From 37428bb2034abc1aba5d4cd52cf8f2885623bada Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?=E5=BC=A0=E6=9E=97=E4=BC=9F?= <lewiszlw520@gmail.com>
Date: Tue, 9 Jul 2024 01:39:33 +0800
Subject: [PATCH 1/9] Enable `clone_on_ref_ptr` clippy lint on physical-plan
 crate (#11241)

* Enable clone_on_ref_ptr clippy lint on physical-plan crate

* fmt

* Update for clippy

* Update new clippy

---------

Co-authored-by: Andrew Lamb <andrew@nerdnetworks.org>
---
 .../physical-plan/src/aggregates/mod.rs       |  94 +++++------
 .../src/aggregates/no_grouping.rs             |   9 +-
 .../src/aggregates/order/partial.rs           |   3 +-
 .../physical-plan/src/aggregates/row_hash.rs  |  12 +-
 .../src/aggregates/topk_stream.rs             |  10 +-
 datafusion/physical-plan/src/analyze.rs       |  16 +-
 .../physical-plan/src/coalesce_batches.rs     |   8 +-
 .../physical-plan/src/coalesce_partitions.rs  |  10 +-
 datafusion/physical-plan/src/common.rs        |   7 +-
 datafusion/physical-plan/src/empty.rs         |  15 +-
 datafusion/physical-plan/src/explain.rs       |   6 +-
 datafusion/physical-plan/src/filter.rs        |  10 +-
 datafusion/physical-plan/src/insert.rs        |  14 +-
 .../physical-plan/src/joins/cross_join.rs     |  14 +-
 .../physical-plan/src/joins/hash_join.rs      | 156 +++++++++++-------
 .../src/joins/nested_loop_join.rs             |  25 +--
 .../src/joins/sort_merge_join.rs              |  69 +++++---
 .../src/joins/stream_join_utils.rs            |  10 +-
 .../src/joins/symmetric_hash_join.rs          |  20 +--
 .../physical-plan/src/joins/test_utils.rs     |  51 ++++--
 datafusion/physical-plan/src/joins/utils.rs   |  12 +-
 datafusion/physical-plan/src/lib.rs           |   8 +-
 datafusion/physical-plan/src/limit.rs         |  12 +-
 datafusion/physical-plan/src/memory.rs        |   9 +-
 .../physical-plan/src/placeholder_row.rs      |  13 +-
 datafusion/physical-plan/src/projection.rs    |  20 ++-
 .../physical-plan/src/recursive_query.rs      |  23 ++-
 .../physical-plan/src/repartition/mod.rs      |  56 ++++---
 datafusion/physical-plan/src/sorts/builder.rs |   6 +-
 datafusion/physical-plan/src/sorts/merge.rs   |   3 +-
 .../physical-plan/src/sorts/partial_sort.rs   |  53 +++---
 datafusion/physical-plan/src/sorts/sort.rs    |  67 +++++---
 .../src/sorts/sort_preserving_merge.rs        |  34 ++--
 datafusion/physical-plan/src/sorts/stream.rs  |   2 +-
 datafusion/physical-plan/src/stream.rs        |  28 ++--
 datafusion/physical-plan/src/streaming.rs     |   6 +-
 datafusion/physical-plan/src/test.rs          |   5 +-
 datafusion/physical-plan/src/test/exec.rs     |  14 +-
 datafusion/physical-plan/src/topk/mod.rs      |   4 +-
 datafusion/physical-plan/src/tree_node.rs     |   2 +-
 datafusion/physical-plan/src/union.rs         |  10 +-
 datafusion/physical-plan/src/unnest.rs        |  36 ++--
 datafusion/physical-plan/src/values.rs        |  10 +-
 .../src/windows/bounded_window_agg_exec.rs    |  56 ++++---
 datafusion/physical-plan/src/windows/mod.rs   |  33 ++--
 .../src/windows/window_agg_exec.rs            |  15 +-
 datafusion/physical-plan/src/work_table.rs    |   8 +-
 47 files changed, 633 insertions(+), 471 deletions(-)

diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs
index 6a0ae202c067..8caf10acf09b 100644
--- a/datafusion/physical-plan/src/aggregates/mod.rs
+++ b/datafusion/physical-plan/src/aggregates/mod.rs
@@ -188,7 +188,7 @@ impl PhysicalGroupBy {
     pub fn input_exprs(&self) -> Vec<Arc<dyn PhysicalExpr>> {
         self.expr
             .iter()
-            .map(|(expr, _alias)| expr.clone())
+            .map(|(expr, _alias)| Arc::clone(expr))
             .collect()
     }
 
@@ -283,9 +283,9 @@ impl AggregateExec {
             group_by: self.group_by.clone(),
             filter_expr: self.filter_expr.clone(),
             limit: self.limit,
-            input: self.input.clone(),
-            schema: self.schema.clone(),
-            input_schema: self.input_schema.clone(),
+            input: Arc::clone(&self.input),
+            schema: Arc::clone(&self.schema),
+            input_schema: Arc::clone(&self.input_schema),
         }
     }
 
@@ -355,7 +355,7 @@ impl AggregateExec {
         let mut new_requirement = indices
             .iter()
             .map(|&idx| PhysicalSortRequirement {
-                expr: groupby_exprs[idx].clone(),
+                expr: Arc::clone(&groupby_exprs[idx]),
                 options: None,
             })
             .collect::<Vec<_>>();
@@ -399,7 +399,7 @@ impl AggregateExec {
 
         let cache = Self::compute_properties(
             &input,
-            schema.clone(),
+            Arc::clone(&schema),
             &projection_mapping,
             &mode,
             &input_order_mode,
@@ -458,7 +458,7 @@ impl AggregateExec {
 
     /// Get the input schema before any aggregates are applied
     pub fn input_schema(&self) -> SchemaRef {
-        self.input_schema.clone()
+        Arc::clone(&self.input_schema)
     }
 
     /// number of rows soft limit of the AggregateExec
@@ -706,9 +706,9 @@ impl ExecutionPlan for AggregateExec {
             self.group_by.clone(),
             self.aggr_expr.clone(),
             self.filter_expr.clone(),
-            children[0].clone(),
-            self.input_schema.clone(),
-            self.schema.clone(),
+            Arc::clone(&children[0]),
+            Arc::clone(&self.input_schema),
+            Arc::clone(&self.schema),
         )?;
         me.limit = self.limit;
 
@@ -1005,7 +1005,7 @@ fn aggregate_expressions(
                 // way order sensitive aggregators can satisfy requirement
                 // themselves.
                 if let Some(ordering_req) = agg.order_bys() {
-                    result.extend(ordering_req.iter().map(|item| item.expr.clone()));
+                    result.extend(ordering_req.iter().map(|item| Arc::clone(&item.expr)));
                 }
                 result
             })
@@ -1165,9 +1165,9 @@ pub(crate) fn evaluate_group_by(
                 .enumerate()
                 .map(|(idx, is_null)| {
                     if *is_null {
-                        null_exprs[idx].clone()
+                        Arc::clone(&null_exprs[idx])
                     } else {
-                        exprs[idx].clone()
+                        Arc::clone(&exprs[idx])
                     }
                 })
                 .collect()
@@ -1235,10 +1235,10 @@ mod tests {
 
         // define data.
         (
-            schema.clone(),
+            Arc::clone(&schema),
             vec![
                 RecordBatch::try_new(
-                    schema.clone(),
+                    Arc::clone(&schema),
                     vec![
                         Arc::new(UInt32Array::from(vec![2, 3, 4, 4])),
                         Arc::new(Float64Array::from(vec![1.0, 2.0, 3.0, 4.0])),
@@ -1270,10 +1270,10 @@ mod tests {
         // the expected result by accident, but merging actually works properly;
         // i.e. it doesn't depend on the data insertion order.
         (
-            schema.clone(),
+            Arc::clone(&schema),
             vec![
                 RecordBatch::try_new(
-                    schema.clone(),
+                    Arc::clone(&schema),
                     vec![
                         Arc::new(UInt32Array::from(vec![2, 3, 4, 4])),
                         Arc::new(Float64Array::from(vec![1.0, 2.0, 3.0, 4.0])),
@@ -1281,7 +1281,7 @@ mod tests {
                 )
                 .unwrap(),
                 RecordBatch::try_new(
-                    schema.clone(),
+                    Arc::clone(&schema),
                     vec![
                         Arc::new(UInt32Array::from(vec![2, 3, 3, 4])),
                         Arc::new(Float64Array::from(vec![0.0, 1.0, 2.0, 3.0])),
@@ -1289,7 +1289,7 @@ mod tests {
                 )
                 .unwrap(),
                 RecordBatch::try_new(
-                    schema.clone(),
+                    Arc::clone(&schema),
                     vec![
                         Arc::new(UInt32Array::from(vec![2, 3, 3, 4])),
                         Arc::new(Float64Array::from(vec![3.0, 4.0, 5.0, 6.0])),
@@ -1369,11 +1369,11 @@ mod tests {
             aggregates.clone(),
             vec![None],
             input,
-            input_schema.clone(),
+            Arc::clone(&input_schema),
         )?);
 
         let result =
-            common::collect(partial_aggregate.execute(0, task_ctx.clone())?).await?;
+            common::collect(partial_aggregate.execute(0, Arc::clone(&task_ctx))?).await?;
 
         let expected = if spill {
             vec![
@@ -1451,7 +1451,7 @@ mod tests {
         )?);
 
         let result =
-            common::collect(merged_aggregate.execute(0, task_ctx.clone())?).await?;
+            common::collect(merged_aggregate.execute(0, Arc::clone(&task_ctx))?).await?;
         let batch = concat_batches(&result[0].schema(), &result)?;
         assert_eq!(batch.num_columns(), 3);
         assert_eq!(batch.num_rows(), 12);
@@ -1519,11 +1519,11 @@ mod tests {
             aggregates.clone(),
             vec![None],
             input,
-            input_schema.clone(),
+            Arc::clone(&input_schema),
         )?);
 
         let result =
-            common::collect(partial_aggregate.execute(0, task_ctx.clone())?).await?;
+            common::collect(partial_aggregate.execute(0, Arc::clone(&task_ctx))?).await?;
 
         let expected = if spill {
             vec![
@@ -1573,7 +1573,7 @@ mod tests {
             // enlarge memory limit to let the final aggregation finish
             new_spill_ctx(2, 2600)
         } else {
-            task_ctx.clone()
+            Arc::clone(&task_ctx)
         };
         let result = common::collect(merged_aggregate.execute(0, task_ctx)?).await?;
         let batch = concat_batches(&result[0].schema(), &result)?;
@@ -1856,11 +1856,11 @@ mod tests {
                 groups,
                 aggregates,
                 vec![None; n_aggr],
-                input.clone(),
-                input_schema.clone(),
+                Arc::clone(&input),
+                Arc::clone(&input_schema),
             )?);
 
-            let stream = partial_aggregate.execute_typed(0, task_ctx.clone())?;
+            let stream = partial_aggregate.execute_typed(0, Arc::clone(&task_ctx))?;
 
             // ensure that we really got the version we wanted
             match version {
@@ -2112,7 +2112,7 @@ mod tests {
                 vec![partition3],
                 vec![partition4],
             ],
-            schema.clone(),
+            Arc::clone(&schema),
             None,
         )?);
         let aggregate_exec = Arc::new(AggregateExec::try_new(
@@ -2121,7 +2121,7 @@ mod tests {
             aggregates.clone(),
             vec![None],
             memory_exec,
-            schema.clone(),
+            Arc::clone(&schema),
         )?);
         let coalesce = if use_coalesce_batches {
             let coalesce = Arc::new(CoalescePartitionsExec::new(aggregate_exec));
@@ -2186,41 +2186,41 @@ mod tests {
         let order_by_exprs = vec![
             None,
             Some(vec![PhysicalSortExpr {
-                expr: col_a.clone(),
+                expr: Arc::clone(col_a),
                 options: options1,
             }]),
             Some(vec![
                 PhysicalSortExpr {
-                    expr: col_a.clone(),
+                    expr: Arc::clone(col_a),
                     options: options1,
                 },
                 PhysicalSortExpr {
-                    expr: col_b.clone(),
+                    expr: Arc::clone(col_b),
                     options: options1,
                 },
                 PhysicalSortExpr {
-                    expr: col_c.clone(),
+                    expr: Arc::clone(col_c),
                     options: options1,
                 },
             ]),
             Some(vec![
                 PhysicalSortExpr {
-                    expr: col_a.clone(),
+                    expr: Arc::clone(col_a),
                     options: options1,
                 },
                 PhysicalSortExpr {
-                    expr: col_b.clone(),
+                    expr: Arc::clone(col_b),
                     options: options1,
                 },
             ]),
         ];
         let common_requirement = vec![
             PhysicalSortExpr {
-                expr: col_a.clone(),
+                expr: Arc::clone(col_a),
                 options: options1,
             },
             PhysicalSortExpr {
-                expr: col_c.clone(),
+                expr: Arc::clone(col_c),
                 options: options1,
             },
         ];
@@ -2228,7 +2228,7 @@ mod tests {
             .into_iter()
             .map(|order_by_expr| {
                 Arc::new(OrderSensitiveArrayAgg::new(
-                    col_a.clone(),
+                    Arc::clone(col_a),
                     "array_agg",
                     DataType::Int32,
                     false,
@@ -2273,12 +2273,11 @@ mod tests {
             groups,
             aggregates.clone(),
             vec![None, None],
-            blocking_exec.clone(),
+            Arc::clone(&blocking_exec) as Arc<dyn ExecutionPlan>,
             schema,
         )?);
-        let new_agg = aggregate_exec
-            .clone()
-            .with_new_children(vec![blocking_exec])?;
+        let new_agg =
+            Arc::clone(&aggregate_exec).with_new_children(vec![blocking_exec])?;
         assert_eq!(new_agg.schema(), aggregate_exec.schema());
         Ok(())
     }
@@ -2340,12 +2339,15 @@ mod tests {
                 let b = Arc::new(Float32Array::from(vec![0.; 8192]));
                 let c = Arc::new(Int32Array::from(vec![1; 8192]));
 
-                RecordBatch::try_new(schema.clone(), vec![a, b, c]).unwrap()
+                RecordBatch::try_new(Arc::clone(&schema), vec![a, b, c]).unwrap()
             })
             .collect();
 
-        let input =
-            Arc::new(MemoryExec::try_new(&[input_batches], schema.clone(), None)?);
+        let input = Arc::new(MemoryExec::try_new(
+            &[input_batches],
+            Arc::clone(&schema),
+            None,
+        )?);
 
         let aggregate_exec = Arc::new(AggregateExec::try_new(
             AggregateMode::Partial,
diff --git a/datafusion/physical-plan/src/aggregates/no_grouping.rs b/datafusion/physical-plan/src/aggregates/no_grouping.rs
index 5ec95bd79942..f85164f7f1e2 100644
--- a/datafusion/physical-plan/src/aggregates/no_grouping.rs
+++ b/datafusion/physical-plan/src/aggregates/no_grouping.rs
@@ -140,8 +140,11 @@ impl AggregateStream {
                         let result =
                             finalize_aggregation(&mut this.accumulators, &this.mode)
                                 .and_then(|columns| {
-                                    RecordBatch::try_new(this.schema.clone(), columns)
-                                        .map_err(Into::into)
+                                    RecordBatch::try_new(
+                                        Arc::clone(&this.schema),
+                                        columns,
+                                    )
+                                    .map_err(Into::into)
                                 })
                                 .record_output(&this.baseline_metrics);
 
@@ -181,7 +184,7 @@ impl Stream for AggregateStream {
 
 impl RecordBatchStream for AggregateStream {
     fn schema(&self) -> SchemaRef {
-        self.schema.clone()
+        Arc::clone(&self.schema)
     }
 }
 
diff --git a/datafusion/physical-plan/src/aggregates/order/partial.rs b/datafusion/physical-plan/src/aggregates/order/partial.rs
index ecd37c913e98..f8fd86ff8b50 100644
--- a/datafusion/physical-plan/src/aggregates/order/partial.rs
+++ b/datafusion/physical-plan/src/aggregates/order/partial.rs
@@ -22,6 +22,7 @@ use datafusion_common::Result;
 use datafusion_execution::memory_pool::proxy::VecAllocExt;
 use datafusion_expr::EmitTo;
 use datafusion_physical_expr::PhysicalSortExpr;
+use std::sync::Arc;
 
 /// Tracks grouping state when the data is ordered by some subset of
 /// the group keys.
@@ -138,7 +139,7 @@ impl GroupOrderingPartial {
         let sort_values: Vec<_> = self
             .order_indices
             .iter()
-            .map(|&idx| group_values[idx].clone())
+            .map(|&idx| Arc::clone(&group_values[idx]))
             .collect();
 
         Ok(self.row_converter.convert_columns(&sort_values)?)
diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs
index 27577e6c8bf8..a1d3378181c2 100644
--- a/datafusion/physical-plan/src/aggregates/row_hash.rs
+++ b/datafusion/physical-plan/src/aggregates/row_hash.rs
@@ -358,7 +358,7 @@ impl GroupedHashAggregateStream {
         let spill_state = SpillState {
             spills: vec![],
             spill_expr,
-            spill_schema: agg_schema.clone(),
+            spill_schema: Arc::clone(&agg_schema),
             is_stream_merging: false,
             merging_aggregate_arguments,
             merging_group_by: PhysicalGroupBy::new_single(agg_group_by.expr.clone()),
@@ -401,7 +401,7 @@ pub(crate) fn create_group_accumulator(
             "Creating GroupsAccumulatorAdapter for {}: {agg_expr:?}",
             agg_expr.name()
         );
-        let agg_expr_captured = agg_expr.clone();
+        let agg_expr_captured = Arc::clone(agg_expr);
         let factory = move || agg_expr_captured.create_accumulator();
         Ok(Box::new(GroupsAccumulatorAdapter::new(factory)))
     }
@@ -515,7 +515,7 @@ impl Stream for GroupedHashAggregateStream {
 
 impl RecordBatchStream for GroupedHashAggregateStream {
     fn schema(&self) -> SchemaRef {
-        self.schema.clone()
+        Arc::clone(&self.schema)
     }
 }
 
@@ -625,7 +625,7 @@ impl GroupedHashAggregateStream {
     /// accumulator states/values specified in emit_to
     fn emit(&mut self, emit_to: EmitTo, spilling: bool) -> Result<RecordBatch> {
         let schema = if spilling {
-            self.spill_state.spill_schema.clone()
+            Arc::clone(&self.spill_state.spill_schema)
         } else {
             self.schema()
         };
@@ -746,13 +746,13 @@ impl GroupedHashAggregateStream {
         let expr = self.spill_state.spill_expr.clone();
         let schema = batch.schema();
         streams.push(Box::pin(RecordBatchStreamAdapter::new(
-            schema.clone(),
+            Arc::clone(&schema),
             futures::stream::once(futures::future::lazy(move |_| {
                 sort_batch(&batch, &expr, None)
             })),
         )));
         for spill in self.spill_state.spills.drain(..) {
-            let stream = read_spill_as_stream(spill, schema.clone(), 2)?;
+            let stream = read_spill_as_stream(spill, Arc::clone(&schema), 2)?;
             streams.push(stream);
         }
         self.spill_state.is_stream_merging = true;
diff --git a/datafusion/physical-plan/src/aggregates/topk_stream.rs b/datafusion/physical-plan/src/aggregates/topk_stream.rs
index 9f25473cb9b4..075d8c5f2883 100644
--- a/datafusion/physical-plan/src/aggregates/topk_stream.rs
+++ b/datafusion/physical-plan/src/aggregates/topk_stream.rs
@@ -84,14 +84,14 @@ impl GroupedTopKAggregateStream {
 
 impl RecordBatchStream for GroupedTopKAggregateStream {
     fn schema(&self) -> SchemaRef {
-        self.schema.clone()
+        Arc::clone(&self.schema)
     }
 }
 
 impl GroupedTopKAggregateStream {
     fn intern(&mut self, ids: ArrayRef, vals: ArrayRef) -> Result<()> {
         let len = ids.len();
-        self.priority_map.set_batch(ids, vals.clone());
+        self.priority_map.set_batch(ids, Arc::clone(&vals));
 
         let has_nulls = vals.null_count() > 0;
         for row_idx in 0..len {
@@ -139,14 +139,14 @@ impl Stream for GroupedTopKAggregateStream {
                         1,
                         "Exactly 1 group value required"
                     );
-                    let group_by_values = group_by_values[0][0].clone();
+                    let group_by_values = Arc::clone(&group_by_values[0][0]);
                     let input_values = evaluate_many(
                         &self.aggregate_arguments,
                         batches.first().unwrap(),
                     )?;
                     assert_eq!(input_values.len(), 1, "Exactly 1 input required");
                     assert_eq!(input_values[0].len(), 1, "Exactly 1 input required");
-                    let input_values = input_values[0][0].clone();
+                    let input_values = Arc::clone(&input_values[0][0]);
 
                     // iterate over each column of group_by values
                     (*self).intern(group_by_values, input_values)?;
@@ -158,7 +158,7 @@ impl Stream for GroupedTopKAggregateStream {
                         return Poll::Ready(None);
                     }
                     let cols = self.priority_map.emit()?;
-                    let batch = RecordBatch::try_new(self.schema.clone(), cols)?;
+                    let batch = RecordBatch::try_new(Arc::clone(&self.schema), cols)?;
                     trace!(
                         "partition {} emit batch with {} rows",
                         self.partition,
diff --git a/datafusion/physical-plan/src/analyze.rs b/datafusion/physical-plan/src/analyze.rs
index 5b859804163b..b4c1e25e6191 100644
--- a/datafusion/physical-plan/src/analyze.rs
+++ b/datafusion/physical-plan/src/analyze.rs
@@ -59,7 +59,7 @@ impl AnalyzeExec {
         input: Arc<dyn ExecutionPlan>,
         schema: SchemaRef,
     ) -> Self {
-        let cache = Self::compute_properties(&input, schema.clone());
+        let cache = Self::compute_properties(&input, Arc::clone(&schema));
         AnalyzeExec {
             verbose,
             show_statistics,
@@ -141,7 +141,7 @@ impl ExecutionPlan for AnalyzeExec {
             self.verbose,
             self.show_statistics,
             children.pop().unwrap(),
-            self.schema.clone(),
+            Arc::clone(&self.schema),
         )))
     }
 
@@ -164,13 +164,17 @@ impl ExecutionPlan for AnalyzeExec {
             RecordBatchReceiverStream::builder(self.schema(), num_input_partitions);
 
         for input_partition in 0..num_input_partitions {
-            builder.run_input(self.input.clone(), input_partition, context.clone());
+            builder.run_input(
+                Arc::clone(&self.input),
+                input_partition,
+                Arc::clone(&context),
+            );
         }
 
         // Create future that computes thefinal output
         let start = Instant::now();
-        let captured_input = self.input.clone();
-        let captured_schema = self.schema.clone();
+        let captured_input = Arc::clone(&self.input);
+        let captured_schema = Arc::clone(&self.schema);
         let verbose = self.verbose;
         let show_statistics = self.show_statistics;
 
@@ -196,7 +200,7 @@ impl ExecutionPlan for AnalyzeExec {
         };
 
         Ok(Box::pin(RecordBatchStreamAdapter::new(
-            self.schema.clone(),
+            Arc::clone(&self.schema),
             futures::stream::once(output),
         )))
     }
diff --git a/datafusion/physical-plan/src/coalesce_batches.rs b/datafusion/physical-plan/src/coalesce_batches.rs
index 804fabff71ac..b9bdfcdee712 100644
--- a/datafusion/physical-plan/src/coalesce_batches.rs
+++ b/datafusion/physical-plan/src/coalesce_batches.rs
@@ -134,7 +134,7 @@ impl ExecutionPlan for CoalesceBatchesExec {
         children: Vec<Arc<dyn ExecutionPlan>>,
     ) -> Result<Arc<dyn ExecutionPlan>> {
         Ok(Arc::new(CoalesceBatchesExec::new(
-            children[0].clone(),
+            Arc::clone(&children[0]),
             self.target_batch_size,
         )))
     }
@@ -272,7 +272,7 @@ impl CoalesceBatchesStream {
 
 impl RecordBatchStream for CoalesceBatchesStream {
     fn schema(&self) -> SchemaRef {
-        self.schema.clone()
+        Arc::clone(&self.schema)
     }
 }
 
@@ -329,7 +329,7 @@ mod tests {
         target_batch_size: usize,
     ) -> Result<Vec<Vec<RecordBatch>>> {
         // create physical plan
-        let exec = MemoryExec::try_new(&input_partitions, schema.clone(), None)?;
+        let exec = MemoryExec::try_new(&input_partitions, Arc::clone(schema), None)?;
         let exec =
             RepartitionExec::try_new(Arc::new(exec), Partitioning::RoundRobinBatch(1))?;
         let exec: Arc<dyn ExecutionPlan> =
@@ -341,7 +341,7 @@ mod tests {
         for i in 0..output_partition_count {
             // execute this *output* partition and collect all batches
             let task_ctx = Arc::new(TaskContext::default());
-            let mut stream = exec.execute(i, task_ctx.clone())?;
+            let mut stream = exec.execute(i, Arc::clone(&task_ctx))?;
             let mut batches = vec![];
             while let Some(result) = stream.next().await {
                 batches.push(result?);
diff --git a/datafusion/physical-plan/src/coalesce_partitions.rs b/datafusion/physical-plan/src/coalesce_partitions.rs
index 93f449f2d39b..ef6afee80307 100644
--- a/datafusion/physical-plan/src/coalesce_partitions.rs
+++ b/datafusion/physical-plan/src/coalesce_partitions.rs
@@ -114,7 +114,9 @@ impl ExecutionPlan for CoalescePartitionsExec {
         self: Arc<Self>,
         children: Vec<Arc<dyn ExecutionPlan>>,
     ) -> Result<Arc<dyn ExecutionPlan>> {
-        Ok(Arc::new(CoalescePartitionsExec::new(children[0].clone())))
+        Ok(Arc::new(CoalescePartitionsExec::new(Arc::clone(
+            &children[0],
+        ))))
     }
 
     fn execute(
@@ -152,7 +154,11 @@ impl ExecutionPlan for CoalescePartitionsExec {
                 // spawn independent tasks whose resulting streams (of batches)
                 // are sent to the channel for consumption.
                 for part_i in 0..input_partitions {
-                    builder.run_input(self.input.clone(), part_i, context.clone());
+                    builder.run_input(
+                        Arc::clone(&self.input),
+                        part_i,
+                        Arc::clone(&context),
+                    );
                 }
 
                 let stream = builder.build();
diff --git a/datafusion/physical-plan/src/common.rs b/datafusion/physical-plan/src/common.rs
index c61e9a05bfa6..bf9d14e73dd8 100644
--- a/datafusion/physical-plan/src/common.rs
+++ b/datafusion/physical-plan/src/common.rs
@@ -618,16 +618,17 @@ mod tests {
             expr: col("f32", &schema).unwrap(),
             options: SortOptions::default(),
         }];
-        let memory_exec = Arc::new(MemoryExec::try_new(&[], schema.clone(), None)?) as _;
+        let memory_exec =
+            Arc::new(MemoryExec::try_new(&[], Arc::clone(&schema), None)?) as _;
         let sort_exec = Arc::new(SortExec::new(sort_expr.clone(), memory_exec))
             as Arc<dyn ExecutionPlan>;
         let memory_exec2 = Arc::new(MemoryExec::try_new(&[], schema, None)?) as _;
         // memory_exec2 doesn't have output ordering
-        let union_exec = UnionExec::new(vec![sort_exec.clone(), memory_exec2]);
+        let union_exec = UnionExec::new(vec![Arc::clone(&sort_exec), memory_exec2]);
         let res = get_meet_of_orderings(union_exec.inputs());
         assert!(res.is_none());
 
-        let union_exec = UnionExec::new(vec![sort_exec.clone(), sort_exec]);
+        let union_exec = UnionExec::new(vec![Arc::clone(&sort_exec), sort_exec]);
         let res = get_meet_of_orderings(union_exec.inputs());
         assert_eq!(res, Some(&sort_expr[..]));
         Ok(())
diff --git a/datafusion/physical-plan/src/empty.rs b/datafusion/physical-plan/src/empty.rs
index 11af0624db15..4bacea48c347 100644
--- a/datafusion/physical-plan/src/empty.rs
+++ b/datafusion/physical-plan/src/empty.rs
@@ -47,7 +47,7 @@ pub struct EmptyExec {
 impl EmptyExec {
     /// Create a new EmptyExec
     pub fn new(schema: SchemaRef) -> Self {
-        let cache = Self::compute_properties(schema.clone(), 1);
+        let cache = Self::compute_properties(Arc::clone(&schema), 1);
         EmptyExec {
             schema,
             partitions: 1,
@@ -142,7 +142,7 @@ impl ExecutionPlan for EmptyExec {
 
         Ok(Box::pin(MemoryStream::try_new(
             self.data()?,
-            self.schema.clone(),
+            Arc::clone(&self.schema),
             None,
         )?))
     }
@@ -170,7 +170,7 @@ mod tests {
         let task_ctx = Arc::new(TaskContext::default());
         let schema = test::aggr_test_schema();
 
-        let empty = EmptyExec::new(schema.clone());
+        let empty = EmptyExec::new(Arc::clone(&schema));
         assert_eq!(empty.schema(), schema);
 
         // we should have no results
@@ -184,9 +184,12 @@ mod tests {
     #[test]
     fn with_new_children() -> Result<()> {
         let schema = test::aggr_test_schema();
-        let empty = Arc::new(EmptyExec::new(schema.clone()));
+        let empty = Arc::new(EmptyExec::new(Arc::clone(&schema)));
 
-        let empty2 = with_new_children_if_necessary(empty.clone(), vec![])?;
+        let empty2 = with_new_children_if_necessary(
+            Arc::clone(&empty) as Arc<dyn ExecutionPlan>,
+            vec![],
+        )?;
         assert_eq!(empty.schema(), empty2.schema());
 
         let too_many_kids = vec![empty2];
@@ -204,7 +207,7 @@ mod tests {
         let empty = EmptyExec::new(schema);
 
         // ask for the wrong partition
-        assert!(empty.execute(1, task_ctx.clone()).is_err());
+        assert!(empty.execute(1, Arc::clone(&task_ctx)).is_err());
         assert!(empty.execute(20, task_ctx).is_err());
         Ok(())
     }
diff --git a/datafusion/physical-plan/src/explain.rs b/datafusion/physical-plan/src/explain.rs
index 4b2edbf2045d..56dc35e8819d 100644
--- a/datafusion/physical-plan/src/explain.rs
+++ b/datafusion/physical-plan/src/explain.rs
@@ -53,7 +53,7 @@ impl ExplainExec {
         stringified_plans: Vec<StringifiedPlan>,
         verbose: bool,
     ) -> Self {
-        let cache = Self::compute_properties(schema.clone());
+        let cache = Self::compute_properties(Arc::clone(&schema));
         ExplainExec {
             schema,
             stringified_plans,
@@ -160,7 +160,7 @@ impl ExecutionPlan for ExplainExec {
         }
 
         let record_batch = RecordBatch::try_new(
-            self.schema.clone(),
+            Arc::clone(&self.schema),
             vec![
                 Arc::new(type_builder.finish()),
                 Arc::new(plan_builder.finish()),
@@ -171,7 +171,7 @@ impl ExecutionPlan for ExplainExec {
             "Before returning RecordBatchStream in ExplainExec::execute for partition {} of context session_id {} and task_id {:?}", partition, context.session_id(), context.task_id());
 
         Ok(Box::pin(RecordBatchStreamAdapter::new(
-            self.schema.clone(),
+            Arc::clone(&self.schema),
             futures::stream::iter(vec![Ok(record_batch)]),
         )))
     }
diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs
index ab7a63e44550..96ec6c0cf34d 100644
--- a/datafusion/physical-plan/src/filter.rs
+++ b/datafusion/physical-plan/src/filter.rs
@@ -78,7 +78,7 @@ impl FilterExec {
                     Self::compute_properties(&input, &predicate, default_selectivity)?;
                 Ok(Self {
                     predicate,
-                    input: input.clone(),
+                    input: Arc::clone(&input),
                     metrics: ExecutionPlanMetricsSet::new(),
                     default_selectivity,
                     cache,
@@ -263,7 +263,7 @@ impl ExecutionPlan for FilterExec {
         self: Arc<Self>,
         mut children: Vec<Arc<dyn ExecutionPlan>>,
     ) -> Result<Arc<dyn ExecutionPlan>> {
-        FilterExec::try_new(self.predicate.clone(), children.swap_remove(0))
+        FilterExec::try_new(Arc::clone(&self.predicate), children.swap_remove(0))
             .and_then(|e| {
                 let selectivity = e.default_selectivity();
                 e.with_default_selectivity(selectivity)
@@ -280,7 +280,7 @@ impl ExecutionPlan for FilterExec {
         let baseline_metrics = BaselineMetrics::new(&self.metrics, partition);
         Ok(Box::pin(FilterExecStream {
             schema: self.input.schema(),
-            predicate: self.predicate.clone(),
+            predicate: Arc::clone(&self.predicate),
             input: self.input.execute(partition, context)?,
             baseline_metrics,
         }))
@@ -405,7 +405,7 @@ impl Stream for FilterExecStream {
 
 impl RecordBatchStream for FilterExecStream {
     fn schema(&self) -> SchemaRef {
-        self.schema.clone()
+        Arc::clone(&self.schema)
     }
 }
 
@@ -1124,7 +1124,7 @@ mod tests {
                 binary(col("c1", &schema)?, Operator::LtEq, lit(4i32), &schema)?,
                 &schema,
             )?,
-            Arc::new(EmptyExec::new(schema.clone())),
+            Arc::new(EmptyExec::new(Arc::clone(&schema))),
         )?;
 
         exec.statistics().unwrap();
diff --git a/datafusion/physical-plan/src/insert.rs b/datafusion/physical-plan/src/insert.rs
index 30c3353d4b71..1c21991d93c5 100644
--- a/datafusion/physical-plan/src/insert.rs
+++ b/datafusion/physical-plan/src/insert.rs
@@ -153,7 +153,7 @@ impl DataSinkExec {
         } else {
             // Check not null constraint on the input stream
             Ok(Box::pin(RecordBatchStreamAdapter::new(
-                self.sink_schema.clone(),
+                Arc::clone(&self.sink_schema),
                 input_stream
                     .map(move |batch| check_not_null_contraits(batch?, &risky_columns)),
             )))
@@ -252,9 +252,9 @@ impl ExecutionPlan for DataSinkExec {
         children: Vec<Arc<dyn ExecutionPlan>>,
     ) -> Result<Arc<dyn ExecutionPlan>> {
         Ok(Arc::new(Self::new(
-            children[0].clone(),
-            self.sink.clone(),
-            self.sink_schema.clone(),
+            Arc::clone(&children[0]),
+            Arc::clone(&self.sink),
+            Arc::clone(&self.sink_schema),
             self.sort_order.clone(),
         )))
     }
@@ -269,10 +269,10 @@ impl ExecutionPlan for DataSinkExec {
         if partition != 0 {
             return internal_err!("DataSinkExec can only be called on partition 0!");
         }
-        let data = self.execute_input_stream(0, context.clone())?;
+        let data = self.execute_input_stream(0, Arc::clone(&context))?;
 
-        let count_schema = self.count_schema.clone();
-        let sink = self.sink.clone();
+        let count_schema = Arc::clone(&self.count_schema);
+        let sink = Arc::clone(&self.sink);
 
         let stream = futures::stream::once(async move {
             sink.write_all(data, &context).await.map(make_count_batch)
diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs
index 92443d06856a..33a9c061bf31 100644
--- a/datafusion/physical-plan/src/joins/cross_join.rs
+++ b/datafusion/physical-plan/src/joins/cross_join.rs
@@ -79,7 +79,7 @@ impl CrossJoinExec {
         };
 
         let schema = Arc::new(Schema::new(all_columns));
-        let cache = Self::compute_properties(&left, &right, schema.clone());
+        let cache = Self::compute_properties(&left, &right, Arc::clone(&schema));
         CrossJoinExec {
             left,
             right,
@@ -220,8 +220,8 @@ impl ExecutionPlan for CrossJoinExec {
         children: Vec<Arc<dyn ExecutionPlan>>,
     ) -> Result<Arc<dyn ExecutionPlan>> {
         Ok(Arc::new(CrossJoinExec::new(
-            children[0].clone(),
-            children[1].clone(),
+            Arc::clone(&children[0]),
+            Arc::clone(&children[1]),
         )))
     }
 
@@ -237,7 +237,7 @@ impl ExecutionPlan for CrossJoinExec {
         partition: usize,
         context: Arc<TaskContext>,
     ) -> Result<SendableRecordBatchStream> {
-        let stream = self.right.execute(partition, context.clone())?;
+        let stream = self.right.execute(partition, Arc::clone(&context))?;
 
         let join_metrics = BuildProbeJoinMetrics::new(partition, &self.metrics);
 
@@ -247,7 +247,7 @@ impl ExecutionPlan for CrossJoinExec {
 
         let left_fut = self.left_fut.once(|| {
             load_left_input(
-                self.left.clone(),
+                Arc::clone(&self.left),
                 context,
                 join_metrics.clone(),
                 reservation,
@@ -255,7 +255,7 @@ impl ExecutionPlan for CrossJoinExec {
         });
 
         Ok(Box::pin(CrossJoinStream {
-            schema: self.schema.clone(),
+            schema: Arc::clone(&self.schema),
             left_fut,
             right: stream,
             left_index: 0,
@@ -337,7 +337,7 @@ struct CrossJoinStream {
 
 impl RecordBatchStream for CrossJoinStream {
     fn schema(&self) -> SchemaRef {
-        self.schema.clone()
+        Arc::clone(&self.schema)
     }
 }
 
diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs
index e91cc84b3bee..2f4ee00da35f 100644
--- a/datafusion/physical-plan/src/joins/hash_join.rs
+++ b/datafusion/physical-plan/src/joins/hash_join.rs
@@ -367,7 +367,7 @@ impl HashJoinExec {
         let cache = Self::compute_properties(
             &left,
             &right,
-            join_schema.clone(),
+            Arc::clone(&join_schema),
             *join_type,
             &on,
             partition_mode,
@@ -464,8 +464,8 @@ impl HashJoinExec {
             None => None,
         };
         Self::try_new(
-            self.left.clone(),
-            self.right.clone(),
+            Arc::clone(&self.left),
+            Arc::clone(&self.right),
             self.on.clone(),
             self.filter.clone(),
             &self.join_type,
@@ -490,7 +490,7 @@ impl HashJoinExec {
             left.equivalence_properties().clone(),
             right.equivalence_properties().clone(),
             &join_type,
-            schema.clone(),
+            Arc::clone(&schema),
             &Self::maintains_input_order(join_type),
             Some(Self::probe_side()),
             on,
@@ -638,8 +638,11 @@ impl ExecutionPlan for HashJoinExec {
                 Distribution::UnspecifiedDistribution,
             ],
             PartitionMode::Partitioned => {
-                let (left_expr, right_expr) =
-                    self.on.iter().map(|(l, r)| (l.clone(), r.clone())).unzip();
+                let (left_expr, right_expr) = self
+                    .on
+                    .iter()
+                    .map(|(l, r)| (Arc::clone(l), Arc::clone(r)))
+                    .unzip();
                 vec![
                     Distribution::HashPartitioned(left_expr),
                     Distribution::HashPartitioned(right_expr),
@@ -681,8 +684,8 @@ impl ExecutionPlan for HashJoinExec {
         children: Vec<Arc<dyn ExecutionPlan>>,
     ) -> Result<Arc<dyn ExecutionPlan>> {
         Ok(Arc::new(HashJoinExec::try_new(
-            children[0].clone(),
-            children[1].clone(),
+            Arc::clone(&children[0]),
+            Arc::clone(&children[1]),
             self.on.clone(),
             self.filter.clone(),
             &self.join_type,
@@ -697,8 +700,16 @@ impl ExecutionPlan for HashJoinExec {
         partition: usize,
         context: Arc<TaskContext>,
     ) -> Result<SendableRecordBatchStream> {
-        let on_left = self.on.iter().map(|on| on.0.clone()).collect::<Vec<_>>();
-        let on_right = self.on.iter().map(|on| on.1.clone()).collect::<Vec<_>>();
+        let on_left = self
+            .on
+            .iter()
+            .map(|on| Arc::clone(&on.0))
+            .collect::<Vec<_>>();
+        let on_right = self
+            .on
+            .iter()
+            .map(|on| Arc::clone(&on.1))
+            .collect::<Vec<_>>();
         let left_partitions = self.left.output_partitioning().partition_count();
         let right_partitions = self.right.output_partitioning().partition_count();
 
@@ -718,9 +729,9 @@ impl ExecutionPlan for HashJoinExec {
                 collect_left_input(
                     None,
                     self.random_state.clone(),
-                    self.left.clone(),
+                    Arc::clone(&self.left),
                     on_left.clone(),
-                    context.clone(),
+                    Arc::clone(&context),
                     join_metrics.clone(),
                     reservation,
                     need_produce_result_in_final(self.join_type),
@@ -735,9 +746,9 @@ impl ExecutionPlan for HashJoinExec {
                 OnceFut::new(collect_left_input(
                     Some(partition),
                     self.random_state.clone(),
-                    self.left.clone(),
+                    Arc::clone(&self.left),
                     on_left.clone(),
-                    context.clone(),
+                    Arc::clone(&context),
                     join_metrics.clone(),
                     reservation,
                     need_produce_result_in_final(self.join_type),
@@ -795,8 +806,8 @@ impl ExecutionPlan for HashJoinExec {
         // There are some special cases though, for example:
         // - `A LEFT JOIN B ON A.col=B.col` with `COUNT_DISTINCT(B.col)=COUNT(B.col)`
         let mut stats = estimate_join_statistics(
-            self.left.clone(),
-            self.right.clone(),
+            Arc::clone(&self.left),
+            Arc::clone(&self.right),
             self.on.clone(),
             &self.join_type,
             &self.join_schema,
@@ -840,7 +851,7 @@ async fn collect_left_input(
     };
 
     // Depending on partition argument load single partition or whole left side in memory
-    let stream = left_input.execute(left_input_partition, context.clone())?;
+    let stream = left_input.execute(left_input_partition, Arc::clone(&context))?;
 
     // This operation performs 2 steps at once:
     // 1. creates a [JoinHashMap] of all batches from the stream
@@ -1117,7 +1128,7 @@ struct HashJoinStream {
 
 impl RecordBatchStream for HashJoinStream {
     fn schema(&self) -> SchemaRef {
-        self.schema.clone()
+        Arc::clone(&self.schema)
     }
 }
 
@@ -1682,8 +1693,10 @@ mod tests {
     ) -> Result<(Vec<String>, Vec<RecordBatch>)> {
         let partition_count = 4;
 
-        let (left_expr, right_expr) =
-            on.iter().map(|(l, r)| (l.clone(), r.clone())).unzip();
+        let (left_expr, right_expr) = on
+            .iter()
+            .map(|(l, r)| (Arc::clone(l), Arc::clone(r)))
+            .unzip();
 
         let left_repartitioned: Arc<dyn ExecutionPlan> = match partition_mode {
             PartitionMode::CollectLeft => Arc::new(CoalescePartitionsExec::new(left)),
@@ -1732,7 +1745,7 @@ mod tests {
 
         let mut batches = vec![];
         for i in 0..partition_count {
-            let stream = join.execute(i, context.clone())?;
+            let stream = join.execute(i, Arc::clone(&context))?;
             let more_batches = common::collect(stream).await?;
             batches.extend(
                 more_batches
@@ -1766,8 +1779,8 @@ mod tests {
         )];
 
         let (columns, batches) = join_collect(
-            left.clone(),
-            right.clone(),
+            Arc::clone(&left),
+            Arc::clone(&right),
             on.clone(),
             &JoinType::Inner,
             false,
@@ -1813,8 +1826,8 @@ mod tests {
         )];
 
         let (columns, batches) = partitioned_join_collect(
-            left.clone(),
-            right.clone(),
+            Arc::clone(&left),
+            Arc::clone(&right),
             on.clone(),
             &JoinType::Inner,
             false,
@@ -2117,7 +2130,7 @@ mod tests {
         assert_eq!(columns, vec!["a1", "b1", "c1", "a2", "b1", "c2"]);
 
         // first part
-        let stream = join.execute(0, task_ctx.clone())?;
+        let stream = join.execute(0, Arc::clone(&task_ctx))?;
         let batches = common::collect(stream).await?;
 
         // expected joined records = 1 (first right batch)
@@ -2140,7 +2153,7 @@ mod tests {
         assert_batches_eq!(expected, &batches);
 
         // second part
-        let stream = join.execute(1, task_ctx.clone())?;
+        let stream = join.execute(1, Arc::clone(&task_ctx))?;
         let batches = common::collect(stream).await?;
 
         // expected joined records = 2 (second right batch)
@@ -2355,8 +2368,8 @@ mod tests {
         )];
 
         let (columns, batches) = join_collect(
-            left.clone(),
-            right.clone(),
+            Arc::clone(&left),
+            Arc::clone(&right),
             on.clone(),
             &JoinType::Left,
             false,
@@ -2399,8 +2412,8 @@ mod tests {
         )];
 
         let (columns, batches) = partitioned_join_collect(
-            left.clone(),
-            right.clone(),
+            Arc::clone(&left),
+            Arc::clone(&right),
             on.clone(),
             &JoinType::Left,
             false,
@@ -2511,8 +2524,8 @@ mod tests {
         );
 
         let join = join_with_filter(
-            left.clone(),
-            right.clone(),
+            Arc::clone(&left),
+            Arc::clone(&right),
             on.clone(),
             filter,
             &JoinType::LeftSemi,
@@ -2522,7 +2535,7 @@ mod tests {
         let columns_header = columns(&join.schema());
         assert_eq!(columns_header.clone(), vec!["a1", "b1", "c1"]);
 
-        let stream = join.execute(0, task_ctx.clone())?;
+        let stream = join.execute(0, Arc::clone(&task_ctx))?;
         let batches = common::collect(stream).await?;
 
         let expected = [
@@ -2635,8 +2648,8 @@ mod tests {
         );
 
         let join = join_with_filter(
-            left.clone(),
-            right.clone(),
+            Arc::clone(&left),
+            Arc::clone(&right),
             on.clone(),
             filter,
             &JoinType::RightSemi,
@@ -2646,7 +2659,7 @@ mod tests {
         let columns = columns(&join.schema());
         assert_eq!(columns, vec!["a2", "b2", "c2"]);
 
-        let stream = join.execute(0, task_ctx.clone())?;
+        let stream = join.execute(0, Arc::clone(&task_ctx))?;
         let batches = common::collect(stream).await?;
 
         let expected = [
@@ -2757,8 +2770,8 @@ mod tests {
         );
 
         let join = join_with_filter(
-            left.clone(),
-            right.clone(),
+            Arc::clone(&left),
+            Arc::clone(&right),
             on.clone(),
             filter,
             &JoinType::LeftAnti,
@@ -2768,7 +2781,7 @@ mod tests {
         let columns_header = columns(&join.schema());
         assert_eq!(columns_header, vec!["a1", "b1", "c1"]);
 
-        let stream = join.execute(0, task_ctx.clone())?;
+        let stream = join.execute(0, Arc::clone(&task_ctx))?;
         let batches = common::collect(stream).await?;
 
         let expected = [
@@ -2886,8 +2899,8 @@ mod tests {
         );
 
         let join = join_with_filter(
-            left.clone(),
-            right.clone(),
+            Arc::clone(&left),
+            Arc::clone(&right),
             on.clone(),
             filter,
             &JoinType::RightAnti,
@@ -2897,7 +2910,7 @@ mod tests {
         let columns_header = columns(&join.schema());
         assert_eq!(columns_header, vec!["a2", "b2", "c2"]);
 
-        let stream = join.execute(0, task_ctx.clone())?;
+        let stream = join.execute(0, Arc::clone(&task_ctx))?;
         let batches = common::collect(stream).await?;
 
         let expected = [
@@ -3087,8 +3100,11 @@ mod tests {
 
         let random_state = RandomState::with_seeds(0, 0, 0, 0);
         let hashes_buff = &mut vec![0; left.num_rows()];
-        let hashes =
-            create_hashes(&[left.columns()[0].clone()], &random_state, hashes_buff)?;
+        let hashes = create_hashes(
+            &[Arc::clone(&left.columns()[0])],
+            &random_state,
+            hashes_buff,
+        )?;
 
         // Create hash collisions (same hashes)
         hashmap_left.insert(hashes[0], (hashes[0], 1), |(h, _)| *h);
@@ -3116,7 +3132,7 @@ mod tests {
             &join_hash_map,
             &left,
             &right,
-            &[key_column.clone()],
+            &[Arc::clone(&key_column)],
             &[key_column],
             false,
             &hashes_buffer,
@@ -3476,13 +3492,13 @@ mod tests {
 
         for (join_type, expected) in test_cases {
             let (_, batches) = join_collect_with_partition_mode(
-                left.clone(),
-                right.clone(),
+                Arc::clone(&left),
+                Arc::clone(&right),
                 on.clone(),
                 &join_type,
                 PartitionMode::CollectLeft,
                 false,
-                task_ctx.clone(),
+                Arc::clone(&task_ctx),
             )
             .await?;
             assert_batches_sorted_eq!(expected, &batches);
@@ -3500,13 +3516,14 @@ mod tests {
 
         let dates: ArrayRef = Arc::new(Date32Array::from(vec![19107, 19108, 19109]));
         let n: ArrayRef = Arc::new(Int32Array::from(vec![1, 2, 3]));
-        let batch = RecordBatch::try_new(schema.clone(), vec![dates, n])?;
-        let left =
-            Arc::new(MemoryExec::try_new(&[vec![batch]], schema.clone(), None).unwrap());
+        let batch = RecordBatch::try_new(Arc::clone(&schema), vec![dates, n])?;
+        let left = Arc::new(
+            MemoryExec::try_new(&[vec![batch]], Arc::clone(&schema), None).unwrap(),
+        );
 
         let dates: ArrayRef = Arc::new(Date32Array::from(vec![19108, 19108, 19109]));
         let n: ArrayRef = Arc::new(Int32Array::from(vec![4, 5, 6]));
-        let batch = RecordBatch::try_new(schema.clone(), vec![dates, n])?;
+        let batch = RecordBatch::try_new(Arc::clone(&schema), vec![dates, n])?;
         let right = Arc::new(MemoryExec::try_new(&[vec![batch]], schema, None).unwrap());
 
         let on = vec![(
@@ -3568,8 +3585,8 @@ mod tests {
 
         for join_type in join_types {
             let join = join(
-                left.clone(),
-                right_input.clone(),
+                Arc::clone(&left),
+                Arc::clone(&right_input) as Arc<dyn ExecutionPlan>,
                 on.clone(),
                 &join_type,
                 false,
@@ -3684,9 +3701,14 @@ mod tests {
             for batch_size in (1..21).rev() {
                 let task_ctx = prepare_task_ctx(batch_size);
 
-                let join =
-                    join(left.clone(), right.clone(), on.clone(), &join_type, false)
-                        .unwrap();
+                let join = join(
+                    Arc::clone(&left),
+                    Arc::clone(&right),
+                    on.clone(),
+                    &join_type,
+                    false,
+                )
+                .unwrap();
 
                 let stream = join.execute(0, task_ctx).unwrap();
                 let batches = common::collect(stream).await.unwrap();
@@ -3759,7 +3781,13 @@ mod tests {
             let task_ctx = TaskContext::default().with_runtime(runtime);
             let task_ctx = Arc::new(task_ctx);
 
-            let join = join(left.clone(), right.clone(), on.clone(), &join_type, false)?;
+            let join = join(
+                Arc::clone(&left),
+                Arc::clone(&right),
+                on.clone(),
+                &join_type,
+                false,
+            )?;
 
             let stream = join.execute(0, task_ctx)?;
             let err = common::collect(stream).await.unwrap_err();
@@ -3832,8 +3860,8 @@ mod tests {
             let task_ctx = Arc::new(task_ctx);
 
             let join = HashJoinExec::try_new(
-                left.clone(),
-                right.clone(),
+                Arc::clone(&left) as Arc<dyn ExecutionPlan>,
+                Arc::clone(&right) as Arc<dyn ExecutionPlan>,
                 on.clone(),
                 None,
                 &join_type,
@@ -3927,12 +3955,12 @@ mod tests {
         )];
 
         let (_, batches_null_eq) = join_collect(
-            left.clone(),
-            right.clone(),
+            Arc::clone(&left),
+            Arc::clone(&right),
             on.clone(),
             &JoinType::Inner,
             true,
-            task_ctx.clone(),
+            Arc::clone(&task_ctx),
         )
         .await?;
 
diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs
index 0a0612f31127..754e55e49650 100644
--- a/datafusion/physical-plan/src/joins/nested_loop_join.rs
+++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs
@@ -173,7 +173,8 @@ impl NestedLoopJoinExec {
         let (schema, column_indices) =
             build_join_schema(&left_schema, &right_schema, join_type);
         let schema = Arc::new(schema);
-        let cache = Self::compute_properties(&left, &right, schema.clone(), *join_type);
+        let cache =
+            Self::compute_properties(&left, &right, Arc::clone(&schema), *join_type);
 
         Ok(NestedLoopJoinExec {
             left,
@@ -287,8 +288,8 @@ impl ExecutionPlan for NestedLoopJoinExec {
         children: Vec<Arc<dyn ExecutionPlan>>,
     ) -> Result<Arc<dyn ExecutionPlan>> {
         Ok(Arc::new(NestedLoopJoinExec::try_new(
-            children[0].clone(),
-            children[1].clone(),
+            Arc::clone(&children[0]),
+            Arc::clone(&children[1]),
             self.filter.clone(),
             &self.join_type,
         )?))
@@ -308,8 +309,8 @@ impl ExecutionPlan for NestedLoopJoinExec {
 
         let inner_table = self.inner_table.once(|| {
             collect_left_input(
-                self.left.clone(),
-                context.clone(),
+                Arc::clone(&self.left),
+                Arc::clone(&context),
                 join_metrics.clone(),
                 load_reservation,
                 need_produce_result_in_final(self.join_type),
@@ -319,7 +320,7 @@ impl ExecutionPlan for NestedLoopJoinExec {
         let outer_table = self.right.execute(partition, context)?;
 
         Ok(Box::pin(NestedLoopJoinStream {
-            schema: self.schema.clone(),
+            schema: Arc::clone(&self.schema),
             filter: self.filter.clone(),
             join_type: self.join_type,
             outer_table,
@@ -336,8 +337,8 @@ impl ExecutionPlan for NestedLoopJoinExec {
 
     fn statistics(&self) -> Result<Statistics> {
         estimate_join_statistics(
-            self.left.clone(),
-            self.right.clone(),
+            Arc::clone(&self.left),
+            Arc::clone(&self.right),
             vec![],
             &self.join_type,
             &self.schema,
@@ -642,7 +643,7 @@ impl Stream for NestedLoopJoinStream {
 
 impl RecordBatchStream for NestedLoopJoinStream {
     fn schema(&self) -> SchemaRef {
-        self.schema.clone()
+        Arc::clone(&self.schema)
     }
 }
 
@@ -752,7 +753,7 @@ mod tests {
         let columns = columns(&nested_loop_join.schema());
         let mut batches = vec![];
         for i in 0..partition_count {
-            let stream = nested_loop_join.execute(i, context.clone())?;
+            let stream = nested_loop_join.execute(i, Arc::clone(&context))?;
             let more_batches = common::collect(stream).await?;
             batches.extend(
                 more_batches
@@ -1037,8 +1038,8 @@ mod tests {
             let task_ctx = Arc::new(task_ctx);
 
             let err = multi_partitioned_join_collect(
-                left.clone(),
-                right.clone(),
+                Arc::clone(&left),
+                Arc::clone(&right),
                 &join_type,
                 Some(filter.clone()),
                 task_ctx,
diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs
index 91b2151d32e7..e9124a72970a 100644
--- a/datafusion/physical-plan/src/joins/sort_merge_join.rs
+++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs
@@ -126,11 +126,11 @@ impl SortMergeJoinExec {
             .zip(sort_options.iter())
             .map(|((l, r), sort_op)| {
                 let left = PhysicalSortExpr {
-                    expr: l.clone(),
+                    expr: Arc::clone(l),
                     options: *sort_op,
                 };
                 let right = PhysicalSortExpr {
-                    expr: r.clone(),
+                    expr: Arc::clone(r),
                     options: *sort_op,
                 };
                 (left, right)
@@ -140,7 +140,7 @@ impl SortMergeJoinExec {
         let schema =
             Arc::new(build_join_schema(&left_schema, &right_schema, &join_type).0);
         let cache =
-            Self::compute_properties(&left, &right, schema.clone(), join_type, &on);
+            Self::compute_properties(&left, &right, Arc::clone(&schema), join_type, &on);
         Ok(Self {
             left,
             right,
@@ -271,8 +271,11 @@ impl ExecutionPlan for SortMergeJoinExec {
     }
 
     fn required_input_distribution(&self) -> Vec<Distribution> {
-        let (left_expr, right_expr) =
-            self.on.iter().map(|(l, r)| (l.clone(), r.clone())).unzip();
+        let (left_expr, right_expr) = self
+            .on
+            .iter()
+            .map(|(l, r)| (Arc::clone(l), Arc::clone(r)))
+            .unzip();
         vec![
             Distribution::HashPartitioned(left_expr),
             Distribution::HashPartitioned(right_expr),
@@ -304,8 +307,8 @@ impl ExecutionPlan for SortMergeJoinExec {
     ) -> Result<Arc<dyn ExecutionPlan>> {
         match &children[..] {
             [left, right] => Ok(Arc::new(SortMergeJoinExec::try_new(
-                left.clone(),
-                right.clone(),
+                Arc::clone(left),
+                Arc::clone(right),
                 self.on.clone(),
                 self.filter.clone(),
                 self.join_type,
@@ -332,14 +335,24 @@ impl ExecutionPlan for SortMergeJoinExec {
         let (on_left, on_right) = self.on.iter().cloned().unzip();
         let (streamed, buffered, on_streamed, on_buffered) =
             if SortMergeJoinExec::probe_side(&self.join_type) == JoinSide::Left {
-                (self.left.clone(), self.right.clone(), on_left, on_right)
+                (
+                    Arc::clone(&self.left),
+                    Arc::clone(&self.right),
+                    on_left,
+                    on_right,
+                )
             } else {
-                (self.right.clone(), self.left.clone(), on_right, on_left)
+                (
+                    Arc::clone(&self.right),
+                    Arc::clone(&self.left),
+                    on_right,
+                    on_left,
+                )
             };
 
         // execute children plans
-        let streamed = streamed.execute(partition, context.clone())?;
-        let buffered = buffered.execute(partition, context.clone())?;
+        let streamed = streamed.execute(partition, Arc::clone(&context))?;
+        let buffered = buffered.execute(partition, Arc::clone(&context))?;
 
         // create output buffer
         let batch_size = context.session_config().batch_size();
@@ -350,7 +363,7 @@ impl ExecutionPlan for SortMergeJoinExec {
 
         // create join stream
         Ok(Box::pin(SMJStream::try_new(
-            self.schema.clone(),
+            Arc::clone(&self.schema),
             self.sort_options.clone(),
             self.null_equals_null,
             streamed,
@@ -374,8 +387,8 @@ impl ExecutionPlan for SortMergeJoinExec {
         // There are some special cases though, for example:
         // - `A LEFT JOIN B ON A.col=B.col` with `COUNT_DISTINCT(B.col)=COUNT(B.col)`
         estimate_join_statistics(
-            self.left.clone(),
-            self.right.clone(),
+            Arc::clone(&self.left),
+            Arc::clone(&self.right),
             self.on.clone(),
             &self.join_type,
             &self.schema,
@@ -657,7 +670,7 @@ struct SMJStream {
 
 impl RecordBatchStream for SMJStream {
     fn schema(&self) -> SchemaRef {
-        self.schema.clone()
+        Arc::clone(&self.schema)
     }
 }
 
@@ -780,7 +793,7 @@ impl SMJStream {
             sort_options,
             null_equals_null,
             schema,
-            streamed_schema: streamed_schema.clone(),
+            streamed_schema: Arc::clone(&streamed_schema),
             buffered_schema,
             streamed,
             buffered,
@@ -1233,7 +1246,7 @@ impl SMJStream {
             };
 
             let output_batch =
-                RecordBatch::try_new(self.schema.clone(), columns.clone())?;
+                RecordBatch::try_new(Arc::clone(&self.schema), columns.clone())?;
 
             // Apply join filter if any
             if !filter_columns.is_empty() {
@@ -1353,8 +1366,10 @@ impl SMJStream {
                         };
 
                         // Push the streamed/buffered batch joined nulls to the output
-                        let null_joined_streamed_batch =
-                            RecordBatch::try_new(self.schema.clone(), columns.clone())?;
+                        let null_joined_streamed_batch = RecordBatch::try_new(
+                            Arc::clone(&self.schema),
+                            columns.clone(),
+                        )?;
                         self.output_record_batches.push(null_joined_streamed_batch);
 
                         // For full join, we also need to output the null joined rows from the buffered side.
@@ -1430,14 +1445,14 @@ fn get_filter_column(
             .column_indices()
             .iter()
             .filter(|col_index| col_index.side == JoinSide::Left)
-            .map(|i| streamed_columns[i.index].clone())
+            .map(|i| Arc::clone(&streamed_columns[i.index]))
             .collect::<Vec<_>>();
 
         let right_columns = f
             .column_indices()
             .iter()
             .filter(|col_index| col_index.side == JoinSide::Right)
-            .map(|i| buffered_columns[i.index].clone())
+            .map(|i| Arc::clone(&buffered_columns[i.index]))
             .collect::<Vec<_>>();
 
         filter_columns.extend(left_columns);
@@ -1476,7 +1491,7 @@ fn produce_buffered_null_batch(
     streamed_columns.extend(buffered_columns);
 
     Ok(Some(RecordBatch::try_new(
-        schema.clone(),
+        Arc::clone(schema),
         streamed_columns,
     )?))
 }
@@ -1927,7 +1942,7 @@ mod tests {
             Field::new(c.0, DataType::Int32, true),
         ]));
         let batch = RecordBatch::try_new(
-            schema.clone(),
+            Arc::clone(&schema),
             vec![
                 Arc::new(Int32Array::from(a.1.clone())),
                 Arc::new(Int32Array::from(b.1.clone())),
@@ -2771,8 +2786,8 @@ mod tests {
             let task_ctx = Arc::new(task_ctx);
 
             let join = join_with_options(
-                left.clone(),
-                right.clone(),
+                Arc::clone(&left),
+                Arc::clone(&right),
                 on.clone(),
                 join_type,
                 sort_options.clone(),
@@ -2849,8 +2864,8 @@ mod tests {
                 .with_runtime(runtime);
             let task_ctx = Arc::new(task_ctx);
             let join = join_with_options(
-                left.clone(),
-                right.clone(),
+                Arc::clone(&left),
+                Arc::clone(&right),
                 on.clone(),
                 join_type,
                 sort_options.clone(),
diff --git a/datafusion/physical-plan/src/joins/stream_join_utils.rs b/datafusion/physical-plan/src/joins/stream_join_utils.rs
index 46d3ac5acf1e..ba9384aef1a6 100644
--- a/datafusion/physical-plan/src/joins/stream_join_utils.rs
+++ b/datafusion/physical-plan/src/joins/stream_join_utils.rs
@@ -270,7 +270,7 @@ pub fn convert_sort_expr_with_filter_schema(
     sort_expr: &PhysicalSortExpr,
 ) -> Result<Option<Arc<dyn PhysicalExpr>>> {
     let column_map = map_origin_col_to_filter_col(filter, schema, side)?;
-    let expr = sort_expr.expr.clone();
+    let expr = Arc::clone(&sort_expr.expr);
     // Get main schema columns:
     let expr_columns = collect_columns(&expr);
     // Calculation is possible with `column_map` since sort exprs belong to a child.
@@ -697,7 +697,7 @@ fn update_sorted_exprs_with_node_indices(
     // Extract filter expressions from the sorted expressions:
     let filter_exprs = sorted_exprs
         .iter()
-        .map(|expr| expr.filter_expr().clone())
+        .map(|expr| Arc::clone(expr.filter_expr()))
         .collect::<Vec<_>>();
 
     // Gather corresponding node indices for the extracted filter expressions from the graph:
@@ -756,7 +756,7 @@ pub fn prepare_sorted_exprs(
 
     // Build the expression interval graph
     let mut graph =
-        ExprIntervalGraph::try_new(filter.expression().clone(), filter.schema())?;
+        ExprIntervalGraph::try_new(Arc::clone(filter.expression()), filter.schema())?;
 
     // Update sorted expressions with node indices
     update_sorted_exprs_with_node_indices(&mut graph, &mut sorted_exprs);
@@ -818,9 +818,9 @@ pub mod tests {
             &intermediate_schema,
         )?;
         let filter_expr = binary(
-            filter_left.clone(),
+            Arc::clone(&filter_left),
             Operator::Gt,
-            filter_right.clone(),
+            Arc::clone(&filter_right),
             &intermediate_schema,
         )?;
         let column_indices = vec![
diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs
index 813f670147bc..c23dc2032c4b 100644
--- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs
+++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs
@@ -233,7 +233,7 @@ impl SymmetricHashJoinExec {
         let random_state = RandomState::with_seeds(0, 0, 0, 0);
         let schema = Arc::new(schema);
         let cache =
-            Self::compute_properties(&left, &right, schema.clone(), *join_type, &on);
+            Self::compute_properties(&left, &right, Arc::clone(&schema), *join_type, &on);
         Ok(SymmetricHashJoinExec {
             left,
             right,
@@ -397,7 +397,7 @@ impl ExecutionPlan for SymmetricHashJoinExec {
                 let (left_expr, right_expr) = self
                     .on
                     .iter()
-                    .map(|(l, r)| (l.clone() as _, r.clone() as _))
+                    .map(|(l, r)| (Arc::clone(l) as _, Arc::clone(r) as _))
                     .unzip();
                 vec![
                     Distribution::HashPartitioned(left_expr),
@@ -430,8 +430,8 @@ impl ExecutionPlan for SymmetricHashJoinExec {
         children: Vec<Arc<dyn ExecutionPlan>>,
     ) -> Result<Arc<dyn ExecutionPlan>> {
         Ok(Arc::new(SymmetricHashJoinExec::try_new(
-            children[0].clone(),
-            children[1].clone(),
+            Arc::clone(&children[0]),
+            Arc::clone(&children[1]),
             self.on.clone(),
             self.filter.clone(),
             &self.join_type,
@@ -489,9 +489,9 @@ impl ExecutionPlan for SymmetricHashJoinExec {
         let right_side_joiner =
             OneSideHashJoiner::new(JoinSide::Right, on_right, self.right.schema());
 
-        let left_stream = self.left.execute(partition, context.clone())?;
+        let left_stream = self.left.execute(partition, Arc::clone(&context))?;
 
-        let right_stream = self.right.execute(partition, context.clone())?;
+        let right_stream = self.right.execute(partition, Arc::clone(&context))?;
 
         let reservation = Arc::new(Mutex::new(
             MemoryConsumer::new(format!("SymmetricHashJoinStream[{partition}]"))
@@ -559,7 +559,7 @@ struct SymmetricHashJoinStream {
 
 impl RecordBatchStream for SymmetricHashJoinStream {
     fn schema(&self) -> SchemaRef {
-        self.schema.clone()
+        Arc::clone(&self.schema)
     }
 }
 
@@ -1634,13 +1634,13 @@ mod tests {
         task_ctx: Arc<TaskContext>,
     ) -> Result<()> {
         let first_batches = partitioned_sym_join_with_filter(
-            left.clone(),
-            right.clone(),
+            Arc::clone(&left),
+            Arc::clone(&right),
             on.clone(),
             filter.clone(),
             &join_type,
             false,
-            task_ctx.clone(),
+            Arc::clone(&task_ctx),
         )
         .await?;
         let second_batches = partitioned_hash_join_with_filter(
diff --git a/datafusion/physical-plan/src/joins/test_utils.rs b/datafusion/physical-plan/src/joins/test_utils.rs
index 7e05ded6f69d..264f297ffb4c 100644
--- a/datafusion/physical-plan/src/joins/test_utils.rs
+++ b/datafusion/physical-plan/src/joins/test_utils.rs
@@ -78,17 +78,23 @@ pub async fn partitioned_sym_join_with_filter(
 ) -> Result<Vec<RecordBatch>> {
     let partition_count = 4;
 
-    let left_expr = on.iter().map(|(l, _)| l.clone() as _).collect::<Vec<_>>();
+    let left_expr = on
+        .iter()
+        .map(|(l, _)| Arc::clone(l) as _)
+        .collect::<Vec<_>>();
 
-    let right_expr = on.iter().map(|(_, r)| r.clone() as _).collect::<Vec<_>>();
+    let right_expr = on
+        .iter()
+        .map(|(_, r)| Arc::clone(r) as _)
+        .collect::<Vec<_>>();
 
     let join = SymmetricHashJoinExec::try_new(
         Arc::new(RepartitionExec::try_new(
-            left.clone(),
+            Arc::clone(&left),
             Partitioning::Hash(left_expr, partition_count),
         )?),
         Arc::new(RepartitionExec::try_new(
-            right.clone(),
+            Arc::clone(&right),
             Partitioning::Hash(right_expr, partition_count),
         )?),
         on,
@@ -102,7 +108,7 @@ pub async fn partitioned_sym_join_with_filter(
 
     let mut batches = vec![];
     for i in 0..partition_count {
-        let stream = join.execute(i, context.clone())?;
+        let stream = join.execute(i, Arc::clone(&context))?;
         let more_batches = common::collect(stream).await?;
         batches.extend(
             more_batches
@@ -127,7 +133,7 @@ pub async fn partitioned_hash_join_with_filter(
     let partition_count = 4;
     let (left_expr, right_expr) = on
         .iter()
-        .map(|(l, r)| (l.clone() as _, r.clone() as _))
+        .map(|(l, r)| (Arc::clone(l) as _, Arc::clone(r) as _))
         .unzip();
 
     let join = Arc::new(HashJoinExec::try_new(
@@ -149,7 +155,7 @@ pub async fn partitioned_hash_join_with_filter(
 
     let mut batches = vec![];
     for i in 0..partition_count {
-        let stream = join.execute(i, context.clone())?;
+        let stream = join.execute(i, Arc::clone(&context))?;
         let more_batches = common::collect(stream).await?;
         batches.extend(
             more_batches
@@ -475,20 +481,29 @@ pub fn build_sides_record_batches(
     ));
 
     let left = RecordBatch::try_from_iter(vec![
-        ("la1", ordered.clone()),
-        ("lb1", cardinality.clone()),
+        ("la1", Arc::clone(&ordered)),
+        ("lb1", Arc::clone(&cardinality) as ArrayRef),
         ("lc1", cardinality_key_left),
-        ("lt1", time.clone()),
-        ("la2", ordered.clone()),
-        ("la1_des", ordered_des.clone()),
-        ("l_asc_null_first", ordered_asc_null_first.clone()),
-        ("l_asc_null_last", ordered_asc_null_last.clone()),
-        ("l_desc_null_first", ordered_desc_null_first.clone()),
-        ("li1", interval_time.clone()),
-        ("l_float", float_asc.clone()),
+        ("lt1", Arc::clone(&time) as ArrayRef),
+        ("la2", Arc::clone(&ordered)),
+        ("la1_des", Arc::clone(&ordered_des) as ArrayRef),
+        (
+            "l_asc_null_first",
+            Arc::clone(&ordered_asc_null_first) as ArrayRef,
+        ),
+        (
+            "l_asc_null_last",
+            Arc::clone(&ordered_asc_null_last) as ArrayRef,
+        ),
+        (
+            "l_desc_null_first",
+            Arc::clone(&ordered_desc_null_first) as ArrayRef,
+        ),
+        ("li1", Arc::clone(&interval_time)),
+        ("l_float", Arc::clone(&float_asc) as ArrayRef),
     ])?;
     let right = RecordBatch::try_from_iter(vec![
-        ("ra1", ordered.clone()),
+        ("ra1", Arc::clone(&ordered)),
         ("rb1", cardinality),
         ("rc1", cardinality_key_right),
         ("rt1", time),
diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs
index f00bbfbfbb1f..e3ec242ce8de 100644
--- a/datafusion/physical-plan/src/joins/utils.rs
+++ b/datafusion/physical-plan/src/joins/utils.rs
@@ -440,7 +440,7 @@ pub fn adjust_right_output_partitioning(
         Partitioning::Hash(exprs, size) => {
             let new_exprs = exprs
                 .iter()
-                .map(|expr| add_offset_to_expr(expr.clone(), left_columns_len))
+                .map(|expr| add_offset_to_expr(Arc::clone(expr), left_columns_len))
                 .collect();
             Partitioning::Hash(new_exprs, *size)
         }
@@ -456,12 +456,10 @@ fn replace_on_columns_of_right_ordering(
 ) -> Result<()> {
     for (left_col, right_col) in on_columns {
         for item in right_ordering.iter_mut() {
-            let new_expr = item
-                .expr
-                .clone()
+            let new_expr = Arc::clone(&item.expr)
                 .transform(|e| {
                     if e.eq(right_col) {
-                        Ok(Transformed::yes(left_col.clone()))
+                        Ok(Transformed::yes(Arc::clone(left_col)))
                     } else {
                         Ok(Transformed::no(e))
                     }
@@ -484,7 +482,7 @@ fn offset_ordering(
         JoinType::Inner | JoinType::Left | JoinType::Full | JoinType::Right => ordering
             .iter()
             .map(|sort_expr| PhysicalSortExpr {
-                expr: add_offset_to_expr(sort_expr.expr.clone(), offset),
+                expr: add_offset_to_expr(Arc::clone(&sort_expr.expr), offset),
                 options: sort_expr.options,
             })
             .collect(),
@@ -1122,7 +1120,7 @@ impl<T: 'static> OnceFut<T> {
             OnceFutState::Ready(r) => Poll::Ready(
                 r.as_ref()
                     .map(|r| r.as_ref())
-                    .map_err(|e| DataFusionError::External(Box::new(e.clone()))),
+                    .map_err(|e| DataFusionError::External(Box::new(Arc::clone(e)))),
             ),
         }
     }
diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs
index aef5b307968c..f3a709ff7670 100644
--- a/datafusion/physical-plan/src/lib.rs
+++ b/datafusion/physical-plan/src/lib.rs
@@ -14,6 +14,8 @@
 // KIND, either express or implied.  See the License for the
 // specific language governing permissions and limitations
 // under the License.
+// Make cheap clones clear: https://github.com/apache/datafusion/issues/11143
+#![deny(clippy::clone_on_ref_ptr)]
 
 //! Traits for physical query plan, supporting parallel execution for partitioned relations.
 
@@ -155,7 +157,7 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync {
 
     /// Get the schema for this execution plan
     fn schema(&self) -> SchemaRef {
-        self.properties().schema().clone()
+        Arc::clone(self.properties().schema())
     }
 
     /// Return properties of the output of the `ExecutionPlan`, such as output
@@ -736,7 +738,7 @@ pub fn execute_stream(
         1 => plan.execute(0, context),
         _ => {
             // merge into a single partition
-            let plan = CoalescePartitionsExec::new(plan.clone());
+            let plan = CoalescePartitionsExec::new(Arc::clone(&plan));
             // CoalescePartitionsExec must produce a single partition
             assert_eq!(1, plan.properties().output_partitioning().partition_count());
             plan.execute(0, context)
@@ -798,7 +800,7 @@ pub fn execute_stream_partitioned(
     let num_partitions = plan.output_partitioning().partition_count();
     let mut streams = Vec::with_capacity(num_partitions);
     for i in 0..num_partitions {
-        streams.push(plan.execute(i, context.clone())?);
+        streams.push(plan.execute(i, Arc::clone(&context))?);
     }
     Ok(streams)
 }
diff --git a/datafusion/physical-plan/src/limit.rs b/datafusion/physical-plan/src/limit.rs
index 4c6d1b3674d5..9c77a3d05cc2 100644
--- a/datafusion/physical-plan/src/limit.rs
+++ b/datafusion/physical-plan/src/limit.rs
@@ -145,7 +145,7 @@ impl ExecutionPlan for GlobalLimitExec {
         children: Vec<Arc<dyn ExecutionPlan>>,
     ) -> Result<Arc<dyn ExecutionPlan>> {
         Ok(Arc::new(GlobalLimitExec::new(
-            children[0].clone(),
+            Arc::clone(&children[0]),
             self.skip,
             self.fetch,
         )))
@@ -352,7 +352,7 @@ impl ExecutionPlan for LocalLimitExec {
     ) -> Result<Arc<dyn ExecutionPlan>> {
         match children.len() {
             1 => Ok(Arc::new(LocalLimitExec::new(
-                children[0].clone(),
+                Arc::clone(&children[0]),
                 self.fetch,
             ))),
             _ => internal_err!("LocalLimitExec wrong number of children"),
@@ -551,7 +551,7 @@ impl Stream for LimitStream {
 impl RecordBatchStream for LimitStream {
     /// Get the schema
     fn schema(&self) -> SchemaRef {
-        self.schema.clone()
+        Arc::clone(&self.schema)
     }
 }
 
@@ -864,11 +864,11 @@ mod tests {
         // Adding a "GROUP BY i" changes the input stats from Exact to Inexact.
         let agg = AggregateExec::try_new(
             AggregateMode::Final,
-            build_group_by(&csv.schema().clone(), vec!["i".to_string()]),
+            build_group_by(&csv.schema(), vec!["i".to_string()]),
             vec![],
             vec![],
-            csv.clone(),
-            csv.schema().clone(),
+            Arc::clone(&csv),
+            Arc::clone(&csv.schema()),
         )?;
         let agg_exec: Arc<dyn ExecutionPlan> = Arc::new(agg);
 
diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs
index 39ae8d551f4b..6b2c78902eae 100644
--- a/datafusion/physical-plan/src/memory.rs
+++ b/datafusion/physical-plan/src/memory.rs
@@ -140,7 +140,7 @@ impl ExecutionPlan for MemoryExec {
     ) -> Result<SendableRecordBatchStream> {
         Ok(Box::pin(MemoryStream::try_new(
             self.partitions[partition].clone(),
-            self.projected_schema.clone(),
+            Arc::clone(&self.projected_schema),
             self.projection.clone(),
         )?))
     }
@@ -164,7 +164,8 @@ impl MemoryExec {
         projection: Option<Vec<usize>>,
     ) -> Result<Self> {
         let projected_schema = project_schema(&schema, projection.as_ref())?;
-        let cache = Self::compute_properties(projected_schema.clone(), &[], partitions);
+        let cache =
+            Self::compute_properties(Arc::clone(&projected_schema), &[], partitions);
         Ok(Self {
             partitions: partitions.to_vec(),
             schema,
@@ -219,7 +220,7 @@ impl MemoryExec {
     }
 
     pub fn original_schema(&self) -> SchemaRef {
-        self.schema.clone()
+        Arc::clone(&self.schema)
     }
 
     /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc.
@@ -305,7 +306,7 @@ impl Stream for MemoryStream {
 impl RecordBatchStream for MemoryStream {
     /// Get the schema
     fn schema(&self) -> SchemaRef {
-        self.schema.clone()
+        Arc::clone(&self.schema)
     }
 }
 
diff --git a/datafusion/physical-plan/src/placeholder_row.rs b/datafusion/physical-plan/src/placeholder_row.rs
index 3b10cc0ac435..272211d5056e 100644
--- a/datafusion/physical-plan/src/placeholder_row.rs
+++ b/datafusion/physical-plan/src/placeholder_row.rs
@@ -50,7 +50,7 @@ impl PlaceholderRowExec {
     /// Create a new PlaceholderRowExec
     pub fn new(schema: SchemaRef) -> Self {
         let partitions = 1;
-        let cache = Self::compute_properties(schema.clone(), partitions);
+        let cache = Self::compute_properties(Arc::clone(&schema), partitions);
         PlaceholderRowExec {
             schema,
             partitions,
@@ -160,7 +160,7 @@ impl ExecutionPlan for PlaceholderRowExec {
 
         Ok(Box::pin(MemoryStream::try_new(
             self.data()?,
-            self.schema.clone(),
+            Arc::clone(&self.schema),
             None,
         )?))
     }
@@ -188,7 +188,10 @@ mod tests {
 
         let placeholder = Arc::new(PlaceholderRowExec::new(schema));
 
-        let placeholder_2 = with_new_children_if_necessary(placeholder.clone(), vec![])?;
+        let placeholder_2 = with_new_children_if_necessary(
+            Arc::clone(&placeholder) as Arc<dyn ExecutionPlan>,
+            vec![],
+        )?;
         assert_eq!(placeholder.schema(), placeholder_2.schema());
 
         let too_many_kids = vec![placeholder_2];
@@ -206,7 +209,7 @@ mod tests {
         let placeholder = PlaceholderRowExec::new(schema);
 
         // ask for the wrong partition
-        assert!(placeholder.execute(1, task_ctx.clone()).is_err());
+        assert!(placeholder.execute(1, Arc::clone(&task_ctx)).is_err());
         assert!(placeholder.execute(20, task_ctx).is_err());
         Ok(())
     }
@@ -234,7 +237,7 @@ mod tests {
         let placeholder = PlaceholderRowExec::new(schema).with_partitions(partitions);
 
         for n in 0..partitions {
-            let iter = placeholder.execute(n, task_ctx.clone())?;
+            let iter = placeholder.execute(n, Arc::clone(&task_ctx))?;
             let batches = common::collect(iter).await?;
 
             // should have one item
diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs
index 8341549340dd..9efa0422ec75 100644
--- a/datafusion/physical-plan/src/projection.rs
+++ b/datafusion/physical-plan/src/projection.rs
@@ -94,7 +94,7 @@ impl ProjectionExec {
         // construct a map from the input expressions to the output expression of the Projection
         let projection_mapping = ProjectionMapping::try_new(&expr, &input_schema)?;
         let cache =
-            Self::compute_properties(&input, &projection_mapping, schema.clone())?;
+            Self::compute_properties(&input, &projection_mapping, Arc::clone(&schema))?;
         Ok(Self {
             expr,
             schema,
@@ -227,8 +227,8 @@ impl ExecutionPlan for ProjectionExec {
     ) -> Result<SendableRecordBatchStream> {
         trace!("Start ProjectionExec::execute for partition {} of context session_id {} and task_id {:?}", partition, context.session_id(), context.task_id());
         Ok(Box::pin(ProjectionStream {
-            schema: self.schema.clone(),
-            expr: self.expr.iter().map(|x| x.0.clone()).collect(),
+            schema: Arc::clone(&self.schema),
+            expr: self.expr.iter().map(|x| Arc::clone(&x.0)).collect(),
             input: self.input.execute(partition, context)?,
             baseline_metrics: BaselineMetrics::new(&self.metrics, partition),
         }))
@@ -242,7 +242,7 @@ impl ExecutionPlan for ProjectionExec {
         Ok(stats_projection(
             self.input.statistics()?,
             self.expr.iter().map(|(e, _)| Arc::clone(e)),
-            self.schema.clone(),
+            Arc::clone(&self.schema),
         ))
     }
 }
@@ -311,10 +311,10 @@ impl ProjectionStream {
         if arrays.is_empty() {
             let options =
                 RecordBatchOptions::new().with_row_count(Some(batch.num_rows()));
-            RecordBatch::try_new_with_options(self.schema.clone(), arrays, &options)
+            RecordBatch::try_new_with_options(Arc::clone(&self.schema), arrays, &options)
                 .map_err(Into::into)
         } else {
-            RecordBatch::try_new(self.schema.clone(), arrays).map_err(Into::into)
+            RecordBatch::try_new(Arc::clone(&self.schema), arrays).map_err(Into::into)
         }
     }
 }
@@ -351,7 +351,7 @@ impl Stream for ProjectionStream {
 impl RecordBatchStream for ProjectionStream {
     /// Get the schema
     fn schema(&self) -> SchemaRef {
-        self.schema.clone()
+        Arc::clone(&self.schema)
     }
 }
 
@@ -370,10 +370,12 @@ mod tests {
         let task_ctx = Arc::new(TaskContext::default());
 
         let exec = test::scan_partitioned(1);
-        let expected = collect(exec.execute(0, task_ctx.clone())?).await.unwrap();
+        let expected = collect(exec.execute(0, Arc::clone(&task_ctx))?)
+            .await
+            .unwrap();
 
         let projection = ProjectionExec::try_new(vec![], exec)?;
-        let stream = projection.execute(0, task_ctx.clone())?;
+        let stream = projection.execute(0, Arc::clone(&task_ctx))?;
         let output = collect(stream).await.unwrap();
         assert_eq!(output.len(), expected.len());
 
diff --git a/datafusion/physical-plan/src/recursive_query.rs b/datafusion/physical-plan/src/recursive_query.rs
index 9a0b66caba31..bd9303f97db0 100644
--- a/datafusion/physical-plan/src/recursive_query.rs
+++ b/datafusion/physical-plan/src/recursive_query.rs
@@ -82,7 +82,7 @@ impl RecursiveQueryExec {
         // Each recursive query needs its own work table
         let work_table = Arc::new(WorkTable::new());
         // Use the same work table for both the WorkTableExec and the recursive term
-        let recursive_term = assign_work_table(recursive_term, work_table.clone())?;
+        let recursive_term = assign_work_table(recursive_term, Arc::clone(&work_table))?;
         let cache = Self::compute_properties(static_term.schema());
         Ok(RecursiveQueryExec {
             name,
@@ -147,8 +147,8 @@ impl ExecutionPlan for RecursiveQueryExec {
     ) -> Result<Arc<dyn ExecutionPlan>> {
         RecursiveQueryExec::try_new(
             self.name.clone(),
-            children[0].clone(),
-            children[1].clone(),
+            Arc::clone(&children[0]),
+            Arc::clone(&children[1]),
             self.is_distinct,
         )
         .map(|e| Arc::new(e) as _)
@@ -167,12 +167,12 @@ impl ExecutionPlan for RecursiveQueryExec {
             )));
         }
 
-        let static_stream = self.static_term.execute(partition, context.clone())?;
+        let static_stream = self.static_term.execute(partition, Arc::clone(&context))?;
         let baseline_metrics = BaselineMetrics::new(&self.metrics, partition);
         Ok(Box::pin(RecursiveQueryStream::new(
             context,
-            self.work_table.clone(),
-            self.recursive_term.clone(),
+            Arc::clone(&self.work_table),
+            Arc::clone(&self.recursive_term),
             static_stream,
             baseline_metrics,
         )))
@@ -313,9 +313,9 @@ impl RecursiveQueryStream {
         // Downstream plans should not expect any partitioning.
         let partition = 0;
 
-        let recursive_plan = reset_plan_states(self.recursive_term.clone())?;
+        let recursive_plan = reset_plan_states(Arc::clone(&self.recursive_term))?;
         self.recursive_stream =
-            Some(recursive_plan.execute(partition, self.task_context.clone())?);
+            Some(recursive_plan.execute(partition, Arc::clone(&self.task_context))?);
         self.poll_next(cx)
     }
 }
@@ -334,7 +334,7 @@ fn assign_work_table(
             } else {
                 work_table_refs += 1;
                 Ok(Transformed::yes(Arc::new(
-                    exec.with_work_table(work_table.clone()),
+                    exec.with_work_table(Arc::clone(&work_table)),
                 )))
             }
         } else if plan.as_any().is::<RecursiveQueryExec>() {
@@ -358,8 +358,7 @@ fn reset_plan_states(plan: Arc<dyn ExecutionPlan>) -> Result<Arc<dyn ExecutionPl
         if plan.as_any().is::<WorkTableExec>() {
             Ok(Transformed::no(plan))
         } else {
-            let new_plan = plan
-                .clone()
+            let new_plan = Arc::clone(&plan)
                 .with_new_children(plan.children().into_iter().cloned().collect())?;
             Ok(Transformed::yes(new_plan))
         }
@@ -407,7 +406,7 @@ impl Stream for RecursiveQueryStream {
 impl RecordBatchStream for RecursiveQueryStream {
     /// Get the schema
     fn schema(&self) -> SchemaRef {
-        self.schema.clone()
+        Arc::clone(&self.schema)
     }
 }
 
diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs
index d9e16c98eee8..3d4d3058393e 100644
--- a/datafusion/physical-plan/src/repartition/mod.rs
+++ b/datafusion/physical-plan/src/repartition/mod.rs
@@ -133,12 +133,12 @@ impl RepartitionExecState {
             let r_metrics = RepartitionMetrics::new(i, num_output_partitions, &metrics);
 
             let input_task = SpawnedTask::spawn(RepartitionExec::pull_from_input(
-                input.clone(),
+                Arc::clone(&input),
                 i,
                 txs.clone(),
                 partitioning.clone(),
                 r_metrics,
-                context.clone(),
+                Arc::clone(&context),
             ));
 
             // In a separate task, wait for each input to be done
@@ -616,7 +616,7 @@ impl ExecutionPlan for RepartitionExec {
                             schema: Arc::clone(&schema_captured),
                             receiver,
                             drop_helper: Arc::clone(&abort_helper),
-                            reservation: reservation.clone(),
+                            reservation: Arc::clone(&reservation),
                         }) as SendableRecordBatchStream
                     })
                     .collect::<Vec<_>>();
@@ -866,7 +866,7 @@ impl RepartitionExec {
 
                 for (_, tx) in txs {
                     // wrap it because need to send error to all output partitions
-                    let err = Err(DataFusionError::External(Box::new(e.clone())));
+                    let err = Err(DataFusionError::External(Box::new(Arc::clone(&e))));
                     tx.send(Some(err)).await.ok();
                 }
             }
@@ -945,7 +945,7 @@ impl Stream for RepartitionStream {
 impl RecordBatchStream for RepartitionStream {
     /// Get the schema
     fn schema(&self) -> SchemaRef {
-        self.schema.clone()
+        Arc::clone(&self.schema)
     }
 }
 
@@ -995,7 +995,7 @@ impl Stream for PerPartitionStream {
 impl RecordBatchStream for PerPartitionStream {
     /// Get the schema
     fn schema(&self) -> SchemaRef {
-        self.schema.clone()
+        Arc::clone(&self.schema)
     }
 }
 
@@ -1117,14 +1117,14 @@ mod tests {
     ) -> Result<Vec<Vec<RecordBatch>>> {
         let task_ctx = Arc::new(TaskContext::default());
         // create physical plan
-        let exec = MemoryExec::try_new(&input_partitions, schema.clone(), None)?;
+        let exec = MemoryExec::try_new(&input_partitions, Arc::clone(schema), None)?;
         let exec = RepartitionExec::try_new(Arc::new(exec), partitioning)?;
 
         // execute and collect results
         let mut output_partitions = vec![];
         for i in 0..exec.partitioning.partition_count() {
             // execute this *output* partition and collect all batches
-            let mut stream = exec.execute(i, task_ctx.clone())?;
+            let mut stream = exec.execute(i, Arc::clone(&task_ctx))?;
             let mut batches = vec![];
             while let Some(result) = stream.next().await {
                 batches.push(result?);
@@ -1301,10 +1301,14 @@ mod tests {
         let input = Arc::new(make_barrier_exec());
 
         // partition into two output streams
-        let exec = RepartitionExec::try_new(input.clone(), partitioning).unwrap();
+        let exec = RepartitionExec::try_new(
+            Arc::clone(&input) as Arc<dyn ExecutionPlan>,
+            partitioning,
+        )
+        .unwrap();
 
-        let output_stream0 = exec.execute(0, task_ctx.clone()).unwrap();
-        let output_stream1 = exec.execute(1, task_ctx.clone()).unwrap();
+        let output_stream0 = exec.execute(0, Arc::clone(&task_ctx)).unwrap();
+        let output_stream1 = exec.execute(1, Arc::clone(&task_ctx)).unwrap();
 
         // now, purposely drop output stream 0
         // *before* any outputs are produced
@@ -1349,8 +1353,12 @@ mod tests {
 
         // We first collect the results without droping the output stream.
         let input = Arc::new(make_barrier_exec());
-        let exec = RepartitionExec::try_new(input.clone(), partitioning.clone()).unwrap();
-        let output_stream1 = exec.execute(1, task_ctx.clone()).unwrap();
+        let exec = RepartitionExec::try_new(
+            Arc::clone(&input) as Arc<dyn ExecutionPlan>,
+            partitioning.clone(),
+        )
+        .unwrap();
+        let output_stream1 = exec.execute(1, Arc::clone(&task_ctx)).unwrap();
         let mut background_task = JoinSet::new();
         background_task.spawn(async move {
             input.wait().await;
@@ -1370,9 +1378,13 @@ mod tests {
 
         // Now do the same but dropping the stream before waiting for the barrier
         let input = Arc::new(make_barrier_exec());
-        let exec = RepartitionExec::try_new(input.clone(), partitioning).unwrap();
-        let output_stream0 = exec.execute(0, task_ctx.clone()).unwrap();
-        let output_stream1 = exec.execute(1, task_ctx.clone()).unwrap();
+        let exec = RepartitionExec::try_new(
+            Arc::clone(&input) as Arc<dyn ExecutionPlan>,
+            partitioning,
+        )
+        .unwrap();
+        let output_stream0 = exec.execute(0, Arc::clone(&task_ctx)).unwrap();
+        let output_stream1 = exec.execute(1, Arc::clone(&task_ctx)).unwrap();
         // now, purposely drop output stream 0
         // *before* any outputs are produced
         std::mem::drop(output_stream0);
@@ -1471,9 +1483,9 @@ mod tests {
         let schema = batch.schema();
         let input = MockExec::new(vec![Ok(batch)], schema);
         let exec = RepartitionExec::try_new(Arc::new(input), partitioning).unwrap();
-        let output_stream0 = exec.execute(0, task_ctx.clone()).unwrap();
+        let output_stream0 = exec.execute(0, Arc::clone(&task_ctx)).unwrap();
         let batch0 = crate::common::collect(output_stream0).await.unwrap();
-        let output_stream1 = exec.execute(1, task_ctx.clone()).unwrap();
+        let output_stream1 = exec.execute(1, Arc::clone(&task_ctx)).unwrap();
         let batch1 = crate::common::collect(output_stream1).await.unwrap();
         assert!(batch0.is_empty() || batch1.is_empty());
         Ok(())
@@ -1496,12 +1508,12 @@ mod tests {
         let task_ctx = Arc::new(task_ctx);
 
         // create physical plan
-        let exec = MemoryExec::try_new(&input_partitions, schema.clone(), None)?;
+        let exec = MemoryExec::try_new(&input_partitions, Arc::clone(&schema), None)?;
         let exec = RepartitionExec::try_new(Arc::new(exec), partitioning)?;
 
         // pull partitions
         for i in 0..exec.partitioning.partition_count() {
-            let mut stream = exec.execute(i, task_ctx.clone())?;
+            let mut stream = exec.execute(i, Arc::clone(&task_ctx))?;
             let err =
                 arrow_datafusion_err!(stream.next().await.unwrap().unwrap_err().into());
             let err = err.find_root();
@@ -1642,7 +1654,7 @@ mod test {
     }
 
     fn memory_exec(schema: &SchemaRef) -> Arc<dyn ExecutionPlan> {
-        Arc::new(MemoryExec::try_new(&[vec![]], schema.clone(), None).unwrap())
+        Arc::new(MemoryExec::try_new(&[vec![]], Arc::clone(schema), None).unwrap())
     }
 
     fn sorted_memory_exec(
@@ -1650,7 +1662,7 @@ mod test {
         sort_exprs: Vec<PhysicalSortExpr>,
     ) -> Arc<dyn ExecutionPlan> {
         Arc::new(
-            MemoryExec::try_new(&[vec![]], schema.clone(), None)
+            MemoryExec::try_new(&[vec![]], Arc::clone(schema), None)
                 .unwrap()
                 .with_sort_information(vec![sort_exprs]),
         )
diff --git a/datafusion/physical-plan/src/sorts/builder.rs b/datafusion/physical-plan/src/sorts/builder.rs
index 3527d5738223..d32c60697ec8 100644
--- a/datafusion/physical-plan/src/sorts/builder.rs
+++ b/datafusion/physical-plan/src/sorts/builder.rs
@@ -20,6 +20,7 @@ use arrow::datatypes::SchemaRef;
 use arrow::record_batch::RecordBatch;
 use datafusion_common::Result;
 use datafusion_execution::memory_pool::MemoryReservation;
+use std::sync::Arc;
 
 #[derive(Debug, Copy, Clone, Default)]
 struct BatchCursor {
@@ -145,6 +146,9 @@ impl BatchBuilder {
             retain
         });
 
-        Ok(Some(RecordBatch::try_new(self.schema.clone(), columns)?))
+        Ok(Some(RecordBatch::try_new(
+            Arc::clone(&self.schema),
+            columns,
+        )?))
     }
 }
diff --git a/datafusion/physical-plan/src/sorts/merge.rs b/datafusion/physical-plan/src/sorts/merge.rs
index 422ff3aebdb3..85418ff36119 100644
--- a/datafusion/physical-plan/src/sorts/merge.rs
+++ b/datafusion/physical-plan/src/sorts/merge.rs
@@ -29,6 +29,7 @@ use datafusion_common::Result;
 use datafusion_execution::memory_pool::MemoryReservation;
 use futures::Stream;
 use std::pin::Pin;
+use std::sync::Arc;
 use std::task::{ready, Context, Poll};
 
 /// A fallible [`PartitionedStream`] of [`Cursor`] and [`RecordBatch`]
@@ -324,6 +325,6 @@ impl<C: CursorValues + Unpin> Stream for SortPreservingMergeStream<C> {
 
 impl<C: CursorValues + Unpin> RecordBatchStream for SortPreservingMergeStream<C> {
     fn schema(&self) -> SchemaRef {
-        self.in_progress.schema().clone()
+        Arc::clone(self.in_progress.schema())
     }
 }
diff --git a/datafusion/physical-plan/src/sorts/partial_sort.rs b/datafusion/physical-plan/src/sorts/partial_sort.rs
index ad5d485cffc9..fe6b744935fb 100644
--- a/datafusion/physical-plan/src/sorts/partial_sort.rs
+++ b/datafusion/physical-plan/src/sorts/partial_sort.rs
@@ -260,7 +260,7 @@ impl ExecutionPlan for PartialSortExec {
     ) -> Result<Arc<dyn ExecutionPlan>> {
         let new_partial_sort = PartialSortExec::new(
             self.expr.clone(),
-            children[0].clone(),
+            Arc::clone(&children[0]),
             self.common_prefix_length,
         )
         .with_fetch(self.fetch)
@@ -276,7 +276,7 @@ impl ExecutionPlan for PartialSortExec {
     ) -> Result<SendableRecordBatchStream> {
         trace!("Start PartialSortExec::execute for partition {} of context session_id {} and task_id {:?}", partition, context.session_id(), context.task_id());
 
-        let input = self.input.execute(partition, context.clone())?;
+        let input = self.input.execute(partition, Arc::clone(&context))?;
 
         trace!(
             "End PartialSortExec's input.execute for partition: {}",
@@ -485,11 +485,11 @@ mod tests {
                     options: option_asc,
                 },
             ],
-            source.clone(),
+            Arc::clone(&source),
             2,
         )) as Arc<dyn ExecutionPlan>;
 
-        let result = collect(partial_sort_exec, task_ctx.clone()).await?;
+        let result = collect(partial_sort_exec, Arc::clone(&task_ctx)).await?;
 
         let expected_after_sort = [
             "+---+---+---+",
@@ -549,13 +549,13 @@ mod tests {
                             options: option_asc,
                         },
                     ],
-                    source.clone(),
+                    Arc::clone(&source),
                     common_prefix_length,
                 )
                 .with_fetch(Some(4)),
             ) as Arc<dyn ExecutionPlan>;
 
-            let result = collect(partial_sort_exec, task_ctx.clone()).await?;
+            let result = collect(partial_sort_exec, Arc::clone(&task_ctx)).await?;
 
             let expected_after_sort = [
                 "+---+---+---+",
@@ -621,11 +621,11 @@ mod tests {
                         options: option_asc,
                     },
                 ],
-                source.clone(),
+                Arc::clone(source),
                 common_prefix_length,
             ));
 
-            let result = collect(partial_sort_exec, task_ctx.clone()).await?;
+            let result = collect(partial_sort_exec, Arc::clone(&task_ctx)).await?;
             assert_eq!(2, result.len());
             assert_eq!(
                 task_ctx.runtime_env().memory_pool.reserved(),
@@ -676,7 +676,7 @@ mod tests {
         Arc::new(
             MemoryExec::try_new(
                 &[vec![batch1, batch2, batch3, batch4]],
-                schema.clone(),
+                Arc::clone(&schema),
                 None,
             )
             .unwrap(),
@@ -711,7 +711,7 @@ mod tests {
                     options: option_asc,
                 },
             ],
-            mem_exec.clone(),
+            Arc::clone(&mem_exec),
             1,
         );
         let partial_sort_exec =
@@ -720,7 +720,7 @@ mod tests {
             partial_sort_executor.expr,
             partial_sort_executor.input,
         )) as Arc<dyn ExecutionPlan>;
-        let result = collect(partial_sort_exec, task_ctx.clone()).await?;
+        let result = collect(partial_sort_exec, Arc::clone(&task_ctx)).await?;
         assert_eq!(
             result.iter().map(|r| r.num_rows()).collect_vec(),
             [0, 125, 125, 0, 150]
@@ -732,7 +732,7 @@ mod tests {
             "The sort should have returned all memory used back to the memory manager"
         );
         let partial_sort_result = concat_batches(&schema, &result).unwrap();
-        let sort_result = collect(sort_exec, task_ctx.clone()).await?;
+        let sort_result = collect(sort_exec, Arc::clone(&task_ctx)).await?;
         assert_eq!(sort_result[0], partial_sort_result);
 
         Ok(())
@@ -772,7 +772,7 @@ mod tests {
                         options: option_asc,
                     },
                 ],
-                mem_exec.clone(),
+                Arc::clone(&mem_exec),
                 1,
             )
             .with_fetch(fetch_size);
@@ -783,7 +783,7 @@ mod tests {
                 SortExec::new(partial_sort_executor.expr, partial_sort_executor.input)
                     .with_fetch(fetch_size),
             ) as Arc<dyn ExecutionPlan>;
-            let result = collect(partial_sort_exec, task_ctx.clone()).await?;
+            let result = collect(partial_sort_exec, Arc::clone(&task_ctx)).await?;
             assert_eq!(
                 result.iter().map(|r| r.num_rows()).collect_vec(),
                 expected_batch_num_rows
@@ -795,7 +795,7 @@ mod tests {
                 "The sort should have returned all memory used back to the memory manager"
             );
             let partial_sort_result = concat_batches(&schema, &result)?;
-            let sort_result = collect(sort_exec, task_ctx.clone()).await?;
+            let sort_result = collect(sort_exec, Arc::clone(&task_ctx)).await?;
             assert_eq!(sort_result[0], partial_sort_result);
         }
 
@@ -822,8 +822,12 @@ mod tests {
         let data: ArrayRef =
             Arc::new(vec![1, 1, 2].into_iter().map(Some).collect::<UInt64Array>());
 
-        let batch = RecordBatch::try_new(schema.clone(), vec![data])?;
-        let input = Arc::new(MemoryExec::try_new(&[vec![batch]], schema.clone(), None)?);
+        let batch = RecordBatch::try_new(Arc::clone(&schema), vec![data])?;
+        let input = Arc::new(MemoryExec::try_new(
+            &[vec![batch]],
+            Arc::clone(&schema),
+            None,
+        )?);
 
         let partial_sort_exec = Arc::new(PartialSortExec::new(
             vec![PhysicalSortExpr {
@@ -837,13 +841,13 @@ mod tests {
         let result: Vec<RecordBatch> = collect(partial_sort_exec, task_ctx).await?;
         let expected_batch = vec![
             RecordBatch::try_new(
-                schema.clone(),
+                Arc::clone(&schema),
                 vec![Arc::new(
                     vec![1, 1].into_iter().map(Some).collect::<UInt64Array>(),
                 )],
             )?,
             RecordBatch::try_new(
-                schema.clone(),
+                Arc::clone(&schema),
                 vec![Arc::new(
                     vec![2].into_iter().map(Some).collect::<UInt64Array>(),
                 )],
@@ -879,7 +883,7 @@ mod tests {
 
         // define data.
         let batch = RecordBatch::try_new(
-            schema.clone(),
+            Arc::clone(&schema),
             vec![
                 Arc::new(Float32Array::from(vec![
                     Some(1.0_f32),
@@ -961,8 +965,11 @@ mod tests {
             *partial_sort_exec.schema().field(2).data_type()
         );
 
-        let result: Vec<RecordBatch> =
-            collect(partial_sort_exec.clone(), task_ctx).await?;
+        let result: Vec<RecordBatch> = collect(
+            Arc::clone(&partial_sort_exec) as Arc<dyn ExecutionPlan>,
+            task_ctx,
+        )
+        .await?;
         assert_batches_eq!(expected, &result);
         assert_eq!(result.len(), 2);
         let metrics = partial_sort_exec.metrics().unwrap();
@@ -997,7 +1004,7 @@ mod tests {
             1,
         ));
 
-        let fut = collect(sort_exec, task_ctx.clone());
+        let fut = collect(sort_exec, Arc::clone(&task_ctx));
         let mut fut = fut.boxed();
 
         assert_is_pending(&mut fut);
diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs
index 0bf66bc6e522..f347a0f5b6d5 100644
--- a/datafusion/physical-plan/src/sorts/sort.rs
+++ b/datafusion/physical-plan/src/sorts/sort.rs
@@ -338,13 +338,13 @@ impl ExternalSorter {
                         spill.path()
                     )));
                 }
-                let stream = read_spill_as_stream(spill, self.schema.clone(), 2)?;
+                let stream = read_spill_as_stream(spill, Arc::clone(&self.schema), 2)?;
                 streams.push(stream);
             }
 
             streaming_merge(
                 streams,
-                self.schema.clone(),
+                Arc::clone(&self.schema),
                 &self.expr,
                 self.metrics.baseline.clone(),
                 self.batch_size,
@@ -354,7 +354,9 @@ impl ExternalSorter {
         } else if !self.in_mem_batches.is_empty() {
             self.in_mem_sort_stream(self.metrics.baseline.clone())
         } else {
-            Ok(Box::pin(EmptyRecordBatchStream::new(self.schema.clone())))
+            Ok(Box::pin(EmptyRecordBatchStream::new(Arc::clone(
+                &self.schema,
+            ))))
         }
     }
 
@@ -394,8 +396,11 @@ impl ExternalSorter {
 
         let spill_file = self.runtime.disk_manager.create_tmp_file("Sorting")?;
         let batches = std::mem::take(&mut self.in_mem_batches);
-        let spilled_rows =
-            spill_record_batches(batches, spill_file.path().into(), self.schema.clone())?;
+        let spilled_rows = spill_record_batches(
+            batches,
+            spill_file.path().into(),
+            Arc::clone(&self.schema),
+        )?;
         let used = self.reservation.free();
         self.metrics.spill_count.add(1);
         self.metrics.spilled_bytes.add(used);
@@ -525,7 +530,7 @@ impl ExternalSorter {
 
         streaming_merge(
             streams,
-            self.schema.clone(),
+            Arc::clone(&self.schema),
             &self.expr,
             metrics,
             self.batch_size,
@@ -548,7 +553,7 @@ impl ExternalSorter {
         let schema = batch.schema();
 
         let fetch = self.fetch;
-        let expressions = self.expr.clone();
+        let expressions = Arc::clone(&self.expr);
         let stream = futures::stream::once(futures::future::lazy(move |_| {
             let sorted = sort_batch(&batch, &expressions, fetch)?;
             metrics.record_output(sorted.num_rows());
@@ -844,7 +849,7 @@ impl ExecutionPlan for SortExec {
         self: Arc<Self>,
         children: Vec<Arc<dyn ExecutionPlan>>,
     ) -> Result<Arc<dyn ExecutionPlan>> {
-        let new_sort = SortExec::new(self.expr.clone(), children[0].clone())
+        let new_sort = SortExec::new(self.expr.clone(), Arc::clone(&children[0]))
             .with_fetch(self.fetch)
             .with_preserve_partitioning(self.preserve_partitioning);
 
@@ -858,7 +863,7 @@ impl ExecutionPlan for SortExec {
     ) -> Result<SendableRecordBatchStream> {
         trace!("Start SortExec::execute for partition {} of context session_id {} and task_id {:?}", partition, context.session_id(), context.task_id());
 
-        let mut input = self.input.execute(partition, context.clone())?;
+        let mut input = self.input.execute(partition, Arc::clone(&context))?;
 
         let execution_options = &context.session_config().options().execution;
 
@@ -962,7 +967,7 @@ mod tests {
             Arc::new(CoalescePartitionsExec::new(csv)),
         ));
 
-        let result = collect(sort_exec, task_ctx.clone()).await?;
+        let result = collect(sort_exec, Arc::clone(&task_ctx)).await?;
 
         assert_eq!(result.len(), 1);
         assert_eq!(result[0].num_rows(), 400);
@@ -1005,7 +1010,11 @@ mod tests {
             Arc::new(CoalescePartitionsExec::new(input)),
         ));
 
-        let result = collect(sort_exec.clone(), task_ctx.clone()).await?;
+        let result = collect(
+            Arc::clone(&sort_exec) as Arc<dyn ExecutionPlan>,
+            Arc::clone(&task_ctx),
+        )
+        .await?;
 
         assert_eq!(result.len(), 2);
 
@@ -1081,7 +1090,11 @@ mod tests {
                 .with_fetch(fetch),
             );
 
-            let result = collect(sort_exec.clone(), task_ctx.clone()).await?;
+            let result = collect(
+                Arc::clone(&sort_exec) as Arc<dyn ExecutionPlan>,
+                Arc::clone(&task_ctx),
+            )
+            .await?;
             assert_eq!(result.len(), 1);
 
             let metrics = sort_exec.metrics().unwrap();
@@ -1111,9 +1124,10 @@ mod tests {
         let data: ArrayRef =
             Arc::new(vec![3, 2, 1].into_iter().map(Some).collect::<UInt64Array>());
 
-        let batch = RecordBatch::try_new(schema.clone(), vec![data]).unwrap();
-        let input =
-            Arc::new(MemoryExec::try_new(&[vec![batch]], schema.clone(), None).unwrap());
+        let batch = RecordBatch::try_new(Arc::clone(&schema), vec![data]).unwrap();
+        let input = Arc::new(
+            MemoryExec::try_new(&[vec![batch]], Arc::clone(&schema), None).unwrap(),
+        );
 
         let sort_exec = Arc::new(SortExec::new(
             vec![PhysicalSortExpr {
@@ -1128,7 +1142,7 @@ mod tests {
         let expected_data: ArrayRef =
             Arc::new(vec![1, 2, 3].into_iter().map(Some).collect::<UInt64Array>());
         let expected_batch =
-            RecordBatch::try_new(schema.clone(), vec![expected_data]).unwrap();
+            RecordBatch::try_new(Arc::clone(&schema), vec![expected_data]).unwrap();
 
         // Data is correct
         assert_eq!(&vec![expected_batch], &result);
@@ -1154,7 +1168,7 @@ mod tests {
 
         // define data.
         let batch = RecordBatch::try_new(
-            schema.clone(),
+            Arc::clone(&schema),
             vec![
                 Arc::new(Int32Array::from(vec![Some(2), None, Some(1), Some(2)])),
                 Arc::new(ListArray::from_iter_primitive::<Int32Type, _, _>(vec![
@@ -1183,7 +1197,11 @@ mod tests {
                     },
                 },
             ],
-            Arc::new(MemoryExec::try_new(&[vec![batch]], schema.clone(), None)?),
+            Arc::new(MemoryExec::try_new(
+                &[vec![batch]],
+                Arc::clone(&schema),
+                None,
+            )?),
         ));
 
         assert_eq!(DataType::Int32, *sort_exec.schema().field(0).data_type());
@@ -1192,7 +1210,8 @@ mod tests {
             *sort_exec.schema().field(1).data_type()
         );
 
-        let result: Vec<RecordBatch> = collect(sort_exec.clone(), task_ctx).await?;
+        let result: Vec<RecordBatch> =
+            collect(Arc::clone(&sort_exec) as Arc<dyn ExecutionPlan>, task_ctx).await?;
         let metrics = sort_exec.metrics().unwrap();
         assert!(metrics.elapsed_compute().unwrap() > 0);
         assert_eq!(metrics.output_rows().unwrap(), 4);
@@ -1226,7 +1245,7 @@ mod tests {
 
         // define data.
         let batch = RecordBatch::try_new(
-            schema.clone(),
+            Arc::clone(&schema),
             vec![
                 Arc::new(Float32Array::from(vec![
                     Some(f32::NAN),
@@ -1274,7 +1293,8 @@ mod tests {
         assert_eq!(DataType::Float32, *sort_exec.schema().field(0).data_type());
         assert_eq!(DataType::Float64, *sort_exec.schema().field(1).data_type());
 
-        let result: Vec<RecordBatch> = collect(sort_exec.clone(), task_ctx).await?;
+        let result: Vec<RecordBatch> =
+            collect(Arc::clone(&sort_exec) as Arc<dyn ExecutionPlan>, task_ctx).await?;
         let metrics = sort_exec.metrics().unwrap();
         assert!(metrics.elapsed_compute().unwrap() > 0);
         assert_eq!(metrics.output_rows().unwrap(), 8);
@@ -1337,7 +1357,7 @@ mod tests {
             blocking_exec,
         ));
 
-        let fut = collect(sort_exec, task_ctx.clone());
+        let fut = collect(sort_exec, Arc::clone(&task_ctx));
         let mut fut = fut.boxed();
 
         assert_is_pending(&mut fut);
@@ -1358,7 +1378,8 @@ mod tests {
         let schema = Arc::new(Schema::empty());
         let options = RecordBatchOptions::new().with_row_count(Some(1));
         let batch =
-            RecordBatch::try_new_with_options(schema.clone(), vec![], &options).unwrap();
+            RecordBatch::try_new_with_options(Arc::clone(&schema), vec![], &options)
+                .unwrap();
 
         let expressions = vec![PhysicalSortExpr {
             expr: Arc::new(Literal::new(ScalarValue::Int64(Some(1)))),
diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs
index e364aca3791c..41dfd449dd82 100644
--- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs
+++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs
@@ -188,7 +188,7 @@ impl ExecutionPlan for SortPreservingMergeExec {
         children: Vec<Arc<dyn ExecutionPlan>>,
     ) -> Result<Arc<dyn ExecutionPlan>> {
         Ok(Arc::new(
-            SortPreservingMergeExec::new(self.expr.clone(), children[0].clone())
+            SortPreservingMergeExec::new(self.expr.clone(), Arc::clone(&children[0]))
                 .with_fetch(self.fetch),
         ))
     }
@@ -232,7 +232,8 @@ impl ExecutionPlan for SortPreservingMergeExec {
             _ => {
                 let receivers = (0..input_partitions)
                     .map(|partition| {
-                        let stream = self.input.execute(partition, context.clone())?;
+                        let stream =
+                            self.input.execute(partition, Arc::clone(&context))?;
                         Ok(spawn_buffered(stream, 1))
                     })
                     .collect::<Result<_>>()?;
@@ -587,8 +588,9 @@ mod tests {
             },
         }];
 
-        let basic = basic_sort(csv.clone(), sort.clone(), task_ctx.clone()).await;
-        let partition = partition_sort(csv, sort, task_ctx.clone()).await;
+        let basic =
+            basic_sort(Arc::clone(&csv), sort.clone(), Arc::clone(&task_ctx)).await;
+        let partition = partition_sort(csv, sort, Arc::clone(&task_ctx)).await;
 
         let basic = arrow::util::pretty::pretty_format_batches(&[basic])
             .unwrap()
@@ -654,10 +656,11 @@ mod tests {
         }];
 
         let input =
-            sorted_partitioned_input(sort.clone(), &[10, 3, 11], task_ctx.clone())
+            sorted_partitioned_input(sort.clone(), &[10, 3, 11], Arc::clone(&task_ctx))
                 .await?;
-        let basic = basic_sort(input.clone(), sort.clone(), task_ctx.clone()).await;
-        let partition = sorted_merge(input, sort, task_ctx.clone()).await;
+        let basic =
+            basic_sort(Arc::clone(&input), sort.clone(), Arc::clone(&task_ctx)).await;
+        let partition = sorted_merge(input, sort, Arc::clone(&task_ctx)).await;
 
         assert_eq!(basic.num_rows(), 1200);
         assert_eq!(partition.num_rows(), 1200);
@@ -685,9 +688,9 @@ mod tests {
         // Test streaming with default batch size
         let task_ctx = Arc::new(TaskContext::default());
         let input =
-            sorted_partitioned_input(sort.clone(), &[10, 5, 13], task_ctx.clone())
+            sorted_partitioned_input(sort.clone(), &[10, 5, 13], Arc::clone(&task_ctx))
                 .await?;
-        let basic = basic_sort(input.clone(), sort.clone(), task_ctx).await;
+        let basic = basic_sort(Arc::clone(&input), sort.clone(), task_ctx).await;
 
         // batch size of 23
         let task_ctx = TaskContext::default()
@@ -805,17 +808,18 @@ mod tests {
         }];
 
         let batches =
-            sorted_partitioned_input(sort.clone(), &[5, 7, 3], task_ctx.clone()).await?;
+            sorted_partitioned_input(sort.clone(), &[5, 7, 3], Arc::clone(&task_ctx))
+                .await?;
 
         let partition_count = batches.output_partitioning().partition_count();
         let mut streams = Vec::with_capacity(partition_count);
 
         for partition in 0..partition_count {
-            let mut builder = RecordBatchReceiverStream::builder(schema.clone(), 1);
+            let mut builder = RecordBatchReceiverStream::builder(Arc::clone(&schema), 1);
 
             let sender = builder.tx();
 
-            let mut stream = batches.execute(partition, task_ctx.clone()).unwrap();
+            let mut stream = batches.execute(partition, Arc::clone(&task_ctx)).unwrap();
             builder.spawn(async move {
                 while let Some(batch) = stream.next().await {
                     sender.send(batch).await.unwrap();
@@ -849,7 +853,7 @@ mod tests {
 
         assert_eq!(merged.len(), 1);
         let merged = merged.remove(0);
-        let basic = basic_sort(batches, sort.clone(), task_ctx.clone()).await;
+        let basic = basic_sort(batches, sort.clone(), Arc::clone(&task_ctx)).await;
 
         let basic = arrow::util::pretty::pretty_format_batches(&[basic])
             .unwrap()
@@ -885,7 +889,9 @@ mod tests {
         let exec = MemoryExec::try_new(&[vec![b1], vec![b2]], schema, None).unwrap();
         let merge = Arc::new(SortPreservingMergeExec::new(sort, Arc::new(exec)));
 
-        let collected = collect(merge.clone(), task_ctx).await.unwrap();
+        let collected = collect(Arc::clone(&merge) as Arc<dyn ExecutionPlan>, task_ctx)
+            .await
+            .unwrap();
         let expected = [
             "+----+---+",
             "| a  | b |",
diff --git a/datafusion/physical-plan/src/sorts/stream.rs b/datafusion/physical-plan/src/sorts/stream.rs
index 135b4fbdece4..c7924edfb1eb 100644
--- a/datafusion/physical-plan/src/sorts/stream.rs
+++ b/datafusion/physical-plan/src/sorts/stream.rs
@@ -109,7 +109,7 @@ impl RowCursorStream {
         Ok(Self {
             converter,
             reservation,
-            column_expressions: expressions.iter().map(|x| x.expr.clone()).collect(),
+            column_expressions: expressions.iter().map(|x| Arc::clone(&x.expr)).collect(),
             streams: FusedStreams(streams),
         })
     }
diff --git a/datafusion/physical-plan/src/stream.rs b/datafusion/physical-plan/src/stream.rs
index 99d9367740be..faeb4799f5af 100644
--- a/datafusion/physical-plan/src/stream.rs
+++ b/datafusion/physical-plan/src/stream.rs
@@ -382,7 +382,7 @@ where
     S: Stream<Item = Result<RecordBatch>>,
 {
     fn schema(&self) -> SchemaRef {
-        self.schema.clone()
+        Arc::clone(&self.schema)
     }
 }
 
@@ -402,7 +402,7 @@ impl EmptyRecordBatchStream {
 
 impl RecordBatchStream for EmptyRecordBatchStream {
     fn schema(&self) -> SchemaRef {
-        self.schema.clone()
+        Arc::clone(&self.schema)
     }
 }
 
@@ -474,7 +474,7 @@ mod test {
         let schema = schema();
 
         let num_partitions = 10;
-        let input = PanicExec::new(schema.clone(), num_partitions);
+        let input = PanicExec::new(Arc::clone(&schema), num_partitions);
         consume(input, 10).await
     }
 
@@ -485,7 +485,7 @@ mod test {
 
         // make 2 partitions, second partition panics before the first
         let num_partitions = 2;
-        let input = PanicExec::new(schema.clone(), num_partitions)
+        let input = PanicExec::new(Arc::clone(&schema), num_partitions)
             .with_partition_panic(0, 10)
             .with_partition_panic(1, 3); // partition 1 should panic first (after 3 )
 
@@ -504,12 +504,12 @@ mod test {
         let schema = schema();
 
         // Make an input that never proceeds
-        let input = BlockingExec::new(schema.clone(), 1);
+        let input = BlockingExec::new(Arc::clone(&schema), 1);
         let refs = input.refs();
 
         // Configure a RecordBatchReceiverStream to consume the input
         let mut builder = RecordBatchReceiverStream::builder(schema, 2);
-        builder.run_input(Arc::new(input), 0, task_ctx.clone());
+        builder.run_input(Arc::new(input), 0, Arc::clone(&task_ctx));
         let stream = builder.build();
 
         // input should still be present
@@ -529,12 +529,14 @@ mod test {
         let schema = schema();
 
         // make an input that will error twice
-        let error_stream =
-            MockExec::new(vec![exec_err!("Test1"), exec_err!("Test2")], schema.clone())
-                .with_use_task(false);
+        let error_stream = MockExec::new(
+            vec![exec_err!("Test1"), exec_err!("Test2")],
+            Arc::clone(&schema),
+        )
+        .with_use_task(false);
 
         let mut builder = RecordBatchReceiverStream::builder(schema, 2);
-        builder.run_input(Arc::new(error_stream), 0, task_ctx.clone());
+        builder.run_input(Arc::new(error_stream), 0, Arc::clone(&task_ctx));
         let mut stream = builder.build();
 
         // get the first result, which should be an error
@@ -560,7 +562,11 @@ mod test {
         let mut builder =
             RecordBatchReceiverStream::builder(input.schema(), num_partitions);
         for partition in 0..num_partitions {
-            builder.run_input(input.clone(), partition, task_ctx.clone());
+            builder.run_input(
+                Arc::clone(&input) as Arc<dyn ExecutionPlan>,
+                partition,
+                Arc::clone(&task_ctx),
+            );
         }
         let mut stream = builder.build();
 
diff --git a/datafusion/physical-plan/src/streaming.rs b/datafusion/physical-plan/src/streaming.rs
index ff57adde4e2e..5a9035c8dbfc 100644
--- a/datafusion/physical-plan/src/streaming.rs
+++ b/datafusion/physical-plan/src/streaming.rs
@@ -93,7 +93,7 @@ impl StreamingTableExec {
         let projected_output_ordering =
             projected_output_ordering.into_iter().collect::<Vec<_>>();
         let cache = Self::compute_properties(
-            projected_schema.clone(),
+            Arc::clone(&projected_schema),
             &projected_output_ordering,
             &partitions,
             infinite,
@@ -240,7 +240,7 @@ impl ExecutionPlan for StreamingTableExec {
         let stream = self.partitions[partition].execute(ctx);
         let projected_stream = match self.projection.clone() {
             Some(projection) => Box::pin(RecordBatchStreamAdapter::new(
-                self.projected_schema.clone(),
+                Arc::clone(&self.projected_schema),
                 stream.map(move |x| {
                     x.and_then(|b| b.project(projection.as_ref()).map_err(Into::into))
                 }),
@@ -327,7 +327,7 @@ mod test {
         /// Set the batches for the stream
         fn with_batches(mut self, batches: Vec<RecordBatch>) -> Self {
             let stream = TestPartitionStream::new_with_batches(batches);
-            self.schema = Some(stream.schema().clone());
+            self.schema = Some(Arc::clone(stream.schema()));
             self.partitions = vec![Arc::new(stream)];
             self
         }
diff --git a/datafusion/physical-plan/src/test.rs b/datafusion/physical-plan/src/test.rs
index 377b919bb407..f5b4a096018f 100644
--- a/datafusion/physical-plan/src/test.rs
+++ b/datafusion/physical-plan/src/test.rs
@@ -144,6 +144,9 @@ impl PartitionStream for TestPartitionStream {
     }
     fn execute(&self, _ctx: Arc<TaskContext>) -> SendableRecordBatchStream {
         let stream = futures::stream::iter(self.batches.clone().into_iter().map(Ok));
-        Box::pin(RecordBatchStreamAdapter::new(self.schema.clone(), stream))
+        Box::pin(RecordBatchStreamAdapter::new(
+            Arc::clone(&self.schema),
+            stream,
+        ))
     }
 }
diff --git a/datafusion/physical-plan/src/test/exec.rs b/datafusion/physical-plan/src/test/exec.rs
index ad47a484c9f7..ac4eb1ca9e58 100644
--- a/datafusion/physical-plan/src/test/exec.rs
+++ b/datafusion/physical-plan/src/test/exec.rs
@@ -133,7 +133,7 @@ impl MockExec {
     /// ensure any poll loops are correct. This behavior can be
     /// changed with `with_use_task`
     pub fn new(data: Vec<Result<RecordBatch>>, schema: SchemaRef) -> Self {
-        let cache = Self::compute_properties(schema.clone());
+        let cache = Self::compute_properties(Arc::clone(&schema));
         Self {
             data,
             schema,
@@ -294,7 +294,7 @@ impl BarrierExec {
     pub fn new(data: Vec<Vec<RecordBatch>>, schema: SchemaRef) -> Self {
         // wait for all streams and the input
         let barrier = Arc::new(Barrier::new(data.len() + 1));
-        let cache = Self::compute_properties(schema.clone(), &data);
+        let cache = Self::compute_properties(Arc::clone(&schema), &data);
         Self {
             data,
             schema,
@@ -374,7 +374,7 @@ impl ExecutionPlan for BarrierExec {
 
         // task simply sends data in order after barrier is reached
         let data = self.data[partition].clone();
-        let b = self.barrier.clone();
+        let b = Arc::clone(&self.barrier);
         let tx = builder.tx();
         builder.spawn(async move {
             println!("Partition {partition} waiting on barrier");
@@ -421,7 +421,7 @@ impl ErrorExec {
             DataType::Int64,
             true,
         )]));
-        let cache = Self::compute_properties(schema.clone());
+        let cache = Self::compute_properties(schema);
         Self { cache }
     }
 
@@ -591,7 +591,7 @@ pub struct BlockingExec {
 impl BlockingExec {
     /// Create new [`BlockingExec`] with a give schema and number of partitions.
     pub fn new(schema: SchemaRef, n_partitions: usize) -> Self {
-        let cache = Self::compute_properties(schema.clone(), n_partitions);
+        let cache = Self::compute_properties(Arc::clone(&schema), n_partitions);
         Self {
             schema,
             refs: Default::default(),
@@ -735,7 +735,7 @@ impl PanicExec {
     /// partitions, which will each panic immediately.
     pub fn new(schema: SchemaRef, n_partitions: usize) -> Self {
         let batches_until_panics = vec![0; n_partitions];
-        let cache = Self::compute_properties(schema.clone(), &batches_until_panics);
+        let cache = Self::compute_properties(Arc::clone(&schema), &batches_until_panics);
         Self {
             schema,
             batches_until_panics,
@@ -845,7 +845,7 @@ impl Stream for PanicStream {
             if self.ready {
                 self.batches_until_panic -= 1;
                 self.ready = false;
-                let batch = RecordBatch::new_empty(self.schema.clone());
+                let batch = RecordBatch::new_empty(Arc::clone(&self.schema));
                 return Poll::Ready(Some(Ok(batch)));
             } else {
                 self.ready = true;
diff --git a/datafusion/physical-plan/src/topk/mod.rs b/datafusion/physical-plan/src/topk/mod.rs
index 6a77bfaf3ccd..5366a5707696 100644
--- a/datafusion/physical-plan/src/topk/mod.rs
+++ b/datafusion/physical-plan/src/topk/mod.rs
@@ -131,7 +131,7 @@ impl TopK {
         );
 
         Ok(Self {
-            schema: schema.clone(),
+            schema: Arc::clone(&schema),
             metrics: TopKMetrics::new(metrics, partition),
             reservation,
             batch_size,
@@ -355,7 +355,7 @@ impl TopKHeap {
     /// high, as a single [`RecordBatch`], and a sorted vec of the
     /// current heap's contents
     pub fn emit_with_state(&mut self) -> Result<(RecordBatch, Vec<TopKRow>)> {
-        let schema = self.store.schema().clone();
+        let schema = Arc::clone(self.store.schema());
 
         // generate sorted rows
         let topk_rows = std::mem::take(&mut self.inner).into_sorted_vec();
diff --git a/datafusion/physical-plan/src/tree_node.rs b/datafusion/physical-plan/src/tree_node.rs
index 1570778be69b..96bd0de3d37c 100644
--- a/datafusion/physical-plan/src/tree_node.rs
+++ b/datafusion/physical-plan/src/tree_node.rs
@@ -62,7 +62,7 @@ impl<T> PlanContext<T> {
     }
 
     pub fn update_plan_from_children(mut self) -> Result<Self> {
-        let children_plans = self.children.iter().map(|c| c.plan.clone()).collect();
+        let children_plans = self.children.iter().map(|c| Arc::clone(&c.plan)).collect();
         self.plan = with_new_children_if_necessary(self.plan, children_plans)?;
 
         Ok(self)
diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs
index 867cddeb7b41..b39c6aee82b9 100644
--- a/datafusion/physical-plan/src/union.rs
+++ b/datafusion/physical-plan/src/union.rs
@@ -449,7 +449,7 @@ impl ExecutionPlan for InterleaveExec {
         let mut input_stream_vec = vec![];
         for input in self.inputs.iter() {
             if partition < input.output_partitioning().partition_count() {
-                input_stream_vec.push(input.execute(partition, context.clone())?);
+                input_stream_vec.push(input.execute(partition, Arc::clone(&context))?);
             } else {
                 // Do not find a partition to execute
                 break;
@@ -550,7 +550,7 @@ impl CombinedRecordBatchStream {
 
 impl RecordBatchStream for CombinedRecordBatchStream {
     fn schema(&self) -> SchemaRef {
-        self.schema.clone()
+        Arc::clone(&self.schema)
     }
 }
 
@@ -657,7 +657,7 @@ mod tests {
         in_data
             .iter()
             .map(|(expr, options)| PhysicalSortExpr {
-                expr: (*expr).clone(),
+                expr: Arc::clone(*expr),
                 options: *options,
             })
             .collect::<Vec<_>>()
@@ -842,11 +842,11 @@ mod tests {
                 .map(|ordering| convert_to_sort_exprs(ordering))
                 .collect::<Vec<_>>();
             let child1 = Arc::new(
-                MemoryExec::try_new(&[], schema.clone(), None)?
+                MemoryExec::try_new(&[], Arc::clone(&schema), None)?
                     .with_sort_information(first_orderings),
             );
             let child2 = Arc::new(
-                MemoryExec::try_new(&[], schema.clone(), None)?
+                MemoryExec::try_new(&[], Arc::clone(&schema), None)?
                     .with_sort_information(second_orderings),
             );
 
diff --git a/datafusion/physical-plan/src/unnest.rs b/datafusion/physical-plan/src/unnest.rs
index e072b214fd36..bdd56f4b5aa4 100644
--- a/datafusion/physical-plan/src/unnest.rs
+++ b/datafusion/physical-plan/src/unnest.rs
@@ -83,7 +83,7 @@ impl UnnestExec {
         schema: SchemaRef,
         options: UnnestOptions,
     ) -> Self {
-        let cache = Self::compute_properties(&input, schema.clone());
+        let cache = Self::compute_properties(&input, Arc::clone(&schema));
 
         UnnestExec {
             input,
@@ -147,10 +147,10 @@ impl ExecutionPlan for UnnestExec {
         children: Vec<Arc<dyn ExecutionPlan>>,
     ) -> Result<Arc<dyn ExecutionPlan>> {
         Ok(Arc::new(UnnestExec::new(
-            children[0].clone(),
+            Arc::clone(&children[0]),
             self.list_column_indices.clone(),
             self.struct_column_indices.clone(),
-            self.schema.clone(),
+            Arc::clone(&self.schema),
             self.options.clone(),
         )))
     }
@@ -169,7 +169,7 @@ impl ExecutionPlan for UnnestExec {
 
         Ok(Box::pin(UnnestStream {
             input,
-            schema: self.schema.clone(),
+            schema: Arc::clone(&self.schema),
             list_type_columns: self.list_column_indices.clone(),
             struct_column_indices: self.struct_column_indices.iter().copied().collect(),
             options: self.options.clone(),
@@ -237,7 +237,7 @@ struct UnnestStream {
 
 impl RecordBatchStream for UnnestStream {
     fn schema(&self) -> SchemaRef {
-        self.schema.clone()
+        Arc::clone(&self.schema)
     }
 }
 
@@ -330,13 +330,13 @@ fn flatten_struct_cols(
                     data_type
                 ),
             },
-            None => Ok(vec![column_data.clone()]),
+            None => Ok(vec![Arc::clone(column_data)]),
         })
         .collect::<Result<Vec<_>>>()?
         .into_iter()
         .flatten()
         .collect();
-    Ok(RecordBatch::try_new(schema.clone(), columns_expanded)?)
+    Ok(RecordBatch::try_new(Arc::clone(schema), columns_expanded)?)
 }
 
 /// For each row in a `RecordBatch`, some list/struct columns need to be unnested.
@@ -357,7 +357,7 @@ fn build_batch(
             let list_arrays: Vec<ArrayRef> = list_type_columns
                 .iter()
                 .map(|index| {
-                    ColumnarValue::Array(batch.column(*index).clone())
+                    ColumnarValue::Array(Arc::clone(batch.column(*index)))
                         .into_array(batch.num_rows())
                 })
                 .collect::<Result<_>>()?;
@@ -372,7 +372,7 @@ fn build_batch(
                 })? as usize
             };
             if total_length == 0 {
-                return Ok(RecordBatch::new_empty(schema.clone()));
+                return Ok(RecordBatch::new_empty(Arc::clone(schema)));
             }
 
             // Unnest all the list arrays
@@ -444,7 +444,7 @@ fn find_longest_length(
         .collect::<Result<_>>()?;
 
     let longest_length = list_lengths.iter().skip(1).try_fold(
-        list_lengths[0].clone(),
+        Arc::clone(&list_lengths[0]),
         |longest, current| {
             let is_lt = lt(&longest, &current)?;
             zip(&is_lt, &current, &longest)
@@ -649,7 +649,7 @@ fn flatten_list_cols_from_indices(
         .iter()
         .enumerate()
         .map(|(col_idx, arr)| match unnested_list_arrays.get(&col_idx) {
-            Some(unnested_array) => Ok(unnested_array.clone()),
+            Some(unnested_array) => Ok(Arc::clone(unnested_array)),
             None => Ok(kernels::take::take(arr, indices, None)?),
         })
         .collect::<Result<Vec<_>>>()?;
@@ -813,27 +813,27 @@ mod tests {
         // Test with single ListArray
         //  [A, B, C], [], NULL, [D], NULL, [NULL, F]
         let list_array = Arc::new(make_generic_array::<i32>()) as ArrayRef;
-        verify_longest_length(&[list_array.clone()], false, vec![3, 0, 0, 1, 0, 2])?;
-        verify_longest_length(&[list_array.clone()], true, vec![3, 0, 1, 1, 1, 2])?;
+        verify_longest_length(&[Arc::clone(&list_array)], false, vec![3, 0, 0, 1, 0, 2])?;
+        verify_longest_length(&[Arc::clone(&list_array)], true, vec![3, 0, 1, 1, 1, 2])?;
 
         // Test with single LargeListArray
         //  [A, B, C], [], NULL, [D], NULL, [NULL, F]
         let list_array = Arc::new(make_generic_array::<i64>()) as ArrayRef;
-        verify_longest_length(&[list_array.clone()], false, vec![3, 0, 0, 1, 0, 2])?;
-        verify_longest_length(&[list_array.clone()], true, vec![3, 0, 1, 1, 1, 2])?;
+        verify_longest_length(&[Arc::clone(&list_array)], false, vec![3, 0, 0, 1, 0, 2])?;
+        verify_longest_length(&[Arc::clone(&list_array)], true, vec![3, 0, 1, 1, 1, 2])?;
 
         // Test with single FixedSizeListArray
         //  [A, B], NULL, [C, D], NULL, [NULL, F], [NULL, NULL]
         let list_array = Arc::new(make_fixed_list()) as ArrayRef;
-        verify_longest_length(&[list_array.clone()], false, vec![2, 0, 2, 0, 2, 2])?;
-        verify_longest_length(&[list_array.clone()], true, vec![2, 1, 2, 1, 2, 2])?;
+        verify_longest_length(&[Arc::clone(&list_array)], false, vec![2, 0, 2, 0, 2, 2])?;
+        verify_longest_length(&[Arc::clone(&list_array)], true, vec![2, 1, 2, 1, 2, 2])?;
 
         // Test with multiple list arrays
         //  [A, B, C], [], NULL, [D], NULL, [NULL, F]
         //  [A, B], NULL, [C, D], NULL, [NULL, F], [NULL, NULL]
         let list1 = Arc::new(make_generic_array::<i32>()) as ArrayRef;
         let list2 = Arc::new(make_fixed_list()) as ArrayRef;
-        let list_arrays = vec![list1.clone(), list2.clone()];
+        let list_arrays = vec![Arc::clone(&list1), Arc::clone(&list2)];
         verify_longest_length(&list_arrays, false, vec![3, 0, 2, 1, 2, 2])?;
         verify_longest_length(&list_arrays, true, vec![3, 1, 2, 1, 2, 2])?;
 
diff --git a/datafusion/physical-plan/src/values.rs b/datafusion/physical-plan/src/values.rs
index 4d385812d4a8..3ea27d62d80b 100644
--- a/datafusion/physical-plan/src/values.rs
+++ b/datafusion/physical-plan/src/values.rs
@@ -88,7 +88,7 @@ impl ValuesExec {
                     .and_then(ScalarValue::iter_to_array)
             })
             .collect::<Result<Vec<_>>>()?;
-        let batch = RecordBatch::try_new(schema.clone(), arr)?;
+        let batch = RecordBatch::try_new(Arc::clone(&schema), arr)?;
         let data: Vec<RecordBatch> = vec![batch];
         Self::try_new_from_batches(schema, data)
     }
@@ -114,7 +114,7 @@ impl ValuesExec {
             }
         }
 
-        let cache = Self::compute_properties(schema.clone());
+        let cache = Self::compute_properties(Arc::clone(&schema));
         Ok(ValuesExec {
             schema,
             data: batches,
@@ -175,7 +175,7 @@ impl ExecutionPlan for ValuesExec {
         self: Arc<Self>,
         _: Vec<Arc<dyn ExecutionPlan>>,
     ) -> Result<Arc<dyn ExecutionPlan>> {
-        ValuesExec::try_new_from_batches(self.schema.clone(), self.data.clone())
+        ValuesExec::try_new_from_batches(Arc::clone(&self.schema), self.data.clone())
             .map(|e| Arc::new(e) as _)
     }
 
@@ -193,7 +193,7 @@ impl ExecutionPlan for ValuesExec {
 
         Ok(Box::pin(MemoryStream::try_new(
             self.data(),
-            self.schema.clone(),
+            Arc::clone(&self.schema),
             None,
         )?))
     }
@@ -260,7 +260,7 @@ mod tests {
             DataType::UInt32,
             false,
         )]));
-        let _ = ValuesExec::try_new(schema.clone(), vec![vec![lit(1u32)]]).unwrap();
+        let _ = ValuesExec::try_new(Arc::clone(&schema), vec![vec![lit(1u32)]]).unwrap();
         // Test that a null value is rejected
         let _ = ValuesExec::try_new(schema, vec![vec![lit(ScalarValue::UInt32(None))]])
             .unwrap_err();
diff --git a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs
index 9eb29891703e..6311107f7b58 100644
--- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs
+++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs
@@ -289,7 +289,7 @@ impl ExecutionPlan for BoundedWindowAggExec {
     ) -> Result<Arc<dyn ExecutionPlan>> {
         Ok(Arc::new(BoundedWindowAggExec::try_new(
             self.window_expr.clone(),
-            children[0].clone(),
+            Arc::clone(&children[0]),
             self.partition_keys.clone(),
             self.input_order_mode.clone(),
         )?))
@@ -303,7 +303,7 @@ impl ExecutionPlan for BoundedWindowAggExec {
         let input = self.input.execute(partition, context)?;
         let search_mode = self.get_search_algo()?;
         let stream = Box::pin(BoundedWindowAggStream::new(
-            self.schema.clone(),
+            Arc::clone(&self.schema),
             self.window_expr.clone(),
             input,
             BaselineMetrics::new(&self.metrics, partition),
@@ -394,7 +394,9 @@ trait PartitionSearcher: Send {
                 // as it may not have the "correct" schema in terms of output
                 // nullability constraints. For details, see the following issue:
                 // https://github.com/apache/datafusion/issues/9320
-                .or_insert_with(|| PartitionBatchState::new(self.input_schema().clone()));
+                .or_insert_with(|| {
+                    PartitionBatchState::new(Arc::clone(self.input_schema()))
+                });
             partition_batch_state.extend(&partition_batch)?;
         }
 
@@ -513,7 +515,7 @@ impl PartitionSearcher for LinearSearch {
             let length = indices.len();
             for (idx, window_agg_state) in window_agg_states.iter().enumerate() {
                 let partition = &window_agg_state[&row];
-                let values = partition.state.out_col.slice(0, length).clone();
+                let values = Arc::clone(&partition.state.out_col.slice(0, length));
                 new_columns[idx].push(values);
             }
             let partition_batch_state = &mut partition_buffers[&row];
@@ -935,7 +937,7 @@ impl BoundedWindowAggStream {
         search_mode: Box<dyn PartitionSearcher>,
     ) -> Result<Self> {
         let state = window_expr.iter().map(|_| IndexMap::new()).collect();
-        let empty_batch = RecordBatch::new_empty(schema.clone());
+        let empty_batch = RecordBatch::new_empty(Arc::clone(&schema));
         Ok(Self {
             schema,
             input,
@@ -957,7 +959,7 @@ impl BoundedWindowAggStream {
             cur_window_expr.evaluate_stateful(&self.partition_buffers, state)?;
         }
 
-        let schema = self.schema.clone();
+        let schema = Arc::clone(&self.schema);
         let window_expr_out = self.search_mode.calculate_out_columns(
             &self.input_buffer,
             &self.window_agg_states,
@@ -1114,7 +1116,7 @@ impl BoundedWindowAggStream {
 impl RecordBatchStream for BoundedWindowAggStream {
     /// Get the schema
     fn schema(&self) -> SchemaRef {
-        self.schema.clone()
+        Arc::clone(&self.schema)
     }
 }
 
@@ -1287,7 +1289,7 @@ mod tests {
 
     impl RecordBatchStream for TestStreamPartition {
         fn schema(&self) -> SchemaRef {
-            self.schema.clone()
+            Arc::clone(&self.schema)
         }
     }
 
@@ -1467,7 +1469,7 @@ mod tests {
             }
 
             let batch = RecordBatch::try_new(
-                schema.clone(),
+                Arc::clone(schema),
                 vec![Arc::new(sn1_array.finish()), Arc::new(hash_array.finish())],
             )?;
             batches.push(batch);
@@ -1500,7 +1502,7 @@ mod tests {
         // Source has 2 partitions
         let partitions = vec![
             Arc::new(TestStreamPartition {
-                schema: schema.clone(),
+                schema: Arc::clone(&schema),
                 batches: batches.clone(),
                 idx: 0,
                 state: PolingState::BatchReturn,
@@ -1510,7 +1512,7 @@ mod tests {
             n_partition
         ];
         let source = Arc::new(StreamingTableExec::try_new(
-            schema.clone(),
+            Arc::clone(&schema),
             partitions,
             None,
             orderings,
@@ -1533,28 +1535,38 @@ mod tests {
         let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, false)]));
         // Create a new batch of data to insert into the table
         let batch = RecordBatch::try_new(
-            schema.clone(),
+            Arc::clone(&schema),
             vec![Arc::new(arrow_array::Int32Array::from(vec![1, 2, 3]))],
         )?;
 
         let memory_exec = MemoryExec::try_new(
             &[vec![batch.clone(), batch.clone(), batch.clone()]],
-            schema.clone(),
+            Arc::clone(&schema),
             None,
         )
         .map(|e| Arc::new(e) as Arc<dyn ExecutionPlan>)?;
         let col_a = col("a", &schema)?;
-        let nth_value_func1 =
-            NthValue::nth("nth_value(-1)", col_a.clone(), DataType::Int32, 1, false)?
-                .reverse_expr()
-                .unwrap();
-        let nth_value_func2 =
-            NthValue::nth("nth_value(-2)", col_a.clone(), DataType::Int32, 2, false)?
-                .reverse_expr()
-                .unwrap();
+        let nth_value_func1 = NthValue::nth(
+            "nth_value(-1)",
+            Arc::clone(&col_a),
+            DataType::Int32,
+            1,
+            false,
+        )?
+        .reverse_expr()
+        .unwrap();
+        let nth_value_func2 = NthValue::nth(
+            "nth_value(-2)",
+            Arc::clone(&col_a),
+            DataType::Int32,
+            2,
+            false,
+        )?
+        .reverse_expr()
+        .unwrap();
         let last_value_func = Arc::new(NthValue::last(
             "last",
-            col_a.clone(),
+            Arc::clone(&col_a),
             DataType::Int32,
             false,
         )) as _;
diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs
index 0622aad74cad..7f794556a241 100644
--- a/datafusion/physical-plan/src/windows/mod.rs
+++ b/datafusion/physical-plan/src/windows/mod.rs
@@ -64,11 +64,11 @@ pub fn schema_add_window_field(
 ) -> Result<Arc<Schema>> {
     let data_types = args
         .iter()
-        .map(|e| e.clone().as_ref().data_type(schema))
+        .map(|e| Arc::clone(e).as_ref().data_type(schema))
         .collect::<Result<Vec<_>>>()?;
     let nullability = args
         .iter()
-        .map(|e| e.clone().as_ref().nullable(schema))
+        .map(|e| Arc::clone(e).as_ref().nullable(schema))
         .collect::<Result<Vec<_>>>()?;
     let window_expr_return_type = window_fn.return_type(&data_types, &nullability)?;
     let mut window_fields = schema
@@ -288,7 +288,7 @@ fn create_built_in_window_expr(
             }
         }
         BuiltInWindowFunction::Lag => {
-            let arg = args[0].clone();
+            let arg = Arc::clone(&args[0]);
             let shift_offset = get_scalar_value_from_args(args, 1)?
                 .map(get_signed_integer)
                 .map_or(Ok(None), |v| v.map(Some))?;
@@ -304,7 +304,7 @@ fn create_built_in_window_expr(
             ))
         }
         BuiltInWindowFunction::Lead => {
-            let arg = args[0].clone();
+            let arg = Arc::clone(&args[0]);
             let shift_offset = get_scalar_value_from_args(args, 1)?
                 .map(get_signed_integer)
                 .map_or(Ok(None), |v| v.map(Some))?;
@@ -320,7 +320,7 @@ fn create_built_in_window_expr(
             ))
         }
         BuiltInWindowFunction::NthValue => {
-            let arg = args[0].clone();
+            let arg = Arc::clone(&args[0]);
             let n = get_signed_integer(
                 args[1]
                     .as_any()
@@ -338,7 +338,7 @@ fn create_built_in_window_expr(
             )?)
         }
         BuiltInWindowFunction::FirstValue => {
-            let arg = args[0].clone();
+            let arg = Arc::clone(&args[0]);
             Arc::new(NthValue::first(
                 name,
                 arg,
@@ -347,7 +347,7 @@ fn create_built_in_window_expr(
             ))
         }
         BuiltInWindowFunction::LastValue => {
-            let arg = args[0].clone();
+            let arg = Arc::clone(&args[0]);
             Arc::new(NthValue::last(
                 name,
                 arg,
@@ -429,13 +429,16 @@ pub(crate) fn calc_requirements<
     let mut sort_reqs = partition_by_exprs
         .into_iter()
         .map(|partition_by| {
-            PhysicalSortRequirement::new(partition_by.borrow().clone(), None)
+            PhysicalSortRequirement::new(Arc::clone(partition_by.borrow()), None)
         })
         .collect::<Vec<_>>();
     for element in orderby_sort_exprs.into_iter() {
         let PhysicalSortExpr { expr, options } = element.borrow();
         if !sort_reqs.iter().any(|e| e.expr.eq(expr)) {
-            sort_reqs.push(PhysicalSortRequirement::new(expr.clone(), Some(*options)));
+            sort_reqs.push(PhysicalSortRequirement::new(
+                Arc::clone(expr),
+                Some(*options),
+            ));
         }
     }
     // Convert empty result to None. Otherwise wrap result inside Some()
@@ -464,7 +467,7 @@ pub(crate) fn get_partition_by_sort_exprs(
 ) -> Result<LexOrdering> {
     let ordered_partition_exprs = ordered_partition_by_indices
         .iter()
-        .map(|idx| partition_by_exprs[*idx].clone())
+        .map(|idx| Arc::clone(&partition_by_exprs[*idx]))
         .collect::<Vec<_>>();
     // Make sure ordered section doesn't move over the partition by expression
     assert!(ordered_partition_by_indices.len() <= partition_by_exprs.len());
@@ -485,7 +488,7 @@ pub(crate) fn window_equivalence_properties(
 ) -> EquivalenceProperties {
     // We need to update the schema, so we can not directly use
     // `input.equivalence_properties()`.
-    let mut window_eq_properties = EquivalenceProperties::new(schema.clone())
+    let mut window_eq_properties = EquivalenceProperties::new(Arc::clone(schema))
         .extend(input.equivalence_properties().clone());
 
     for expr in window_expr {
@@ -556,7 +559,7 @@ pub fn get_best_fitting_window(
     if window_expr.iter().all(|e| e.uses_bounded_memory()) {
         Ok(Some(Arc::new(BoundedWindowAggExec::try_new(
             window_expr,
-            input.clone(),
+            Arc::clone(input),
             physical_partition_keys.to_vec(),
             input_order_mode,
         )?) as _))
@@ -569,7 +572,7 @@ pub fn get_best_fitting_window(
     } else {
         Ok(Some(Arc::new(WindowAggExec::try_new(
             window_expr,
-            input.clone(),
+            Arc::clone(input),
             physical_partition_keys.to_vec(),
         )?) as _))
     }
@@ -594,7 +597,7 @@ pub fn get_window_mode(
     let mut partition_by_reqs: Vec<PhysicalSortRequirement> = vec![];
     let (_, indices) = input_eqs.find_longest_permutation(partitionby_exprs);
     partition_by_reqs.extend(indices.iter().map(|&idx| PhysicalSortRequirement {
-        expr: partitionby_exprs[idx].clone(),
+        expr: Arc::clone(&partitionby_exprs[idx]),
         options: None,
     }));
     // Treat partition by exprs as constant. During analysis of requirements are satisfied.
@@ -694,7 +697,7 @@ mod tests {
         let sort_exprs = sort_exprs.into_iter().collect();
 
         Ok(Arc::new(StreamingTableExec::try_new(
-            schema.clone(),
+            Arc::clone(schema),
             vec![],
             None,
             Some(sort_exprs),
diff --git a/datafusion/physical-plan/src/windows/window_agg_exec.rs b/datafusion/physical-plan/src/windows/window_agg_exec.rs
index eb01da2ec094..b6330f65e0b7 100644
--- a/datafusion/physical-plan/src/windows/window_agg_exec.rs
+++ b/datafusion/physical-plan/src/windows/window_agg_exec.rs
@@ -79,7 +79,7 @@ impl WindowAggExec {
 
         let ordered_partition_by_indices =
             get_ordered_partition_by_indices(window_expr[0].partition_by(), &input);
-        let cache = Self::compute_properties(schema.clone(), &input, &window_expr);
+        let cache = Self::compute_properties(Arc::clone(&schema), &input, &window_expr);
         Ok(Self {
             input,
             window_expr,
@@ -220,7 +220,7 @@ impl ExecutionPlan for WindowAggExec {
     ) -> Result<Arc<dyn ExecutionPlan>> {
         Ok(Arc::new(WindowAggExec::try_new(
             self.window_expr.clone(),
-            children[0].clone(),
+            Arc::clone(&children[0]),
             self.partition_keys.clone(),
         )?))
     }
@@ -232,7 +232,7 @@ impl ExecutionPlan for WindowAggExec {
     ) -> Result<SendableRecordBatchStream> {
         let input = self.input.execute(partition, context)?;
         let stream = Box::pin(WindowAggStream::new(
-            self.schema.clone(),
+            Arc::clone(&self.schema),
             self.window_expr.clone(),
             input,
             BaselineMetrics::new(&self.metrics, partition),
@@ -333,7 +333,7 @@ impl WindowAggStream {
         let _timer = self.baseline_metrics.elapsed_compute().timer();
         let batch = concat_batches(&self.input.schema(), &self.batches)?;
         if batch.num_rows() == 0 {
-            return Ok(RecordBatch::new_empty(self.schema.clone()));
+            return Ok(RecordBatch::new_empty(Arc::clone(&self.schema)));
         }
 
         let partition_by_sort_keys = self
@@ -366,7 +366,10 @@ impl WindowAggStream {
         let mut batch_columns = batch.columns().to_vec();
         // calculate window cols
         batch_columns.extend_from_slice(&columns);
-        Ok(RecordBatch::try_new(self.schema.clone(), batch_columns)?)
+        Ok(RecordBatch::try_new(
+            Arc::clone(&self.schema),
+            batch_columns,
+        )?)
     }
 }
 
@@ -412,6 +415,6 @@ impl WindowAggStream {
 impl RecordBatchStream for WindowAggStream {
     /// Get the schema
     fn schema(&self) -> SchemaRef {
-        self.schema.clone()
+        Arc::clone(&self.schema)
     }
 }
diff --git a/datafusion/physical-plan/src/work_table.rs b/datafusion/physical-plan/src/work_table.rs
index 003957947fec..5f3cf6e2aee8 100644
--- a/datafusion/physical-plan/src/work_table.rs
+++ b/datafusion/physical-plan/src/work_table.rs
@@ -110,7 +110,7 @@ pub struct WorkTableExec {
 impl WorkTableExec {
     /// Create a new execution plan for a worktable exec.
     pub fn new(name: String, schema: SchemaRef) -> Self {
-        let cache = Self::compute_properties(schema.clone());
+        let cache = Self::compute_properties(Arc::clone(&schema));
         Self {
             name,
             schema,
@@ -123,7 +123,7 @@ impl WorkTableExec {
     pub(super) fn with_work_table(&self, work_table: Arc<WorkTable>) -> Self {
         Self {
             name: self.name.clone(),
-            schema: self.schema.clone(),
+            schema: Arc::clone(&self.schema),
             metrics: ExecutionPlanMetricsSet::new(),
             work_table,
             cache: self.cache.clone(),
@@ -185,7 +185,7 @@ impl ExecutionPlan for WorkTableExec {
         self: Arc<Self>,
         _: Vec<Arc<dyn ExecutionPlan>>,
     ) -> Result<Arc<dyn ExecutionPlan>> {
-        Ok(self.clone())
+        Ok(Arc::clone(&self) as Arc<dyn ExecutionPlan>)
     }
 
     /// Stream the batches that were written to the work table.
@@ -202,7 +202,7 @@ impl ExecutionPlan for WorkTableExec {
         }
         let batch = self.work_table.take()?;
         Ok(Box::pin(
-            MemoryStream::try_new(batch.batches, self.schema.clone(), None)?
+            MemoryStream::try_new(batch.batches, Arc::clone(&self.schema), None)?
                 .with_reservation(batch.reservation),
         ))
     }

From 894a8794d11be148cc60db8eac16f105a74d96b1 Mon Sep 17 00:00:00 2001
From: Arttu <Blizzara@users.noreply.github.com>
Date: Mon, 8 Jul 2024 20:30:33 +0200
Subject: [PATCH 2/9] fix: When consuming Substrait, temporarily rename
 clashing duplicate columns (#11329)

* cleanup project internals

* alias intermediate duplicate columns

* fix test

* fix clippy
---
 .../substrait/src/logical_plan/consumer.rs    | 33 ++++++++++++-------
 .../tests/cases/roundtrip_logical_plan.rs     | 16 +++++++++
 2 files changed, 38 insertions(+), 11 deletions(-)

diff --git a/datafusion/substrait/src/logical_plan/consumer.rs b/datafusion/substrait/src/logical_plan/consumer.rs
index 905475eaca7a..77fd5fe44d44 100644
--- a/datafusion/substrait/src/logical_plan/consumer.rs
+++ b/datafusion/substrait/src/logical_plan/consumer.rs
@@ -60,7 +60,7 @@ use datafusion::{
     prelude::{Column, SessionContext},
     scalar::ScalarValue,
 };
-use std::collections::HashMap;
+use std::collections::{HashMap, HashSet};
 use std::str::FromStr;
 use std::sync::Arc;
 use substrait::proto::exchange_rel::ExchangeKind;
@@ -404,22 +404,33 @@ pub async fn from_substrait_rel(
                 let mut input = LogicalPlanBuilder::from(
                     from_substrait_rel(ctx, input, extensions).await?,
                 );
+                let mut names: HashSet<String> = HashSet::new();
                 let mut exprs: Vec<Expr> = vec![];
                 for e in &p.expressions {
                     let x =
                         from_substrait_rex(ctx, e, input.clone().schema(), extensions)
                             .await?;
                     // if the expression is WindowFunction, wrap in a Window relation
-                    //   before returning and do not add to list of this Projection's expression list
-                    // otherwise, add expression to the Projection's expression list
-                    match &*x {
-                        Expr::WindowFunction(_) => {
-                            input = input.window(vec![x.as_ref().clone()])?;
-                            exprs.push(x.as_ref().clone());
-                        }
-                        _ => {
-                            exprs.push(x.as_ref().clone());
-                        }
+                    if let Expr::WindowFunction(_) = x.as_ref() {
+                        // Adding the same expression here and in the project below
+                        // works because the project's builder uses columnize_expr(..)
+                        // to transform it into a column reference
+                        input = input.window(vec![x.as_ref().clone()])?
+                    }
+                    // Ensure the expression has a unique display name, so that project's
+                    // validate_unique_names doesn't fail
+                    let name = x.display_name()?;
+                    let mut new_name = name.clone();
+                    let mut i = 0;
+                    while names.contains(&new_name) {
+                        new_name = format!("{}__temp__{}", name, i);
+                        i += 1;
+                    }
+                    names.insert(new_name.clone());
+                    if new_name != name {
+                        exprs.push(x.as_ref().clone().alias(new_name.clone()));
+                    } else {
+                        exprs.push(x.as_ref().clone());
                     }
                 }
                 input.project(exprs)?.build()
diff --git a/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs b/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs
index 52cfa50683a0..2893b1a31a26 100644
--- a/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs
+++ b/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs
@@ -751,6 +751,22 @@ async fn roundtrip_values_duplicate_column_join() -> Result<()> {
     .await
 }
 
+#[tokio::test]
+async fn duplicate_column() -> Result<()> {
+    // Substrait does not keep column names (aliases) in the plan, rather it operates on column indices
+    // only. DataFusion however, is strict about not having duplicate column names appear in the plan.
+    // This test confirms that we generate aliases for columns in the plan which would otherwise have
+    // colliding names.
+    assert_expected_plan(
+        "SELECT a + 1 as sum_a, a + 1 as sum_a_2 FROM data",
+        "Projection: data.a + Int64(1) AS sum_a, data.a + Int64(1) AS data.a + Int64(1)__temp__0 AS sum_a_2\
+            \n  Projection: data.a + Int64(1)\
+            \n    TableScan: data projection=[a]",
+        true,
+    )
+    .await
+}
+
 /// Construct a plan that cast columns. Only those SQL types are supported for now.
 #[tokio::test]
 async fn new_test_grammar() -> Result<()> {

From 99911449bc470dbefbff78675fbcff743eaa37f4 Mon Sep 17 00:00:00 2001
From: Samuel Colvin <s@muelcolvin.com>
Date: Mon, 8 Jul 2024 20:42:05 +0100
Subject: [PATCH 3/9] Remove any aliases in `Filter::try_new` rather than
 erroring (#11307)

* allow alias in predicate, fix #11306

* Fix typo.

Co-authored-by: Andrew Lamb <andrew@nerdnetworks.org>

* unalise predicate

* use unalias_nested

---------

Co-authored-by: Andrew Lamb <andrew@nerdnetworks.org>
---
 .../user_defined/user_defined_sql_planner.rs  | 37 ++++++++++++++++++-
 datafusion/expr/src/logical_plan/plan.rs      | 16 +++-----
 2 files changed, 41 insertions(+), 12 deletions(-)

diff --git a/datafusion/core/tests/user_defined/user_defined_sql_planner.rs b/datafusion/core/tests/user_defined/user_defined_sql_planner.rs
index 37df7e0900b4..6b660e15a197 100644
--- a/datafusion/core/tests/user_defined/user_defined_sql_planner.rs
+++ b/datafusion/core/tests/user_defined/user_defined_sql_planner.rs
@@ -24,6 +24,8 @@ use datafusion::execution::FunctionRegistry;
 use datafusion::logical_expr::Operator;
 use datafusion::prelude::*;
 use datafusion::sql::sqlparser::ast::BinaryOperator;
+use datafusion_common::ScalarValue;
+use datafusion_expr::expr::Alias;
 use datafusion_expr::planner::{PlannerResult, RawBinaryExpr, UserDefinedSQLPlanner};
 use datafusion_expr::BinaryExpr;
 
@@ -50,13 +52,22 @@ impl UserDefinedSQLPlanner for MyCustomPlanner {
                     op: Operator::Plus,
                 })))
             }
+            BinaryOperator::Question => {
+                Ok(PlannerResult::Planned(Expr::Alias(Alias::new(
+                    Expr::Literal(ScalarValue::Boolean(Some(true))),
+                    None::<&str>,
+                    format!("{} ? {}", expr.left, expr.right),
+                ))))
+            }
             _ => Ok(PlannerResult::Original(expr)),
         }
     }
 }
 
 async fn plan_and_collect(sql: &str) -> Result<Vec<RecordBatch>> {
-    let mut ctx = SessionContext::new();
+    let config =
+        SessionConfig::new().set_str("datafusion.sql_parser.dialect", "postgres");
+    let mut ctx = SessionContext::new_with_config(config);
     ctx.register_user_defined_sql_planner(Arc::new(MyCustomPlanner))?;
     ctx.sql(sql).await?.collect().await
 }
@@ -86,3 +97,27 @@ async fn test_custom_operators_long_arrow() {
     ];
     assert_batches_eq!(&expected, &actual);
 }
+
+#[tokio::test]
+async fn test_question_select() {
+    let actual = plan_and_collect("select a ? 2 from (select 1 as a);")
+        .await
+        .unwrap();
+    let expected = [
+        "+--------------+",
+        "| a ? Int64(2) |",
+        "+--------------+",
+        "| true         |",
+        "+--------------+",
+    ];
+    assert_batches_eq!(&expected, &actual);
+}
+
+#[tokio::test]
+async fn test_question_filter() {
+    let actual = plan_and_collect("select a from (select 1 as a) where a ? 2;")
+        .await
+        .unwrap();
+    let expected = ["+---+", "| a |", "+---+", "| 1 |", "+---+"];
+    assert_batches_eq!(&expected, &actual);
+}
diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs
index 8fd5982a0f2e..bda03fb7087a 100644
--- a/datafusion/expr/src/logical_plan/plan.rs
+++ b/datafusion/expr/src/logical_plan/plan.rs
@@ -25,7 +25,7 @@ use std::sync::Arc;
 use super::dml::CopyTo;
 use super::DdlStatement;
 use crate::builder::{change_redundant_column, unnest_with_options};
-use crate::expr::{Alias, Placeholder, Sort as SortExpr, WindowFunction};
+use crate::expr::{Placeholder, Sort as SortExpr, WindowFunction};
 use crate::expr_rewriter::{create_col_from_scalar_expr, normalize_cols};
 use crate::logical_plan::display::{GraphvizVisitor, IndentVisitor};
 use crate::logical_plan::extension::UserDefinedLogicalNode;
@@ -2130,16 +2130,10 @@ impl Filter {
             }
         }
 
-        // filter predicates should not be aliased
-        if let Expr::Alias(Alias { expr, name, .. }) = predicate {
-            return plan_err!(
-                "Attempted to create Filter predicate with \
-                expression `{expr}` aliased as '{name}'. Filter predicates should not be \
-                aliased."
-            );
-        }
-
-        Ok(Self { predicate, input })
+        Ok(Self {
+            predicate: predicate.unalias_nested().data,
+            input,
+        })
     }
 
     /// Is this filter guaranteed to return 0 or 1 row in a given instantiation?

From 8ae56fc2b8c8b283daa16d540fbbf84dd49e1469 Mon Sep 17 00:00:00 2001
From: Andrew Lamb <andrew@nerdnetworks.org>
Date: Mon, 8 Jul 2024 17:00:10 -0400
Subject: [PATCH 4/9] Improve `DataFrame` Users Guide (#11324)

* Improve `DataFrame` Users Guide

* typo

* Update docs/source/user-guide/dataframe.md

Co-authored-by: Oleks V <comphead@users.noreply.github.com>

---------

Co-authored-by: Oleks V <comphead@users.noreply.github.com>
---
 datafusion/core/src/lib.rs          |   6 ++
 docs/source/user-guide/dataframe.md | 123 +++++++++++-----------------
 2 files changed, 53 insertions(+), 76 deletions(-)

diff --git a/datafusion/core/src/lib.rs b/datafusion/core/src/lib.rs
index fb7abcd795e8..956e9f7246a3 100644
--- a/datafusion/core/src/lib.rs
+++ b/datafusion/core/src/lib.rs
@@ -626,6 +626,12 @@ doc_comment::doctest!(
     user_guide_configs
 );
 
+#[cfg(doctest)]
+doc_comment::doctest!(
+    "../../../docs/source/user-guide/dataframe.md",
+    user_guide_dataframe
+);
+
 #[cfg(doctest)]
 doc_comment::doctest!(
     "../../../docs/source/user-guide/expressions.md",
diff --git a/docs/source/user-guide/dataframe.md b/docs/source/user-guide/dataframe.md
index f011e68fadb2..c3d0b6c2d688 100644
--- a/docs/source/user-guide/dataframe.md
+++ b/docs/source/user-guide/dataframe.md
@@ -19,17 +19,30 @@
 
 # DataFrame API
 
-A DataFrame represents a logical set of rows with the same named columns, similar to a [Pandas DataFrame](https://pandas.pydata.org/pandas-docs/stable/reference/api/pandas.DataFrame.html) or
-[Spark DataFrame](https://spark.apache.org/docs/latest/sql-programming-guide.html).
+A DataFrame represents a logical set of rows with the same named columns,
+similar to a [Pandas DataFrame] or [Spark DataFrame].
 
-DataFrames are typically created by calling a method on
-`SessionContext`, such as `read_csv`, and can then be modified
-by calling the transformation methods, such as `filter`, `select`, `aggregate`, and `limit`
-to build up a query definition.
+DataFrames are typically created by calling a method on [`SessionContext`], such
+as [`read_csv`], and can then be modified by calling the transformation methods,
+such as [`filter`], [`select`], [`aggregate`], and [`limit`] to build up a query
+definition.
 
-The query can be executed by calling the `collect` method.
+The query can be executed by calling the [`collect`] method.
 
-The DataFrame struct is part of DataFusion's prelude and can be imported with the following statement.
+DataFusion DataFrames use lazy evaluation, meaning that each transformation
+creates a new plan but does not actually perform any immediate actions. This
+approach allows for the overall plan to be optimized before execution. The plan
+is evaluated (executed) when an action method is invoked, such as [`collect`].
+See the [Library Users Guide] for more details.
+
+The DataFrame API is well documented in the [API reference on docs.rs].
+Please refer to the [Expressions Reference] for more information on
+building logical expressions (`Expr`) to use with the DataFrame API.
+
+## Example
+
+The DataFrame struct is part of DataFusion's `prelude` and can be imported with
+the following statement.
 
 ```rust
 use datafusion::prelude::*;
@@ -38,73 +51,31 @@ use datafusion::prelude::*;
 Here is a minimal example showing the execution of a query using the DataFrame API.
 
 ```rust
-let ctx = SessionContext::new();
-let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?;
-let df = df.filter(col("a").lt_eq(col("b")))?
-           .aggregate(vec![col("a")], vec![min(col("b"))])?
-           .limit(0, Some(100))?;
-// Print results
-df.show().await?;
+use datafusion::prelude::*;
+use datafusion::error::Result;
+
+#[tokio::main]
+async fn main() -> Result<()> {
+    let ctx = SessionContext::new();
+    let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?;
+    let df = df.filter(col("a").lt_eq(col("b")))?
+        .aggregate(vec![col("a")], vec![min(col("b"))])?
+        .limit(0, Some(100))?;
+    // Print results
+    df.show().await?;
+    Ok(())
+}
 ```
 
-The DataFrame API is well documented in the [API reference on docs.rs](https://docs.rs/datafusion/latest/datafusion/dataframe/struct.DataFrame.html).
-
-Refer to the [Expressions Reference](expressions) for available functions for building logical expressions for use with the
-DataFrame API.
-
-## DataFrame Transformations
-
-These methods create a new DataFrame after applying a transformation to the logical plan that the DataFrame represents.
-
-DataFusion DataFrames use lazy evaluation, meaning that each transformation is just creating a new query plan and
-not actually performing any transformations. This approach allows for the overall plan to be optimized before
-execution. The plan is evaluated (executed) when an action method is invoked, such as `collect`.
-
-| Function            | Notes                                                                                                                                      |
-| ------------------- | ------------------------------------------------------------------------------------------------------------------------------------------ |
-| aggregate           | Perform an aggregate query with optional grouping expressions.                                                                             |
-| distinct            | Filter out duplicate rows.                                                                                                                 |
-| distinct_on         | Filter out duplicate rows based on provided expressions.                                                                                   |
-| drop_columns        | Create a projection with all but the provided column names.                                                                                |
-| except              | Calculate the exception of two DataFrames. The two DataFrames must have exactly the same schema                                            |
-| filter              | Filter a DataFrame to only include rows that match the specified filter expression.                                                        |
-| intersect           | Calculate the intersection of two DataFrames. The two DataFrames must have exactly the same schema                                         |
-| join                | Join this DataFrame with another DataFrame using the specified columns as join keys.                                                       |
-| join_on             | Join this DataFrame with another DataFrame using arbitrary expressions.                                                                    |
-| limit               | Limit the number of rows returned from this DataFrame.                                                                                     |
-| repartition         | Repartition a DataFrame based on a logical partitioning scheme.                                                                            |
-| sort                | Sort the DataFrame by the specified sorting expressions. Any expression can be turned into a sort expression by calling its `sort` method. |
-| select              | Create a projection based on arbitrary expressions. Example: `df.select(vec![col("c1"), abs(col("c2"))])?`                                 |
-| select_columns      | Create a projection based on column names. Example: `df.select_columns(&["id", "name"])?`.                                                 |
-| union               | Calculate the union of two DataFrames, preserving duplicate rows. The two DataFrames must have exactly the same schema.                    |
-| union_distinct      | Calculate the distinct union of two DataFrames. The two DataFrames must have exactly the same schema.                                      |
-| with_column         | Add an additional column to the DataFrame.                                                                                                 |
-| with_column_renamed | Rename one column by applying a new projection.                                                                                            |
-
-## DataFrame Actions
-
-These methods execute the logical plan represented by the DataFrame and either collects the results into memory, prints them to stdout, or writes them to disk.
-
-| Function                   | Notes                                                                                                                       |
-| -------------------------- | --------------------------------------------------------------------------------------------------------------------------- |
-| collect                    | Executes this DataFrame and collects all results into a vector of RecordBatch.                                              |
-| collect_partitioned        | Executes this DataFrame and collects all results into a vector of vector of RecordBatch maintaining the input partitioning. |
-| count                      | Executes this DataFrame to get the total number of rows.                                                                    |
-| execute_stream             | Executes this DataFrame and returns a stream over a single partition.                                                       |
-| execute_stream_partitioned | Executes this DataFrame and returns one stream per partition.                                                               |
-| show                       | Execute this DataFrame and print the results to stdout.                                                                     |
-| show_limit                 | Execute this DataFrame and print a subset of results to stdout.                                                             |
-| write_csv                  | Execute this DataFrame and write the results to disk in CSV format.                                                         |
-| write_json                 | Execute this DataFrame and write the results to disk in JSON format.                                                        |
-| write_parquet              | Execute this DataFrame and write the results to disk in Parquet format.                                                     |
-| write_table                | Execute this DataFrame and write the results via the insert_into method of the registered TableProvider                     |
-
-## Other DataFrame Methods
-
-| Function            | Notes                                                                                                                                                        |
-| ------------------- | ------------------------------------------------------------------------------------------------------------------------------------------------------------ |
-| explain             | Return a DataFrame with the explanation of its plan so far.                                                                                                  |
-| registry            | Return a `FunctionRegistry` used to plan udf's calls.                                                                                                        |
-| schema              | Returns the schema describing the output of this DataFrame in terms of columns returned, where each column has a name, data type, and nullability attribute. |
-| to_logical_plan     | Return the optimized logical plan represented by this DataFrame.                                                                                             |
-| to_unoptimized_plan | Return the unoptimized logical plan represented by this DataFrame.                                                                                           |
+[pandas dataframe]: https://pandas.pydata.org/pandas-docs/stable/reference/api/pandas.DataFrame.html
+[spark dataframe]: https://spark.apache.org/docs/latest/sql-programming-guide.html
+[`sessioncontext`]: https://docs.rs/datafusion/latest/datafusion/execution/context/struct.SessionContext.html
+[`read_csv`]: https://docs.rs/datafusion/latest/datafusion/execution/context/struct.SessionContext.html#method.read_csv
+[`filter`]: https://docs.rs/datafusion/latest/datafusion/dataframe/struct.DataFrame.html#method.filter
+[`select`]: https://docs.rs/datafusion/latest/datafusion/dataframe/struct.DataFrame.html#method.select
+[`aggregate`]: https://docs.rs/datafusion/latest/datafusion/dataframe/struct.DataFrame.html#method.aggregate
+[`limit`]: https://docs.rs/datafusion/latest/datafusion/dataframe/struct.DataFrame.html#method.limit
+[`collect`]: https://docs.rs/datafusion/latest/datafusion/dataframe/struct.DataFrame.html#method.collect
+[library users guide]: ../library-user-guide/using-the-dataframe-api.md
+[api reference on docs.rs]: https://docs.rs/datafusion/latest/datafusion/dataframe/struct.DataFrame.html
+[expressions reference]: expressions

From 782df390078b1aee157d999898424c23530c3eca Mon Sep 17 00:00:00 2001
From: Andy Grove <agrove@apache.org>
Date: Mon, 8 Jul 2024 15:44:06 -0600
Subject: [PATCH 5/9] chore: Rename UserDefinedSQLPlanner to ExprPlanner
 (#11338)

* rename UserDefinedSQLPlanner to ExprPlanner

* Clippy

---------

Co-authored-by: Andrew Lamb <andrew@nerdnetworks.org>
---
 datafusion/core/src/execution/context/mod.rs  | 12 +++++------
 .../core/src/execution/session_state.rs       | 21 +++++++++----------
 ...defined_sql_planner.rs => expr_planner.rs} |  6 +++---
 datafusion/core/tests/user_defined/mod.rs     |  4 ++--
 .../user_defined_scalar_functions.rs          |  4 ++--
 datafusion/execution/src/task.rs              |  4 ++--
 datafusion/expr/src/planner.rs                | 12 +++++------
 datafusion/expr/src/registry.rs               | 16 +++++++-------
 datafusion/functions-array/src/planner.rs     |  6 +++---
 datafusion/functions/src/core/planner.rs      |  4 ++--
 datafusion/functions/src/planner.rs           |  4 ++--
 datafusion/proto/src/bytes/mod.rs             |  4 ++--
 datafusion/proto/src/bytes/registry.rs        |  4 ++--
 datafusion/sql/src/planner.rs                 |  9 +++-----
 14 files changed, 52 insertions(+), 58 deletions(-)
 rename datafusion/core/tests/user_defined/{user_defined_sql_planner.rs => expr_planner.rs} (95%)

diff --git a/datafusion/core/src/execution/context/mod.rs b/datafusion/core/src/execution/context/mod.rs
index 04debf498aa9..4b9e3e843341 100644
--- a/datafusion/core/src/execution/context/mod.rs
+++ b/datafusion/core/src/execution/context/mod.rs
@@ -60,7 +60,7 @@ use datafusion_execution::registry::SerializerRegistry;
 use datafusion_expr::{
     expr_rewriter::FunctionRewrite,
     logical_plan::{DdlStatement, Statement},
-    planner::UserDefinedSQLPlanner,
+    planner::ExprPlanner,
     Expr, UserDefinedLogicalNode, WindowUDF,
 };
 
@@ -1392,17 +1392,15 @@ impl FunctionRegistry for SessionContext {
         self.state.write().register_function_rewrite(rewrite)
     }
 
-    fn expr_planners(&self) -> Vec<Arc<dyn UserDefinedSQLPlanner>> {
+    fn expr_planners(&self) -> Vec<Arc<dyn ExprPlanner>> {
         self.state.read().expr_planners()
     }
 
-    fn register_user_defined_sql_planner(
+    fn register_expr_planner(
         &mut self,
-        user_defined_sql_planner: Arc<dyn UserDefinedSQLPlanner>,
+        expr_planner: Arc<dyn ExprPlanner>,
     ) -> Result<()> {
-        self.state
-            .write()
-            .register_user_defined_sql_planner(user_defined_sql_planner)
+        self.state.write().register_expr_planner(expr_planner)
     }
 }
 
diff --git a/datafusion/core/src/execution/session_state.rs b/datafusion/core/src/execution/session_state.rs
index d056b91c2747..c123ebb22ecb 100644
--- a/datafusion/core/src/execution/session_state.rs
+++ b/datafusion/core/src/execution/session_state.rs
@@ -60,7 +60,7 @@ use datafusion_execution::runtime_env::RuntimeEnv;
 use datafusion_execution::TaskContext;
 use datafusion_expr::execution_props::ExecutionProps;
 use datafusion_expr::expr_rewriter::FunctionRewrite;
-use datafusion_expr::planner::UserDefinedSQLPlanner;
+use datafusion_expr::planner::ExprPlanner;
 use datafusion_expr::registry::{FunctionRegistry, SerializerRegistry};
 use datafusion_expr::simplify::SimplifyInfo;
 use datafusion_expr::var_provider::{is_system_variables, VarType};
@@ -101,7 +101,7 @@ pub struct SessionState {
     /// Responsible for analyzing and rewrite a logical plan before optimization
     analyzer: Analyzer,
     /// Provides support for customising the SQL planner, e.g. to add support for custom operators like `->>` or `?`
-    user_defined_sql_planners: Vec<Arc<dyn UserDefinedSQLPlanner>>,
+    expr_planners: Vec<Arc<dyn ExprPlanner>>,
     /// Responsible for optimizing a logical plan
     optimizer: Optimizer,
     /// Responsible for optimizing a physical execution plan
@@ -231,7 +231,7 @@ impl SessionState {
             );
         }
 
-        let user_defined_sql_planners: Vec<Arc<dyn UserDefinedSQLPlanner>> = vec![
+        let expr_planners: Vec<Arc<dyn ExprPlanner>> = vec![
             Arc::new(functions::core::planner::CoreFunctionPlanner::default()),
             // register crate of array expressions (if enabled)
             #[cfg(feature = "array_expressions")]
@@ -248,7 +248,7 @@ impl SessionState {
         let mut new_self = SessionState {
             session_id,
             analyzer: Analyzer::new(),
-            user_defined_sql_planners,
+            expr_planners,
             optimizer: Optimizer::new(),
             physical_optimizers: PhysicalOptimizer::new(),
             query_planner: Arc::new(DefaultQueryPlanner {}),
@@ -968,7 +968,7 @@ impl SessionState {
         let mut query = SqlToRel::new_with_options(provider, self.get_parser_options());
 
         // custom planners are registered first, so they're run first and take precedence over built-in planners
-        for planner in self.user_defined_sql_planners.iter() {
+        for planner in self.expr_planners.iter() {
             query = query.with_user_defined_planner(planner.clone());
         }
 
@@ -1184,16 +1184,15 @@ impl FunctionRegistry for SessionState {
         Ok(())
     }
 
-    fn expr_planners(&self) -> Vec<Arc<dyn UserDefinedSQLPlanner>> {
-        self.user_defined_sql_planners.clone()
+    fn expr_planners(&self) -> Vec<Arc<dyn ExprPlanner>> {
+        self.expr_planners.clone()
     }
 
-    fn register_user_defined_sql_planner(
+    fn register_expr_planner(
         &mut self,
-        user_defined_sql_planner: Arc<dyn UserDefinedSQLPlanner>,
+        expr_planner: Arc<dyn ExprPlanner>,
     ) -> datafusion_common::Result<()> {
-        self.user_defined_sql_planners
-            .push(user_defined_sql_planner);
+        self.expr_planners.push(expr_planner);
         Ok(())
     }
 }
diff --git a/datafusion/core/tests/user_defined/user_defined_sql_planner.rs b/datafusion/core/tests/user_defined/expr_planner.rs
similarity index 95%
rename from datafusion/core/tests/user_defined/user_defined_sql_planner.rs
rename to datafusion/core/tests/user_defined/expr_planner.rs
index 6b660e15a197..1b23bf9ab2ef 100644
--- a/datafusion/core/tests/user_defined/user_defined_sql_planner.rs
+++ b/datafusion/core/tests/user_defined/expr_planner.rs
@@ -26,12 +26,12 @@ use datafusion::prelude::*;
 use datafusion::sql::sqlparser::ast::BinaryOperator;
 use datafusion_common::ScalarValue;
 use datafusion_expr::expr::Alias;
-use datafusion_expr::planner::{PlannerResult, RawBinaryExpr, UserDefinedSQLPlanner};
+use datafusion_expr::planner::{ExprPlanner, PlannerResult, RawBinaryExpr};
 use datafusion_expr::BinaryExpr;
 
 struct MyCustomPlanner;
 
-impl UserDefinedSQLPlanner for MyCustomPlanner {
+impl ExprPlanner for MyCustomPlanner {
     fn plan_binary_op(
         &self,
         expr: RawBinaryExpr,
@@ -68,7 +68,7 @@ async fn plan_and_collect(sql: &str) -> Result<Vec<RecordBatch>> {
     let config =
         SessionConfig::new().set_str("datafusion.sql_parser.dialect", "postgres");
     let mut ctx = SessionContext::new_with_config(config);
-    ctx.register_user_defined_sql_planner(Arc::new(MyCustomPlanner))?;
+    ctx.register_expr_planner(Arc::new(MyCustomPlanner))?;
     ctx.sql(sql).await?.collect().await
 }
 
diff --git a/datafusion/core/tests/user_defined/mod.rs b/datafusion/core/tests/user_defined/mod.rs
index 9b83a9fdd408..56cec8df468b 100644
--- a/datafusion/core/tests/user_defined/mod.rs
+++ b/datafusion/core/tests/user_defined/mod.rs
@@ -30,5 +30,5 @@ mod user_defined_window_functions;
 /// Tests for User Defined Table Functions
 mod user_defined_table_functions;
 
-/// Tests for User Defined SQL Planner
-mod user_defined_sql_planner;
+/// Tests for Expression Planner
+mod expr_planner;
diff --git a/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs b/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs
index ae8a009c6292..1733068debb9 100644
--- a/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs
+++ b/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs
@@ -575,9 +575,9 @@ async fn test_user_defined_functions_cast_to_i64() -> Result<()> {
 async fn test_user_defined_sql_functions() -> Result<()> {
     let ctx = SessionContext::new();
 
-    let sql_planners = ctx.expr_planners();
+    let expr_planners = ctx.expr_planners();
 
-    assert!(!sql_planners.is_empty());
+    assert!(!expr_planners.is_empty());
 
     Ok(())
 }
diff --git a/datafusion/execution/src/task.rs b/datafusion/execution/src/task.rs
index 24d61e6a8b72..df7fd0dbd92c 100644
--- a/datafusion/execution/src/task.rs
+++ b/datafusion/execution/src/task.rs
@@ -27,7 +27,7 @@ use crate::{
     runtime_env::{RuntimeConfig, RuntimeEnv},
 };
 use datafusion_common::{plan_datafusion_err, DataFusionError, Result};
-use datafusion_expr::planner::UserDefinedSQLPlanner;
+use datafusion_expr::planner::ExprPlanner;
 use datafusion_expr::{AggregateUDF, ScalarUDF, WindowUDF};
 
 /// Task Execution Context
@@ -192,7 +192,7 @@ impl FunctionRegistry for TaskContext {
         Ok(self.scalar_functions.insert(udf.name().into(), udf))
     }
 
-    fn expr_planners(&self) -> Vec<Arc<dyn UserDefinedSQLPlanner>> {
+    fn expr_planners(&self) -> Vec<Arc<dyn ExprPlanner>> {
         vec![]
     }
 }
diff --git a/datafusion/expr/src/planner.rs b/datafusion/expr/src/planner.rs
index c255edbea5ae..aeb8ed8372b7 100644
--- a/datafusion/expr/src/planner.rs
+++ b/datafusion/expr/src/planner.rs
@@ -15,7 +15,7 @@
 // specific language governing permissions and limitations
 // under the License.
 
-//! [`ContextProvider`] and [`UserDefinedSQLPlanner`] APIs to customize SQL query planning
+//! [`ContextProvider`] and [`ExprPlanner`] APIs to customize SQL query planning
 
 use std::sync::Arc;
 
@@ -83,7 +83,7 @@ pub trait ContextProvider {
 }
 
 /// This trait allows users to customize the behavior of the SQL planner
-pub trait UserDefinedSQLPlanner: Send + Sync {
+pub trait ExprPlanner: Send + Sync {
     /// Plan the binary operation between two expressions, returns original
     /// BinaryExpr if not possible
     fn plan_binary_op(
@@ -168,7 +168,7 @@ pub trait UserDefinedSQLPlanner: Send + Sync {
 /// Note `left` and `right` are DataFusion [`Expr`]s but the `op` is the SQL AST
 /// operator.
 ///
-/// This structure is used by [`UserDefinedSQLPlanner`] to plan operators with
+/// This structure is used by [`ExprPlanner`] to plan operators with
 /// custom expressions.
 #[derive(Debug, Clone)]
 pub struct RawBinaryExpr {
@@ -179,7 +179,7 @@ pub struct RawBinaryExpr {
 
 /// An expression with GetFieldAccess to plan
 ///
-/// This structure is used by [`UserDefinedSQLPlanner`] to plan operators with
+/// This structure is used by [`ExprPlanner`] to plan operators with
 /// custom expressions.
 #[derive(Debug, Clone)]
 pub struct RawFieldAccessExpr {
@@ -189,7 +189,7 @@ pub struct RawFieldAccessExpr {
 
 /// A Dictionary literal expression `{ key: value, ...}`
 ///
-/// This structure is used by [`UserDefinedSQLPlanner`] to plan operators with
+/// This structure is used by [`ExprPlanner`] to plan operators with
 /// custom expressions.
 #[derive(Debug, Clone)]
 pub struct RawDictionaryExpr {
@@ -197,7 +197,7 @@ pub struct RawDictionaryExpr {
     pub values: Vec<Expr>,
 }
 
-/// Result of planning a raw expr with [`UserDefinedSQLPlanner`]
+/// Result of planning a raw expr with [`ExprPlanner`]
 #[derive(Debug, Clone)]
 pub enum PlannerResult<T> {
     /// The raw expression was successfully planned as a new [`Expr`]
diff --git a/datafusion/expr/src/registry.rs b/datafusion/expr/src/registry.rs
index 6a27c05bb451..988dc0f5aeda 100644
--- a/datafusion/expr/src/registry.rs
+++ b/datafusion/expr/src/registry.rs
@@ -18,7 +18,7 @@
 //! FunctionRegistry trait
 
 use crate::expr_rewriter::FunctionRewrite;
-use crate::planner::UserDefinedSQLPlanner;
+use crate::planner::ExprPlanner;
 use crate::{AggregateUDF, ScalarUDF, UserDefinedLogicalNode, WindowUDF};
 use datafusion_common::{not_impl_err, plan_datafusion_err, Result};
 use std::collections::HashMap;
@@ -110,15 +110,15 @@ pub trait FunctionRegistry {
         not_impl_err!("Registering FunctionRewrite")
     }
 
-    /// Set of all registered [`UserDefinedSQLPlanner`]s
-    fn expr_planners(&self) -> Vec<Arc<dyn UserDefinedSQLPlanner>>;
+    /// Set of all registered [`ExprPlanner`]s
+    fn expr_planners(&self) -> Vec<Arc<dyn ExprPlanner>>;
 
-    /// Registers a new [`UserDefinedSQLPlanner`] with the registry.
-    fn register_user_defined_sql_planner(
+    /// Registers a new [`ExprPlanner`] with the registry.
+    fn register_expr_planner(
         &mut self,
-        _user_defined_sql_planner: Arc<dyn UserDefinedSQLPlanner>,
+        _expr_planner: Arc<dyn ExprPlanner>,
     ) -> Result<()> {
-        not_impl_err!("Registering UserDefinedSQLPlanner")
+        not_impl_err!("Registering ExprPlanner")
     }
 }
 
@@ -196,7 +196,7 @@ impl FunctionRegistry for MemoryFunctionRegistry {
         Ok(self.udwfs.insert(udaf.name().into(), udaf))
     }
 
-    fn expr_planners(&self) -> Vec<Arc<dyn UserDefinedSQLPlanner>> {
+    fn expr_planners(&self) -> Vec<Arc<dyn ExprPlanner>> {
         vec![]
     }
 }
diff --git a/datafusion/functions-array/src/planner.rs b/datafusion/functions-array/src/planner.rs
index 01853fb56908..cfbe99b4b7fd 100644
--- a/datafusion/functions-array/src/planner.rs
+++ b/datafusion/functions-array/src/planner.rs
@@ -19,7 +19,7 @@
 
 use datafusion_common::{utils::list_ndims, DFSchema, Result};
 use datafusion_expr::{
-    planner::{PlannerResult, RawBinaryExpr, RawFieldAccessExpr, UserDefinedSQLPlanner},
+    planner::{ExprPlanner, PlannerResult, RawBinaryExpr, RawFieldAccessExpr},
     sqlparser, AggregateFunction, Expr, ExprSchemable, GetFieldAccess,
 };
 use datafusion_functions::expr_fn::get_field;
@@ -34,7 +34,7 @@ use crate::{
 
 pub struct ArrayFunctionPlanner;
 
-impl UserDefinedSQLPlanner for ArrayFunctionPlanner {
+impl ExprPlanner for ArrayFunctionPlanner {
     fn plan_binary_op(
         &self,
         expr: RawBinaryExpr,
@@ -101,7 +101,7 @@ impl UserDefinedSQLPlanner for ArrayFunctionPlanner {
 
 pub struct FieldAccessPlanner;
 
-impl UserDefinedSQLPlanner for FieldAccessPlanner {
+impl ExprPlanner for FieldAccessPlanner {
     fn plan_field_access(
         &self,
         expr: RawFieldAccessExpr,
diff --git a/datafusion/functions/src/core/planner.rs b/datafusion/functions/src/core/planner.rs
index e803c92dd0b3..748b598d292f 100644
--- a/datafusion/functions/src/core/planner.rs
+++ b/datafusion/functions/src/core/planner.rs
@@ -18,7 +18,7 @@
 use datafusion_common::DFSchema;
 use datafusion_common::Result;
 use datafusion_expr::expr::ScalarFunction;
-use datafusion_expr::planner::{PlannerResult, RawDictionaryExpr, UserDefinedSQLPlanner};
+use datafusion_expr::planner::{ExprPlanner, PlannerResult, RawDictionaryExpr};
 use datafusion_expr::Expr;
 
 use super::named_struct;
@@ -26,7 +26,7 @@ use super::named_struct;
 #[derive(Default)]
 pub struct CoreFunctionPlanner {}
 
-impl UserDefinedSQLPlanner for CoreFunctionPlanner {
+impl ExprPlanner for CoreFunctionPlanner {
     fn plan_dictionary_literal(
         &self,
         expr: RawDictionaryExpr,
diff --git a/datafusion/functions/src/planner.rs b/datafusion/functions/src/planner.rs
index 41ff92f26111..ad42c5edd6e6 100644
--- a/datafusion/functions/src/planner.rs
+++ b/datafusion/functions/src/planner.rs
@@ -20,14 +20,14 @@
 use datafusion_common::Result;
 use datafusion_expr::{
     expr::ScalarFunction,
-    planner::{PlannerResult, UserDefinedSQLPlanner},
+    planner::{ExprPlanner, PlannerResult},
     Expr,
 };
 
 #[derive(Default)]
 pub struct UserDefinedFunctionPlanner;
 
-impl UserDefinedSQLPlanner for UserDefinedFunctionPlanner {
+impl ExprPlanner for UserDefinedFunctionPlanner {
     #[cfg(feature = "datetime_expressions")]
     fn plan_extract(&self, args: Vec<Expr>) -> Result<PlannerResult<Vec<Expr>>> {
         Ok(PlannerResult::Planned(Expr::ScalarFunction(
diff --git a/datafusion/proto/src/bytes/mod.rs b/datafusion/proto/src/bytes/mod.rs
index 83210cb4e41f..9188480431aa 100644
--- a/datafusion/proto/src/bytes/mod.rs
+++ b/datafusion/proto/src/bytes/mod.rs
@@ -39,7 +39,7 @@ use std::sync::Arc;
 use datafusion::execution::registry::FunctionRegistry;
 use datafusion::physical_plan::ExecutionPlan;
 use datafusion::prelude::SessionContext;
-use datafusion_expr::planner::UserDefinedSQLPlanner;
+use datafusion_expr::planner::ExprPlanner;
 
 mod registry;
 
@@ -167,7 +167,7 @@ impl Serializeable for Expr {
                 )
             }
 
-            fn expr_planners(&self) -> Vec<Arc<dyn UserDefinedSQLPlanner>> {
+            fn expr_planners(&self) -> Vec<Arc<dyn ExprPlanner>> {
                 vec![]
             }
         }
diff --git a/datafusion/proto/src/bytes/registry.rs b/datafusion/proto/src/bytes/registry.rs
index 075993e2ba76..eae2425f8ac1 100644
--- a/datafusion/proto/src/bytes/registry.rs
+++ b/datafusion/proto/src/bytes/registry.rs
@@ -20,7 +20,7 @@ use std::{collections::HashSet, sync::Arc};
 use datafusion::execution::registry::FunctionRegistry;
 use datafusion_common::plan_err;
 use datafusion_common::Result;
-use datafusion_expr::planner::UserDefinedSQLPlanner;
+use datafusion_expr::planner::ExprPlanner;
 use datafusion_expr::{AggregateUDF, ScalarUDF, WindowUDF};
 
 /// A default [`FunctionRegistry`] registry that does not resolve any
@@ -56,7 +56,7 @@ impl FunctionRegistry for NoRegistry {
         plan_err!("No function registry provided to deserialize, so can not deserialize User Defined Window Function '{}'", udwf.inner().name())
     }
 
-    fn expr_planners(&self) -> Vec<Arc<dyn UserDefinedSQLPlanner>> {
+    fn expr_planners(&self) -> Vec<Arc<dyn ExprPlanner>> {
         vec![]
     }
 }
diff --git a/datafusion/sql/src/planner.rs b/datafusion/sql/src/planner.rs
index 443cd64a940c..a77f0003f738 100644
--- a/datafusion/sql/src/planner.rs
+++ b/datafusion/sql/src/planner.rs
@@ -24,7 +24,7 @@ use arrow_schema::*;
 use datafusion_common::{
     field_not_found, internal_err, plan_datafusion_err, DFSchemaRef, SchemaError,
 };
-use datafusion_expr::planner::UserDefinedSQLPlanner;
+use datafusion_expr::planner::ExprPlanner;
 use sqlparser::ast::TimezoneInfo;
 use sqlparser::ast::{ArrayElemTypeDef, ExactNumberInfo};
 use sqlparser::ast::{ColumnDef as SQLColumnDef, ColumnOption};
@@ -187,7 +187,7 @@ pub struct SqlToRel<'a, S: ContextProvider> {
     pub(crate) options: ParserOptions,
     pub(crate) normalizer: IdentNormalizer,
     /// user defined planner extensions
-    pub(crate) planners: Vec<Arc<dyn UserDefinedSQLPlanner>>,
+    pub(crate) planners: Vec<Arc<dyn ExprPlanner>>,
 }
 
 impl<'a, S: ContextProvider> SqlToRel<'a, S> {
@@ -197,10 +197,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> {
     }
 
     /// add an user defined planner
-    pub fn with_user_defined_planner(
-        mut self,
-        planner: Arc<dyn UserDefinedSQLPlanner>,
-    ) -> Self {
+    pub fn with_user_defined_planner(mut self, planner: Arc<dyn ExprPlanner>) -> Self {
         self.planners.push(planner);
         self
     }

From a6898d36c4bde7ab77d5cc912a6aff5eb620df77 Mon Sep 17 00:00:00 2001
From: Andrew Lamb <andrew@nerdnetworks.org>
Date: Mon, 8 Jul 2024 20:27:58 -0400
Subject: [PATCH 6/9] Revert "remove `derive(Copy)` from `Operator` (#11132)"
 (#11341)

This reverts commit b468ba788310d5dcd6ea4549f0d5b0f215ea49c9.
---
 .../core/src/physical_optimizer/pruning.rs    | 24 +++++++------------
 datafusion/expr/src/operator.rs               |  2 +-
 datafusion/expr/src/type_coercion/binary.rs   |  4 ++--
 datafusion/expr/src/utils.rs                  | 22 ++++++++---------
 .../optimizer/src/analyzer/type_coercion.rs   |  8 +++----
 .../src/simplify_expressions/utils.rs         |  8 +++----
 datafusion/optimizer/src/utils.rs             |  6 ++---
 datafusion/physical-expr-common/src/datum.rs  |  4 ++--
 .../physical-expr/src/expressions/binary.rs   |  4 ++--
 .../physical-expr/src/intervals/cp_solver.rs  |  2 +-
 .../physical-expr/src/intervals/utils.rs      |  2 +-
 datafusion/physical-expr/src/planner.rs       |  2 +-
 datafusion/physical-expr/src/utils/mod.rs     |  8 +++----
 .../physical-plan/src/joins/hash_join.rs      |  2 +-
 .../proto/src/logical_plan/from_proto.rs      |  6 +----
 .../substrait/src/logical_plan/consumer.rs    |  2 +-
 .../substrait/src/logical_plan/producer.rs    | 14 ++---------
 17 files changed, 50 insertions(+), 70 deletions(-)

diff --git a/datafusion/core/src/physical_optimizer/pruning.rs b/datafusion/core/src/physical_optimizer/pruning.rs
index e8f2f34abda0..a7ce29bdc7e3 100644
--- a/datafusion/core/src/physical_optimizer/pruning.rs
+++ b/datafusion/core/src/physical_optimizer/pruning.rs
@@ -987,8 +987,8 @@ impl<'a> PruningExpressionBuilder<'a> {
         })
     }
 
-    fn op(&self) -> &Operator {
-        &self.op
+    fn op(&self) -> Operator {
+        self.op
     }
 
     fn scalar_expr(&self) -> &Arc<dyn PhysicalExpr> {
@@ -1064,7 +1064,7 @@ fn rewrite_expr_to_prunable(
     scalar_expr: &PhysicalExprRef,
     schema: DFSchema,
 ) -> Result<(PhysicalExprRef, Operator, PhysicalExprRef)> {
-    if !is_compare_op(&op) {
+    if !is_compare_op(op) {
         return plan_err!("rewrite_expr_to_prunable only support compare expression");
     }
 
@@ -1131,7 +1131,7 @@ fn rewrite_expr_to_prunable(
     }
 }
 
-fn is_compare_op(op: &Operator) -> bool {
+fn is_compare_op(op: Operator) -> bool {
     matches!(
         op,
         Operator::Eq
@@ -1358,13 +1358,11 @@ fn build_predicate_expression(
                 .map(|e| {
                     Arc::new(phys_expr::BinaryExpr::new(
                         in_list.expr().clone(),
-                        eq_op.clone(),
+                        eq_op,
                         e.clone(),
                     )) as _
                 })
-                .reduce(|a, b| {
-                    Arc::new(phys_expr::BinaryExpr::new(a, re_op.clone(), b)) as _
-                })
+                .reduce(|a, b| Arc::new(phys_expr::BinaryExpr::new(a, re_op, b)) as _)
                 .unwrap();
             return build_predicate_expression(&change_expr, schema, required_columns);
         } else {
@@ -1376,7 +1374,7 @@ fn build_predicate_expression(
         if let Some(bin_expr) = expr_any.downcast_ref::<phys_expr::BinaryExpr>() {
             (
                 bin_expr.left().clone(),
-                bin_expr.op().clone(),
+                *bin_expr.op(),
                 bin_expr.right().clone(),
             )
         } else {
@@ -1388,7 +1386,7 @@ fn build_predicate_expression(
         let left_expr = build_predicate_expression(&left, schema, required_columns);
         let right_expr = build_predicate_expression(&right, schema, required_columns);
         // simplify boolean expression if applicable
-        let expr = match (&left_expr, &op, &right_expr) {
+        let expr = match (&left_expr, op, &right_expr) {
             (left, Operator::And, _) if is_always_true(left) => right_expr,
             (_, Operator::And, right) if is_always_true(right) => left_expr,
             (left, Operator::Or, right)
@@ -1396,11 +1394,7 @@ fn build_predicate_expression(
             {
                 unhandled
             }
-            _ => Arc::new(phys_expr::BinaryExpr::new(
-                left_expr,
-                op.clone(),
-                right_expr,
-            )),
+            _ => Arc::new(phys_expr::BinaryExpr::new(left_expr, op, right_expr)),
         };
         return expr;
     }
diff --git a/datafusion/expr/src/operator.rs b/datafusion/expr/src/operator.rs
index 742511822a0f..a10312e23446 100644
--- a/datafusion/expr/src/operator.rs
+++ b/datafusion/expr/src/operator.rs
@@ -25,7 +25,7 @@ use std::ops;
 use std::ops::Not;
 
 /// Operators applied to expressions
-#[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Hash)]
+#[derive(Debug, Copy, Clone, PartialEq, Eq, PartialOrd, Hash)]
 pub enum Operator {
     /// Expressions are equal
     Eq,
diff --git a/datafusion/expr/src/type_coercion/binary.rs b/datafusion/expr/src/type_coercion/binary.rs
index 83a7da046844..4f79f3fa2b22 100644
--- a/datafusion/expr/src/type_coercion/binary.rs
+++ b/datafusion/expr/src/type_coercion/binary.rs
@@ -1152,8 +1152,8 @@ mod tests {
         ];
         for (i, input_type) in input_types.iter().enumerate() {
             let expect_type = &result_types[i];
-            for op in &comparison_op_types {
-                let (lhs, rhs) = get_input_types(&input_decimal, op, input_type)?;
+            for op in comparison_op_types {
+                let (lhs, rhs) = get_input_types(&input_decimal, &op, input_type)?;
                 assert_eq!(expect_type, &lhs);
                 assert_eq!(expect_type, &rhs);
             }
diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs
index 34e007207427..45155cbd2c27 100644
--- a/datafusion/expr/src/utils.rs
+++ b/datafusion/expr/src/utils.rs
@@ -997,7 +997,7 @@ fn split_conjunction_impl<'a>(expr: &'a Expr, mut exprs: Vec<&'a Expr>) -> Vec<&
 /// assert_eq!(split_conjunction_owned(expr), split);
 /// ```
 pub fn split_conjunction_owned(expr: Expr) -> Vec<Expr> {
-    split_binary_owned(expr, &Operator::And)
+    split_binary_owned(expr, Operator::And)
 }
 
 /// Splits an owned binary operator tree [`Expr`] such as `A <OP> B <OP> C` => `[A, B, C]`
@@ -1020,19 +1020,19 @@ pub fn split_conjunction_owned(expr: Expr) -> Vec<Expr> {
 /// ];
 ///
 /// // use split_binary_owned to split them
-/// assert_eq!(split_binary_owned(expr, &Operator::Plus), split);
+/// assert_eq!(split_binary_owned(expr, Operator::Plus), split);
 /// ```
-pub fn split_binary_owned(expr: Expr, op: &Operator) -> Vec<Expr> {
+pub fn split_binary_owned(expr: Expr, op: Operator) -> Vec<Expr> {
     split_binary_owned_impl(expr, op, vec![])
 }
 
 fn split_binary_owned_impl(
     expr: Expr,
-    operator: &Operator,
+    operator: Operator,
     mut exprs: Vec<Expr>,
 ) -> Vec<Expr> {
     match expr {
-        Expr::BinaryExpr(BinaryExpr { right, op, left }) if &op == operator => {
+        Expr::BinaryExpr(BinaryExpr { right, op, left }) if op == operator => {
             let exprs = split_binary_owned_impl(*left, operator, exprs);
             split_binary_owned_impl(*right, operator, exprs)
         }
@@ -1049,17 +1049,17 @@ fn split_binary_owned_impl(
 /// Splits an binary operator tree [`Expr`] such as `A <OP> B <OP> C` => `[A, B, C]`
 ///
 /// See [`split_binary_owned`] for more details and an example.
-pub fn split_binary<'a>(expr: &'a Expr, op: &Operator) -> Vec<&'a Expr> {
+pub fn split_binary(expr: &Expr, op: Operator) -> Vec<&Expr> {
     split_binary_impl(expr, op, vec![])
 }
 
 fn split_binary_impl<'a>(
     expr: &'a Expr,
-    operator: &Operator,
+    operator: Operator,
     mut exprs: Vec<&'a Expr>,
 ) -> Vec<&'a Expr> {
     match expr {
-        Expr::BinaryExpr(BinaryExpr { right, op, left }) if op == operator => {
+        Expr::BinaryExpr(BinaryExpr { right, op, left }) if *op == operator => {
             let exprs = split_binary_impl(left, operator, exprs);
             split_binary_impl(right, operator, exprs)
         }
@@ -1613,13 +1613,13 @@ mod tests {
     #[test]
     fn test_split_binary_owned() {
         let expr = col("a");
-        assert_eq!(split_binary_owned(expr.clone(), &Operator::And), vec![expr]);
+        assert_eq!(split_binary_owned(expr.clone(), Operator::And), vec![expr]);
     }
 
     #[test]
     fn test_split_binary_owned_two() {
         assert_eq!(
-            split_binary_owned(col("a").eq(lit(5)).and(col("b")), &Operator::And),
+            split_binary_owned(col("a").eq(lit(5)).and(col("b")), Operator::And),
             vec![col("a").eq(lit(5)), col("b")]
         );
     }
@@ -1629,7 +1629,7 @@ mod tests {
         let expr = col("a").eq(lit(5)).or(col("b"));
         assert_eq!(
             // expr is connected by OR, but pass in AND
-            split_binary_owned(expr.clone(), &Operator::And),
+            split_binary_owned(expr.clone(), Operator::And),
             vec![expr]
         );
     }
diff --git a/datafusion/optimizer/src/analyzer/type_coercion.rs b/datafusion/optimizer/src/analyzer/type_coercion.rs
index 64d9c508f3f6..6c08b3e998b3 100644
--- a/datafusion/optimizer/src/analyzer/type_coercion.rs
+++ b/datafusion/optimizer/src/analyzer/type_coercion.rs
@@ -146,7 +146,7 @@ impl<'a> TypeCoercionRewriter<'a> {
             .map(|(lhs, rhs)| {
                 // coerce the arguments as though they were a single binary equality
                 // expression
-                let (lhs, rhs) = self.coerce_binary_op(lhs, &Operator::Eq, rhs)?;
+                let (lhs, rhs) = self.coerce_binary_op(lhs, Operator::Eq, rhs)?;
                 Ok((lhs, rhs))
             })
             .collect::<Result<Vec<_>>>()?;
@@ -157,12 +157,12 @@ impl<'a> TypeCoercionRewriter<'a> {
     fn coerce_binary_op(
         &self,
         left: Expr,
-        op: &Operator,
+        op: Operator,
         right: Expr,
     ) -> Result<(Expr, Expr)> {
         let (left_type, right_type) = get_input_types(
             &left.get_type(self.schema)?,
-            op,
+            &op,
             &right.get_type(self.schema)?,
         )?;
         Ok((
@@ -279,7 +279,7 @@ impl<'a> TreeNodeRewriter for TypeCoercionRewriter<'a> {
                 ))))
             }
             Expr::BinaryExpr(BinaryExpr { left, op, right }) => {
-                let (left, right) = self.coerce_binary_op(*left, &op, *right)?;
+                let (left, right) = self.coerce_binary_op(*left, op, *right)?;
                 Ok(Transformed::yes(Expr::BinaryExpr(BinaryExpr::new(
                     Box::new(left),
                     op,
diff --git a/datafusion/optimizer/src/simplify_expressions/utils.rs b/datafusion/optimizer/src/simplify_expressions/utils.rs
index ed3fd75f3efd..5da727cb5990 100644
--- a/datafusion/optimizer/src/simplify_expressions/utils.rs
+++ b/datafusion/optimizer/src/simplify_expressions/utils.rs
@@ -69,8 +69,8 @@ pub static POWS_OF_TEN: [i128; 38] = [
 /// expressions. Such as: (A AND B) AND C
 pub fn expr_contains(expr: &Expr, needle: &Expr, search_op: Operator) -> bool {
     match expr {
-        Expr::BinaryExpr(BinaryExpr { left, op, right }) if op == &search_op => {
-            expr_contains(left, needle, search_op.clone())
+        Expr::BinaryExpr(BinaryExpr { left, op, right }) if *op == search_op => {
+            expr_contains(left, needle, search_op)
                 || expr_contains(right, needle, search_op)
         }
         _ => expr == needle,
@@ -88,7 +88,7 @@ pub fn delete_xor_in_complex_expr(expr: &Expr, needle: &Expr, is_left: bool) ->
     ) -> Expr {
         match expr {
             Expr::BinaryExpr(BinaryExpr { left, op, right })
-                if op == &Operator::BitwiseXor =>
+                if *op == Operator::BitwiseXor =>
             {
                 let left_expr = recursive_delete_xor_in_expr(left, needle, xor_counter);
                 let right_expr = recursive_delete_xor_in_expr(right, needle, xor_counter);
@@ -102,7 +102,7 @@ pub fn delete_xor_in_complex_expr(expr: &Expr, needle: &Expr, is_left: bool) ->
 
                 Expr::BinaryExpr(BinaryExpr::new(
                     Box::new(left_expr),
-                    op.clone(),
+                    *op,
                     Box::new(right_expr),
                 ))
             }
diff --git a/datafusion/optimizer/src/utils.rs b/datafusion/optimizer/src/utils.rs
index 0549845430a6..05b1744d90c5 100644
--- a/datafusion/optimizer/src/utils.rs
+++ b/datafusion/optimizer/src/utils.rs
@@ -177,13 +177,13 @@ pub fn split_conjunction_owned(expr: Expr) -> Vec<Expr> {
 /// ];
 ///
 /// // use split_binary_owned to split them
-/// assert_eq!(split_binary_owned(expr, &Operator::Plus), split);
+/// assert_eq!(split_binary_owned(expr, Operator::Plus), split);
 /// ```
 #[deprecated(
     since = "34.0.0",
     note = "use `datafusion_expr::utils::split_binary_owned` instead"
 )]
-pub fn split_binary_owned(expr: Expr, op: &Operator) -> Vec<Expr> {
+pub fn split_binary_owned(expr: Expr, op: Operator) -> Vec<Expr> {
     expr_utils::split_binary_owned(expr, op)
 }
 
@@ -194,7 +194,7 @@ pub fn split_binary_owned(expr: Expr, op: &Operator) -> Vec<Expr> {
     since = "34.0.0",
     note = "use `datafusion_expr::utils::split_binary` instead"
 )]
-pub fn split_binary<'a>(expr: &'a Expr, op: &Operator) -> Vec<&'a Expr> {
+pub fn split_binary(expr: &Expr, op: Operator) -> Vec<&Expr> {
     expr_utils::split_binary(expr, op)
 }
 
diff --git a/datafusion/physical-expr-common/src/datum.rs b/datafusion/physical-expr-common/src/datum.rs
index 790e742c4221..d0ba5f113b6f 100644
--- a/datafusion/physical-expr-common/src/datum.rs
+++ b/datafusion/physical-expr-common/src/datum.rs
@@ -63,7 +63,7 @@ pub fn apply_cmp(
 /// Applies a binary [`Datum`] comparison kernel `f` to `lhs` and `rhs` for nested type like
 /// List, FixedSizeList, LargeList, Struct, Union, Map, or a dictionary of a nested type
 pub fn apply_cmp_for_nested(
-    op: &Operator,
+    op: Operator,
     lhs: &ColumnarValue,
     rhs: &ColumnarValue,
 ) -> Result<ColumnarValue> {
@@ -88,7 +88,7 @@ pub fn apply_cmp_for_nested(
 
 /// Compare on nested type List, Struct, and so on
 pub fn compare_op_for_nested(
-    op: &Operator,
+    op: Operator,
     lhs: &dyn Datum,
     rhs: &dyn Datum,
 ) -> Result<BooleanArray> {
diff --git a/datafusion/physical-expr/src/expressions/binary.rs b/datafusion/physical-expr/src/expressions/binary.rs
index f1e40575bc64..c153ead9639f 100644
--- a/datafusion/physical-expr/src/expressions/binary.rs
+++ b/datafusion/physical-expr/src/expressions/binary.rs
@@ -269,7 +269,7 @@ impl PhysicalExpr for BinaryExpr {
             if right_data_type != left_data_type {
                 return internal_err!("type mismatch");
             }
-            return apply_cmp_for_nested(&self.op, &lhs, &rhs);
+            return apply_cmp_for_nested(self.op, &lhs, &rhs);
         }
 
         match self.op {
@@ -329,7 +329,7 @@ impl PhysicalExpr for BinaryExpr {
     ) -> Result<Arc<dyn PhysicalExpr>> {
         Ok(Arc::new(BinaryExpr::new(
             Arc::clone(&children[0]),
-            self.op.clone(),
+            self.op,
             Arc::clone(&children[1]),
         )))
     }
diff --git a/datafusion/physical-expr/src/intervals/cp_solver.rs b/datafusion/physical-expr/src/intervals/cp_solver.rs
index ef9dd36cfb50..fc4950ae4e7c 100644
--- a/datafusion/physical-expr/src/intervals/cp_solver.rs
+++ b/datafusion/physical-expr/src/intervals/cp_solver.rs
@@ -222,7 +222,7 @@ pub fn propagate_arithmetic(
     left_child: &Interval,
     right_child: &Interval,
 ) -> Result<Option<(Interval, Interval)>> {
-    let inverse_op = get_inverse_op(op)?;
+    let inverse_op = get_inverse_op(*op)?;
     match (left_child.data_type(), right_child.data_type()) {
         // If we have a child whose type is a time interval (i.e. DataType::Interval),
         // we need special handling since timestamp differencing results in a
diff --git a/datafusion/physical-expr/src/intervals/utils.rs b/datafusion/physical-expr/src/intervals/utils.rs
index 37527802f84d..b426a656fba9 100644
--- a/datafusion/physical-expr/src/intervals/utils.rs
+++ b/datafusion/physical-expr/src/intervals/utils.rs
@@ -63,7 +63,7 @@ pub fn check_support(expr: &Arc<dyn PhysicalExpr>, schema: &SchemaRef) -> bool {
 }
 
 // This function returns the inverse operator of the given operator.
-pub fn get_inverse_op(op: &Operator) -> Result<Operator> {
+pub fn get_inverse_op(op: Operator) -> Result<Operator> {
     match op {
         Operator::Plus => Ok(Operator::Minus),
         Operator::Minus => Ok(Operator::Plus),
diff --git a/datafusion/physical-expr/src/planner.rs b/datafusion/physical-expr/src/planner.rs
index dbebf4c18b79..a975f0c6ef83 100644
--- a/datafusion/physical-expr/src/planner.rs
+++ b/datafusion/physical-expr/src/planner.rs
@@ -195,7 +195,7 @@ pub fn create_physical_expr(
             //
             // There should be no coercion during physical
             // planning.
-            binary(lhs, op.clone(), rhs, input_schema)
+            binary(lhs, *op, rhs, input_schema)
         }
         Expr::Like(Like {
             negated,
diff --git a/datafusion/physical-expr/src/utils/mod.rs b/datafusion/physical-expr/src/utils/mod.rs
index a33f65f92a61..6c4791b158c8 100644
--- a/datafusion/physical-expr/src/utils/mod.rs
+++ b/datafusion/physical-expr/src/utils/mod.rs
@@ -44,7 +44,7 @@ use petgraph::stable_graph::StableGraph;
 pub fn split_conjunction(
     predicate: &Arc<dyn PhysicalExpr>,
 ) -> Vec<&Arc<dyn PhysicalExpr>> {
-    split_impl(&Operator::And, predicate, vec![])
+    split_impl(Operator::And, predicate, vec![])
 }
 
 /// Assume the predicate is in the form of DNF, split the predicate to a Vec of PhysicalExprs.
@@ -53,16 +53,16 @@ pub fn split_conjunction(
 pub fn split_disjunction(
     predicate: &Arc<dyn PhysicalExpr>,
 ) -> Vec<&Arc<dyn PhysicalExpr>> {
-    split_impl(&Operator::Or, predicate, vec![])
+    split_impl(Operator::Or, predicate, vec![])
 }
 
 fn split_impl<'a>(
-    operator: &Operator,
+    operator: Operator,
     predicate: &'a Arc<dyn PhysicalExpr>,
     mut exprs: Vec<&'a Arc<dyn PhysicalExpr>>,
 ) -> Vec<&'a Arc<dyn PhysicalExpr>> {
     match predicate.as_any().downcast_ref::<BinaryExpr>() {
-        Some(binary) if binary.op() == operator => {
+        Some(binary) if binary.op() == &operator => {
             let exprs = split_impl(operator, binary.left(), exprs);
             split_impl(operator, binary.right(), exprs)
         }
diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs
index 2f4ee00da35f..16b3a4f2febd 100644
--- a/datafusion/physical-plan/src/joins/hash_join.rs
+++ b/datafusion/physical-plan/src/joins/hash_join.rs
@@ -1238,7 +1238,7 @@ fn eq_dyn_null(
         } else {
             Operator::Eq
         };
-        return Ok(compare_op_for_nested(&op, &left, &right)?);
+        return Ok(compare_op_for_nested(op, &left, &right)?);
     }
     match (left.data_type(), right.data_type()) {
         _ if null_equals_null => not_distinct(&left, &right),
diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs
index a58af8afdd04..095c6a50973a 100644
--- a/datafusion/proto/src/logical_plan/from_proto.rs
+++ b/datafusion/proto/src/logical_plan/from_proto.rs
@@ -269,11 +269,7 @@ pub fn parse_expr(
             Ok(operands
                 .into_iter()
                 .reduce(|left, right| {
-                    Expr::BinaryExpr(BinaryExpr::new(
-                        Box::new(left),
-                        op.clone(),
-                        Box::new(right),
-                    ))
+                    Expr::BinaryExpr(BinaryExpr::new(Box::new(left), op, Box::new(right)))
                 })
                 .expect("Binary expression could not be reduced to a single expression."))
         }
diff --git a/datafusion/substrait/src/logical_plan/consumer.rs b/datafusion/substrait/src/logical_plan/consumer.rs
index 77fd5fe44d44..89a6dde51e42 100644
--- a/datafusion/substrait/src/logical_plan/consumer.rs
+++ b/datafusion/substrait/src/logical_plan/consumer.rs
@@ -1154,7 +1154,7 @@ pub async fn from_substrait_rex(
                                     Arc::try_unwrap(expr)
                                         .unwrap_or_else(|arc: Arc<Expr>| (*arc).clone()),
                                 ), // Avoid cloning if possible
-                                op: op.clone(),
+                                op,
                                 right: Box::new(arg),
                             })),
                             None => Arc::new(arg),
diff --git a/datafusion/substrait/src/logical_plan/producer.rs b/datafusion/substrait/src/logical_plan/producer.rs
index 959542080161..8d039a050249 100644
--- a/datafusion/substrait/src/logical_plan/producer.rs
+++ b/datafusion/substrait/src/logical_plan/producer.rs
@@ -664,12 +664,7 @@ fn to_substrait_join_expr(
             extension_info,
         )?;
         // AND with existing expression
-        exprs.push(make_binary_op_scalar_func(
-            &l,
-            &r,
-            eq_op.clone(),
-            extension_info,
-        ));
+        exprs.push(make_binary_op_scalar_func(&l, &r, eq_op, extension_info));
     }
     let join_expr: Option<Expression> =
         exprs.into_iter().reduce(|acc: Expression, e: Expression| {
@@ -1167,12 +1162,7 @@ pub fn to_substrait_rex(
             let l = to_substrait_rex(ctx, left, schema, col_ref_offset, extension_info)?;
             let r = to_substrait_rex(ctx, right, schema, col_ref_offset, extension_info)?;
 
-            Ok(make_binary_op_scalar_func(
-                &l,
-                &r,
-                op.clone(),
-                extension_info,
-            ))
+            Ok(make_binary_op_scalar_func(&l, &r, *op, extension_info))
         }
         Expr::Case(Case {
             expr,

From 4123ad6ca7bb7bdcb4cca65e97d30b4677233de3 Mon Sep 17 00:00:00 2001
From: Andy Grove <agrove@apache.org>
Date: Mon, 8 Jul 2024 21:51:36 -0600
Subject: [PATCH 7/9] Prepare 40.0.0 Release (#11343)

---
 Cargo.toml                        |  36 +--
 datafusion-cli/Cargo.lock         |  96 ++++----
 datafusion-cli/Cargo.toml         |   4 +-
 dev/changelog/40.0.0.md           | 371 ++++++++++++++++++++++++++++++
 docs/source/user-guide/configs.md |   2 +-
 5 files changed, 440 insertions(+), 69 deletions(-)
 create mode 100644 dev/changelog/40.0.0.md

diff --git a/Cargo.toml b/Cargo.toml
index 968a74e37f10..f87205f0d067 100644
--- a/Cargo.toml
+++ b/Cargo.toml
@@ -53,7 +53,7 @@ license = "Apache-2.0"
 readme = "README.md"
 repository = "https://github.com/apache/datafusion"
 rust-version = "1.76"
-version = "39.0.0"
+version = "40.0.0"
 
 [workspace.dependencies]
 # We turn off default-features for some dependencies here so the workspaces which inherit them can
@@ -86,23 +86,23 @@ bytes = "1.4"
 chrono = { version = "0.4.34", default-features = false }
 ctor = "0.2.0"
 dashmap = "5.5.0"
-datafusion = { path = "datafusion/core", version = "39.0.0", default-features = false }
-datafusion-common = { path = "datafusion/common", version = "39.0.0", default-features = false }
-datafusion-common-runtime = { path = "datafusion/common-runtime", version = "39.0.0" }
-datafusion-execution = { path = "datafusion/execution", version = "39.0.0" }
-datafusion-expr = { path = "datafusion/expr", version = "39.0.0" }
-datafusion-functions = { path = "datafusion/functions", version = "39.0.0" }
-datafusion-functions-aggregate = { path = "datafusion/functions-aggregate", version = "39.0.0" }
-datafusion-functions-array = { path = "datafusion/functions-array", version = "39.0.0" }
-datafusion-optimizer = { path = "datafusion/optimizer", version = "39.0.0", default-features = false }
-datafusion-physical-expr = { path = "datafusion/physical-expr", version = "39.0.0", default-features = false }
-datafusion-physical-expr-common = { path = "datafusion/physical-expr-common", version = "39.0.0", default-features = false }
-datafusion-physical-plan = { path = "datafusion/physical-plan", version = "39.0.0" }
-datafusion-proto = { path = "datafusion/proto", version = "39.0.0" }
-datafusion-proto-common = { path = "datafusion/proto-common", version = "39.0.0" }
-datafusion-sql = { path = "datafusion/sql", version = "39.0.0" }
-datafusion-sqllogictest = { path = "datafusion/sqllogictest", version = "39.0.0" }
-datafusion-substrait = { path = "datafusion/substrait", version = "39.0.0" }
+datafusion = { path = "datafusion/core", version = "40.0.0", default-features = false }
+datafusion-common = { path = "datafusion/common", version = "40.0.0", default-features = false }
+datafusion-common-runtime = { path = "datafusion/common-runtime", version = "40.0.0" }
+datafusion-execution = { path = "datafusion/execution", version = "40.0.0" }
+datafusion-expr = { path = "datafusion/expr", version = "40.0.0" }
+datafusion-functions = { path = "datafusion/functions", version = "40.0.0" }
+datafusion-functions-aggregate = { path = "datafusion/functions-aggregate", version = "40.0.0" }
+datafusion-functions-array = { path = "datafusion/functions-array", version = "40.0.0" }
+datafusion-optimizer = { path = "datafusion/optimizer", version = "40.0.0", default-features = false }
+datafusion-physical-expr = { path = "datafusion/physical-expr", version = "40.0.0", default-features = false }
+datafusion-physical-expr-common = { path = "datafusion/physical-expr-common", version = "40.0.0", default-features = false }
+datafusion-physical-plan = { path = "datafusion/physical-plan", version = "40.0.0" }
+datafusion-proto = { path = "datafusion/proto", version = "40.0.0" }
+datafusion-proto-common = { path = "datafusion/proto-common", version = "40.0.0" }
+datafusion-sql = { path = "datafusion/sql", version = "40.0.0" }
+datafusion-sqllogictest = { path = "datafusion/sqllogictest", version = "40.0.0" }
+datafusion-substrait = { path = "datafusion/substrait", version = "40.0.0" }
 doc-comment = "0.3"
 env_logger = "0.11"
 futures = "0.3"
diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock
index 500e731a5b4f..42ec5922a73f 100644
--- a/datafusion-cli/Cargo.lock
+++ b/datafusion-cli/Cargo.lock
@@ -381,13 +381,13 @@ dependencies = [
 
 [[package]]
 name = "async-trait"
-version = "0.1.80"
+version = "0.1.81"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "c6fa2087f2753a7da8cc1c0dbfcf89579dd57458e36769de5ac750b4671737ca"
+checksum = "6e0c28dcc82d7c8ead5cb13beb15405b57b8546e93215673ff8ca0349a028107"
 dependencies = [
  "proc-macro2",
  "quote",
- "syn 2.0.68",
+ "syn 2.0.70",
 ]
 
 [[package]]
@@ -875,9 +875,9 @@ dependencies = [
 
 [[package]]
 name = "cc"
-version = "1.0.104"
+version = "1.0.106"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "74b6a57f98764a267ff415d50a25e6e166f3831a5071af4995296ea97d210490"
+checksum = "066fce287b1d4eafef758e89e09d724a24808a9196fe9756b8ca90e86d0719a2"
 dependencies = [
  "jobserver",
  "libc",
@@ -1099,7 +1099,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index"
 checksum = "edb49164822f3ee45b17acd4a208cfc1251410cf0cad9a833234c9890774dd9f"
 dependencies = [
  "quote",
- "syn 2.0.68",
+ "syn 2.0.70",
 ]
 
 [[package]]
@@ -1123,7 +1123,7 @@ dependencies = [
 
 [[package]]
 name = "datafusion"
-version = "39.0.0"
+version = "40.0.0"
 dependencies = [
  "ahash",
  "apache-avro",
@@ -1177,7 +1177,7 @@ dependencies = [
 
 [[package]]
 name = "datafusion-cli"
-version = "39.0.0"
+version = "40.0.0"
 dependencies = [
  "arrow",
  "assert_cmd",
@@ -1204,7 +1204,7 @@ dependencies = [
 
 [[package]]
 name = "datafusion-common"
-version = "39.0.0"
+version = "40.0.0"
 dependencies = [
  "ahash",
  "apache-avro",
@@ -1225,14 +1225,14 @@ dependencies = [
 
 [[package]]
 name = "datafusion-common-runtime"
-version = "39.0.0"
+version = "40.0.0"
 dependencies = [
  "tokio",
 ]
 
 [[package]]
 name = "datafusion-execution"
-version = "39.0.0"
+version = "40.0.0"
 dependencies = [
  "arrow",
  "chrono",
@@ -1251,7 +1251,7 @@ dependencies = [
 
 [[package]]
 name = "datafusion-expr"
-version = "39.0.0"
+version = "40.0.0"
 dependencies = [
  "ahash",
  "arrow",
@@ -1268,7 +1268,7 @@ dependencies = [
 
 [[package]]
 name = "datafusion-functions"
-version = "39.0.0"
+version = "40.0.0"
 dependencies = [
  "arrow",
  "base64 0.22.1",
@@ -1292,7 +1292,7 @@ dependencies = [
 
 [[package]]
 name = "datafusion-functions-aggregate"
-version = "39.0.0"
+version = "40.0.0"
 dependencies = [
  "ahash",
  "arrow",
@@ -1308,7 +1308,7 @@ dependencies = [
 
 [[package]]
 name = "datafusion-functions-array"
-version = "39.0.0"
+version = "40.0.0"
 dependencies = [
  "arrow",
  "arrow-array",
@@ -1327,7 +1327,7 @@ dependencies = [
 
 [[package]]
 name = "datafusion-optimizer"
-version = "39.0.0"
+version = "40.0.0"
 dependencies = [
  "arrow",
  "async-trait",
@@ -1345,7 +1345,7 @@ dependencies = [
 
 [[package]]
 name = "datafusion-physical-expr"
-version = "39.0.0"
+version = "40.0.0"
 dependencies = [
  "ahash",
  "arrow",
@@ -1373,7 +1373,7 @@ dependencies = [
 
 [[package]]
 name = "datafusion-physical-expr-common"
-version = "39.0.0"
+version = "40.0.0"
 dependencies = [
  "ahash",
  "arrow",
@@ -1385,7 +1385,7 @@ dependencies = [
 
 [[package]]
 name = "datafusion-physical-plan"
-version = "39.0.0"
+version = "40.0.0"
 dependencies = [
  "ahash",
  "arrow",
@@ -1417,7 +1417,7 @@ dependencies = [
 
 [[package]]
 name = "datafusion-sql"
-version = "39.0.0"
+version = "40.0.0"
 dependencies = [
  "arrow",
  "arrow-array",
@@ -1686,7 +1686,7 @@ checksum = "87750cf4b7a4c0625b1529e4c543c2182106e4dedc60a2a6455e00d212c489ac"
 dependencies = [
  "proc-macro2",
  "quote",
- "syn 2.0.68",
+ "syn 2.0.70",
 ]
 
 [[package]]
@@ -2008,7 +2008,7 @@ dependencies = [
  "http 1.1.0",
  "hyper 1.4.0",
  "hyper-util",
- "rustls 0.23.10",
+ "rustls 0.23.11",
  "rustls-native-certs 0.7.1",
  "rustls-pki-types",
  "tokio",
@@ -2699,7 +2699,7 @@ checksum = "2f38a4412a78282e09a2cf38d195ea5420d15ba0602cb375210efbc877243965"
 dependencies = [
  "proc-macro2",
  "quote",
- "syn 2.0.68",
+ "syn 2.0.70",
 ]
 
 [[package]]
@@ -2822,7 +2822,7 @@ dependencies = [
  "quinn-proto",
  "quinn-udp",
  "rustc-hash",
- "rustls 0.23.10",
+ "rustls 0.23.11",
  "thiserror",
  "tokio",
  "tracing",
@@ -2838,7 +2838,7 @@ dependencies = [
  "rand",
  "ring 0.17.8",
  "rustc-hash",
- "rustls 0.23.10",
+ "rustls 0.23.11",
  "slab",
  "thiserror",
  "tinyvec",
@@ -2987,7 +2987,7 @@ dependencies = [
  "percent-encoding",
  "pin-project-lite",
  "quinn",
- "rustls 0.23.10",
+ "rustls 0.23.11",
  "rustls-native-certs 0.7.1",
  "rustls-pemfile 2.1.2",
  "rustls-pki-types",
@@ -3117,9 +3117,9 @@ dependencies = [
 
 [[package]]
 name = "rustls"
-version = "0.23.10"
+version = "0.23.11"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "05cff451f60db80f490f3c182b77c35260baace73209e9cdbbe526bfe3a4d402"
+checksum = "4828ea528154ae444e5a642dbb7d5623354030dc9822b83fd9bb79683c7399d0"
 dependencies = [
  "once_cell",
  "ring 0.17.8",
@@ -3296,22 +3296,22 @@ checksum = "a3f0bf26fd526d2a95683cd0f87bf103b8539e2ca1ef48ce002d67aad59aa0b4"
 
 [[package]]
 name = "serde"
-version = "1.0.203"
+version = "1.0.204"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "7253ab4de971e72fb7be983802300c30b5a7f0c2e56fab8abfc6a214307c0094"
+checksum = "bc76f558e0cbb2a839d37354c575f1dc3fdc6546b5be373ba43d95f231bf7c12"
 dependencies = [
  "serde_derive",
 ]
 
 [[package]]
 name = "serde_derive"
-version = "1.0.203"
+version = "1.0.204"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "500cbc0ebeb6f46627f50f3f5811ccf6bf00643be300b4c3eabc0ef55dc5b5ba"
+checksum = "e0cd7e117be63d3c3678776753929474f3b04a43a080c744d6b0ae2a8c28e222"
 dependencies = [
  "proc-macro2",
  "quote",
- "syn 2.0.68",
+ "syn 2.0.70",
 ]
 
 [[package]]
@@ -3446,7 +3446,7 @@ checksum = "01b2e185515564f15375f593fb966b5718bc624ba77fe49fa4616ad619690554"
 dependencies = [
  "proc-macro2",
  "quote",
- "syn 2.0.68",
+ "syn 2.0.70",
 ]
 
 [[package]]
@@ -3492,7 +3492,7 @@ dependencies = [
  "proc-macro2",
  "quote",
  "rustversion",
- "syn 2.0.68",
+ "syn 2.0.70",
 ]
 
 [[package]]
@@ -3505,7 +3505,7 @@ dependencies = [
  "proc-macro2",
  "quote",
  "rustversion",
- "syn 2.0.68",
+ "syn 2.0.70",
 ]
 
 [[package]]
@@ -3527,9 +3527,9 @@ dependencies = [
 
 [[package]]
 name = "syn"
-version = "2.0.68"
+version = "2.0.70"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "901fa70d88b9d6c98022e23b4136f9f3e54e4662c3bc1bd1d84a42a9a0f0c1e9"
+checksum = "2f0209b68b3613b093e0ec905354eccaedcfe83b8cb37cbdeae64026c3064c16"
 dependencies = [
  "proc-macro2",
  "quote",
@@ -3592,7 +3592,7 @@ checksum = "46c3384250002a6d5af4d114f2845d37b57521033f30d5c3f46c4d70e1197533"
 dependencies = [
  "proc-macro2",
  "quote",
- "syn 2.0.68",
+ "syn 2.0.70",
 ]
 
 [[package]]
@@ -3647,9 +3647,9 @@ dependencies = [
 
 [[package]]
 name = "tinyvec"
-version = "1.7.0"
+version = "1.8.0"
 source = "registry+https://github.com/rust-lang/crates.io-index"
-checksum = "ce6b6a2fb3a985e99cebfaefa9faa3024743da73304ca1c683a36429613d3d22"
+checksum = "445e881f4f6d382d5f27c034e25eb92edd7c784ceab92a0937db7f2e9471b938"
 dependencies = [
  "tinyvec_macros",
 ]
@@ -3687,7 +3687,7 @@ checksum = "5f5ae998a069d4b5aba8ee9dad856af7d520c3699e6159b185c2acd48155d39a"
 dependencies = [
  "proc-macro2",
  "quote",
- "syn 2.0.68",
+ "syn 2.0.70",
 ]
 
 [[package]]
@@ -3707,7 +3707,7 @@ version = "0.26.0"
 source = "registry+https://github.com/rust-lang/crates.io-index"
 checksum = "0c7bc40d0e5a97695bb96e27995cd3a08538541b0a846f65bba7a359f36700d4"
 dependencies = [
- "rustls 0.23.10",
+ "rustls 0.23.11",
  "rustls-pki-types",
  "tokio",
 ]
@@ -3784,7 +3784,7 @@ checksum = "34704c8d6ebcbc939824180af020566b01a7c01f80641264eba0999f6c2b6be7"
 dependencies = [
  "proc-macro2",
  "quote",
- "syn 2.0.68",
+ "syn 2.0.70",
 ]
 
 [[package]]
@@ -3829,7 +3829,7 @@ checksum = "f03ca4cb38206e2bef0700092660bb74d696f808514dae47fa1467cbfe26e96e"
 dependencies = [
  "proc-macro2",
  "quote",
- "syn 2.0.68",
+ "syn 2.0.70",
 ]
 
 [[package]]
@@ -3983,7 +3983,7 @@ dependencies = [
  "once_cell",
  "proc-macro2",
  "quote",
- "syn 2.0.68",
+ "syn 2.0.70",
  "wasm-bindgen-shared",
 ]
 
@@ -4017,7 +4017,7 @@ checksum = "e94f17b526d0a461a191c78ea52bbce64071ed5c04c9ffe424dcb38f74171bb7"
 dependencies = [
  "proc-macro2",
  "quote",
- "syn 2.0.68",
+ "syn 2.0.70",
  "wasm-bindgen-backend",
  "wasm-bindgen-shared",
 ]
@@ -4282,7 +4282,7 @@ checksum = "fa4f8080344d4671fb4e831a13ad1e68092748387dfc4f55e356242fae12ce3e"
 dependencies = [
  "proc-macro2",
  "quote",
- "syn 2.0.68",
+ "syn 2.0.70",
 ]
 
 [[package]]
diff --git a/datafusion-cli/Cargo.toml b/datafusion-cli/Cargo.toml
index bcacf1d52a9b..860dc123fa94 100644
--- a/datafusion-cli/Cargo.toml
+++ b/datafusion-cli/Cargo.toml
@@ -18,7 +18,7 @@
 [package]
 name = "datafusion-cli"
 description = "Command Line Client for DataFusion query engine."
-version = "39.0.0"
+version = "40.0.0"
 authors = ["Apache DataFusion <dev@datafusion.apache.org>"]
 edition = "2021"
 keywords = ["arrow", "datafusion", "query", "sql"]
@@ -35,7 +35,7 @@ async-trait = "0.1.41"
 aws-config = "0.55"
 aws-credential-types = "0.55"
 clap = { version = "3", features = ["derive", "cargo"] }
-datafusion = { path = "../datafusion/core", version = "39.0.0", features = [
+datafusion = { path = "../datafusion/core", version = "40.0.0", features = [
     "avro",
     "crypto_expressions",
     "datetime_expressions",
diff --git a/dev/changelog/40.0.0.md b/dev/changelog/40.0.0.md
new file mode 100644
index 000000000000..72143ae48b28
--- /dev/null
+++ b/dev/changelog/40.0.0.md
@@ -0,0 +1,371 @@
+<!--
+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.
+-->
+
+# Apache DataFusion 40.0.0 Changelog
+
+This release consists of 263 commits from 64 contributors. See credits at the end of this changelog for more information.
+
+**Breaking changes:**
+
+- Convert `StringAgg` to UDAF [#10945](https://github.com/apache/datafusion/pull/10945) (lewiszlw)
+- Convert `bool_and` & `bool_or` to UDAF [#11009](https://github.com/apache/datafusion/pull/11009) (jcsherin)
+- Convert Average to UDAF #10942 [#10964](https://github.com/apache/datafusion/pull/10964) (dharanad)
+- fix: remove the Sized requirement on ExecutionPlan::name() [#11047](https://github.com/apache/datafusion/pull/11047) (waynexia)
+- Return `&Arc` reference to inner trait object [#11103](https://github.com/apache/datafusion/pull/11103) (linhr)
+- Support COPY TO Externally Defined File Formats, add FileType trait [#11060](https://github.com/apache/datafusion/pull/11060) (devinjdangelo)
+- expose table name in proto extension codec [#11139](https://github.com/apache/datafusion/pull/11139) (leoyvens)
+- fix(typo): unqualifed to unqualified [#11159](https://github.com/apache/datafusion/pull/11159) (waynexia)
+- Consolidate `Filter::remove_aliases` into `Expr::unalias_nested` [#11001](https://github.com/apache/datafusion/pull/11001) (alamb)
+- Convert `nth_value` to UDAF [#11287](https://github.com/apache/datafusion/pull/11287) (jcsherin)
+
+**Implemented enhancements:**
+
+- feat: Add support for Int8 and Int16 data types in data page statistics [#10931](https://github.com/apache/datafusion/pull/10931) (Weijun-H)
+- feat: add CliSessionContext trait for cli [#10890](https://github.com/apache/datafusion/pull/10890) (tshauck)
+- feat(optimizer): handle partial anchored regex cases and improve doc [#10977](https://github.com/apache/datafusion/pull/10977) (waynexia)
+- feat: support uint data page extraction [#11018](https://github.com/apache/datafusion/pull/11018) (tshauck)
+- feat: propagate EmptyRelation for more join types [#10963](https://github.com/apache/datafusion/pull/10963) (tshauck)
+- feat: Add method to add analyzer rules to SessionContext [#10849](https://github.com/apache/datafusion/pull/10849) (pingsutw)
+- feat: Support duplicate column names in Joins in Substrait consumer [#11049](https://github.com/apache/datafusion/pull/11049) (Blizzara)
+- feat: Add support for Timestamp data types in data page statistics. [#11123](https://github.com/apache/datafusion/pull/11123) (efredine)
+- feat: Add support for `Binary`/`LargeBinary`/`Utf8`/`LargeUtf8` data types in data page statistics [#11136](https://github.com/apache/datafusion/pull/11136) (PsiACE)
+- feat: Support Map type in Substrait conversions [#11129](https://github.com/apache/datafusion/pull/11129) (Blizzara)
+- feat: Conditionally allow to keep partition_by columns when using PARTITIONED BY enhancement [#11107](https://github.com/apache/datafusion/pull/11107) (hveiga)
+- feat: enable "substring" as a UDF in addition to "substr" [#11277](https://github.com/apache/datafusion/pull/11277) (Blizzara)
+
+**Fixed bugs:**
+
+- fix: use total ordering in the min & max accumulator for floats [#10627](https://github.com/apache/datafusion/pull/10627) (westonpace)
+- fix: Support double quotes in `date_part` [#10833](https://github.com/apache/datafusion/pull/10833) (Weijun-H)
+- fix: Ignore nullability of list elements when consuming Substrait [#10874](https://github.com/apache/datafusion/pull/10874) (Blizzara)
+- fix: Support `NOT <field> IN (<subquery>)` via anti join [#10936](https://github.com/apache/datafusion/pull/10936) (akoshchiy)
+- fix: CTEs defined in a subquery can escape their scope [#10954](https://github.com/apache/datafusion/pull/10954) (jonahgao)
+- fix: Fix the incorrect null joined rows for SMJ outer join with join filter [#10892](https://github.com/apache/datafusion/pull/10892) (viirya)
+- fix: gcd returns negative results [#11099](https://github.com/apache/datafusion/pull/11099) (jonahgao)
+- fix: LCM panicked due to overflow [#11131](https://github.com/apache/datafusion/pull/11131) (jonahgao)
+- fix: Support dictionary type in parquet metadata statistics. [#11169](https://github.com/apache/datafusion/pull/11169) (efredine)
+- fix: Ignore nullability in Substrait structs [#11130](https://github.com/apache/datafusion/pull/11130) (Blizzara)
+- fix: typo in comment about FinalPhysicalPlan [#11181](https://github.com/apache/datafusion/pull/11181) (c8ef)
+- fix: Support Substrait's compound names also for window functions [#11163](https://github.com/apache/datafusion/pull/11163) (Blizzara)
+- fix: Incorrect LEFT JOIN evaluation result on OR conditions [#11203](https://github.com/apache/datafusion/pull/11203) (viirya)
+- fix: Be more lenient in interpreting input args for builtin window functions [#11199](https://github.com/apache/datafusion/pull/11199) (Blizzara)
+- fix: correctly handle Substrait windows with rows bounds (and validate executability of test plans) [#11278](https://github.com/apache/datafusion/pull/11278) (Blizzara)
+- fix: When consuming Substrait, temporarily rename clashing duplicate columns [#11329](https://github.com/apache/datafusion/pull/11329) (Blizzara)
+
+**Documentation updates:**
+
+- Minor: Clarify `SessionContext::state` docs [#10847](https://github.com/apache/datafusion/pull/10847) (alamb)
+- Minor: Update SIGMOD paper reference url [#10860](https://github.com/apache/datafusion/pull/10860) (alamb)
+- docs(variance): Correct typos in comments [#10844](https://github.com/apache/datafusion/pull/10844) (pingsutw)
+- Add missing code close tick in LiteralGuarantee docs [#10859](https://github.com/apache/datafusion/pull/10859) (adriangb)
+- Minor: Add more docs and examples for `Transformed` and `TransformedResult` [#11003](https://github.com/apache/datafusion/pull/11003) (alamb)
+- doc: Update links in the documantation [#11044](https://github.com/apache/datafusion/pull/11044) (Weijun-H)
+- Minor: Examples cleanup + more docs in pruning example [#11086](https://github.com/apache/datafusion/pull/11086) (alamb)
+- Minor: refine documentation pointing to examples [#11110](https://github.com/apache/datafusion/pull/11110) (alamb)
+- Fix running in Docker instructions [#11141](https://github.com/apache/datafusion/pull/11141) (findepi)
+- docs: add example for custom file format with `COPY TO` [#11174](https://github.com/apache/datafusion/pull/11174) (tshauck)
+- Fix docs wordings [#11226](https://github.com/apache/datafusion/pull/11226) (findepi)
+- Fix count() docs around including null values [#11293](https://github.com/apache/datafusion/pull/11293) (findepi)
+
+**Other:**
+
+- chore: Prepare 39.0.0-rc1 [#10828](https://github.com/apache/datafusion/pull/10828) (andygrove)
+- Remove expr_fn::sum and replace them with function stub [#10816](https://github.com/apache/datafusion/pull/10816) (jayzhan211)
+- Debug print as many fields as possible for `SessionState` [#10818](https://github.com/apache/datafusion/pull/10818) (lewiszlw)
+- Prune Parquet RowGroup in a single call to `PruningPredicate::prune`, update StatisticsExtractor API [#10802](https://github.com/apache/datafusion/pull/10802) (alamb)
+- Remove Built-in sum and Rename to lowercase `sum` [#10831](https://github.com/apache/datafusion/pull/10831) (jayzhan211)
+- Convert `stddev` and `stddev_pop` to UDAF [#10834](https://github.com/apache/datafusion/pull/10834) (goldmedal)
+- Introduce expr builder for aggregate function [#10560](https://github.com/apache/datafusion/pull/10560) (jayzhan211)
+- chore: Improve change log generator [#10841](https://github.com/apache/datafusion/pull/10841) (andygrove)
+- Support user defined `ParquetAccessPlan` in `ParquetExec`, validation to `ParquetAccessPlan::select` [#10813](https://github.com/apache/datafusion/pull/10813) (alamb)
+- Convert `VariancePopulation` to UDAF [#10836](https://github.com/apache/datafusion/pull/10836) (mknaw)
+- Convert `approx_median` to UDAF [#10840](https://github.com/apache/datafusion/pull/10840) (goldmedal)
+- MINOR: use workspace deps in proto-common (upgrade object store dependency) [#10848](https://github.com/apache/datafusion/pull/10848) (waynexia)
+- Minor: add `Window::try_new_with_schema` constructor [#10850](https://github.com/apache/datafusion/pull/10850) (sadboy)
+- Add support for reading CSV files with comments [#10467](https://github.com/apache/datafusion/pull/10467) (bbannier)
+- Convert approx_distinct to UDAF [#10851](https://github.com/apache/datafusion/pull/10851) (Lordworms)
+- minor: add proto-common crate to release instructions [#10858](https://github.com/apache/datafusion/pull/10858) (andygrove)
+- Implement TPCH substrait integration teset, support tpch_1 [#10842](https://github.com/apache/datafusion/pull/10842) (Lordworms)
+- Remove unecessary passing around of `suffix: &str` in `pruning.rs`'s `RequiredColumns` [#10863](https://github.com/apache/datafusion/pull/10863) (adriangb)
+- chore: Make DFSchema::datatype_is_logically_equal function public [#10867](https://github.com/apache/datafusion/pull/10867) (advancedxy)
+- Bump braces from 3.0.2 to 3.0.3 in /datafusion/wasmtest/datafusion-wasm-app [#10865](https://github.com/apache/datafusion/pull/10865) (dependabot[bot])
+- Docs: Add `unnest` to SQL Reference [#10839](https://github.com/apache/datafusion/pull/10839) (gloomweaver)
+- Support correct output column names and struct field names when consuming/producing Substrait [#10829](https://github.com/apache/datafusion/pull/10829) (Blizzara)
+- Make Logical Plans more readable by removing extra aliases [#10832](https://github.com/apache/datafusion/pull/10832) (MohamedAbdeen21)
+- Minor: Improve `ListingTable` documentation [#10854](https://github.com/apache/datafusion/pull/10854) (alamb)
+- Extending join fuzz tests to support join filtering [#10728](https://github.com/apache/datafusion/pull/10728) (edmondop)
+- replace and(_, not(_)) with and_not(\*) [#10885](https://github.com/apache/datafusion/pull/10885) (RTEnzyme)
+- Disabling test for semi join with filters [#10887](https://github.com/apache/datafusion/pull/10887) (edmondop)
+- Minor: Update `min_statistics` and `max_statistics` to be helpers, update docs [#10866](https://github.com/apache/datafusion/pull/10866) (alamb)
+- Remove `Interval` column test // parquet extraction [#10888](https://github.com/apache/datafusion/pull/10888) (marvinlanhenke)
+- Minor: SMJ fuzz tests fix for rowcounts [#10891](https://github.com/apache/datafusion/pull/10891) (comphead)
+- Move `Count` to `functions-aggregate`, update MSRV to rust 1.75 [#10484](https://github.com/apache/datafusion/pull/10484) (jayzhan211)
+- refactor: fetch statistics for a given ParquetMetaData [#10880](https://github.com/apache/datafusion/pull/10880) (NGA-TRAN)
+- Move FileSinkExec::metrics to the correct place [#10901](https://github.com/apache/datafusion/pull/10901) (joroKr21)
+- Refine ParquetAccessPlan comments and tests [#10896](https://github.com/apache/datafusion/pull/10896) (alamb)
+- ci: fix clippy failures on main [#10903](https://github.com/apache/datafusion/pull/10903) (jonahgao)
+- Minor: disable flaky fuzz test [#10904](https://github.com/apache/datafusion/pull/10904) (comphead)
+- Remove builtin count [#10893](https://github.com/apache/datafusion/pull/10893) (jayzhan211)
+- Move Regr\_\* functions to use UDAF [#10898](https://github.com/apache/datafusion/pull/10898) (eejbyfeldt)
+- Docs: clarify when the parquet reader will read from object store when using cached metadata [#10909](https://github.com/apache/datafusion/pull/10909) (alamb)
+- Minor: Fix `bench.sh tpch data` [#10905](https://github.com/apache/datafusion/pull/10905) (alamb)
+- Minor: use venv in benchmark compare [#10894](https://github.com/apache/datafusion/pull/10894) (tmi)
+- Support explicit type and name during table creation [#10273](https://github.com/apache/datafusion/pull/10273) (duongcongtoai)
+- Simplify Join Partition Rules [#10911](https://github.com/apache/datafusion/pull/10911) (berkaysynnada)
+- Move `Literal` to `physical-expr-common` [#10910](https://github.com/apache/datafusion/pull/10910) (lewiszlw)
+- chore: update some error messages for clarity [#10916](https://github.com/apache/datafusion/pull/10916) (jeffreyssmith2nd)
+- Initial Extract parquet data page statistics API [#10852](https://github.com/apache/datafusion/pull/10852) (marvinlanhenke)
+- Add contains function, and support in datafusion substrait consumer [#10879](https://github.com/apache/datafusion/pull/10879) (Lordworms)
+- Minor: Improve `arrow_statistics` tests [#10927](https://github.com/apache/datafusion/pull/10927) (alamb)
+- Minor: Remove `prefer_hash_join` env variable for clickbench [#10933](https://github.com/apache/datafusion/pull/10933) (jayzhan211)
+- Convert ApproxPercentileCont and ApproxPercentileContWithWeight to UDAF [#10917](https://github.com/apache/datafusion/pull/10917) (goldmedal)
+- refactor: remove extra default in max rows [#10941](https://github.com/apache/datafusion/pull/10941) (tshauck)
+- chore: Improve performance of Parquet statistics conversion [#10932](https://github.com/apache/datafusion/pull/10932) (Weijun-H)
+- Add catalog::resolve_table_references [#10876](https://github.com/apache/datafusion/pull/10876) (leoyvens)
+- Convert BitAnd, BitOr, BitXor to UDAF [#10930](https://github.com/apache/datafusion/pull/10930) (dharanad)
+- refactor: improve PoolType argument handling for CLI [#10940](https://github.com/apache/datafusion/pull/10940) (tshauck)
+- Minor: remove potential string copy from Column::from_qualified_name [#10947](https://github.com/apache/datafusion/pull/10947) (alamb)
+- Fix: StatisticsConverter `counts` for missing columns [#10946](https://github.com/apache/datafusion/pull/10946) (marvinlanhenke)
+- Add initial support for Utf8View and BinaryView types [#10925](https://github.com/apache/datafusion/pull/10925) (XiangpengHao)
+- Use shorter aliases in CSE [#10939](https://github.com/apache/datafusion/pull/10939) (peter-toth)
+- Substrait support for ParquetExec round trip for simple select [#10949](https://github.com/apache/datafusion/pull/10949) (xinlifoobar)
+- Support to unparse `ScalarValue::IntervalMonthDayNano` to String [#10956](https://github.com/apache/datafusion/pull/10956) (goldmedal)
+- Minor: Return option from row_group_row_count [#10973](https://github.com/apache/datafusion/pull/10973) (marvinlanhenke)
+- Minor: Add routine to debug join fuzz tests [#10970](https://github.com/apache/datafusion/pull/10970) (comphead)
+- Support to unparse `ScalarValue::TimestampNanosecond` to String [#10984](https://github.com/apache/datafusion/pull/10984) (goldmedal)
+- build(deps-dev): bump ws from 8.14.2 to 8.17.1 in /datafusion/wasmtest/datafusion-wasm-app [#10988](https://github.com/apache/datafusion/pull/10988) (dependabot[bot])
+- Minor: reuse Rows buffer in GroupValuesRows [#10980](https://github.com/apache/datafusion/pull/10980) (alamb)
+- Add example for writing SQL analysis using DataFusion structures [#10938](https://github.com/apache/datafusion/pull/10938) (LorrensP-2158466)
+- Push down filter for Unnest plan [#10974](https://github.com/apache/datafusion/pull/10974) (jayzhan211)
+- Add parquet page stats for float{16, 32, 64} [#10982](https://github.com/apache/datafusion/pull/10982) (tmi)
+- Fix `file_stream_provider` example compilation failure on windows [#10975](https://github.com/apache/datafusion/pull/10975) (lewiszlw)
+- Stop copying LogicalPlan and Exprs in `CommonSubexprEliminate` (2-3% planning speed improvement) [#10835](https://github.com/apache/datafusion/pull/10835) (alamb)
+- chore: Update documentation link in `PhysicalOptimizerRule` comment [#11002](https://github.com/apache/datafusion/pull/11002) (Weijun-H)
+- Push down filter plan for unnest on non-unnest column only [#10991](https://github.com/apache/datafusion/pull/10991) (jayzhan211)
+- Minor: add test for pushdown past unnest [#11017](https://github.com/apache/datafusion/pull/11017) (alamb)
+- Update docs for `protoc` minimum installed version [#11006](https://github.com/apache/datafusion/pull/11006) (jcsherin)
+- propagate error instead of panicking on out of bounds in physical-expr/src/analysis.rs [#10992](https://github.com/apache/datafusion/pull/10992) (LorrensP-2158466)
+- Add drop_columns to dataframe api [#11010](https://github.com/apache/datafusion/pull/11010) (Omega359)
+- Push down filter plan for non-unnest column [#11019](https://github.com/apache/datafusion/pull/11019) (jayzhan211)
+- Consider timezones with `UTC` and `+00:00` to be the same [#10960](https://github.com/apache/datafusion/pull/10960) (marvinlanhenke)
+- Deprecate `OptimizerRule::try_optimize` [#11022](https://github.com/apache/datafusion/pull/11022) (lewiszlw)
+- Relax combine partial final rule [#10913](https://github.com/apache/datafusion/pull/10913) (mustafasrepo)
+- Compute gcd with u64 instead of i64 because of overflows [#11036](https://github.com/apache/datafusion/pull/11036) (LorrensP-2158466)
+- Add distinct_on to dataframe api [#11012](https://github.com/apache/datafusion/pull/11012) (Omega359)
+- chore: add test to show current behavior of `AT TIME ZONE` for string vs. timestamp [#11056](https://github.com/apache/datafusion/pull/11056) (appletreeisyellow)
+- Boolean parquet get datapage stat [#11054](https://github.com/apache/datafusion/pull/11054) (LorrensP-2158466)
+- Using display_name for Expr::Aggregation [#11020](https://github.com/apache/datafusion/pull/11020) (Lordworms)
+- Minor: Convert `Count`'s name to lowercase [#11028](https://github.com/apache/datafusion/pull/11028) (jayzhan211)
+- Minor: Move `function::Hint` to `datafusion-expr` crate to avoid physical-expr dependency for `datafusion-function` crate [#11061](https://github.com/apache/datafusion/pull/11061) (jayzhan211)
+- Support to unparse ScalarValue::TimestampMillisecond to String [#11046](https://github.com/apache/datafusion/pull/11046) (pingsutw)
+- Support to unparse IntervalYearMonth and IntervalDayTime to String [#11065](https://github.com/apache/datafusion/pull/11065) (goldmedal)
+- SMJ: fix streaming row concurrency issue for LEFT SEMI filtered join [#11041](https://github.com/apache/datafusion/pull/11041) (comphead)
+- Add `advanced_parquet_index.rs` example of index in into parquet files [#10701](https://github.com/apache/datafusion/pull/10701) (alamb)
+- Add Expr::column_refs to find column references without copying [#10948](https://github.com/apache/datafusion/pull/10948) (alamb)
+- Give `OptimizerRule::try_optimize` default implementation and cleanup duplicated custom implementations [#11059](https://github.com/apache/datafusion/pull/11059) (lewiszlw)
+- Fix `FormatOptions::CSV` propagation [#10912](https://github.com/apache/datafusion/pull/10912) (svranesevic)
+- Support parsing SQL strings to Exprs [#10995](https://github.com/apache/datafusion/pull/10995) (xinlifoobar)
+- Support dictionary data type in array_to_string [#10908](https://github.com/apache/datafusion/pull/10908) (EduardoVega)
+- Implement min/max for interval types [#11015](https://github.com/apache/datafusion/pull/11015) (maxburke)
+- Improve LIKE performance for Dictionary arrays [#11058](https://github.com/apache/datafusion/pull/11058) (Lordworms)
+- handle overflow in gcd and return this as an error [#11057](https://github.com/apache/datafusion/pull/11057) (LorrensP-2158466)
+- Convert Correlation to UDAF [#11064](https://github.com/apache/datafusion/pull/11064) (pingsutw)
+- Migrate more code from `Expr::to_columns` to `Expr::column_refs` [#11067](https://github.com/apache/datafusion/pull/11067) (alamb)
+- decimal support for unparser [#11092](https://github.com/apache/datafusion/pull/11092) (y-f-u)
+- Improve `CommonSubexprEliminate` identifier management (10% faster planning) [#10473](https://github.com/apache/datafusion/pull/10473) (peter-toth)
+- Change wildcard qualifier type from `String` to `TableReference` [#11073](https://github.com/apache/datafusion/pull/11073) (linhr)
+- Allow access to UDTF in `SessionContext` [#11071](https://github.com/apache/datafusion/pull/11071) (linhr)
+- Strip table qualifiers from schema in `UNION ALL` for unparser [#11082](https://github.com/apache/datafusion/pull/11082) (phillipleblanc)
+- Update ListingTable to use StatisticsConverter [#11068](https://github.com/apache/datafusion/pull/11068) (xinlifoobar)
+- to_timestamp functions should preserve timezone [#11038](https://github.com/apache/datafusion/pull/11038) (maxburke)
+- Rewrite array operator to function in parser [#11101](https://github.com/apache/datafusion/pull/11101) (jayzhan211)
+- Resolve empty relation opt for join types [#11066](https://github.com/apache/datafusion/pull/11066) (LorrensP-2158466)
+- Add composed extension codec example [#11095](https://github.com/apache/datafusion/pull/11095) (lewiszlw)
+- Minor: Avoid some repetition in to_timestamp [#11116](https://github.com/apache/datafusion/pull/11116) (alamb)
+- Minor: fix ScalarValue::new_ten error message (cites one not ten) [#11126](https://github.com/apache/datafusion/pull/11126) (gstvg)
+- Deprecate Expr::column_refs [#11115](https://github.com/apache/datafusion/pull/11115) (alamb)
+- Overflow in negate operator [#11084](https://github.com/apache/datafusion/pull/11084) (LorrensP-2158466)
+- Minor: Add Architectural Goals to the docs [#11109](https://github.com/apache/datafusion/pull/11109) (alamb)
+- Fix overflow in pow [#11124](https://github.com/apache/datafusion/pull/11124) (LorrensP-2158466)
+- Support to unparse Time scalar value to String [#11121](https://github.com/apache/datafusion/pull/11121) (goldmedal)
+- Support to unparse `TimestampSecond` and `TimestampMicrosecond` to String [#11120](https://github.com/apache/datafusion/pull/11120) (goldmedal)
+- Add standalone example for `OptimizerRule` [#11087](https://github.com/apache/datafusion/pull/11087) (alamb)
+- Fix overflow in factorial [#11134](https://github.com/apache/datafusion/pull/11134) (LorrensP-2158466)
+- Temporary Fix: Query error when grouping by case expressions [#11133](https://github.com/apache/datafusion/pull/11133) (jonahgao)
+- Fix nullability of return value of array_agg [#11093](https://github.com/apache/datafusion/pull/11093) (eejbyfeldt)
+- Support filter for List [#11091](https://github.com/apache/datafusion/pull/11091) (jayzhan211)
+- [MINOR]: Fix some minor silent bugs [#11127](https://github.com/apache/datafusion/pull/11127) (mustafasrepo)
+- Minor Fix for Logical and Physical Expr Conversions [#11142](https://github.com/apache/datafusion/pull/11142) (berkaysynnada)
+- Support Date Parquet Data Page Statistics [#11135](https://github.com/apache/datafusion/pull/11135) (dharanad)
+- fix flaky array query slt test [#11140](https://github.com/apache/datafusion/pull/11140) (leoyvens)
+- Support Decimal and Decimal256 Parquet Data Page Statistics [#11138](https://github.com/apache/datafusion/pull/11138) (Lordworms)
+- Implement comparisons on nested data types such that distinct/except would work [#11117](https://github.com/apache/datafusion/pull/11117) (rtyler)
+- Minor: dont panic with bad arguments to round [#10899](https://github.com/apache/datafusion/pull/10899) (tmi)
+- Minor: reduce replication for nested comparison [#11149](https://github.com/apache/datafusion/pull/11149) (alamb)
+- [Minor]: Remove datafusion-functions-aggregate dependency from physical-expr crate [#11158](https://github.com/apache/datafusion/pull/11158) (mustafasrepo)
+- adding config to control Varchar behavior [#11090](https://github.com/apache/datafusion/pull/11090) (Lordworms)
+- minor: consolidate `gcd` related tests [#11164](https://github.com/apache/datafusion/pull/11164) (jonahgao)
+- Minor: move batch spilling methods to `lib.rs` to make it reusable [#11154](https://github.com/apache/datafusion/pull/11154) (comphead)
+- Move schema projection to where it's used in ListingTable [#11167](https://github.com/apache/datafusion/pull/11167) (adriangb)
+- Make running in docker instruction be copy-pastable [#11148](https://github.com/apache/datafusion/pull/11148) (findepi)
+- Rewrite `array @> array` and `array <@ array` in sql_expr_to_logical_expr [#11155](https://github.com/apache/datafusion/pull/11155) (jayzhan211)
+- Minor: make some physical_optimizer rules public [#11171](https://github.com/apache/datafusion/pull/11171) (askalt)
+- Remove pr_benchmarks.yml [#11165](https://github.com/apache/datafusion/pull/11165) (alamb)
+- Optionally display schema in explain plan [#11177](https://github.com/apache/datafusion/pull/11177) (alamb)
+- Minor: Add more support for ScalarValue::Float16 [#11156](https://github.com/apache/datafusion/pull/11156) (Lordworms)
+- Minor: fix SQLOptions::with_allow_ddl comments [#11166](https://github.com/apache/datafusion/pull/11166) (alamb)
+- Update sqllogictest requirement from 0.20.0 to 0.21.0 [#11189](https://github.com/apache/datafusion/pull/11189) (dependabot[bot])
+- Support Time Parquet Data Page Statistics [#11187](https://github.com/apache/datafusion/pull/11187) (dharanad)
+- Adds support for Dictionary data type statistics from parquet data pages. [#11195](https://github.com/apache/datafusion/pull/11195) (efredine)
+- [Minor]: Make sort_batch public [#11191](https://github.com/apache/datafusion/pull/11191) (mustafasrepo)
+- Introduce user defined SQL planner API [#11180](https://github.com/apache/datafusion/pull/11180) (jayzhan211)
+- Covert grouping to udaf [#11147](https://github.com/apache/datafusion/pull/11147) (Rachelint)
+- Make statistics_from_parquet_meta a sync function [#11205](https://github.com/apache/datafusion/pull/11205) (adriangb)
+- Allow user defined SQL planners to be registered [#11208](https://github.com/apache/datafusion/pull/11208) (samuelcolvin)
+- Recursive `unnest` [#11062](https://github.com/apache/datafusion/pull/11062) (duongcongtoai)
+- Document how to test examples in user guide, add some more coverage [#11178](https://github.com/apache/datafusion/pull/11178) (alamb)
+- Minor: Move MemoryCatalog\*Provider into a module, improve comments [#11183](https://github.com/apache/datafusion/pull/11183) (alamb)
+- Add standalone example of using the SQL frontend [#11088](https://github.com/apache/datafusion/pull/11088) (alamb)
+- Add Optimizer Sanity Checker, improve sortedness equivalence properties [#11196](https://github.com/apache/datafusion/pull/11196) (mustafasrepo)
+- Implement user defined planner for extract [#11215](https://github.com/apache/datafusion/pull/11215) (xinlifoobar)
+- Move basic SQL query examples to user guide [#11217](https://github.com/apache/datafusion/pull/11217) (alamb)
+- Support FixedSizedBinaryArray Parquet Data Page Statistics [#11200](https://github.com/apache/datafusion/pull/11200) (dharanad)
+- Implement ScalarValue::Map [#11224](https://github.com/apache/datafusion/pull/11224) (goldmedal)
+- Remove unmaintained python pre-commit configuration [#11255](https://github.com/apache/datafusion/pull/11255) (findepi)
+- Enable `clone_on_ref_ptr` clippy lint on execution crate [#11239](https://github.com/apache/datafusion/pull/11239) (lewiszlw)
+- Minor: Improve documentation about pushdown join predicates [#11209](https://github.com/apache/datafusion/pull/11209) (alamb)
+- Minor: clean up data page statistics tests and fix bugs [#11236](https://github.com/apache/datafusion/pull/11236) (efredine)
+- Replacing pattern matching through downcast with trait method [#11257](https://github.com/apache/datafusion/pull/11257) (edmondop)
+- Update substrait requirement from 0.34.0 to 0.35.0 [#11206](https://github.com/apache/datafusion/pull/11206) (dependabot[bot])
+- Enhance short circuit handling in `CommonSubexprEliminate` [#11197](https://github.com/apache/datafusion/pull/11197) (peter-toth)
+- Add bench for data page statistics parquet extraction [#10950](https://github.com/apache/datafusion/pull/10950) (marvinlanhenke)
+- Register SQL planners in `SessionState` constructor [#11253](https://github.com/apache/datafusion/pull/11253) (dharanad)
+- Support DuckDB style struct syntax [#11214](https://github.com/apache/datafusion/pull/11214) (jayzhan211)
+- Enable `clone_on_ref_ptr` clippy lint on expr crate [#11238](https://github.com/apache/datafusion/pull/11238) (lewiszlw)
+- Optimize PushDownFilter to avoid recreating schema columns [#11211](https://github.com/apache/datafusion/pull/11211) (alamb)
+- Remove outdated `rewrite_expr.rs` example [#11085](https://github.com/apache/datafusion/pull/11085) (alamb)
+- Implement TPCH substrait integration teset, support tpch_2 [#11234](https://github.com/apache/datafusion/pull/11234) (Lordworms)
+- Enable `clone_on_ref_ptr` clippy lint on physical-expr crate [#11240](https://github.com/apache/datafusion/pull/11240) (lewiszlw)
+- Add standalone `AnalyzerRule` example that implements row level access control [#11089](https://github.com/apache/datafusion/pull/11089) (alamb)
+- Replace println! with assert! if possible in DataFusion examples [#11237](https://github.com/apache/datafusion/pull/11237) (Nishi46)
+- minor: format `Expr::get_type()` [#11267](https://github.com/apache/datafusion/pull/11267) (jonahgao)
+- Fix hash join for nested types [#11232](https://github.com/apache/datafusion/pull/11232) (eejbyfeldt)
+- Infer count() aggregation is not null [#11256](https://github.com/apache/datafusion/pull/11256) (findepi)
+- Remove unnecessary qualified names [#11292](https://github.com/apache/datafusion/pull/11292) (findepi)
+- Fix running examples readme [#11225](https://github.com/apache/datafusion/pull/11225) (findepi)
+- Minor: Add `ConstExpr::from` and use in physical optimizer [#11283](https://github.com/apache/datafusion/pull/11283) (alamb)
+- Implement TPCH substrait integration teset, support tpch_3 [#11298](https://github.com/apache/datafusion/pull/11298) (Lordworms)
+- Implement user defined planner for position [#11243](https://github.com/apache/datafusion/pull/11243) (xinlifoobar)
+- Upgrade to arrow 52.1.0 (and fix clippy issues on main) [#11302](https://github.com/apache/datafusion/pull/11302) (alamb)
+- AggregateExec: Take grouping sets into account for InputOrderMode [#11301](https://github.com/apache/datafusion/pull/11301) (thinkharderdev)
+- Add user_defined_sql_planners(..) to FunctionRegistry [#11296](https://github.com/apache/datafusion/pull/11296) (Omega359)
+- use safe cast in propagate_constraints [#11297](https://github.com/apache/datafusion/pull/11297) (Lordworms)
+- Minor: Remove clone in optimizer [#11315](https://github.com/apache/datafusion/pull/11315) (jayzhan211)
+- minor: Add `PhysicalSortExpr::new` [#11310](https://github.com/apache/datafusion/pull/11310) (andygrove)
+- Fix data page statistics when all rows are null in a data page [#11295](https://github.com/apache/datafusion/pull/11295) (efredine)
+- Made UserDefinedFunctionPlanner to uniform the usages [#11318](https://github.com/apache/datafusion/pull/11318) (xinlifoobar)
+- Implement user defined planner for `create_struct` & `create_named_struct` [#11273](https://github.com/apache/datafusion/pull/11273) (dharanad)
+- Improve stats convert performance for Binary/String/Boolean arrays [#11319](https://github.com/apache/datafusion/pull/11319) (Rachelint)
+- Fix typos in datafusion-examples/datafusion-cli/docs [#11259](https://github.com/apache/datafusion/pull/11259) (lewiszlw)
+- Minor: Fix Failing TPC-DS Test [#11331](https://github.com/apache/datafusion/pull/11331) (berkaysynnada)
+- HashJoin can preserve the right ordering when join type is Right [#11276](https://github.com/apache/datafusion/pull/11276) (berkaysynnada)
+- Update substrait requirement from 0.35.0 to 0.36.0 [#11328](https://github.com/apache/datafusion/pull/11328) (dependabot[bot])
+- Support to uparse logical plans with timestamp cast to string [#11326](https://github.com/apache/datafusion/pull/11326) (sgrebnov)
+- Implement user defined planner for sql_substring_to_expr [#11327](https://github.com/apache/datafusion/pull/11327) (xinlifoobar)
+- Improve volatile expression handling in `CommonSubexprEliminate` [#11265](https://github.com/apache/datafusion/pull/11265) (peter-toth)
+- Support `IS NULL` and `IS NOT NULL` on Unions [#11321](https://github.com/apache/datafusion/pull/11321) (samuelcolvin)
+- Implement TPCH substrait integration test, support tpch_4 and tpch_5 [#11311](https://github.com/apache/datafusion/pull/11311) (Lordworms)
+- Enable `clone_on_ref_ptr` clippy lint on physical-plan crate [#11241](https://github.com/apache/datafusion/pull/11241) (lewiszlw)
+- Remove any aliases in `Filter::try_new` rather than erroring [#11307](https://github.com/apache/datafusion/pull/11307) (samuelcolvin)
+- Improve `DataFrame` Users Guide [#11324](https://github.com/apache/datafusion/pull/11324) (alamb)
+- chore: Rename UserDefinedSQLPlanner to ExprPlanner [#11338](https://github.com/apache/datafusion/pull/11338) (andygrove)
+- Revert "remove `derive(Copy)` from `Operator` (#11132)" [#11341](https://github.com/apache/datafusion/pull/11341) (alamb)
+
+## Credits
+
+Thank you to everyone who contributed to this release. Here is a breakdown of commits (PRs merged) per contributor.
+
+```
+    41	Andrew Lamb
+    17	Jay Zhan
+    12	Lordworms
+    12	张林伟
+    10	Arttu
+     9	Jax Liu
+     9	Lorrens Pantelis
+     8	Piotr Findeisen
+     7	Dharan Aditya
+     7	Jonah Gao
+     7	Xin Li
+     6	Andy Grove
+     6	Marvin Lanhenke
+     6	Trent Hauck
+     5	Alex Huang
+     5	Eric Fredine
+     5	Mustafa Akur
+     5	Oleks V
+     5	dependabot[bot]
+     4	Adrian Garcia Badaracco
+     4	Berkay Şahin
+     4	Kevin Su
+     4	Peter Toth
+     4	Ruihang Xia
+     4	Samuel Colvin
+     3	Bruce Ritchie
+     3	Edmondo Porcu
+     3	Emil Ejbyfeldt
+     3	Heran Lin
+     3	Leonardo Yvens
+     3	jcsherin
+     3	tmi
+     2	Duong Cong Toai
+     2	Liang-Chi Hsieh
+     2	Max Burke
+     2	kamille
+     1	Albert Skalt
+     1	Andrey Koshchiy
+     1	Benjamin Bannier
+     1	Bo Lin
+     1	Chojan Shang
+     1	Chunchun Ye
+     1	Dan Harris
+     1	Devin D'Angelo
+     1	Eduardo Vega
+     1	Georgi Krastev
+     1	Hector Veiga
+     1	Jeffrey Smith II
+     1	Kirill Khramkov
+     1	Matt Nawara
+     1	Mohamed Abdeen
+     1	Nga Tran
+     1	Nishi
+     1	Phillip LeBlanc
+     1	R. Tyler Croy
+     1	RT_Enzyme
+     1	Sava Vranešević
+     1	Sergei Grebnov
+     1	Weston Pace
+     1	Xiangpeng Hao
+     1	advancedxy
+     1	c8ef
+     1	gstvg
+     1	yfu
+```
+
+Thank you also to everyone who contributed in other ways such as filing issues, reviewing PRs, and providing feedback on this release.
diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md
index 303caef57700..579088f991ef 100644
--- a/docs/source/user-guide/configs.md
+++ b/docs/source/user-guide/configs.md
@@ -64,7 +64,7 @@ Environment variables are read during `SessionConfig` initialisation so they mus
 | datafusion.execution.parquet.statistics_enabled                         | NULL                      | Sets if statistics are enabled for any column Valid values are: "none", "chunk", and "page" These values are not case sensitive. If NULL, uses default parquet writer setting                                                                                                                                                                                                                                                                                                                                                                                                                           |
 | datafusion.execution.parquet.max_statistics_size                        | NULL                      | Sets max statistics size for any column. If NULL, uses default parquet writer setting                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                   |
 | datafusion.execution.parquet.max_row_group_size                         | 1048576                   | Target maximum number of rows in each row group (defaults to 1M rows). Writing larger row groups requires more memory to write, but can get better compression and be faster to read.                                                                                                                                                                                                                                                                                                                                                                                                                   |
-| datafusion.execution.parquet.created_by                                 | datafusion version 39.0.0 | Sets "created by" property                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                              |
+| datafusion.execution.parquet.created_by                                 | datafusion version 40.0.0 | Sets "created by" property                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                              |
 | datafusion.execution.parquet.column_index_truncate_length               | NULL                      | Sets column index truncate length                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                       |
 | datafusion.execution.parquet.data_page_row_count_limit                  | 18446744073709551615      | Sets best effort maximum number of rows in data page                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                    |
 | datafusion.execution.parquet.encoding                                   | NULL                      | Sets default encoding for any column Valid values are: plain, plain_dictionary, rle, bit_packed, delta_binary_packed, delta_length_byte_array, delta_byte_array, rle_dictionary, and byte_stream_split. These values are not case sensitive. If NULL, uses default parquet writer setting                                                                                                                                                                                                                                                                                                               |

From fa0191772e87e04da2598aedb7fe11dd49f88f88 Mon Sep 17 00:00:00 2001
From: Xin Li <33629085+xinlifoobar@users.noreply.github.com>
Date: Tue, 9 Jul 2024 22:12:48 +0900
Subject: [PATCH 8/9] Support `NULL` literals in where clause  (#11266)

* Try fix where clause incorrectly reject NULL literal

* check null in filter
---
 datafusion/expr/src/logical_plan/plan.rs    |  3 +-
 datafusion/physical-plan/src/filter.rs      | 39 +++++++++++++++++----
 datafusion/sqllogictest/test_files/misc.slt | 14 ++++++++
 3 files changed, 49 insertions(+), 7 deletions(-)

diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs
index bda03fb7087a..998b5bdcb60c 100644
--- a/datafusion/expr/src/logical_plan/plan.rs
+++ b/datafusion/expr/src/logical_plan/plan.rs
@@ -2123,7 +2123,8 @@ impl Filter {
         // construction (such as with correlated subqueries) so we make a best effort here and
         // ignore errors resolving the expression against the schema.
         if let Ok(predicate_type) = predicate.get_type(input.schema()) {
-            if predicate_type != DataType::Boolean {
+            // Interpret NULL as a missing boolean value.
+            if predicate_type != DataType::Boolean && predicate_type != DataType::Null {
                 return plan_err!(
                     "Cannot create filter with non-boolean predicate '{predicate}' returning {predicate_type}"
                 );
diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs
index 96ec6c0cf34d..84afc227578f 100644
--- a/datafusion/physical-plan/src/filter.rs
+++ b/datafusion/physical-plan/src/filter.rs
@@ -31,13 +31,13 @@ use crate::{
     metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet},
     DisplayFormatType, ExecutionPlan,
 };
-
 use arrow::compute::filter_record_batch;
 use arrow::datatypes::{DataType, SchemaRef};
 use arrow::record_batch::RecordBatch;
-use datafusion_common::cast::as_boolean_array;
+use arrow_array::{Array, BooleanArray};
+use datafusion_common::cast::{as_boolean_array, as_null_array};
 use datafusion_common::stats::Precision;
-use datafusion_common::{plan_err, DataFusionError, Result};
+use datafusion_common::{internal_err, plan_err, DataFusionError, Result};
 use datafusion_execution::TaskContext;
 use datafusion_expr::Operator;
 use datafusion_physical_expr::expressions::BinaryExpr;
@@ -84,6 +84,19 @@ impl FilterExec {
                     cache,
                 })
             }
+            DataType::Null => {
+                let default_selectivity = 0;
+                let cache =
+                    Self::compute_properties(&input, &predicate, default_selectivity)?;
+
+                Ok(Self {
+                    predicate,
+                    input: input.clone(),
+                    metrics: ExecutionPlanMetricsSet::new(),
+                    default_selectivity,
+                    cache,
+                })
+            }
             other => {
                 plan_err!("Filter predicate must return boolean values, not {other:?}")
             }
@@ -355,9 +368,23 @@ pub(crate) fn batch_filter(
         .evaluate(batch)
         .and_then(|v| v.into_array(batch.num_rows()))
         .and_then(|array| {
-            Ok(as_boolean_array(&array)?)
-                // apply filter array to record batch
-                .and_then(|filter_array| Ok(filter_record_batch(batch, filter_array)?))
+            let filter_array = match as_boolean_array(&array) {
+                Ok(boolean_array) => {
+                    Ok(boolean_array.to_owned())
+                },
+                Err(_) => {
+                    let Ok(null_array) = as_null_array(&array) else {
+                        return internal_err!("Cannot create filter_array from non-boolean predicates, unable to continute");
+                    };
+
+                    // if the predicate is null, then the result is also null
+                    Ok::<BooleanArray, DataFusionError>(BooleanArray::new_null(
+                        null_array.len(),
+                    ))
+                }
+            }?;
+
+            Ok(filter_record_batch(batch, &filter_array)?)
         })
 }
 
diff --git a/datafusion/sqllogictest/test_files/misc.slt b/datafusion/sqllogictest/test_files/misc.slt
index 848cdc943914..66606df83480 100644
--- a/datafusion/sqllogictest/test_files/misc.slt
+++ b/datafusion/sqllogictest/test_files/misc.slt
@@ -24,3 +24,17 @@ query TT?
 select 'foo', '', NULL
 ----
 foo (empty) NULL
+
+# Where clause accept NULL literal
+query I
+select 1 where NULL
+----
+
+query I
+select 1 where NULL and 1 = 1
+----
+
+query I
+select 1 where NULL or 1 = 1
+----
+1
\ No newline at end of file

From e65c3e919855c9977cf4d80c0630ee26b7fd03ee Mon Sep 17 00:00:00 2001
From: Andrew Lamb <andrew@nerdnetworks.org>
Date: Tue, 9 Jul 2024 09:13:11 -0400
Subject: [PATCH 9/9] Minor: Add link to blog to main DataFusion website
 (#11356)

---
 docs/source/index.rst | 1 +
 1 file changed, 1 insertion(+)

diff --git a/docs/source/index.rst b/docs/source/index.rst
index 8677b63c916a..d491df04f7fe 100644
--- a/docs/source/index.rst
+++ b/docs/source/index.rst
@@ -69,6 +69,7 @@ See the `developer’s guide`_ for contributing and `communication`_ for getting
    GitHub and Issue Tracker <https://github.com/apache/datafusion>
    crates.io <https://crates.io/crates/datafusion>
    API Docs <https://docs.rs/datafusion/latest/datafusion/>
+   Blog <https://datafusion.apache.org/blog/>
    Code of conduct <https://github.com/apache/datafusion/blob/main/CODE_OF_CONDUCT.md>
    Download <download>