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 2023/01/26 19:19:58 UTC

[arrow-datafusion] branch master updated: [sqllogictests] Port orderby.rs to sqllogictests (#5062)

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 d588b479b [sqllogictests] Port orderby.rs to sqllogictests (#5062)
d588b479b is described below

commit d588b479bb5b22d160b850677ff43ade5797cffa
Author: Andrew Lamb <an...@nerdnetworks.org>
AuthorDate: Thu Jan 26 20:19:51 2023 +0100

    [sqllogictests] Port orderby.rs to sqllogictests (#5062)
    
    * [sqllogictests] Port orderby.rs to sqllogictests
    
    * Apply suggestions from code review
    
    Co-authored-by: Yevhenii Melnyk <me...@gmail.com>
    
    * restore lost test
    
    Co-authored-by: Yevhenii Melnyk <me...@gmail.com>
---
 .../tests/data/partitioned_csv/partition-0.csv     |  11 ++
 .../tests/data/partitioned_csv/partition-1.csv     |  11 ++
 .../tests/data/partitioned_csv/partition-2.csv     |  11 ++
 .../tests/data/partitioned_csv/partition-3.csv     |  11 ++
 datafusion/core/tests/sql/order.rs                 | 193 ---------------------
 .../core/tests/sqllogictests/test_files/order.slt  | 165 ++++++++++++++++++
 6 files changed, 209 insertions(+), 193 deletions(-)

diff --git a/datafusion/core/tests/data/partitioned_csv/partition-0.csv b/datafusion/core/tests/data/partitioned_csv/partition-0.csv
new file mode 100644
index 000000000..f54554228
--- /dev/null
+++ b/datafusion/core/tests/data/partitioned_csv/partition-0.csv
@@ -0,0 +1,11 @@
+0,0,true
+0,1,false
+0,2,true
+0,3,false
+0,4,true
+0,5,false
+0,6,true
+0,7,false
+0,8,true
+0,9,false
+0,10,true
diff --git a/datafusion/core/tests/data/partitioned_csv/partition-1.csv b/datafusion/core/tests/data/partitioned_csv/partition-1.csv
new file mode 100644
index 000000000..b8977f1c5
--- /dev/null
+++ b/datafusion/core/tests/data/partitioned_csv/partition-1.csv
@@ -0,0 +1,11 @@
+1,0,true
+1,1,false
+1,2,true
+1,3,false
+1,4,true
+1,5,false
+1,6,true
+1,7,false
+1,8,true
+1,9,false
+1,10,true
diff --git a/datafusion/core/tests/data/partitioned_csv/partition-2.csv b/datafusion/core/tests/data/partitioned_csv/partition-2.csv
new file mode 100644
index 000000000..7e42564ab
--- /dev/null
+++ b/datafusion/core/tests/data/partitioned_csv/partition-2.csv
@@ -0,0 +1,11 @@
+2,0,true
+2,1,false
+2,2,true
+2,3,false
+2,4,true
+2,5,false
+2,6,true
+2,7,false
+2,8,true
+2,9,false
+2,10,true
diff --git a/datafusion/core/tests/data/partitioned_csv/partition-3.csv b/datafusion/core/tests/data/partitioned_csv/partition-3.csv
new file mode 100644
index 000000000..29aabba4a
--- /dev/null
+++ b/datafusion/core/tests/data/partitioned_csv/partition-3.csv
@@ -0,0 +1,11 @@
+3,0,true
+3,1,false
+3,2,true
+3,3,false
+3,4,true
+3,5,false
+3,6,true
+3,7,false
+3,8,true
+3,9,false
+3,10,true
diff --git a/datafusion/core/tests/sql/order.rs b/datafusion/core/tests/sql/order.rs
index 8f0861245..2388eebef 100644
--- a/datafusion/core/tests/sql/order.rs
+++ b/datafusion/core/tests/sql/order.rs
@@ -18,199 +18,6 @@
 use super::*;
 use test_utils::{batches_to_vec, partitions_to_sorted_vec};
 
-#[tokio::test]
-async fn test_sort_unprojected_col() -> Result<()> {
-    let ctx = SessionContext::new();
-    register_alltypes_parquet(&ctx).await;
-    // execute the query
-    let sql = "SELECT id FROM alltypes_plain ORDER BY int_col, double_col";
-    let actual = execute_to_batches(&ctx, sql).await;
-    #[rustfmt::skip]
-    let expected = vec![
-        "+----+",
-        "| id |",
-        "+----+",
-        "| 4  |",
-        "| 6  |",
-        "| 2  |",
-        "| 0  |",
-        "| 5  |",
-        "| 7  |",
-        "| 3  |",
-        "| 1  |",
-        "+----+",
-    ];
-    assert_batches_eq!(expected, &actual);
-    Ok(())
-}
-
-#[tokio::test]
-async fn test_order_by_agg_expr() -> Result<()> {
-    let ctx = SessionContext::new();
-    register_aggregate_csv(&ctx).await?;
-    let sql = "SELECT MIN(c12) FROM aggregate_test_100 ORDER BY MIN(c12)";
-    let actual = execute_to_batches(&ctx, sql).await;
-    let expected = vec![
-        "+-----------------------------+",
-        "| MIN(aggregate_test_100.c12) |",
-        "+-----------------------------+",
-        "| 0.01479305307777301         |",
-        "+-----------------------------+",
-    ];
-    assert_batches_eq!(expected, &actual);
-
-    let sql = "SELECT MIN(c12) FROM aggregate_test_100 ORDER BY MIN(c12) + 0.1";
-    let actual = execute_to_batches(&ctx, sql).await;
-    assert_batches_eq!(expected, &actual);
-    Ok(())
-}
-
-#[tokio::test]
-async fn test_nulls_first_asc() -> Result<()> {
-    let ctx = SessionContext::new();
-    let sql = "SELECT * FROM (VALUES (1, 'one'), (2, 'two'), (null, 'three')) AS t (num,letter) ORDER BY num";
-    let actual = execute_to_batches(&ctx, sql).await;
-    let expected = vec![
-        "+-----+--------+",
-        "| num | letter |",
-        "+-----+--------+",
-        "| 1   | one    |",
-        "| 2   | two    |",
-        "|     | three  |",
-        "+-----+--------+",
-    ];
-    assert_batches_eq!(expected, &actual);
-    Ok(())
-}
-
-#[tokio::test]
-async fn test_nulls_first_desc() -> Result<()> {
-    let ctx = SessionContext::new();
-    let sql = "SELECT * FROM (VALUES (1, 'one'), (2, 'two'), (null, 'three')) AS t (num,letter) ORDER BY num DESC";
-    let actual = execute_to_batches(&ctx, sql).await;
-    let expected = vec![
-        "+-----+--------+",
-        "| num | letter |",
-        "+-----+--------+",
-        "|     | three  |",
-        "| 2   | two    |",
-        "| 1   | one    |",
-        "+-----+--------+",
-    ];
-    assert_batches_eq!(expected, &actual);
-    Ok(())
-}
-
-#[tokio::test]
-async fn test_specific_nulls_last_desc() -> Result<()> {
-    let ctx = SessionContext::new();
-    let sql = "SELECT * FROM (VALUES (1, 'one'), (2, 'two'), (null, 'three')) AS t (num,letter) ORDER BY num DESC NULLS LAST";
-    let actual = execute_to_batches(&ctx, sql).await;
-    let expected = vec![
-        "+-----+--------+",
-        "| num | letter |",
-        "+-----+--------+",
-        "| 2   | two    |",
-        "| 1   | one    |",
-        "|     | three  |",
-        "+-----+--------+",
-    ];
-    assert_batches_eq!(expected, &actual);
-    Ok(())
-}
-
-#[tokio::test]
-async fn test_specific_nulls_first_asc() -> Result<()> {
-    let ctx = SessionContext::new();
-    let sql = "SELECT * FROM (VALUES (1, 'one'), (2, 'two'), (null, 'three')) AS t (num,letter) ORDER BY num ASC NULLS FIRST";
-    let actual = execute_to_batches(&ctx, sql).await;
-    let expected = vec![
-        "+-----+--------+",
-        "| num | letter |",
-        "+-----+--------+",
-        "|     | three  |",
-        "| 1   | one    |",
-        "| 2   | two    |",
-        "+-----+--------+",
-    ];
-    assert_batches_eq!(expected, &actual);
-    Ok(())
-}
-
-#[tokio::test]
-async fn sort() -> Result<()> {
-    let results =
-        partitioned_csv::execute("SELECT c1, c2 FROM test ORDER BY c1 DESC, c2 ASC", 4)
-            .await?;
-    assert_eq!(results.len(), 1);
-
-    let expected: Vec<&str> = vec![
-        "+----+----+",
-        "| c1 | c2 |",
-        "+----+----+",
-        "| 3  | 1  |",
-        "| 3  | 2  |",
-        "| 3  | 3  |",
-        "| 3  | 4  |",
-        "| 3  | 5  |",
-        "| 3  | 6  |",
-        "| 3  | 7  |",
-        "| 3  | 8  |",
-        "| 3  | 9  |",
-        "| 3  | 10 |",
-        "| 2  | 1  |",
-        "| 2  | 2  |",
-        "| 2  | 3  |",
-        "| 2  | 4  |",
-        "| 2  | 5  |",
-        "| 2  | 6  |",
-        "| 2  | 7  |",
-        "| 2  | 8  |",
-        "| 2  | 9  |",
-        "| 2  | 10 |",
-        "| 1  | 1  |",
-        "| 1  | 2  |",
-        "| 1  | 3  |",
-        "| 1  | 4  |",
-        "| 1  | 5  |",
-        "| 1  | 6  |",
-        "| 1  | 7  |",
-        "| 1  | 8  |",
-        "| 1  | 9  |",
-        "| 1  | 10 |",
-        "| 0  | 1  |",
-        "| 0  | 2  |",
-        "| 0  | 3  |",
-        "| 0  | 4  |",
-        "| 0  | 5  |",
-        "| 0  | 6  |",
-        "| 0  | 7  |",
-        "| 0  | 8  |",
-        "| 0  | 9  |",
-        "| 0  | 10 |",
-        "+----+----+",
-    ];
-
-    // Note it is important to NOT use assert_batches_sorted_eq
-    // here as we are testing the sortedness of the output
-    assert_batches_eq!(expected, &results);
-
-    Ok(())
-}
-
-#[tokio::test]
-async fn sort_empty() -> Result<()> {
-    // The predicate on this query purposely generates no results
-    let results = partitioned_csv::execute(
-        "SELECT c1, c2 FROM test WHERE c1 > 100000 ORDER BY c1 DESC, c2 ASC",
-        4,
-    )
-    .await
-    .unwrap();
-    assert_eq!(results.len(), 0);
-    Ok(())
-}
-
 #[tokio::test]
 async fn sort_with_lots_of_repetition_values() -> Result<()> {
     let ctx = SessionContext::new();
diff --git a/datafusion/core/tests/sqllogictests/test_files/order.slt b/datafusion/core/tests/sqllogictests/test_files/order.slt
new file mode 100644
index 000000000..5afdd0131
--- /dev/null
+++ b/datafusion/core/tests/sqllogictests/test_files/order.slt
@@ -0,0 +1,165 @@
+# 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.
+
+
+statement ok
+CREATE EXTERNAL TABLE alltypes_plain STORED AS PARQUET LOCATION '../../parquet-testing/data/alltypes_plain.parquet';
+
+statement ok
+CREATE EXTERNAL TABLE aggregate_test_100 (
+  c1  VARCHAR NOT NULL,
+  c2  TINYINT NOT NULL,
+  c3  SMALLINT NOT NULL,
+  c4  SMALLINT,
+  c5  INT,
+  c6  BIGINT NOT NULL,
+  c7  SMALLINT NOT NULL,
+  c8  INT NOT NULL,
+  c9  BIGINT UNSIGNED NOT NULL,
+  c10 VARCHAR NOT NULL,
+  c11 FLOAT NOT NULL,
+  c12 DOUBLE NOT NULL,
+  c13 VARCHAR NOT NULL
+)
+STORED AS CSV
+WITH HEADER ROW
+LOCATION '../../testing/data/csv/aggregate_test_100.csv'
+
+# test_sort_unprojected_col
+query I
+SELECT id FROM alltypes_plain ORDER BY int_col, double_col
+----
+4
+6
+2
+0
+5
+7
+3
+1
+
+
+# test_order_by_agg_expr
+query R
+SELECT MIN(c12) FROM aggregate_test_100 ORDER BY MIN(c12)
+----
+0.014793053078
+
+# test_nulls_first_asc
+
+query RC
+SELECT * FROM (VALUES (1, 'one'), (2, 'two'), (null, 'three')) AS t (num,letter) ORDER BY num
+----
+1 one
+2 two
+NULL three
+
+# test_nulls_first_desc()
+
+query RC
+SELECT * FROM (VALUES (1, 'one'), (2, 'two'), (null, 'three')) AS t (num,letter) ORDER BY num DESC
+----
+NULL three
+2 two
+1 one
+
+# test_specific_nulls_last_desc
+
+query RC
+SELECT * FROM (VALUES (1, 'one'), (2, 'two'), (null, 'three')) AS t (num,letter) ORDER BY num DESC NULLS LAST
+----
+2 two
+1 one
+NULL three
+
+# test_specific_nulls_first_asc
+query RT
+SELECT * FROM (VALUES (1, 'one'), (2, 'two'), (null, 'three')) AS t (num,letter) ORDER BY num ASC NULLS FIRST
+----
+NULL three
+1 one
+2 two
+
+# sort
+
+statement ok
+CREATE EXTERNAL TABLE test (c1 int, c2 bigint, c3 boolean)
+STORED AS CSV LOCATION 'tests/data/partitioned_csv';
+
+# Demonstrate types
+query RC
+SELECT arrow_typeof(c1), arrow_typeof(c2), arrow_typeof(c3) FROM test LIMIT 1;
+----
+Int32 Int64 Boolean
+
+
+query RC
+SELECT c1, c2 FROM test ORDER BY c1 DESC, c2 ASC
+----
+3 0
+3 1
+3 2
+3 3
+3 4
+3 5
+3 6
+3 7
+3 8
+3 9
+3 10
+2 0
+2 1
+2 2
+2 3
+2 4
+2 5
+2 6
+2 7
+2 8
+2 9
+2 10
+1 0
+1 1
+1 2
+1 3
+1 4
+1 5
+1 6
+1 7
+1 8
+1 9
+1 10
+0 0
+0 1
+0 2
+0 3
+0 4
+0 5
+0 6
+0 7
+0 8
+0 9
+0 10
+
+
+
+# sort_empty
+# The predicate on this query purposely generates no results
+
+query RC
+SELECT c1, c2 FROM test WHERE c1 > 100000 ORDER BY c1 DESC, c2 ASC
+----