From d3cfc45cef28a2d707bdd4b14da33a4d38a209d6 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 9 Dec 2024 04:05:37 -0800 Subject: [PATCH 01/14] Minor: Use `div_ceil` --- .../physical-plan/src/joins/cross_join.rs | 27 ++++++++++--------- .../src/joins/nested_loop_join.rs | 24 ++++++++--------- 2 files changed, 26 insertions(+), 25 deletions(-) diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index f53fe13df15e..8bf675e87362 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -190,18 +190,21 @@ async fn load_left_input( // Load all batches and count the rows let (batches, _metrics, reservation) = stream - .try_fold((Vec::new(), metrics, reservation), |mut acc, batch| async { - let batch_size = batch.get_array_memory_size(); - // Reserve memory for incoming batch - acc.2.try_grow(batch_size)?; - // Update metrics - acc.1.build_mem_used.add(batch_size); - acc.1.build_input_batches.add(1); - acc.1.build_input_rows.add(batch.num_rows()); - // Push batch to output - acc.0.push(batch); - Ok(acc) - }) + .try_fold( + (Vec::new(), metrics, reservation), + |(mut batches, metrics, mut reservation), batch| async { + let batch_size = batch.get_array_memory_size(); + // Reserve memory for incoming batch + reservation.try_grow(batch_size)?; + // Update metrics + metrics.build_mem_used.add(batch_size); + metrics.build_input_batches.add(1); + metrics.build_input_rows.add(batch.num_rows()); + // Push batch to output + batches.push(batch); + Ok((batches, metrics, reservation)) + }, + ) .await?; let merged_batch = concat_batches(&left_schema, &batches)?; diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index 2beeb92da499..d174564178df 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -45,7 +45,6 @@ use arrow::array::{BooleanBufferBuilder, UInt32Array, UInt64Array}; use arrow::compute::concat_batches; use arrow::datatypes::{Schema, SchemaRef}; use arrow::record_batch::RecordBatch; -use arrow::util::bit_util; use datafusion_common::{ exec_datafusion_err, internal_err, JoinSide, Result, Statistics, }; @@ -440,17 +439,17 @@ async fn collect_left_input( let (batches, metrics, mut reservation) = stream .try_fold( (Vec::new(), join_metrics, reservation), - |mut acc, batch| async { + |(mut batches, metrics, mut reservation), batch| async { let batch_size = batch.get_array_memory_size(); // Reserve memory for incoming batch - acc.2.try_grow(batch_size)?; + reservation.try_grow(batch_size)?; // Update metrics - acc.1.build_mem_used.add(batch_size); - acc.1.build_input_batches.add(1); - acc.1.build_input_rows.add(batch.num_rows()); + metrics.build_mem_used.add(batch_size); + metrics.build_input_batches.add(1); + metrics.build_input_rows.add(batch.num_rows()); // Push batch to output - acc.0.push(batch); - Ok(acc) + batches.push(batch); + Ok((batches, metrics, reservation)) }, ) .await?; @@ -459,14 +458,13 @@ async fn collect_left_input( // Reserve memory for visited_left_side bitmap if required by join type let visited_left_side = if with_visited_left_side { - // TODO: Replace `ceil` wrapper with stable `div_cell` after - // https://github.com/rust-lang/rust/issues/88581 - let buffer_size = bit_util::ceil(merged_batch.num_rows(), 8); + let n_rows = merged_batch.num_rows(); + let buffer_size = n_rows.div_ceil(8); reservation.try_grow(buffer_size)?; metrics.build_mem_used.add(buffer_size); - let mut buffer = BooleanBufferBuilder::new(merged_batch.num_rows()); - buffer.append_n(merged_batch.num_rows(), false); + let mut buffer = BooleanBufferBuilder::new(n_rows); + buffer.append_n(n_rows, false); buffer } else { BooleanBufferBuilder::new(0) From 6d7b902ff28a0d2de29c90f15ea3568f24888120 Mon Sep 17 00:00:00 2001 From: Huaijin Date: Tue, 10 Dec 2024 04:26:19 +0800 Subject: [PATCH 02/14] Fix hash join with sort push down (#13560) * fix: join with sort push down * chore: insert some value * apply suggestion * recover handle_costom_pushdown change * apply suggestion * add more test * add partition --- .../src/physical_optimizer/sort_pushdown.rs | 101 +++++++++++ datafusion/sqllogictest/test_files/joins.slt | 171 +++++++++++++----- 2 files changed, 228 insertions(+), 44 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index d48c7118cb8e..6c761f674b3b 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -28,6 +28,7 @@ use crate::physical_plan::repartition::RepartitionExec; use crate::physical_plan::sorts::sort::SortExec; use crate::physical_plan::tree_node::PlanContext; use crate::physical_plan::{ExecutionPlan, ExecutionPlanProperties}; +use arrow_schema::SchemaRef; use datafusion_common::tree_node::{ ConcreteTreeNode, Transformed, TreeNode, TreeNodeRecursion, @@ -38,6 +39,8 @@ use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::utils::collect_columns; use datafusion_physical_expr::PhysicalSortRequirement; use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; +use datafusion_physical_plan::joins::utils::ColumnIndex; +use datafusion_physical_plan::joins::HashJoinExec; /// This is a "data class" we use within the [`EnforceSorting`] rule to push /// down [`SortExec`] in the plan. In some cases, we can reduce the total @@ -294,6 +297,8 @@ fn pushdown_requirement_to_children( .then(|| LexRequirement::new(parent_required.to_vec())); Ok(Some(vec![req])) } + } else if let Some(hash_join) = plan.as_any().downcast_ref::() { + handle_hash_join(hash_join, parent_required) } else { handle_custom_pushdown(plan, parent_required, maintains_input_order) } @@ -606,6 +611,102 @@ fn handle_custom_pushdown( } } +// For hash join we only maintain the input order for the right child +// for join type: Inner, Right, RightSemi, RightAnti +fn handle_hash_join( + plan: &HashJoinExec, + parent_required: &LexRequirement, +) -> Result>>> { + // If there's no requirement from the parent or the plan has no children + // or the join type is not Inner, Right, RightSemi, RightAnti, return early + if parent_required.is_empty() || !plan.maintains_input_order()[1] { + return Ok(None); + } + + // Collect all unique column indices used in the parent-required sorting expression + let all_indices: HashSet = parent_required + .iter() + .flat_map(|order| { + collect_columns(&order.expr) + .into_iter() + .map(|col| col.index()) + .collect::>() + }) + .collect(); + + let column_indices = build_join_column_index(plan); + let projected_indices: Vec<_> = if let Some(projection) = &plan.projection { + projection.iter().map(|&i| &column_indices[i]).collect() + } else { + column_indices.iter().collect() + }; + let len_of_left_fields = projected_indices + .iter() + .filter(|ci| ci.side == JoinSide::Left) + .count(); + + let all_from_right_child = all_indices.iter().all(|i| *i >= len_of_left_fields); + + // If all columns are from the right child, update the parent requirements + if all_from_right_child { + // Transform the parent-required expression for the child schema by adjusting columns + let updated_parent_req = parent_required + .iter() + .map(|req| { + let child_schema = plan.children()[1].schema(); + let updated_columns = Arc::clone(&req.expr) + .transform_up(|expr| { + if let Some(col) = expr.as_any().downcast_ref::() { + let index = projected_indices[col.index()].index; + Ok(Transformed::yes(Arc::new(Column::new( + child_schema.field(index).name(), + index, + )))) + } else { + Ok(Transformed::no(expr)) + } + })? + .data; + Ok(PhysicalSortRequirement::new(updated_columns, req.options)) + }) + .collect::>>()?; + + // Populating with the updated requirements for children that maintain order + Ok(Some(vec![ + None, + Some(LexRequirement::new(updated_parent_req)), + ])) + } else { + Ok(None) + } +} + +// this function is used to build the column index for the hash join +// push down sort requirements to the right child +fn build_join_column_index(plan: &HashJoinExec) -> Vec { + let map_fields = |schema: SchemaRef, side: JoinSide| { + schema + .fields() + .iter() + .enumerate() + .map(|(index, _)| ColumnIndex { index, side }) + .collect::>() + }; + + match plan.join_type() { + JoinType::Inner | JoinType::Right => { + map_fields(plan.left().schema(), JoinSide::Left) + .into_iter() + .chain(map_fields(plan.right().schema(), JoinSide::Right)) + .collect::>() + } + JoinType::RightSemi | JoinType::RightAnti => { + map_fields(plan.right().schema(), JoinSide::Right) + } + _ => unreachable!("unexpected join type: {}", plan.join_type()), + } +} + /// Define the Requirements Compatibility #[derive(Debug)] enum RequirementsCompatibility { diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index e636e93007a4..62f625119897 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -2864,13 +2864,13 @@ explain SELECT t1_id, t1_name FROM left_semi_anti_join_table_t1 t1 WHERE t1_id I ---- physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] -02)--SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----CoalesceBatchesExec: target_batch_size=2 -04)------HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] -05)--------CoalesceBatchesExec: target_batch_size=2 -06)----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 -07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -08)--------------MemoryExec: partitions=1, partition_sizes=[1] +02)--CoalesceBatchesExec: target_batch_size=2 +03)----HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] +04)------CoalesceBatchesExec: target_batch_size=2 +05)--------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 +06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)------------MemoryExec: partitions=1, partition_sizes=[1] +08)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] 09)--------CoalesceBatchesExec: target_batch_size=2 10)----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 11)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -2905,13 +2905,13 @@ explain SELECT t1_id, t1_name FROM left_semi_anti_join_table_t1 t1 LEFT SEMI JOI ---- physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] -02)--SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----CoalesceBatchesExec: target_batch_size=2 -04)------HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] -05)--------CoalesceBatchesExec: target_batch_size=2 -06)----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 -07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -08)--------------MemoryExec: partitions=1, partition_sizes=[1] +02)--CoalesceBatchesExec: target_batch_size=2 +03)----HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] +04)------CoalesceBatchesExec: target_batch_size=2 +05)--------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 +06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)------------MemoryExec: partitions=1, partition_sizes=[1] +08)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] 09)--------CoalesceBatchesExec: target_batch_size=2 10)----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 11)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -2967,10 +2967,10 @@ explain SELECT t1_id, t1_name FROM left_semi_anti_join_table_t1 t1 WHERE t1_id I ---- physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] -02)--SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----CoalesceBatchesExec: target_batch_size=2 -04)------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] -05)--------MemoryExec: partitions=1, partition_sizes=[1] +02)--CoalesceBatchesExec: target_batch_size=2 +03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] +04)------MemoryExec: partitions=1, partition_sizes=[1] +05)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] 06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 07)----------MemoryExec: partitions=1, partition_sizes=[1] @@ -3003,10 +3003,10 @@ explain SELECT t1_id, t1_name FROM left_semi_anti_join_table_t1 t1 LEFT SEMI JOI ---- physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] -02)--SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----CoalesceBatchesExec: target_batch_size=2 -04)------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] -05)--------MemoryExec: partitions=1, partition_sizes=[1] +02)--CoalesceBatchesExec: target_batch_size=2 +03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] +04)------MemoryExec: partitions=1, partition_sizes=[1] +05)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] 06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 07)----------MemoryExec: partitions=1, partition_sizes=[1] @@ -3061,13 +3061,13 @@ explain SELECT t1_id, t1_name, t1_int FROM right_semi_anti_join_table_t1 t1 WHER ---- physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] -02)--SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----CoalesceBatchesExec: target_batch_size=2 -04)------HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@1 != t1_name@0 -05)--------CoalesceBatchesExec: target_batch_size=2 -06)----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 -07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -08)--------------MemoryExec: partitions=1, partition_sizes=[1] +02)--CoalesceBatchesExec: target_batch_size=2 +03)----HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@1 != t1_name@0 +04)------CoalesceBatchesExec: target_batch_size=2 +05)--------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 +06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)------------MemoryExec: partitions=1, partition_sizes=[1] +08)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] 09)--------CoalesceBatchesExec: target_batch_size=2 10)----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 11)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -3083,13 +3083,13 @@ explain SELECT t1_id, t1_name, t1_int FROM right_semi_anti_join_table_t2 t2 RIGH ---- physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] -02)--SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----CoalesceBatchesExec: target_batch_size=2 -04)------HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@0 != t1_name@1 -05)--------CoalesceBatchesExec: target_batch_size=2 -06)----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 -07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -08)--------------MemoryExec: partitions=1, partition_sizes=[1] +02)--CoalesceBatchesExec: target_batch_size=2 +03)----HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@0 != t1_name@1 +04)------CoalesceBatchesExec: target_batch_size=2 +05)--------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 +06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)------------MemoryExec: partitions=1, partition_sizes=[1] +08)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] 09)--------CoalesceBatchesExec: target_batch_size=2 10)----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 11)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -3143,10 +3143,10 @@ explain SELECT t1_id, t1_name, t1_int FROM right_semi_anti_join_table_t1 t1 WHER ---- physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] -02)--SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----CoalesceBatchesExec: target_batch_size=2 -04)------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@1 != t1_name@0 -05)--------MemoryExec: partitions=1, partition_sizes=[1] +02)--CoalesceBatchesExec: target_batch_size=2 +03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@1 != t1_name@0 +04)------MemoryExec: partitions=1, partition_sizes=[1] +05)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] 06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 07)----------MemoryExec: partitions=1, partition_sizes=[1] @@ -3160,10 +3160,10 @@ explain SELECT t1_id, t1_name, t1_int FROM right_semi_anti_join_table_t2 t2 RIGH ---- physical_plan 01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] -02)--SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----CoalesceBatchesExec: target_batch_size=2 -04)------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@0 != t1_name@1 -05)--------MemoryExec: partitions=1, partition_sizes=[1] +02)--CoalesceBatchesExec: target_batch_size=2 +03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@0 != t1_name@1 +04)------MemoryExec: partitions=1, partition_sizes=[1] +05)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] 06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 07)----------MemoryExec: partitions=1, partition_sizes=[1] @@ -4313,3 +4313,86 @@ physical_plan 04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(binary_col@0, binary_col@0)] 05)--------MemoryExec: partitions=1, partition_sizes=[1] 06)--------MemoryExec: partitions=1, partition_sizes=[1] + +# Test hash join sort push down +# Issue: https://github.com/apache/datafusion/issues/13559 +statement ok +CREATE TABLE test(a INT, b INT, c INT) + +statement ok +insert into test values (1,2,3), (4,5,6), (null, 7, 8), (8, null, 9), (9, 10, null) + +statement ok +set datafusion.execution.target_partitions = 2; + +query TT +explain select * from test where a in (select a from test where b > 3) order by c desc nulls first; +---- +logical_plan +01)Sort: test.c DESC NULLS FIRST +02)--LeftSemi Join: test.a = __correlated_sq_1.a +03)----TableScan: test projection=[a, b, c] +04)----SubqueryAlias: __correlated_sq_1 +05)------Projection: test.a +06)--------Filter: test.b > Int32(3) +07)----------TableScan: test projection=[a, b] +physical_plan +01)SortPreservingMergeExec: [c@2 DESC] +02)--CoalesceBatchesExec: target_batch_size=3 +03)----HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(a@0, a@0)] +04)------CoalesceBatchesExec: target_batch_size=3 +05)--------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 +06)----------CoalesceBatchesExec: target_batch_size=3 +07)------------FilterExec: b@1 > 3, projection=[a@0] +08)--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +09)----------------MemoryExec: partitions=1, partition_sizes=[1] +10)------SortExec: expr=[c@2 DESC], preserve_partitioning=[true] +11)--------CoalesceBatchesExec: target_batch_size=3 +12)----------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 +13)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +14)--------------MemoryExec: partitions=1, partition_sizes=[1] + +query TT +explain select * from test where a in (select a from test where b > 3) order by c desc nulls last; +---- +logical_plan +01)Sort: test.c DESC NULLS LAST +02)--LeftSemi Join: test.a = __correlated_sq_1.a +03)----TableScan: test projection=[a, b, c] +04)----SubqueryAlias: __correlated_sq_1 +05)------Projection: test.a +06)--------Filter: test.b > Int32(3) +07)----------TableScan: test projection=[a, b] +physical_plan +01)SortPreservingMergeExec: [c@2 DESC NULLS LAST] +02)--CoalesceBatchesExec: target_batch_size=3 +03)----HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(a@0, a@0)] +04)------CoalesceBatchesExec: target_batch_size=3 +05)--------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 +06)----------CoalesceBatchesExec: target_batch_size=3 +07)------------FilterExec: b@1 > 3, projection=[a@0] +08)--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +09)----------------MemoryExec: partitions=1, partition_sizes=[1] +10)------SortExec: expr=[c@2 DESC NULLS LAST], preserve_partitioning=[true] +11)--------CoalesceBatchesExec: target_batch_size=3 +12)----------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 +13)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +14)--------------MemoryExec: partitions=1, partition_sizes=[1] + +query III +select * from test where a in (select a from test where b > 3) order by c desc nulls first; +---- +9 10 NULL +4 5 6 + +query III +select * from test where a in (select a from test where b > 3) order by c desc nulls last; +---- +4 5 6 +9 10 NULL + +statement ok +DROP TABLE test + +statement ok +set datafusion.execution.target_partitions = 1; From b73734f9b4277b421ec790436e641d6f40dd1560 Mon Sep 17 00:00:00 2001 From: Zhang Li Date: Tue, 10 Dec 2024 05:49:19 +0800 Subject: [PATCH 03/14] Improve substr() performance by avoiding using owned string (#13688) Co-authored-by: zhangli20 --- datafusion/functions/src/unicode/substr.rs | 77 +++++++++++----------- 1 file changed, 40 insertions(+), 37 deletions(-) diff --git a/datafusion/functions/src/unicode/substr.rs b/datafusion/functions/src/unicode/substr.rs index 141984cf2674..687f77dbef5b 100644 --- a/datafusion/functions/src/unicode/substr.rs +++ b/datafusion/functions/src/unicode/substr.rs @@ -21,8 +21,8 @@ use std::sync::{Arc, OnceLock}; use crate::strings::{make_and_append_view, StringArrayType}; use crate::utils::{make_scalar_function, utf8_to_str_type}; use arrow::array::{ - Array, ArrayIter, ArrayRef, AsArray, GenericStringArray, Int64Array, OffsetSizeTrait, - StringViewArray, + Array, ArrayIter, ArrayRef, AsArray, GenericStringBuilder, Int64Array, + OffsetSizeTrait, StringViewArray, }; use arrow::datatypes::DataType; use arrow_buffer::{NullBufferBuilder, ScalarBuffer}; @@ -448,10 +448,9 @@ where match args.len() { 1 => { let iter = ArrayIter::new(string_array); - - let result = iter - .zip(start_array.iter()) - .map(|(string, start)| match (string, start) { + let mut result_builder = GenericStringBuilder::::new(); + for (string, start) in iter.zip(start_array.iter()) { + match (string, start) { (Some(string), Some(start)) => { let (start, end) = get_true_start_end( string, @@ -460,47 +459,51 @@ where enable_ascii_fast_path, ); // start, end is byte-based let substr = &string[start..end]; - Some(substr.to_string()) + result_builder.append_value(substr); } - _ => None, - }) - .collect::>(); - Ok(Arc::new(result) as ArrayRef) + _ => { + result_builder.append_null(); + } + } + } + Ok(Arc::new(result_builder.finish()) as ArrayRef) } 2 => { let iter = ArrayIter::new(string_array); let count_array = count_array_opt.unwrap(); + let mut result_builder = GenericStringBuilder::::new(); - let result = iter - .zip(start_array.iter()) - .zip(count_array.iter()) - .map(|((string, start), count)| { - match (string, start, count) { - (Some(string), Some(start), Some(count)) => { - if count < 0 { - exec_err!( + for ((string, start), count) in + iter.zip(start_array.iter()).zip(count_array.iter()) + { + match (string, start, count) { + (Some(string), Some(start), Some(count)) => { + if count < 0 { + return exec_err!( "negative substring length not allowed: substr(, {start}, {count})" - ) - } else { - if start == i64::MIN { - return exec_err!("negative overflow when calculating skip value"); - } - let (start, end) = get_true_start_end( - string, - start, - Some(count as u64), - enable_ascii_fast_path, - ); // start, end is byte-based - let substr = &string[start..end]; - Ok(Some(substr.to_string())) + ); + } else { + if start == i64::MIN { + return exec_err!( + "negative overflow when calculating skip value" + ); } + let (start, end) = get_true_start_end( + string, + start, + Some(count as u64), + enable_ascii_fast_path, + ); // start, end is byte-based + let substr = &string[start..end]; + result_builder.append_value(substr); } - _ => Ok(None), } - }) - .collect::>>()?; - - Ok(Arc::new(result) as ArrayRef) + _ => { + result_builder.append_null(); + } + } + } + Ok(Arc::new(result_builder.finish()) as ArrayRef) } other => { exec_err!("substr was called with {other} arguments. It requires 2 or 3.") From 6fae5a078b43466ea6f2ecd47e521744dff8e384 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 9 Dec 2024 14:49:58 -0700 Subject: [PATCH 04/14] reinstate down_cast_any_ref (#13705) --- .../physical-expr-common/src/physical_expr.rs | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/datafusion/physical-expr-common/src/physical_expr.rs b/datafusion/physical-expr-common/src/physical_expr.rs index 93bdcdef8ea0..c2e892d63da0 100644 --- a/datafusion/physical-expr-common/src/physical_expr.rs +++ b/datafusion/physical-expr-common/src/physical_expr.rs @@ -214,6 +214,21 @@ pub fn with_new_children_if_necessary( } } +#[deprecated(since = "44.0.0")] +pub fn down_cast_any_ref(any: &dyn Any) -> &dyn Any { + if any.is::>() { + any.downcast_ref::>() + .unwrap() + .as_any() + } else if any.is::>() { + any.downcast_ref::>() + .unwrap() + .as_any() + } else { + any + } +} + /// Returns [`Display`] able a list of [`PhysicalExpr`] /// /// Example output: `[a + 1, b]` From 437cbf847a80beae791af821d73c56d8226473be Mon Sep 17 00:00:00 2001 From: Tai Le Manh Date: Tue, 10 Dec 2024 09:33:25 +0700 Subject: [PATCH 05/14] Optimize performance of `character_length` function (#13696) * Optimize performance of function Signed-off-by: Tai Le Manh * Add pre-check array is null * Fix clippy warnings --------- Signed-off-by: Tai Le Manh --- .../functions/src/unicode/character_length.rs | 57 +++++++++++++------ 1 file changed, 39 insertions(+), 18 deletions(-) diff --git a/datafusion/functions/src/unicode/character_length.rs b/datafusion/functions/src/unicode/character_length.rs index 822bdca9aca8..ad51a8ef72fb 100644 --- a/datafusion/functions/src/unicode/character_length.rs +++ b/datafusion/functions/src/unicode/character_length.rs @@ -18,7 +18,7 @@ use crate::strings::StringArrayType; use crate::utils::{make_scalar_function, utf8_to_int_type}; use arrow::array::{ - Array, ArrayRef, ArrowPrimitiveType, AsArray, OffsetSizeTrait, PrimitiveArray, + Array, ArrayRef, ArrowPrimitiveType, AsArray, OffsetSizeTrait, PrimitiveBuilder, }; use arrow::datatypes::{ArrowNativeType, DataType, Int32Type, Int64Type}; use datafusion_common::Result; @@ -136,31 +136,52 @@ fn character_length(args: &[ArrayRef]) -> Result { } } -fn character_length_general<'a, T: ArrowPrimitiveType, V: StringArrayType<'a>>( - array: V, -) -> Result +fn character_length_general<'a, T, V>(array: V) -> Result where + T: ArrowPrimitiveType, T::Native: OffsetSizeTrait, + V: StringArrayType<'a>, { + let mut builder = PrimitiveBuilder::::with_capacity(array.len()); + // String characters are variable length encoded in UTF-8, counting the // number of chars requires expensive decoding, however checking if the // string is ASCII only is relatively cheap. // If strings are ASCII only, count bytes instead. let is_array_ascii_only = array.is_ascii(); - let iter = array.iter(); - let result = iter - .map(|string| { - string.map(|string: &str| { - if is_array_ascii_only { - T::Native::usize_as(string.len()) - } else { - T::Native::usize_as(string.chars().count()) - } - }) - }) - .collect::>(); - - Ok(Arc::new(result) as ArrayRef) + if array.null_count() == 0 { + if is_array_ascii_only { + for i in 0..array.len() { + let value = array.value(i); + builder.append_value(T::Native::usize_as(value.len())); + } + } else { + for i in 0..array.len() { + let value = array.value(i); + builder.append_value(T::Native::usize_as(value.chars().count())); + } + } + } else if is_array_ascii_only { + for i in 0..array.len() { + if array.is_null(i) { + builder.append_null(); + } else { + let value = array.value(i); + builder.append_value(T::Native::usize_as(value.len())); + } + } + } else { + for i in 0..array.len() { + if array.is_null(i) { + builder.append_null(); + } else { + let value = array.value(i); + builder.append_value(T::Native::usize_as(value.chars().count())); + } + } + } + + Ok(Arc::new(builder.finish()) as ArrayRef) } #[cfg(test)] From becd0c9c7544c5dbde99d03a9002bb2d93553c70 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 10 Dec 2024 02:21:58 -0800 Subject: [PATCH 06/14] Update prost-build requirement from =0.13.3 to =0.13.4 (#13698) Updates the requirements on [prost-build](https://github.com/tokio-rs/prost) to permit the latest version. - [Release notes](https://github.com/tokio-rs/prost/releases) - [Changelog](https://github.com/tokio-rs/prost/blob/master/CHANGELOG.md) - [Commits](https://github.com/tokio-rs/prost/compare/v0.13.3...v0.13.4) --- updated-dependencies: - dependency-name: prost-build dependency-type: direct:production ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- datafusion/proto-common/gen/Cargo.toml | 2 +- datafusion/proto/gen/Cargo.toml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/proto-common/gen/Cargo.toml b/datafusion/proto-common/gen/Cargo.toml index da5bc6029ff9..21fc9eccb40c 100644 --- a/datafusion/proto-common/gen/Cargo.toml +++ b/datafusion/proto-common/gen/Cargo.toml @@ -35,4 +35,4 @@ workspace = true [dependencies] # Pin these dependencies so that the generated output is deterministic pbjson-build = "=0.7.0" -prost-build = "=0.13.3" +prost-build = "=0.13.4" diff --git a/datafusion/proto/gen/Cargo.toml b/datafusion/proto/gen/Cargo.toml index 297406becada..dda72d20a159 100644 --- a/datafusion/proto/gen/Cargo.toml +++ b/datafusion/proto/gen/Cargo.toml @@ -35,4 +35,4 @@ workspace = true [dependencies] # Pin these dependencies so that the generated output is deterministic pbjson-build = "=0.7.0" -prost-build = "=0.13.3" +prost-build = "=0.13.4" From 62fc4dbfde65a0376388a48ae8c82b3b72c8dd57 Mon Sep 17 00:00:00 2001 From: Oleks V Date: Tue, 10 Dec 2024 12:18:39 -0800 Subject: [PATCH 07/14] Minor: Output elapsed time for sql logic test (#13718) * Minor: Output elapsed time for sql logic test --- datafusion/sqllogictest/bin/sqllogictests.rs | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/datafusion/sqllogictest/bin/sqllogictests.rs b/datafusion/sqllogictest/bin/sqllogictests.rs index 12c0e27ea911..176bd3229125 100644 --- a/datafusion/sqllogictest/bin/sqllogictests.rs +++ b/datafusion/sqllogictest/bin/sqllogictests.rs @@ -15,10 +15,6 @@ // specific language governing permissions and limitations // under the License. -use std::ffi::OsStr; -use std::fs; -use std::path::{Path, PathBuf}; - use clap::Parser; use datafusion_common::utils::get_available_parallelism; use datafusion_sqllogictest::{DataFusion, TestContext}; @@ -26,6 +22,9 @@ use futures::stream::StreamExt; use itertools::Itertools; use log::info; use sqllogictest::strict_column_validator; +use std::ffi::OsStr; +use std::fs; +use std::path::{Path, PathBuf}; use datafusion_common::{exec_datafusion_err, exec_err, DataFusionError, Result}; use datafusion_common_runtime::SpawnedTask; @@ -100,7 +99,8 @@ async fn run_tests() -> Result<()> { let errors: Vec<_> = futures::stream::iter(read_test_files(&options)?) .map(|test_file| { SpawnedTask::spawn(async move { - println!("Running {:?}", test_file.relative_path); + let file_path = test_file.relative_path.clone(); + let start = datafusion::common::instant::Instant::now(); if options.complete { run_complete_file(test_file).await?; } else if options.postgres_runner { @@ -108,6 +108,7 @@ async fn run_tests() -> Result<()> { } else { run_test_file(test_file).await?; } + println!("Executed {:?}. Took {:?}", file_path, start.elapsed()); Ok(()) as Result<()> }) .join() From 0f5634ec3de8cae77804aae883b655db38da4648 Mon Sep 17 00:00:00 2001 From: Jonah Gao Date: Wed, 11 Dec 2024 09:14:44 +0800 Subject: [PATCH 08/14] refactor: simplify the `make_udf_function` macro (#13712) --- datafusion/functions/src/core/mod.rs | 22 +++++----- datafusion/functions/src/crypto/mod.rs | 12 +++--- datafusion/functions/src/datetime/mod.rs | 54 ++++++++---------------- datafusion/functions/src/encoding/mod.rs | 4 +- datafusion/functions/src/macros.rs | 25 ++++++----- datafusion/functions/src/math/mod.rs | 53 +++++++---------------- datafusion/functions/src/regex/mod.rs | 12 ++---- datafusion/functions/src/string/mod.rs | 44 +++++++++---------- datafusion/functions/src/unicode/mod.rs | 28 ++++++------ 9 files changed, 101 insertions(+), 153 deletions(-) diff --git a/datafusion/functions/src/core/mod.rs b/datafusion/functions/src/core/mod.rs index 24d26c539539..bd8305cd56d8 100644 --- a/datafusion/functions/src/core/mod.rs +++ b/datafusion/functions/src/core/mod.rs @@ -35,17 +35,17 @@ pub mod r#struct; pub mod version; // create UDFs -make_udf_function!(arrow_cast::ArrowCastFunc, ARROW_CAST, arrow_cast); -make_udf_function!(nullif::NullIfFunc, NULLIF, nullif); -make_udf_function!(nvl::NVLFunc, NVL, nvl); -make_udf_function!(nvl2::NVL2Func, NVL2, nvl2); -make_udf_function!(arrowtypeof::ArrowTypeOfFunc, ARROWTYPEOF, arrow_typeof); -make_udf_function!(r#struct::StructFunc, STRUCT, r#struct); -make_udf_function!(named_struct::NamedStructFunc, NAMED_STRUCT, named_struct); -make_udf_function!(getfield::GetFieldFunc, GET_FIELD, get_field); -make_udf_function!(coalesce::CoalesceFunc, COALESCE, coalesce); -make_udf_function!(greatest::GreatestFunc, GREATEST, greatest); -make_udf_function!(version::VersionFunc, VERSION, version); +make_udf_function!(arrow_cast::ArrowCastFunc, arrow_cast); +make_udf_function!(nullif::NullIfFunc, nullif); +make_udf_function!(nvl::NVLFunc, nvl); +make_udf_function!(nvl2::NVL2Func, nvl2); +make_udf_function!(arrowtypeof::ArrowTypeOfFunc, arrow_typeof); +make_udf_function!(r#struct::StructFunc, r#struct); +make_udf_function!(named_struct::NamedStructFunc, named_struct); +make_udf_function!(getfield::GetFieldFunc, get_field); +make_udf_function!(coalesce::CoalesceFunc, coalesce); +make_udf_function!(greatest::GreatestFunc, greatest); +make_udf_function!(version::VersionFunc, version); pub mod expr_fn { use datafusion_expr::{Expr, Literal}; diff --git a/datafusion/functions/src/crypto/mod.rs b/datafusion/functions/src/crypto/mod.rs index 46177fc22b60..62ea3c2e2737 100644 --- a/datafusion/functions/src/crypto/mod.rs +++ b/datafusion/functions/src/crypto/mod.rs @@ -27,12 +27,12 @@ pub mod sha224; pub mod sha256; pub mod sha384; pub mod sha512; -make_udf_function!(digest::DigestFunc, DIGEST, digest); -make_udf_function!(md5::Md5Func, MD5, md5); -make_udf_function!(sha224::SHA224Func, SHA224, sha224); -make_udf_function!(sha256::SHA256Func, SHA256, sha256); -make_udf_function!(sha384::SHA384Func, SHA384, sha384); -make_udf_function!(sha512::SHA512Func, SHA512, sha512); +make_udf_function!(digest::DigestFunc, digest); +make_udf_function!(md5::Md5Func, md5); +make_udf_function!(sha224::SHA224Func, sha224); +make_udf_function!(sha256::SHA256Func, sha256); +make_udf_function!(sha384::SHA384Func, sha384); +make_udf_function!(sha512::SHA512Func, sha512); pub mod expr_fn { export_functions!(( diff --git a/datafusion/functions/src/datetime/mod.rs b/datafusion/functions/src/datetime/mod.rs index db4e365267dd..96ca63010ee4 100644 --- a/datafusion/functions/src/datetime/mod.rs +++ b/datafusion/functions/src/datetime/mod.rs @@ -37,43 +37,23 @@ pub mod to_timestamp; pub mod to_unixtime; // create UDFs -make_udf_function!(current_date::CurrentDateFunc, CURRENT_DATE, current_date); -make_udf_function!(current_time::CurrentTimeFunc, CURRENT_TIME, current_time); -make_udf_function!(date_bin::DateBinFunc, DATE_BIN, date_bin); -make_udf_function!(date_part::DatePartFunc, DATE_PART, date_part); -make_udf_function!(date_trunc::DateTruncFunc, DATE_TRUNC, date_trunc); -make_udf_function!(make_date::MakeDateFunc, MAKE_DATE, make_date); -make_udf_function!( - from_unixtime::FromUnixtimeFunc, - FROM_UNIXTIME, - from_unixtime -); -make_udf_function!(now::NowFunc, NOW, now); -make_udf_function!(to_char::ToCharFunc, TO_CHAR, to_char); -make_udf_function!(to_date::ToDateFunc, TO_DATE, to_date); -make_udf_function!(to_local_time::ToLocalTimeFunc, TO_LOCAL_TIME, to_local_time); -make_udf_function!(to_unixtime::ToUnixtimeFunc, TO_UNIXTIME, to_unixtime); -make_udf_function!(to_timestamp::ToTimestampFunc, TO_TIMESTAMP, to_timestamp); -make_udf_function!( - to_timestamp::ToTimestampSecondsFunc, - TO_TIMESTAMP_SECONDS, - to_timestamp_seconds -); -make_udf_function!( - to_timestamp::ToTimestampMillisFunc, - TO_TIMESTAMP_MILLIS, - to_timestamp_millis -); -make_udf_function!( - to_timestamp::ToTimestampMicrosFunc, - TO_TIMESTAMP_MICROS, - to_timestamp_micros -); -make_udf_function!( - to_timestamp::ToTimestampNanosFunc, - TO_TIMESTAMP_NANOS, - to_timestamp_nanos -); +make_udf_function!(current_date::CurrentDateFunc, current_date); +make_udf_function!(current_time::CurrentTimeFunc, current_time); +make_udf_function!(date_bin::DateBinFunc, date_bin); +make_udf_function!(date_part::DatePartFunc, date_part); +make_udf_function!(date_trunc::DateTruncFunc, date_trunc); +make_udf_function!(make_date::MakeDateFunc, make_date); +make_udf_function!(from_unixtime::FromUnixtimeFunc, from_unixtime); +make_udf_function!(now::NowFunc, now); +make_udf_function!(to_char::ToCharFunc, to_char); +make_udf_function!(to_date::ToDateFunc, to_date); +make_udf_function!(to_local_time::ToLocalTimeFunc, to_local_time); +make_udf_function!(to_unixtime::ToUnixtimeFunc, to_unixtime); +make_udf_function!(to_timestamp::ToTimestampFunc, to_timestamp); +make_udf_function!(to_timestamp::ToTimestampSecondsFunc, to_timestamp_seconds); +make_udf_function!(to_timestamp::ToTimestampMillisFunc, to_timestamp_millis); +make_udf_function!(to_timestamp::ToTimestampMicrosFunc, to_timestamp_micros); +make_udf_function!(to_timestamp::ToTimestampNanosFunc, to_timestamp_nanos); // we cannot currently use the export_functions macro since it doesn't handle // functions with varargs currently diff --git a/datafusion/functions/src/encoding/mod.rs b/datafusion/functions/src/encoding/mod.rs index 48171370ad58..b0ddbd368a6b 100644 --- a/datafusion/functions/src/encoding/mod.rs +++ b/datafusion/functions/src/encoding/mod.rs @@ -21,8 +21,8 @@ use std::sync::Arc; pub mod inner; // create `encode` and `decode` UDFs -make_udf_function!(inner::EncodeFunc, ENCODE, encode); -make_udf_function!(inner::DecodeFunc, DECODE, decode); +make_udf_function!(inner::EncodeFunc, encode); +make_udf_function!(inner::DecodeFunc, decode); // Export the functions out of this package, both as expr_fn as well as a list of functions pub mod expr_fn { diff --git a/datafusion/functions/src/macros.rs b/datafusion/functions/src/macros.rs index bedec9bb2e6f..82308601490c 100644 --- a/datafusion/functions/src/macros.rs +++ b/datafusion/functions/src/macros.rs @@ -65,24 +65,23 @@ macro_rules! export_functions { }; } -/// Creates a singleton `ScalarUDF` of the `$UDF` function named `$GNAME` and a -/// function named `$NAME` which returns that singleton. +/// Creates a singleton `ScalarUDF` of the `$UDF` function and a function +/// named `$NAME` which returns that singleton. /// /// This is used to ensure creating the list of `ScalarUDF` only happens once. macro_rules! make_udf_function { - ($UDF:ty, $GNAME:ident, $NAME:ident) => { - #[doc = "Return a [`ScalarUDF`](datafusion_expr::ScalarUDF) implementation "] - #[doc = stringify!($UDF)] + ($UDF:ty, $NAME:ident) => { + #[doc = concat!("Return a [`ScalarUDF`](datafusion_expr::ScalarUDF) implementation of ", stringify!($NAME))] pub fn $NAME() -> std::sync::Arc { // Singleton instance of the function - static $GNAME: std::sync::LazyLock< + static INSTANCE: std::sync::LazyLock< std::sync::Arc, > = std::sync::LazyLock::new(|| { std::sync::Arc::new(datafusion_expr::ScalarUDF::new_from_impl( <$UDF>::new(), )) }); - std::sync::Arc::clone(&$GNAME) + std::sync::Arc::clone(&INSTANCE) } }; } @@ -134,13 +133,13 @@ macro_rules! downcast_arg { /// applies a unary floating function to the argument, and returns a value of the same type. /// /// $UDF: the name of the UDF struct that implements `ScalarUDFImpl` -/// $GNAME: a singleton instance of the UDF /// $NAME: the name of the function /// $UNARY_FUNC: the unary function to apply to the argument /// $OUTPUT_ORDERING: the output ordering calculation method of the function +/// $GET_DOC: the function to get the documentation of the UDF macro_rules! make_math_unary_udf { - ($UDF:ident, $GNAME:ident, $NAME:ident, $UNARY_FUNC:ident, $OUTPUT_ORDERING:expr, $EVALUATE_BOUNDS:expr, $GET_DOC:expr) => { - make_udf_function!($NAME::$UDF, $GNAME, $NAME); + ($UDF:ident, $NAME:ident, $UNARY_FUNC:ident, $OUTPUT_ORDERING:expr, $EVALUATE_BOUNDS:expr, $GET_DOC:expr) => { + make_udf_function!($NAME::$UDF, $NAME); mod $NAME { use std::any::Any; @@ -248,13 +247,13 @@ macro_rules! make_math_unary_udf { /// applies a binary floating function to the argument, and returns a value of the same type. /// /// $UDF: the name of the UDF struct that implements `ScalarUDFImpl` -/// $GNAME: a singleton instance of the UDF /// $NAME: the name of the function /// $BINARY_FUNC: the binary function to apply to the argument /// $OUTPUT_ORDERING: the output ordering calculation method of the function +/// $GET_DOC: the function to get the documentation of the UDF macro_rules! make_math_binary_udf { - ($UDF:ident, $GNAME:ident, $NAME:ident, $BINARY_FUNC:ident, $OUTPUT_ORDERING:expr, $GET_DOC:expr) => { - make_udf_function!($NAME::$UDF, $GNAME, $NAME); + ($UDF:ident, $NAME:ident, $BINARY_FUNC:ident, $OUTPUT_ORDERING:expr, $GET_DOC:expr) => { + make_udf_function!($NAME::$UDF, $NAME); mod $NAME { use std::any::Any; diff --git a/datafusion/functions/src/math/mod.rs b/datafusion/functions/src/math/mod.rs index 1452bfdee5a0..4eb337a30110 100644 --- a/datafusion/functions/src/math/mod.rs +++ b/datafusion/functions/src/math/mod.rs @@ -40,10 +40,9 @@ pub mod signum; pub mod trunc; // Create UDFs -make_udf_function!(abs::AbsFunc, ABS, abs); +make_udf_function!(abs::AbsFunc, abs); make_math_unary_udf!( AcosFunc, - ACOS, acos, acos, super::acos_order, @@ -52,7 +51,6 @@ make_math_unary_udf!( ); make_math_unary_udf!( AcoshFunc, - ACOSH, acosh, acosh, super::acosh_order, @@ -61,7 +59,6 @@ make_math_unary_udf!( ); make_math_unary_udf!( AsinFunc, - ASIN, asin, asin, super::asin_order, @@ -70,7 +67,6 @@ make_math_unary_udf!( ); make_math_unary_udf!( AsinhFunc, - ASINH, asinh, asinh, super::asinh_order, @@ -79,7 +75,6 @@ make_math_unary_udf!( ); make_math_unary_udf!( AtanFunc, - ATAN, atan, atan, super::atan_order, @@ -88,7 +83,6 @@ make_math_unary_udf!( ); make_math_unary_udf!( AtanhFunc, - ATANH, atanh, atanh, super::atanh_order, @@ -97,7 +91,6 @@ make_math_unary_udf!( ); make_math_binary_udf!( Atan2, - ATAN2, atan2, atan2, super::atan2_order, @@ -105,7 +98,6 @@ make_math_binary_udf!( ); make_math_unary_udf!( CbrtFunc, - CBRT, cbrt, cbrt, super::cbrt_order, @@ -114,7 +106,6 @@ make_math_unary_udf!( ); make_math_unary_udf!( CeilFunc, - CEIL, ceil, ceil, super::ceil_order, @@ -123,7 +114,6 @@ make_math_unary_udf!( ); make_math_unary_udf!( CosFunc, - COS, cos, cos, super::cos_order, @@ -132,17 +122,15 @@ make_math_unary_udf!( ); make_math_unary_udf!( CoshFunc, - COSH, cosh, cosh, super::cosh_order, super::bounds::cosh_bounds, super::get_cosh_doc ); -make_udf_function!(cot::CotFunc, COT, cot); +make_udf_function!(cot::CotFunc, cot); make_math_unary_udf!( DegreesFunc, - DEGREES, degrees, to_degrees, super::degrees_order, @@ -151,31 +139,28 @@ make_math_unary_udf!( ); make_math_unary_udf!( ExpFunc, - EXP, exp, exp, super::exp_order, super::bounds::exp_bounds, super::get_exp_doc ); -make_udf_function!(factorial::FactorialFunc, FACTORIAL, factorial); +make_udf_function!(factorial::FactorialFunc, factorial); make_math_unary_udf!( FloorFunc, - FLOOR, floor, floor, super::floor_order, super::bounds::unbounded_bounds, super::get_floor_doc ); -make_udf_function!(log::LogFunc, LOG, log); -make_udf_function!(gcd::GcdFunc, GCD, gcd); -make_udf_function!(nans::IsNanFunc, ISNAN, isnan); -make_udf_function!(iszero::IsZeroFunc, ISZERO, iszero); -make_udf_function!(lcm::LcmFunc, LCM, lcm); +make_udf_function!(log::LogFunc, log); +make_udf_function!(gcd::GcdFunc, gcd); +make_udf_function!(nans::IsNanFunc, isnan); +make_udf_function!(iszero::IsZeroFunc, iszero); +make_udf_function!(lcm::LcmFunc, lcm); make_math_unary_udf!( LnFunc, - LN, ln, ln, super::ln_order, @@ -184,7 +169,6 @@ make_math_unary_udf!( ); make_math_unary_udf!( Log2Func, - LOG2, log2, log2, super::log2_order, @@ -193,31 +177,28 @@ make_math_unary_udf!( ); make_math_unary_udf!( Log10Func, - LOG10, log10, log10, super::log10_order, super::bounds::unbounded_bounds, super::get_log10_doc ); -make_udf_function!(nanvl::NanvlFunc, NANVL, nanvl); -make_udf_function!(pi::PiFunc, PI, pi); -make_udf_function!(power::PowerFunc, POWER, power); +make_udf_function!(nanvl::NanvlFunc, nanvl); +make_udf_function!(pi::PiFunc, pi); +make_udf_function!(power::PowerFunc, power); make_math_unary_udf!( RadiansFunc, - RADIANS, radians, to_radians, super::radians_order, super::bounds::radians_bounds, super::get_radians_doc ); -make_udf_function!(random::RandomFunc, RANDOM, random); -make_udf_function!(round::RoundFunc, ROUND, round); -make_udf_function!(signum::SignumFunc, SIGNUM, signum); +make_udf_function!(random::RandomFunc, random); +make_udf_function!(round::RoundFunc, round); +make_udf_function!(signum::SignumFunc, signum); make_math_unary_udf!( SinFunc, - SIN, sin, sin, super::sin_order, @@ -226,7 +207,6 @@ make_math_unary_udf!( ); make_math_unary_udf!( SinhFunc, - SINH, sinh, sinh, super::sinh_order, @@ -235,7 +215,6 @@ make_math_unary_udf!( ); make_math_unary_udf!( SqrtFunc, - SQRT, sqrt, sqrt, super::sqrt_order, @@ -244,7 +223,6 @@ make_math_unary_udf!( ); make_math_unary_udf!( TanFunc, - TAN, tan, tan, super::tan_order, @@ -253,14 +231,13 @@ make_math_unary_udf!( ); make_math_unary_udf!( TanhFunc, - TANH, tanh, tanh, super::tanh_order, super::bounds::tanh_bounds, super::get_tanh_doc ); -make_udf_function!(trunc::TruncFunc, TRUNC, trunc); +make_udf_function!(trunc::TruncFunc, trunc); pub mod expr_fn { export_functions!( diff --git a/datafusion/functions/src/regex/mod.rs b/datafusion/functions/src/regex/mod.rs index 803f51e915a9..13fbc049af58 100644 --- a/datafusion/functions/src/regex/mod.rs +++ b/datafusion/functions/src/regex/mod.rs @@ -25,14 +25,10 @@ pub mod regexpmatch; pub mod regexpreplace; // create UDFs -make_udf_function!(regexpcount::RegexpCountFunc, REGEXP_COUNT, regexp_count); -make_udf_function!(regexpmatch::RegexpMatchFunc, REGEXP_MATCH, regexp_match); -make_udf_function!(regexplike::RegexpLikeFunc, REGEXP_LIKE, regexp_like); -make_udf_function!( - regexpreplace::RegexpReplaceFunc, - REGEXP_REPLACE, - regexp_replace -); +make_udf_function!(regexpcount::RegexpCountFunc, regexp_count); +make_udf_function!(regexpmatch::RegexpMatchFunc, regexp_match); +make_udf_function!(regexplike::RegexpLikeFunc, regexp_like); +make_udf_function!(regexpreplace::RegexpReplaceFunc, regexp_replace); pub mod expr_fn { use datafusion_expr::Expr; diff --git a/datafusion/functions/src/string/mod.rs b/datafusion/functions/src/string/mod.rs index 622802f0142b..f156f070d960 100644 --- a/datafusion/functions/src/string/mod.rs +++ b/datafusion/functions/src/string/mod.rs @@ -45,28 +45,28 @@ pub mod to_hex; pub mod upper; pub mod uuid; // create UDFs -make_udf_function!(ascii::AsciiFunc, ASCII, ascii); -make_udf_function!(bit_length::BitLengthFunc, BIT_LENGTH, bit_length); -make_udf_function!(btrim::BTrimFunc, BTRIM, btrim); -make_udf_function!(chr::ChrFunc, CHR, chr); -make_udf_function!(concat::ConcatFunc, CONCAT, concat); -make_udf_function!(concat_ws::ConcatWsFunc, CONCAT_WS, concat_ws); -make_udf_function!(ends_with::EndsWithFunc, ENDS_WITH, ends_with); -make_udf_function!(initcap::InitcapFunc, INITCAP, initcap); -make_udf_function!(levenshtein::LevenshteinFunc, LEVENSHTEIN, levenshtein); -make_udf_function!(ltrim::LtrimFunc, LTRIM, ltrim); -make_udf_function!(lower::LowerFunc, LOWER, lower); -make_udf_function!(octet_length::OctetLengthFunc, OCTET_LENGTH, octet_length); -make_udf_function!(overlay::OverlayFunc, OVERLAY, overlay); -make_udf_function!(repeat::RepeatFunc, REPEAT, repeat); -make_udf_function!(replace::ReplaceFunc, REPLACE, replace); -make_udf_function!(rtrim::RtrimFunc, RTRIM, rtrim); -make_udf_function!(starts_with::StartsWithFunc, STARTS_WITH, starts_with); -make_udf_function!(split_part::SplitPartFunc, SPLIT_PART, split_part); -make_udf_function!(to_hex::ToHexFunc, TO_HEX, to_hex); -make_udf_function!(upper::UpperFunc, UPPER, upper); -make_udf_function!(uuid::UuidFunc, UUID, uuid); -make_udf_function!(contains::ContainsFunc, CONTAINS, contains); +make_udf_function!(ascii::AsciiFunc, ascii); +make_udf_function!(bit_length::BitLengthFunc, bit_length); +make_udf_function!(btrim::BTrimFunc, btrim); +make_udf_function!(chr::ChrFunc, chr); +make_udf_function!(concat::ConcatFunc, concat); +make_udf_function!(concat_ws::ConcatWsFunc, concat_ws); +make_udf_function!(ends_with::EndsWithFunc, ends_with); +make_udf_function!(initcap::InitcapFunc, initcap); +make_udf_function!(levenshtein::LevenshteinFunc, levenshtein); +make_udf_function!(ltrim::LtrimFunc, ltrim); +make_udf_function!(lower::LowerFunc, lower); +make_udf_function!(octet_length::OctetLengthFunc, octet_length); +make_udf_function!(overlay::OverlayFunc, overlay); +make_udf_function!(repeat::RepeatFunc, repeat); +make_udf_function!(replace::ReplaceFunc, replace); +make_udf_function!(rtrim::RtrimFunc, rtrim); +make_udf_function!(starts_with::StartsWithFunc, starts_with); +make_udf_function!(split_part::SplitPartFunc, split_part); +make_udf_function!(to_hex::ToHexFunc, to_hex); +make_udf_function!(upper::UpperFunc, upper); +make_udf_function!(uuid::UuidFunc, uuid); +make_udf_function!(contains::ContainsFunc, contains); pub mod expr_fn { use datafusion_expr::Expr; diff --git a/datafusion/functions/src/unicode/mod.rs b/datafusion/functions/src/unicode/mod.rs index 40915bc9efde..f31ece9196d8 100644 --- a/datafusion/functions/src/unicode/mod.rs +++ b/datafusion/functions/src/unicode/mod.rs @@ -34,22 +34,18 @@ pub mod substrindex; pub mod translate; // create UDFs -make_udf_function!( - character_length::CharacterLengthFunc, - CHARACTER_LENGTH, - character_length -); -make_udf_function!(find_in_set::FindInSetFunc, FIND_IN_SET, find_in_set); -make_udf_function!(left::LeftFunc, LEFT, left); -make_udf_function!(lpad::LPadFunc, LPAD, lpad); -make_udf_function!(right::RightFunc, RIGHT, right); -make_udf_function!(reverse::ReverseFunc, REVERSE, reverse); -make_udf_function!(rpad::RPadFunc, RPAD, rpad); -make_udf_function!(strpos::StrposFunc, STRPOS, strpos); -make_udf_function!(substr::SubstrFunc, SUBSTR, substr); -make_udf_function!(substr::SubstrFunc, SUBSTRING, substring); -make_udf_function!(substrindex::SubstrIndexFunc, SUBSTR_INDEX, substr_index); -make_udf_function!(translate::TranslateFunc, TRANSLATE, translate); +make_udf_function!(character_length::CharacterLengthFunc, character_length); +make_udf_function!(find_in_set::FindInSetFunc, find_in_set); +make_udf_function!(left::LeftFunc, left); +make_udf_function!(lpad::LPadFunc, lpad); +make_udf_function!(right::RightFunc, right); +make_udf_function!(reverse::ReverseFunc, reverse); +make_udf_function!(rpad::RPadFunc, rpad); +make_udf_function!(strpos::StrposFunc, strpos); +make_udf_function!(substr::SubstrFunc, substr); +make_udf_function!(substr::SubstrFunc, substring); +make_udf_function!(substrindex::SubstrIndexFunc, substr_index); +make_udf_function!(translate::TranslateFunc, translate); pub mod expr_fn { use datafusion_expr::Expr; From 6196ff2d3f221466c7f86c07c884b03db1568866 Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Wed, 11 Dec 2024 20:02:10 +0800 Subject: [PATCH 09/14] refactor: replace `Vec` with `IndexMap` for expression mappings in `ProjectionMapping` and `EquivalenceGroup` (#13675) * refactor: replace Vec with IndexMap for expression mappings in ProjectionMapping and EquivalenceGroup * chore * chore: Fix CI * chore: comment * chore: simplify --- .../physical-expr/src/equivalence/class.rs | 34 +++++++------------ 1 file changed, 13 insertions(+), 21 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/class.rs b/datafusion/physical-expr/src/equivalence/class.rs index d06a495d970a..cc26d12fb029 100644 --- a/datafusion/physical-expr/src/equivalence/class.rs +++ b/datafusion/physical-expr/src/equivalence/class.rs @@ -17,8 +17,8 @@ use super::{add_offset_to_expr, collapse_lex_req, ProjectionMapping}; use crate::{ - expressions::Column, physical_exprs_contains, LexOrdering, LexRequirement, - PhysicalExpr, PhysicalExprRef, PhysicalSortExpr, PhysicalSortRequirement, + expressions::Column, LexOrdering, LexRequirement, PhysicalExpr, PhysicalExprRef, + PhysicalSortExpr, PhysicalSortRequirement, }; use std::fmt::Display; use std::sync::Arc; @@ -27,7 +27,7 @@ use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::JoinType; use datafusion_physical_expr_common::physical_expr::format_physical_expr_list; -use indexmap::IndexSet; +use indexmap::{IndexMap, IndexSet}; /// A structure representing a expression known to be constant in a physical execution plan. /// @@ -546,28 +546,20 @@ impl EquivalenceGroup { .collect::>(); (new_class.len() > 1).then_some(EquivalenceClass::new(new_class)) }); - // TODO: Convert the algorithm below to a version that uses `HashMap`. - // once `Arc` can be stored in `HashMap`. - // See issue: https://github.com/apache/datafusion/issues/8027 - let mut new_classes = vec![]; - for (source, target) in mapping.iter() { - if new_classes.is_empty() { - new_classes.push((source, vec![Arc::clone(target)])); - } - if let Some((_, values)) = - new_classes.iter_mut().find(|(key, _)| *key == source) - { - if !physical_exprs_contains(values, target) { - values.push(Arc::clone(target)); - } - } - } + // the key is the source expression and the value is the EquivalenceClass that contains the target expression of the source expression. + let mut new_classes: IndexMap, EquivalenceClass> = + IndexMap::new(); + mapping.iter().for_each(|(source, target)| { + new_classes + .entry(Arc::clone(source)) + .or_insert_with(EquivalenceClass::new_empty) + .push(Arc::clone(target)); + }); // Only add equivalence classes with at least two members as singleton // equivalence classes are meaningless. let new_classes = new_classes .into_iter() - .filter_map(|(_, values)| (values.len() > 1).then_some(values)) - .map(EquivalenceClass::new); + .filter_map(|(_, cls)| (cls.len() > 1).then_some(cls)); let classes = projected_classes.chain(new_classes).collect(); Self::new(classes) From 93b3d9cbfa1ba8ed237ca59f686c32b94ee4bc0a Mon Sep 17 00:00:00 2001 From: Eason <30045503+Eason0729@users.noreply.github.com> Date: Wed, 11 Dec 2024 20:21:17 +0800 Subject: [PATCH 10/14] Handle alias when parsing sql(parse_sql_expr) (#12939) * fix: Fix parse_sql_expr not handling alias * cargo fmt * fix parse_sql_expr example(remove alias) * add testing * add SUM udaf to TestContextProvider and modify test_sql_to_expr_with_alias for function * revert change on example `parse_sql_expr` --- .../examples/parse_sql_expr.rs | 10 ++-- .../core/src/execution/session_state.rs | 21 +++++-- datafusion/sql/src/expr/mod.rs | 60 +++++++++++++++++-- datafusion/sql/src/parser.rs | 9 +-- 4 files changed, 82 insertions(+), 18 deletions(-) diff --git a/datafusion-examples/examples/parse_sql_expr.rs b/datafusion-examples/examples/parse_sql_expr.rs index e23e5accae39..d8f0778e19e3 100644 --- a/datafusion-examples/examples/parse_sql_expr.rs +++ b/datafusion-examples/examples/parse_sql_expr.rs @@ -121,11 +121,11 @@ async fn query_parquet_demo() -> Result<()> { assert_batches_eq!( &[ - "+------------+----------------------+", - "| double_col | sum(?table?.int_col) |", - "+------------+----------------------+", - "| 10.1 | 4 |", - "+------------+----------------------+", + "+------------+-------------+", + "| double_col | sum_int_col |", + "+------------+-------------+", + "| 10.1 | 4 |", + "+------------+-------------+", ], &result ); diff --git a/datafusion/core/src/execution/session_state.rs b/datafusion/core/src/execution/session_state.rs index 4ccad5ffd323..cef5d4c1ee2a 100644 --- a/datafusion/core/src/execution/session_state.rs +++ b/datafusion/core/src/execution/session_state.rs @@ -68,7 +68,7 @@ use datafusion_sql::planner::{ContextProvider, ParserOptions, PlannerContext, Sq use itertools::Itertools; use log::{debug, info}; use object_store::ObjectStore; -use sqlparser::ast::Expr as SQLExpr; +use sqlparser::ast::{Expr as SQLExpr, ExprWithAlias as SQLExprWithAlias}; use sqlparser::dialect::dialect_from_str; use std::any::Any; use std::collections::hash_map::Entry; @@ -500,11 +500,22 @@ impl SessionState { sql: &str, dialect: &str, ) -> datafusion_common::Result { + self.sql_to_expr_with_alias(sql, dialect).map(|x| x.expr) + } + + /// parse a sql string into a sqlparser-rs AST [`SQLExprWithAlias`]. + /// + /// See [`Self::create_logical_expr`] for parsing sql to [`Expr`]. + pub fn sql_to_expr_with_alias( + &self, + sql: &str, + dialect: &str, + ) -> datafusion_common::Result { let dialect = dialect_from_str(dialect).ok_or_else(|| { plan_datafusion_err!( "Unsupported SQL dialect: {dialect}. Available dialects: \ - Generic, MySQL, PostgreSQL, Hive, SQLite, Snowflake, Redshift, \ - MsSQL, ClickHouse, BigQuery, Ansi." + Generic, MySQL, PostgreSQL, Hive, SQLite, Snowflake, Redshift, \ + MsSQL, ClickHouse, BigQuery, Ansi." ) })?; @@ -603,7 +614,7 @@ impl SessionState { ) -> datafusion_common::Result { let dialect = self.config.options().sql_parser.dialect.as_str(); - let sql_expr = self.sql_to_expr(sql, dialect)?; + let sql_expr = self.sql_to_expr_with_alias(sql, dialect)?; let provider = SessionContextProvider { state: self, @@ -611,7 +622,7 @@ impl SessionState { }; let query = SqlToRel::new_with_options(&provider, self.get_parser_options()); - query.sql_to_expr(sql_expr, df_schema, &mut PlannerContext::new()) + query.sql_to_expr_with_alias(sql_expr, df_schema, &mut PlannerContext::new()) } /// Returns the [`Analyzer`] for this session diff --git a/datafusion/sql/src/expr/mod.rs b/datafusion/sql/src/expr/mod.rs index 57ac96951f1f..e8ec8d7b7d1c 100644 --- a/datafusion/sql/src/expr/mod.rs +++ b/datafusion/sql/src/expr/mod.rs @@ -23,7 +23,8 @@ use datafusion_expr::planner::{ use recursive::recursive; use sqlparser::ast::{ BinaryOperator, CastFormat, CastKind, DataType as SQLDataType, DictionaryField, - Expr as SQLExpr, MapEntry, StructField, Subscript, TrimWhereField, Value, + Expr as SQLExpr, ExprWithAlias as SQLExprWithAlias, MapEntry, StructField, Subscript, + TrimWhereField, Value, }; use datafusion_common::{ @@ -50,6 +51,19 @@ mod unary_op; mod value; impl SqlToRel<'_, S> { + pub(crate) fn sql_expr_to_logical_expr_with_alias( + &self, + sql: SQLExprWithAlias, + schema: &DFSchema, + planner_context: &mut PlannerContext, + ) -> Result { + let mut expr = + self.sql_expr_to_logical_expr(sql.expr, schema, planner_context)?; + if let Some(alias) = sql.alias { + expr = expr.alias(alias.value); + } + Ok(expr) + } pub(crate) fn sql_expr_to_logical_expr( &self, sql: SQLExpr, @@ -131,6 +145,20 @@ impl SqlToRel<'_, S> { ))) } + pub fn sql_to_expr_with_alias( + &self, + sql: SQLExprWithAlias, + schema: &DFSchema, + planner_context: &mut PlannerContext, + ) -> Result { + let mut expr = + self.sql_expr_to_logical_expr_with_alias(sql, schema, planner_context)?; + expr = self.rewrite_partial_qualifier(expr, schema); + self.validate_schema_satisfies_exprs(schema, &[expr.clone()])?; + let (expr, _) = expr.infer_placeholder_types(schema)?; + Ok(expr) + } + /// Generate a relational expression from a SQL expression pub fn sql_to_expr( &self, @@ -1091,8 +1119,11 @@ mod tests { None } - fn get_aggregate_meta(&self, _name: &str) -> Option> { - None + fn get_aggregate_meta(&self, name: &str) -> Option> { + match name { + "sum" => Some(datafusion_functions_aggregate::sum::sum_udaf()), + _ => None, + } } fn get_variable_type(&self, _variable_names: &[String]) -> Option { @@ -1112,7 +1143,7 @@ mod tests { } fn udaf_names(&self) -> Vec { - Vec::new() + vec!["sum".to_string()] } fn udwf_names(&self) -> Vec { @@ -1167,4 +1198,25 @@ mod tests { test_stack_overflow!(2048); test_stack_overflow!(4096); test_stack_overflow!(8192); + #[test] + fn test_sql_to_expr_with_alias() { + let schema = DFSchema::empty(); + let mut planner_context = PlannerContext::default(); + + let expr_str = "SUM(int_col) as sum_int_col"; + + let dialect = GenericDialect {}; + let mut parser = Parser::new(&dialect).try_with_sql(expr_str).unwrap(); + // from sqlparser + let sql_expr = parser.parse_expr_with_alias().unwrap(); + + let context_provider = TestContextProvider::new(); + let sql_to_rel = SqlToRel::new(&context_provider); + + let expr = sql_to_rel + .sql_expr_to_logical_expr_with_alias(sql_expr, &schema, &mut planner_context) + .unwrap(); + + assert!(matches!(expr, Expr::Alias(_))); + } } diff --git a/datafusion/sql/src/parser.rs b/datafusion/sql/src/parser.rs index bd1ed3145ef5..efec6020641c 100644 --- a/datafusion/sql/src/parser.rs +++ b/datafusion/sql/src/parser.rs @@ -20,9 +20,10 @@ use std::collections::VecDeque; use std::fmt; +use sqlparser::ast::ExprWithAlias; use sqlparser::{ ast::{ - ColumnDef, ColumnOptionDef, Expr, ObjectName, OrderByExpr, Query, + ColumnDef, ColumnOptionDef, ObjectName, OrderByExpr, Query, Statement as SQLStatement, TableConstraint, Value, }, dialect::{keywords::Keyword, Dialect, GenericDialect}, @@ -328,7 +329,7 @@ impl<'a> DFParser<'a> { pub fn parse_sql_into_expr_with_dialect( sql: &str, dialect: &dyn Dialect, - ) -> Result { + ) -> Result { let mut parser = DFParser::new_with_dialect(sql, dialect)?; parser.parse_expr() } @@ -377,7 +378,7 @@ impl<'a> DFParser<'a> { } } - pub fn parse_expr(&mut self) -> Result { + pub fn parse_expr(&mut self) -> Result { if let Token::Word(w) = self.parser.peek_token().token { match w.keyword { Keyword::CREATE | Keyword::COPY | Keyword::EXPLAIN => { @@ -387,7 +388,7 @@ impl<'a> DFParser<'a> { } } - self.parser.parse_expr() + self.parser.parse_expr_with_alias() } /// Parse a SQL `COPY TO` statement From a52eb47cf8ed0f74936cf5c2ae179e006f87826f Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 11 Dec 2024 07:27:46 -0500 Subject: [PATCH 11/14] Improve documentation for TableProvider (#13724) --- datafusion/catalog/src/table.rs | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/datafusion/catalog/src/table.rs b/datafusion/catalog/src/table.rs index b6752191d9a7..3c8960495588 100644 --- a/datafusion/catalog/src/table.rs +++ b/datafusion/catalog/src/table.rs @@ -33,7 +33,19 @@ use datafusion_expr::{ }; use datafusion_physical_plan::ExecutionPlan; -/// Source table +/// A named table which can be queried. +/// +/// Please see [`CatalogProvider`] for details of implementing a custom catalog. +/// +/// [`TableProvider`] represents a source of data which can provide data as +/// Apache Arrow `RecordBatch`es. Implementations of this trait provide +/// important information for planning such as: +/// +/// 1. [`Self::schema`]: The schema (columns and their types) of the table +/// 2. [`Self::supports_filters_pushdown`]: Should filters be pushed into this scan +/// 2. [`Self::scan`]: An [`ExecutionPlan`] that can read data +/// +/// [`CatalogProvider`]: super::CatalogProvider #[async_trait] pub trait TableProvider: Debug + Sync + Send { /// Returns the table provider as [`Any`](std::any::Any) so that it can be From 95405a965555c557ca99afb9052b5ac12a9fd574 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Wed, 11 Dec 2024 14:29:23 +0100 Subject: [PATCH 12/14] Reveal implementing type and return type in simple UDF implementations (#13730) Debug trait is useful for understanding what something is and how it's configured, especially if the implementation is behind dyn trait. --- datafusion/expr/src/expr_fn.rs | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/datafusion/expr/src/expr_fn.rs b/datafusion/expr/src/expr_fn.rs index 681eb3c0afd5..a44dd24039dc 100644 --- a/datafusion/expr/src/expr_fn.rs +++ b/datafusion/expr/src/expr_fn.rs @@ -416,9 +416,10 @@ pub struct SimpleScalarUDF { impl Debug for SimpleScalarUDF { fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { - f.debug_struct("ScalarUDF") + f.debug_struct("SimpleScalarUDF") .field("name", &self.name) .field("signature", &self.signature) + .field("return_type", &self.return_type) .field("fun", &"") .finish() } @@ -524,9 +525,10 @@ pub struct SimpleAggregateUDF { impl Debug for SimpleAggregateUDF { fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { - f.debug_struct("AggregateUDF") + f.debug_struct("SimpleAggregateUDF") .field("name", &self.name) .field("signature", &self.signature) + .field("return_type", &self.return_type) .field("fun", &"") .finish() } From 28e4c64dc738227cd6a4cdf7db48685338582c04 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 11 Dec 2024 09:49:23 -0500 Subject: [PATCH 13/14] minor: Extract tests for `EXTRACT` AND `date_part` to their own file (#13731) --- datafusion/sqllogictest/test_files/expr.slt | 861 ----------------- .../test_files/expr/date_part.slt | 878 ++++++++++++++++++ 2 files changed, 878 insertions(+), 861 deletions(-) create mode 100644 datafusion/sqllogictest/test_files/expr/date_part.slt diff --git a/datafusion/sqllogictest/test_files/expr.slt b/datafusion/sqllogictest/test_files/expr.slt index 9b8dfc2186be..2306eda77d35 100644 --- a/datafusion/sqllogictest/test_files/expr.slt +++ b/datafusion/sqllogictest/test_files/expr.slt @@ -832,867 +832,6 @@ SELECT ---- 0 NULL 0 NULL -# test_extract_date_part - -query error -SELECT EXTRACT("'''year'''" FROM timestamp '2020-09-08T12:00:00+00:00') - -query error -SELECT EXTRACT("'year'" FROM timestamp '2020-09-08T12:00:00+00:00') - -query I -SELECT date_part('YEAR', CAST('2000-01-01' AS DATE)) ----- -2000 - -query I -SELECT EXTRACT(year FROM timestamp '2020-09-08T12:00:00+00:00') ----- -2020 - -query I -SELECT EXTRACT("year" FROM timestamp '2020-09-08T12:00:00+00:00') ----- -2020 - -query I -SELECT EXTRACT('year' FROM timestamp '2020-09-08T12:00:00+00:00') ----- -2020 - -query I -SELECT date_part('QUARTER', CAST('2000-01-01' AS DATE)) ----- -1 - -query I -SELECT EXTRACT(quarter FROM to_timestamp('2020-09-08T12:00:00+00:00')) ----- -3 - -query I -SELECT EXTRACT("quarter" FROM to_timestamp('2020-09-08T12:00:00+00:00')) ----- -3 - -query I -SELECT EXTRACT('quarter' FROM to_timestamp('2020-09-08T12:00:00+00:00')) ----- -3 - -query I -SELECT date_part('MONTH', CAST('2000-01-01' AS DATE)) ----- -1 - -query I -SELECT EXTRACT(month FROM to_timestamp('2020-09-08T12:00:00+00:00')) ----- -9 - -query I -SELECT EXTRACT("month" FROM to_timestamp('2020-09-08T12:00:00+00:00')) ----- -9 - -query I -SELECT EXTRACT('month' FROM to_timestamp('2020-09-08T12:00:00+00:00')) ----- -9 - -query I -SELECT date_part('WEEK', CAST('2003-01-01' AS DATE)) ----- -1 - -query I -SELECT EXTRACT(WEEK FROM to_timestamp('2020-09-08T12:00:00+00:00')) ----- -37 - -query I -SELECT EXTRACT("WEEK" FROM to_timestamp('2020-09-08T12:00:00+00:00')) ----- -37 - -query I -SELECT EXTRACT('WEEK' FROM to_timestamp('2020-09-08T12:00:00+00:00')) ----- -37 - -query I -SELECT date_part('DAY', CAST('2000-01-01' AS DATE)) ----- -1 - -query I -SELECT EXTRACT(day FROM to_timestamp('2020-09-08T12:00:00+00:00')) ----- -8 - -query I -SELECT EXTRACT("day" FROM to_timestamp('2020-09-08T12:00:00+00:00')) ----- -8 - -query I -SELECT EXTRACT('day' FROM to_timestamp('2020-09-08T12:00:00+00:00')) ----- -8 - -query I -SELECT date_part('DOY', CAST('2000-01-01' AS DATE)) ----- -1 - -query I -SELECT EXTRACT(doy FROM to_timestamp('2020-09-08T12:00:00+00:00')) ----- -252 - -query I -SELECT EXTRACT("doy" FROM to_timestamp('2020-09-08T12:00:00+00:00')) ----- -252 - -query I -SELECT EXTRACT('doy' FROM to_timestamp('2020-09-08T12:00:00+00:00')) ----- -252 - -query I -SELECT date_part('DOW', CAST('2000-01-01' AS DATE)) ----- -6 - -query I -SELECT EXTRACT(dow FROM to_timestamp('2020-09-08T12:00:00+00:00')) ----- -2 - -query I -SELECT EXTRACT("dow" FROM to_timestamp('2020-09-08T12:00:00+00:00')) ----- -2 - -query I -SELECT EXTRACT('dow' FROM to_timestamp('2020-09-08T12:00:00+00:00')) ----- -2 - -query I -SELECT date_part('HOUR', CAST('2000-01-01' AS DATE)) ----- -0 - -query I -SELECT EXTRACT(hour FROM to_timestamp('2020-09-08T12:03:03+00:00')) ----- -12 - -query I -SELECT EXTRACT("hour" FROM to_timestamp('2020-09-08T12:03:03+00:00')) ----- -12 - -query I -SELECT EXTRACT('hour' FROM to_timestamp('2020-09-08T12:03:03+00:00')) ----- -12 - -query I -SELECT EXTRACT(minute FROM to_timestamp('2020-09-08T12:12:00+00:00')) ----- -12 - -query I -SELECT EXTRACT("minute" FROM to_timestamp('2020-09-08T12:12:00+00:00')) ----- -12 - -query I -SELECT EXTRACT('minute' FROM to_timestamp('2020-09-08T12:12:00+00:00')) ----- -12 - -query I -SELECT date_part('minute', to_timestamp('2020-09-08T12:12:00+00:00')) ----- -12 - -# make sure the return type is integer -query T -SELECT arrow_typeof(date_part('minute', to_timestamp('2020-09-08T12:12:00+00:00'))) ----- -Int32 - -query I -SELECT EXTRACT(second FROM timestamp '2020-09-08T12:00:12.12345678+00:00') ----- -12 - -query I -SELECT EXTRACT(millisecond FROM timestamp '2020-09-08T12:00:12.12345678+00:00') ----- -12123 - -query I -SELECT EXTRACT(microsecond FROM timestamp '2020-09-08T12:00:12.12345678+00:00') ----- -12123456 - -query error DataFusion error: Internal error: unit Nanosecond not supported -SELECT EXTRACT(nanosecond FROM timestamp '2020-09-08T12:00:12.12345678+00:00') - -query I -SELECT EXTRACT("second" FROM timestamp '2020-09-08T12:00:12.12345678+00:00') ----- -12 - -query I -SELECT EXTRACT("millisecond" FROM timestamp '2020-09-08T12:00:12.12345678+00:00') ----- -12123 - -query I -SELECT EXTRACT("microsecond" FROM timestamp '2020-09-08T12:00:12.12345678+00:00') ----- -12123456 - -query error DataFusion error: Internal error: unit Nanosecond not supported -SELECT EXTRACT("nanosecond" FROM timestamp '2020-09-08T12:00:12.12345678+00:00') - -query I -SELECT EXTRACT('second' FROM timestamp '2020-09-08T12:00:12.12345678+00:00') ----- -12 - -query I -SELECT EXTRACT('millisecond' FROM timestamp '2020-09-08T12:00:12.12345678+00:00') ----- -12123 - -query I -SELECT EXTRACT('microsecond' FROM timestamp '2020-09-08T12:00:12.12345678+00:00') ----- -12123456 - -query error DataFusion error: Internal error: unit Nanosecond not supported -SELECT EXTRACT('nanosecond' FROM timestamp '2020-09-08T12:00:12.12345678+00:00') - - -# Keep precision when coercing Utf8 to Timestamp -query I -SELECT date_part('second', timestamp '2020-09-08T12:00:12.12345678+00:00') ----- -12 - -query I -SELECT date_part('millisecond', timestamp '2020-09-08T12:00:12.12345678+00:00') ----- -12123 - -query I -SELECT date_part('microsecond', timestamp '2020-09-08T12:00:12.12345678+00:00') ----- -12123456 - -query error DataFusion error: Internal error: unit Nanosecond not supported -SELECT date_part('nanosecond', timestamp '2020-09-08T12:00:12.12345678+00:00') - - -query I -SELECT date_part('second', '2020-09-08T12:00:12.12345678+00:00') ----- -12 - -query I -SELECT date_part('millisecond', '2020-09-08T12:00:12.12345678+00:00') ----- -12123 - -query I -SELECT date_part('microsecond', '2020-09-08T12:00:12.12345678+00:00') ----- -12123456 - -query error DataFusion error: Internal error: unit Nanosecond not supported -SELECT date_part('nanosecond', '2020-09-08T12:00:12.12345678+00:00') - -# test_date_part_time - -## time32 seconds -query I -SELECT date_part('hour', arrow_cast('23:32:50'::time, 'Time32(Second)')) ----- -23 - -query I -SELECT extract(hour from arrow_cast('23:32:50'::time, 'Time32(Second)')) ----- -23 - -query I -SELECT date_part('minute', arrow_cast('23:32:50'::time, 'Time32(Second)')) ----- -32 - -query I -SELECT extract(minute from arrow_cast('23:32:50'::time, 'Time32(Second)')) ----- -32 - -query I -SELECT date_part('second', arrow_cast('23:32:50'::time, 'Time32(Second)')) ----- -50 - -query I -SELECT extract(second from arrow_cast('23:32:50'::time, 'Time32(Second)')) ----- -50 - -query I -SELECT date_part('millisecond', arrow_cast('23:32:50'::time, 'Time32(Second)')) ----- -50000 - -query I -SELECT extract(millisecond from arrow_cast('23:32:50'::time, 'Time32(Second)')) ----- -50000 - -query I -SELECT date_part('microsecond', arrow_cast('23:32:50'::time, 'Time32(Second)')) ----- -50000000 - -query I -SELECT extract(microsecond from arrow_cast('23:32:50'::time, 'Time32(Second)')) ----- -50000000 - -query error DataFusion error: Internal error: unit Nanosecond not supported -SELECT extract(nanosecond from arrow_cast('23:32:50'::time, 'Time32(Second)')) - -query R -SELECT date_part('epoch', arrow_cast('23:32:50'::time, 'Time32(Second)')) ----- -84770 - -query R -SELECT extract(epoch from arrow_cast('23:32:50'::time, 'Time32(Second)')) ----- -84770 - -## time32 milliseconds -query I -SELECT date_part('hour', arrow_cast('23:32:50.123'::time, 'Time32(Millisecond)')) ----- -23 - -query I -SELECT extract(hour from arrow_cast('23:32:50.123'::time, 'Time32(Millisecond)')) ----- -23 - -query I -SELECT date_part('minute', arrow_cast('23:32:50.123'::time, 'Time32(Millisecond)')) ----- -32 - -query I -SELECT extract(minute from arrow_cast('23:32:50.123'::time, 'Time32(Millisecond)')) ----- -32 - -query I -SELECT date_part('second', arrow_cast('23:32:50.123'::time, 'Time32(Millisecond)')) ----- -50 - -query I -SELECT extract(second from arrow_cast('23:32:50.123'::time, 'Time32(Millisecond)')) ----- -50 - -query I -SELECT date_part('millisecond', arrow_cast('23:32:50.123'::time, 'Time32(Millisecond)')) ----- -50123 - -query I -SELECT extract(millisecond from arrow_cast('23:32:50.123'::time, 'Time32(Millisecond)')) ----- -50123 - -query I -SELECT date_part('microsecond', arrow_cast('23:32:50.123'::time, 'Time32(Millisecond)')) ----- -50123000 - -query I -SELECT extract(microsecond from arrow_cast('23:32:50.123'::time, 'Time32(Millisecond)')) ----- -50123000 - -query error DataFusion error: Internal error: unit Nanosecond not supported -SELECT extract(nanosecond from arrow_cast('23:32:50.123'::time, 'Time32(Millisecond)')) - -query R -SELECT date_part('epoch', arrow_cast('23:32:50.123'::time, 'Time32(Millisecond)')) ----- -84770.123 - -query R -SELECT extract(epoch from arrow_cast('23:32:50.123'::time, 'Time32(Millisecond)')) ----- -84770.123 - -## time64 microseconds -query I -SELECT date_part('hour', arrow_cast('23:32:50.123456'::time, 'Time64(Microsecond)')) ----- -23 - -query I -SELECT extract(hour from arrow_cast('23:32:50.123456'::time, 'Time64(Microsecond)')) ----- -23 - -query I -SELECT date_part('minute', arrow_cast('23:32:50.123456'::time, 'Time64(Microsecond)')) ----- -32 - -query I -SELECT extract(minute from arrow_cast('23:32:50.123456'::time, 'Time64(Microsecond)')) ----- -32 - -query I -SELECT date_part('second', arrow_cast('23:32:50.123456'::time, 'Time64(Microsecond)')) ----- -50 - -query I -SELECT extract(second from arrow_cast('23:32:50.123456'::time, 'Time64(Microsecond)')) ----- -50 - -query I -SELECT date_part('millisecond', arrow_cast('23:32:50.123456'::time, 'Time64(Microsecond)')) ----- -50123 - -query I -SELECT extract(millisecond from arrow_cast('23:32:50.123456'::time, 'Time64(Microsecond)')) ----- -50123 - -query I -SELECT date_part('microsecond', arrow_cast('23:32:50.123456'::time, 'Time64(Microsecond)')) ----- -50123456 - -query I -SELECT extract(microsecond from arrow_cast('23:32:50.123456'::time, 'Time64(Microsecond)')) ----- -50123456 - -query error DataFusion error: Internal error: unit Nanosecond not supported -SELECT extract(nanosecond from arrow_cast('23:32:50.123456'::time, 'Time64(Microsecond)')) - -query R -SELECT date_part('epoch', arrow_cast('23:32:50.123456'::time, 'Time64(Microsecond)')) ----- -84770.123456 - -query R -SELECT extract(epoch from arrow_cast('23:32:50.123456'::time, 'Time64(Microsecond)')) ----- -84770.123456 - -## time64 nanoseconds -query I -SELECT date_part('hour', arrow_cast('23:32:50.123456789'::time, 'Time64(Nanosecond)')) ----- -23 - -query I -SELECT extract(hour from arrow_cast('23:32:50.123456789'::time, 'Time64(Nanosecond)')) ----- -23 - -query I -SELECT date_part('minute', arrow_cast('23:32:50.123456789'::time, 'Time64(Nanosecond)')) ----- -32 - -query I -SELECT extract(minute from arrow_cast('23:32:50.123456789'::time, 'Time64(Nanosecond)')) ----- -32 - -query I -SELECT date_part('second', arrow_cast('23:32:50.123456789'::time, 'Time64(Nanosecond)')) ----- -50 - -query I -select extract(second from '2024-08-09T12:13:14') ----- -14 - -query I -select extract(seconds from '2024-08-09T12:13:14') ----- -14 - -query I -SELECT extract(second from arrow_cast('23:32:50.123456789'::time, 'Time64(Nanosecond)')) ----- -50 - -query I -SELECT date_part('millisecond', arrow_cast('23:32:50.123456789'::time, 'Time64(Nanosecond)')) ----- -50123 - -query I -SELECT extract(millisecond from arrow_cast('23:32:50.123456789'::time, 'Time64(Nanosecond)')) ----- -50123 - -# just some floating point stuff happening in the result here -query I -SELECT date_part('microsecond', arrow_cast('23:32:50.123456789'::time, 'Time64(Nanosecond)')) ----- -50123456 - -query I -SELECT extract(microsecond from arrow_cast('23:32:50.123456789'::time, 'Time64(Nanosecond)')) ----- -50123456 - -query I -SELECT extract(us from arrow_cast('23:32:50.123456789'::time, 'Time64(Nanosecond)')) ----- -50123456 - -query error DataFusion error: Internal error: unit Nanosecond not supported -SELECT date_part('nanosecond', arrow_cast('23:32:50.123456789'::time, 'Time64(Nanosecond)')) - -query R -SELECT date_part('epoch', arrow_cast('23:32:50.123456789'::time, 'Time64(Nanosecond)')) ----- -84770.123456789 - -query R -SELECT extract(epoch from arrow_cast('23:32:50.123456789'::time, 'Time64(Nanosecond)')) ----- -84770.123456789 - -# test_extract_epoch - -query R -SELECT extract(epoch from '1870-01-01T07:29:10.256'::timestamp) ----- --3155646649.744 - -query R -SELECT extract(epoch from '2000-01-01T00:00:00.000'::timestamp) ----- -946684800 - -query R -SELECT extract(epoch from to_timestamp('2000-01-01T00:00:00+00:00')) ----- -946684800 - -query R -SELECT extract(epoch from NULL::timestamp) ----- -NULL - -query R -SELECT extract(epoch from arrow_cast('1970-01-01', 'Date32')) ----- -0 - -query R -SELECT extract(epoch from arrow_cast('1970-01-02', 'Date32')) ----- -86400 - -query R -SELECT extract(epoch from arrow_cast('1970-01-11', 'Date32')) ----- -864000 - -query R -SELECT extract(epoch from arrow_cast('1969-12-31', 'Date32')) ----- --86400 - -query R -SELECT extract(epoch from arrow_cast('1970-01-01', 'Date64')) ----- -0 - -query R -SELECT extract(epoch from arrow_cast('1970-01-02', 'Date64')) ----- -86400 - -query R -SELECT extract(epoch from arrow_cast('1970-01-11', 'Date64')) ----- -864000 - -query R -SELECT extract(epoch from arrow_cast('1969-12-31', 'Date64')) ----- --86400 - -# test_extract_interval - -query I -SELECT extract(year from arrow_cast('10 years', 'Interval(YearMonth)')) ----- -10 - -query I -SELECT extract(month from arrow_cast('10 years', 'Interval(YearMonth)')) ----- -0 - -query I -SELECT extract(year from arrow_cast('10 months', 'Interval(YearMonth)')) ----- -0 - -query I -SELECT extract(month from arrow_cast('10 months', 'Interval(YearMonth)')) ----- -10 - -query I -SELECT extract(year from arrow_cast('20 months', 'Interval(YearMonth)')) ----- -1 - -query I -SELECT extract(month from arrow_cast('20 months', 'Interval(YearMonth)')) ----- -8 - -query error DataFusion error: Arrow error: Compute error: Year does not support: Interval\(DayTime\) -SELECT extract(year from arrow_cast('10 days', 'Interval(DayTime)')) - -query error DataFusion error: Arrow error: Compute error: Month does not support: Interval\(DayTime\) -SELECT extract(month from arrow_cast('10 days', 'Interval(DayTime)')) - -query I -SELECT extract(day from arrow_cast('10 days', 'Interval(DayTime)')) ----- -10 - -query I -SELECT extract(day from arrow_cast('14400 minutes', 'Interval(DayTime)')) ----- -0 - -query I -SELECT extract(minute from arrow_cast('14400 minutes', 'Interval(DayTime)')) ----- -14400 - -query I -SELECT extract(second from arrow_cast('5.1 seconds', 'Interval(DayTime)')) ----- -5 - -query I -SELECT extract(second from arrow_cast('14400 minutes', 'Interval(DayTime)')) ----- -864000 - -query I -SELECT extract(second from arrow_cast('2 months', 'Interval(MonthDayNano)')) ----- -0 - -query I -SELECT extract(second from arrow_cast('2 days', 'Interval(MonthDayNano)')) ----- -0 - -query I -SELECT extract(second from arrow_cast('2 seconds', 'Interval(MonthDayNano)')) ----- -2 - -query I -SELECT extract(seconds from arrow_cast('2 seconds', 'Interval(MonthDayNano)')) ----- -2 - -query R -SELECT extract(epoch from arrow_cast('2 seconds', 'Interval(MonthDayNano)')) ----- -2 - -query I -SELECT extract(milliseconds from arrow_cast('2 seconds', 'Interval(MonthDayNano)')) ----- -2000 - -query I -SELECT extract(second from arrow_cast('2030 milliseconds', 'Interval(MonthDayNano)')) ----- -2 - -query I -SELECT extract(second from arrow_cast(NULL, 'Interval(MonthDayNano)')) ----- -NULL - -statement ok -create table t (id int, i interval) as values - (0, interval '5 months 1 day 10 nanoseconds'), - (1, interval '1 year 3 months'), - (2, interval '3 days 2 milliseconds'), - (3, interval '2 seconds'), - (4, interval '8 months'), - (5, NULL); - -query III -select - id, - extract(second from i), - extract(month from i) -from t -order by id; ----- -0 0 5 -1 0 15 -2 0 0 -3 2 0 -4 0 8 -5 NULL NULL - -statement ok -drop table t; - -# test_extract_duration - -query I -SELECT extract(second from arrow_cast(2, 'Duration(Second)')) ----- -2 - -query I -SELECT extract(seconds from arrow_cast(2, 'Duration(Second)')) ----- -2 - -query R -SELECT extract(epoch from arrow_cast(2, 'Duration(Second)')) ----- -2 - -query I -SELECT extract(millisecond from arrow_cast(2, 'Duration(Second)')) ----- -2000 - -query I -SELECT extract(second from arrow_cast(2, 'Duration(Millisecond)')) ----- -0 - -query I -SELECT extract(second from arrow_cast(2002, 'Duration(Millisecond)')) ----- -2 - -query I -SELECT extract(millisecond from arrow_cast(2002, 'Duration(Millisecond)')) ----- -2002 - -query I -SELECT extract(day from arrow_cast(864000, 'Duration(Second)')) ----- -10 - -query error DataFusion error: Arrow error: Compute error: Month does not support: Duration\(Second\) -SELECT extract(month from arrow_cast(864000, 'Duration(Second)')) - -query error DataFusion error: Arrow error: Compute error: Year does not support: Duration\(Second\) -SELECT extract(year from arrow_cast(864000, 'Duration(Second)')) - -query I -SELECT extract(day from arrow_cast(NULL, 'Duration(Second)')) ----- -NULL - -# test_extract_date_part_func - -query B -SELECT (date_part('year', now()) = EXTRACT(year FROM now())) ----- -true - -query B -SELECT (date_part('quarter', now()) = EXTRACT(quarter FROM now())) ----- -true - -query B -SELECT (date_part('month', now()) = EXTRACT(month FROM now())) ----- -true - -query B -SELECT (date_part('week', now()) = EXTRACT(week FROM now())) ----- -true - -query B -SELECT (date_part('day', now()) = EXTRACT(day FROM now())) ----- -true - -query B -SELECT (date_part('hour', now()) = EXTRACT(hour FROM now())) ----- -true - -query B -SELECT (date_part('minute', now()) = EXTRACT(minute FROM now())) ----- -true - -query B -SELECT (date_part('second', now()) = EXTRACT(second FROM now())) ----- -true - -query B -SELECT (date_part('millisecond', now()) = EXTRACT(millisecond FROM now())) ----- -true - -query B -SELECT (date_part('microsecond', now()) = EXTRACT(microsecond FROM now())) ----- -true - -query error DataFusion error: Internal error: unit Nanosecond not supported -SELECT (date_part('nanosecond', now()) = EXTRACT(nanosecond FROM now())) - query B SELECT 'a' IN ('a','b') ---- diff --git a/datafusion/sqllogictest/test_files/expr/date_part.slt b/datafusion/sqllogictest/test_files/expr/date_part.slt new file mode 100644 index 000000000000..cec80a165f30 --- /dev/null +++ b/datafusion/sqllogictest/test_files/expr/date_part.slt @@ -0,0 +1,878 @@ +# 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. + +# Tests for `date_part` and `EXTRACT` (which is a different syntax +# for the same function). + +query error +SELECT EXTRACT("'''year'''" FROM timestamp '2020-09-08T12:00:00+00:00') + +query error +SELECT EXTRACT("'year'" FROM timestamp '2020-09-08T12:00:00+00:00') + +query I +SELECT date_part('YEAR', CAST('2000-01-01' AS DATE)) +---- +2000 + +query I +SELECT EXTRACT(year FROM timestamp '2020-09-08T12:00:00+00:00') +---- +2020 + +query I +SELECT EXTRACT("year" FROM timestamp '2020-09-08T12:00:00+00:00') +---- +2020 + +query I +SELECT EXTRACT('year' FROM timestamp '2020-09-08T12:00:00+00:00') +---- +2020 + +query I +SELECT date_part('QUARTER', CAST('2000-01-01' AS DATE)) +---- +1 + +query I +SELECT EXTRACT(quarter FROM to_timestamp('2020-09-08T12:00:00+00:00')) +---- +3 + +query I +SELECT EXTRACT("quarter" FROM to_timestamp('2020-09-08T12:00:00+00:00')) +---- +3 + +query I +SELECT EXTRACT('quarter' FROM to_timestamp('2020-09-08T12:00:00+00:00')) +---- +3 + +query I +SELECT date_part('MONTH', CAST('2000-01-01' AS DATE)) +---- +1 + +query I +SELECT EXTRACT(month FROM to_timestamp('2020-09-08T12:00:00+00:00')) +---- +9 + +query I +SELECT EXTRACT("month" FROM to_timestamp('2020-09-08T12:00:00+00:00')) +---- +9 + +query I +SELECT EXTRACT('month' FROM to_timestamp('2020-09-08T12:00:00+00:00')) +---- +9 + +query I +SELECT date_part('WEEK', CAST('2003-01-01' AS DATE)) +---- +1 + +query I +SELECT EXTRACT(WEEK FROM to_timestamp('2020-09-08T12:00:00+00:00')) +---- +37 + +query I +SELECT EXTRACT("WEEK" FROM to_timestamp('2020-09-08T12:00:00+00:00')) +---- +37 + +query I +SELECT EXTRACT('WEEK' FROM to_timestamp('2020-09-08T12:00:00+00:00')) +---- +37 + +query I +SELECT date_part('DAY', CAST('2000-01-01' AS DATE)) +---- +1 + +query I +SELECT EXTRACT(day FROM to_timestamp('2020-09-08T12:00:00+00:00')) +---- +8 + +query I +SELECT EXTRACT("day" FROM to_timestamp('2020-09-08T12:00:00+00:00')) +---- +8 + +query I +SELECT EXTRACT('day' FROM to_timestamp('2020-09-08T12:00:00+00:00')) +---- +8 + +query I +SELECT date_part('DOY', CAST('2000-01-01' AS DATE)) +---- +1 + +query I +SELECT EXTRACT(doy FROM to_timestamp('2020-09-08T12:00:00+00:00')) +---- +252 + +query I +SELECT EXTRACT("doy" FROM to_timestamp('2020-09-08T12:00:00+00:00')) +---- +252 + +query I +SELECT EXTRACT('doy' FROM to_timestamp('2020-09-08T12:00:00+00:00')) +---- +252 + +query I +SELECT date_part('DOW', CAST('2000-01-01' AS DATE)) +---- +6 + +query I +SELECT EXTRACT(dow FROM to_timestamp('2020-09-08T12:00:00+00:00')) +---- +2 + +query I +SELECT EXTRACT("dow" FROM to_timestamp('2020-09-08T12:00:00+00:00')) +---- +2 + +query I +SELECT EXTRACT('dow' FROM to_timestamp('2020-09-08T12:00:00+00:00')) +---- +2 + +query I +SELECT date_part('HOUR', CAST('2000-01-01' AS DATE)) +---- +0 + +query I +SELECT EXTRACT(hour FROM to_timestamp('2020-09-08T12:03:03+00:00')) +---- +12 + +query I +SELECT EXTRACT("hour" FROM to_timestamp('2020-09-08T12:03:03+00:00')) +---- +12 + +query I +SELECT EXTRACT('hour' FROM to_timestamp('2020-09-08T12:03:03+00:00')) +---- +12 + +query I +SELECT EXTRACT(minute FROM to_timestamp('2020-09-08T12:12:00+00:00')) +---- +12 + +query I +SELECT EXTRACT("minute" FROM to_timestamp('2020-09-08T12:12:00+00:00')) +---- +12 + +query I +SELECT EXTRACT('minute' FROM to_timestamp('2020-09-08T12:12:00+00:00')) +---- +12 + +query I +SELECT date_part('minute', to_timestamp('2020-09-08T12:12:00+00:00')) +---- +12 + +# make sure the return type is integer +query T +SELECT arrow_typeof(date_part('minute', to_timestamp('2020-09-08T12:12:00+00:00'))) +---- +Int32 + +query I +SELECT EXTRACT(second FROM timestamp '2020-09-08T12:00:12.12345678+00:00') +---- +12 + +query I +SELECT EXTRACT(millisecond FROM timestamp '2020-09-08T12:00:12.12345678+00:00') +---- +12123 + +query I +SELECT EXTRACT(microsecond FROM timestamp '2020-09-08T12:00:12.12345678+00:00') +---- +12123456 + +query error DataFusion error: Internal error: unit Nanosecond not supported +SELECT EXTRACT(nanosecond FROM timestamp '2020-09-08T12:00:12.12345678+00:00') + +query I +SELECT EXTRACT("second" FROM timestamp '2020-09-08T12:00:12.12345678+00:00') +---- +12 + +query I +SELECT EXTRACT("millisecond" FROM timestamp '2020-09-08T12:00:12.12345678+00:00') +---- +12123 + +query I +SELECT EXTRACT("microsecond" FROM timestamp '2020-09-08T12:00:12.12345678+00:00') +---- +12123456 + +query error DataFusion error: Internal error: unit Nanosecond not supported +SELECT EXTRACT("nanosecond" FROM timestamp '2020-09-08T12:00:12.12345678+00:00') + +query I +SELECT EXTRACT('second' FROM timestamp '2020-09-08T12:00:12.12345678+00:00') +---- +12 + +query I +SELECT EXTRACT('millisecond' FROM timestamp '2020-09-08T12:00:12.12345678+00:00') +---- +12123 + +query I +SELECT EXTRACT('microsecond' FROM timestamp '2020-09-08T12:00:12.12345678+00:00') +---- +12123456 + +query error DataFusion error: Internal error: unit Nanosecond not supported +SELECT EXTRACT('nanosecond' FROM timestamp '2020-09-08T12:00:12.12345678+00:00') + + +# Keep precision when coercing Utf8 to Timestamp +query I +SELECT date_part('second', timestamp '2020-09-08T12:00:12.12345678+00:00') +---- +12 + +query I +SELECT date_part('millisecond', timestamp '2020-09-08T12:00:12.12345678+00:00') +---- +12123 + +query I +SELECT date_part('microsecond', timestamp '2020-09-08T12:00:12.12345678+00:00') +---- +12123456 + +query error DataFusion error: Internal error: unit Nanosecond not supported +SELECT date_part('nanosecond', timestamp '2020-09-08T12:00:12.12345678+00:00') + + +query I +SELECT date_part('second', '2020-09-08T12:00:12.12345678+00:00') +---- +12 + +query I +SELECT date_part('millisecond', '2020-09-08T12:00:12.12345678+00:00') +---- +12123 + +query I +SELECT date_part('microsecond', '2020-09-08T12:00:12.12345678+00:00') +---- +12123456 + +query error DataFusion error: Internal error: unit Nanosecond not supported +SELECT date_part('nanosecond', '2020-09-08T12:00:12.12345678+00:00') + +# test_date_part_time + +## time32 seconds +query I +SELECT date_part('hour', arrow_cast('23:32:50'::time, 'Time32(Second)')) +---- +23 + +query I +SELECT extract(hour from arrow_cast('23:32:50'::time, 'Time32(Second)')) +---- +23 + +query I +SELECT date_part('minute', arrow_cast('23:32:50'::time, 'Time32(Second)')) +---- +32 + +query I +SELECT extract(minute from arrow_cast('23:32:50'::time, 'Time32(Second)')) +---- +32 + +query I +SELECT date_part('second', arrow_cast('23:32:50'::time, 'Time32(Second)')) +---- +50 + +query I +SELECT extract(second from arrow_cast('23:32:50'::time, 'Time32(Second)')) +---- +50 + +query I +SELECT date_part('millisecond', arrow_cast('23:32:50'::time, 'Time32(Second)')) +---- +50000 + +query I +SELECT extract(millisecond from arrow_cast('23:32:50'::time, 'Time32(Second)')) +---- +50000 + +query I +SELECT date_part('microsecond', arrow_cast('23:32:50'::time, 'Time32(Second)')) +---- +50000000 + +query I +SELECT extract(microsecond from arrow_cast('23:32:50'::time, 'Time32(Second)')) +---- +50000000 + +query error DataFusion error: Internal error: unit Nanosecond not supported +SELECT extract(nanosecond from arrow_cast('23:32:50'::time, 'Time32(Second)')) + +query R +SELECT date_part('epoch', arrow_cast('23:32:50'::time, 'Time32(Second)')) +---- +84770 + +query R +SELECT extract(epoch from arrow_cast('23:32:50'::time, 'Time32(Second)')) +---- +84770 + +## time32 milliseconds +query I +SELECT date_part('hour', arrow_cast('23:32:50.123'::time, 'Time32(Millisecond)')) +---- +23 + +query I +SELECT extract(hour from arrow_cast('23:32:50.123'::time, 'Time32(Millisecond)')) +---- +23 + +query I +SELECT date_part('minute', arrow_cast('23:32:50.123'::time, 'Time32(Millisecond)')) +---- +32 + +query I +SELECT extract(minute from arrow_cast('23:32:50.123'::time, 'Time32(Millisecond)')) +---- +32 + +query I +SELECT date_part('second', arrow_cast('23:32:50.123'::time, 'Time32(Millisecond)')) +---- +50 + +query I +SELECT extract(second from arrow_cast('23:32:50.123'::time, 'Time32(Millisecond)')) +---- +50 + +query I +SELECT date_part('millisecond', arrow_cast('23:32:50.123'::time, 'Time32(Millisecond)')) +---- +50123 + +query I +SELECT extract(millisecond from arrow_cast('23:32:50.123'::time, 'Time32(Millisecond)')) +---- +50123 + +query I +SELECT date_part('microsecond', arrow_cast('23:32:50.123'::time, 'Time32(Millisecond)')) +---- +50123000 + +query I +SELECT extract(microsecond from arrow_cast('23:32:50.123'::time, 'Time32(Millisecond)')) +---- +50123000 + +query error DataFusion error: Internal error: unit Nanosecond not supported +SELECT extract(nanosecond from arrow_cast('23:32:50.123'::time, 'Time32(Millisecond)')) + +query R +SELECT date_part('epoch', arrow_cast('23:32:50.123'::time, 'Time32(Millisecond)')) +---- +84770.123 + +query R +SELECT extract(epoch from arrow_cast('23:32:50.123'::time, 'Time32(Millisecond)')) +---- +84770.123 + +## time64 microseconds +query I +SELECT date_part('hour', arrow_cast('23:32:50.123456'::time, 'Time64(Microsecond)')) +---- +23 + +query I +SELECT extract(hour from arrow_cast('23:32:50.123456'::time, 'Time64(Microsecond)')) +---- +23 + +query I +SELECT date_part('minute', arrow_cast('23:32:50.123456'::time, 'Time64(Microsecond)')) +---- +32 + +query I +SELECT extract(minute from arrow_cast('23:32:50.123456'::time, 'Time64(Microsecond)')) +---- +32 + +query I +SELECT date_part('second', arrow_cast('23:32:50.123456'::time, 'Time64(Microsecond)')) +---- +50 + +query I +SELECT extract(second from arrow_cast('23:32:50.123456'::time, 'Time64(Microsecond)')) +---- +50 + +query I +SELECT date_part('millisecond', arrow_cast('23:32:50.123456'::time, 'Time64(Microsecond)')) +---- +50123 + +query I +SELECT extract(millisecond from arrow_cast('23:32:50.123456'::time, 'Time64(Microsecond)')) +---- +50123 + +query I +SELECT date_part('microsecond', arrow_cast('23:32:50.123456'::time, 'Time64(Microsecond)')) +---- +50123456 + +query I +SELECT extract(microsecond from arrow_cast('23:32:50.123456'::time, 'Time64(Microsecond)')) +---- +50123456 + +query error DataFusion error: Internal error: unit Nanosecond not supported +SELECT extract(nanosecond from arrow_cast('23:32:50.123456'::time, 'Time64(Microsecond)')) + +query R +SELECT date_part('epoch', arrow_cast('23:32:50.123456'::time, 'Time64(Microsecond)')) +---- +84770.123456 + +query R +SELECT extract(epoch from arrow_cast('23:32:50.123456'::time, 'Time64(Microsecond)')) +---- +84770.123456 + +## time64 nanoseconds +query I +SELECT date_part('hour', arrow_cast('23:32:50.123456789'::time, 'Time64(Nanosecond)')) +---- +23 + +query I +SELECT extract(hour from arrow_cast('23:32:50.123456789'::time, 'Time64(Nanosecond)')) +---- +23 + +query I +SELECT date_part('minute', arrow_cast('23:32:50.123456789'::time, 'Time64(Nanosecond)')) +---- +32 + +query I +SELECT extract(minute from arrow_cast('23:32:50.123456789'::time, 'Time64(Nanosecond)')) +---- +32 + +query I +SELECT date_part('second', arrow_cast('23:32:50.123456789'::time, 'Time64(Nanosecond)')) +---- +50 + +query I +select extract(second from '2024-08-09T12:13:14') +---- +14 + +query I +select extract(seconds from '2024-08-09T12:13:14') +---- +14 + +query I +SELECT extract(second from arrow_cast('23:32:50.123456789'::time, 'Time64(Nanosecond)')) +---- +50 + +query I +SELECT date_part('millisecond', arrow_cast('23:32:50.123456789'::time, 'Time64(Nanosecond)')) +---- +50123 + +query I +SELECT extract(millisecond from arrow_cast('23:32:50.123456789'::time, 'Time64(Nanosecond)')) +---- +50123 + +# just some floating point stuff happening in the result here +query I +SELECT date_part('microsecond', arrow_cast('23:32:50.123456789'::time, 'Time64(Nanosecond)')) +---- +50123456 + +query I +SELECT extract(microsecond from arrow_cast('23:32:50.123456789'::time, 'Time64(Nanosecond)')) +---- +50123456 + +query I +SELECT extract(us from arrow_cast('23:32:50.123456789'::time, 'Time64(Nanosecond)')) +---- +50123456 + +query error DataFusion error: Internal error: unit Nanosecond not supported +SELECT date_part('nanosecond', arrow_cast('23:32:50.123456789'::time, 'Time64(Nanosecond)')) + +query R +SELECT date_part('epoch', arrow_cast('23:32:50.123456789'::time, 'Time64(Nanosecond)')) +---- +84770.123456789 + +query R +SELECT extract(epoch from arrow_cast('23:32:50.123456789'::time, 'Time64(Nanosecond)')) +---- +84770.123456789 + +# test_extract_epoch + +query R +SELECT extract(epoch from '1870-01-01T07:29:10.256'::timestamp) +---- +-3155646649.744 + +query R +SELECT extract(epoch from '2000-01-01T00:00:00.000'::timestamp) +---- +946684800 + +query R +SELECT extract(epoch from to_timestamp('2000-01-01T00:00:00+00:00')) +---- +946684800 + +query R +SELECT extract(epoch from NULL::timestamp) +---- +NULL + +query R +SELECT extract(epoch from arrow_cast('1970-01-01', 'Date32')) +---- +0 + +query R +SELECT extract(epoch from arrow_cast('1970-01-02', 'Date32')) +---- +86400 + +query R +SELECT extract(epoch from arrow_cast('1970-01-11', 'Date32')) +---- +864000 + +query R +SELECT extract(epoch from arrow_cast('1969-12-31', 'Date32')) +---- +-86400 + +query R +SELECT extract(epoch from arrow_cast('1970-01-01', 'Date64')) +---- +0 + +query R +SELECT extract(epoch from arrow_cast('1970-01-02', 'Date64')) +---- +86400 + +query R +SELECT extract(epoch from arrow_cast('1970-01-11', 'Date64')) +---- +864000 + +query R +SELECT extract(epoch from arrow_cast('1969-12-31', 'Date64')) +---- +-86400 + +# test_extract_interval + +query I +SELECT extract(year from arrow_cast('10 years', 'Interval(YearMonth)')) +---- +10 + +query I +SELECT extract(month from arrow_cast('10 years', 'Interval(YearMonth)')) +---- +0 + +query I +SELECT extract(year from arrow_cast('10 months', 'Interval(YearMonth)')) +---- +0 + +query I +SELECT extract(month from arrow_cast('10 months', 'Interval(YearMonth)')) +---- +10 + +query I +SELECT extract(year from arrow_cast('20 months', 'Interval(YearMonth)')) +---- +1 + +query I +SELECT extract(month from arrow_cast('20 months', 'Interval(YearMonth)')) +---- +8 + +query error DataFusion error: Arrow error: Compute error: Year does not support: Interval\(DayTime\) +SELECT extract(year from arrow_cast('10 days', 'Interval(DayTime)')) + +query error DataFusion error: Arrow error: Compute error: Month does not support: Interval\(DayTime\) +SELECT extract(month from arrow_cast('10 days', 'Interval(DayTime)')) + +query I +SELECT extract(day from arrow_cast('10 days', 'Interval(DayTime)')) +---- +10 + +query I +SELECT extract(day from arrow_cast('14400 minutes', 'Interval(DayTime)')) +---- +0 + +query I +SELECT extract(minute from arrow_cast('14400 minutes', 'Interval(DayTime)')) +---- +14400 + +query I +SELECT extract(second from arrow_cast('5.1 seconds', 'Interval(DayTime)')) +---- +5 + +query I +SELECT extract(second from arrow_cast('14400 minutes', 'Interval(DayTime)')) +---- +864000 + +query I +SELECT extract(second from arrow_cast('2 months', 'Interval(MonthDayNano)')) +---- +0 + +query I +SELECT extract(second from arrow_cast('2 days', 'Interval(MonthDayNano)')) +---- +0 + +query I +SELECT extract(second from arrow_cast('2 seconds', 'Interval(MonthDayNano)')) +---- +2 + +query I +SELECT extract(seconds from arrow_cast('2 seconds', 'Interval(MonthDayNano)')) +---- +2 + +query R +SELECT extract(epoch from arrow_cast('2 seconds', 'Interval(MonthDayNano)')) +---- +2 + +query I +SELECT extract(milliseconds from arrow_cast('2 seconds', 'Interval(MonthDayNano)')) +---- +2000 + +query I +SELECT extract(second from arrow_cast('2030 milliseconds', 'Interval(MonthDayNano)')) +---- +2 + +query I +SELECT extract(second from arrow_cast(NULL, 'Interval(MonthDayNano)')) +---- +NULL + +statement ok +create table t (id int, i interval) as values + (0, interval '5 months 1 day 10 nanoseconds'), + (1, interval '1 year 3 months'), + (2, interval '3 days 2 milliseconds'), + (3, interval '2 seconds'), + (4, interval '8 months'), + (5, NULL); + +query III +select + id, + extract(second from i), + extract(month from i) +from t +order by id; +---- +0 0 5 +1 0 15 +2 0 0 +3 2 0 +4 0 8 +5 NULL NULL + +statement ok +drop table t; + +# test_extract_duration + +query I +SELECT extract(second from arrow_cast(2, 'Duration(Second)')) +---- +2 + +query I +SELECT extract(seconds from arrow_cast(2, 'Duration(Second)')) +---- +2 + +query R +SELECT extract(epoch from arrow_cast(2, 'Duration(Second)')) +---- +2 + +query I +SELECT extract(millisecond from arrow_cast(2, 'Duration(Second)')) +---- +2000 + +query I +SELECT extract(second from arrow_cast(2, 'Duration(Millisecond)')) +---- +0 + +query I +SELECT extract(second from arrow_cast(2002, 'Duration(Millisecond)')) +---- +2 + +query I +SELECT extract(millisecond from arrow_cast(2002, 'Duration(Millisecond)')) +---- +2002 + +query I +SELECT extract(day from arrow_cast(864000, 'Duration(Second)')) +---- +10 + +query error DataFusion error: Arrow error: Compute error: Month does not support: Duration\(Second\) +SELECT extract(month from arrow_cast(864000, 'Duration(Second)')) + +query error DataFusion error: Arrow error: Compute error: Year does not support: Duration\(Second\) +SELECT extract(year from arrow_cast(864000, 'Duration(Second)')) + +query I +SELECT extract(day from arrow_cast(NULL, 'Duration(Second)')) +---- +NULL + +# test_extract_date_part_func + +query B +SELECT (date_part('year', now()) = EXTRACT(year FROM now())) +---- +true + +query B +SELECT (date_part('quarter', now()) = EXTRACT(quarter FROM now())) +---- +true + +query B +SELECT (date_part('month', now()) = EXTRACT(month FROM now())) +---- +true + +query B +SELECT (date_part('week', now()) = EXTRACT(week FROM now())) +---- +true + +query B +SELECT (date_part('day', now()) = EXTRACT(day FROM now())) +---- +true + +query B +SELECT (date_part('hour', now()) = EXTRACT(hour FROM now())) +---- +true + +query B +SELECT (date_part('minute', now()) = EXTRACT(minute FROM now())) +---- +true + +query B +SELECT (date_part('second', now()) = EXTRACT(second FROM now())) +---- +true + +query B +SELECT (date_part('millisecond', now()) = EXTRACT(millisecond FROM now())) +---- +true + +query B +SELECT (date_part('microsecond', now()) = EXTRACT(microsecond FROM now())) +---- +true + +query error DataFusion error: Internal error: unit Nanosecond not supported +SELECT (date_part('nanosecond', now()) = EXTRACT(nanosecond FROM now())) From 08119e634e1740801bf543d8106f92de340ec011 Mon Sep 17 00:00:00 2001 From: Jax Liu Date: Wed, 11 Dec 2024 23:27:31 +0800 Subject: [PATCH 14/14] Support unparsing `UNNEST` plan to `UNNEST` table factor SQL (#13660) * add `unnest_as_table_factor` and `UnnestRelationBuilder` * unparse unnest as table factor * fix typo * add tests for the default configs * add a static const for unnest_placeholder * fix tests * fix tests --- datafusion/sql/src/unparser/ast.rs | 73 ++++++++++++++ datafusion/sql/src/unparser/dialect.rs | 23 +++++ datafusion/sql/src/unparser/plan.rs | 55 ++++++++++- datafusion/sql/src/unparser/utils.rs | 2 +- datafusion/sql/src/utils.rs | 50 +++++----- datafusion/sql/tests/cases/plan_to_sql.rs | 99 ++++++++++++++++++- .../sqllogictest/test_files/encoding.slt | 2 +- datafusion/sqllogictest/test_files/joins.slt | 12 +-- .../test_files/push_down_filter.slt | 40 ++++---- .../test_files/table_functions.slt | 2 +- datafusion/sqllogictest/test_files/unnest.slt | 28 +++--- datafusion/sqllogictest/test_files/window.slt | 1 - 12 files changed, 313 insertions(+), 74 deletions(-) diff --git a/datafusion/sql/src/unparser/ast.rs b/datafusion/sql/src/unparser/ast.rs index cc0812cd71e1..ad0b5f16b283 100644 --- a/datafusion/sql/src/unparser/ast.rs +++ b/datafusion/sql/src/unparser/ast.rs @@ -353,6 +353,7 @@ pub(super) struct RelationBuilder { enum TableFactorBuilder { Table(TableRelationBuilder), Derived(DerivedRelationBuilder), + Unnest(UnnestRelationBuilder), Empty, } @@ -369,6 +370,12 @@ impl RelationBuilder { self.relation = Some(TableFactorBuilder::Derived(value)); self } + + pub fn unnest(&mut self, value: UnnestRelationBuilder) -> &mut Self { + self.relation = Some(TableFactorBuilder::Unnest(value)); + self + } + pub fn empty(&mut self) -> &mut Self { self.relation = Some(TableFactorBuilder::Empty); self @@ -382,6 +389,9 @@ impl RelationBuilder { Some(TableFactorBuilder::Derived(ref mut rel_builder)) => { rel_builder.alias = value; } + Some(TableFactorBuilder::Unnest(ref mut rel_builder)) => { + rel_builder.alias = value; + } Some(TableFactorBuilder::Empty) => (), None => (), } @@ -391,6 +401,7 @@ impl RelationBuilder { Ok(match self.relation { Some(TableFactorBuilder::Table(ref value)) => Some(value.build()?), Some(TableFactorBuilder::Derived(ref value)) => Some(value.build()?), + Some(TableFactorBuilder::Unnest(ref value)) => Some(value.build()?), Some(TableFactorBuilder::Empty) => None, None => return Err(Into::into(UninitializedFieldError::from("relation"))), }) @@ -526,6 +537,68 @@ impl Default for DerivedRelationBuilder { } } +#[derive(Clone)] +pub(super) struct UnnestRelationBuilder { + pub alias: Option, + pub array_exprs: Vec, + with_offset: bool, + with_offset_alias: Option, + with_ordinality: bool, +} + +#[allow(dead_code)] +impl UnnestRelationBuilder { + pub fn alias(&mut self, value: Option) -> &mut Self { + self.alias = value; + self + } + pub fn array_exprs(&mut self, value: Vec) -> &mut Self { + self.array_exprs = value; + self + } + + pub fn with_offset(&mut self, value: bool) -> &mut Self { + self.with_offset = value; + self + } + + pub fn with_offset_alias(&mut self, value: Option) -> &mut Self { + self.with_offset_alias = value; + self + } + + pub fn with_ordinality(&mut self, value: bool) -> &mut Self { + self.with_ordinality = value; + self + } + + pub fn build(&self) -> Result { + Ok(ast::TableFactor::UNNEST { + alias: self.alias.clone(), + array_exprs: self.array_exprs.clone(), + with_offset: self.with_offset, + with_offset_alias: self.with_offset_alias.clone(), + with_ordinality: self.with_ordinality, + }) + } + + fn create_empty() -> Self { + Self { + alias: Default::default(), + array_exprs: Default::default(), + with_offset: Default::default(), + with_offset_alias: Default::default(), + with_ordinality: Default::default(), + } + } +} + +impl Default for UnnestRelationBuilder { + fn default() -> Self { + Self::create_empty() + } +} + /// Runtime error when a `build()` method is called and one or more required fields /// do not have a value. #[derive(Debug, Clone)] diff --git a/datafusion/sql/src/unparser/dialect.rs b/datafusion/sql/src/unparser/dialect.rs index e979d8fd4ebd..ae387d441fa2 100644 --- a/datafusion/sql/src/unparser/dialect.rs +++ b/datafusion/sql/src/unparser/dialect.rs @@ -157,6 +157,15 @@ pub trait Dialect: Send + Sync { fn full_qualified_col(&self) -> bool { false } + + /// Allow to unparse the unnest plan as [ast::TableFactor::UNNEST]. + /// + /// Some dialects like BigQuery require UNNEST to be used in the FROM clause but + /// the LogicalPlan planner always puts UNNEST in the SELECT clause. This flag allows + /// to unparse the UNNEST plan as [ast::TableFactor::UNNEST] instead of a subquery. + fn unnest_as_table_factor(&self) -> bool { + false + } } /// `IntervalStyle` to use for unparsing @@ -448,6 +457,7 @@ pub struct CustomDialect { requires_derived_table_alias: bool, division_operator: BinaryOperator, full_qualified_col: bool, + unnest_as_table_factor: bool, } impl Default for CustomDialect { @@ -474,6 +484,7 @@ impl Default for CustomDialect { requires_derived_table_alias: false, division_operator: BinaryOperator::Divide, full_qualified_col: false, + unnest_as_table_factor: false, } } } @@ -582,6 +593,10 @@ impl Dialect for CustomDialect { fn full_qualified_col(&self) -> bool { self.full_qualified_col } + + fn unnest_as_table_factor(&self) -> bool { + self.unnest_as_table_factor + } } /// `CustomDialectBuilder` to build `CustomDialect` using builder pattern @@ -617,6 +632,7 @@ pub struct CustomDialectBuilder { requires_derived_table_alias: bool, division_operator: BinaryOperator, full_qualified_col: bool, + unnest_as_table_factor: bool, } impl Default for CustomDialectBuilder { @@ -649,6 +665,7 @@ impl CustomDialectBuilder { requires_derived_table_alias: false, division_operator: BinaryOperator::Divide, full_qualified_col: false, + unnest_as_table_factor: false, } } @@ -673,6 +690,7 @@ impl CustomDialectBuilder { requires_derived_table_alias: self.requires_derived_table_alias, division_operator: self.division_operator, full_qualified_col: self.full_qualified_col, + unnest_as_table_factor: self.unnest_as_table_factor, } } @@ -800,4 +818,9 @@ impl CustomDialectBuilder { self.full_qualified_col = full_qualified_col; self } + + pub fn with_unnest_as_table_factor(mut self, _unnest_as_table_factor: bool) -> Self { + self.unnest_as_table_factor = _unnest_as_table_factor; + self + } } diff --git a/datafusion/sql/src/unparser/plan.rs b/datafusion/sql/src/unparser/plan.rs index eaae4fe73d8c..e9f9f486ea9a 100644 --- a/datafusion/sql/src/unparser/plan.rs +++ b/datafusion/sql/src/unparser/plan.rs @@ -32,7 +32,9 @@ use super::{ }, Unparser, }; +use crate::unparser::ast::UnnestRelationBuilder; use crate::unparser::utils::unproject_agg_exprs; +use crate::utils::UNNEST_PLACEHOLDER; use datafusion_common::{ internal_err, not_impl_err, tree_node::{TransformedResult, TreeNode}, @@ -40,7 +42,7 @@ use datafusion_common::{ }; use datafusion_expr::{ expr::Alias, BinaryExpr, Distinct, Expr, JoinConstraint, JoinType, LogicalPlan, - LogicalPlanBuilder, Operator, Projection, SortExpr, TableScan, + LogicalPlanBuilder, Operator, Projection, SortExpr, TableScan, Unnest, }; use sqlparser::ast::{self, Ident, SetExpr}; use std::sync::Arc; @@ -312,6 +314,19 @@ impl Unparser<'_> { .select_to_sql_recursively(&new_plan, query, select, relation); } + // Projection can be top-level plan for unnest relation + // The projection generated by the `RecursiveUnnestRewriter` from a UNNEST relation will have + // only one expression, which is the placeholder column generated by the rewriter. + if self.dialect.unnest_as_table_factor() + && p.expr.len() == 1 + && Self::is_unnest_placeholder(&p.expr[0]) + { + if let LogicalPlan::Unnest(unnest) = &p.input.as_ref() { + return self + .unnest_to_table_factor_sql(unnest, query, select, relation); + } + } + // Projection can be top-level plan for derived table if select.already_projected() { return self.derive_with_dialect_alias( @@ -678,7 +693,11 @@ impl Unparser<'_> { ) } LogicalPlan::EmptyRelation(_) => { - relation.empty(); + // An EmptyRelation could be behind an UNNEST node. If the dialect supports UNNEST as a table factor, + // a TableRelationBuilder will be created for the UNNEST node first. + if !relation.has_relation() { + relation.empty(); + } Ok(()) } LogicalPlan::Extension(_) => not_impl_err!("Unsupported operator: {plan:?}"), @@ -708,6 +727,38 @@ impl Unparser<'_> { } } + /// Try to find the placeholder column name generated by `RecursiveUnnestRewriter` + /// Only match the pattern `Expr::Alias(Expr::Column("__unnest_placeholder(...)"))` + fn is_unnest_placeholder(expr: &Expr) -> bool { + if let Expr::Alias(Alias { expr, .. }) = expr { + if let Expr::Column(Column { name, .. }) = expr.as_ref() { + return name.starts_with(UNNEST_PLACEHOLDER); + } + } + false + } + + fn unnest_to_table_factor_sql( + &self, + unnest: &Unnest, + query: &mut Option, + select: &mut SelectBuilder, + relation: &mut RelationBuilder, + ) -> Result<()> { + let mut unnest_relation = UnnestRelationBuilder::default(); + let LogicalPlan::Projection(p) = unnest.input.as_ref() else { + return internal_err!("Unnest input is not a Projection: {unnest:?}"); + }; + let exprs = p + .expr + .iter() + .map(|e| self.expr_to_sql(e)) + .collect::>>()?; + unnest_relation.array_exprs(exprs); + relation.unnest(unnest_relation); + self.select_to_sql_recursively(p.input.as_ref(), query, select, relation) + } + fn is_scan_with_pushdown(scan: &TableScan) -> bool { scan.projection.is_some() || !scan.filters.is_empty() || scan.fetch.is_some() } diff --git a/datafusion/sql/src/unparser/utils.rs b/datafusion/sql/src/unparser/utils.rs index 518781106c3b..354a68f60964 100644 --- a/datafusion/sql/src/unparser/utils.rs +++ b/datafusion/sql/src/unparser/utils.rs @@ -133,7 +133,7 @@ pub(crate) fn find_window_nodes_within_select<'a>( /// Recursively identify Column expressions and transform them into the appropriate unnest expression /// -/// For example, if expr contains the column expr "unnest_placeholder(make_array(Int64(1),Int64(2),Int64(2),Int64(5),NULL),depth=1)" +/// For example, if expr contains the column expr "__unnest_placeholder(make_array(Int64(1),Int64(2),Int64(2),Int64(5),NULL),depth=1)" /// it will be transformed into an actual unnest expression UNNEST([1, 2, 2, 5, NULL]) pub(crate) fn unproject_unnest_expr(expr: Expr, unnest: &Unnest) -> Result { expr.transform(|sub_expr| { diff --git a/datafusion/sql/src/utils.rs b/datafusion/sql/src/utils.rs index 69e3953341ef..1c2a3ea91a2b 100644 --- a/datafusion/sql/src/utils.rs +++ b/datafusion/sql/src/utils.rs @@ -315,6 +315,8 @@ pub(crate) fn rewrite_recursive_unnests_bottom_up( .collect::>()) } +pub const UNNEST_PLACEHOLDER: &str = "__unnest_placeholder"; + /* This is only usedful when used with transform down up A full example of how the transformation works: @@ -360,9 +362,9 @@ impl RecursiveUnnestRewriter<'_> { // Full context, we are trying to plan the execution as InnerProjection->Unnest->OuterProjection // inside unnest execution, each column inside the inner projection // will be transformed into new columns. Thus we need to keep track of these placeholding column names - let placeholder_name = format!("unnest_placeholder({})", inner_expr_name); + let placeholder_name = format!("{UNNEST_PLACEHOLDER}({})", inner_expr_name); let post_unnest_name = - format!("unnest_placeholder({},depth={})", inner_expr_name, level); + format!("{UNNEST_PLACEHOLDER}({},depth={})", inner_expr_name, level); // This is due to the fact that unnest transformation should keep the original // column name as is, to comply with group by and order by let placeholder_column = Column::from_name(placeholder_name.clone()); @@ -693,17 +695,17 @@ mod tests { // Only the bottom most unnest exprs are transformed assert_eq!( transformed_exprs, - vec![col("unnest_placeholder(3d_col,depth=2)") + vec![col("__unnest_placeholder(3d_col,depth=2)") .alias("UNNEST(UNNEST(3d_col))") .add( - col("unnest_placeholder(3d_col,depth=2)") + col("__unnest_placeholder(3d_col,depth=2)") .alias("UNNEST(UNNEST(3d_col))") ) .add(col("i64_col"))] ); column_unnests_eq( vec![ - "unnest_placeholder(3d_col)=>[unnest_placeholder(3d_col,depth=2)|depth=2]", + "__unnest_placeholder(3d_col)=>[__unnest_placeholder(3d_col,depth=2)|depth=2]", ], &unnest_placeholder_columns, ); @@ -713,7 +715,7 @@ mod tests { assert_eq!( inner_projection_exprs, vec![ - col("3d_col").alias("unnest_placeholder(3d_col)"), + col("3d_col").alias("__unnest_placeholder(3d_col)"), col("i64_col") ] ); @@ -730,12 +732,12 @@ mod tests { assert_eq!( transformed_exprs, vec![ - (col("unnest_placeholder(3d_col,depth=1)").alias("UNNEST(3d_col)")) + (col("__unnest_placeholder(3d_col,depth=1)").alias("UNNEST(3d_col)")) .alias("2d_col") ] ); column_unnests_eq( - vec!["unnest_placeholder(3d_col)=>[unnest_placeholder(3d_col,depth=2)|depth=2, unnest_placeholder(3d_col,depth=1)|depth=1]"], + vec!["__unnest_placeholder(3d_col)=>[__unnest_placeholder(3d_col,depth=2)|depth=2, __unnest_placeholder(3d_col,depth=1)|depth=1]"], &unnest_placeholder_columns, ); // Still reference struct_col in original schema but with alias, @@ -743,7 +745,7 @@ mod tests { assert_eq!( inner_projection_exprs, vec![ - col("3d_col").alias("unnest_placeholder(3d_col)"), + col("3d_col").alias("__unnest_placeholder(3d_col)"), col("i64_col") ] ); @@ -794,19 +796,19 @@ mod tests { assert_eq!( transformed_exprs, vec![ - col("unnest_placeholder(struct_col).field1"), - col("unnest_placeholder(struct_col).field2"), + col("__unnest_placeholder(struct_col).field1"), + col("__unnest_placeholder(struct_col).field2"), ] ); column_unnests_eq( - vec!["unnest_placeholder(struct_col)"], + vec!["__unnest_placeholder(struct_col)"], &unnest_placeholder_columns, ); // Still reference struct_col in original schema but with alias, // to avoid colliding with the projection on the column itself if any assert_eq!( inner_projection_exprs, - vec![col("struct_col").alias("unnest_placeholder(struct_col)"),] + vec![col("struct_col").alias("__unnest_placeholder(struct_col)"),] ); // unnest(array_col) + 1 @@ -819,15 +821,15 @@ mod tests { )?; column_unnests_eq( vec![ - "unnest_placeholder(struct_col)", - "unnest_placeholder(array_col)=>[unnest_placeholder(array_col,depth=1)|depth=1]", + "__unnest_placeholder(struct_col)", + "__unnest_placeholder(array_col)=>[__unnest_placeholder(array_col,depth=1)|depth=1]", ], &unnest_placeholder_columns, ); // Only transform the unnest children assert_eq!( transformed_exprs, - vec![col("unnest_placeholder(array_col,depth=1)") + vec![col("__unnest_placeholder(array_col,depth=1)") .alias("UNNEST(array_col)") .add(lit(1i64))] ); @@ -838,8 +840,8 @@ mod tests { assert_eq!( inner_projection_exprs, vec![ - col("struct_col").alias("unnest_placeholder(struct_col)"), - col("array_col").alias("unnest_placeholder(array_col)") + col("struct_col").alias("__unnest_placeholder(struct_col)"), + col("array_col").alias("__unnest_placeholder(array_col)") ] ); @@ -907,7 +909,7 @@ mod tests { assert_eq!( transformed_exprs, vec![unnest( - col("unnest_placeholder(struct_list,depth=1)") + col("__unnest_placeholder(struct_list,depth=1)") .alias("UNNEST(struct_list)") .field("subfield1") )] @@ -915,14 +917,14 @@ mod tests { column_unnests_eq( vec![ - "unnest_placeholder(struct_list)=>[unnest_placeholder(struct_list,depth=1)|depth=1]", + "__unnest_placeholder(struct_list)=>[__unnest_placeholder(struct_list,depth=1)|depth=1]", ], &unnest_placeholder_columns, ); assert_eq!( inner_projection_exprs, - vec![col("struct_list").alias("unnest_placeholder(struct_list)")] + vec![col("struct_list").alias("__unnest_placeholder(struct_list)")] ); // continue rewrite another expr in select @@ -937,7 +939,7 @@ mod tests { assert_eq!( transformed_exprs, vec![unnest( - col("unnest_placeholder(struct_list,depth=1)") + col("__unnest_placeholder(struct_list,depth=1)") .alias("UNNEST(struct_list)") .field("subfield2") )] @@ -947,14 +949,14 @@ mod tests { // because expr1 and expr2 derive from the same unnest result column_unnests_eq( vec![ - "unnest_placeholder(struct_list)=>[unnest_placeholder(struct_list,depth=1)|depth=1]", + "__unnest_placeholder(struct_list)=>[__unnest_placeholder(struct_list,depth=1)|depth=1]", ], &unnest_placeholder_columns, ); assert_eq!( inner_projection_exprs, - vec![col("struct_list").alias("unnest_placeholder(struct_list)")] + vec![col("struct_list").alias("__unnest_placeholder(struct_list)")] ); Ok(()) diff --git a/datafusion/sql/tests/cases/plan_to_sql.rs b/datafusion/sql/tests/cases/plan_to_sql.rs index fcfee29f6ac9..236b59432a5f 100644 --- a/datafusion/sql/tests/cases/plan_to_sql.rs +++ b/datafusion/sql/tests/cases/plan_to_sql.rs @@ -525,6 +525,96 @@ fn roundtrip_statement_with_dialect() -> Result<()> { parser_dialect: Box::new(GenericDialect {}), unparser_dialect: Box::new(SqliteDialect {}), }, + TestStatementWithDialect { + sql: "SELECT * FROM UNNEST([1,2,3])", + expected: r#"SELECT * FROM (SELECT UNNEST([1, 2, 3]) AS "UNNEST(make_array(Int64(1),Int64(2),Int64(3)))")"#, + parser_dialect: Box::new(GenericDialect {}), + unparser_dialect: Box::new(UnparserDefaultDialect {}), + }, + TestStatementWithDialect { + sql: "SELECT * FROM UNNEST([1,2,3]) AS t1 (c1)", + expected: r#"SELECT * FROM (SELECT UNNEST([1, 2, 3]) AS "UNNEST(make_array(Int64(1),Int64(2),Int64(3)))") AS t1 (c1)"#, + parser_dialect: Box::new(GenericDialect {}), + unparser_dialect: Box::new(UnparserDefaultDialect {}), + }, + TestStatementWithDialect { + sql: "SELECT * FROM UNNEST([1,2,3]) AS t1 (c1)", + expected: r#"SELECT * FROM (SELECT UNNEST([1, 2, 3]) AS "UNNEST(make_array(Int64(1),Int64(2),Int64(3)))") AS t1 (c1)"#, + parser_dialect: Box::new(GenericDialect {}), + unparser_dialect: Box::new(UnparserDefaultDialect {}), + }, + TestStatementWithDialect { + sql: "SELECT * FROM UNNEST([1,2,3]), j1", + expected: r#"SELECT * FROM (SELECT UNNEST([1, 2, 3]) AS "UNNEST(make_array(Int64(1),Int64(2),Int64(3)))") CROSS JOIN j1"#, + parser_dialect: Box::new(GenericDialect {}), + unparser_dialect: Box::new(UnparserDefaultDialect {}), + }, + TestStatementWithDialect { + sql: "SELECT * FROM UNNEST([1,2,3]) u(c1) JOIN j1 ON u.c1 = j1.j1_id", + expected: r#"SELECT * FROM (SELECT UNNEST([1, 2, 3]) AS "UNNEST(make_array(Int64(1),Int64(2),Int64(3)))") AS u (c1) JOIN j1 ON (u.c1 = j1.j1_id)"#, + parser_dialect: Box::new(GenericDialect {}), + unparser_dialect: Box::new(UnparserDefaultDialect {}), + }, + TestStatementWithDialect { + sql: "SELECT * FROM UNNEST([1,2,3]) u(c1) UNION ALL SELECT * FROM UNNEST([4,5,6]) u(c1)", + expected: r#"SELECT * FROM (SELECT UNNEST([1, 2, 3]) AS "UNNEST(make_array(Int64(1),Int64(2),Int64(3)))") AS u (c1) UNION ALL SELECT * FROM (SELECT UNNEST([4, 5, 6]) AS "UNNEST(make_array(Int64(4),Int64(5),Int64(6)))") AS u (c1)"#, + parser_dialect: Box::new(GenericDialect {}), + unparser_dialect: Box::new(UnparserDefaultDialect {}), + }, + TestStatementWithDialect { + sql: "SELECT * FROM UNNEST([1,2,3])", + expected: r#"SELECT * FROM UNNEST([1, 2, 3])"#, + parser_dialect: Box::new(GenericDialect {}), + unparser_dialect: Box::new(CustomDialectBuilder::default().with_unnest_as_table_factor(true).build()), + }, + TestStatementWithDialect { + sql: "SELECT * FROM UNNEST([1,2,3]) AS t1 (c1)", + expected: r#"SELECT * FROM UNNEST([1, 2, 3]) AS t1 (c1)"#, + parser_dialect: Box::new(GenericDialect {}), + unparser_dialect: Box::new(CustomDialectBuilder::default().with_unnest_as_table_factor(true).build()), + }, + TestStatementWithDialect { + sql: "SELECT * FROM UNNEST([1,2,3]) AS t1 (c1)", + expected: r#"SELECT * FROM UNNEST([1, 2, 3]) AS t1 (c1)"#, + parser_dialect: Box::new(GenericDialect {}), + unparser_dialect: Box::new(CustomDialectBuilder::default().with_unnest_as_table_factor(true).build()), + }, + TestStatementWithDialect { + sql: "SELECT * FROM UNNEST([1,2,3]), j1", + expected: r#"SELECT * FROM UNNEST([1, 2, 3]) CROSS JOIN j1"#, + parser_dialect: Box::new(GenericDialect {}), + unparser_dialect: Box::new(CustomDialectBuilder::default().with_unnest_as_table_factor(true).build()), + }, + TestStatementWithDialect { + sql: "SELECT * FROM UNNEST([1,2,3]) u(c1) JOIN j1 ON u.c1 = j1.j1_id", + expected: r#"SELECT * FROM UNNEST([1, 2, 3]) AS u (c1) JOIN j1 ON (u.c1 = j1.j1_id)"#, + parser_dialect: Box::new(GenericDialect {}), + unparser_dialect: Box::new(CustomDialectBuilder::default().with_unnest_as_table_factor(true).build()), + }, + TestStatementWithDialect { + sql: "SELECT * FROM UNNEST([1,2,3]) u(c1) UNION ALL SELECT * FROM UNNEST([4,5,6]) u(c1)", + expected: r#"SELECT * FROM UNNEST([1, 2, 3]) AS u (c1) UNION ALL SELECT * FROM UNNEST([4, 5, 6]) AS u (c1)"#, + parser_dialect: Box::new(GenericDialect {}), + unparser_dialect: Box::new(CustomDialectBuilder::default().with_unnest_as_table_factor(true).build()), + }, + TestStatementWithDialect { + sql: "SELECT UNNEST([1,2,3])", + expected: r#"SELECT * FROM UNNEST([1, 2, 3])"#, + parser_dialect: Box::new(GenericDialect {}), + unparser_dialect: Box::new(CustomDialectBuilder::default().with_unnest_as_table_factor(true).build()), + }, + TestStatementWithDialect { + sql: "SELECT UNNEST([1,2,3]) as c1", + expected: r#"SELECT UNNEST([1, 2, 3]) AS c1"#, + parser_dialect: Box::new(GenericDialect {}), + unparser_dialect: Box::new(CustomDialectBuilder::default().with_unnest_as_table_factor(true).build()), + }, + TestStatementWithDialect { + sql: "SELECT UNNEST([1,2,3]), 1", + expected: r#"SELECT UNNEST([1, 2, 3]) AS UNNEST(make_array(Int64(1),Int64(2),Int64(3))), Int64(1)"#, + parser_dialect: Box::new(GenericDialect {}), + unparser_dialect: Box::new(CustomDialectBuilder::default().with_unnest_as_table_factor(true).build()), + }, ]; for query in tests { @@ -535,7 +625,8 @@ fn roundtrip_statement_with_dialect() -> Result<()> { let state = MockSessionState::default() .with_aggregate_function(max_udaf()) .with_aggregate_function(min_udaf()) - .with_expr_planner(Arc::new(CoreFunctionPlanner::default())); + .with_expr_planner(Arc::new(CoreFunctionPlanner::default())) + .with_expr_planner(Arc::new(NestedFunctionPlanner)); let context = MockContextProvider { state }; let sql_to_rel = SqlToRel::new(&context); @@ -571,9 +662,9 @@ fn test_unnest_logical_plan() -> Result<()> { let sql_to_rel = SqlToRel::new(&context); let plan = sql_to_rel.sql_statement_to_plan(statement).unwrap(); let expected = r#" -Projection: unnest_placeholder(unnest_table.struct_col).field1, unnest_placeholder(unnest_table.struct_col).field2, unnest_placeholder(unnest_table.array_col,depth=1) AS UNNEST(unnest_table.array_col), unnest_table.struct_col, unnest_table.array_col - Unnest: lists[unnest_placeholder(unnest_table.array_col)|depth=1] structs[unnest_placeholder(unnest_table.struct_col)] - Projection: unnest_table.struct_col AS unnest_placeholder(unnest_table.struct_col), unnest_table.array_col AS unnest_placeholder(unnest_table.array_col), unnest_table.struct_col, unnest_table.array_col +Projection: __unnest_placeholder(unnest_table.struct_col).field1, __unnest_placeholder(unnest_table.struct_col).field2, __unnest_placeholder(unnest_table.array_col,depth=1) AS UNNEST(unnest_table.array_col), unnest_table.struct_col, unnest_table.array_col + Unnest: lists[__unnest_placeholder(unnest_table.array_col)|depth=1] structs[__unnest_placeholder(unnest_table.struct_col)] + Projection: unnest_table.struct_col AS __unnest_placeholder(unnest_table.struct_col), unnest_table.array_col AS __unnest_placeholder(unnest_table.array_col), unnest_table.struct_col, unnest_table.array_col TableScan: unnest_table"#.trim_start(); assert_eq!(plan.to_string(), expected); diff --git a/datafusion/sqllogictest/test_files/encoding.slt b/datafusion/sqllogictest/test_files/encoding.slt index fc22cc8bf7a7..24efb33f7896 100644 --- a/datafusion/sqllogictest/test_files/encoding.slt +++ b/datafusion/sqllogictest/test_files/encoding.slt @@ -101,4 +101,4 @@ FROM test_utf8view; Andrew QW5kcmV3 416e64726577 X WA 58 Xiangpeng WGlhbmdwZW5n 5869616e6770656e67 Xiangpeng WGlhbmdwZW5n 5869616e6770656e67 Raphael UmFwaGFlbA 5261706861656c R Ug 52 -NULL NULL NULL R Ug 52 \ No newline at end of file +NULL NULL NULL R Ug 52 diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index 62f625119897..49aaa877caa6 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -4058,9 +4058,9 @@ logical_plan 03)----TableScan: join_t1 projection=[t1_id, t1_name] 04)--SubqueryAlias: series 05)----Subquery: -06)------Projection: unnest_placeholder(generate_series(Int64(1),outer_ref(t1.t1_int)),depth=1) AS i -07)--------Unnest: lists[unnest_placeholder(generate_series(Int64(1),outer_ref(t1.t1_int)))|depth=1] structs[] -08)----------Projection: generate_series(Int64(1), CAST(outer_ref(t1.t1_int) AS Int64)) AS unnest_placeholder(generate_series(Int64(1),outer_ref(t1.t1_int))) +06)------Projection: __unnest_placeholder(generate_series(Int64(1),outer_ref(t1.t1_int)),depth=1) AS i +07)--------Unnest: lists[__unnest_placeholder(generate_series(Int64(1),outer_ref(t1.t1_int)))|depth=1] structs[] +08)----------Projection: generate_series(Int64(1), CAST(outer_ref(t1.t1_int) AS Int64)) AS __unnest_placeholder(generate_series(Int64(1),outer_ref(t1.t1_int))) 09)------------EmptyRelation physical_plan_error This feature is not implemented: Physical plan does not support logical expression OuterReferenceColumn(UInt32, Column { relation: Some(Bare { table: "t1" }), name: "t1_int" }) @@ -4081,9 +4081,9 @@ logical_plan 03)----TableScan: join_t1 projection=[t1_id, t1_name] 04)--SubqueryAlias: series 05)----Subquery: -06)------Projection: unnest_placeholder(generate_series(Int64(1),outer_ref(t2.t1_int)),depth=1) AS i -07)--------Unnest: lists[unnest_placeholder(generate_series(Int64(1),outer_ref(t2.t1_int)))|depth=1] structs[] -08)----------Projection: generate_series(Int64(1), CAST(outer_ref(t2.t1_int) AS Int64)) AS unnest_placeholder(generate_series(Int64(1),outer_ref(t2.t1_int))) +06)------Projection: __unnest_placeholder(generate_series(Int64(1),outer_ref(t2.t1_int)),depth=1) AS i +07)--------Unnest: lists[__unnest_placeholder(generate_series(Int64(1),outer_ref(t2.t1_int)))|depth=1] structs[] +08)----------Projection: generate_series(Int64(1), CAST(outer_ref(t2.t1_int) AS Int64)) AS __unnest_placeholder(generate_series(Int64(1),outer_ref(t2.t1_int))) 09)------------EmptyRelation physical_plan_error This feature is not implemented: Physical plan does not support logical expression OuterReferenceColumn(UInt32, Column { relation: Some(Bare { table: "t2" }), name: "t1_int" }) diff --git a/datafusion/sqllogictest/test_files/push_down_filter.slt b/datafusion/sqllogictest/test_files/push_down_filter.slt index 86aa07b04ce1..64cc51b3c4ff 100644 --- a/datafusion/sqllogictest/test_files/push_down_filter.slt +++ b/datafusion/sqllogictest/test_files/push_down_filter.slt @@ -36,9 +36,9 @@ query TT explain select uc2 from (select unnest(column2) as uc2, column1 from v) where column1 = 2; ---- logical_plan -01)Projection: unnest_placeholder(v.column2,depth=1) AS uc2 -02)--Unnest: lists[unnest_placeholder(v.column2)|depth=1] structs[] -03)----Projection: v.column2 AS unnest_placeholder(v.column2), v.column1 +01)Projection: __unnest_placeholder(v.column2,depth=1) AS uc2 +02)--Unnest: lists[__unnest_placeholder(v.column2)|depth=1] structs[] +03)----Projection: v.column2 AS __unnest_placeholder(v.column2), v.column1 04)------Filter: v.column1 = Int64(2) 05)--------TableScan: v projection=[column1, column2] @@ -53,11 +53,11 @@ query TT explain select uc2 from (select unnest(column2) as uc2, column1 from v) where uc2 > 3; ---- logical_plan -01)Projection: unnest_placeholder(v.column2,depth=1) AS uc2 -02)--Filter: unnest_placeholder(v.column2,depth=1) > Int64(3) -03)----Projection: unnest_placeholder(v.column2,depth=1) -04)------Unnest: lists[unnest_placeholder(v.column2)|depth=1] structs[] -05)--------Projection: v.column2 AS unnest_placeholder(v.column2), v.column1 +01)Projection: __unnest_placeholder(v.column2,depth=1) AS uc2 +02)--Filter: __unnest_placeholder(v.column2,depth=1) > Int64(3) +03)----Projection: __unnest_placeholder(v.column2,depth=1) +04)------Unnest: lists[__unnest_placeholder(v.column2)|depth=1] structs[] +05)--------Projection: v.column2 AS __unnest_placeholder(v.column2), v.column1 06)----------TableScan: v projection=[column1, column2] query II @@ -71,10 +71,10 @@ query TT explain select uc2, column1 from (select unnest(column2) as uc2, column1 from v) where uc2 > 3 AND column1 = 2; ---- logical_plan -01)Projection: unnest_placeholder(v.column2,depth=1) AS uc2, v.column1 -02)--Filter: unnest_placeholder(v.column2,depth=1) > Int64(3) -03)----Unnest: lists[unnest_placeholder(v.column2)|depth=1] structs[] -04)------Projection: v.column2 AS unnest_placeholder(v.column2), v.column1 +01)Projection: __unnest_placeholder(v.column2,depth=1) AS uc2, v.column1 +02)--Filter: __unnest_placeholder(v.column2,depth=1) > Int64(3) +03)----Unnest: lists[__unnest_placeholder(v.column2)|depth=1] structs[] +04)------Projection: v.column2 AS __unnest_placeholder(v.column2), v.column1 05)--------Filter: v.column1 = Int64(2) 06)----------TableScan: v projection=[column1, column2] @@ -90,10 +90,10 @@ query TT explain select uc2, column1 from (select unnest(column2) as uc2, column1 from v) where uc2 > 3 OR column1 = 2; ---- logical_plan -01)Projection: unnest_placeholder(v.column2,depth=1) AS uc2, v.column1 -02)--Filter: unnest_placeholder(v.column2,depth=1) > Int64(3) OR v.column1 = Int64(2) -03)----Unnest: lists[unnest_placeholder(v.column2)|depth=1] structs[] -04)------Projection: v.column2 AS unnest_placeholder(v.column2), v.column1 +01)Projection: __unnest_placeholder(v.column2,depth=1) AS uc2, v.column1 +02)--Filter: __unnest_placeholder(v.column2,depth=1) > Int64(3) OR v.column1 = Int64(2) +03)----Unnest: lists[__unnest_placeholder(v.column2)|depth=1] structs[] +04)------Projection: v.column2 AS __unnest_placeholder(v.column2), v.column1 05)--------TableScan: v projection=[column1, column2] statement ok @@ -112,10 +112,10 @@ query TT explain select * from (select column1, unnest(column2) as o from d) where o['a'] = 1; ---- logical_plan -01)Projection: d.column1, unnest_placeholder(d.column2,depth=1) AS o -02)--Filter: get_field(unnest_placeholder(d.column2,depth=1), Utf8("a")) = Int64(1) -03)----Unnest: lists[unnest_placeholder(d.column2)|depth=1] structs[] -04)------Projection: d.column1, d.column2 AS unnest_placeholder(d.column2) +01)Projection: d.column1, __unnest_placeholder(d.column2,depth=1) AS o +02)--Filter: get_field(__unnest_placeholder(d.column2,depth=1), Utf8("a")) = Int64(1) +03)----Unnest: lists[__unnest_placeholder(d.column2)|depth=1] structs[] +04)------Projection: d.column1, d.column2 AS __unnest_placeholder(d.column2) 05)--------TableScan: d projection=[column1, column2] diff --git a/datafusion/sqllogictest/test_files/table_functions.slt b/datafusion/sqllogictest/test_files/table_functions.slt index 12402e0d70c5..79294993dded 100644 --- a/datafusion/sqllogictest/test_files/table_functions.slt +++ b/datafusion/sqllogictest/test_files/table_functions.slt @@ -139,4 +139,4 @@ SELECT generate_series(1, t1.end) FROM generate_series(3, 5) as t1(end) ---- [1, 2, 3, 4, 5] [1, 2, 3, 4] -[1, 2, 3] \ No newline at end of file +[1, 2, 3] diff --git a/datafusion/sqllogictest/test_files/unnest.slt b/datafusion/sqllogictest/test_files/unnest.slt index d409e0902f7e..1c54006bd2a0 100644 --- a/datafusion/sqllogictest/test_files/unnest.slt +++ b/datafusion/sqllogictest/test_files/unnest.slt @@ -594,17 +594,17 @@ query TT explain select unnest(unnest(column3)), column3 from recursive_unnest_table; ---- logical_plan -01)Unnest: lists[] structs[unnest_placeholder(UNNEST(recursive_unnest_table.column3))] -02)--Projection: unnest_placeholder(recursive_unnest_table.column3,depth=1) AS UNNEST(recursive_unnest_table.column3) AS unnest_placeholder(UNNEST(recursive_unnest_table.column3)), recursive_unnest_table.column3 -03)----Unnest: lists[unnest_placeholder(recursive_unnest_table.column3)|depth=1] structs[] -04)------Projection: recursive_unnest_table.column3 AS unnest_placeholder(recursive_unnest_table.column3), recursive_unnest_table.column3 +01)Unnest: lists[] structs[__unnest_placeholder(UNNEST(recursive_unnest_table.column3))] +02)--Projection: __unnest_placeholder(recursive_unnest_table.column3,depth=1) AS UNNEST(recursive_unnest_table.column3) AS __unnest_placeholder(UNNEST(recursive_unnest_table.column3)), recursive_unnest_table.column3 +03)----Unnest: lists[__unnest_placeholder(recursive_unnest_table.column3)|depth=1] structs[] +04)------Projection: recursive_unnest_table.column3 AS __unnest_placeholder(recursive_unnest_table.column3), recursive_unnest_table.column3 05)--------TableScan: recursive_unnest_table projection=[column3] physical_plan 01)UnnestExec 02)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -03)----ProjectionExec: expr=[unnest_placeholder(recursive_unnest_table.column3,depth=1)@0 as unnest_placeholder(UNNEST(recursive_unnest_table.column3)), column3@1 as column3] +03)----ProjectionExec: expr=[__unnest_placeholder(recursive_unnest_table.column3,depth=1)@0 as __unnest_placeholder(UNNEST(recursive_unnest_table.column3)), column3@1 as column3] 04)------UnnestExec -05)--------ProjectionExec: expr=[column3@0 as unnest_placeholder(recursive_unnest_table.column3), column3@0 as column3] +05)--------ProjectionExec: expr=[column3@0 as __unnest_placeholder(recursive_unnest_table.column3), column3@0 as column3] 06)----------MemoryExec: partitions=1, partition_sizes=[1] ## unnest->field_access->unnest->unnest @@ -650,19 +650,19 @@ query TT explain select unnest(unnest(unnest(column3)['c1'])), column3 from recursive_unnest_table; ---- logical_plan -01)Projection: unnest_placeholder(UNNEST(recursive_unnest_table.column3)[c1],depth=2) AS UNNEST(UNNEST(UNNEST(recursive_unnest_table.column3)[c1])), recursive_unnest_table.column3 -02)--Unnest: lists[unnest_placeholder(UNNEST(recursive_unnest_table.column3)[c1])|depth=2] structs[] -03)----Projection: get_field(unnest_placeholder(recursive_unnest_table.column3,depth=1) AS UNNEST(recursive_unnest_table.column3), Utf8("c1")) AS unnest_placeholder(UNNEST(recursive_unnest_table.column3)[c1]), recursive_unnest_table.column3 -04)------Unnest: lists[unnest_placeholder(recursive_unnest_table.column3)|depth=1] structs[] -05)--------Projection: recursive_unnest_table.column3 AS unnest_placeholder(recursive_unnest_table.column3), recursive_unnest_table.column3 +01)Projection: __unnest_placeholder(UNNEST(recursive_unnest_table.column3)[c1],depth=2) AS UNNEST(UNNEST(UNNEST(recursive_unnest_table.column3)[c1])), recursive_unnest_table.column3 +02)--Unnest: lists[__unnest_placeholder(UNNEST(recursive_unnest_table.column3)[c1])|depth=2] structs[] +03)----Projection: get_field(__unnest_placeholder(recursive_unnest_table.column3,depth=1) AS UNNEST(recursive_unnest_table.column3), Utf8("c1")) AS __unnest_placeholder(UNNEST(recursive_unnest_table.column3)[c1]), recursive_unnest_table.column3 +04)------Unnest: lists[__unnest_placeholder(recursive_unnest_table.column3)|depth=1] structs[] +05)--------Projection: recursive_unnest_table.column3 AS __unnest_placeholder(recursive_unnest_table.column3), recursive_unnest_table.column3 06)----------TableScan: recursive_unnest_table projection=[column3] physical_plan -01)ProjectionExec: expr=[unnest_placeholder(UNNEST(recursive_unnest_table.column3)[c1],depth=2)@0 as UNNEST(UNNEST(UNNEST(recursive_unnest_table.column3)[c1])), column3@1 as column3] +01)ProjectionExec: expr=[__unnest_placeholder(UNNEST(recursive_unnest_table.column3)[c1],depth=2)@0 as UNNEST(UNNEST(UNNEST(recursive_unnest_table.column3)[c1])), column3@1 as column3] 02)--UnnestExec -03)----ProjectionExec: expr=[get_field(unnest_placeholder(recursive_unnest_table.column3,depth=1)@0, c1) as unnest_placeholder(UNNEST(recursive_unnest_table.column3)[c1]), column3@1 as column3] +03)----ProjectionExec: expr=[get_field(__unnest_placeholder(recursive_unnest_table.column3,depth=1)@0, c1) as __unnest_placeholder(UNNEST(recursive_unnest_table.column3)[c1]), column3@1 as column3] 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 05)--------UnnestExec -06)----------ProjectionExec: expr=[column3@0 as unnest_placeholder(recursive_unnest_table.column3), column3@0 as column3] +06)----------ProjectionExec: expr=[column3@0 as __unnest_placeholder(recursive_unnest_table.column3), column3@0 as column3] 07)------------MemoryExec: partitions=1, partition_sizes=[1] diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 6c48ac68ab6b..188e2ae0915f 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -5127,4 +5127,3 @@ order by id; statement ok drop table t1; -