Skip to content

Commit

Permalink
[FEAT]: add sql DISTINCT (#3087)
Browse files Browse the repository at this point in the history
  • Loading branch information
universalmind303 authored Oct 22, 2024
1 parent 7d600c2 commit 26d639b
Show file tree
Hide file tree
Showing 2 changed files with 18 additions and 4 deletions.
15 changes: 11 additions & 4 deletions src/daft-sql/src/planner.rs
Original file line number Diff line number Diff line change
Expand Up @@ -11,7 +11,7 @@ use daft_functions::numeric::{ceil::ceil, floor::floor};
use daft_plan::{LogicalPlanBuilder, LogicalPlanRef};
use sqlparser::{
ast::{
ArrayElemTypeDef, BinaryOperator, CastKind, ExactNumberInfo, ExcludeSelectItem,
ArrayElemTypeDef, BinaryOperator, CastKind, Distinct, ExactNumberInfo, ExcludeSelectItem,
GroupByExpr, Ident, Query, SelectItem, Statement, StructField, Subscript, TableAlias,
TableWithJoins, TimezoneInfo, UnaryOperator, Value, WildcardAdditionalOptions,
},
Expand Down Expand Up @@ -202,6 +202,15 @@ impl SQLPlanner {
}
}

match &selection.distinct {
Some(Distinct::Distinct) => {
let rel = self.relation_mut();
rel.inner = rel.inner.distinct()?;
}
Some(Distinct::On(_)) => unsupported_sql_err!("DISTINCT ON"),
None => {}
}

if let Some(order_by) = &query.order_by {
if order_by.interpolate.is_some() {
unsupported_sql_err!("ORDER BY [query] [INTERPOLATE]");
Expand Down Expand Up @@ -1186,9 +1195,7 @@ fn check_select_features(selection: &sqlparser::ast::Select) -> SQLPlannerResult
if selection.top.is_some() {
unsupported_sql_err!("TOP");
}
if selection.distinct.is_some() {
unsupported_sql_err!("DISTINCT");
}

if selection.into.is_some() {
unsupported_sql_err!("INTO");
}
Expand Down
7 changes: 7 additions & 0 deletions tests/sql/test_sql.py
Original file line number Diff line number Diff line change
Expand Up @@ -214,3 +214,10 @@ def test_sql_tbl_alias():
catalog = SQLCatalog({"df": daft.from_pydict({"n": [1, 2, 3]})})
df = daft.sql("SELECT df_alias.n FROM df AS df_alias where df_alias.n = 2", catalog)
assert df.collect().to_pydict() == {"n": [2]}


def test_sql_distinct():
df = daft.from_pydict({"n": [1, 1, 2, 2]})
actual = daft.sql("SELECT DISTINCT n FROM df").collect().to_pydict()
expected = df.distinct().collect().to_pydict()
assert actual == expected

0 comments on commit 26d639b

Please sign in to comment.