You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by yj...@apache.org on 2022/04/16 10:30:26 UTC

[arrow-datafusion] branch master updated: Move identifer case tests to `sql_integ`, add negative cases, Debug for `DataFrame` (#2243)

This is an automated email from the ASF dual-hosted git repository.

yjshen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/arrow-datafusion.git


The following commit(s) were added to refs/heads/master by this push:
     new f5592cf29 Move identifer case tests to `sql_integ`, add negative cases, Debug for `DataFrame` (#2243)
f5592cf29 is described below

commit f5592cf296624e35bf06b3716ed60deefcc38cf8
Author: Andrew Lamb <an...@nerdnetworks.org>
AuthorDate: Sat Apr 16 06:30:21 2022 -0400

    Move identifer case tests to `sql_integ`, add negative cases, Debug for `DataFrame` (#2243)
    
    * Move normalized_column_identier test to sql_integ
    
    * Add test for invalid identifiers
    
    * Add Minimal Debug impls for DataFrame
    
    * RAT
---
 datafusion/core/src/dataframe.rs         |   1 +
 datafusion/core/src/execution/context.rs | 181 ++---------------------
 datafusion/core/tests/sql/idenfifers.rs  | 243 +++++++++++++++++++++++++++++++
 datafusion/core/tests/sql/mod.rs         |   1 +
 4 files changed, 258 insertions(+), 168 deletions(-)

diff --git a/datafusion/core/src/dataframe.rs b/datafusion/core/src/dataframe.rs
index 845791166..9414f2887 100644
--- a/datafusion/core/src/dataframe.rs
+++ b/datafusion/core/src/dataframe.rs
@@ -68,6 +68,7 @@ use std::any::Any;
 /// # Ok(())
 /// # }
 /// ```
+#[derive(Debug)]
 pub struct DataFrame {
     session_state: Arc<RwLock<SessionState>>,
     plan: LogicalPlan,
diff --git a/datafusion/core/src/execution/context.rs b/datafusion/core/src/execution/context.rs
index 85022019b..4bb77fb93 100644
--- a/datafusion/core/src/execution/context.rs
+++ b/datafusion/core/src/execution/context.rs
@@ -42,9 +42,12 @@ use crate::{
 };
 use log::{debug, trace};
 use parking_lot::RwLock;
-use std::collections::{HashMap, HashSet};
 use std::string::String;
 use std::sync::Arc;
+use std::{
+    collections::{HashMap, HashSet},
+    fmt::Debug,
+};
 
 use arrow::datatypes::{DataType, SchemaRef};
 
@@ -1148,6 +1151,15 @@ pub struct SessionState {
     pub runtime_env: Arc<RuntimeEnv>,
 }
 
+impl Debug for SessionState {
+    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
+        f.debug_struct("SessionState")
+            .field("session_id", &self.session_id)
+            // TODO should we print out more?
+            .finish()
+    }
+}
+
 /// Default session builder using the provided configuration
 pub fn default_session_builder(config: SessionConfig) -> SessionState {
     SessionState::with_config_rt(
@@ -3322,173 +3334,6 @@ mod tests {
         Ok(())
     }
 
-    #[tokio::test]
-    async fn normalized_column_identifiers() {
-        // create local execution context
-        let ctx = SessionContext::new();
-
-        // register csv file with the execution context
-        ctx.register_csv(
-            "case_insensitive_test",
-            "tests/example.csv",
-            CsvReadOptions::new(),
-        )
-        .await
-        .unwrap();
-
-        let sql = "SELECT A, b FROM case_insensitive_test";
-        let result = plan_and_collect(&ctx, sql)
-            .await
-            .expect("ran plan correctly");
-        let expected = vec![
-            "+---+---+",
-            "| a | b |",
-            "+---+---+",
-            "| 1 | 2 |",
-            "+---+---+",
-        ];
-        assert_batches_sorted_eq!(expected, &result);
-
-        let sql = "SELECT t.A, b FROM case_insensitive_test AS t";
-        let result = plan_and_collect(&ctx, sql)
-            .await
-            .expect("ran plan correctly");
-        let expected = vec![
-            "+---+---+",
-            "| a | b |",
-            "+---+---+",
-            "| 1 | 2 |",
-            "+---+---+",
-        ];
-        assert_batches_sorted_eq!(expected, &result);
-
-        // Aliases
-
-        let sql = "SELECT t.A as x, b FROM case_insensitive_test AS t";
-        let result = plan_and_collect(&ctx, sql)
-            .await
-            .expect("ran plan correctly");
-        let expected = vec![
-            "+---+---+",
-            "| x | b |",
-            "+---+---+",
-            "| 1 | 2 |",
-            "+---+---+",
-        ];
-        assert_batches_sorted_eq!(expected, &result);
-
-        let sql = "SELECT t.A AS X, b FROM case_insensitive_test AS t";
-        let result = plan_and_collect(&ctx, sql)
-            .await
-            .expect("ran plan correctly");
-        let expected = vec![
-            "+---+---+",
-            "| x | b |",
-            "+---+---+",
-            "| 1 | 2 |",
-            "+---+---+",
-        ];
-        assert_batches_sorted_eq!(expected, &result);
-
-        let sql = r#"SELECT t.A AS "X", b FROM case_insensitive_test AS t"#;
-        let result = plan_and_collect(&ctx, sql)
-            .await
-            .expect("ran plan correctly");
-        let expected = vec![
-            "+---+---+",
-            "| X | b |",
-            "+---+---+",
-            "| 1 | 2 |",
-            "+---+---+",
-        ];
-        assert_batches_sorted_eq!(expected, &result);
-
-        // Order by
-
-        let sql = "SELECT t.A AS x, b FROM case_insensitive_test AS t ORDER BY x";
-        let result = plan_and_collect(&ctx, sql)
-            .await
-            .expect("ran plan correctly");
-        let expected = vec![
-            "+---+---+",
-            "| x | b |",
-            "+---+---+",
-            "| 1 | 2 |",
-            "+---+---+",
-        ];
-        assert_batches_sorted_eq!(expected, &result);
-
-        let sql = "SELECT t.A AS x, b FROM case_insensitive_test AS t ORDER BY X";
-        let result = plan_and_collect(&ctx, sql)
-            .await
-            .expect("ran plan correctly");
-        let expected = vec![
-            "+---+---+",
-            "| x | b |",
-            "+---+---+",
-            "| 1 | 2 |",
-            "+---+---+",
-        ];
-        assert_batches_sorted_eq!(expected, &result);
-
-        let sql = r#"SELECT t.A AS "X", b FROM case_insensitive_test AS t ORDER BY "X""#;
-        let result = plan_and_collect(&ctx, sql)
-            .await
-            .expect("ran plan correctly");
-        let expected = vec![
-            "+---+---+",
-            "| X | b |",
-            "+---+---+",
-            "| 1 | 2 |",
-            "+---+---+",
-        ];
-        assert_batches_sorted_eq!(expected, &result);
-
-        // Where
-
-        let sql = "SELECT a, b FROM case_insensitive_test where A IS NOT null";
-        let result = plan_and_collect(&ctx, sql)
-            .await
-            .expect("ran plan correctly");
-        let expected = vec![
-            "+---+---+",
-            "| a | b |",
-            "+---+---+",
-            "| 1 | 2 |",
-            "+---+---+",
-        ];
-        assert_batches_sorted_eq!(expected, &result);
-
-        // Group by
-
-        let sql = "SELECT a as x, count(*) as c FROM case_insensitive_test GROUP BY X";
-        let result = plan_and_collect(&ctx, sql)
-            .await
-            .expect("ran plan correctly");
-        let expected = vec![
-            "+---+---+",
-            "| x | c |",
-            "+---+---+",
-            "| 1 | 1 |",
-            "+---+---+",
-        ];
-        assert_batches_sorted_eq!(expected, &result);
-
-        let sql =
-            r#"SELECT a as "X", count(*) as c FROM case_insensitive_test GROUP BY "X""#;
-        let result = plan_and_collect(&ctx, sql)
-            .await
-            .expect("ran plan correctly");
-        let expected = vec![
-            "+---+---+",
-            "| X | c |",
-            "+---+---+",
-            "| 1 | 1 |",
-            "+---+---+",
-        ];
-        assert_batches_sorted_eq!(expected, &result);
-    }
-
     struct MyPhysicalPlanner {}
 
     #[async_trait]
diff --git a/datafusion/core/tests/sql/idenfifers.rs b/datafusion/core/tests/sql/idenfifers.rs
new file mode 100644
index 000000000..71083e274
--- /dev/null
+++ b/datafusion/core/tests/sql/idenfifers.rs
@@ -0,0 +1,243 @@
+// 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 std::sync::Arc;
+
+use arrow::{array::StringArray, record_batch::RecordBatch};
+use datafusion::{
+    assert_batches_sorted_eq, assert_contains, datasource::MemTable, prelude::*,
+};
+
+use crate::sql::plan_and_collect;
+
+#[tokio::test]
+async fn normalized_column_identifiers() {
+    // create local execution context
+    let ctx = SessionContext::new();
+
+    // register csv file with the execution context
+    ctx.register_csv(
+        "case_insensitive_test",
+        "tests/example.csv",
+        CsvReadOptions::new(),
+    )
+    .await
+    .unwrap();
+
+    let sql = "SELECT A, b FROM case_insensitive_test";
+    let result = plan_and_collect(&ctx, sql)
+        .await
+        .expect("ran plan correctly");
+    let expected = vec![
+        "+---+---+",
+        "| a | b |",
+        "+---+---+",
+        "| 1 | 2 |",
+        "+---+---+",
+    ];
+    assert_batches_sorted_eq!(expected, &result);
+
+    let sql = "SELECT t.A, b FROM case_insensitive_test AS t";
+    let result = plan_and_collect(&ctx, sql)
+        .await
+        .expect("ran plan correctly");
+    let expected = vec![
+        "+---+---+",
+        "| a | b |",
+        "+---+---+",
+        "| 1 | 2 |",
+        "+---+---+",
+    ];
+    assert_batches_sorted_eq!(expected, &result);
+
+    // Aliases
+
+    let sql = "SELECT t.A as x, b FROM case_insensitive_test AS t";
+    let result = plan_and_collect(&ctx, sql)
+        .await
+        .expect("ran plan correctly");
+    let expected = vec![
+        "+---+---+",
+        "| x | b |",
+        "+---+---+",
+        "| 1 | 2 |",
+        "+---+---+",
+    ];
+    assert_batches_sorted_eq!(expected, &result);
+
+    let sql = "SELECT t.A AS X, b FROM case_insensitive_test AS t";
+    let result = plan_and_collect(&ctx, sql)
+        .await
+        .expect("ran plan correctly");
+    let expected = vec![
+        "+---+---+",
+        "| x | b |",
+        "+---+---+",
+        "| 1 | 2 |",
+        "+---+---+",
+    ];
+    assert_batches_sorted_eq!(expected, &result);
+
+    let sql = r#"SELECT t.A AS "X", b FROM case_insensitive_test AS t"#;
+    let result = plan_and_collect(&ctx, sql)
+        .await
+        .expect("ran plan correctly");
+    let expected = vec![
+        "+---+---+",
+        "| X | b |",
+        "+---+---+",
+        "| 1 | 2 |",
+        "+---+---+",
+    ];
+    assert_batches_sorted_eq!(expected, &result);
+
+    // Order by
+
+    let sql = "SELECT t.A AS x, b FROM case_insensitive_test AS t ORDER BY x";
+    let result = plan_and_collect(&ctx, sql)
+        .await
+        .expect("ran plan correctly");
+    let expected = vec![
+        "+---+---+",
+        "| x | b |",
+        "+---+---+",
+        "| 1 | 2 |",
+        "+---+---+",
+    ];
+    assert_batches_sorted_eq!(expected, &result);
+
+    let sql = "SELECT t.A AS x, b FROM case_insensitive_test AS t ORDER BY X";
+    let result = plan_and_collect(&ctx, sql)
+        .await
+        .expect("ran plan correctly");
+    let expected = vec![
+        "+---+---+",
+        "| x | b |",
+        "+---+---+",
+        "| 1 | 2 |",
+        "+---+---+",
+    ];
+    assert_batches_sorted_eq!(expected, &result);
+
+    let sql = r#"SELECT t.A AS "X", b FROM case_insensitive_test AS t ORDER BY "X""#;
+    let result = plan_and_collect(&ctx, sql)
+        .await
+        .expect("ran plan correctly");
+    let expected = vec![
+        "+---+---+",
+        "| X | b |",
+        "+---+---+",
+        "| 1 | 2 |",
+        "+---+---+",
+    ];
+    assert_batches_sorted_eq!(expected, &result);
+
+    // Where
+
+    let sql = "SELECT a, b FROM case_insensitive_test where A IS NOT null";
+    let result = plan_and_collect(&ctx, sql)
+        .await
+        .expect("ran plan correctly");
+    let expected = vec![
+        "+---+---+",
+        "| a | b |",
+        "+---+---+",
+        "| 1 | 2 |",
+        "+---+---+",
+    ];
+    assert_batches_sorted_eq!(expected, &result);
+
+    // Group by
+
+    let sql = "SELECT a as x, count(*) as c FROM case_insensitive_test GROUP BY X";
+    let result = plan_and_collect(&ctx, sql)
+        .await
+        .expect("ran plan correctly");
+    let expected = vec![
+        "+---+---+",
+        "| x | c |",
+        "+---+---+",
+        "| 1 | 1 |",
+        "+---+---+",
+    ];
+    assert_batches_sorted_eq!(expected, &result);
+
+    let sql = r#"SELECT a as "X", count(*) as c FROM case_insensitive_test GROUP BY "X""#;
+    let result = plan_and_collect(&ctx, sql)
+        .await
+        .expect("ran plan correctly");
+    let expected = vec![
+        "+---+---+",
+        "| X | c |",
+        "+---+---+",
+        "| 1 | 1 |",
+        "+---+---+",
+    ];
+    assert_batches_sorted_eq!(expected, &result);
+}
+
+#[tokio::test]
+async fn case_insensitive_in_sql_errors() {
+    let record_batch = RecordBatch::try_from_iter(vec![
+        // The proper way to refer to this column is "Column1" -- it
+        // should not be possible to use `column1` or `COLUMN1` or
+        // other variants
+        (
+            "Column1",
+            Arc::new(StringArray::from(vec!["content1"])) as _,
+        ),
+    ])
+    .unwrap();
+
+    let table =
+        MemTable::try_new(record_batch.schema(), vec![vec![record_batch]]).unwrap();
+
+    let ctx = SessionContext::new();
+    ctx.register_table("test", Arc::new(table)).unwrap();
+
+    // None of these tests shoud pass
+    let actual = ctx
+        .sql("SELECT COLumn1 from test")
+        .await
+        .unwrap_err()
+        .to_string();
+    assert_contains!(actual, "Invalid identifier '#column1'");
+
+    let actual = ctx
+        .sql("SELECT Column1 from test")
+        .await
+        .unwrap_err()
+        .to_string();
+    assert_contains!(actual, "Invalid identifier '#column1'");
+
+    let actual = ctx
+        .sql("SELECT column1 from test")
+        .await
+        .unwrap_err()
+        .to_string();
+    assert_contains!(actual, "Invalid identifier '#column1'");
+
+    let actual = ctx
+        .sql(r#"SELECT "column1" from test"#)
+        .await
+        .unwrap_err()
+        .to_string();
+    assert_contains!(actual, "Invalid identifier '#column1'");
+
+    // This should pass (note the quotes)
+    ctx.sql(r#"SELECT "Column1" from test"#).await.unwrap();
+}
diff --git a/datafusion/core/tests/sql/mod.rs b/datafusion/core/tests/sql/mod.rs
index ee1015f64..05f328d3a 100644
--- a/datafusion/core/tests/sql/mod.rs
+++ b/datafusion/core/tests/sql/mod.rs
@@ -100,6 +100,7 @@ pub mod window;
 
 pub mod decimal;
 mod explain;
+mod idenfifers;
 pub mod information_schema;
 mod partitioned_csv;
 #[cfg(feature = "unicode_expressions")]