Skip to content
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

Refactor code for insert in sqllogictest #4503

Merged
merged 4 commits into from
Dec 4, 2022
Merged
Show file tree
Hide file tree
Changes from 2 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
24 changes: 8 additions & 16 deletions datafusion/core/src/execution/context.rs
Original file line number Diff line number Diff line change
Expand Up @@ -940,22 +940,14 @@ impl SessionContext {
table_ref: impl Into<TableReference<'a>>,
) -> Result<Arc<DataFrame>> {
let table_ref = table_ref.into();
let schema = self.state.read().schema_for_ref(table_ref)?;
match schema.table(table_ref.table()) {
Some(ref provider) => {
let plan = LogicalPlanBuilder::scan(
table_ref.table(),
provider_as_source(Arc::clone(provider)),
None,
)?
.build()?;
Ok(Arc::new(DataFrame::new(self.state.clone(), &plan)))
}
_ => Err(DataFusionError::Plan(format!(
"No table named '{}'",
table_ref.table()
))),
}
let provider = self.table_provider(table_ref)?;
let plan = LogicalPlanBuilder::scan(
table_ref.table(),
provider_as_source(Arc::clone(&provider)),
None,
)?
.build()?;
Ok(Arc::new(DataFrame::new(self.state.clone(), &plan)))
}

/// Return a [`TabelProvider`] for the specified table.
Expand Down
12 changes: 0 additions & 12 deletions datafusion/core/tests/sqllogictests/src/error.rs
Original file line number Diff line number Diff line change
Expand Up @@ -32,12 +32,6 @@ pub enum DFSqlLogicTestError {
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),
}

impl From<TestError> for DFSqlLogicTestError {
Expand Down Expand Up @@ -70,12 +64,6 @@ impl Display for DFSqlLogicTestError {
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)
}
}
}
}
Expand Down
33 changes: 11 additions & 22 deletions datafusion/core/tests/sqllogictests/src/insert/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -17,46 +17,39 @@

mod util;

use crate::error::{DFSqlLogicTestError, Result};
use crate::error::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> {
pub async fn insert(ctx: &SessionContext, insert_stmt: &SQLStatement) -> 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 {
let table_name;
let insert_values: Vec<Vec<Expr>>;
match insert_stmt {
SQLStatement::Insert {
table_name: name,
source,
..
} = &**statement
{
} => {
// Todo: check columns match table schema
table_name = name.to_string();
match &*source.body {
SetExpr::Values(values) => {
insert_values = values.0.clone();
}
_ => {
return Err(DFSqlLogicTestError::NotImplemented(
"Only support insert values".to_string(),
));
// Directly panic: make it easy to find the location of the error.
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Checked in my local, and it works

[insert.slt] Running query: "insert into users select * from users;"
thread 'main' panicked at 'explicit panic', datafusion/core/tests/sqllogictests/src/insert/mod.rs:48:21
stack backtrace:
   0: rust_begin_unwind
             at /rustc/96ddd32c4bfb1d78f0cd03eb068b1710a8cebeef/library/std/src/panicking.rs:575:5
   1: core::panicking::panic_fmt
             at /rustc/96ddd32c4bfb1d78f0cd03eb068b1710a8cebeef/library/core/src/panicking.rs:65:14
   2: core::panicking::panic
             at /rustc/96ddd32c4bfb1d78f0cd03eb068b1710a8cebeef/library/core/src/panicking.rs:114:5
   3: sqllogictests::insert::insert::{{closure}}
             at ./tests/sqllogictests/src/insert/mod.rs:48:21
   4: <core::future::from_generator::GenFuture<T> as core::future::future::Future>::poll
             at /rustc/96ddd32c4bfb1d78f0cd03eb068b1710a8cebeef/library/core/src/future/mod.rs:91:19
   5: sqllogictests::run_query::{{closure}}
             at ./tests/sqllogictests/src/main.rs:151:53
   6: <core::future::from_generator::GenFuture<T> as core::future::future::Future>::poll
             at /rustc/96ddd32c4bfb1d78f0cd03eb068b1710a8cebeef/library/core/src/future/mod.rs:91:19
   7: <sqllogictests::DataFusion as sqllogictest::runner::AsyncDB>::run::{{closure}}
             at ./tests/sqllogictests/src/main.rs:48:47

panic!()
}
}
}
} else {
return Err(DFSqlLogicTestError::Internal(format!(
"{:?} not an insert statement",
sql
)));
_ => unreachable!(),
}

// Second, get table by table name
Expand All @@ -65,11 +58,7 @@ pub async fn insert(ctx: &SessionContext, sql: String) -> Result<String> {
let table_batches = table_provider
.as_any()
.downcast_ref::<MemTable>()
.ok_or_else(|| {
DFSqlLogicTestError::NotImplemented(
"only support use memory table in logictest".to_string(),
)
})?
.unwrap()
.get_batches();

// Third, transfer insert values to `RecordBatch`
Expand Down
15 changes: 11 additions & 4 deletions datafusion/core/tests/sqllogictests/src/main.rs
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,8 @@ use async_trait::async_trait;
use datafusion::arrow::csv::WriterBuilder;
use datafusion::arrow::record_batch::RecordBatch;
use datafusion::prelude::{SessionConfig, SessionContext};
use datafusion_sql::parser::{DFParser, Statement};
use sqlparser::ast::Statement as SQLStatement;
use std::path::Path;
use std::time::Duration;

Expand Down Expand Up @@ -142,10 +144,15 @@ fn format_batches(batches: &[RecordBatch]) -> Result<String> {
async fn run_query(ctx: &SessionContext, sql: impl Into<String>) -> Result<String> {
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());
match DFParser::parse_sql(&sql) {
Ok(statements) => {
if let Statement::Statement(statement) = &statements[0] {
if let SQLStatement::Insert { .. } = &**statement {
return insert(ctx, &**statement).await;
}
}
}
Err(_) => {}
}
let df = ctx.sql(sql.as_str()).await.unwrap();
let results: Vec<RecordBatch> = df.collect().await.unwrap();
Expand Down