From 2935abdf4b844bbd2cc4c9746604d78008d5fdec Mon Sep 17 00:00:00 2001 From: Arik Mitschang Date: Fri, 27 Sep 2024 11:58:40 -0400 Subject: [PATCH] [BUG] improve error reporting for multistatement sql (#2916) Called it a bug because the error is confusing, this fixes error behavior of multistatement sql. Before: ```py daft.sql('''SELECT * FROM df; SELECT count(*) FROM df''', cat) ``` results in this error: ```py daft.exceptions.InvalidSQLException: Unsupported SQL: 'SELECT * FROM df' ``` but of course that SQL is supported! With this change, error becomes: ```py daft.exceptions.InvalidSQLException: Unsupported SQL: 'Only exactly one SQL statement allowed, found 2' ``` which I believe is at least currently correct, and any future support of multiple statements would have to touch this area of code anyway. --- src/daft-sql/src/planner.rs | 19 ++++++++++++++----- tests/sql/test_sql.py | 6 ++++++ 2 files changed, 20 insertions(+), 5 deletions(-) diff --git a/src/daft-sql/src/planner.rs b/src/daft-sql/src/planner.rs index bac774a4fc..b9f326ce99 100644 --- a/src/daft-sql/src/planner.rs +++ b/src/daft-sql/src/planner.rs @@ -12,8 +12,8 @@ use daft_plan::{LogicalPlanBuilder, LogicalPlanRef}; use sqlparser::{ ast::{ ArrayElemTypeDef, BinaryOperator, CastKind, ExactNumberInfo, GroupByExpr, Ident, Query, - SelectItem, StructField, Subscript, TableWithJoins, TimezoneInfo, UnaryOperator, Value, - WildcardAdditionalOptions, + SelectItem, Statement, StructField, Subscript, TableWithJoins, TimezoneInfo, UnaryOperator, + Value, WildcardAdditionalOptions, }, dialect::GenericDialect, parser::{Parser, ParserOptions}, @@ -88,9 +88,18 @@ impl SQLPlanner { let statements = parser.parse_statements()?; - match statements.as_slice() { - [sqlparser::ast::Statement::Query(query)] => Ok(self.plan_query(query)?.build()), - other => unsupported_sql_err!("{}", other[0]), + match statements.len() { + 1 => Ok(self.plan_statement(&statements[0])?), + other => { + unsupported_sql_err!("Only exactly one SQL statement allowed, found {}", other) + } + } + } + + fn plan_statement(&mut self, statement: &Statement) -> SQLPlannerResult { + match statement { + Statement::Query(query) => Ok(self.plan_query(query)?.build()), + other => unsupported_sql_err!("{}", other), } } diff --git a/tests/sql/test_sql.py b/tests/sql/test_sql.py index 12384e34e0..3802a3de20 100644 --- a/tests/sql/test_sql.py +++ b/tests/sql/test_sql.py @@ -199,3 +199,9 @@ def test_sql_function_raises_when_cant_get_frame(monkeypatch): monkeypatch.setattr("inspect.currentframe", lambda: None) with pytest.raises(DaftCoreException, match="Cannot get caller environment"): daft.sql("SELECT * FROM df") + + +def test_sql_multi_statement_sql_error(): + catalog = SQLCatalog({}) + with pytest.raises(Exception, match="one SQL statement allowed"): + daft.sql("SELECT * FROM df; SELECT * FROM df", catalog)