-
Notifications
You must be signed in to change notification settings - Fork 1.3k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Support insert into
statement in sqllogictest
#4496
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,83 @@ | ||
// Licensed to the Apache Software Foundation (ASF) under one | ||
// or more contributor license agreements. See the NOTICE file | ||
// distributed with this work for additional information | ||
// regarding copyright ownership. The ASF licenses this file | ||
// to you under the Apache License, Version 2.0 (the | ||
// "License"); you may not use this file except in compliance | ||
// with the License. You may obtain a copy of the License at | ||
// | ||
// http://www.apache.org/licenses/LICENSE-2.0 | ||
// | ||
// Unless required by applicable law or agreed to in writing, | ||
// software distributed under the License is distributed on an | ||
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY | ||
// KIND, either express or implied. See the License for the | ||
// specific language governing permissions and limitations | ||
// under the License. | ||
|
||
use datafusion_common::DataFusionError; | ||
use sqllogictest::TestError; | ||
use sqlparser::parser::ParserError; | ||
use std::error; | ||
use std::fmt::{Display, Formatter}; | ||
|
||
pub type Result<T> = std::result::Result<T, DFSqlLogicTestError>; | ||
|
||
/// DataFusion sql-logicaltest error | ||
#[derive(Debug)] | ||
pub enum DFSqlLogicTestError { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. 👍 this is a good idea |
||
/// Error from sqllogictest-rs | ||
SqlLogicTest(TestError), | ||
/// Error from datafusion | ||
DataFusion(DataFusionError), | ||
/// Error returned when SQL is syntactically incorrect. | ||
Sql(ParserError), | ||
/// Error returned on a branch that we know it is possible | ||
/// but to which we still have no implementation for. | ||
/// Often, these errors are tracked in our issue tracker. | ||
NotImplemented(String), | ||
/// Error returned from DFSqlLogicTest inner | ||
Internal(String), | ||
Comment on lines
+38
to
+40
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I suggest we simply panic in the sqllogic runner in these cases so the location of the error is easier to see There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. LGTM |
||
} | ||
|
||
impl From<TestError> for DFSqlLogicTestError { | ||
fn from(value: TestError) -> Self { | ||
DFSqlLogicTestError::SqlLogicTest(value) | ||
} | ||
} | ||
|
||
impl From<DataFusionError> for DFSqlLogicTestError { | ||
fn from(value: DataFusionError) -> Self { | ||
DFSqlLogicTestError::DataFusion(value) | ||
} | ||
} | ||
|
||
impl From<ParserError> for DFSqlLogicTestError { | ||
fn from(value: ParserError) -> Self { | ||
DFSqlLogicTestError::Sql(value) | ||
} | ||
} | ||
|
||
impl Display for DFSqlLogicTestError { | ||
fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { | ||
match self { | ||
DFSqlLogicTestError::SqlLogicTest(error) => write!( | ||
f, | ||
"SqlLogicTest error(from sqllogictest-rs crate): {}", | ||
error | ||
), | ||
DFSqlLogicTestError::DataFusion(error) => { | ||
write!(f, "DataFusion error: {}", error) | ||
} | ||
DFSqlLogicTestError::Sql(error) => write!(f, "SQL Parser error: {}", error), | ||
DFSqlLogicTestError::NotImplemented(error) => { | ||
write!(f, "This feature is not implemented yet: {}", error) | ||
} | ||
DFSqlLogicTestError::Internal(error) => { | ||
write!(f, "Internal error: {}", error) | ||
} | ||
} | ||
} | ||
} | ||
|
||
impl error::Error for DFSqlLogicTestError {} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,96 @@ | ||
// Licensed to the Apache Software Foundation (ASF) under one | ||
// or more contributor license agreements. See the NOTICE file | ||
// distributed with this work for additional information | ||
// regarding copyright ownership. The ASF licenses this file | ||
// to you under the Apache License, Version 2.0 (the | ||
// "License"); you may not use this file except in compliance | ||
// with the License. You may obtain a copy of the License at | ||
// | ||
// http://www.apache.org/licenses/LICENSE-2.0 | ||
// | ||
// Unless required by applicable law or agreed to in writing, | ||
// software distributed under the License is distributed on an | ||
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY | ||
// KIND, either express or implied. See the License for the | ||
// specific language governing permissions and limitations | ||
// under the License. | ||
|
||
mod util; | ||
|
||
use crate::error::{DFSqlLogicTestError, Result}; | ||
use crate::insert::util::LogicTestContextProvider; | ||
use datafusion::datasource::MemTable; | ||
use datafusion::prelude::SessionContext; | ||
use datafusion_common::{DFSchema, DataFusionError}; | ||
use datafusion_expr::Expr as DFExpr; | ||
use datafusion_sql::parser::{DFParser, Statement}; | ||
use datafusion_sql::planner::SqlToRel; | ||
use sqlparser::ast::{Expr, SetExpr, Statement as SQLStatement}; | ||
use std::collections::HashMap; | ||
|
||
pub async fn insert(ctx: &SessionContext, sql: String) -> Result<String> { | ||
// First, use sqlparser to get table name and insert values | ||
let mut table_name = "".to_string(); | ||
let mut insert_values: Vec<Vec<Expr>> = vec![]; | ||
if let Statement::Statement(statement) = &DFParser::parse_sql(&sql)?[0] { | ||
if let SQLStatement::Insert { | ||
table_name: name, | ||
source, | ||
.. | ||
} = &**statement | ||
{ | ||
// Todo: check columns match table schema | ||
table_name = name.to_string(); | ||
match &*source.body { | ||
SetExpr::Values(values) => { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This is very clever |
||
insert_values = values.0.clone(); | ||
} | ||
_ => { | ||
return Err(DFSqlLogicTestError::NotImplemented( | ||
"Only support insert values".to_string(), | ||
)); | ||
} | ||
} | ||
} | ||
} else { | ||
return Err(DFSqlLogicTestError::Internal(format!( | ||
"{:?} not an insert statement", | ||
sql | ||
))); | ||
} | ||
|
||
// Second, get table by table name | ||
// Here we assume table must be in memory table. | ||
let table_provider = ctx.table_provider(table_name.as_str())?; | ||
let table_batches = table_provider | ||
.as_any() | ||
.downcast_ref::<MemTable>() | ||
.ok_or_else(|| { | ||
DFSqlLogicTestError::NotImplemented( | ||
"only support use memory table in logictest".to_string(), | ||
) | ||
})? | ||
.get_batches(); | ||
|
||
// Third, transfer insert values to `RecordBatch` | ||
// Attention: schema info can be ignored. (insert values don't contain schema info) | ||
let sql_to_rel = SqlToRel::new(&LogicTestContextProvider {}); | ||
let mut insert_batches = Vec::with_capacity(insert_values.len()); | ||
for row in insert_values.into_iter() { | ||
let logical_exprs = row | ||
.into_iter() | ||
.map(|expr| { | ||
sql_to_rel.sql_to_rex(expr, &DFSchema::empty(), &mut HashMap::new()) | ||
}) | ||
.collect::<std::result::Result<Vec<DFExpr>, DataFusionError>>()?; | ||
// Directly use `select` to get `RecordBatch` | ||
let dataframe = ctx.read_empty()?; | ||
insert_batches.push(dataframe.select(logical_exprs)?.collect().await?) | ||
} | ||
|
||
// Final, append the `RecordBatch` to memtable's batches | ||
let mut table_batches = table_batches.write(); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Rather than changing the batches in the existing memtable, what would you think about creating a new memtable with the same name with the new values (rather than modifying the original one) I think you might be able to avoid changes to SessionContext and MemTable entirely. Something like this (untested) // fetch existing batches
let mut existing_batches = ctx.table(table_name.as_str()).collect();
// append new batch
exsiting_batches.extend(insert_batches)
// Replace table provider provider
let new_provider = MemTable::try_new(batches[0].schema(), vec![batches]);
ctx.register_table(table_name, new_provider) There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yes, I have thought about the way you mentioned, (also need to delete original memtable). -- But for performance reasons, I choose the current way: modifying the original one. If you think the changes to add interior mutability to MemTable don't make sense, I can change it in the following ticket! (I don't have a strong bias for both ways) There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think keeping MemTable as simple as possible is likely the best approach -- so for that reason I prefer to remove the interior mutability. I can give it a shot if you agree -- I think the performance of copying record batches (for reasonably small data such as what is in the test) will be ok There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Ok, agree! Thanks @alamb . I'll refactor it later. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Thank you very much! |
||
table_batches.extend(insert_batches); | ||
|
||
Ok("".to_string()) | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,50 @@ | ||
// Licensed to the Apache Software Foundation (ASF) under one | ||
// or more contributor license agreements. See the NOTICE file | ||
// distributed with this work for additional information | ||
// regarding copyright ownership. The ASF licenses this file | ||
// to you under the Apache License, Version 2.0 (the | ||
// "License"); you may not use this file except in compliance | ||
// with the License. You may obtain a copy of the License at | ||
// | ||
// http://www.apache.org/licenses/LICENSE-2.0 | ||
// | ||
// Unless required by applicable law or agreed to in writing, | ||
// software distributed under the License is distributed on an | ||
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY | ||
// KIND, either express or implied. See the License for the | ||
// specific language governing permissions and limitations | ||
// under the License. | ||
|
||
use arrow::datatypes::DataType; | ||
use datafusion_common::{ScalarValue, TableReference}; | ||
use datafusion_expr::{AggregateUDF, ScalarUDF, TableSource}; | ||
use datafusion_sql::planner::ContextProvider; | ||
use std::sync::Arc; | ||
|
||
pub struct LogicTestContextProvider {} | ||
|
||
// Only a mock, don't need to implement | ||
impl ContextProvider for LogicTestContextProvider { | ||
fn get_table_provider( | ||
&self, | ||
_name: TableReference, | ||
) -> datafusion_common::Result<Arc<dyn TableSource>> { | ||
todo!() | ||
} | ||
|
||
fn get_function_meta(&self, _name: &str) -> Option<Arc<ScalarUDF>> { | ||
todo!() | ||
} | ||
|
||
fn get_aggregate_meta(&self, _name: &str) -> Option<Arc<AggregateUDF>> { | ||
todo!() | ||
} | ||
|
||
fn get_variable_type(&self, _variable_names: &[String]) -> Option<DataType> { | ||
todo!() | ||
} | ||
|
||
fn get_config_option(&self, _variable: &str) -> Option<ScalarValue> { | ||
todo!() | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -22,9 +22,11 @@ use datafusion::prelude::{SessionConfig, SessionContext}; | |
use std::path::Path; | ||
use std::time::Duration; | ||
|
||
use sqllogictest::TestError; | ||
pub type Result<T> = std::result::Result<T, TestError>; | ||
use crate::error::{DFSqlLogicTestError, Result}; | ||
use crate::insert::insert; | ||
|
||
mod error; | ||
mod insert; | ||
mod setup; | ||
mod utils; | ||
|
||
|
@@ -37,7 +39,7 @@ pub struct DataFusion { | |
|
||
#[async_trait] | ||
impl sqllogictest::AsyncDB for DataFusion { | ||
type Error = TestError; | ||
type Error = DFSqlLogicTestError; | ||
|
||
async fn run(&mut self, sql: &str) -> Result<String> { | ||
println!("[{}] Running query: \"{}\"", self.file_name, sql); | ||
|
@@ -138,7 +140,14 @@ fn format_batches(batches: &[RecordBatch]) -> Result<String> { | |
} | ||
|
||
async fn run_query(ctx: &SessionContext, sql: impl Into<String>) -> Result<String> { | ||
let df = ctx.sql(&sql.into()).await.unwrap(); | ||
let sql = sql.into(); | ||
// Check if the sql is `insert` | ||
if sql.trim_start().to_lowercase().starts_with("insert") { | ||
// Process the insert statement | ||
insert(ctx, sql).await?; | ||
return Ok("".to_string()); | ||
} | ||
Comment on lines
+144
to
+149
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. 👍 I like this basic approach (special case the sql and route it to the test runner implementation). One thing that might be worth doing is to actually try and parse the input into sql, to detect INSERT statements though I think string manipulation is fine too or we could do this later // Handle any test only special case statements
let sql = sql.into();
match DFParser::parse_sql(&sql) {
Ok(Statement(Insert)) => {
//debug!("Parsed statement: {:#?}", stmt);
}
Err(_) => {
// ignore anything else, including errors -- they will be handled by the sql context below
}
}; There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yes, more generic way, will fix in next PR. |
||
let df = ctx.sql(sql.as_str()).await.unwrap(); | ||
let results: Vec<RecordBatch> = df.collect().await.unwrap(); | ||
let formatted_batches = format_batches(&results)?; | ||
Ok(formatted_batches) | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
👍 it might be nice to refactor
fn table()
to call this function now to avoid some duplication.