You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by al...@apache.org on 2021/12/15 20:41:36 UTC

[arrow-datafusion] branch master updated: Support identifiers with `.` in them (#1449)

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

alamb 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 0052667  Support identifiers with `.` in them (#1449)
0052667 is described below

commit 0052667afae33ba9e549256d0d5d47e2f45e6ffb
Author: Andrew Lamb <an...@nerdnetworks.org>
AuthorDate: Wed Dec 15 15:41:28 2021 -0500

    Support identifiers with `.` in them (#1449)
    
    * Support identifiers with `.` in them
    
    * simplify
    
    * fix: clippy
    
    * fix: more clippy
    
    * Add test for "...."
---
 datafusion/src/sql/planner.rs | 14 +++++---
 datafusion/tests/sql.rs       | 80 +++++++++++++++++++++++++++++++++++++++++++
 2 files changed, 89 insertions(+), 5 deletions(-)

diff --git a/datafusion/src/sql/planner.rs b/datafusion/src/sql/planner.rs
index e668163..bbd5aa7 100644
--- a/datafusion/src/sql/planner.rs
+++ b/datafusion/src/sql/planner.rs
@@ -1062,8 +1062,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> {
                 }
 
                 let field = schema.field(field_index - 1);
-                let col_ident = SQLExpr::Identifier(Ident::new(field.qualified_name()));
-                self.sql_expr_to_logical_expr(&col_ident, schema)?
+                Expr::Column(field.qualified_column())
             }
             e => self.sql_expr_to_logical_expr(e, schema)?,
         };
@@ -1323,9 +1322,14 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> {
                     let var_names = vec![id.value.clone()];
                     Ok(Expr::ScalarVariable(var_names))
                 } else {
-                    // create a column expression based on raw user input, this column will be
-                    // normalized with qualifer later by the SQL planner.
-                    Ok(col(&id.value))
+                    // Don't use `col()` here because it will try to
+                    // interpret names with '.' as if they were
+                    // compound indenfiers, but this is not a compound
+                    // identifier. (e.g. it is "foo.bar" not foo.bar)
+                    Ok(Expr::Column(Column {
+                        relation: None,
+                        name: id.value.clone(),
+                    }))
                 }
             }
 
diff --git a/datafusion/tests/sql.rs b/datafusion/tests/sql.rs
index 0b1abbe..b72606f 100644
--- a/datafusion/tests/sql.rs
+++ b/datafusion/tests/sql.rs
@@ -5450,6 +5450,86 @@ async fn qualified_table_references() -> Result<()> {
 }
 
 #[tokio::test]
+async fn qualified_table_references_and_fields() -> Result<()> {
+    let mut ctx = ExecutionContext::new();
+
+    let c1: StringArray = vec!["foofoo", "foobar", "foobaz"]
+        .into_iter()
+        .map(Some)
+        .collect();
+    let c2: Int64Array = vec![1, 2, 3].into_iter().map(Some).collect();
+    let c3: Int64Array = vec![10, 20, 30].into_iter().map(Some).collect();
+
+    let batch = RecordBatch::try_from_iter(vec![
+        ("f.c1", Arc::new(c1) as ArrayRef),
+        //  evil -- use the same name as the table
+        ("test.c2", Arc::new(c2) as ArrayRef),
+        //  more evil still
+        ("....", Arc::new(c3) as ArrayRef),
+    ])?;
+
+    let table = MemTable::try_new(batch.schema(), vec![vec![batch]])?;
+    ctx.register_table("test", Arc::new(table))?;
+
+    // referring to the unquoted column is an error
+    let sql = r#"SELECT f1.c1 from test"#;
+    let error = ctx.create_logical_plan(sql).unwrap_err();
+    assert_contains!(
+        error.to_string(),
+        "No field named 'f1.c1'. Valid fields are 'test.f.c1', 'test.test.c2'"
+    );
+
+    // however, enclosing it in double quotes is ok
+    let sql = r#"SELECT "f.c1" from test"#;
+    let actual = execute_to_batches(&mut ctx, sql).await;
+    let expected = vec![
+        "+--------+",
+        "| f.c1   |",
+        "+--------+",
+        "| foofoo |",
+        "| foobar |",
+        "| foobaz |",
+        "+--------+",
+    ];
+    assert_batches_eq!(expected, &actual);
+    // Works fully qualified too
+    let sql = r#"SELECT test."f.c1" from test"#;
+    let actual = execute_to_batches(&mut ctx, sql).await;
+    assert_batches_eq!(expected, &actual);
+
+    // check that duplicated table name and column name are ok
+    let sql = r#"SELECT "test.c2" as expr1, test."test.c2" as expr2 from test"#;
+    let actual = execute_to_batches(&mut ctx, sql).await;
+    let expected = vec![
+        "+-------+-------+",
+        "| expr1 | expr2 |",
+        "+-------+-------+",
+        "| 1     | 1     |",
+        "| 2     | 2     |",
+        "| 3     | 3     |",
+        "+-------+-------+",
+    ];
+    assert_batches_eq!(expected, &actual);
+
+    // check that '....' is also an ok column name (in the sense that
+    // datafusion should run the query, not that someone should write
+    // this
+    let sql = r#"SELECT "....", "...." as c3 from test order by "....""#;
+    let actual = execute_to_batches(&mut ctx, sql).await;
+    let expected = vec![
+        "+------+----+",
+        "| .... | c3 |",
+        "+------+----+",
+        "| 10   | 10 |",
+        "| 20   | 20 |",
+        "| 30   | 30 |",
+        "+------+----+",
+    ];
+    assert_batches_eq!(expected, &actual);
+    Ok(())
+}
+
+#[tokio::test]
 async fn invalid_qualified_table_references() -> Result<()> {
     let mut ctx = ExecutionContext::new();
     register_aggregate_csv(&mut ctx).await?;