From 1152e9799d996adb9a31fa52475aeaa0ebb404f8 Mon Sep 17 00:00:00 2001
From: Jk Xu <54522439+Dousir9@users.noreply.github.com>
Date: Sat, 25 May 2024 17:17:01 +0800
Subject: [PATCH 1/3] feat(planner): support ExpressionScan and CacheScan
 (#15561)

* refactor(planner): refactor binder

* chore: merge main

* chore: remove eq_scalars

* chore: fix bind join

* feat(binder): support expression scan

* feat(planner): add ExpressionScan and CacheScan logical operator

* feat(planner): add hash join build cache

* feat(planner): add ExpressionScan and CacheScan physical operator

* feat(query): add TransformExpressionScan and TransformCacheScan

* chore: make lint

* chore: remove useless code

* chore: merge main

* chore: merge main

* chore: fix join bind context

* chore: refine Cache Scan

* chore: add sqllogictest

* chore: refine code

* chore: fix sqllogictest

* chore: refine bind_values

* chore: add sqllogictest

* chore: rename file

* chore: fix logical operator prop

* chore: make lint

* chore: refine code

* chore: add more comments

* chore: make lint

* chore: fix and add sqllogictest

* chore: fix and add sqllogictest

* chore: merge main

* chore: merge main

* chore: merge main

* chore: remove useless code
---
 .../src/pipelines/builders/builder_join.rs    |   9 +
 .../src/pipelines/builders/builder_scan.rs    |  65 ++
 .../pipelines/builders/builder_union_all.rs   |   1 +
 .../service/src/pipelines/pipeline_builder.rs |   8 +
 .../transforms/hash_join/hash_join_state.rs   |  49 +
 .../pipelines/processors/transforms/mod.rs    |   6 +
 .../transforms/transform_cache_scan.rs        | 133 +++
 .../transforms/transform_expression_scan.rs   |  77 ++
 .../src/schedulers/fragments/fragmenter.rs    |   1 +
 .../it/pipelines/builders/runtime_filter.rs   |   1 +
 src/query/sql/src/executor/format.rs          |  69 ++
 src/query/sql/src/executor/physical_plan.rs   |  22 +
 .../sql/src/executor/physical_plan_builder.rs |   4 +
 .../sql/src/executor/physical_plan_display.rs |  38 +
 .../sql/src/executor/physical_plan_visitor.rs |  15 +
 .../sql/src/executor/physical_plans/mod.rs    |   6 +
 .../physical_plans/physical_cache_scan.rs     |  62 ++
 .../physical_expression_scan.rs               |  78 ++
 .../physical_plans/physical_hash_join.rs      |  31 +-
 .../executor/physical_plans/physical_join.rs  |  15 +-
 .../sql/src/planner/binder/bind_query/bind.rs | 170 +++
 .../{limit.rs => bind_query/bind_limit.rs}    |  56 +-
 .../planner/binder/bind_query/bind_select.rs  | 559 ++++++++++
 .../binder/bind_query/bind_set_expr.rs        |  52 +
 .../planner/binder/bind_query/bind_value.rs   | 726 +++++++++++++
 .../sql/src/planner/binder/bind_query/mod.rs  |  23 +
 .../binder/bind_table_reference/bind.rs       |  87 ++
 .../bind_join.rs}                             | 339 +++---
 .../bind_table_reference/bind_location.rs     |  65 ++
 .../bind_table_reference/bind_subquery.rs     |  60 ++
 .../binder/bind_table_reference/bind_table.rs | 291 ++++++
 .../bind_table_function.rs                    | 495 +++++++++
 .../binder/bind_table_reference/mod.rs        |  23 +
 src/query/sql/src/planner/binder/binder.rs    | 170 ++-
 .../src/planner/binder/copy_into_location.rs  |   4 +-
 .../sql/src/planner/binder/copy_into_table.rs |   6 +-
 src/query/sql/src/planner/binder/delete.rs    |   8 +-
 src/query/sql/src/planner/binder/insert.rs    |   6 +-
 .../src/planner/binder/insert_multi_table.rs  |   4 +-
 .../sql/src/planner/binder/merge_into.rs      |  15 +-
 src/query/sql/src/planner/binder/mod.rs       |   9 +-
 src/query/sql/src/planner/binder/select.rs    | 640 +-----------
 src/query/sql/src/planner/binder/sort.rs      |  56 -
 src/query/sql/src/planner/binder/table.rs     | 973 +-----------------
 src/query/sql/src/planner/binder/update.rs    |   6 +-
 src/query/sql/src/planner/binder/values.rs    | 224 ----
 src/query/sql/src/planner/dataframe.rs        |  38 +-
 .../optimizer/decorrelate/decorrelate.rs      |   8 +-
 .../optimizer/decorrelate/flatten_plan.rs     |  24 +
 .../decorrelate/subquery_rewriter.rs          |  10 +-
 src/query/sql/src/planner/optimizer/format.rs |   2 +
 .../src/planner/optimizer/hyper_dp/dphyp.rs   |   4 +-
 .../planner/optimizer/hyper_dp/join_node.rs   |   1 +
 .../rule/rewrite/rule_commute_join.rs         |   5 +
 .../rule/rewrite/rule_semi_to_inner_join.rs   |   2 +
 src/query/sql/src/planner/optimizer/s_expr.rs |  12 +-
 src/query/sql/src/planner/plans/cache_scan.rs | 136 +++
 .../sql/src/planner/plans/expression_scan.rs  | 122 +++
 src/query/sql/src/planner/plans/join.rs       |   8 +
 src/query/sql/src/planner/plans/mod.rs        |   4 +
 src/query/sql/src/planner/plans/operator.rs   |  34 +
 src/query/sql/src/planner/semantic/mod.rs     |   2 +-
 .../sql/src/planner/semantic/type_check.rs    |   1 +
 .../standalone/explain/expression_scan.test   | 268 +++++
 .../explain_native/expression_scan.test       | 269 +++++
 tests/sqllogictests/suites/query/lateral.test |  46 +
 66 files changed, 4585 insertions(+), 2168 deletions(-)
 create mode 100644 src/query/service/src/pipelines/processors/transforms/transform_cache_scan.rs
 create mode 100644 src/query/service/src/pipelines/processors/transforms/transform_expression_scan.rs
 create mode 100644 src/query/sql/src/executor/physical_plans/physical_cache_scan.rs
 create mode 100644 src/query/sql/src/executor/physical_plans/physical_expression_scan.rs
 create mode 100644 src/query/sql/src/planner/binder/bind_query/bind.rs
 rename src/query/sql/src/planner/binder/{limit.rs => bind_query/bind_limit.rs} (62%)
 create mode 100644 src/query/sql/src/planner/binder/bind_query/bind_select.rs
 create mode 100644 src/query/sql/src/planner/binder/bind_query/bind_set_expr.rs
 create mode 100644 src/query/sql/src/planner/binder/bind_query/bind_value.rs
 create mode 100644 src/query/sql/src/planner/binder/bind_query/mod.rs
 create mode 100644 src/query/sql/src/planner/binder/bind_table_reference/bind.rs
 rename src/query/sql/src/planner/binder/{join.rs => bind_table_reference/bind_join.rs} (78%)
 create mode 100644 src/query/sql/src/planner/binder/bind_table_reference/bind_location.rs
 create mode 100644 src/query/sql/src/planner/binder/bind_table_reference/bind_subquery.rs
 create mode 100644 src/query/sql/src/planner/binder/bind_table_reference/bind_table.rs
 create mode 100644 src/query/sql/src/planner/binder/bind_table_reference/bind_table_function.rs
 create mode 100644 src/query/sql/src/planner/binder/bind_table_reference/mod.rs
 delete mode 100644 src/query/sql/src/planner/binder/values.rs
 create mode 100644 src/query/sql/src/planner/plans/cache_scan.rs
 create mode 100644 src/query/sql/src/planner/plans/expression_scan.rs
 create mode 100644 tests/sqllogictests/suites/mode/standalone/explain/expression_scan.test
 create mode 100644 tests/sqllogictests/suites/mode/standalone/explain_native/expression_scan.test

diff --git a/src/query/service/src/pipelines/builders/builder_join.rs b/src/query/service/src/pipelines/builders/builder_join.rs
index f81299b917758..99aa9c9c6f5d4 100644
--- a/src/query/service/src/pipelines/builders/builder_join.rs
+++ b/src/query/service/src/pipelines/builders/builder_join.rs
@@ -80,6 +80,8 @@ impl PipelineBuilder {
             self.main_pipeline.get_scopes(),
         );
         right_side_builder.cte_state = self.cte_state.clone();
+        right_side_builder.hash_join_states = self.hash_join_states.clone();
+
         let mut right_res = right_side_builder.finalize(&range_join.right)?;
         right_res.main_pipeline.add_sink(|input| {
             Ok(ProcessorPtr::create(
@@ -104,6 +106,10 @@ impl PipelineBuilder {
             merge_into_is_distributed,
             enable_merge_into_optimization,
         )?;
+        if let Some((build_cache_index, _)) = join.build_side_cache_info {
+            self.hash_join_states
+                .insert(build_cache_index, state.clone());
+        }
         self.expand_build_side_pipeline(&join.build, join, state.clone())?;
         self.build_join_probe(join, state)
     }
@@ -122,6 +128,7 @@ impl PipelineBuilder {
             &join.probe_to_build,
             merge_into_is_distributed,
             enable_merge_into_optimization,
+            join.build_side_cache_info.clone(),
         )
     }
 
@@ -139,6 +146,7 @@ impl PipelineBuilder {
             self.main_pipeline.get_scopes(),
         );
         build_side_builder.cte_state = self.cte_state.clone();
+        build_side_builder.hash_join_states = self.hash_join_states.clone();
         let mut build_res = build_side_builder.finalize(build)?;
 
         assert!(build_res.main_pipeline.is_pulling_pipeline()?);
@@ -267,6 +275,7 @@ impl PipelineBuilder {
             self.main_pipeline.get_scopes(),
         );
         left_side_builder.cte_state = self.cte_state.clone();
+        left_side_builder.hash_join_states = self.hash_join_states.clone();
         let mut left_side_pipeline = left_side_builder.finalize(left_side)?;
         assert!(left_side_pipeline.main_pipeline.is_pulling_pipeline()?);
 
diff --git a/src/query/service/src/pipelines/builders/builder_scan.rs b/src/query/service/src/pipelines/builders/builder_scan.rs
index a10ad04900110..dd4e07bbf6bcf 100644
--- a/src/query/service/src/pipelines/builders/builder_scan.rs
+++ b/src/query/service/src/pipelines/builders/builder_scan.rs
@@ -13,19 +13,28 @@
 // limitations under the License.
 
 use databend_common_catalog::table_context::TableContext;
+use databend_common_exception::ErrorCode;
 use databend_common_exception::Result;
 use databend_common_expression::DataBlock;
+use databend_common_functions::BUILTIN_FUNCTIONS;
 use databend_common_pipeline_core::processors::ProcessorPtr;
 use databend_common_pipeline_sources::OneBlockSource;
 use databend_common_sql::evaluator::BlockOperator;
 use databend_common_sql::evaluator::CompoundBlockOperator;
+use databend_common_sql::executor::physical_plans::CacheScan;
 use databend_common_sql::executor::physical_plans::ConstantTableScan;
 use databend_common_sql::executor::physical_plans::CteScan;
+use databend_common_sql::executor::physical_plans::ExpressionScan;
 use databend_common_sql::executor::physical_plans::TableScan;
+use databend_common_sql::plans::CacheSource;
 use databend_common_sql::StreamContext;
 
+use crate::pipelines::processors::transforms::CacheSourceState;
+use crate::pipelines::processors::transforms::HashJoinCacheState;
 use crate::pipelines::processors::transforms::MaterializedCteSource;
 use crate::pipelines::processors::transforms::TransformAddInternalColumns;
+use crate::pipelines::processors::transforms::TransformCacheScan;
+use crate::pipelines::processors::transforms::TransformExpressionScan;
 use crate::pipelines::processors::TransformAddStreamColumns;
 use crate::pipelines::PipelineBuilder;
 
@@ -115,4 +124,60 @@ impl PipelineBuilder {
             1,
         )
     }
+
+    pub(crate) fn build_cache_scan(&mut self, scan: &CacheScan) -> Result<()> {
+        let max_threads = self.settings.get_max_threads()?;
+        let max_block_size = self.settings.get_max_block_size()? as usize;
+        let cache_source_state = match &scan.cache_source {
+            CacheSource::HashJoinBuild((cache_index, column_indexes)) => {
+                let hash_join_state = match self.hash_join_states.get(cache_index) {
+                    Some(hash_join_state) => hash_join_state.clone(),
+                    None => {
+                        return Err(ErrorCode::Internal(
+                            "Hash join state not found during building cache scan".to_string(),
+                        ));
+                    }
+                };
+                CacheSourceState::HashJoinCacheState(HashJoinCacheState::new(
+                    column_indexes.clone(),
+                    hash_join_state,
+                    max_block_size,
+                ))
+            }
+        };
+
+        self.main_pipeline.add_source(
+            |output| {
+                TransformCacheScan::create(self.ctx.clone(), output, cache_source_state.clone())
+            },
+            max_threads as usize,
+        )
+    }
+
+    pub(crate) fn build_expression_scan(&mut self, scan: &ExpressionScan) -> Result<()> {
+        self.build_pipeline(&scan.input)?;
+
+        let values = scan
+            .values
+            .iter()
+            .map(|row| {
+                row.iter()
+                    .map(|scalar| scalar.as_expr(&BUILTIN_FUNCTIONS))
+                    .collect::<Vec<_>>()
+            })
+            .collect::<Vec<_>>();
+
+        let fun_ctx = self.func_ctx.clone();
+
+        self.main_pipeline.add_transform(|input, output| {
+            Ok(ProcessorPtr::create(TransformExpressionScan::create(
+                input,
+                output,
+                values.clone(),
+                fun_ctx.clone(),
+            )))
+        })?;
+
+        Ok(())
+    }
 }
diff --git a/src/query/service/src/pipelines/builders/builder_union_all.rs b/src/query/service/src/pipelines/builders/builder_union_all.rs
index c7b55f57f05af..b411c13b7939c 100644
--- a/src/query/service/src/pipelines/builders/builder_union_all.rs
+++ b/src/query/service/src/pipelines/builders/builder_union_all.rs
@@ -51,6 +51,7 @@ impl PipelineBuilder {
             self.main_pipeline.get_scopes(),
         );
         pipeline_builder.cte_state = self.cte_state.clone();
+        pipeline_builder.hash_join_states = self.hash_join_states.clone();
 
         let mut build_res = pipeline_builder.finalize(input)?;
 
diff --git a/src/query/service/src/pipelines/pipeline_builder.rs b/src/query/service/src/pipelines/pipeline_builder.rs
index 9e154894cef19..eea5cdd209dfc 100644
--- a/src/query/service/src/pipelines/pipeline_builder.rs
+++ b/src/query/service/src/pipelines/pipeline_builder.rs
@@ -31,6 +31,7 @@ use databend_common_sql::IndexType;
 use super::PipelineBuilderData;
 use crate::pipelines::processors::transforms::HashJoinBuildState;
 use crate::pipelines::processors::transforms::MaterializedCteState;
+use crate::pipelines::processors::HashJoinState;
 use crate::pipelines::PipelineBuildResult;
 use crate::servers::flight::v1::exchange::DefaultExchangeInjector;
 use crate::servers::flight::v1::exchange::ExchangeInjector;
@@ -52,6 +53,8 @@ pub struct PipelineBuilder {
     pub cte_state: HashMap<IndexType, Arc<MaterializedCteState>>,
 
     pub(crate) exchange_injector: Arc<dyn ExchangeInjector>,
+
+    pub hash_join_states: HashMap<usize, Arc<HashJoinState>>,
 }
 
 impl PipelineBuilder {
@@ -71,6 +74,7 @@ impl PipelineBuilder {
             cte_state: HashMap::new(),
             merge_into_probe_data_fields: None,
             join_state: None,
+            hash_join_states: HashMap::new(),
         }
     }
 
@@ -157,6 +161,10 @@ impl PipelineBuilder {
             PhysicalPlan::MaterializedCte(materialized_cte) => {
                 self.build_materialized_cte(materialized_cte)
             }
+            PhysicalPlan::CacheScan(cache_scan) => self.build_cache_scan(cache_scan),
+            PhysicalPlan::ExpressionScan(expression_scan) => {
+                self.build_expression_scan(expression_scan)
+            }
 
             // Copy into.
             PhysicalPlan::CopyIntoTable(copy) => self.build_copy_into_table(copy),
diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_state.rs
index 7a2817f948d30..1cc620218c248 100644
--- a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_state.rs
+++ b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_state.rs
@@ -13,6 +13,7 @@
 // limitations under the License.
 
 use std::cell::SyncUnsafeCell;
+use std::collections::HashMap;
 use std::collections::HashSet;
 use std::sync::atomic::AtomicBool;
 use std::sync::atomic::AtomicI8;
@@ -26,6 +27,7 @@ use databend_common_base::base::tokio::sync::watch::Sender;
 use databend_common_catalog::table_context::TableContext;
 use databend_common_exception::ErrorCode;
 use databend_common_exception::Result;
+use databend_common_expression::BlockEntry;
 use databend_common_expression::DataSchemaRef;
 use databend_common_expression::HashMethodFixedKeys;
 use databend_common_expression::HashMethodSerializer;
@@ -44,6 +46,7 @@ use crate::pipelines::processors::transforms::hash_join::row::RowSpace;
 use crate::pipelines::processors::transforms::hash_join::util::build_schema_wrap_nullable;
 use crate::pipelines::processors::HashJoinDesc;
 use crate::sessions::QueryContext;
+use crate::sql::IndexType;
 
 pub struct SerializerHashJoinHashTable {
     pub(crate) hash_table: BinaryHashJoinHashMap,
@@ -120,6 +123,12 @@ pub struct HashJoinState {
     pub(crate) enable_spill: bool,
 
     pub(crate) merge_into_state: Option<SyncUnsafeCell<MergeIntoState>>,
+
+    /// Build side cache info.
+    /// A HashMap for mapping the column indexes to the BlockEntry indexes in DataBlock.
+    pub(crate) column_map: HashMap<usize, usize>,
+    // The index of the next cache block to be read.
+    pub(crate) next_cache_block_index: AtomicUsize,
 }
 
 impl HashJoinState {
@@ -131,6 +140,7 @@ impl HashJoinState {
         probe_to_build: &[(usize, (bool, bool))],
         merge_into_is_distributed: bool,
         enable_merge_into_optimization: bool,
+        build_side_cache_info: Option<(usize, HashMap<IndexType, usize>)>,
     ) -> Result<Arc<HashJoinState>> {
         if matches!(
             hash_join_desc.join_type,
@@ -144,6 +154,11 @@ impl HashJoinState {
         if ctx.get_settings().get_join_spilling_memory_ratio()? != 0 {
             enable_spill = true;
         }
+        let column_map = if let Some((_, column_map)) = build_side_cache_info {
+            column_map
+        } else {
+            HashMap::new()
+        };
         Ok(Arc::new(HashJoinState {
             hash_table: SyncUnsafeCell::new(HashJoinHashTable::Null),
             hash_table_builders: AtomicUsize::new(0),
@@ -166,6 +181,8 @@ impl HashJoinState {
                     merge_into_is_distributed,
                 )),
             },
+            column_map,
+            next_cache_block_index: AtomicUsize::new(0),
         }))
     }
 
@@ -256,4 +273,36 @@ impl HashJoinState {
         }
         build_state.generation_state.is_build_projected = true;
     }
+
+    pub fn num_build_chunks(&self) -> usize {
+        let build_state = unsafe { &*self.build_state.get() };
+        build_state.generation_state.chunks.len()
+    }
+
+    pub fn get_cached_columns(&self, column_index: usize) -> Vec<BlockEntry> {
+        let index = self.column_map.get(&column_index).unwrap();
+        let build_state = unsafe { &*self.build_state.get() };
+        let columns = build_state
+            .generation_state
+            .chunks
+            .iter()
+            .map(|data_block| data_block.get_by_offset(*index).clone())
+            .collect::<Vec<_>>();
+        columns
+    }
+
+    pub fn get_cached_num_rows(&self) -> Vec<usize> {
+        let build_state = unsafe { &*self.build_state.get() };
+        let num_rows = build_state
+            .generation_state
+            .chunks
+            .iter()
+            .map(|data_block| data_block.num_rows())
+            .collect::<Vec<_>>();
+        num_rows
+    }
+
+    pub fn next_cache_block_index(&self) -> usize {
+        self.next_cache_block_index.fetch_add(1, Ordering::Relaxed)
+    }
 }
diff --git a/src/query/service/src/pipelines/processors/transforms/mod.rs b/src/query/service/src/pipelines/processors/transforms/mod.rs
index 6b3784301876a..b3cce44c3b55b 100644
--- a/src/query/service/src/pipelines/processors/transforms/mod.rs
+++ b/src/query/service/src/pipelines/processors/transforms/mod.rs
@@ -23,8 +23,10 @@ mod transform_add_computed_columns;
 mod transform_add_const_columns;
 mod transform_add_internal_columns;
 mod transform_add_stream_columns;
+mod transform_cache_scan;
 mod transform_cast_schema;
 mod transform_create_sets;
+mod transform_expression_scan;
 mod transform_filter;
 mod transform_limit;
 mod transform_materialized_cte;
@@ -47,8 +49,12 @@ pub use transform_add_computed_columns::TransformAddComputedColumns;
 pub use transform_add_const_columns::TransformAddConstColumns;
 pub use transform_add_internal_columns::TransformAddInternalColumns;
 pub use transform_add_stream_columns::TransformAddStreamColumns;
+pub use transform_cache_scan::CacheSourceState;
+pub use transform_cache_scan::HashJoinCacheState;
+pub use transform_cache_scan::TransformCacheScan;
 pub use transform_cast_schema::TransformCastSchema;
 pub use transform_create_sets::TransformCreateSets;
+pub use transform_expression_scan::TransformExpressionScan;
 pub use transform_filter::TransformFilter;
 pub use transform_limit::TransformLimit;
 pub use transform_materialized_cte::MaterializedCteSink;
diff --git a/src/query/service/src/pipelines/processors/transforms/transform_cache_scan.rs b/src/query/service/src/pipelines/processors/transforms/transform_cache_scan.rs
new file mode 100644
index 0000000000000..6b8f2bf53f745
--- /dev/null
+++ b/src/query/service/src/pipelines/processors/transforms/transform_cache_scan.rs
@@ -0,0 +1,133 @@
+// Copyright 2021 Datafuse Labs
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+use std::collections::VecDeque;
+use std::sync::Arc;
+
+use databend_common_exception::Result;
+use databend_common_expression::BlockEntry;
+use databend_common_expression::DataBlock;
+use databend_common_pipeline_core::processors::OutputPort;
+use databend_common_pipeline_core::processors::ProcessorPtr;
+use databend_common_pipeline_sources::AsyncSource;
+use databend_common_pipeline_sources::AsyncSourcer;
+
+use crate::pipelines::processors::HashJoinState;
+use crate::sessions::QueryContext;
+
+#[derive(Clone)]
+pub enum CacheSourceState {
+    HashJoinCacheState(HashJoinCacheState),
+}
+
+impl CacheSourceState {
+    fn next_data_block(&mut self) -> Option<DataBlock> {
+        match self {
+            CacheSourceState::HashJoinCacheState(state) => state.next_data_block(),
+        }
+    }
+}
+
+#[derive(Clone)]
+pub struct HashJoinCacheState {
+    initilized: bool,
+    column_indexes: Vec<usize>,
+    columns: Vec<Vec<BlockEntry>>,
+    num_rows: Vec<usize>,
+    num_cache_blocks: usize,
+    hash_join_state: Arc<HashJoinState>,
+    output_buffer: VecDeque<DataBlock>,
+    max_block_size: usize,
+}
+
+impl HashJoinCacheState {
+    pub fn new(
+        column_indexes: Vec<usize>,
+        hash_join_state: Arc<HashJoinState>,
+        max_block_size: usize,
+    ) -> Self {
+        Self {
+            initilized: false,
+            column_indexes,
+            columns: Vec::new(),
+            num_rows: Vec::new(),
+            num_cache_blocks: 0,
+            hash_join_state,
+            output_buffer: VecDeque::new(),
+            max_block_size,
+        }
+    }
+    fn next_data_block(&mut self) -> Option<DataBlock> {
+        if !self.initilized {
+            self.num_cache_blocks = self.hash_join_state.num_build_chunks();
+            for column_index in self.column_indexes.iter() {
+                let column = self.hash_join_state.get_cached_columns(*column_index);
+                self.columns.push(column);
+            }
+            self.num_rows = self.hash_join_state.get_cached_num_rows();
+            self.initilized = true;
+        }
+
+        if let Some(data_block) = self.output_buffer.pop_front() {
+            return Some(data_block);
+        }
+
+        let next_cache_block_index = self.hash_join_state.next_cache_block_index();
+        if next_cache_block_index >= self.num_cache_blocks {
+            // Release memory.
+            self.columns.clear();
+            return None;
+        }
+
+        let block_entries = (0..self.columns.len())
+            .map(|idx| self.columns[idx][next_cache_block_index].clone())
+            .collect::<Vec<BlockEntry>>();
+        let num_rows = self.num_rows[next_cache_block_index];
+        let data_block = DataBlock::new(block_entries, num_rows);
+
+        for data_block in data_block.split_by_rows_no_tail(self.max_block_size) {
+            self.output_buffer.push_back(data_block);
+        }
+
+        self.output_buffer.pop_front()
+    }
+}
+
+pub struct TransformCacheScan {
+    cache_source_state: CacheSourceState,
+}
+
+impl TransformCacheScan {
+    pub fn create(
+        ctx: Arc<QueryContext>,
+        output_port: Arc<OutputPort>,
+        cache_source_state: CacheSourceState,
+    ) -> Result<ProcessorPtr> {
+        AsyncSourcer::create(ctx.clone(), output_port, TransformCacheScan {
+            cache_source_state,
+        })
+    }
+}
+
+#[async_trait::async_trait]
+impl AsyncSource for TransformCacheScan {
+    const NAME: &'static str = "TransformCacheScan";
+
+    #[async_trait::unboxed_simple]
+    #[async_backtrace::framed]
+    async fn generate(&mut self) -> Result<Option<DataBlock>> {
+        let data_block = self.cache_source_state.next_data_block();
+        Ok(data_block)
+    }
+}
diff --git a/src/query/service/src/pipelines/processors/transforms/transform_expression_scan.rs b/src/query/service/src/pipelines/processors/transforms/transform_expression_scan.rs
new file mode 100644
index 0000000000000..957f82d5b47f5
--- /dev/null
+++ b/src/query/service/src/pipelines/processors/transforms/transform_expression_scan.rs
@@ -0,0 +1,77 @@
+// Copyright 2021 Datafuse Labs
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+use std::collections::VecDeque;
+use std::sync::Arc;
+
+use databend_common_exception::Result;
+use databend_common_expression::BlockEntry;
+use databend_common_expression::DataBlock;
+use databend_common_expression::Evaluator;
+use databend_common_expression::Expr;
+use databend_common_expression::FunctionContext;
+use databend_common_functions::BUILTIN_FUNCTIONS;
+use databend_common_pipeline_transforms::processors::BlockingTransform;
+use databend_common_pipeline_transforms::processors::BlockingTransformer;
+
+use crate::pipelines::processors::InputPort;
+use crate::pipelines::processors::OutputPort;
+use crate::pipelines::processors::Processor;
+
+pub struct TransformExpressionScan {
+    values: Vec<Vec<Expr>>,
+    output_buffer: VecDeque<DataBlock>,
+    func_ctx: FunctionContext,
+}
+
+impl TransformExpressionScan {
+    pub fn create(
+        input: Arc<InputPort>,
+        output: Arc<OutputPort>,
+        values: Vec<Vec<Expr>>,
+        func_ctx: FunctionContext,
+    ) -> Box<dyn Processor> {
+        BlockingTransformer::create(input, output, TransformExpressionScan {
+            values,
+            output_buffer: VecDeque::new(),
+            func_ctx,
+        })
+    }
+}
+
+impl BlockingTransform for TransformExpressionScan {
+    const NAME: &'static str = "TransformExpressionScan";
+
+    fn consume(&mut self, input: DataBlock) -> Result<()> {
+        let evaluator = Evaluator::new(&input, &self.func_ctx, &BUILTIN_FUNCTIONS);
+        for row in self.values.iter() {
+            let mut columns = Vec::with_capacity(row.len());
+            for expr in row {
+                let result = evaluator.run(expr)?;
+                let column = BlockEntry::new(expr.data_type().clone(), result);
+                columns.push(column);
+            }
+            self.output_buffer
+                .push_back(DataBlock::new(columns, input.num_rows()));
+        }
+        Ok(())
+    }
+
+    fn transform(&mut self) -> Result<Option<DataBlock>> {
+        match !self.output_buffer.is_empty() {
+            true => Ok(Some(self.output_buffer.pop_front().unwrap())),
+            false => Ok(None),
+        }
+    }
+}
diff --git a/src/query/service/src/schedulers/fragments/fragmenter.rs b/src/query/service/src/schedulers/fragments/fragmenter.rs
index b97f16fffe7dd..b647457b71b3d 100644
--- a/src/query/service/src/schedulers/fragments/fragmenter.rs
+++ b/src/query/service/src/schedulers/fragments/fragmenter.rs
@@ -257,6 +257,7 @@ impl PhysicalPlanReplacer for Fragmenter {
             enable_bloom_runtime_filter: plan.enable_bloom_runtime_filter,
             broadcast: plan.broadcast,
             single_to_inner: plan.single_to_inner.clone(),
+            build_side_cache_info: plan.build_side_cache_info.clone(),
         }))
     }
 
diff --git a/src/query/service/tests/it/pipelines/builders/runtime_filter.rs b/src/query/service/tests/it/pipelines/builders/runtime_filter.rs
index f1f5426c28539..811153ec118bd 100644
--- a/src/query/service/tests/it/pipelines/builders/runtime_filter.rs
+++ b/src/query/service/tests/it/pipelines/builders/runtime_filter.rs
@@ -91,6 +91,7 @@ async fn join_build_state(
         &join.probe_to_build,
         false,
         true,
+        None,
     )?;
     let build_state = HashJoinBuildState::try_create(
         ctx.clone(),
diff --git a/src/query/sql/src/executor/format.rs b/src/query/sql/src/executor/format.rs
index fce401cf340ba..ec7466ce7b80f 100644
--- a/src/query/sql/src/executor/format.rs
+++ b/src/query/sql/src/executor/format.rs
@@ -25,6 +25,8 @@ use databend_common_pipeline_core::processors::PlanProfile;
 use itertools::Itertools;
 
 use super::physical_plans::AsyncFunction;
+use super::physical_plans::CacheScan;
+use super::physical_plans::ExpressionScan;
 use crate::executor::explain::PlanStatsInfo;
 use crate::executor::physical_plans::AggregateExpand;
 use crate::executor::physical_plans::AggregateFinal;
@@ -60,6 +62,7 @@ use crate::executor::PhysicalPlan;
 use crate::planner::Metadata;
 use crate::planner::MetadataRef;
 use crate::planner::DUMMY_TABLE_INDEX;
+use crate::plans::CacheSource;
 
 impl PhysicalPlan {
     pub fn format(
@@ -250,6 +253,8 @@ fn to_format_tree(
             materialized_cte_to_format_tree(plan, metadata, profs)
         }
         PhysicalPlan::ConstantTableScan(plan) => constant_table_scan_to_format_tree(plan, metadata),
+        PhysicalPlan::ExpressionScan(plan) => expression_scan_to_format_tree(plan, metadata, profs),
+        PhysicalPlan::CacheScan(plan) => cache_scan_to_format_tree(plan, metadata),
         PhysicalPlan::Duplicate(plan) => {
             let mut children = Vec::new();
             children.push(FormatTreeNode::new(format!(
@@ -523,6 +528,60 @@ fn constant_table_scan_to_format_tree(
     ))
 }
 
+fn expression_scan_to_format_tree(
+    plan: &ExpressionScan,
+    metadata: &Metadata,
+    profs: &HashMap<u32, PlanProfile>,
+) -> Result<FormatTreeNode<String>> {
+    let mut children = Vec::with_capacity(plan.values.len() + 1);
+    children.push(FormatTreeNode::new(format!(
+        "output columns: [{}]",
+        format_output_columns(plan.output_schema()?, metadata, true)
+    )));
+    for (i, value) in plan.values.iter().enumerate() {
+        let column = value
+            .iter()
+            .map(|val| val.as_expr(&BUILTIN_FUNCTIONS).sql_display())
+            .join(", ");
+        children.push(FormatTreeNode::new(format!("column {}: [{}]", i, column)));
+    }
+
+    children.push(to_format_tree(&plan.input, metadata, profs)?);
+
+    Ok(FormatTreeNode::with_children(
+        "ExpressionScan".to_string(),
+        children,
+    ))
+}
+
+fn cache_scan_to_format_tree(
+    plan: &CacheScan,
+    metadata: &Metadata,
+) -> Result<FormatTreeNode<String>> {
+    let mut children = Vec::with_capacity(2);
+    children.push(FormatTreeNode::new(format!(
+        "output columns: [{}]",
+        format_output_columns(plan.output_schema()?, metadata, true)
+    )));
+
+    match &plan.cache_source {
+        CacheSource::HashJoinBuild((cache_index, column_indexes)) => {
+            let mut column_indexes = column_indexes.clone();
+            column_indexes.sort();
+            children.push(FormatTreeNode::new(format!("cache index: {}", cache_index)));
+            children.push(FormatTreeNode::new(format!(
+                "column indexes: {:?}",
+                column_indexes
+            )));
+        }
+    }
+
+    Ok(FormatTreeNode::with_children(
+        "CacheScan".to_string(),
+        children,
+    ))
+}
+
 fn filter_to_format_tree(
     plan: &Filter,
     metadata: &Metadata,
@@ -1026,6 +1085,16 @@ fn hash_join_to_format_tree(
         FormatTreeNode::new(format!("filters: [{filters}]")),
     ];
 
+    if let Some((cache_index, column_map)) = &plan.build_side_cache_info {
+        let mut column_indexes = column_map.keys().collect::<Vec<_>>();
+        column_indexes.sort();
+        children.push(FormatTreeNode::new(format!("cache index: {}", cache_index)));
+        children.push(FormatTreeNode::new(format!(
+            "cache columns: {:?}",
+            column_indexes
+        )));
+    }
+
     if let Some(info) = &plan.stat_info {
         let items = plan_stats_info_to_format_tree(info);
         children.extend(items);
diff --git a/src/query/sql/src/executor/physical_plan.rs b/src/query/sql/src/executor/physical_plan.rs
index d394e552ef054..d5686a46da46a 100644
--- a/src/query/sql/src/executor/physical_plan.rs
+++ b/src/query/sql/src/executor/physical_plan.rs
@@ -26,6 +26,7 @@ use crate::executor::physical_plans::AggregateExpand;
 use crate::executor::physical_plans::AggregateFinal;
 use crate::executor::physical_plans::AggregatePartial;
 use crate::executor::physical_plans::AsyncFunction;
+use crate::executor::physical_plans::CacheScan;
 use crate::executor::physical_plans::ChunkAppendData;
 use crate::executor::physical_plans::ChunkCastSchema;
 use crate::executor::physical_plans::ChunkCommitInsert;
@@ -47,6 +48,7 @@ use crate::executor::physical_plans::EvalScalar;
 use crate::executor::physical_plans::Exchange;
 use crate::executor::physical_plans::ExchangeSink;
 use crate::executor::physical_plans::ExchangeSource;
+use crate::executor::physical_plans::ExpressionScan;
 use crate::executor::physical_plans::Filter;
 use crate::executor::physical_plans::HashJoin;
 use crate::executor::physical_plans::Limit;
@@ -91,6 +93,8 @@ pub enum PhysicalPlan {
     CteScan(CteScan),
     MaterializedCte(MaterializedCte),
     ConstantTableScan(ConstantTableScan),
+    ExpressionScan(ExpressionScan),
+    CacheScan(CacheScan),
     Udf(Udf),
 
     /// For insert into ... select ... in cluster
@@ -244,6 +248,14 @@ impl PhysicalPlan {
                 plan.plan_id = *next_id;
                 *next_id += 1;
             }
+            PhysicalPlan::ExpressionScan(plan) => {
+                plan.plan_id = *next_id;
+                *next_id += 1;
+            }
+            PhysicalPlan::CacheScan(plan) => {
+                plan.plan_id = *next_id;
+                *next_id += 1;
+            }
             PhysicalPlan::Udf(plan) => {
                 plan.plan_id = *next_id;
                 *next_id += 1;
@@ -403,6 +415,8 @@ impl PhysicalPlan {
             PhysicalPlan::CteScan(v) => v.plan_id,
             PhysicalPlan::MaterializedCte(v) => v.plan_id,
             PhysicalPlan::ConstantTableScan(v) => v.plan_id,
+            PhysicalPlan::ExpressionScan(v) => v.plan_id,
+            PhysicalPlan::CacheScan(v) => v.plan_id,
             PhysicalPlan::Udf(v) => v.plan_id,
             PhysicalPlan::DeleteSource(v) => v.plan_id,
             PhysicalPlan::MergeInto(v) => v.plan_id,
@@ -455,6 +469,8 @@ impl PhysicalPlan {
             PhysicalPlan::CteScan(plan) => plan.output_schema(),
             PhysicalPlan::MaterializedCte(plan) => plan.output_schema(),
             PhysicalPlan::ConstantTableScan(plan) => plan.output_schema(),
+            PhysicalPlan::ExpressionScan(plan) => plan.output_schema(),
+            PhysicalPlan::CacheScan(plan) => plan.output_schema(),
             PhysicalPlan::Udf(plan) => plan.output_schema(),
             PhysicalPlan::MergeInto(plan) => Ok(plan.output_schema.clone()),
             PhysicalPlan::MergeIntoAddRowNumber(plan) => plan.output_schema(),
@@ -520,6 +536,8 @@ impl PhysicalPlan {
             PhysicalPlan::CteScan(_) => "PhysicalCteScan".to_string(),
             PhysicalPlan::MaterializedCte(_) => "PhysicalMaterializedCte".to_string(),
             PhysicalPlan::ConstantTableScan(_) => "PhysicalConstantTableScan".to_string(),
+            PhysicalPlan::ExpressionScan(_) => "ExpressionScan".to_string(),
+            PhysicalPlan::CacheScan(_) => "CacheScan".to_string(),
             PhysicalPlan::MergeIntoAddRowNumber(_) => "AddRowNumber".to_string(),
             PhysicalPlan::ReclusterSource(_) => "ReclusterSource".to_string(),
             PhysicalPlan::ReclusterSink(_) => "ReclusterSink".to_string(),
@@ -542,6 +560,7 @@ impl PhysicalPlan {
             PhysicalPlan::TableScan(_)
             | PhysicalPlan::CteScan(_)
             | PhysicalPlan::ConstantTableScan(_)
+            | PhysicalPlan::CacheScan(_)
             | PhysicalPlan::ExchangeSource(_)
             | PhysicalPlan::CompactSource(_)
             | PhysicalPlan::DeleteSource(_)
@@ -562,6 +581,7 @@ impl PhysicalPlan {
             PhysicalPlan::HashJoin(plan) => Box::new(
                 std::iter::once(plan.probe.as_ref()).chain(std::iter::once(plan.build.as_ref())),
             ),
+            PhysicalPlan::ExpressionScan(plan) => Box::new(std::iter::once(plan.input.as_ref())),
             PhysicalPlan::Exchange(plan) => Box::new(std::iter::once(plan.input.as_ref())),
             PhysicalPlan::ExchangeSink(plan) => Box::new(std::iter::once(plan.input.as_ref())),
             PhysicalPlan::UnionAll(plan) => Box::new(
@@ -641,6 +661,8 @@ impl PhysicalPlan {
             | PhysicalPlan::MergeIntoAddRowNumber(_)
             | PhysicalPlan::MergeIntoAppendNotMatched(_)
             | PhysicalPlan::ConstantTableScan(_)
+            | PhysicalPlan::ExpressionScan(_)
+            | PhysicalPlan::CacheScan(_)
             | PhysicalPlan::CteScan(_)
             | PhysicalPlan::ReclusterSource(_)
             | PhysicalPlan::ReclusterSink(_)
diff --git a/src/query/sql/src/executor/physical_plan_builder.rs b/src/query/sql/src/executor/physical_plan_builder.rs
index 9ac117150bcc7..8355e085b1699 100644
--- a/src/query/sql/src/executor/physical_plan_builder.rs
+++ b/src/query/sql/src/executor/physical_plan_builder.rs
@@ -112,6 +112,10 @@ impl PhysicalPlanBuilder {
             RelOperator::ConstantTableScan(scan) => {
                 self.build_constant_table_scan(scan, required).await
             }
+            RelOperator::ExpressionScan(scan) => {
+                self.build_expression_scan(s_expr, scan, required).await
+            }
+            RelOperator::CacheScan(scan) => self.build_cache_scan(scan, required).await,
             RelOperator::AddRowNumber(_) => self.build_add_row_number(s_expr, required).await,
             RelOperator::Udf(udf) => self.build_udf(s_expr, udf, required, stat_info).await,
             RelOperator::AsyncFunction(async_func) => {
diff --git a/src/query/sql/src/executor/physical_plan_display.rs b/src/query/sql/src/executor/physical_plan_display.rs
index a1739c57bd809..0091bd7d15a19 100644
--- a/src/query/sql/src/executor/physical_plan_display.rs
+++ b/src/query/sql/src/executor/physical_plan_display.rs
@@ -23,6 +23,7 @@ use crate::executor::physical_plan::PhysicalPlan;
 use crate::executor::physical_plans::AggregateExpand;
 use crate::executor::physical_plans::AggregateFinal;
 use crate::executor::physical_plans::AggregatePartial;
+use crate::executor::physical_plans::CacheScan;
 use crate::executor::physical_plans::CommitSink;
 use crate::executor::physical_plans::CompactSource;
 use crate::executor::physical_plans::ConstantTableScan;
@@ -35,6 +36,7 @@ use crate::executor::physical_plans::EvalScalar;
 use crate::executor::physical_plans::Exchange;
 use crate::executor::physical_plans::ExchangeSink;
 use crate::executor::physical_plans::ExchangeSource;
+use crate::executor::physical_plans::ExpressionScan;
 use crate::executor::physical_plans::Filter;
 use crate::executor::physical_plans::HashJoin;
 use crate::executor::physical_plans::Limit;
@@ -56,6 +58,7 @@ use crate::executor::physical_plans::Udf;
 use crate::executor::physical_plans::UnionAll;
 use crate::executor::physical_plans::UpdateSource;
 use crate::executor::physical_plans::Window;
+use crate::plans::CacheSource;
 use crate::plans::JoinType;
 
 impl PhysicalPlan {
@@ -110,6 +113,8 @@ impl<'a> Display for PhysicalPlanIndentFormatDisplay<'a> {
             PhysicalPlan::CteScan(cte_scan) => write!(f, "{}", cte_scan)?,
             PhysicalPlan::MaterializedCte(plan) => write!(f, "{}", plan)?,
             PhysicalPlan::ConstantTableScan(scan) => write!(f, "{}", scan)?,
+            PhysicalPlan::ExpressionScan(scan) => write!(f, "{}", scan)?,
+            PhysicalPlan::CacheScan(scan) => write!(f, "{}", scan)?,
             PhysicalPlan::ReclusterSource(plan) => write!(f, "{}", plan)?,
             PhysicalPlan::ReclusterSink(plan) => write!(f, "{}", plan)?,
             PhysicalPlan::UpdateSource(plan) => write!(f, "{}", plan)?,
@@ -169,6 +174,39 @@ impl Display for ConstantTableScan {
     }
 }
 
+impl Display for ExpressionScan {
+    fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
+        let columns = self
+            .values
+            .iter()
+            .enumerate()
+            .map(|(i, value)| {
+                let column = value
+                    .iter()
+                    .map(|val| val.as_expr(&BUILTIN_FUNCTIONS).sql_display())
+                    .join(", ");
+                format!("column {}: [{}]", i, column)
+            })
+            .collect::<Vec<String>>();
+
+        write!(f, "ExpressionScan: {}", columns.join(", "))
+    }
+}
+
+impl Display for CacheScan {
+    fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
+        match &self.cache_source {
+            CacheSource::HashJoinBuild((cache_index, column_indexes)) => {
+                write!(
+                    f,
+                    "CacheScan: [cache_index: {}, column_indexes: {:?}]",
+                    cache_index, column_indexes
+                )
+            }
+        }
+    }
+}
+
 impl Display for Filter {
     fn fmt(&self, f: &mut Formatter) -> std::fmt::Result {
         let predicates = self
diff --git a/src/query/sql/src/executor/physical_plan_visitor.rs b/src/query/sql/src/executor/physical_plan_visitor.rs
index b8385e5bc64f2..3cc6c35d37d69 100644
--- a/src/query/sql/src/executor/physical_plan_visitor.rs
+++ b/src/query/sql/src/executor/physical_plan_visitor.rs
@@ -14,6 +14,8 @@
 
 use databend_common_exception::Result;
 
+use super::physical_plans::CacheScan;
+use super::physical_plans::ExpressionScan;
 use crate::executor::physical_plan::PhysicalPlan;
 use crate::executor::physical_plans::AggregateExpand;
 use crate::executor::physical_plans::AggregateFinal;
@@ -99,6 +101,8 @@ pub trait PhysicalPlanReplacer {
             }
             PhysicalPlan::MaterializedCte(plan) => self.replace_materialized_cte(plan),
             PhysicalPlan::ConstantTableScan(plan) => self.replace_constant_table_scan(plan),
+            PhysicalPlan::ExpressionScan(plan) => self.replace_expression_scan(plan),
+            PhysicalPlan::CacheScan(plan) => self.replace_cache_scan(plan),
             PhysicalPlan::ReclusterSource(plan) => self.replace_recluster_source(plan),
             PhysicalPlan::ReclusterSink(plan) => self.replace_recluster_sink(plan),
             PhysicalPlan::UpdateSource(plan) => self.replace_update_source(plan),
@@ -140,6 +144,14 @@ pub trait PhysicalPlanReplacer {
         Ok(PhysicalPlan::ConstantTableScan(plan.clone()))
     }
 
+    fn replace_expression_scan(&mut self, plan: &ExpressionScan) -> Result<PhysicalPlan> {
+        Ok(PhysicalPlan::ExpressionScan(plan.clone()))
+    }
+
+    fn replace_cache_scan(&mut self, plan: &CacheScan) -> Result<PhysicalPlan> {
+        Ok(PhysicalPlan::CacheScan(plan.clone()))
+    }
+
     fn replace_filter(&mut self, plan: &Filter) -> Result<PhysicalPlan> {
         let input = self.replace(&plan.input)?;
 
@@ -245,6 +257,7 @@ pub trait PhysicalPlanReplacer {
             enable_bloom_runtime_filter: plan.enable_bloom_runtime_filter,
             broadcast: plan.broadcast,
             single_to_inner: plan.single_to_inner.clone(),
+            build_side_cache_info: plan.build_side_cache_info.clone(),
         }))
     }
 
@@ -601,6 +614,8 @@ impl PhysicalPlan {
                 | PhysicalPlan::ReplaceAsyncSourcer(_)
                 | PhysicalPlan::CteScan(_)
                 | PhysicalPlan::ConstantTableScan(_)
+                | PhysicalPlan::ExpressionScan(_)
+                | PhysicalPlan::CacheScan(_)
                 | PhysicalPlan::ReclusterSource(_)
                 | PhysicalPlan::ExchangeSource(_)
                 | PhysicalPlan::CompactSource(_)
diff --git a/src/query/sql/src/executor/physical_plans/mod.rs b/src/query/sql/src/executor/physical_plans/mod.rs
index 06bc00177dd53..34dc09d5c5acb 100644
--- a/src/query/sql/src/executor/physical_plans/mod.rs
+++ b/src/query/sql/src/executor/physical_plans/mod.rs
@@ -85,6 +85,12 @@ pub use physical_table_scan::TableScan;
 mod physical_async_func;
 pub use physical_async_func::AsyncFunction;
 
+mod physical_expression_scan;
+pub use physical_expression_scan::ExpressionScan;
+
+mod physical_cache_scan;
+pub use physical_cache_scan::CacheScan;
+
 mod physical_union_all;
 pub use physical_union_all::UnionAll;
 mod physical_window;
diff --git a/src/query/sql/src/executor/physical_plans/physical_cache_scan.rs b/src/query/sql/src/executor/physical_plans/physical_cache_scan.rs
new file mode 100644
index 0000000000000..98575dca5b3d5
--- /dev/null
+++ b/src/query/sql/src/executor/physical_plans/physical_cache_scan.rs
@@ -0,0 +1,62 @@
+// Copyright 2021 Datafuse Labs
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+use databend_common_exception::Result;
+use databend_common_expression::DataSchemaRef;
+use databend_common_expression::DataSchemaRefExt;
+
+use crate::executor::PhysicalPlan;
+use crate::executor::PhysicalPlanBuilder;
+use crate::plans::CacheSource;
+use crate::ColumnSet;
+
+#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)]
+pub struct CacheScan {
+    // A unique id of operator in a `PhysicalPlan` tree, only used for display.
+    pub plan_id: u32,
+    pub cache_source: CacheSource,
+    pub output_schema: DataSchemaRef,
+}
+
+impl CacheScan {
+    pub fn output_schema(&self) -> Result<DataSchemaRef> {
+        Ok(self.output_schema.clone())
+    }
+}
+
+impl PhysicalPlanBuilder {
+    pub(crate) async fn build_cache_scan(
+        &mut self,
+        scan: &crate::plans::CacheScan,
+        required: ColumnSet,
+    ) -> Result<PhysicalPlan> {
+        // 1. Prune unused Columns.
+        let used: ColumnSet = required.intersection(&scan.columns).cloned().collect();
+        let (cache_source, fields) = if used == scan.columns {
+            (scan.cache_source.clone(), scan.schema.fields().clone())
+        } else {
+            let new_scan = scan.prune_columns(used);
+            (
+                new_scan.cache_source.clone(),
+                new_scan.schema.fields().clone(),
+            )
+        };
+        // 2. Build physical plan.
+        Ok(PhysicalPlan::CacheScan(CacheScan {
+            plan_id: 0,
+            cache_source,
+            output_schema: DataSchemaRefExt::create(fields),
+        }))
+    }
+}
diff --git a/src/query/sql/src/executor/physical_plans/physical_expression_scan.rs b/src/query/sql/src/executor/physical_plans/physical_expression_scan.rs
new file mode 100644
index 0000000000000..f0b0d2f29bf68
--- /dev/null
+++ b/src/query/sql/src/executor/physical_plans/physical_expression_scan.rs
@@ -0,0 +1,78 @@
+// Copyright 2021 Datafuse Labs
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+use databend_common_exception::Result;
+use databend_common_expression::ConstantFolder;
+use databend_common_expression::DataSchemaRef;
+use databend_common_expression::RemoteExpr;
+use databend_common_functions::BUILTIN_FUNCTIONS;
+
+use crate::executor::PhysicalPlan;
+use crate::executor::PhysicalPlanBuilder;
+use crate::optimizer::SExpr;
+use crate::ColumnSet;
+use crate::TypeCheck;
+
+#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)]
+pub struct ExpressionScan {
+    // A unique id of operator in a `PhysicalPlan` tree, only used for display.
+    pub plan_id: u32,
+    pub values: Vec<Vec<RemoteExpr>>,
+    pub input: Box<PhysicalPlan>,
+    pub output_schema: DataSchemaRef,
+}
+
+impl ExpressionScan {
+    pub fn output_schema(&self) -> Result<DataSchemaRef> {
+        Ok(self.output_schema.clone())
+    }
+}
+
+impl PhysicalPlanBuilder {
+    pub(crate) async fn build_expression_scan(
+        &mut self,
+        s_expr: &SExpr,
+        scan: &crate::plans::ExpressionScan,
+        required: ColumnSet,
+    ) -> Result<PhysicalPlan> {
+        let input = self.build(s_expr.child(0)?, required).await?;
+        let input_schema = input.output_schema()?;
+
+        let values = scan
+            .values
+            .iter()
+            .map(|row| {
+                row.iter()
+                    .map(|scalar| {
+                        let expr = scalar
+                            .type_check(input_schema.as_ref())?
+                            .project_column_ref(|index| {
+                                input_schema.index_of(&index.to_string()).unwrap()
+                            });
+                        let (expr, _) =
+                            ConstantFolder::fold(&expr, &self.func_ctx, &BUILTIN_FUNCTIONS);
+                        Ok(expr.as_remote_expr())
+                    })
+                    .collect::<Result<Vec<_>>>()
+            })
+            .collect::<Result<Vec<_>>>()?;
+
+        Ok(PhysicalPlan::ExpressionScan(ExpressionScan {
+            plan_id: 0,
+            values,
+            input: Box::new(input),
+            output_schema: scan.schema.clone(),
+        }))
+    }
+}
diff --git a/src/query/sql/src/executor/physical_plans/physical_hash_join.rs b/src/query/sql/src/executor/physical_plans/physical_hash_join.rs
index f38a1bff8b14a..a1bfd80d818e1 100644
--- a/src/query/sql/src/executor/physical_plans/physical_hash_join.rs
+++ b/src/query/sql/src/executor/physical_plans/physical_hash_join.rs
@@ -12,6 +12,8 @@
 // See the License for the specific language governing permissions and
 // limitations under the License.
 
+use std::collections::HashMap;
+use std::collections::HashSet;
 use std::sync::Arc;
 
 use databend_common_catalog::table_context::TableContext;
@@ -85,6 +87,10 @@ pub struct HashJoin {
     // When left/right single join converted to inner join, record the original join type
     // and do some special processing during runtime.
     pub single_to_inner: Option<JoinType>,
+
+    // Hash join build side cache information for ExpressionScan, which includes the cache index and
+    // a HashMap for mapping the column indexes to the BlockEntry indexes in DataBlock.
+    pub build_side_cache_info: Option<(usize, HashMap<IndexType, usize>)>,
 }
 
 impl HashJoin {
@@ -98,13 +104,14 @@ impl PhysicalPlanBuilder {
         &mut self,
         join: &Join,
         s_expr: &SExpr,
-        required: (ColumnSet, ColumnSet),
+        left_required: ColumnSet,
+        right_required: ColumnSet,
         mut pre_column_projections: Vec<IndexType>,
         column_projections: Vec<IndexType>,
         stat_info: PlanStatsInfo,
     ) -> Result<PhysicalPlan> {
-        let mut probe_side = Box::new(self.build(s_expr.child(0)?, required.0).await?);
-        let mut build_side = Box::new(self.build(s_expr.child(1)?, required.1).await?);
+        let mut probe_side = Box::new(self.build(s_expr.child(0)?, left_required).await?);
+        let mut build_side = Box::new(self.build(s_expr.child(1)?, right_required).await?);
 
         let mut is_broadcast = false;
         // Check if join is broadcast join
@@ -326,6 +333,13 @@ impl PhysicalPlanBuilder {
                 .push(left_expr_for_runtime_filter.map(|(expr, idx)| (expr.as_remote_expr(), idx)));
         }
 
+        let mut cache_column_map = HashMap::new();
+        let cached_column = if let Some(cache_info) = &join.build_side_cache_info {
+            cache_info.columns.clone().into_iter().collect()
+        } else {
+            HashSet::new()
+        };
+        pre_column_projections.extend(cached_column.iter());
         let mut probe_projections = ColumnSet::new();
         let mut build_projections = ColumnSet::new();
         for column in pre_column_projections.iter() {
@@ -333,10 +347,20 @@ impl PhysicalPlanBuilder {
                 probe_projections.insert(index);
             }
             if let Some((index, _)) = build_schema.column_with_name(&column.to_string()) {
+                if cached_column.contains(column) {
+                    cache_column_map.insert(*column, index);
+                }
                 build_projections.insert(index);
             }
         }
 
+        let build_side_cache_info = if let Some(cache_info) = &join.build_side_cache_info {
+            probe_to_build_index.clear();
+            Some((cache_info.cache_idx, cache_column_map))
+        } else {
+            None
+        };
+
         // for distributed merge into, there is a field called "_row_number", but
         // it's not an internal row_number, we need to add it here
         if let Some((index, _)) = build_schema.column_with_name(ROW_NUMBER_COL_NAME) {
@@ -518,6 +542,7 @@ impl PhysicalPlanBuilder {
                 s_expr,
             )
             .await?,
+            build_side_cache_info,
         }))
     }
 }
diff --git a/src/query/sql/src/executor/physical_plans/physical_join.rs b/src/query/sql/src/executor/physical_plans/physical_join.rs
index 51b52a7d329c8..07b2002db3f0b 100644
--- a/src/query/sql/src/executor/physical_plans/physical_join.rs
+++ b/src/query/sql/src/executor/physical_plans/physical_join.rs
@@ -39,6 +39,11 @@ pub fn physical_join(join: &Join, s_expr: &SExpr) -> Result<PhysicalJoinType> {
         return Ok(PhysicalJoinType::Hash);
     }
 
+    if join.build_side_cache_info.is_some() {
+        // There is a build side cache, use hash join.
+        return Ok(PhysicalJoinType::Hash);
+    }
+
     let left_prop = RelExpr::with_s_expr(s_expr.child(1)?).derive_relational_prop()?;
     let right_prop = RelExpr::with_s_expr(s_expr.child(0)?).derive_relational_prop()?;
     let mut range_conditions = vec![];
@@ -108,12 +113,17 @@ impl PhysicalPlanBuilder {
     ) -> Result<PhysicalPlan> {
         // 1. Prune unused Columns.
         let column_projections = required.clone().into_iter().collect::<Vec<_>>();
-        let others_required = join
+        let mut others_required = join
             .non_equi_conditions
             .iter()
             .fold(required.clone(), |acc, v| {
                 acc.union(&v.used_columns()).cloned().collect()
             });
+        if let Some(cache_info) = &join.build_side_cache_info {
+            for column in &cache_info.columns {
+                others_required.insert(*column);
+            }
+        }
         let pre_column_projections = others_required.clone().into_iter().collect::<Vec<_>>();
         // Include columns referenced in left conditions and right conditions.
         let left_required = join
@@ -143,7 +153,8 @@ impl PhysicalPlanBuilder {
                 self.build_hash_join(
                     join,
                     s_expr,
-                    (left_required, right_required),
+                    left_required,
+                    right_required,
                     pre_column_projections,
                     column_projections,
                     stat_info,
diff --git a/src/query/sql/src/planner/binder/bind_query/bind.rs b/src/query/sql/src/planner/binder/bind_query/bind.rs
new file mode 100644
index 0000000000000..62cb67853f06f
--- /dev/null
+++ b/src/query/sql/src/planner/binder/bind_query/bind.rs
@@ -0,0 +1,170 @@
+// Copyright 2021 Datafuse Labs
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+use std::sync::Arc;
+
+use databend_common_ast::ast::Expr;
+use databend_common_ast::ast::Query;
+use databend_common_ast::ast::SetExpr;
+use databend_common_ast::ast::With;
+use databend_common_exception::ErrorCode;
+use databend_common_exception::Result;
+
+use crate::binder::CteInfo;
+use crate::optimizer::SExpr;
+use crate::planner::binder::scalar::ScalarBinder;
+use crate::planner::binder::BindContext;
+use crate::planner::binder::Binder;
+use crate::plans::BoundColumnRef;
+use crate::plans::ScalarExpr;
+use crate::plans::Sort;
+use crate::plans::SortItem;
+
+impl Binder {
+    #[async_backtrace::framed]
+    pub(crate) async fn bind_query(
+        &mut self,
+        bind_context: &mut BindContext,
+        query: &Query,
+    ) -> Result<(SExpr, BindContext)> {
+        // Initialize cte map.
+        self.init_cte(bind_context, &query.with)?;
+
+        // Extract limit and offset from query.
+        let (limit, offset) = self.extract_limit_and_offset(query)?;
+
+        // Bind query body.
+        let (mut s_expr, mut bind_context) = Box::pin(self.bind_set_expr(
+            bind_context,
+            &query.body,
+            &query.order_by,
+            limit.unwrap_or_default(),
+        ))
+        .await?;
+
+        // Bind order by for `SetOperation` and `Values`.
+        s_expr = self
+            .bind_query_order_by(&mut bind_context, query, s_expr)
+            .await?;
+
+        // Bind limit.
+        s_expr = self.bind_query_limit(query, s_expr, limit, offset);
+
+        Ok((s_expr, bind_context))
+    }
+
+    // Initialize cte map.
+    pub(crate) fn init_cte(
+        &mut self,
+        bind_context: &mut BindContext,
+        with: &Option<With>,
+    ) -> Result<()> {
+        let with = if let Some(with) = with {
+            with
+        } else {
+            return Ok(());
+        };
+
+        for (idx, cte) in with.ctes.iter().enumerate() {
+            let table_name = self.normalize_identifier(&cte.alias.name).name;
+            if bind_context.cte_map_ref.contains_key(&table_name) {
+                return Err(ErrorCode::SemanticError(format!(
+                    "Duplicate common table expression: {table_name}"
+                )));
+            }
+            let column_name = cte
+                .alias
+                .columns
+                .iter()
+                .map(|ident| self.normalize_identifier(ident).name)
+                .collect();
+            let cte_info = CteInfo {
+                columns_alias: column_name,
+                query: *cte.query.clone(),
+                materialized: cte.materialized,
+                cte_idx: idx,
+                used_count: 0,
+                columns: vec![],
+            };
+            self.ctes_map.insert(table_name.clone(), cte_info.clone());
+            bind_context.cte_map_ref.insert(table_name, cte_info);
+        }
+
+        Ok(())
+    }
+
+    #[async_backtrace::framed]
+    pub(crate) async fn bind_query_order_by(
+        &mut self,
+        bind_context: &mut BindContext,
+        query: &Query,
+        child: SExpr,
+    ) -> Result<SExpr> {
+        if !matches!(
+            query.body,
+            SetExpr::SetOperation(_) | SetExpr::Values { .. }
+        ) || query.order_by.is_empty()
+        {
+            return Ok(child);
+        }
+
+        let mut scalar_binder = ScalarBinder::new(
+            bind_context,
+            self.ctx.clone(),
+            &self.name_resolution_ctx,
+            self.metadata.clone(),
+            &[],
+            self.m_cte_bound_ctx.clone(),
+            self.ctes_map.clone(),
+        );
+        let mut order_by_items = Vec::with_capacity(query.order_by.len());
+        for order in query.order_by.iter() {
+            match order.expr {
+                Expr::ColumnRef { .. } => {
+                    let scalar = scalar_binder.bind(&order.expr).await?.0;
+                    match scalar {
+                        ScalarExpr::BoundColumnRef(BoundColumnRef { column, .. }) => {
+                            let order_by_item = SortItem {
+                                index: column.index,
+                                asc: order.asc.unwrap_or(true),
+                                nulls_first: order.nulls_first.unwrap_or(false),
+                            };
+                            order_by_items.push(order_by_item);
+                        }
+                        _ => {
+                            return Err(ErrorCode::Internal("scalar should be BoundColumnRef")
+                                .set_span(order.expr.span()));
+                        }
+                    }
+                }
+                _ => {
+                    return Err(
+                        ErrorCode::SemanticError("can only order by column".to_string())
+                            .set_span(order.expr.span()),
+                    );
+                }
+            }
+        }
+        let sort_plan = Sort {
+            items: order_by_items,
+            limit: None,
+            after_exchange: None,
+            pre_projection: None,
+        };
+        Ok(SExpr::create_unary(
+            Arc::new(sort_plan.into()),
+            Arc::new(child),
+        ))
+    }
+}
diff --git a/src/query/sql/src/planner/binder/limit.rs b/src/query/sql/src/planner/binder/bind_query/bind_limit.rs
similarity index 62%
rename from src/query/sql/src/planner/binder/limit.rs
rename to src/query/sql/src/planner/binder/bind_query/bind_limit.rs
index a096afad66c3b..8670bd97d4735 100644
--- a/src/query/sql/src/planner/binder/limit.rs
+++ b/src/query/sql/src/planner/binder/bind_query/bind_limit.rs
@@ -16,6 +16,8 @@ use std::sync::Arc;
 
 use databend_common_ast::ast::Expr;
 use databend_common_ast::ast::Literal;
+use databend_common_ast::ast::Query;
+use databend_common_ast::ast::SetExpr;
 use databend_common_exception::ErrorCode;
 use databend_common_exception::Result;
 
@@ -24,6 +26,51 @@ use crate::optimizer::SExpr;
 use crate::plans::Limit;
 
 impl Binder {
+    pub(super) fn bind_query_limit(
+        &self,
+        query: &Query,
+        s_expr: SExpr,
+        limit: Option<usize>,
+        offset: usize,
+    ) -> SExpr {
+        if limit.is_none() && query.offset.is_none() {
+            return s_expr;
+        }
+
+        let limit_plan = Limit {
+            before_exchange: false,
+            limit,
+            offset,
+        };
+        SExpr::create_unary(Arc::new(limit_plan.into()), Arc::new(s_expr))
+    }
+
+    pub(crate) fn extract_limit_and_offset(&self, query: &Query) -> Result<(Option<usize>, usize)> {
+        let (mut limit, offset) = if !query.limit.is_empty() {
+            if query.limit.len() == 1 {
+                Self::analyze_limit(Some(&query.limit[0]), &query.offset)?
+            } else {
+                Self::analyze_limit(Some(&query.limit[1]), &Some(query.limit[0].clone()))?
+            }
+        } else if query.offset.is_some() {
+            Self::analyze_limit(None, &query.offset)?
+        } else {
+            (None, 0)
+        };
+
+        if let SetExpr::Select(stmt) = &query.body {
+            if !query.limit.is_empty() && stmt.top_n.is_some() {
+                return Err(ErrorCode::SemanticError(
+                    "Duplicate LIMIT: TopN and Limit cannot be used together",
+                ));
+            } else if let Some(n) = stmt.top_n {
+                limit = Some(n as usize);
+            }
+        }
+
+        Ok((limit, offset))
+    }
+
     pub(super) fn analyze_limit(
         limit: Option<&Expr>,
         offset: &Option<Expr>,
@@ -48,15 +95,6 @@ impl Binder {
         Ok((limit_cnt, offset_cnt))
     }
 
-    pub(super) fn bind_limit(child: SExpr, limit: Option<usize>, offset: usize) -> SExpr {
-        let limit_plan = Limit {
-            before_exchange: false,
-            limit,
-            offset,
-        };
-        SExpr::create_unary(Arc::new(limit_plan.into()), Arc::new(child))
-    }
-
     /// So far, we only support integer literal as limit argument.
     /// So we will try to extract the integer value from the AST directly.
     /// In the future it's possible to treat the argument as an expression.
diff --git a/src/query/sql/src/planner/binder/bind_query/bind_select.rs b/src/query/sql/src/planner/binder/bind_query/bind_select.rs
new file mode 100644
index 0000000000000..ff223627dcfd2
--- /dev/null
+++ b/src/query/sql/src/planner/binder/bind_query/bind_select.rs
@@ -0,0 +1,559 @@
+// Copyright 2021 Datafuse Labs
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+use databend_common_ast::ast::BinaryOperator;
+use databend_common_ast::ast::ColumnID;
+use databend_common_ast::ast::ColumnPosition;
+use databend_common_ast::ast::ColumnRef;
+use databend_common_ast::ast::Expr;
+use databend_common_ast::ast::Expr::Array;
+use databend_common_ast::ast::FunctionCall;
+use databend_common_ast::ast::GroupBy;
+use databend_common_ast::ast::Identifier;
+use databend_common_ast::ast::Join;
+use databend_common_ast::ast::JoinCondition;
+use databend_common_ast::ast::JoinOperator;
+use databend_common_ast::ast::Literal;
+use databend_common_ast::ast::OrderByExpr;
+use databend_common_ast::ast::SelectStmt;
+use databend_common_ast::ast::SelectTarget;
+use databend_common_ast::ast::TableReference;
+use databend_common_ast::Span;
+use databend_common_exception::ErrorCode;
+use databend_common_exception::Result;
+use databend_common_license::license::Feature;
+use databend_common_license::license_manager::get_license_manager;
+use derive_visitor::Drive;
+use derive_visitor::Visitor;
+use log::warn;
+
+use crate::binder::project_set::SrfCollector;
+use crate::optimizer::SExpr;
+use crate::planner::binder::BindContext;
+use crate::planner::binder::Binder;
+use crate::plans::ScalarExpr;
+use crate::AsyncFunctionRewriter;
+use crate::ColumnBinding;
+use crate::UdfRewriter;
+use crate::VirtualColumnRewriter;
+
+// A normalized IR for `SELECT` clause.
+#[derive(Debug, Default)]
+pub struct SelectList<'a> {
+    pub items: Vec<SelectItem<'a>>,
+}
+
+#[derive(Debug)]
+pub struct SelectItem<'a> {
+    pub select_target: &'a SelectTarget,
+    pub scalar: ScalarExpr,
+    pub alias: String,
+}
+
+impl Binder {
+    #[async_backtrace::framed]
+    pub(crate) async fn bind_select(
+        &mut self,
+        bind_context: &mut BindContext,
+        stmt: &SelectStmt,
+        order_by: &[OrderByExpr],
+        limit: usize,
+    ) -> Result<(SExpr, BindContext)> {
+        if let Some(hints) = &stmt.hints {
+            if let Some(e) = self.opt_hints_set_var(bind_context, hints).await.err() {
+                warn!(
+                    "In SELECT resolve optimize hints {:?} failed, err: {:?}",
+                    hints, e
+                );
+            }
+        }
+        let (mut s_expr, mut from_context) = if stmt.from.is_empty() {
+            let select_list = &stmt.select_list;
+            self.bind_dummy_table(bind_context, select_list).await?
+        } else {
+            let mut max_column_position = MaxColumnPosition::new();
+            stmt.drive(&mut max_column_position);
+            self.metadata
+                .write()
+                .set_max_column_position(max_column_position.max_pos);
+
+            let cross_joins = stmt
+                .from
+                .iter()
+                .cloned()
+                .reduce(|left, right| TableReference::Join {
+                    span: None,
+                    join: Join {
+                        op: JoinOperator::CrossJoin,
+                        condition: JoinCondition::None,
+                        left: Box::new(left),
+                        right: Box::new(right),
+                    },
+                })
+                .unwrap();
+            self.bind_table_reference(bind_context, &cross_joins)
+                .await?
+        };
+
+        let mut rewriter = SelectRewriter::new(
+            from_context.all_column_bindings(),
+            self.name_resolution_ctx.unquoted_ident_case_sensitive,
+        );
+        let new_stmt = rewriter.rewrite(stmt)?;
+        let stmt = new_stmt.as_ref().unwrap_or(stmt);
+
+        // Collect set returning functions
+        let set_returning_functions = {
+            let mut collector = SrfCollector::new();
+            stmt.select_list.iter().for_each(|item| {
+                if let SelectTarget::AliasedExpr { expr, .. } = item {
+                    collector.visit(expr);
+                }
+            });
+            collector.into_srfs()
+        };
+
+        // Bind set returning functions
+        s_expr = self
+            .bind_project_set(&mut from_context, &set_returning_functions, s_expr)
+            .await?;
+
+        // Try put window definitions into bind context.
+        // This operation should be before `normalize_select_list` because window functions can be used in select list.
+        self.analyze_window_definition(&mut from_context, &stmt.window_list)?;
+
+        // Generate a analyzed select list with from context
+        let mut select_list = self
+            .normalize_select_list(&mut from_context, &stmt.select_list)
+            .await?;
+
+        // This will potentially add some alias group items to `from_context` if find some.
+        if let Some(group_by) = stmt.group_by.as_ref() {
+            self.analyze_group_items(&mut from_context, &select_list, group_by)
+                .await?;
+        }
+
+        self.analyze_aggregate_select(&mut from_context, &mut select_list)?;
+
+        // `analyze_window` should behind `analyze_aggregate_select`,
+        // because `analyze_window` will rewrite the aggregate functions in the window function's arguments.
+        self.analyze_window(&mut from_context, &mut select_list)?;
+
+        let aliases = select_list
+            .items
+            .iter()
+            .map(|item| (item.alias.clone(), item.scalar.clone()))
+            .collect::<Vec<_>>();
+
+        // To support using aliased column in `WHERE` clause,
+        // we should bind where after `select_list` is rewritten.
+        let where_scalar = if let Some(expr) = &stmt.selection {
+            let (new_expr, scalar) = self
+                .bind_where(&mut from_context, &aliases, expr, s_expr)
+                .await?;
+            s_expr = new_expr;
+            Some(scalar)
+        } else {
+            None
+        };
+
+        // `analyze_projection` should behind `analyze_aggregate_select` because `analyze_aggregate_select` will rewrite `grouping`.
+        let (mut scalar_items, projections) = self.analyze_projection(
+            &from_context.aggregate_info,
+            &from_context.windows,
+            &select_list,
+        )?;
+
+        let having = if let Some(having) = &stmt.having {
+            Some(
+                self.analyze_aggregate_having(&mut from_context, &aliases, having)
+                    .await?,
+            )
+        } else {
+            None
+        };
+
+        let qualify = if let Some(qualify) = &stmt.qualify {
+            Some(
+                self.analyze_window_qualify(&mut from_context, &aliases, qualify)
+                    .await?,
+            )
+        } else {
+            None
+        };
+
+        let order_items = self
+            .analyze_order_items(
+                &mut from_context,
+                &mut scalar_items,
+                &aliases,
+                &projections,
+                order_by,
+                stmt.distinct,
+            )
+            .await?;
+
+        // After all analysis is done.
+        if set_returning_functions.is_empty() {
+            // Ignore SRFs.
+            self.analyze_lazy_materialization(
+                &from_context,
+                stmt,
+                &scalar_items,
+                &select_list,
+                &where_scalar,
+                &order_items.items,
+                limit,
+            )?;
+        }
+
+        if !from_context.aggregate_info.aggregate_functions.is_empty()
+            || !from_context.aggregate_info.group_items.is_empty()
+        {
+            s_expr = self.bind_aggregate(&mut from_context, s_expr).await?;
+        }
+
+        if let Some(having) = having {
+            s_expr = self.bind_having(&mut from_context, having, s_expr).await?;
+        }
+
+        // bind window
+        // window run after the HAVING clause but before the ORDER BY clause.
+        for window_info in &from_context.windows.window_functions {
+            s_expr = self.bind_window_function(window_info, s_expr).await?;
+        }
+
+        if let Some(qualify) = qualify {
+            s_expr = self
+                .bind_qualify(&mut from_context, qualify, s_expr)
+                .await?;
+        }
+
+        if stmt.distinct {
+            s_expr = self.bind_distinct(
+                stmt.span,
+                &from_context,
+                &projections,
+                &mut scalar_items,
+                s_expr,
+            )?;
+        }
+
+        if !order_by.is_empty() {
+            s_expr = self
+                .bind_order_by(
+                    &from_context,
+                    order_items,
+                    &select_list,
+                    &mut scalar_items,
+                    s_expr,
+                )
+                .await?;
+        }
+
+        s_expr = self.bind_projection(&mut from_context, &projections, &scalar_items, s_expr)?;
+
+        // rewrite async function to async function plan
+        let mut async_func_rewriter = AsyncFunctionRewriter::new();
+        s_expr = async_func_rewriter.rewrite(&s_expr)?;
+
+        // rewrite udf for interpreter udf
+        let mut udf_rewriter = UdfRewriter::new(self.metadata.clone(), true);
+        s_expr = udf_rewriter.rewrite(&s_expr)?;
+
+        // rewrite udf for server udf
+        let mut udf_rewriter = UdfRewriter::new(self.metadata.clone(), false);
+        s_expr = udf_rewriter.rewrite(&s_expr)?;
+
+        // rewrite variant inner fields as virtual columns
+        let mut virtual_column_rewriter =
+            VirtualColumnRewriter::new(self.ctx.clone(), self.metadata.clone());
+        s_expr = virtual_column_rewriter.rewrite(&s_expr).await?;
+
+        // check inverted index license
+        if !from_context.inverted_index_map.is_empty() {
+            let license_manager = get_license_manager();
+            license_manager
+                .manager
+                .check_enterprise_enabled(self.ctx.get_license_key(), Feature::InvertedIndex)?;
+        }
+        // add internal column binding into expr
+        s_expr = from_context.add_internal_column_into_expr(s_expr)?;
+
+        let mut output_context = BindContext::new();
+        output_context.parent = from_context.parent;
+        output_context.columns = from_context.columns;
+
+        Ok((s_expr, output_context))
+    }
+}
+
+/// It is useful when implementing some SQL syntax sugar,
+///
+/// [`column_binding`] contains the column binding information of the SelectStmt.
+///
+/// to rewrite the SelectStmt, just add a new rewrite_* function and call it in the `rewrite` function.
+struct SelectRewriter<'a> {
+    column_binding: &'a [ColumnBinding],
+    new_stmt: Option<SelectStmt>,
+    is_unquoted_ident_case_sensitive: bool,
+}
+
+// helper functions to SelectRewriter
+impl<'a> SelectRewriter<'a> {
+    fn compare_unquoted_ident(&self, a: &str, b: &str) -> bool {
+        if self.is_unquoted_ident_case_sensitive {
+            a == b
+        } else {
+            a.eq_ignore_ascii_case(b)
+        }
+    }
+
+    fn parse_aggregate_function(expr: &Expr) -> Result<(&Identifier, &[Expr])> {
+        match expr {
+            Expr::FunctionCall {
+                func: FunctionCall { name, args, .. },
+                ..
+            } => Ok((name, args)),
+            _ => Err(ErrorCode::SyntaxException("Aggregate function is required")),
+        }
+    }
+
+    fn expr_eq_from_col_and_value(col: Identifier, value: Expr) -> Expr {
+        Expr::BinaryOp {
+            span: None,
+            left: Box::new(Expr::ColumnRef {
+                span: None,
+                column: ColumnRef {
+                    database: None,
+                    table: None,
+                    column: ColumnID::Name(col),
+                },
+            }),
+            op: BinaryOperator::Eq,
+            right: Box::new(value),
+        }
+    }
+
+    fn target_func_from_name_args(
+        name: Identifier,
+        args: Vec<Expr>,
+        alias: Option<Identifier>,
+    ) -> SelectTarget {
+        SelectTarget::AliasedExpr {
+            expr: Box::new(Expr::FunctionCall {
+                span: Span::default(),
+                func: FunctionCall {
+                    distinct: false,
+                    name,
+                    args,
+                    params: vec![],
+                    window: None,
+                    lambda: None,
+                },
+            }),
+            alias,
+        }
+    }
+
+    fn expr_literal_array_from_vec_ident(exprs: Vec<Identifier>) -> Expr {
+        Array {
+            span: Span::default(),
+            exprs: exprs
+                .into_iter()
+                .map(|expr| Expr::Literal {
+                    span: None,
+                    value: Literal::String(expr.name),
+                })
+                .collect(),
+        }
+    }
+
+    fn expr_column_ref_array_from_vec_ident(exprs: Vec<Identifier>) -> Expr {
+        Array {
+            span: Span::default(),
+            exprs: exprs
+                .into_iter()
+                .map(|expr| Expr::ColumnRef {
+                    span: None,
+                    column: ColumnRef {
+                        database: None,
+                        table: None,
+                        column: ColumnID::Name(expr),
+                    },
+                })
+                .collect(),
+        }
+    }
+
+    // For Expr::Literal, expr.to_string() is quoted, sometimes we need the raw string.
+    fn raw_string_from_literal_expr(expr: &Expr) -> Option<String> {
+        match expr {
+            Expr::Literal { value, .. } => match value {
+                Literal::String(v) => Some(v.clone()),
+                _ => Some(expr.to_string()),
+            },
+            _ => None,
+        }
+    }
+}
+
+impl<'a> SelectRewriter<'a> {
+    fn new(column_binding: &'a [ColumnBinding], is_unquoted_ident_case_sensitive: bool) -> Self {
+        SelectRewriter {
+            column_binding,
+            new_stmt: None,
+            is_unquoted_ident_case_sensitive,
+        }
+    }
+
+    fn rewrite(&mut self, stmt: &SelectStmt) -> Result<Option<SelectStmt>> {
+        self.rewrite_pivot(stmt)?;
+        self.rewrite_unpivot(stmt)?;
+        Ok(self.new_stmt.take())
+    }
+
+    fn rewrite_pivot(&mut self, stmt: &SelectStmt) -> Result<()> {
+        if stmt.from.len() != 1 || stmt.from[0].pivot().is_none() {
+            return Ok(());
+        }
+        let pivot = stmt.from[0].pivot().unwrap();
+        let (aggregate_name, aggregate_args) = Self::parse_aggregate_function(&pivot.aggregate)?;
+        let aggregate_columns = aggregate_args
+            .iter()
+            .map(|expr| match expr {
+                Expr::ColumnRef { column, .. } => Some(column.clone()),
+                _ => None,
+            })
+            .collect::<Option<Vec<_>>>()
+            .ok_or_else(|| ErrorCode::SyntaxException("Aggregate column not found"))?;
+        let aggregate_column_names = aggregate_columns
+            .iter()
+            .map(|col| col.column.name())
+            .collect::<Vec<_>>();
+        let new_group_by = stmt.group_by.clone().unwrap_or_else(|| {
+            GroupBy::Normal(
+                self.column_binding
+                    .iter()
+                    .filter(|col_bind| {
+                        !self
+                            .compare_unquoted_ident(&col_bind.column_name, &pivot.value_column.name)
+                            && !aggregate_column_names
+                                .iter()
+                                .any(|col| self.compare_unquoted_ident(col, &col_bind.column_name))
+                    })
+                    .map(|col| Expr::Literal {
+                        span: Span::default(),
+                        value: Literal::UInt64(col.index as u64 + 1),
+                    })
+                    .collect(),
+            )
+        });
+
+        let mut new_select_list = stmt.select_list.clone();
+        if let Some(star) = new_select_list.iter_mut().find(|target| target.is_star()) {
+            let mut exclude_columns: Vec<_> = aggregate_columns
+                .iter()
+                .map(|c| Identifier::from_name(stmt.span, c.column.name()))
+                .collect();
+            exclude_columns.push(pivot.value_column.clone());
+            star.exclude(exclude_columns);
+        };
+        let new_aggregate_name = Identifier {
+            name: format!("{}_if", aggregate_name.name),
+            ..aggregate_name.clone()
+        };
+        for value in &pivot.values {
+            let mut args = aggregate_args.to_vec();
+            args.push(Self::expr_eq_from_col_and_value(
+                pivot.value_column.clone(),
+                value.clone(),
+            ));
+            let alias = Self::raw_string_from_literal_expr(value)
+                .ok_or_else(|| ErrorCode::SyntaxException("Pivot value should be literal"))?;
+            new_select_list.push(Self::target_func_from_name_args(
+                new_aggregate_name.clone(),
+                args,
+                Some(Identifier::from_name(stmt.span, &alias)),
+            ));
+        }
+
+        if let Some(ref mut new_stmt) = self.new_stmt {
+            new_stmt.select_list = new_select_list;
+            new_stmt.group_by = Some(new_group_by);
+        } else {
+            self.new_stmt = Some(SelectStmt {
+                select_list: new_select_list,
+                group_by: Some(new_group_by),
+                ..stmt.clone()
+            });
+        }
+        Ok(())
+    }
+
+    fn rewrite_unpivot(&mut self, stmt: &SelectStmt) -> Result<()> {
+        if stmt.from.len() != 1 || stmt.from[0].unpivot().is_none() {
+            return Ok(());
+        }
+        let unpivot = stmt.from[0].unpivot().unwrap();
+        let mut new_select_list = stmt.select_list.clone();
+        if let Some(star) = new_select_list.iter_mut().find(|target| target.is_star()) {
+            star.exclude(unpivot.names.clone());
+        };
+        new_select_list.push(Self::target_func_from_name_args(
+            Identifier::from_name(stmt.span, "unnest"),
+            vec![Self::expr_literal_array_from_vec_ident(
+                unpivot.names.clone(),
+            )],
+            Some(unpivot.column_name.clone()),
+        ));
+        new_select_list.push(Self::target_func_from_name_args(
+            Identifier::from_name(stmt.span, "unnest"),
+            vec![Self::expr_column_ref_array_from_vec_ident(
+                unpivot.names.clone(),
+            )],
+            Some(unpivot.value_column.clone()),
+        ));
+
+        if let Some(ref mut new_stmt) = self.new_stmt {
+            new_stmt.select_list = new_select_list;
+        } else {
+            self.new_stmt = Some(SelectStmt {
+                select_list: new_select_list,
+                ..stmt.clone()
+            });
+        };
+        Ok(())
+    }
+}
+
+#[derive(Visitor)]
+#[visitor(ColumnPosition(enter))]
+pub struct MaxColumnPosition {
+    pub max_pos: usize,
+}
+
+impl MaxColumnPosition {
+    pub fn new() -> Self {
+        Self { max_pos: 0 }
+    }
+}
+
+impl MaxColumnPosition {
+    fn enter_column_position(&mut self, pos: &ColumnPosition) {
+        if pos.pos > self.max_pos {
+            self.max_pos = pos.pos;
+        }
+    }
+}
diff --git a/src/query/sql/src/planner/binder/bind_query/bind_set_expr.rs b/src/query/sql/src/planner/binder/bind_query/bind_set_expr.rs
new file mode 100644
index 0000000000000..d5f937a99682b
--- /dev/null
+++ b/src/query/sql/src/planner/binder/bind_query/bind_set_expr.rs
@@ -0,0 +1,52 @@
+// Copyright 2021 Datafuse Labs
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+use databend_common_ast::ast::OrderByExpr;
+use databend_common_ast::ast::SetExpr;
+use databend_common_exception::Result;
+
+use crate::optimizer::SExpr;
+use crate::planner::binder::BindContext;
+use crate::planner::binder::Binder;
+
+impl Binder {
+    #[async_backtrace::framed]
+    pub(crate) async fn bind_set_expr(
+        &mut self,
+        bind_context: &mut BindContext,
+        set_expr: &SetExpr,
+        order_by: &[OrderByExpr],
+        limit: usize,
+    ) -> Result<(SExpr, BindContext)> {
+        match set_expr {
+            SetExpr::Select(stmt) => {
+                Box::pin(self.bind_select(bind_context, stmt, order_by, limit)).await
+            }
+            SetExpr::Query(stmt) => Box::pin(self.bind_query(bind_context, stmt)).await,
+            SetExpr::SetOperation(set_operation) => {
+                Box::pin(self.bind_set_operator(
+                    bind_context,
+                    &set_operation.left,
+                    &set_operation.right,
+                    &set_operation.op,
+                    &set_operation.all,
+                ))
+                .await
+            }
+            SetExpr::Values { span, values } => {
+                Box::pin(self.bind_values(bind_context, *span, values)).await
+            }
+        }
+    }
+}
diff --git a/src/query/sql/src/planner/binder/bind_query/bind_value.rs b/src/query/sql/src/planner/binder/bind_query/bind_value.rs
new file mode 100644
index 0000000000000..eb3d19a15d5d7
--- /dev/null
+++ b/src/query/sql/src/planner/binder/bind_query/bind_value.rs
@@ -0,0 +1,726 @@
+// Copyright 2021 Datafuse Labs
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+use std::collections::HashMap;
+use std::collections::HashSet;
+use std::sync::Arc;
+
+use databend_common_ast::ast::Expr as AExpr;
+use databend_common_ast::Span;
+use databend_common_catalog::table_context::TableContext;
+use databend_common_exception::ErrorCode;
+use databend_common_exception::Result;
+use databend_common_expression::type_check::common_super_type;
+use databend_common_expression::types::DataType;
+use databend_common_expression::ColumnBuilder;
+use databend_common_expression::DataBlock;
+use databend_common_expression::DataField;
+use databend_common_expression::DataSchema;
+use databend_common_expression::DataSchemaRefExt;
+use databend_common_expression::Evaluator;
+use databend_common_functions::BUILTIN_FUNCTIONS;
+use indexmap::IndexMap;
+
+use crate::binder::wrap_cast;
+use crate::optimizer::ColumnSet;
+use crate::optimizer::SExpr;
+use crate::plans::Aggregate;
+use crate::plans::AggregateMode;
+use crate::plans::BoundColumnRef;
+use crate::plans::CacheScan;
+use crate::plans::CacheSource;
+use crate::plans::ConstantTableScan;
+use crate::plans::ExpressionScan;
+use crate::plans::HashJoinBuildCacheInfo;
+use crate::plans::RelOperator;
+use crate::plans::ScalarItem;
+use crate::BindContext;
+use crate::Binder;
+use crate::ColumnBinding;
+use crate::ColumnBindingBuilder;
+use crate::MetadataRef;
+use crate::NameResolutionContext;
+use crate::ScalarBinder;
+use crate::ScalarExpr;
+use crate::Visibility;
+
+// The `ExpressionScanContext` is used to store the information of
+// expression scan and hash join build cache.
+#[derive(Clone)]
+pub struct ExpressionScanContext {
+    // Hash join build cache info.
+    // Cache column bindings in hash join build side.
+    pub cache_columns: Vec<Vec<ColumnBinding>>,
+    // Cache column indexes in hash join build side.
+    pub column_indexes: Vec<Vec<usize>>,
+    // The hash join build cache index which are used in expression scan.
+    pub used_cache_indexes: HashSet<usize>,
+    // The hash join build cache columns which are used in expression scan.
+    pub used_column_indexes: Vec<HashSet<usize>>,
+
+    // Expression scan info.
+    // Derived column indexes for each expression scan.
+    pub derived_indexes: Vec<HashMap<usize, usize>>,
+    // Original column indexes for each expression scan.
+    pub originnal_columns: Vec<HashMap<usize, usize>>,
+}
+
+impl Default for ExpressionScanContext {
+    fn default() -> Self {
+        Self::new()
+    }
+}
+
+impl ExpressionScanContext {
+    pub fn new() -> Self {
+        ExpressionScanContext {
+            cache_columns: vec![],
+            column_indexes: vec![],
+            used_cache_indexes: HashSet::new(),
+            used_column_indexes: vec![],
+            derived_indexes: vec![],
+            originnal_columns: vec![],
+        }
+    }
+
+    // Add expression scan and return expression scan index.
+    pub fn add_expression_scan(&mut self) -> usize {
+        self.derived_indexes.push(HashMap::new());
+        self.originnal_columns.push(HashMap::new());
+        self.derived_indexes.len() - 1
+    }
+
+    pub fn add_expression_scan_column(
+        &mut self,
+        expression_scan_index: usize,
+        original_column_index: usize,
+        derived_column_index: usize,
+    ) {
+        self.derived_indexes[expression_scan_index]
+            .insert(original_column_index, derived_column_index);
+        self.originnal_columns[expression_scan_index]
+            .insert(derived_column_index, original_column_index);
+    }
+
+    pub fn get_derived_column(&self, expression_scan_index: usize, column_index: usize) -> usize {
+        self.derived_indexes[expression_scan_index]
+            .get(&column_index)
+            .cloned()
+            .unwrap()
+    }
+
+    pub fn get_original_column(&self, expression_scan_index: usize, column_index: usize) -> usize {
+        self.originnal_columns[expression_scan_index]
+            .get(&column_index)
+            .cloned()
+            .unwrap()
+    }
+
+    // Add hash join build cache and return cache index.
+    pub fn add_hash_join_build_cache(
+        &mut self,
+        columns: Vec<ColumnBinding>,
+        column_indexes: Vec<usize>,
+    ) -> usize {
+        self.cache_columns.push(columns);
+        self.column_indexes.push(column_indexes);
+        self.used_column_indexes.push(HashSet::new());
+        self.derived_indexes.push(HashMap::new());
+        self.originnal_columns.push(HashMap::new());
+        self.cache_columns.len() - 1
+    }
+
+    pub fn add_used_cache_index(&mut self, cache_index: usize) {
+        self.used_cache_indexes.insert(cache_index);
+    }
+
+    pub fn add_used_cache_column_index(&mut self, cache_index: usize, column_index: usize) {
+        self.used_column_indexes[cache_index].insert(column_index);
+    }
+
+    // Try to find the cache index for the column index.
+    pub fn find_cache_index(&self, column_index: usize) -> Option<usize> {
+        for idx in (0..self.column_indexes.len()).rev() {
+            for index in &self.column_indexes[idx] {
+                if *index == column_index {
+                    return Some(idx);
+                }
+            }
+        }
+        None
+    }
+
+    pub fn get_cache_columns(&self, cache_idx: usize) -> &Vec<ColumnBinding> {
+        &self.cache_columns[cache_idx]
+    }
+
+    pub fn generate_cache_info(&self, cache_idx: usize) -> Option<HashJoinBuildCacheInfo> {
+        self.used_cache_indexes.get(&cache_idx)?;
+        Some(HashJoinBuildCacheInfo {
+            cache_idx,
+            columns: self.column_indexes[cache_idx].clone(),
+        })
+    }
+}
+
+impl Binder {
+    #[async_backtrace::framed]
+    pub(crate) async fn bind_values(
+        &mut self,
+        bind_context: &mut BindContext,
+        span: Span,
+        values: &[Vec<AExpr>],
+    ) -> Result<(SExpr, BindContext)> {
+        bind_values(
+            self.ctx.clone(),
+            &self.name_resolution_ctx,
+            self.metadata.clone(),
+            bind_context,
+            span,
+            values,
+            Some(&mut self.expression_scan_context),
+        )
+        .await
+    }
+
+    fn check_values_semantic(span: Span, values: &[Vec<AExpr>]) -> Result<()> {
+        if values.is_empty() {
+            return Err(ErrorCode::SemanticError(
+                "Values lists must have at least one row".to_string(),
+            )
+            .set_span(span));
+        }
+        let same_length = values.windows(2).all(|v| v[0].len() == v[1].len());
+        if !same_length {
+            return Err(ErrorCode::SemanticError(
+                "Values lists must all be the same length".to_string(),
+            )
+            .set_span(span));
+        }
+        Ok(())
+    }
+
+    // Remove unused cache columns and join conditions and construct ExpressionScan's child.
+    pub fn construct_expression_scan(
+        &mut self,
+        s_expr: &SExpr,
+        metadata: MetadataRef,
+    ) -> Result<(SExpr, ColumnSet)> {
+        match s_expr.plan.as_ref() {
+            RelOperator::Join(join) if join.build_side_cache_info.is_some() => {
+                let mut join = join.clone();
+                let build_side_cache_info = join.build_side_cache_info.as_mut().unwrap();
+
+                let (left, left_correlated_columns) =
+                    self.construct_expression_scan(s_expr.child(0)?, metadata.clone())?;
+
+                let (right, right_correlated_columns) =
+                    self.construct_expression_scan(s_expr.child(1)?, metadata.clone())?;
+
+                // Remove unused cache columns from hash join build side.
+                let used_cache_columns = &self.expression_scan_context.used_column_indexes
+                    [build_side_cache_info.cache_idx];
+                for index in (0..build_side_cache_info.columns.len()).rev() {
+                    let column_index = build_side_cache_info.columns[index];
+                    if !used_cache_columns.contains(&column_index) {
+                        build_side_cache_info.columns.remove(index);
+                    }
+                }
+
+                // Remove unused join conditions.
+                let join_conditions = [join.left_conditions.clone(), join.right_conditions.clone()];
+                for index in (0..join.left_conditions.len()).rev() {
+                    let mut used = false;
+                    for conditions in join_conditions.iter() {
+                        let used_columns = conditions[index].used_columns();
+                        if used_columns.is_subset(&left_correlated_columns) {
+                            used = true;
+                            break;
+                        }
+                    }
+
+                    if !used {
+                        join.left_conditions.remove(index);
+                        join.right_conditions.remove(index);
+                    }
+                }
+
+                let s_expr = s_expr
+                    .replace_plan(Arc::new(RelOperator::Join(join)))
+                    .replace_children(vec![Arc::new(left), Arc::new(right)]);
+                Ok((s_expr, right_correlated_columns))
+            }
+            RelOperator::ExpressionScan(scan) => {
+                // The join condition columns may consist of the following two parts:
+                // (1) expression scan columns.
+                // (2) correlated columns in values.
+                let mut join_condition_columns = ColumnSet::new();
+                for row in scan.values.iter() {
+                    for (scalar, column_index) in row
+                        .iter()
+                        .zip(scan.column_indexes.iter())
+                        .take(scan.num_scalar_columns)
+                    {
+                        join_condition_columns.insert(*column_index);
+                        for index in scalar.used_columns() {
+                            let derived_index = self
+                                .expression_scan_context
+                                .get_derived_column(scan.expression_scan_index, index);
+                            join_condition_columns.insert(derived_index);
+                        }
+                    }
+                }
+
+                let mut scan = scan.clone();
+
+                // Remove ExpressionScan unused cache columns.
+                let mut cache_scan_columns = vec![];
+                let mut cache_scan_column_indexes = vec![];
+                for index in (scan.num_scalar_columns..scan.values[0].len()).rev() {
+                    let column_index = scan.column_indexes[index];
+                    if join_condition_columns.contains(&column_index) {
+                        cache_scan_columns.push(scan.values[0][index].clone());
+                        let original_index = self
+                            .expression_scan_context
+                            .get_original_column(scan.expression_scan_index, column_index);
+                        cache_scan_column_indexes.push(original_index);
+                        self.expression_scan_context
+                            .add_used_cache_column_index(scan.cache_index, original_index)
+                    } else {
+                        scan.remove_cache_column(index);
+                    }
+                }
+
+                // Construct ExpressionScan schema.
+                let mut expression_scan_field = Vec::with_capacity(scan.values[0].len());
+                for (column_index, data_type) in
+                    scan.column_indexes.iter().zip(scan.data_types.iter())
+                {
+                    let field = DataField::new(&column_index.to_string(), data_type.clone());
+                    expression_scan_field.push(field);
+                }
+                let expression_scan_schema = DataSchemaRefExt::create(expression_scan_field);
+                scan.schema = expression_scan_schema;
+
+                // Construct CacheScan.
+                let mut cache_scan_fields = Vec::with_capacity(cache_scan_columns.len());
+                for (column, column_index) in cache_scan_columns
+                    .iter()
+                    .zip(cache_scan_column_indexes.iter())
+                {
+                    let field = DataField::new(&column_index.to_string(), column.data_type()?);
+                    cache_scan_fields.push(field);
+                }
+
+                let cache_source = CacheSource::HashJoinBuild((
+                    scan.cache_index,
+                    cache_scan_column_indexes.clone(),
+                ));
+                let cache_scan = SExpr::create_leaf(Arc::new(RelOperator::CacheScan(CacheScan {
+                    cache_source,
+                    columns: ColumnSet::new(),
+                    schema: DataSchemaRefExt::create(cache_scan_fields),
+                })));
+
+                let mut distinct_columns = Vec::new();
+                for column in scan.values[0].iter().skip(scan.num_scalar_columns) {
+                    distinct_columns.push(column);
+                }
+
+                // Wrap CacheScan with distinct to eliminate duplicates rows.
+                let mut group_items = Vec::with_capacity(cache_scan_column_indexes.len());
+                for (index, column_index) in cache_scan_column_indexes.iter().enumerate() {
+                    group_items.push(ScalarItem {
+                        scalar: cache_scan_columns[index].clone(),
+                        index: *column_index,
+                    });
+                }
+
+                let s_expr = SExpr::create_unary(
+                    Arc::new(RelOperator::ExpressionScan(scan)),
+                    Arc::new(SExpr::create_unary(
+                        Arc::new(
+                            Aggregate {
+                                mode: AggregateMode::Initial,
+                                group_items,
+                                aggregate_functions: vec![],
+                                from_distinct: false,
+                                limit: None,
+                                grouping_sets: None,
+                            }
+                            .into(),
+                        ),
+                        Arc::new(cache_scan),
+                    )),
+                );
+
+                Ok((s_expr, join_condition_columns))
+            }
+            _ => {
+                let mut correlated_columns = ColumnSet::new();
+                let mut children = Vec::with_capacity(s_expr.arity());
+                for child in s_expr.children() {
+                    let (child, columns) =
+                        self.construct_expression_scan(child, metadata.clone())?;
+                    children.push(Arc::new(child));
+                    correlated_columns.extend(columns);
+                }
+                Ok((s_expr.replace_children(children), correlated_columns))
+            }
+        }
+    }
+}
+
+pub async fn bind_values(
+    ctx: Arc<dyn TableContext>,
+    name_resolution_ctx: &NameResolutionContext,
+    metadata: MetadataRef,
+    bind_context: &mut BindContext,
+    span: Span,
+    values: &[Vec<AExpr>],
+    expression_scan_info: Option<&mut ExpressionScanContext>,
+) -> Result<(SExpr, BindContext)> {
+    // Check the semantic of values lists.
+    Binder::check_values_semantic(span, values)?;
+
+    let mut scalar_binder = ScalarBinder::new(
+        bind_context,
+        ctx.clone(),
+        name_resolution_ctx,
+        metadata.clone(),
+        &[],
+        HashMap::new(),
+        Box::new(IndexMap::new()),
+    );
+
+    let num_values = values.len();
+    let num_columns = values[0].len();
+    // Scalar expressions for each column.
+    let mut column_scalars = vec![Vec::with_capacity(num_values); num_columns];
+    // Common data type for each column.
+    let mut column_common_type = Vec::with_capacity(num_columns);
+    // Used cache indexes, if it's not empty, we will use ExpressionScan.
+    let mut cache_indexes = HashSet::new();
+
+    for (row_idx, row) in values.iter().enumerate() {
+        for (column_idx, expr) in row.iter().enumerate() {
+            let (scalar, data_type) = scalar_binder.bind(expr).await?;
+            let used_columns = scalar.used_columns();
+            if !used_columns.is_empty() {
+                if let Some(expression_scan_info) = expression_scan_info.as_ref() {
+                    for column_index in used_columns {
+                        if let Some(cache_index) =
+                            expression_scan_info.find_cache_index(column_index)
+                        {
+                            cache_indexes.insert(cache_index);
+                        } else {
+                            return Err(ErrorCode::SemanticError(format!(
+                                "Can find cache index for {:?}",
+                                &scalar
+                            ))
+                            .set_span(span));
+                        }
+                    }
+                } else {
+                    return Err(ErrorCode::SemanticError(format!(
+                        "There is no HashJoinBuildSideInfo for column reference {:?}",
+                        &scalar
+                    ))
+                    .set_span(span));
+                }
+            }
+            column_scalars[column_idx].push((scalar, data_type.clone()));
+
+            // Get the common data type for each column.
+            if row_idx == 0 {
+                column_common_type.push(data_type);
+            } else {
+                let common_type = &column_common_type[column_idx];
+                if common_type != &data_type {
+                    match common_super_type(
+                        common_type.clone(),
+                        data_type.clone(),
+                        &BUILTIN_FUNCTIONS.default_cast_rules,
+                    ) {
+                        Some(new_common_type) => column_common_type[column_idx] = new_common_type,
+                        None => {
+                            return Err(ErrorCode::SemanticError(format!(
+                                "{} and {} don't have common data type",
+                                common_type, data_type
+                            ))
+                            .set_span(span));
+                        }
+                    }
+                }
+            }
+        }
+    }
+
+    if !cache_indexes.is_empty() {
+        if cache_indexes.len() > 1 {
+            return Err(ErrorCode::SemanticError(
+                "Values can only reference one cache".to_string(),
+            )
+            .set_span(span));
+        }
+        let cache_index = *cache_indexes.iter().next().unwrap();
+        let expression_scan_info = expression_scan_info.unwrap();
+        bind_expression_scan(
+            metadata,
+            bind_context,
+            span,
+            num_values,
+            num_columns,
+            column_scalars,
+            column_common_type,
+            cache_index,
+            expression_scan_info,
+        )
+    } else {
+        bind_constant_scan(
+            ctx,
+            metadata,
+            bind_context,
+            span,
+            num_values,
+            num_columns,
+            column_scalars,
+            column_common_type,
+        )
+    }
+}
+
+pub fn bind_expression_scan(
+    metadata: MetadataRef,
+    bind_context: &mut BindContext,
+    span: Span,
+    num_values: usize,
+    num_columns: usize,
+    column_scalars: Vec<Vec<(ScalarExpr, DataType)>>,
+    mut column_common_type: Vec<DataType>,
+    cache_index: usize,
+    expression_scan_info: &mut ExpressionScanContext,
+) -> Result<(SExpr, BindContext)> {
+    // Add the cache index to the used cache indexes.
+    expression_scan_info.add_used_cache_index(cache_index);
+    // Get the columns in the cache.
+    let cache_columns = expression_scan_info.get_cache_columns(cache_index).clone();
+    // Generate expression scan index.
+    let expression_scan_index = expression_scan_info.add_expression_scan();
+    // The scalars include the values and the cache columns.
+    let mut scalars = vec![Vec::with_capacity(num_columns + cache_columns.len()); num_values];
+    for (column_idx, column) in column_scalars.iter().enumerate() {
+        for (row_idx, (scalar, data_type)) in column.iter().enumerate() {
+            let scalar = if data_type != &column_common_type[column_idx] {
+                wrap_cast(scalar, &column_common_type[column_idx])
+            } else {
+                scalar.clone()
+            };
+            scalars[row_idx].push(scalar);
+        }
+    }
+
+    // Assigns default column names col0, col1, etc.
+    let names = (0..num_columns + cache_columns.len())
+        .map(|i| format!("col{}", i))
+        .collect::<Vec<_>>();
+    let mut expression_scan_fields = Vec::with_capacity(names.len());
+    for column in cache_columns.iter() {
+        column_common_type.push(*column.data_type.clone());
+    }
+    // Build expression scan fields.
+    for (name, data_type) in names.into_iter().zip(column_common_type.iter()) {
+        let value_field = DataField::new(&name, data_type.clone());
+        expression_scan_fields.push(value_field);
+    }
+
+    let mut metadata = metadata.write();
+    // Column index for the each column.
+    let mut column_indexes = Vec::with_capacity(num_columns + cache_columns.len());
+    // Add column bindings for expression scan columns.
+    for (idx, field) in expression_scan_fields.iter().take(num_columns).enumerate() {
+        let index = metadata.columns().len();
+        let column_binding = ColumnBindingBuilder::new(
+            format!("expr_scan_{}", idx),
+            index,
+            Box::new(field.data_type().clone()),
+            Visibility::Visible,
+        )
+        .build();
+        let _ = metadata.add_derived_column(
+            field.name().clone(),
+            field.data_type().clone(),
+            Some(ScalarExpr::BoundColumnRef(BoundColumnRef {
+                span,
+                column: column_binding.clone(),
+            })),
+        );
+        bind_context.add_column_binding(column_binding);
+        column_indexes.push(index);
+    }
+
+    // Add column bindings for cache columns.
+    for (idx, cache_column) in cache_columns.iter().enumerate() {
+        for row_scalars in scalars.iter_mut() {
+            let scalar = ScalarExpr::BoundColumnRef(BoundColumnRef {
+                span: None,
+                column: cache_column.clone(),
+            });
+            row_scalars.push(scalar);
+        }
+
+        let column_entry = metadata.column(cache_column.index);
+        let name = column_entry.name();
+        let data_type = column_entry.data_type();
+        let new_column_index = metadata.columns().len();
+        let new_column_binding = ColumnBindingBuilder::new(
+            format!("expr_scan_{}", idx + num_columns),
+            new_column_index,
+            Box::new(data_type.clone()),
+            Visibility::Visible,
+        )
+        .build();
+
+        let _ = metadata.add_derived_column(
+            name,
+            data_type,
+            Some(ScalarExpr::BoundColumnRef(BoundColumnRef {
+                span,
+                column: new_column_binding.clone(),
+            })),
+        );
+        bind_context.add_column_binding(new_column_binding);
+        column_indexes.push(new_column_index);
+
+        // Record the mapping between original index (cache index in hash join build side) and derived index.
+        expression_scan_info.add_expression_scan_column(
+            expression_scan_index,
+            cache_column.index,
+            new_column_index,
+        );
+    }
+
+    let s_expr = SExpr::create_leaf(Arc::new(
+        ExpressionScan {
+            expression_scan_index,
+            values: scalars,
+            num_scalar_columns: num_columns,
+            cache_index,
+            data_types: column_common_type,
+            column_indexes,
+            schema: DataSchemaRefExt::create(vec![]),
+        }
+        .into(),
+    ));
+
+    Ok((s_expr, bind_context.clone()))
+}
+
+pub fn bind_constant_scan(
+    ctx: Arc<dyn TableContext>,
+    metadata: MetadataRef,
+    bind_context: &mut BindContext,
+    span: Span,
+    num_values: usize,
+    num_columns: usize,
+    column_scalars: Vec<Vec<(ScalarExpr, DataType)>>,
+    column_common_type: Vec<DataType>,
+) -> Result<(SExpr, BindContext)> {
+    let names = (0..num_columns)
+        .map(|i| format!("col{}", i))
+        .collect::<Vec<_>>();
+    let mut value_fields = Vec::with_capacity(names.len());
+    for (name, common_type) in names.into_iter().zip(column_common_type.into_iter()) {
+        let value_field = DataField::new(&name, common_type);
+        value_fields.push(value_field);
+    }
+    let value_schema = DataSchema::new(value_fields);
+
+    let input = DataBlock::empty();
+    let func_ctx = ctx.get_function_context()?;
+    let evaluator = Evaluator::new(&input, &func_ctx, &BUILTIN_FUNCTIONS);
+
+    // use values to build columns
+    let mut value_columns = Vec::with_capacity(column_scalars.len());
+    for (scalars, value_field) in column_scalars.iter().zip(value_schema.fields().iter()) {
+        let mut builder =
+            ColumnBuilder::with_capacity(value_field.data_type(), column_scalars.len());
+        for (scalar, value_type) in scalars {
+            let scalar = if value_type != value_field.data_type() {
+                wrap_cast(scalar, value_field.data_type())
+            } else {
+                scalar.clone()
+            };
+            let expr = scalar
+                .as_expr()?
+                .project_column_ref(|col| value_schema.index_of(&col.index.to_string()).unwrap());
+            let result = evaluator.run(&expr)?;
+
+            match result.as_scalar() {
+                Some(val) => {
+                    builder.push(val.as_ref());
+                }
+                None => {
+                    return Err(ErrorCode::SemanticError(format!(
+                        "Value must be a scalar, but get {}",
+                        result
+                    ))
+                    .set_span(span));
+                }
+            }
+        }
+        value_columns.push(builder.build());
+    }
+
+    // add column bindings
+    let mut columns = ColumnSet::new();
+    let mut fields = Vec::with_capacity(num_values);
+    let mut metadata = metadata.write();
+    for value_field in value_schema.fields() {
+        let index = metadata.columns().len();
+        columns.insert(index);
+        let column_binding = ColumnBindingBuilder::new(
+            value_field.name().clone(),
+            index,
+            Box::new(value_field.data_type().clone()),
+            Visibility::Visible,
+        )
+        .build();
+        let _ = metadata.add_derived_column(
+            value_field.name().clone(),
+            value_field.data_type().clone(),
+            Some(ScalarExpr::BoundColumnRef(BoundColumnRef {
+                span,
+                column: column_binding.clone(),
+            })),
+        );
+        bind_context.add_column_binding(column_binding);
+
+        let field = DataField::new(&index.to_string(), value_field.data_type().clone());
+        fields.push(field);
+    }
+    let schema = DataSchemaRefExt::create(fields);
+
+    let s_expr = SExpr::create_leaf(Arc::new(
+        ConstantTableScan {
+            values: value_columns,
+            num_rows: num_values,
+            schema,
+            columns,
+        }
+        .into(),
+    ));
+
+    Ok((s_expr, bind_context.clone()))
+}
diff --git a/src/query/sql/src/planner/binder/bind_query/mod.rs b/src/query/sql/src/planner/binder/bind_query/mod.rs
new file mode 100644
index 0000000000000..a100b232fa511
--- /dev/null
+++ b/src/query/sql/src/planner/binder/bind_query/mod.rs
@@ -0,0 +1,23 @@
+// Copyright 2021 Datafuse Labs
+//
+// Licensed 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.
+
+mod bind;
+mod bind_limit;
+mod bind_select;
+mod bind_set_expr;
+mod bind_value;
+
+pub use bind_select::MaxColumnPosition;
+pub use bind_value::bind_values;
+pub use bind_value::ExpressionScanContext;
diff --git a/src/query/sql/src/planner/binder/bind_table_reference/bind.rs b/src/query/sql/src/planner/binder/bind_table_reference/bind.rs
new file mode 100644
index 0000000000000..a646eb8d279ee
--- /dev/null
+++ b/src/query/sql/src/planner/binder/bind_table_reference/bind.rs
@@ -0,0 +1,87 @@
+// Copyright 2021 Datafuse Labs
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+use async_recursion::async_recursion;
+use databend_common_ast::ast::TableReference;
+use databend_common_exception::Result;
+
+use crate::binder::Binder;
+use crate::optimizer::SExpr;
+use crate::BindContext;
+
+impl Binder {
+    #[async_recursion]
+    #[async_backtrace::framed]
+    pub(crate) async fn bind_table_reference(
+        &mut self,
+        bind_context: &mut BindContext,
+        table_ref: &TableReference,
+    ) -> Result<(SExpr, BindContext)> {
+        match table_ref {
+            TableReference::Table {
+                span,
+                catalog,
+                database,
+                table,
+                alias,
+                temporal,
+                pivot: _,
+                unpivot: _,
+                consume,
+            } => {
+                self.bind_table(
+                    bind_context,
+                    span,
+                    catalog,
+                    database,
+                    table,
+                    alias,
+                    temporal,
+                    *consume,
+                )
+                .await
+            }
+            TableReference::TableFunction {
+                span,
+                name,
+                params,
+                named_params,
+                alias,
+                ..
+            } => {
+                self.bind_table_function(bind_context, span, name, params, named_params, alias)
+                    .await
+            }
+            TableReference::Subquery {
+                span: _,
+                lateral,
+                subquery,
+                alias,
+            } => {
+                self.bind_subquery(bind_context, *lateral, subquery, alias)
+                    .await
+            }
+            TableReference::Location {
+                span: _,
+                location,
+                options,
+                alias,
+            } => {
+                self.bind_location(bind_context, location, options, alias)
+                    .await
+            }
+            TableReference::Join { join, .. } => self.bind_join(bind_context, join).await,
+        }
+    }
+}
diff --git a/src/query/sql/src/planner/binder/join.rs b/src/query/sql/src/planner/binder/bind_table_reference/bind_join.rs
similarity index 78%
rename from src/query/sql/src/planner/binder/join.rs
rename to src/query/sql/src/planner/binder/bind_table_reference/bind_join.rs
index 32a0605044f6e..355198d0f176d 100644
--- a/src/query/sql/src/planner/binder/join.rs
+++ b/src/query/sql/src/planner/binder/bind_table_reference/bind_join.rs
@@ -27,8 +27,8 @@ use databend_common_exception::ErrorCode;
 use databend_common_exception::Result;
 use indexmap::IndexMap;
 
-use super::Finder;
 use crate::binder::CteInfo;
+use crate::binder::Finder;
 use crate::binder::JoinPredicate;
 use crate::binder::Visibility;
 use crate::normalize_identifier;
@@ -42,11 +42,13 @@ use crate::planner::binder::Binder;
 use crate::planner::semantic::NameResolutionContext;
 use crate::plans::BoundColumnRef;
 use crate::plans::Filter;
+use crate::plans::HashJoinBuildCacheInfo;
 use crate::plans::Join;
 use crate::plans::JoinType;
 use crate::plans::ScalarExpr;
 use crate::plans::Visitor;
 use crate::BindContext;
+use crate::ColumnBinding;
 use crate::IndexType;
 use crate::MetadataRef;
 
@@ -62,33 +64,112 @@ impl Binder {
     #[async_backtrace::framed]
     pub(crate) async fn bind_join(
         &mut self,
-        bind_context: &BindContext,
+        bind_context: &mut BindContext,
+        join: &databend_common_ast::ast::Join,
+    ) -> Result<(SExpr, BindContext)> {
+        let (left_child, mut left_context) =
+            self.bind_table_reference(bind_context, &join.left).await?;
+        let left_column_bindings = left_context.columns.clone();
+
+        let cache_column_bindings = left_column_bindings.clone();
+        let mut cache_column_indexes = Vec::with_capacity(cache_column_bindings.len());
+        for column in cache_column_bindings.iter() {
+            cache_column_indexes.push(column.index);
+        }
+        let cache_idx = self
+            .expression_scan_context
+            .add_hash_join_build_cache(cache_column_bindings, cache_column_indexes);
+
+        if join.right.is_lateral_table_function() {
+            let (result_expr, bind_context) = self
+                .bind_lateral_table_function(&mut left_context, left_child.clone(), &join.right)
+                .await?;
+            return Ok((result_expr, bind_context));
+        }
+
+        let (right_child, right_context) = self
+            .bind_table_reference(&mut left_context, &join.right)
+            .await?;
+
+        let right_column_bindings = right_context.columns.clone();
+
+        let mut bind_context = bind_context.replace();
+
+        self.check_table_name_and_condition(&left_column_bindings, &right_column_bindings, join)?;
+
+        let join_conditions = self
+            .generate_join_condition(
+                &mut bind_context,
+                join,
+                &left_column_bindings,
+                &right_column_bindings,
+            )
+            .await?;
+
+        let build_side_cache_info = self.expression_scan_context.generate_cache_info(cache_idx);
+
+        let join_type = join_type(&join.op);
+        let s_expr = self
+            .bind_join_with_type(
+                join_type.clone(),
+                join_conditions,
+                left_child,
+                right_child,
+                build_side_cache_info,
+            )
+            .await?;
+        let bind_context = join_bind_context(&join_type, bind_context, left_context, right_context);
+
+        Ok((s_expr, bind_context))
+    }
+
+    // TODO: unify this function with bind_join
+    #[async_recursion]
+    #[async_backtrace::framed]
+    pub(crate) async fn bind_merge_into_join(
+        &mut self,
+        bind_context: &mut BindContext,
         left_context: BindContext,
         right_context: BindContext,
         left_child: SExpr,
         right_child: SExpr,
         join: &databend_common_ast::ast::Join,
     ) -> Result<(SExpr, BindContext)> {
-        check_duplicate_join_tables(&left_context, &right_context)?;
+        self.check_table_name_and_condition(&left_context.columns, &right_context.columns, join)?;
 
         let mut bind_context = bind_context.replace();
 
-        match &join.op {
-            JoinOperator::LeftOuter | JoinOperator::RightOuter | JoinOperator::FullOuter
-                if join.condition == JoinCondition::None =>
-            {
-                return Err(ErrorCode::SemanticError(
-                    "outer join should contain join conditions".to_string(),
-                ));
-            }
-            JoinOperator::CrossJoin if join.condition != JoinCondition::None => {
-                return Err(ErrorCode::SemanticError(
-                    "cross join should not contain join conditions".to_string(),
-                ));
-            }
-            _ => (),
-        };
+        let join_conditions = self
+            .generate_join_condition(
+                &mut bind_context,
+                join,
+                &left_context.columns,
+                &right_context.columns,
+            )
+            .await?;
+
+        let join_type = join_type(&join.op);
+        let s_expr = self
+            .bind_join_with_type(
+                join_type.clone(),
+                join_conditions,
+                left_child,
+                right_child,
+                None,
+            )
+            .await?;
+        let bind_context = join_bind_context(&join_type, bind_context, left_context, right_context);
+
+        Ok((s_expr, bind_context))
+    }
 
+    async fn generate_join_condition(
+        &self,
+        bind_context: &mut BindContext,
+        join: &databend_common_ast::ast::Join,
+        left_column_bindings: &[ColumnBinding],
+        right_column_bindings: &[ColumnBinding],
+    ) -> Result<JoinConditions> {
         let mut left_join_conditions: Vec<ScalarExpr> = vec![];
         let mut right_join_conditions: Vec<ScalarExpr> = vec![];
         let mut non_equi_conditions: Vec<ScalarExpr> = vec![];
@@ -100,11 +181,12 @@ impl Binder {
             self.m_cte_bound_ctx.clone(),
             self.ctes_map.clone(),
             join.op.clone(),
-            &left_context,
-            &right_context,
-            &mut bind_context,
+            left_column_bindings,
+            right_column_bindings,
+            bind_context,
             &join.condition,
         );
+
         join_condition_resolver
             .resolve(
                 &mut left_join_conditions,
@@ -115,80 +197,27 @@ impl Binder {
             )
             .await?;
 
-        let join_conditions = JoinConditions {
+        Ok(JoinConditions {
             left_conditions: left_join_conditions,
             right_conditions: right_join_conditions,
             non_equi_conditions,
             other_conditions,
-        };
-
-        let s_expr = match &join.op {
-            JoinOperator::Inner => {
-                self.bind_join_with_type(JoinType::Inner, join_conditions, left_child, right_child)
-            }
-            JoinOperator::LeftOuter => {
-                self.bind_join_with_type(JoinType::Left, join_conditions, left_child, right_child)
-            }
-            JoinOperator::RightOuter => {
-                self.bind_join_with_type(JoinType::Right, join_conditions, left_child, right_child)
-            }
-            JoinOperator::FullOuter => {
-                self.bind_join_with_type(JoinType::Full, join_conditions, left_child, right_child)
-            }
-            JoinOperator::CrossJoin => {
-                self.bind_join_with_type(JoinType::Cross, join_conditions, left_child, right_child)
-            }
-            JoinOperator::LeftSemi => {
-                bind_context = left_context;
-                self.bind_join_with_type(
-                    JoinType::LeftSemi,
-                    join_conditions,
-                    left_child,
-                    right_child,
-                )
-            }
-            JoinOperator::RightSemi => {
-                bind_context = right_context;
-                self.bind_join_with_type(
-                    JoinType::RightSemi,
-                    join_conditions,
-                    left_child,
-                    right_child,
-                )
-            }
-            JoinOperator::LeftAnti => {
-                bind_context = left_context;
-                self.bind_join_with_type(
-                    JoinType::LeftAnti,
-                    join_conditions,
-                    left_child,
-                    right_child,
-                )
-            }
-            JoinOperator::RightAnti => {
-                bind_context = right_context;
-                self.bind_join_with_type(
-                    JoinType::RightAnti,
-                    join_conditions,
-                    left_child,
-                    right_child,
-                )
-            }
-        }?;
-        Ok((s_expr, bind_context))
+        })
     }
 
-    pub fn bind_join_with_type(
+    pub(crate) async fn bind_join_with_type(
         &mut self,
         mut join_type: JoinType,
         join_conditions: JoinConditions,
         mut left_child: SExpr,
         mut right_child: SExpr,
+        build_side_cache_info: Option<HashJoinBuildCacheInfo>,
     ) -> Result<SExpr> {
         let mut left_conditions = join_conditions.left_conditions;
         let mut right_conditions = join_conditions.right_conditions;
         let mut non_equi_conditions = join_conditions.non_equi_conditions;
         let other_conditions = join_conditions.other_conditions;
+
         if join_type == JoinType::Cross
             && (!left_conditions.is_empty() || !right_conditions.is_empty())
         {
@@ -204,15 +233,12 @@ impl Binder {
             &mut non_equi_conditions,
         )?;
 
-        for (left, right) in left_conditions.iter().zip(right_conditions.iter()) {
-            self.eq_scalars.push((left.clone(), right.clone()));
-        }
-
         let right_prop = RelExpr::with_s_expr(&right_child).derive_relational_prop()?;
         let mut is_lateral = false;
         if !right_prop.outer_columns.is_empty() {
             // If there are outer columns in right child, then the join is a correlated lateral join
-            let mut decorrelator = SubqueryRewriter::new(self.ctx.clone(), self.metadata.clone());
+            let mut decorrelator =
+                SubqueryRewriter::new(self.ctx.clone(), self.metadata.clone(), Some(self.clone()));
             right_child = decorrelator.flatten_plan(
                 &right_child,
                 &right_prop.outer_columns,
@@ -233,6 +259,26 @@ impl Binder {
             is_lateral = true;
         }
 
+        let (left_child, right_child, join_type, left_conditions, right_conditions) =
+            // If there are cache indexes used in the expression scan context, we swap the left and right child
+            // to make left child as the build side.
+            if !self.expression_scan_context.used_cache_indexes.is_empty() {
+                (
+                    right_child,
+                    left_child,
+                    join_type.opposite(),
+                    right_conditions,
+                    left_conditions,
+                )
+            } else {
+                (
+                    left_child,
+                    right_child,
+                    join_type,
+                    left_conditions,
+                    right_conditions,
+                )
+            };
         let logical_join = Join {
             left_conditions,
             right_conditions,
@@ -243,6 +289,7 @@ impl Binder {
             need_hold_hash_table: false,
             is_lateral,
             single_to_inner: None,
+            build_side_cache_info,
         };
         Ok(SExpr::create_binary(
             Arc::new(logical_join.into()),
@@ -337,55 +384,82 @@ impl Binder {
 
         Ok(())
     }
+
+    fn check_table_name_and_condition(
+        &self,
+        left_column_bindings: &[ColumnBinding],
+        right_column_bindings: &[ColumnBinding],
+        join: &databend_common_ast::ast::Join,
+    ) -> Result<()> {
+        check_duplicate_join_tables(left_column_bindings, right_column_bindings)?;
+
+        match &join.op {
+            JoinOperator::LeftOuter | JoinOperator::RightOuter | JoinOperator::FullOuter
+                if join.condition == JoinCondition::None =>
+            {
+                return Err(ErrorCode::SemanticError(
+                    "outer join should contain join conditions".to_string(),
+                ));
+            }
+            JoinOperator::CrossJoin if join.condition != JoinCondition::None => {
+                return Err(ErrorCode::SemanticError(
+                    "cross join should not contain join conditions".to_string(),
+                ));
+            }
+            _ => (),
+        };
+
+        Ok(())
+    }
 }
 
 // Wrap nullable for column binding depending on join type.
 fn wrap_nullable_for_column(
     join_type: &JoinOperator,
-    left_context: &BindContext,
-    right_context: &BindContext,
+    left_column_bindings: &[ColumnBinding],
+    right_column_bindings: &[ColumnBinding],
     bind_context: &mut BindContext,
 ) {
     match join_type {
         JoinOperator::LeftOuter => {
-            for column in left_context.all_column_bindings() {
+            for column in left_column_bindings {
                 bind_context.add_column_binding(column.clone());
             }
-            for column in right_context.all_column_bindings().iter() {
+            for column in right_column_bindings {
                 let mut nullable_column = column.clone();
                 nullable_column.data_type = Box::new(column.data_type.wrap_nullable());
                 bind_context.add_column_binding(nullable_column);
             }
         }
         JoinOperator::RightOuter => {
-            for column in left_context.all_column_bindings() {
+            for column in left_column_bindings {
                 let mut nullable_column = column.clone();
                 nullable_column.data_type = Box::new(column.data_type.wrap_nullable());
                 bind_context.add_column_binding(nullable_column);
             }
 
-            for column in right_context.all_column_bindings().iter() {
+            for column in right_column_bindings {
                 bind_context.add_column_binding(column.clone());
             }
         }
         JoinOperator::FullOuter => {
-            for column in left_context.all_column_bindings() {
+            for column in left_column_bindings {
                 let mut nullable_column = column.clone();
                 nullable_column.data_type = Box::new(column.data_type.wrap_nullable());
                 bind_context.add_column_binding(nullable_column);
             }
 
-            for column in right_context.all_column_bindings().iter() {
+            for column in right_column_bindings {
                 let mut nullable_column = column.clone();
                 nullable_column.data_type = Box::new(column.data_type.wrap_nullable());
                 bind_context.add_column_binding(nullable_column);
             }
         }
         _ => {
-            for column in left_context.all_column_bindings() {
+            for column in left_column_bindings {
                 bind_context.add_column_binding(column.clone());
             }
-            for column in right_context.all_column_bindings() {
+            for column in right_column_bindings {
                 bind_context.add_column_binding(column.clone());
             }
         }
@@ -393,17 +467,15 @@ fn wrap_nullable_for_column(
 }
 
 pub fn check_duplicate_join_tables(
-    left_context: &BindContext,
-    right_context: &BindContext,
+    left_column_bindings: &[ColumnBinding],
+    right_column_bindings: &[ColumnBinding],
 ) -> Result<()> {
-    let left_column_bindings = left_context.all_column_bindings();
     let left_table_name = if left_column_bindings.is_empty() {
         None
     } else {
         left_column_bindings[0].table_name.as_ref()
     };
 
-    let right_column_bindings = right_context.all_column_bindings();
     let right_table_name = if right_column_bindings.is_empty() {
         None
     } else {
@@ -430,8 +502,8 @@ struct JoinConditionResolver<'a> {
     m_cte_bound_ctx: HashMap<IndexType, BindContext>,
     ctes_map: Box<IndexMap<String, CteInfo>>,
     join_op: JoinOperator,
-    left_context: &'a BindContext,
-    right_context: &'a BindContext,
+    left_column_bindings: &'a [ColumnBinding],
+    right_column_bindings: &'a [ColumnBinding],
     join_context: &'a mut BindContext,
     join_condition: &'a JoinCondition,
 }
@@ -445,8 +517,8 @@ impl<'a> JoinConditionResolver<'a> {
         m_cte_bound_ctx: HashMap<IndexType, BindContext>,
         ctes_map: Box<IndexMap<String, CteInfo>>,
         join_op: JoinOperator,
-        left_context: &'a BindContext,
-        right_context: &'a BindContext,
+        left_column_bindings: &'a [ColumnBinding],
+        right_column_bindings: &'a [ColumnBinding],
         join_context: &'a mut BindContext,
         join_condition: &'a JoinCondition,
     ) -> Self {
@@ -457,8 +529,8 @@ impl<'a> JoinConditionResolver<'a> {
             m_cte_bound_ctx,
             ctes_map,
             join_op,
-            left_context,
-            right_context,
+            left_column_bindings,
+            right_column_bindings,
             join_context,
             join_condition,
         }
@@ -520,8 +592,8 @@ impl<'a> JoinConditionResolver<'a> {
             JoinCondition::None => {
                 wrap_nullable_for_column(
                     &self.join_op,
-                    self.left_context,
-                    self.right_context,
+                    self.left_column_bindings,
+                    self.right_column_bindings,
                     self.join_context,
                 );
             }
@@ -583,8 +655,8 @@ impl<'a> JoinConditionResolver<'a> {
         }
         wrap_nullable_for_column(
             &self.join_op,
-            self.left_context,
-            self.right_context,
+            self.left_column_bindings,
+            self.right_column_bindings,
             self.join_context,
         );
         Ok(())
@@ -602,8 +674,8 @@ impl<'a> JoinConditionResolver<'a> {
         let mut join_context = (*self.join_context).clone();
         wrap_nullable_for_column(
             &self.join_op,
-            self.left_context,
-            self.right_context,
+            self.left_column_bindings,
+            self.right_column_bindings,
             &mut join_context,
         );
         let mut scalar_binder = ScalarBinder::new(
@@ -653,11 +725,11 @@ impl<'a> JoinConditionResolver<'a> {
     ) -> Result<()> {
         wrap_nullable_for_column(
             &self.join_op,
-            self.left_context,
-            self.right_context,
+            self.left_column_bindings,
+            self.right_column_bindings,
             self.join_context,
         );
-        let left_columns_len = self.left_context.columns.len();
+        let left_columns_len = self.left_column_bindings.len();
         for (span, join_key) in using_columns.iter() {
             let join_key_name = join_key.as_str();
             let left_scalar = if let Some(col_binding) = self.join_context.columns
@@ -757,8 +829,8 @@ impl<'a> JoinConditionResolver<'a> {
         let mut join_context = (*self.join_context).clone();
         wrap_nullable_for_column(
             &JoinOperator::Inner,
-            self.left_context,
-            self.right_context,
+            self.left_column_bindings,
+            self.right_column_bindings,
             &mut join_context,
         );
         let mut scalar_binder = ScalarBinder::new(
@@ -803,16 +875,14 @@ impl<'a> JoinConditionResolver<'a> {
 
     fn left_right_columns(&self) -> Result<(ColumnSet, ColumnSet)> {
         let left_columns: ColumnSet =
-            self.left_context
-                .all_column_bindings()
+            self.left_column_bindings
                 .iter()
                 .fold(ColumnSet::new(), |mut acc, v| {
                     acc.insert(v.index);
                     acc
                 });
         let right_columns: ColumnSet =
-            self.right_context
-                .all_column_bindings()
+            self.right_column_bindings
                 .iter()
                 .fold(ColumnSet::new(), |mut acc, v| {
                     acc.insert(v.index);
@@ -822,8 +892,8 @@ impl<'a> JoinConditionResolver<'a> {
     }
 
     fn find_using_columns(&self, using_columns: &mut Vec<(Span, String)>) -> Result<()> {
-        for left_column in self.left_context.all_column_bindings().iter() {
-            for right_column in self.right_context.all_column_bindings().iter() {
+        for left_column in self.left_column_bindings {
+            for right_column in self.right_column_bindings {
                 if left_column.column_name == right_column.column_name {
                     using_columns.push((None, left_column.column_name.clone()));
                 }
@@ -832,3 +902,30 @@ impl<'a> JoinConditionResolver<'a> {
         Ok(())
     }
 }
+
+fn join_type(join_type: &JoinOperator) -> JoinType {
+    match join_type {
+        JoinOperator::CrossJoin => JoinType::Cross,
+        JoinOperator::Inner => JoinType::Inner,
+        JoinOperator::LeftOuter => JoinType::Left,
+        JoinOperator::RightOuter => JoinType::Right,
+        JoinOperator::FullOuter => JoinType::Full,
+        JoinOperator::LeftSemi => JoinType::LeftSemi,
+        JoinOperator::RightSemi => JoinType::RightSemi,
+        JoinOperator::LeftAnti => JoinType::LeftAnti,
+        JoinOperator::RightAnti => JoinType::RightAnti,
+    }
+}
+
+fn join_bind_context(
+    join_type: &JoinType,
+    bind_context: BindContext,
+    left_context: BindContext,
+    right_context: BindContext,
+) -> BindContext {
+    match join_type {
+        JoinType::LeftSemi | JoinType::LeftAnti => left_context,
+        JoinType::RightSemi | JoinType::RightAnti => right_context,
+        _ => bind_context,
+    }
+}
diff --git a/src/query/sql/src/planner/binder/bind_table_reference/bind_location.rs b/src/query/sql/src/planner/binder/bind_table_reference/bind_location.rs
new file mode 100644
index 0000000000000..a2410c1aeafa9
--- /dev/null
+++ b/src/query/sql/src/planner/binder/bind_table_reference/bind_location.rs
@@ -0,0 +1,65 @@
+// Copyright 2021 Datafuse Labs
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+use std::str::FromStr;
+
+use databend_common_ast::ast::Connection;
+use databend_common_ast::ast::FileLocation;
+use databend_common_ast::ast::SelectStageOptions;
+use databend_common_ast::ast::TableAlias;
+use databend_common_ast::ast::UriLocation;
+use databend_common_exception::Result;
+use databend_common_meta_app::principal::FileFormatParams;
+use databend_common_meta_app::principal::StageFileFormatType;
+use databend_common_storage::StageFilesInfo;
+
+use crate::binder::copy_into_table::resolve_file_location;
+use crate::binder::Binder;
+use crate::optimizer::SExpr;
+use crate::BindContext;
+
+impl Binder {
+    /// Bind a location.
+    #[async_backtrace::framed]
+    pub(crate) async fn bind_location(
+        &mut self,
+        bind_context: &mut BindContext,
+        location: &FileLocation,
+        options: &SelectStageOptions,
+        alias: &Option<TableAlias>,
+    ) -> Result<(SExpr, BindContext)> {
+        let location = match location {
+            FileLocation::Uri(uri) => FileLocation::Uri(UriLocation {
+                connection: Connection::new(options.connection.clone()),
+                ..uri.clone()
+            }),
+            _ => location.clone(),
+        };
+        let (mut stage_info, path) = resolve_file_location(self.ctx.as_ref(), &location).await?;
+        if let Some(f) = &options.file_format {
+            stage_info.file_format_params = match StageFileFormatType::from_str(f) {
+                Ok(t) => FileFormatParams::default_by_type(t)?,
+                _ => self.ctx.get_file_format(f).await?,
+            }
+        }
+        let files_info = StageFilesInfo {
+            path,
+            pattern: options.pattern.clone(),
+            files: options.files.clone(),
+        };
+        let table_ctx = self.ctx.clone();
+        self.bind_stage_table(table_ctx, bind_context, stage_info, files_info, alias, None)
+            .await
+    }
+}
diff --git a/src/query/sql/src/planner/binder/bind_table_reference/bind_subquery.rs b/src/query/sql/src/planner/binder/bind_table_reference/bind_subquery.rs
new file mode 100644
index 0000000000000..df638659aece6
--- /dev/null
+++ b/src/query/sql/src/planner/binder/bind_table_reference/bind_subquery.rs
@@ -0,0 +1,60 @@
+// Copyright 2021 Datafuse Labs
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+use databend_common_ast::ast::Query;
+use databend_common_ast::ast::TableAlias;
+use databend_common_exception::Result;
+
+use crate::binder::Binder;
+use crate::optimizer::SExpr;
+use crate::BindContext;
+
+impl Binder {
+    /// Bind a subquery.
+    #[async_backtrace::framed]
+    pub(crate) async fn bind_subquery(
+        &mut self,
+        bind_context: &mut BindContext,
+        lateral: bool,
+        subquery: &Query,
+        alias: &Option<TableAlias>,
+    ) -> Result<(SExpr, BindContext)> {
+        // If the subquery is a lateral subquery, we need to let it see the columns
+        // from the previous queries.
+        let (result, mut result_bind_context) = if lateral {
+            let mut new_bind_context = BindContext::with_parent(Box::new(bind_context.clone()));
+            self.bind_query(&mut new_bind_context, subquery).await?
+        } else {
+            let mut new_bind_context = BindContext::with_parent(
+                bind_context
+                    .parent
+                    .clone()
+                    .unwrap_or_else(|| Box::new(BindContext::new())),
+            );
+            self.bind_query(&mut new_bind_context, subquery).await?
+        };
+
+        if let Some(alias) = alias {
+            result_bind_context.apply_table_alias(alias, &self.name_resolution_ctx)?;
+            // Reset column name as alias column name
+            for i in 0..alias.columns.len() {
+                let column = &result_bind_context.columns[i];
+                self.metadata
+                    .write()
+                    .change_derived_column_alias(column.index, column.column_name.clone());
+            }
+        }
+        Ok((result, result_bind_context))
+    }
+}
diff --git a/src/query/sql/src/planner/binder/bind_table_reference/bind_table.rs b/src/query/sql/src/planner/binder/bind_table_reference/bind_table.rs
new file mode 100644
index 0000000000000..730726dc25e57
--- /dev/null
+++ b/src/query/sql/src/planner/binder/bind_table_reference/bind_table.rs
@@ -0,0 +1,291 @@
+// Copyright 2021 Datafuse Labs
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+use databend_common_ast::ast::Identifier;
+use databend_common_ast::ast::Statement;
+use databend_common_ast::ast::TableAlias;
+use databend_common_ast::ast::TemporalClause;
+use databend_common_ast::parser::parse_sql;
+use databend_common_ast::parser::tokenize_sql;
+use databend_common_ast::Span;
+use databend_common_catalog::table::TimeNavigation;
+use databend_common_exception::ErrorCode;
+use databend_common_exception::Result;
+use databend_common_storages_view::view_table::QUERY;
+use databend_storages_common_table_meta::table::get_change_type;
+
+use crate::binder::Binder;
+use crate::optimizer::SExpr;
+use crate::BindContext;
+
+impl Binder {
+    /// Bind a base table.
+    /// A base table is a table that is not a view or CTE.
+    #[allow(clippy::too_many_arguments)]
+    #[async_backtrace::framed]
+    pub(crate) async fn bind_table(
+        &mut self,
+        bind_context: &mut BindContext,
+        span: &Span,
+        catalog: &Option<Identifier>,
+        database: &Option<Identifier>,
+        table: &Identifier,
+        alias: &Option<TableAlias>,
+        temporal: &Option<TemporalClause>,
+        consume: bool,
+    ) -> Result<(SExpr, BindContext)> {
+        let (catalog, database, table_name) =
+            self.normalize_object_identifier_triple(catalog, database, table);
+        let table_alias_name = alias
+            .as_ref()
+            .map(|table_alias| self.normalize_identifier(&table_alias.name).name);
+        let mut bind_cte = true;
+        if let Some(cte_name) = &bind_context.cte_name {
+            // If table name equals to cte name, then skip bind cte and find table from catalog
+            // Or will dead loop and stack overflow
+            if cte_name == &table_name {
+                bind_cte = false;
+            }
+        }
+        // Check and bind common table expression
+        let ctes_map = self.ctes_map.clone();
+        if let Some(cte_info) = ctes_map.get(&table_name) {
+            if bind_cte {
+                return if !cte_info.materialized {
+                    self.bind_cte(*span, bind_context, &table_name, alias, cte_info)
+                        .await
+                } else {
+                    self.bind_m_cte(bind_context, cte_info, &table_name, alias, span)
+                        .await
+                };
+            }
+        }
+
+        let tenant = self.ctx.get_tenant();
+
+        let navigation = self.resolve_temporal_clause(bind_context, temporal).await?;
+
+        // Resolve table with catalog
+        let table_meta = match self
+            .resolve_data_source(
+                tenant.tenant_name(),
+                catalog.as_str(),
+                database.as_str(),
+                table_name.as_str(),
+                navigation.as_ref(),
+                self.ctx.clone().get_abort_checker(),
+            )
+            .await
+        {
+            Ok(table) => table,
+            Err(e) => {
+                let mut parent = bind_context.parent.as_mut();
+                loop {
+                    if parent.is_none() {
+                        break;
+                    }
+                    let bind_context = parent.unwrap().as_mut();
+                    let ctes_map = self.ctes_map.clone();
+                    if let Some(cte_info) = ctes_map.get(&table_name) {
+                        return if !cte_info.materialized {
+                            self.bind_cte(*span, bind_context, &table_name, alias, cte_info)
+                                .await
+                        } else {
+                            self.bind_m_cte(bind_context, cte_info, &table_name, alias, span)
+                                .await
+                        };
+                    }
+                    parent = bind_context.parent.as_mut();
+                }
+                if e.code() == ErrorCode::UNKNOWN_DATABASE {
+                    return Err(ErrorCode::UnknownDatabase(format!(
+                        "Unknown database `{}` in catalog '{catalog}'",
+                        database
+                    ))
+                    .set_span(*span));
+                }
+                if e.code() == ErrorCode::UNKNOWN_TABLE {
+                    return Err(ErrorCode::UnknownTable(format!(
+                        "Unknown table `{database}`.`{table_name}` in catalog '{catalog}'"
+                    ))
+                    .set_span(*span));
+                }
+                return Err(e);
+            }
+        };
+
+        if consume && table_meta.engine() != "STREAM" {
+            return Err(ErrorCode::StorageUnsupported(
+                "WITH CONSUME only support in STREAM",
+            ));
+        }
+
+        if navigation.is_some_and(|n| matches!(n, TimeNavigation::Changes { .. }))
+            || table_meta.engine() == "STREAM"
+        {
+            let change_type = get_change_type(&table_alias_name);
+            if change_type.is_some() {
+                let table_index = self.metadata.write().add_table(
+                    catalog,
+                    database.clone(),
+                    table_meta,
+                    table_alias_name,
+                    bind_context.view_info.is_some(),
+                    bind_context.planning_agg_index,
+                    false,
+                    consume,
+                );
+                let (s_expr, mut bind_context) = self
+                    .bind_base_table(bind_context, database.as_str(), table_index, change_type)
+                    .await?;
+
+                if let Some(alias) = alias {
+                    bind_context.apply_table_alias(alias, &self.name_resolution_ctx)?;
+                }
+                return Ok((s_expr, bind_context));
+            }
+
+            let query = table_meta
+                .generage_changes_query(
+                    self.ctx.clone(),
+                    database.as_str(),
+                    table_name.as_str(),
+                    consume,
+                )
+                .await?;
+
+            let mut new_bind_context = BindContext::with_parent(Box::new(bind_context.clone()));
+            let tokens = tokenize_sql(query.as_str())?;
+            let (stmt, _) = parse_sql(&tokens, self.dialect)?;
+            let Statement::Query(query) = &stmt else {
+                unreachable!()
+            };
+            let (s_expr, mut new_bind_context) =
+                self.bind_query(&mut new_bind_context, query).await?;
+
+            let cols = table_meta
+                .schema()
+                .fields()
+                .iter()
+                .map(|f| f.name().clone())
+                .collect::<Vec<_>>();
+            for (index, column_name) in cols.iter().enumerate() {
+                new_bind_context.columns[index].column_name = column_name.clone();
+            }
+
+            if let Some(alias) = alias {
+                new_bind_context.apply_table_alias(alias, &self.name_resolution_ctx)?;
+            } else {
+                for column in new_bind_context.columns.iter_mut() {
+                    column.database_name = None;
+                    column.table_name = Some(table_name.clone());
+                }
+            }
+
+            new_bind_context.parent = Some(Box::new(bind_context.clone()));
+            return Ok((s_expr, new_bind_context));
+        }
+
+        match table_meta.engine() {
+            "VIEW" => {
+                // TODO(leiysky): this check is error-prone,
+                // we should find a better way to do this.
+                Self::check_view_dep(bind_context, &database, &table_name)?;
+                let query = table_meta
+                    .options()
+                    .get(QUERY)
+                    .ok_or_else(|| ErrorCode::Internal("Invalid VIEW object"))?;
+                let tokens = tokenize_sql(query.as_str())?;
+                let (stmt, _) = parse_sql(&tokens, self.dialect)?;
+                // For view, we need use a new context to bind it.
+                let mut new_bind_context = BindContext::with_parent(Box::new(bind_context.clone()));
+                new_bind_context.view_info = Some((database.clone(), table_name));
+                if let Statement::Query(query) = &stmt {
+                    self.metadata.write().add_table(
+                        catalog,
+                        database.clone(),
+                        table_meta,
+                        table_alias_name,
+                        false,
+                        false,
+                        false,
+                        false,
+                    );
+                    let (s_expr, mut new_bind_context) =
+                        self.bind_query(&mut new_bind_context, query).await?;
+                    if let Some(alias) = alias {
+                        // view maybe has alias, e.g. select v1.col1 from v as v1;
+                        new_bind_context.apply_table_alias(alias, &self.name_resolution_ctx)?;
+                    } else {
+                        // e.g. select v0.c0 from v0;
+                        for column in new_bind_context.columns.iter_mut() {
+                            column.database_name = None;
+                            column.table_name = Some(self.normalize_identifier(table).name);
+                        }
+                    }
+                    new_bind_context.parent = Some(Box::new(bind_context.clone()));
+                    Ok((s_expr, new_bind_context))
+                } else {
+                    Err(
+                        ErrorCode::Internal(format!("Invalid VIEW object: {}", table_meta.name()))
+                            .set_span(*span),
+                    )
+                }
+            }
+            _ => {
+                let table_index = self.metadata.write().add_table(
+                    catalog,
+                    database.clone(),
+                    table_meta,
+                    table_alias_name,
+                    bind_context.view_info.is_some(),
+                    bind_context.planning_agg_index,
+                    false,
+                    false,
+                );
+
+                let (s_expr, mut bind_context) = self
+                    .bind_base_table(bind_context, database.as_str(), table_index, None)
+                    .await?;
+                if let Some(alias) = alias {
+                    bind_context.apply_table_alias(alias, &self.name_resolution_ctx)?;
+                }
+                Ok((s_expr, bind_context))
+            }
+        }
+    }
+
+    pub(crate) fn check_view_dep(
+        bind_context: &BindContext,
+        database: &str,
+        view_name: &str,
+    ) -> Result<()> {
+        match &bind_context.parent {
+            Some(parent) => match &parent.view_info {
+                Some((db, v)) => {
+                    if db == database && v == view_name {
+                        Err(ErrorCode::Internal(format!(
+                            "View dependency loop detected (view: {}.{})",
+                            database, view_name
+                        )))
+                    } else {
+                        Self::check_view_dep(parent, database, view_name)
+                    }
+                }
+                _ => Ok(()),
+            },
+            _ => Ok(()),
+        }
+    }
+}
diff --git a/src/query/sql/src/planner/binder/bind_table_reference/bind_table_function.rs b/src/query/sql/src/planner/binder/bind_table_reference/bind_table_function.rs
new file mode 100644
index 0000000000000..bbc87d96663fb
--- /dev/null
+++ b/src/query/sql/src/planner/binder/bind_table_reference/bind_table_function.rs
@@ -0,0 +1,495 @@
+// Copyright 2021 Datafuse Labs
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+use std::collections::HashMap;
+use std::sync::Arc;
+
+use databend_common_ast::ast::Expr;
+use databend_common_ast::ast::FunctionCall as ASTFunctionCall;
+use databend_common_ast::ast::Identifier;
+use databend_common_ast::ast::Literal;
+use databend_common_ast::ast::SelectStmt;
+use databend_common_ast::ast::SelectTarget;
+use databend_common_ast::ast::TableAlias;
+use databend_common_ast::ast::TableReference;
+use databend_common_ast::Span;
+use databend_common_catalog::catalog_kind::CATALOG_DEFAULT;
+use databend_common_catalog::table_args::TableArgs;
+use databend_common_catalog::table_function::TableFunction;
+use databend_common_exception::ErrorCode;
+use databend_common_exception::Result;
+use databend_common_expression::types::NumberScalar;
+use databend_common_expression::FunctionKind;
+use databend_common_expression::Scalar;
+use databend_common_functions::BUILTIN_FUNCTIONS;
+use databend_common_storage::DataOperator;
+use databend_common_storages_result_cache::ResultCacheMetaManager;
+use databend_common_storages_result_cache::ResultCacheReader;
+use databend_common_storages_result_cache::ResultScan;
+use databend_common_users::UserApiProvider;
+
+use crate::binder::scalar::ScalarBinder;
+use crate::binder::table_args::bind_table_args;
+use crate::binder::Binder;
+use crate::binder::ColumnBindingBuilder;
+use crate::binder::Visibility;
+use crate::optimizer::SExpr;
+use crate::planner::semantic::normalize_identifier;
+use crate::plans::EvalScalar;
+use crate::plans::FunctionCall;
+use crate::plans::RelOperator;
+use crate::plans::ScalarItem;
+use crate::BindContext;
+use crate::ScalarExpr;
+
+impl Binder {
+    /// Bind a table function.
+    #[async_backtrace::framed]
+    pub(crate) async fn bind_table_function(
+        &mut self,
+        bind_context: &mut BindContext,
+        span: &Span,
+        name: &Identifier,
+        params: &[Expr],
+        named_params: &[(Identifier, Expr)],
+        alias: &Option<TableAlias>,
+    ) -> Result<(SExpr, BindContext)> {
+        let func_name = normalize_identifier(name, &self.name_resolution_ctx);
+
+        if BUILTIN_FUNCTIONS
+            .get_property(&func_name.name)
+            .map(|p| p.kind == FunctionKind::SRF)
+            .unwrap_or(false)
+        {
+            // If it is a set-returning function, we bind it as a subquery.
+            let args = parse_table_function_args(span, &func_name, params, named_params)?;
+
+            let select_stmt = SelectStmt {
+                span: *span,
+                hints: None,
+                distinct: false,
+                top_n: None,
+                select_list: vec![SelectTarget::AliasedExpr {
+                    expr: Box::new(databend_common_ast::ast::Expr::FunctionCall {
+                        span: *span,
+                        func: ASTFunctionCall {
+                            distinct: false,
+                            name: databend_common_ast::ast::Identifier::from_name(
+                                *span,
+                                &func_name.name,
+                            ),
+                            params: vec![],
+                            args,
+                            window: None,
+                            lambda: None,
+                        },
+                    }),
+                    alias: None,
+                }],
+                from: vec![],
+                selection: None,
+                group_by: None,
+                having: None,
+                window_list: None,
+                qualify: None,
+            };
+            let (srf_expr, mut bind_context) =
+                self.bind_select(bind_context, &select_stmt, &[], 0).await?;
+
+            return self
+                .extract_srf_table_function_columns(
+                    &mut bind_context,
+                    span,
+                    &func_name,
+                    srf_expr,
+                    alias,
+                )
+                .await;
+        }
+
+        let mut scalar_binder = ScalarBinder::new(
+            bind_context,
+            self.ctx.clone(),
+            &self.name_resolution_ctx,
+            self.metadata.clone(),
+            &[],
+            self.m_cte_bound_ctx.clone(),
+            self.ctes_map.clone(),
+        );
+        let table_args = bind_table_args(&mut scalar_binder, params, named_params).await?;
+
+        if func_name.name.eq_ignore_ascii_case("result_scan") {
+            let query_id = parse_result_scan_args(&table_args)?;
+            if query_id.is_empty() {
+                return Err(ErrorCode::InvalidArgument("The `RESULT_SCAN` function requires a 'query_id' parameter. Please specify a valid query ID.")
+                    .set_span(*span));
+            }
+            let kv_store = UserApiProvider::instance().get_meta_store_client();
+            let meta_key = self.ctx.get_result_cache_key(&query_id);
+            if meta_key.is_none() {
+                return Err(ErrorCode::EmptyData(format!(
+                    "`RESULT_SCAN` failed: No cache key found in current session for query ID '{}'.",
+                    query_id
+                )).set_span(*span));
+            }
+            let result_cache_mgr = ResultCacheMetaManager::create(kv_store, 0);
+            let meta_key = meta_key.unwrap();
+            let (table_schema, block_raw_data) = match result_cache_mgr
+                .get(meta_key.clone())
+                .await?
+            {
+                Some(value) => {
+                    let op = DataOperator::instance().operator();
+                    ResultCacheReader::read_table_schema_and_data(op, &value.location).await?
+                }
+                None => {
+                    return Err(ErrorCode::EmptyData(format!(
+                        "`RESULT_SCAN` failed: Unable to fetch cached data for query ID '{}'. The data may have exceeded its TTL or been cleaned up. Cache key: '{}'",
+                        query_id, meta_key
+                    )).set_span(*span));
+                }
+            };
+            let table = ResultScan::try_create(table_schema, query_id, block_raw_data)?;
+
+            let table_alias_name = if let Some(table_alias) = alias {
+                Some(normalize_identifier(&table_alias.name, &self.name_resolution_ctx).name)
+            } else {
+                None
+            };
+
+            let table_index = self.metadata.write().add_table(
+                CATALOG_DEFAULT.to_string(),
+                "system".to_string(),
+                table.clone(),
+                table_alias_name,
+                false,
+                false,
+                false,
+                false,
+            );
+
+            let (s_expr, mut bind_context) = self
+                .bind_base_table(bind_context, "system", table_index, None)
+                .await?;
+            if let Some(alias) = alias {
+                bind_context.apply_table_alias(alias, &self.name_resolution_ctx)?;
+            }
+            Ok((s_expr, bind_context))
+        } else {
+            // Other table functions always reside is default catalog
+            let table_meta: Arc<dyn TableFunction> = self
+                .catalogs
+                .get_default_catalog(self.ctx.txn_mgr())?
+                .get_table_function(&func_name.name, table_args)?;
+            let table = table_meta.as_table();
+            let table_alias_name = if let Some(table_alias) = alias {
+                Some(normalize_identifier(&table_alias.name, &self.name_resolution_ctx).name)
+            } else {
+                None
+            };
+            let table_index = self.metadata.write().add_table(
+                CATALOG_DEFAULT.to_string(),
+                "system".to_string(),
+                table.clone(),
+                table_alias_name,
+                false,
+                false,
+                false,
+                false,
+            );
+
+            let (s_expr, mut bind_context) = self
+                .bind_base_table(bind_context, "system", table_index, None)
+                .await?;
+            if let Some(alias) = alias {
+                bind_context.apply_table_alias(alias, &self.name_resolution_ctx)?;
+            }
+            Ok((s_expr, bind_context))
+        }
+    }
+
+    /// Extract the srf inner tuple fields as columns.
+    #[async_backtrace::framed]
+    async fn extract_srf_table_function_columns(
+        &mut self,
+        bind_context: &mut BindContext,
+        span: &Span,
+        func_name: &Identifier,
+        srf_expr: SExpr,
+        alias: &Option<TableAlias>,
+    ) -> Result<(SExpr, BindContext)> {
+        let fields = if func_name.name.eq_ignore_ascii_case("flatten") {
+            Some(vec![
+                "seq".to_string(),
+                "key".to_string(),
+                "path".to_string(),
+                "index".to_string(),
+                "value".to_string(),
+                "this".to_string(),
+            ])
+        } else if func_name.name.eq_ignore_ascii_case("json_each") {
+            Some(vec!["key".to_string(), "value".to_string()])
+        } else {
+            None
+        };
+
+        if let Some(fields) = fields {
+            if let RelOperator::EvalScalar(plan) = (*srf_expr.plan).clone() {
+                if plan.items.len() != 1 {
+                    return Err(ErrorCode::Internal(format!(
+                        "Invalid table function subquery EvalScalar items, expect 1, but got {}",
+                        plan.items.len()
+                    )));
+                }
+                // Delete srf result tuple column, extract tuple inner columns instead
+                let _ = bind_context.columns.pop();
+                let scalar = &plan.items[0].scalar;
+
+                // Add tuple inner columns
+                let mut items = Vec::with_capacity(fields.len());
+                for (i, field) in fields.into_iter().enumerate() {
+                    let field_expr = ScalarExpr::FunctionCall(FunctionCall {
+                        span: *span,
+                        func_name: "get".to_string(),
+                        params: vec![Scalar::Number(NumberScalar::Int64((i + 1) as i64))],
+                        arguments: vec![scalar.clone()],
+                    });
+                    let data_type = field_expr.data_type()?;
+                    let index = self.metadata.write().add_derived_column(
+                        field.clone(),
+                        data_type.clone(),
+                        Some(field_expr.clone()),
+                    );
+
+                    let column_binding = ColumnBindingBuilder::new(
+                        field,
+                        index,
+                        Box::new(data_type),
+                        Visibility::Visible,
+                    )
+                    .build();
+                    bind_context.add_column_binding(column_binding);
+
+                    items.push(ScalarItem {
+                        scalar: field_expr,
+                        index,
+                    });
+                }
+                let eval_scalar = EvalScalar { items };
+                let new_expr =
+                    SExpr::create_unary(Arc::new(eval_scalar.into()), srf_expr.children[0].clone());
+
+                if let Some(alias) = alias {
+                    bind_context.apply_table_alias(alias, &self.name_resolution_ctx)?;
+                }
+                return Ok((new_expr, bind_context.clone()));
+            } else {
+                return Err(ErrorCode::Internal(
+                    "Invalid subquery in table function: Table functions do not support this type of subquery.",
+                ));
+            }
+        }
+        // Set name for srf result column
+        bind_context.columns[0].column_name = "value".to_string();
+        if let Some(alias) = alias {
+            bind_context.apply_table_alias(alias, &self.name_resolution_ctx)?;
+        }
+        Ok((srf_expr, bind_context.clone()))
+    }
+
+    /// Bind a lateral table function.
+    #[async_backtrace::framed]
+    pub(crate) async fn bind_lateral_table_function(
+        &mut self,
+        parent_context: &mut BindContext,
+        child: SExpr,
+        table_ref: &TableReference,
+    ) -> Result<(SExpr, BindContext)> {
+        match table_ref {
+            TableReference::TableFunction {
+                span,
+                name,
+                params,
+                named_params,
+                alias,
+                ..
+            } => {
+                let mut bind_context = BindContext::with_parent(Box::new(parent_context.clone()));
+                let func_name = normalize_identifier(name, &self.name_resolution_ctx);
+
+                if BUILTIN_FUNCTIONS
+                    .get_property(&func_name.name)
+                    .map(|p| p.kind == FunctionKind::SRF)
+                    .unwrap_or(false)
+                {
+                    let args = parse_table_function_args(span, &func_name, params, named_params)?;
+
+                    // convert lateral join table function to srf function
+                    let srf = Expr::FunctionCall {
+                        span: *span,
+                        func: ASTFunctionCall {
+                            distinct: false,
+                            name: func_name.clone(),
+                            args,
+                            params: vec![],
+                            window: None,
+                            lambda: None,
+                        },
+                    };
+                    let srfs = vec![srf.clone()];
+                    let srf_expr = self
+                        .bind_project_set(&mut bind_context, &srfs, child)
+                        .await?;
+
+                    if let Some((_, srf_result)) = bind_context.srfs.remove(&srf.to_string()) {
+                        let column_binding =
+                            if let ScalarExpr::BoundColumnRef(column_ref) = &srf_result {
+                                column_ref.column.clone()
+                            } else {
+                                // Add result column to metadata
+                                let data_type = srf_result.data_type()?;
+                                let index = self.metadata.write().add_derived_column(
+                                    srf.to_string(),
+                                    data_type.clone(),
+                                    Some(srf_result.clone()),
+                                );
+                                ColumnBindingBuilder::new(
+                                    srf.to_string(),
+                                    index,
+                                    Box::new(data_type),
+                                    Visibility::Visible,
+                                )
+                                .build()
+                            };
+
+                        let eval_scalar = EvalScalar {
+                            items: vec![ScalarItem {
+                                scalar: srf_result,
+                                index: column_binding.index,
+                            }],
+                        };
+                        // Add srf result column
+                        bind_context.add_column_binding(column_binding);
+
+                        let flatten_expr =
+                            SExpr::create_unary(Arc::new(eval_scalar.into()), Arc::new(srf_expr));
+
+                        let (new_expr, mut bind_context) = self
+                            .extract_srf_table_function_columns(
+                                &mut bind_context,
+                                span,
+                                &func_name,
+                                flatten_expr,
+                                alias,
+                            )
+                            .await?;
+
+                        // add left table columns.
+                        let mut new_columns = parent_context.columns.clone();
+                        new_columns.extend_from_slice(&bind_context.columns);
+                        bind_context.columns = new_columns;
+
+                        return Ok((new_expr, bind_context));
+                    } else {
+                        return Err(ErrorCode::Internal("Failed to bind project_set for lateral join. This may indicate an issue with the SRF (Set Returning Function) processing or an internal logic error.")
+                            .set_span(*span));
+                    }
+                } else {
+                    return Err(ErrorCode::InvalidArgument(format!(
+                        "The function '{}' is not supported for lateral joins. Lateral joins currently support only Set Returning Functions (SRFs).",
+                        func_name
+                    ))
+                    .set_span(*span));
+                }
+            }
+            _ => unreachable!(),
+        }
+    }
+}
+
+// parse flatten named params to arguments
+fn parse_table_function_args(
+    span: &Span,
+    func_name: &Identifier,
+    params: &[Expr],
+    named_params: &[(Identifier, Expr)],
+) -> Result<Vec<Expr>> {
+    if func_name.name.eq_ignore_ascii_case("flatten") {
+        // build flatten function arguments.
+        let mut named_args: HashMap<String, Expr> = named_params
+            .iter()
+            .map(|(name, value)| (name.name.to_lowercase(), value.clone()))
+            .collect::<HashMap<_, _>>();
+
+        let mut args = Vec::with_capacity(named_args.len() + params.len());
+        let names = vec!["input", "path", "outer", "recursive", "mode"];
+        for name in names {
+            if named_args.is_empty() {
+                break;
+            }
+            match named_args.remove(name) {
+                Some(val) => args.push(val),
+                None => args.push(Expr::Literal {
+                    span: None,
+                    value: Literal::Null,
+                }),
+            }
+        }
+        if !named_args.is_empty() {
+            let invalid_names = named_args.into_keys().collect::<Vec<String>>().join(", ");
+            return Err(ErrorCode::InvalidArgument(format!(
+                "Invalid named parameters for 'flatten': {}, valid parameters are: [input, path, outer, recursive, mode]",
+                invalid_names,
+            ))
+            .set_span(*span));
+        }
+
+        if !params.is_empty() {
+            args.extend(params.iter().cloned());
+        }
+        Ok(args)
+    } else {
+        if !named_params.is_empty() {
+            let invalid_names = named_params
+                .iter()
+                .map(|(name, _)| name.name.clone())
+                .collect::<Vec<String>>()
+                .join(", ");
+            return Err(ErrorCode::InvalidArgument(format!(
+                "Named parameters are not allowed for '{}'. Invalid parameters provided: {}.",
+                func_name.name, invalid_names
+            ))
+            .set_span(*span));
+        }
+
+        Ok(params.to_vec())
+    }
+}
+
+// copy from common-storages-fuse to avoid cyclic dependency.
+fn string_value(value: &Scalar) -> Result<String> {
+    match value {
+        Scalar::String(val) => Ok(val.clone()),
+        other => Err(ErrorCode::BadArguments(format!(
+            "Expected a string value, but found a '{}'.",
+            other
+        ))),
+    }
+}
+
+#[inline(always)]
+pub fn parse_result_scan_args(table_args: &TableArgs) -> Result<String> {
+    let args = table_args.expect_all_positioned("RESULT_SCAN", Some(1))?;
+    string_value(&args[0])
+}
diff --git a/src/query/sql/src/planner/binder/bind_table_reference/mod.rs b/src/query/sql/src/planner/binder/bind_table_reference/mod.rs
new file mode 100644
index 0000000000000..d8e089d54ca63
--- /dev/null
+++ b/src/query/sql/src/planner/binder/bind_table_reference/mod.rs
@@ -0,0 +1,23 @@
+// Copyright 2021 Datafuse Labs
+//
+// Licensed 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.
+
+mod bind;
+mod bind_join;
+mod bind_location;
+mod bind_subquery;
+mod bind_table;
+mod bind_table_function;
+
+pub use bind_join::JoinConditions;
+pub use bind_table_function::parse_result_scan_args;
diff --git a/src/query/sql/src/planner/binder/binder.rs b/src/query/sql/src/planner/binder/binder.rs
index 08b349d194ec8..9c4527c290f6c 100644
--- a/src/query/sql/src/planner/binder/binder.rs
+++ b/src/query/sql/src/planner/binder/binder.rs
@@ -22,7 +22,6 @@ use databend_common_ast::ast::format_statement;
 use databend_common_ast::ast::Hint;
 use databend_common_ast::ast::Identifier;
 use databend_common_ast::ast::Statement;
-use databend_common_ast::ast::With;
 use databend_common_ast::parser::parse_sql;
 use databend_common_ast::parser::tokenize_sql;
 use databend_common_ast::parser::Dialect;
@@ -42,6 +41,7 @@ use indexmap::IndexMap;
 use log::warn;
 
 use super::Finder;
+use crate::binder::bind_query::ExpressionScanContext;
 use crate::binder::util::illegal_ident_name;
 use crate::binder::wrap_cast;
 use crate::binder::ColumnBindingBuilder;
@@ -81,22 +81,22 @@ use crate::Visibility;
 /// - Check semantic of query
 /// - Validate expressions
 /// - Build `Metadata`
+#[derive(Clone)]
 pub struct Binder {
     pub ctx: Arc<dyn TableContext>,
     pub dialect: Dialect,
     pub catalogs: Arc<CatalogManager>,
     pub name_resolution_ctx: NameResolutionContext,
     pub metadata: MetadataRef,
-    // Save the equal scalar exprs for joins
-    // Eg: SELECT * FROM (twocolumn AS a JOIN twocolumn AS b USING(x) JOIN twocolumn AS c on a.x = c.x) ORDER BY x LIMIT 1
-    // The eq_scalars is [(a.x, b.x), (a.x, c.x)]
-    pub eq_scalars: Vec<(ScalarExpr, ScalarExpr)>,
     // Save the bound context for materialized cte, the key is cte_idx
     pub m_cte_bound_ctx: HashMap<IndexType, BindContext>,
     pub m_cte_bound_s_expr: HashMap<IndexType, SExpr>,
     /// Use `IndexMap` because need to keep the insertion order
     /// Then wrap materialized ctes to main plan.
     pub ctes_map: Box<IndexMap<String, CteInfo>>,
+    /// The `ExpressionScanContext` is used to store the information of
+    /// expression scan and hash join build cache.
+    pub expression_scan_context: ExpressionScanContext,
 }
 
 impl<'a> Binder {
@@ -114,29 +114,20 @@ impl<'a> Binder {
             name_resolution_ctx,
             metadata,
             m_cte_bound_ctx: Default::default(),
-            eq_scalars: vec![],
             m_cte_bound_s_expr: Default::default(),
             ctes_map: Box::default(),
+            expression_scan_context: ExpressionScanContext::new(),
         }
     }
 
-    // After the materialized cte was bound, add it to `m_cte_bound_ctx`
-    pub fn set_m_cte_bound_ctx(&mut self, cte_idx: IndexType, bound_ctx: BindContext) {
-        self.m_cte_bound_ctx.insert(cte_idx, bound_ctx);
-    }
-
-    pub fn set_m_cte_bound_s_expr(&mut self, cte_idx: IndexType, s_expr: SExpr) {
-        self.m_cte_bound_s_expr.insert(cte_idx, s_expr);
-    }
-
     #[async_backtrace::framed]
     #[minitrace::trace]
     pub async fn bind(mut self, stmt: &Statement) -> Result<Plan> {
         let start = Instant::now();
         self.ctx.set_status_info("binding");
-        let mut init_bind_context = BindContext::new();
-        let plan = self.bind_statement(&mut init_bind_context, stmt).await?;
-        self.bind_query_index(&mut init_bind_context, &plan).await?;
+        let mut bind_context = BindContext::new();
+        let plan = self.bind_statement(&mut bind_context, stmt).await?;
+        self.bind_query_index(&mut bind_context, &plan).await?;
         self.ctx.set_status_info(&format!(
             "bind stmt to plan done, time used: {:?}",
             start.elapsed()
@@ -144,49 +135,6 @@ impl<'a> Binder {
         Ok(plan)
     }
 
-    pub(crate) async fn opt_hints_set_var(
-        &mut self,
-        bind_context: &mut BindContext,
-        hints: &Hint,
-    ) -> Result<()> {
-        let mut type_checker = TypeChecker::try_create(
-            bind_context,
-            self.ctx.clone(),
-            &self.name_resolution_ctx,
-            self.metadata.clone(),
-            &[],
-            false,
-        )?;
-        let mut hint_settings: HashMap<String, String> = HashMap::new();
-        for hint in &hints.hints_list {
-            let variable = &hint.name.name;
-            let (scalar, _) = *type_checker.resolve(&hint.expr).await?;
-
-            let scalar = wrap_cast(&scalar, &DataType::String);
-            let expr = scalar.as_expr()?;
-
-            let (new_expr, _) =
-                ConstantFolder::fold(&expr, &self.ctx.get_function_context()?, &BUILTIN_FUNCTIONS);
-            match new_expr {
-                Expr::Constant { scalar, .. } => {
-                    let value = scalar.into_string().unwrap();
-                    if variable.to_lowercase().as_str() == "timezone" {
-                        let tz = value.trim_matches(|c| c == '\'' || c == '\"');
-                        tz.parse::<Tz>().map_err(|_| {
-                            ErrorCode::InvalidTimezone(format!("Invalid Timezone: {:?}", value))
-                        })?;
-                    }
-                    hint_settings.entry(variable.to_string()).or_insert(value);
-                }
-                _ => {
-                    warn!("fold hints {:?} failed. value must be constant value", hint);
-                }
-            }
-        }
-
-        self.ctx.get_settings().set_batch_settings(&hint_settings)
-    }
-
     #[async_recursion::async_recursion]
     #[async_backtrace::framed]
     pub(crate) async fn bind_statement(
@@ -197,6 +145,7 @@ impl<'a> Binder {
         let plan = match stmt {
             Statement::Query(query) => {
                 let (mut s_expr, bind_context) = self.bind_query(bind_context, query).await?;
+
                 // Wrap `LogicalMaterializedCte` to `s_expr`
                 for (_, cte_info) in self.ctes_map.iter().rev() {
                     if !cte_info.materialized || cte_info.used_count == 0 {
@@ -210,6 +159,10 @@ impl<'a> Binder {
                         Arc::new(s_expr),
                     );
                 }
+
+                // Remove unused cache columns and join conditions and construct ExpressionScan's child.
+                (s_expr, _) = self.construct_expression_scan(&s_expr, self.metadata.clone())?;
+
                 let formatted_ast = if self.ctx.get_settings().get_enable_query_result_cache()? {
                     Some(format_statement(stmt.clone())?)
                 } else {
@@ -681,6 +634,62 @@ impl<'a> Binder {
         Ok(plan)
     }
 
+    pub(crate) fn normalize_identifier(&self, ident: &Identifier) -> Identifier {
+        normalize_identifier(ident, &self.name_resolution_ctx)
+    }
+
+    pub(crate) async fn opt_hints_set_var(
+        &mut self,
+        bind_context: &mut BindContext,
+        hints: &Hint,
+    ) -> Result<()> {
+        let mut type_checker = TypeChecker::try_create(
+            bind_context,
+            self.ctx.clone(),
+            &self.name_resolution_ctx,
+            self.metadata.clone(),
+            &[],
+            false,
+        )?;
+        let mut hint_settings: HashMap<String, String> = HashMap::new();
+        for hint in &hints.hints_list {
+            let variable = &hint.name.name;
+            let (scalar, _) = *type_checker.resolve(&hint.expr).await?;
+
+            let scalar = wrap_cast(&scalar, &DataType::String);
+            let expr = scalar.as_expr()?;
+
+            let (new_expr, _) =
+                ConstantFolder::fold(&expr, &self.ctx.get_function_context()?, &BUILTIN_FUNCTIONS);
+            match new_expr {
+                Expr::Constant { scalar, .. } => {
+                    let value = scalar.into_string().unwrap();
+                    if variable.to_lowercase().as_str() == "timezone" {
+                        let tz = value.trim_matches(|c| c == '\'' || c == '\"');
+                        tz.parse::<Tz>().map_err(|_| {
+                            ErrorCode::InvalidTimezone(format!("Invalid Timezone: {:?}", value))
+                        })?;
+                    }
+                    hint_settings.entry(variable.to_string()).or_insert(value);
+                }
+                _ => {
+                    warn!("fold hints {:?} failed. value must be constant value", hint);
+                }
+            }
+        }
+
+        self.ctx.get_settings().set_batch_settings(&hint_settings)
+    }
+
+    // After the materialized cte was bound, add it to `m_cte_bound_ctx`
+    pub fn set_m_cte_bound_ctx(&mut self, cte_idx: IndexType, bound_ctx: BindContext) {
+        self.m_cte_bound_ctx.insert(cte_idx, bound_ctx);
+    }
+
+    pub fn set_m_cte_bound_s_expr(&mut self, cte_idx: IndexType, s_expr: SExpr) {
+        self.m_cte_bound_s_expr.insert(cte_idx, s_expr);
+    }
+
     #[async_backtrace::framed]
     pub(crate) async fn bind_rewrite_to_query(
         &mut self,
@@ -724,13 +733,13 @@ impl<'a> Binder {
     ) -> (String, String, String) {
         let catalog_name = catalog
             .as_ref()
-            .map(|ident| normalize_identifier(ident, &self.name_resolution_ctx).name)
+            .map(|ident| self.normalize_identifier(ident).name)
             .unwrap_or_else(|| self.ctx.get_current_catalog());
         let database_name = database
             .as_ref()
-            .map(|ident| normalize_identifier(ident, &self.name_resolution_ctx).name)
+            .map(|ident| self.normalize_identifier(ident).name)
             .unwrap_or_else(|| self.ctx.get_current_database());
-        let object_name = normalize_identifier(object, &self.name_resolution_ctx).name;
+        let object_name = self.normalize_identifier(object).name;
         (catalog_name, database_name, object_name)
     }
 
@@ -739,12 +748,6 @@ impl<'a> Binder {
         normalize_identifier(ident, &self.name_resolution_ctx).name
     }
 
-    pub fn judge_equal_scalars(&self, left: &ScalarExpr, right: &ScalarExpr) -> bool {
-        self.eq_scalars
-            .iter()
-            .any(|(l, r)| (l == left && r == right) || (l == right && r == left))
-    }
-
     pub(crate) fn check_allowed_scalar_expr(&self, scalar: &ScalarExpr) -> Result<bool> {
         let f = |scalar: &ScalarExpr| {
             matches!(
@@ -901,31 +904,4 @@ impl<'a> Binder {
         finder.visit(scalar)?;
         Ok(finder.scalars().is_empty())
     }
-
-    pub(crate) fn add_cte(&mut self, with: &With, bind_context: &mut BindContext) -> Result<()> {
-        for (idx, cte) in with.ctes.iter().enumerate() {
-            let table_name = normalize_identifier(&cte.alias.name, &self.name_resolution_ctx).name;
-            if bind_context.cte_map_ref.contains_key(&table_name) {
-                return Err(ErrorCode::SemanticError(format!(
-                    "duplicate cte {table_name}"
-                )));
-            }
-            let cte_info = CteInfo {
-                columns_alias: cte
-                    .alias
-                    .columns
-                    .iter()
-                    .map(|c| normalize_identifier(c, &self.name_resolution_ctx).name)
-                    .collect(),
-                query: *cte.query.clone(),
-                materialized: cte.materialized,
-                cte_idx: idx,
-                used_count: 0,
-                columns: vec![],
-            };
-            self.ctes_map.insert(table_name.clone(), cte_info.clone());
-            bind_context.cte_map_ref.insert(table_name, cte_info);
-        }
-        Ok(())
-    }
 }
diff --git a/src/query/sql/src/planner/binder/copy_into_location.rs b/src/query/sql/src/planner/binder/copy_into_location.rs
index 6e4eb4080c897..ddd7c575a196b 100644
--- a/src/query/sql/src/planner/binder/copy_into_location.rs
+++ b/src/query/sql/src/planner/binder/copy_into_location.rs
@@ -59,9 +59,7 @@ impl<'a> Binder {
                 }
             }
             CopyIntoLocationSource::Query(query) => {
-                if let Some(with) = &stmt.with {
-                    self.add_cte(with, bind_context)?;
-                }
+                self.init_cte(bind_context, &stmt.with)?;
                 self.bind_statement(bind_context, &Statement::Query(query.clone()))
                     .await
             }
diff --git a/src/query/sql/src/planner/binder/copy_into_table.rs b/src/query/sql/src/planner/binder/copy_into_table.rs
index bb4d916c8a5ce..3abbc6a91cb2d 100644
--- a/src/query/sql/src/planner/binder/copy_into_table.rs
+++ b/src/query/sql/src/planner/binder/copy_into_table.rs
@@ -64,8 +64,8 @@ use log::debug;
 use log::warn;
 use parking_lot::RwLock;
 
+use crate::binder::bind_query::MaxColumnPosition;
 use crate::binder::location::parse_uri_location;
-use crate::binder::select::MaxColumnPosition;
 use crate::binder::Binder;
 use crate::plans::CopyIntoTableMode;
 use crate::plans::CopyIntoTablePlan;
@@ -96,9 +96,7 @@ impl<'a> Binder {
                     .await
             }
             CopyIntoTableSource::Query(query) => {
-                if let Some(with) = &stmt.with {
-                    self.add_cte(with, bind_context)?;
-                }
+                self.init_cte(bind_context, &stmt.with)?;
 
                 let mut max_column_position = MaxColumnPosition::new();
                 query.drive(&mut max_column_position);
diff --git a/src/query/sql/src/planner/binder/delete.rs b/src/query/sql/src/planner/binder/delete.rs
index b497d825debd1..c6d467faf0f20 100644
--- a/src/query/sql/src/planner/binder/delete.rs
+++ b/src/query/sql/src/planner/binder/delete.rs
@@ -70,9 +70,7 @@ impl<'a> Binder {
             ..
         } = stamt;
 
-        if let Some(with) = &with {
-            self.add_cte(with, bind_context)?;
-        }
+        self.init_cte(bind_context, with)?;
 
         let (catalog_name, database_name, table_name) = if let TableReference::Table {
             catalog,
@@ -89,7 +87,7 @@ impl<'a> Binder {
             ));
         };
 
-        let (table_expr, mut context) = self.bind_single_table(bind_context, table).await?;
+        let (table_expr, mut context) = self.bind_table_reference(bind_context, table).await?;
 
         context.allow_internal_columns(false);
         let mut scalar_binder = ScalarBinder::new(
@@ -204,7 +202,7 @@ impl Binder {
         scan.columns.insert(row_id_index.unwrap());
         table_expr.plan = Arc::new(Scan(scan));
         let filter_expr = SExpr::create_unary(Arc::new(filter.into()), Arc::new(table_expr));
-        let mut rewriter = SubqueryRewriter::new(self.ctx.clone(), self.metadata.clone());
+        let mut rewriter = SubqueryRewriter::new(self.ctx.clone(), self.metadata.clone(), None);
         let filter_expr = rewriter.rewrite(&filter_expr)?;
 
         Ok(SubqueryDesc {
diff --git a/src/query/sql/src/planner/binder/insert.rs b/src/query/sql/src/planner/binder/insert.rs
index e29ab7e09a243..5c20302e7a8ac 100644
--- a/src/query/sql/src/planner/binder/insert.rs
+++ b/src/query/sql/src/planner/binder/insert.rs
@@ -86,9 +86,9 @@ impl Binder {
             overwrite,
             ..
         } = stmt;
-        if let Some(with) = &with {
-            self.add_cte(with, bind_context)?;
-        }
+
+        self.init_cte(bind_context, with)?;
+
         let (catalog_name, database_name, table_name) =
             self.normalize_object_identifier_triple(catalog, database, table);
         let table = self
diff --git a/src/query/sql/src/planner/binder/insert_multi_table.rs b/src/query/sql/src/planner/binder/insert_multi_table.rs
index bf51fcae548ac..0d6f44b9642ac 100644
--- a/src/query/sql/src/planner/binder/insert_multi_table.rs
+++ b/src/query/sql/src/planner/binder/insert_multi_table.rs
@@ -59,7 +59,9 @@ impl Binder {
                 alias: None,
             };
 
-            let (s_expr, bind_context) = self.bind_single_table(bind_context, &table_ref).await?;
+            let (s_expr, bind_context) =
+                self.bind_table_reference(bind_context, &table_ref).await?;
+
             let select_plan = Plan::Query {
                 s_expr: Box::new(s_expr),
                 metadata: self.metadata.clone(),
diff --git a/src/query/sql/src/planner/binder/merge_into.rs b/src/query/sql/src/planner/binder/merge_into.rs
index 61f3fd82d6598..a5229ea3dfa03 100644
--- a/src/query/sql/src/planner/binder/merge_into.rs
+++ b/src/query/sql/src/planner/binder/merge_into.rs
@@ -202,8 +202,10 @@ impl Binder {
         let source_data = source.transform_table_reference();
 
         // bind source data
-        let (mut source_expr, mut source_context) =
-            self.bind_single_table(bind_context, &source_data).await?;
+        let (mut source_expr, mut source_context) = self
+            .bind_table_reference(bind_context, &source_data)
+            .await?;
+
         // remove stream column.
         source_context
             .columns
@@ -236,10 +238,9 @@ impl Binder {
                 let default_target_table_schema = table_schema.remove_computed_fields();
                 let mut update_columns =
                     HashMap::with_capacity(default_target_table_schema.num_fields());
-                let source_output_columns = &source_context.columns;
                 // we use Vec as the value, because there could be duplicate names
                 let mut name_map = HashMap::<String, Vec<ColumnBinding>>::new();
-                for column in source_output_columns {
+                for column in source_context.columns.iter() {
                     name_map
                         .entry(column.column_name.clone())
                         .or_default()
@@ -288,7 +289,7 @@ impl Binder {
         // when the target table has been binded in bind_merge_into_source
         // bind table for target table
         let (mut target_expr, mut target_context) = self
-            .bind_single_table(&mut source_context, &target_table)
+            .bind_table_reference(bind_context, &target_table)
             .await?;
 
         if table.change_tracking_enabled() && merge_type != MergeIntoType::InsertOnly {
@@ -344,9 +345,9 @@ impl Binder {
         };
 
         let (join_sexpr, mut bind_ctx) = self
-            .bind_join(
+            .bind_merge_into_join(
                 bind_context,
-                target_context.clone(),
+                target_context,
                 source_context,
                 target_expr,
                 source_expr,
diff --git a/src/query/sql/src/planner/binder/mod.rs b/src/query/sql/src/planner/binder/mod.rs
index 0970879da1990..b1ecfbf351d40 100644
--- a/src/query/sql/src/planner/binder/mod.rs
+++ b/src/query/sql/src/planner/binder/mod.rs
@@ -14,6 +14,8 @@
 
 mod aggregate;
 mod bind_context;
+mod bind_query;
+mod bind_table_reference;
 #[allow(clippy::module_inception)]
 mod binder;
 /// SQL builders;
@@ -30,9 +32,7 @@ mod having;
 mod insert;
 mod insert_multi_table;
 mod internal_column_factory;
-mod join;
 mod kill;
-mod limit;
 mod location;
 mod merge_into;
 mod presign;
@@ -54,11 +54,12 @@ mod table_args;
 mod udf;
 mod update;
 mod util;
-mod values;
 mod window;
 
 pub use aggregate::AggregateInfo;
 pub use bind_context::*;
+pub use bind_query::bind_values;
+pub use bind_table_reference::parse_result_scan_args;
 pub use binder::Binder;
 pub use builders::*;
 pub use column_binding::ColumnBinding;
@@ -74,6 +75,4 @@ pub use merge_into::MergeIntoType;
 pub use scalar::ScalarBinder;
 pub use scalar_common::*;
 pub use stream_column_factory::STREAM_COLUMN_FACTORY;
-pub use table::parse_result_scan_args;
-pub use values::bind_values;
 pub use window::WindowOrderByInfo;
diff --git a/src/query/sql/src/planner/binder/select.rs b/src/query/sql/src/planner/binder/select.rs
index eaa7a96b78fe5..37b54ad0280b3 100644
--- a/src/query/sql/src/planner/binder/select.rs
+++ b/src/query/sql/src/planner/binder/select.rs
@@ -16,27 +16,11 @@ use std::collections::HashMap;
 use std::collections::HashSet;
 use std::sync::Arc;
 
-use async_recursion::async_recursion;
-use databend_common_ast::ast::BinaryOperator;
-use databend_common_ast::ast::ColumnID;
-use databend_common_ast::ast::ColumnPosition;
-use databend_common_ast::ast::ColumnRef;
 use databend_common_ast::ast::Expr;
-use databend_common_ast::ast::Expr::Array;
-use databend_common_ast::ast::FunctionCall;
-use databend_common_ast::ast::GroupBy;
-use databend_common_ast::ast::Identifier;
-use databend_common_ast::ast::Join;
-use databend_common_ast::ast::JoinCondition;
-use databend_common_ast::ast::JoinOperator;
-use databend_common_ast::ast::Literal;
-use databend_common_ast::ast::OrderByExpr;
-use databend_common_ast::ast::Query;
 use databend_common_ast::ast::SelectStmt;
 use databend_common_ast::ast::SelectTarget;
 use databend_common_ast::ast::SetExpr;
 use databend_common_ast::ast::SetOperator;
-use databend_common_ast::ast::TableReference;
 use databend_common_ast::Span;
 use databend_common_exception::ErrorCode;
 use databend_common_exception::Result;
@@ -45,16 +29,10 @@ use databend_common_expression::types::DataType;
 use databend_common_expression::ROW_ID_COLUMN_ID;
 use databend_common_expression::ROW_ID_COL_NAME;
 use databend_common_functions::BUILTIN_FUNCTIONS;
-use databend_common_license::license::Feature;
-use databend_common_license::license_manager::get_license_manager;
-use derive_visitor::Drive;
-use derive_visitor::Visitor;
-use log::warn;
 
 use super::sort::OrderItem;
 use super::Finder;
-use crate::binder::join::JoinConditions;
-use crate::binder::project_set::SrfCollector;
+use crate::binder::bind_table_reference::JoinConditions;
 use crate::binder::scalar_common::split_conjunctions;
 use crate::binder::ColumnBindingBuilder;
 use crate::binder::ExprContext;
@@ -72,12 +50,8 @@ use crate::plans::ScalarExpr;
 use crate::plans::ScalarItem;
 use crate::plans::UnionAll;
 use crate::plans::Visitor as _;
-use crate::AsyncFunctionRewriter;
-use crate::ColumnBinding;
 use crate::ColumnEntry;
 use crate::IndexType;
-use crate::UdfRewriter;
-use crate::VirtualColumnRewriter;
 use crate::Visibility;
 
 // A normalized IR for `SELECT` clause.
@@ -94,343 +68,6 @@ pub struct SelectItem<'a> {
 }
 
 impl Binder {
-    #[async_backtrace::framed]
-    pub(crate) async fn bind_select_stmt(
-        &mut self,
-        bind_context: &mut BindContext,
-        stmt: &SelectStmt,
-        order_by: &[OrderByExpr],
-        limit: usize,
-    ) -> Result<(SExpr, BindContext)> {
-        if let Some(hints) = &stmt.hints {
-            if let Some(e) = self.opt_hints_set_var(bind_context, hints).await.err() {
-                warn!(
-                    "In SELECT resolve optimize hints {:?} failed, err: {:?}",
-                    hints, e
-                );
-            }
-        }
-        let (mut s_expr, mut from_context) = if stmt.from.is_empty() {
-            let select_list = &stmt.select_list;
-            self.bind_one_table(bind_context, select_list).await?
-        } else {
-            let mut max_column_position = MaxColumnPosition::new();
-            stmt.drive(&mut max_column_position);
-            self.metadata
-                .write()
-                .set_max_column_position(max_column_position.max_pos);
-
-            let cross_joins = stmt
-                .from
-                .iter()
-                .cloned()
-                .reduce(|left, right| TableReference::Join {
-                    span: None,
-                    join: Join {
-                        op: JoinOperator::CrossJoin,
-                        condition: JoinCondition::None,
-                        left: Box::new(left),
-                        right: Box::new(right),
-                    },
-                })
-                .unwrap();
-            self.bind_table_reference(bind_context, &cross_joins)
-                .await?
-        };
-
-        let mut rewriter = SelectRewriter::new(
-            from_context.all_column_bindings(),
-            self.name_resolution_ctx.unquoted_ident_case_sensitive,
-        );
-        let new_stmt = rewriter.rewrite(stmt)?;
-        let stmt = new_stmt.as_ref().unwrap_or(stmt);
-
-        // Collect set returning functions
-        let set_returning_functions = {
-            let mut collector = SrfCollector::new();
-            stmt.select_list.iter().for_each(|item| {
-                if let SelectTarget::AliasedExpr { expr, .. } = item {
-                    collector.visit(expr);
-                }
-            });
-            collector.into_srfs()
-        };
-
-        // Bind set returning functions
-        s_expr = self
-            .bind_project_set(&mut from_context, &set_returning_functions, s_expr)
-            .await?;
-
-        // Try put window definitions into bind context.
-        // This operation should be before `normalize_select_list` because window functions can be used in select list.
-        self.analyze_window_definition(&mut from_context, &stmt.window_list)?;
-
-        // Generate a analyzed select list with from context
-        let mut select_list = self
-            .normalize_select_list(&mut from_context, &stmt.select_list)
-            .await?;
-
-        // This will potentially add some alias group items to `from_context` if find some.
-        if let Some(group_by) = stmt.group_by.as_ref() {
-            self.analyze_group_items(&mut from_context, &select_list, group_by)
-                .await?;
-        }
-
-        self.analyze_aggregate_select(&mut from_context, &mut select_list)?;
-
-        // `analyze_window` should behind `analyze_aggregate_select`,
-        // because `analyze_window` will rewrite the aggregate functions in the window function's arguments.
-        self.analyze_window(&mut from_context, &mut select_list)?;
-
-        let aliases = select_list
-            .items
-            .iter()
-            .map(|item| (item.alias.clone(), item.scalar.clone()))
-            .collect::<Vec<_>>();
-
-        // To support using aliased column in `WHERE` clause,
-        // we should bind where after `select_list` is rewritten.
-        let where_scalar = if let Some(expr) = &stmt.selection {
-            let (new_expr, scalar) = self
-                .bind_where(&mut from_context, &aliases, expr, s_expr)
-                .await?;
-            s_expr = new_expr;
-            Some(scalar)
-        } else {
-            None
-        };
-
-        // `analyze_projection` should behind `analyze_aggregate_select` because `analyze_aggregate_select` will rewrite `grouping`.
-        let (mut scalar_items, projections) = self.analyze_projection(
-            &from_context.aggregate_info,
-            &from_context.windows,
-            &select_list,
-        )?;
-
-        let having = if let Some(having) = &stmt.having {
-            Some(
-                self.analyze_aggregate_having(&mut from_context, &aliases, having)
-                    .await?,
-            )
-        } else {
-            None
-        };
-
-        let qualify = if let Some(qualify) = &stmt.qualify {
-            Some(
-                self.analyze_window_qualify(&mut from_context, &aliases, qualify)
-                    .await?,
-            )
-        } else {
-            None
-        };
-
-        let order_items = self
-            .analyze_order_items(
-                &mut from_context,
-                &mut scalar_items,
-                &aliases,
-                &projections,
-                order_by,
-                stmt.distinct,
-            )
-            .await?;
-
-        // After all analysis is done.
-        if set_returning_functions.is_empty() {
-            // Ignore SRFs.
-            self.analyze_lazy_materialization(
-                &from_context,
-                stmt,
-                &scalar_items,
-                &select_list,
-                &where_scalar,
-                &order_items.items,
-                limit,
-            )?;
-        }
-
-        if !from_context.aggregate_info.aggregate_functions.is_empty()
-            || !from_context.aggregate_info.group_items.is_empty()
-        {
-            s_expr = self.bind_aggregate(&mut from_context, s_expr).await?;
-        }
-
-        if let Some(having) = having {
-            s_expr = self.bind_having(&mut from_context, having, s_expr).await?;
-        }
-
-        // bind window
-        // window run after the HAVING clause but before the ORDER BY clause.
-        for window_info in &from_context.windows.window_functions {
-            s_expr = self.bind_window_function(window_info, s_expr).await?;
-        }
-
-        if let Some(qualify) = qualify {
-            s_expr = self
-                .bind_qualify(&mut from_context, qualify, s_expr)
-                .await?;
-        }
-
-        if stmt.distinct {
-            s_expr = self.bind_distinct(
-                stmt.span,
-                &from_context,
-                &projections,
-                &mut scalar_items,
-                s_expr,
-            )?;
-        }
-
-        if !order_by.is_empty() {
-            s_expr = self
-                .bind_order_by(
-                    &from_context,
-                    order_items,
-                    &select_list,
-                    &mut scalar_items,
-                    s_expr,
-                )
-                .await?;
-        }
-
-        s_expr = self.bind_projection(&mut from_context, &projections, &scalar_items, s_expr)?;
-
-        // rewrite async function to async function plan
-        let mut async_func_rewriter = AsyncFunctionRewriter::new();
-        s_expr = async_func_rewriter.rewrite(&s_expr)?;
-
-        // rewrite udf for interpreter udf
-        let mut udf_rewriter = UdfRewriter::new(self.metadata.clone(), true);
-        s_expr = udf_rewriter.rewrite(&s_expr)?;
-
-        // rewrite udf for server udf
-        let mut udf_rewriter = UdfRewriter::new(self.metadata.clone(), false);
-        s_expr = udf_rewriter.rewrite(&s_expr)?;
-
-        // rewrite variant inner fields as virtual columns
-        let mut virtual_column_rewriter =
-            VirtualColumnRewriter::new(self.ctx.clone(), self.metadata.clone());
-        s_expr = virtual_column_rewriter.rewrite(&s_expr).await?;
-
-        // check inverted index license
-        if !from_context.inverted_index_map.is_empty() {
-            let license_manager = get_license_manager();
-            license_manager
-                .manager
-                .check_enterprise_enabled(self.ctx.get_license_key(), Feature::InvertedIndex)?;
-        }
-        // add internal column binding into expr
-        s_expr = from_context.add_internal_column_into_expr(s_expr)?;
-
-        let mut output_context = BindContext::new();
-        output_context.parent = from_context.parent;
-        output_context.columns = from_context.columns;
-
-        Ok((s_expr, output_context))
-    }
-
-    #[async_recursion]
-    #[async_backtrace::framed]
-    pub(crate) async fn bind_set_expr(
-        &mut self,
-        bind_context: &mut BindContext,
-        set_expr: &SetExpr,
-        order_by: &[OrderByExpr],
-        limit: usize,
-    ) -> Result<(SExpr, BindContext)> {
-        match set_expr {
-            SetExpr::Select(stmt) => {
-                self.bind_select_stmt(bind_context, stmt, order_by, limit)
-                    .await
-            }
-            SetExpr::Query(stmt) => self.bind_query(bind_context, stmt).await,
-            SetExpr::SetOperation(set_operation) => {
-                self.bind_set_operator(
-                    bind_context,
-                    &set_operation.left,
-                    &set_operation.right,
-                    &set_operation.op,
-                    &set_operation.all,
-                )
-                .await
-            }
-            SetExpr::Values { span, values } => self.bind_values(bind_context, *span, values).await,
-        }
-    }
-
-    #[async_recursion]
-    #[async_backtrace::framed]
-    pub(crate) async fn bind_query(
-        &mut self,
-        bind_context: &mut BindContext,
-        query: &Query,
-    ) -> Result<(SExpr, BindContext)> {
-        if let Some(with) = &query.with {
-            self.add_cte(with, bind_context)?;
-        }
-
-        let limit_empty = query.limit.is_empty();
-        let (mut limit, offset) = if !limit_empty {
-            if query.limit.len() == 1 {
-                Self::analyze_limit(Some(&query.limit[0]), &query.offset)?
-            } else {
-                Self::analyze_limit(Some(&query.limit[1]), &Some(query.limit[0].clone()))?
-            }
-        } else if query.offset.is_some() {
-            Self::analyze_limit(None, &query.offset)?
-        } else {
-            (None, 0)
-        };
-
-        let mut contain_top_n = false;
-        let (mut s_expr, bind_context) = match &query.body {
-            SetExpr::Select(stmt) => {
-                if !limit_empty && stmt.top_n.is_some() {
-                    return Err(ErrorCode::SemanticError(
-                        "Duplicate LIMIT: TopN and Limit cannot be used together",
-                    ));
-                } else if let Some(n) = stmt.top_n {
-                    contain_top_n = true;
-                    limit = Some(n as usize);
-                }
-                self.bind_set_expr(
-                    bind_context,
-                    &query.body,
-                    &query.order_by,
-                    limit.unwrap_or_default(),
-                )
-                .await?
-            }
-            SetExpr::Query(_) => {
-                self.bind_set_expr(
-                    bind_context,
-                    &query.body,
-                    &query.order_by,
-                    limit.unwrap_or_default(),
-                )
-                .await?
-            }
-            SetExpr::SetOperation(_) | SetExpr::Values { .. } => {
-                let (mut s_expr, mut bind_context) = self
-                    .bind_set_expr(bind_context, &query.body, &[], limit.unwrap_or_default())
-                    .await?;
-                if !query.order_by.is_empty() {
-                    s_expr = self
-                        .bind_order_by_for_set_operation(&mut bind_context, s_expr, &query.order_by)
-                        .await?;
-                }
-                (s_expr, bind_context)
-            }
-        };
-
-        if !query.limit.is_empty() || contain_top_n || query.offset.is_some() {
-            s_expr = Self::bind_limit(s_expr, limit, offset);
-        }
-
-        Ok((s_expr, bind_context))
-    }
-
     #[async_backtrace::framed]
     pub async fn bind_where(
         &mut self,
@@ -509,6 +146,7 @@ impl Binder {
                     left_expr,
                     right_expr,
                 )
+                .await
             }
             (SetOperator::Except, false) => {
                 // Transfer Except to Anti join
@@ -520,6 +158,7 @@ impl Binder {
                     left_expr,
                     right_expr,
                 )
+                .await
             }
             (SetOperator::Union, true) => self.bind_union(
                 left.span(),
@@ -612,7 +251,7 @@ impl Binder {
         Ok((new_expr, new_bind_context))
     }
 
-    pub fn bind_intersect(
+    pub async fn bind_intersect(
         &mut self,
         left_span: Span,
         right_span: Span,
@@ -630,9 +269,10 @@ impl Binder {
             right_expr,
             JoinType::LeftSemi,
         )
+        .await
     }
 
-    pub fn bind_except(
+    pub async fn bind_except(
         &mut self,
         left_span: Span,
         right_span: Span,
@@ -650,10 +290,11 @@ impl Binder {
             right_expr,
             JoinType::LeftAnti,
         )
+        .await
     }
 
     #[allow(clippy::too_many_arguments)]
-    pub fn bind_intersect_or_except(
+    pub async fn bind_intersect_or_except(
         &mut self,
         left_span: Span,
         right_span: Span,
@@ -699,7 +340,9 @@ impl Binder {
             non_equi_conditions: vec![],
             other_conditions: vec![],
         };
-        let s_expr = self.bind_join_with_type(join_type, join_conditions, left_expr, right_expr)?;
+        let s_expr = self
+            .bind_join_with_type(join_type, join_conditions, left_expr, right_expr, None)
+            .await?;
         Ok((s_expr, left_context))
     }
 
@@ -814,7 +457,7 @@ impl Binder {
     }
 
     #[allow(clippy::too_many_arguments)]
-    fn analyze_lazy_materialization(
+    pub(crate) fn analyze_lazy_materialization(
         &self,
         bind_context: &BindContext,
         stmt: &SelectStmt,
@@ -959,262 +602,3 @@ impl Binder {
         Ok(())
     }
 }
-
-/// It is useful when implementing some SQL syntax sugar,
-///
-/// [`column_binding`] contains the column binding information of the SelectStmt.
-///
-/// to rewrite the SelectStmt, just add a new rewrite_* function and call it in the `rewrite` function.
-struct SelectRewriter<'a> {
-    column_binding: &'a [ColumnBinding],
-    new_stmt: Option<SelectStmt>,
-    is_unquoted_ident_case_sensitive: bool,
-}
-
-// helper functions to SelectRewriter
-impl<'a> SelectRewriter<'a> {
-    fn compare_unquoted_ident(&self, a: &str, b: &str) -> bool {
-        if self.is_unquoted_ident_case_sensitive {
-            a == b
-        } else {
-            a.eq_ignore_ascii_case(b)
-        }
-    }
-
-    fn parse_aggregate_function(expr: &Expr) -> Result<(&Identifier, &[Expr])> {
-        match expr {
-            Expr::FunctionCall {
-                func: FunctionCall { name, args, .. },
-                ..
-            } => Ok((name, args)),
-            _ => Err(ErrorCode::SyntaxException("Aggregate function is required")),
-        }
-    }
-
-    fn expr_eq_from_col_and_value(col: Identifier, value: Expr) -> Expr {
-        Expr::BinaryOp {
-            span: None,
-            left: Box::new(Expr::ColumnRef {
-                span: None,
-                column: ColumnRef {
-                    database: None,
-                    table: None,
-                    column: ColumnID::Name(col),
-                },
-            }),
-            op: BinaryOperator::Eq,
-            right: Box::new(value),
-        }
-    }
-
-    fn target_func_from_name_args(
-        name: Identifier,
-        args: Vec<Expr>,
-        alias: Option<Identifier>,
-    ) -> SelectTarget {
-        SelectTarget::AliasedExpr {
-            expr: Box::new(Expr::FunctionCall {
-                span: Span::default(),
-                func: FunctionCall {
-                    distinct: false,
-                    name,
-                    args,
-                    params: vec![],
-                    window: None,
-                    lambda: None,
-                },
-            }),
-            alias,
-        }
-    }
-
-    fn expr_literal_array_from_vec_ident(exprs: Vec<Identifier>) -> Expr {
-        Array {
-            span: Span::default(),
-            exprs: exprs
-                .into_iter()
-                .map(|expr| Expr::Literal {
-                    span: None,
-                    value: Literal::String(expr.name),
-                })
-                .collect(),
-        }
-    }
-
-    fn expr_column_ref_array_from_vec_ident(exprs: Vec<Identifier>) -> Expr {
-        Array {
-            span: Span::default(),
-            exprs: exprs
-                .into_iter()
-                .map(|expr| Expr::ColumnRef {
-                    span: None,
-                    column: ColumnRef {
-                        database: None,
-                        table: None,
-                        column: ColumnID::Name(expr),
-                    },
-                })
-                .collect(),
-        }
-    }
-
-    // For Expr::Literal, expr.to_string() is quoted, sometimes we need the raw string.
-    fn raw_string_from_literal_expr(expr: &Expr) -> Option<String> {
-        match expr {
-            Expr::Literal { value, .. } => match value {
-                Literal::String(v) => Some(v.clone()),
-                _ => Some(expr.to_string()),
-            },
-            _ => None,
-        }
-    }
-}
-
-impl<'a> SelectRewriter<'a> {
-    fn new(column_binding: &'a [ColumnBinding], is_unquoted_ident_case_sensitive: bool) -> Self {
-        SelectRewriter {
-            column_binding,
-            new_stmt: None,
-            is_unquoted_ident_case_sensitive,
-        }
-    }
-
-    fn rewrite(&mut self, stmt: &SelectStmt) -> Result<Option<SelectStmt>> {
-        self.rewrite_pivot(stmt)?;
-        self.rewrite_unpivot(stmt)?;
-        Ok(self.new_stmt.take())
-    }
-
-    fn rewrite_pivot(&mut self, stmt: &SelectStmt) -> Result<()> {
-        if stmt.from.len() != 1 || stmt.from[0].pivot().is_none() {
-            return Ok(());
-        }
-        let pivot = stmt.from[0].pivot().unwrap();
-        let (aggregate_name, aggregate_args) = Self::parse_aggregate_function(&pivot.aggregate)?;
-        let aggregate_columns = aggregate_args
-            .iter()
-            .map(|expr| match expr {
-                Expr::ColumnRef { column, .. } => Some(column.clone()),
-                _ => None,
-            })
-            .collect::<Option<Vec<_>>>()
-            .ok_or_else(|| ErrorCode::SyntaxException("Aggregate column not found"))?;
-        let aggregate_column_names = aggregate_columns
-            .iter()
-            .map(|col| col.column.name())
-            .collect::<Vec<_>>();
-        let new_group_by = stmt.group_by.clone().unwrap_or_else(|| {
-            GroupBy::Normal(
-                self.column_binding
-                    .iter()
-                    .filter(|col_bind| {
-                        !self
-                            .compare_unquoted_ident(&col_bind.column_name, &pivot.value_column.name)
-                            && !aggregate_column_names
-                                .iter()
-                                .any(|col| self.compare_unquoted_ident(col, &col_bind.column_name))
-                    })
-                    .map(|col| Expr::Literal {
-                        span: Span::default(),
-                        value: Literal::UInt64(col.index as u64 + 1),
-                    })
-                    .collect(),
-            )
-        });
-
-        let mut new_select_list = stmt.select_list.clone();
-        if let Some(star) = new_select_list.iter_mut().find(|target| target.is_star()) {
-            let mut exclude_columns: Vec<_> = aggregate_columns
-                .iter()
-                .map(|c| Identifier::from_name(stmt.span, c.column.name()))
-                .collect();
-            exclude_columns.push(pivot.value_column.clone());
-            star.exclude(exclude_columns);
-        };
-        let new_aggregate_name = Identifier {
-            name: format!("{}_if", aggregate_name.name),
-            ..aggregate_name.clone()
-        };
-        for value in &pivot.values {
-            let mut args = aggregate_args.to_vec();
-            args.push(Self::expr_eq_from_col_and_value(
-                pivot.value_column.clone(),
-                value.clone(),
-            ));
-            let alias = Self::raw_string_from_literal_expr(value)
-                .ok_or_else(|| ErrorCode::SyntaxException("Pivot value should be literal"))?;
-            new_select_list.push(Self::target_func_from_name_args(
-                new_aggregate_name.clone(),
-                args,
-                Some(Identifier::from_name(stmt.span, &alias)),
-            ));
-        }
-
-        if let Some(ref mut new_stmt) = self.new_stmt {
-            new_stmt.select_list = new_select_list;
-            new_stmt.group_by = Some(new_group_by);
-        } else {
-            self.new_stmt = Some(SelectStmt {
-                select_list: new_select_list,
-                group_by: Some(new_group_by),
-                ..stmt.clone()
-            });
-        }
-        Ok(())
-    }
-
-    fn rewrite_unpivot(&mut self, stmt: &SelectStmt) -> Result<()> {
-        if stmt.from.len() != 1 || stmt.from[0].unpivot().is_none() {
-            return Ok(());
-        }
-        let unpivot = stmt.from[0].unpivot().unwrap();
-        let mut new_select_list = stmt.select_list.clone();
-        if let Some(star) = new_select_list.iter_mut().find(|target| target.is_star()) {
-            star.exclude(unpivot.names.clone());
-        };
-        new_select_list.push(Self::target_func_from_name_args(
-            Identifier::from_name(stmt.span, "unnest"),
-            vec![Self::expr_literal_array_from_vec_ident(
-                unpivot.names.clone(),
-            )],
-            Some(unpivot.column_name.clone()),
-        ));
-        new_select_list.push(Self::target_func_from_name_args(
-            Identifier::from_name(stmt.span, "unnest"),
-            vec![Self::expr_column_ref_array_from_vec_ident(
-                unpivot.names.clone(),
-            )],
-            Some(unpivot.value_column.clone()),
-        ));
-
-        if let Some(ref mut new_stmt) = self.new_stmt {
-            new_stmt.select_list = new_select_list;
-        } else {
-            self.new_stmt = Some(SelectStmt {
-                select_list: new_select_list,
-                ..stmt.clone()
-            });
-        };
-        Ok(())
-    }
-}
-
-#[derive(Visitor)]
-#[visitor(ColumnPosition(enter))]
-pub struct MaxColumnPosition {
-    pub max_pos: usize,
-}
-
-impl MaxColumnPosition {
-    pub fn new() -> Self {
-        Self { max_pos: 0 }
-    }
-}
-
-impl MaxColumnPosition {
-    fn enter_column_position(&mut self, pos: &ColumnPosition) {
-        if pos.pos > self.max_pos {
-            self.max_pos = pos.pos;
-        }
-    }
-}
diff --git a/src/query/sql/src/planner/binder/sort.rs b/src/query/sql/src/planner/binder/sort.rs
index 961da1ae03bbb..6f4c8c7368a66 100644
--- a/src/query/sql/src/planner/binder/sort.rs
+++ b/src/query/sql/src/planner/binder/sort.rs
@@ -251,62 +251,6 @@ impl Binder {
         Ok(new_expr)
     }
 
-    #[async_backtrace::framed]
-    pub(crate) async fn bind_order_by_for_set_operation(
-        &mut self,
-        bind_context: &mut BindContext,
-        child: SExpr,
-        order_by: &[OrderByExpr],
-    ) -> Result<SExpr> {
-        let mut scalar_binder = ScalarBinder::new(
-            bind_context,
-            self.ctx.clone(),
-            &self.name_resolution_ctx,
-            self.metadata.clone(),
-            &[],
-            self.m_cte_bound_ctx.clone(),
-            self.ctes_map.clone(),
-        );
-        let mut order_by_items = Vec::with_capacity(order_by.len());
-        for order in order_by.iter() {
-            match order.expr {
-                Expr::ColumnRef { .. } => {
-                    let scalar = scalar_binder.bind(&order.expr).await?.0;
-                    match scalar {
-                        ScalarExpr::BoundColumnRef(BoundColumnRef { column, .. }) => {
-                            let order_by_item = SortItem {
-                                index: column.index,
-                                asc: order.asc.unwrap_or(true),
-                                nulls_first: order.nulls_first.unwrap_or(false),
-                            };
-                            order_by_items.push(order_by_item);
-                        }
-                        _ => {
-                            return Err(ErrorCode::Internal("scalar should be BoundColumnRef")
-                                .set_span(order.expr.span()));
-                        }
-                    }
-                }
-                _ => {
-                    return Err(
-                        ErrorCode::SemanticError("can only order by column".to_string())
-                            .set_span(order.expr.span()),
-                    );
-                }
-            }
-        }
-        let sort_plan = Sort {
-            items: order_by_items,
-            limit: None,
-            after_exchange: None,
-            pre_projection: None,
-        };
-        Ok(SExpr::create_unary(
-            Arc::new(sort_plan.into()),
-            Arc::new(child),
-        ))
-    }
-
     #[allow(clippy::only_used_in_recursion)]
     pub(crate) fn rewrite_scalar_with_replacement<F>(
         &self,
diff --git a/src/query/sql/src/planner/binder/table.rs b/src/query/sql/src/planner/binder/table.rs
index e2543361d63a7..4824a78ab563e 100644
--- a/src/query/sql/src/planner/binder/table.rs
+++ b/src/query/sql/src/planner/binder/table.rs
@@ -13,35 +13,17 @@
 // limitations under the License.
 
 use std::collections::BTreeMap;
-use std::collections::HashMap;
 use std::default::Default;
-use std::str::FromStr;
 use std::sync::Arc;
 
-use async_recursion::async_recursion;
 use chrono::TimeZone;
 use chrono::Utc;
 use dashmap::DashMap;
-use databend_common_ast::ast::Connection;
-use databend_common_ast::ast::Expr;
-use databend_common_ast::ast::FileLocation;
-use databend_common_ast::ast::FunctionCall as ASTFunctionCall;
-use databend_common_ast::ast::Identifier;
 use databend_common_ast::ast::Indirection;
-use databend_common_ast::ast::Join;
-use databend_common_ast::ast::Literal;
-use databend_common_ast::ast::Query;
-use databend_common_ast::ast::SelectStageOptions;
-use databend_common_ast::ast::SelectStmt;
 use databend_common_ast::ast::SelectTarget;
-use databend_common_ast::ast::Statement;
 use databend_common_ast::ast::TableAlias;
-use databend_common_ast::ast::TableReference;
 use databend_common_ast::ast::TemporalClause;
 use databend_common_ast::ast::TimeTravelPoint;
-use databend_common_ast::ast::UriLocation;
-use databend_common_ast::parser::parse_sql;
-use databend_common_ast::parser::tokenize_sql;
 use databend_common_ast::Span;
 use databend_common_catalog::catalog_kind::CATALOG_DEFAULT;
 use databend_common_catalog::plan::ParquetReadOptions;
@@ -49,50 +31,34 @@ use databend_common_catalog::plan::StageTableInfo;
 use databend_common_catalog::table::NavigationPoint;
 use databend_common_catalog::table::Table;
 use databend_common_catalog::table::TimeNavigation;
-use databend_common_catalog::table_args::TableArgs;
 use databend_common_catalog::table_context::TableContext;
-use databend_common_catalog::table_function::TableFunction;
 use databend_common_exception::ErrorCode;
 use databend_common_exception::Result;
 use databend_common_expression::is_stream_column;
 use databend_common_expression::type_check::check_number;
 use databend_common_expression::types::DataType;
-use databend_common_expression::types::NumberScalar;
 use databend_common_expression::AbortChecker;
 use databend_common_expression::ConstantFolder;
 use databend_common_expression::DataField;
 use databend_common_expression::FunctionContext;
-use databend_common_expression::FunctionKind;
-use databend_common_expression::Scalar;
 use databend_common_expression::TableDataType;
 use databend_common_expression::TableField;
 use databend_common_expression::TableSchema;
 use databend_common_functions::BUILTIN_FUNCTIONS;
 use databend_common_meta_app::principal::FileFormatParams;
-use databend_common_meta_app::principal::StageFileFormatType;
 use databend_common_meta_app::principal::StageInfo;
 use databend_common_meta_app::schema::IndexMeta;
 use databend_common_meta_app::schema::ListIndexesReq;
 use databend_common_meta_app::tenant::Tenant;
 use databend_common_meta_types::MetaId;
-use databend_common_storage::DataOperator;
 use databend_common_storage::StageFileInfo;
 use databend_common_storage::StageFilesInfo;
 use databend_common_storages_parquet::ParquetRSTable;
-use databend_common_storages_result_cache::ResultCacheMetaManager;
-use databend_common_storages_result_cache::ResultCacheReader;
-use databend_common_storages_result_cache::ResultScan;
 use databend_common_storages_stage::StageTable;
-use databend_common_storages_view::view_table::QUERY;
-use databend_common_users::UserApiProvider;
-use databend_storages_common_table_meta::table::get_change_type;
 use databend_storages_common_table_meta::table::ChangeType;
 use log::info;
 use parking_lot::RwLock;
 
-use crate::binder::copy_into_table::resolve_file_location;
-use crate::binder::scalar::ScalarBinder;
-use crate::binder::table_args::bind_table_args;
 use crate::binder::Binder;
 use crate::binder::ColumnBindingBuilder;
 use crate::binder::CteInfo;
@@ -104,21 +70,16 @@ use crate::planner::semantic::normalize_identifier;
 use crate::planner::semantic::TypeChecker;
 use crate::plans::CteScan;
 use crate::plans::DummyTableScan;
-use crate::plans::EvalScalar;
-use crate::plans::FunctionCall;
-use crate::plans::RelOperator;
-use crate::plans::ScalarItem;
 use crate::plans::Scan;
 use crate::plans::Statistics;
 use crate::BaseTableColumn;
 use crate::BindContext;
 use crate::ColumnEntry;
 use crate::IndexType;
-use crate::ScalarExpr;
 
 impl Binder {
     #[async_backtrace::framed]
-    pub async fn bind_one_table(
+    pub async fn bind_dummy_table(
         &mut self,
         bind_context: &BindContext,
         select_list: &Vec<SelectTarget>,
@@ -146,776 +107,6 @@ impl Binder {
         ))
     }
 
-    fn check_view_dep(bind_context: &BindContext, database: &str, view_name: &str) -> Result<()> {
-        match &bind_context.parent {
-            Some(parent) => match &parent.view_info {
-                Some((db, v)) => {
-                    if db == database && v == view_name {
-                        Err(ErrorCode::Internal(format!(
-                            "View dependency loop detected (view: {}.{})",
-                            database, view_name
-                        )))
-                    } else {
-                        Self::check_view_dep(parent, database, view_name)
-                    }
-                }
-                _ => Ok(()),
-            },
-            _ => Ok(()),
-        }
-    }
-
-    /// Bind a base table.
-    /// A base table is a table that is not a view or CTE.
-    #[allow(clippy::too_many_arguments)]
-    #[async_backtrace::framed]
-    async fn bind_table(
-        &mut self,
-        bind_context: &mut BindContext,
-        span: &Span,
-        catalog: &Option<Identifier>,
-        database: &Option<Identifier>,
-        table: &Identifier,
-        alias: &Option<TableAlias>,
-        temporal: &Option<TemporalClause>,
-        consume: bool,
-    ) -> Result<(SExpr, BindContext)> {
-        let (catalog, database, table_name) =
-            self.normalize_object_identifier_triple(catalog, database, table);
-        let table_alias_name = if let Some(table_alias) = alias {
-            Some(normalize_identifier(&table_alias.name, &self.name_resolution_ctx).name)
-        } else {
-            None
-        };
-        let mut bind_cte = true;
-        if let Some(cte_name) = &bind_context.cte_name {
-            // If table name equals to cte name, then skip bind cte and find table from catalog
-            // Or will dead loop and stack overflow
-            if cte_name == &table_name {
-                bind_cte = false;
-            }
-        }
-        // Check and bind common table expression
-        let ctes_map = self.ctes_map.clone();
-        if let Some(cte_info) = ctes_map.get(&table_name) {
-            if bind_cte {
-                return if !cte_info.materialized {
-                    self.bind_cte(*span, bind_context, &table_name, alias, cte_info)
-                        .await
-                } else {
-                    self.bind_m_cte(bind_context, cte_info, &table_name, alias, span)
-                        .await
-                };
-            }
-        }
-
-        let tenant = self.ctx.get_tenant();
-
-        let navigation = self.resolve_temporal_clause(bind_context, temporal).await?;
-
-        // Resolve table with catalog
-        let table_meta = match self
-            .resolve_data_source(
-                tenant.tenant_name(),
-                catalog.as_str(),
-                database.as_str(),
-                table_name.as_str(),
-                navigation.as_ref(),
-                self.ctx.clone().get_abort_checker(),
-            )
-            .await
-        {
-            Ok(table) => table,
-            Err(e) => {
-                let mut parent = bind_context.parent.as_mut();
-                loop {
-                    if parent.is_none() {
-                        break;
-                    }
-                    let bind_context = parent.unwrap().as_mut();
-                    let ctes_map = self.ctes_map.clone();
-                    if let Some(cte_info) = ctes_map.get(&table_name) {
-                        return if !cte_info.materialized {
-                            self.bind_cte(*span, bind_context, &table_name, alias, cte_info)
-                                .await
-                        } else {
-                            self.bind_m_cte(bind_context, cte_info, &table_name, alias, span)
-                                .await
-                        };
-                    }
-                    parent = bind_context.parent.as_mut();
-                }
-                if e.code() == ErrorCode::UNKNOWN_DATABASE {
-                    return Err(ErrorCode::UnknownDatabase(format!(
-                        "Unknown database `{}` in catalog '{catalog}'",
-                        database
-                    ))
-                    .set_span(*span));
-                }
-                if e.code() == ErrorCode::UNKNOWN_TABLE {
-                    return Err(ErrorCode::UnknownTable(format!(
-                        "Unknown table `{database}`.`{table_name}` in catalog '{catalog}'"
-                    ))
-                    .set_span(*span));
-                }
-                return Err(e);
-            }
-        };
-
-        if consume && table_meta.engine() != "STREAM" {
-            return Err(ErrorCode::StorageUnsupported(
-                "WITH CONSUME only support in STREAM",
-            ));
-        }
-
-        if navigation.is_some_and(|n| matches!(n, TimeNavigation::Changes { .. }))
-            || table_meta.engine() == "STREAM"
-        {
-            let change_type = get_change_type(&table_alias_name);
-            if change_type.is_some() {
-                let table_index = self.metadata.write().add_table(
-                    catalog,
-                    database.clone(),
-                    table_meta,
-                    table_alias_name,
-                    bind_context.view_info.is_some(),
-                    bind_context.planning_agg_index,
-                    false,
-                    consume,
-                );
-                let (s_expr, mut bind_context) = self
-                    .bind_base_table(bind_context, database.as_str(), table_index, change_type)
-                    .await?;
-
-                if let Some(alias) = alias {
-                    bind_context.apply_table_alias(alias, &self.name_resolution_ctx)?;
-                }
-                return Ok((s_expr, bind_context));
-            }
-
-            let query = table_meta
-                .generage_changes_query(
-                    self.ctx.clone(),
-                    database.as_str(),
-                    table_name.as_str(),
-                    consume,
-                )
-                .await?;
-
-            let mut new_bind_context = BindContext::with_parent(Box::new(bind_context.clone()));
-            let tokens = tokenize_sql(query.as_str())?;
-            let (stmt, _) = parse_sql(&tokens, self.dialect)?;
-            let Statement::Query(query) = &stmt else {
-                unreachable!()
-            };
-            let (s_expr, mut new_bind_context) =
-                self.bind_query(&mut new_bind_context, query).await?;
-
-            let cols = table_meta
-                .schema()
-                .fields()
-                .iter()
-                .map(|f| f.name().clone())
-                .collect::<Vec<_>>();
-            for (index, column_name) in cols.iter().enumerate() {
-                new_bind_context.columns[index].column_name = column_name.clone();
-            }
-
-            if let Some(alias) = alias {
-                new_bind_context.apply_table_alias(alias, &self.name_resolution_ctx)?;
-            } else {
-                for column in new_bind_context.columns.iter_mut() {
-                    column.database_name = None;
-                    column.table_name = Some(table_name.clone());
-                }
-            }
-
-            new_bind_context.parent = Some(Box::new(bind_context.clone()));
-            return Ok((s_expr, new_bind_context));
-        }
-
-        match table_meta.engine() {
-            "VIEW" => {
-                // TODO(leiysky): this check is error-prone,
-                // we should find a better way to do this.
-                Self::check_view_dep(bind_context, &database, &table_name)?;
-                let query = table_meta
-                    .options()
-                    .get(QUERY)
-                    .ok_or_else(|| ErrorCode::Internal("Invalid VIEW object"))?;
-                let tokens = tokenize_sql(query.as_str())?;
-                let (stmt, _) = parse_sql(&tokens, self.dialect)?;
-                // For view, we need use a new context to bind it.
-                let mut new_bind_context = BindContext::with_parent(Box::new(bind_context.clone()));
-                new_bind_context.view_info = Some((database.clone(), table_name));
-                if let Statement::Query(query) = &stmt {
-                    self.metadata.write().add_table(
-                        catalog,
-                        database.clone(),
-                        table_meta,
-                        table_alias_name,
-                        false,
-                        false,
-                        false,
-                        false,
-                    );
-                    let (s_expr, mut new_bind_context) =
-                        self.bind_query(&mut new_bind_context, query).await?;
-                    if let Some(alias) = alias {
-                        // view maybe has alias, e.g. select v1.col1 from v as v1;
-                        new_bind_context.apply_table_alias(alias, &self.name_resolution_ctx)?;
-                    } else {
-                        // e.g. select v0.c0 from v0;
-                        for column in new_bind_context.columns.iter_mut() {
-                            column.database_name = None;
-                            column.table_name =
-                                Some(normalize_identifier(table, &self.name_resolution_ctx).name);
-                        }
-                    }
-                    new_bind_context.parent = Some(Box::new(bind_context.clone()));
-                    Ok((s_expr, new_bind_context))
-                } else {
-                    Err(
-                        ErrorCode::Internal(format!("Invalid VIEW object: {}", table_meta.name()))
-                            .set_span(*span),
-                    )
-                }
-            }
-            _ => {
-                let table_index = self.metadata.write().add_table(
-                    catalog,
-                    database.clone(),
-                    table_meta,
-                    table_alias_name,
-                    bind_context.view_info.is_some(),
-                    bind_context.planning_agg_index,
-                    false,
-                    false,
-                );
-
-                let (s_expr, mut bind_context) = self
-                    .bind_base_table(bind_context, database.as_str(), table_index, None)
-                    .await?;
-                if let Some(alias) = alias {
-                    bind_context.apply_table_alias(alias, &self.name_resolution_ctx)?;
-                }
-                Ok((s_expr, bind_context))
-            }
-        }
-    }
-
-    /// Extract the srf inner tuple fields as columns.
-    #[async_backtrace::framed]
-    async fn extract_srf_table_function_columns(
-        &mut self,
-        bind_context: &mut BindContext,
-        span: &Span,
-        func_name: &Identifier,
-        srf_expr: SExpr,
-        alias: &Option<TableAlias>,
-    ) -> Result<(SExpr, BindContext)> {
-        let fields = if func_name.name.eq_ignore_ascii_case("flatten") {
-            Some(vec![
-                "seq".to_string(),
-                "key".to_string(),
-                "path".to_string(),
-                "index".to_string(),
-                "value".to_string(),
-                "this".to_string(),
-            ])
-        } else if func_name.name.eq_ignore_ascii_case("json_each") {
-            Some(vec!["key".to_string(), "value".to_string()])
-        } else {
-            None
-        };
-
-        if let Some(fields) = fields {
-            if let RelOperator::EvalScalar(plan) = (*srf_expr.plan).clone() {
-                if plan.items.len() != 1 {
-                    return Err(ErrorCode::Internal(format!(
-                        "Invalid table function subquery EvalScalar items, expect 1, but got {}",
-                        plan.items.len()
-                    )));
-                }
-                // Delete srf result tuple column, extract tuple inner columns instead
-                let _ = bind_context.columns.pop();
-                let scalar = &plan.items[0].scalar;
-
-                // Add tuple inner columns
-                let mut items = Vec::with_capacity(fields.len());
-                for (i, field) in fields.into_iter().enumerate() {
-                    let field_expr = ScalarExpr::FunctionCall(FunctionCall {
-                        span: *span,
-                        func_name: "get".to_string(),
-                        params: vec![Scalar::Number(NumberScalar::Int64((i + 1) as i64))],
-                        arguments: vec![scalar.clone()],
-                    });
-                    let data_type = field_expr.data_type()?;
-                    let index = self.metadata.write().add_derived_column(
-                        field.clone(),
-                        data_type.clone(),
-                        Some(field_expr.clone()),
-                    );
-
-                    let column_binding = ColumnBindingBuilder::new(
-                        field,
-                        index,
-                        Box::new(data_type),
-                        Visibility::Visible,
-                    )
-                    .build();
-                    bind_context.add_column_binding(column_binding);
-
-                    items.push(ScalarItem {
-                        scalar: field_expr,
-                        index,
-                    });
-                }
-                let eval_scalar = EvalScalar { items };
-                let new_expr =
-                    SExpr::create_unary(Arc::new(eval_scalar.into()), srf_expr.children[0].clone());
-
-                if let Some(alias) = alias {
-                    bind_context.apply_table_alias(alias, &self.name_resolution_ctx)?;
-                }
-                return Ok((new_expr, bind_context.clone()));
-            } else {
-                return Err(ErrorCode::Internal(
-                    "Invalid subquery in table function: Table functions do not support this type of subquery.",
-                ));
-            }
-        }
-        // Set name for srf result column
-        bind_context.columns[0].column_name = "value".to_string();
-        if let Some(alias) = alias {
-            bind_context.apply_table_alias(alias, &self.name_resolution_ctx)?;
-        }
-        Ok((srf_expr, bind_context.clone()))
-    }
-
-    /// Bind a lateral table function.
-    #[async_backtrace::framed]
-    async fn bind_lateral_table_function(
-        &mut self,
-        parent_context: &mut BindContext,
-        child: SExpr,
-        table_ref: &TableReference,
-    ) -> Result<(SExpr, BindContext)> {
-        match table_ref {
-            TableReference::TableFunction {
-                span,
-                name,
-                params,
-                named_params,
-                alias,
-                ..
-            } => {
-                let mut bind_context = BindContext::with_parent(Box::new(parent_context.clone()));
-                let func_name = normalize_identifier(name, &self.name_resolution_ctx);
-
-                if BUILTIN_FUNCTIONS
-                    .get_property(&func_name.name)
-                    .map(|p| p.kind == FunctionKind::SRF)
-                    .unwrap_or(false)
-                {
-                    let args = parse_table_function_args(span, &func_name, params, named_params)?;
-
-                    // convert lateral join table function to srf function
-                    let srf = Expr::FunctionCall {
-                        span: *span,
-                        func: ASTFunctionCall {
-                            distinct: false,
-                            name: func_name.clone(),
-                            args,
-                            params: vec![],
-                            window: None,
-                            lambda: None,
-                        },
-                    };
-                    let srfs = vec![srf.clone()];
-                    let srf_expr = self
-                        .bind_project_set(&mut bind_context, &srfs, child)
-                        .await?;
-
-                    if let Some((_, srf_result)) = bind_context.srfs.remove(&srf.to_string()) {
-                        let column_binding =
-                            if let ScalarExpr::BoundColumnRef(column_ref) = &srf_result {
-                                column_ref.column.clone()
-                            } else {
-                                // Add result column to metadata
-                                let data_type = srf_result.data_type()?;
-                                let index = self.metadata.write().add_derived_column(
-                                    srf.to_string(),
-                                    data_type.clone(),
-                                    Some(srf_result.clone()),
-                                );
-                                ColumnBindingBuilder::new(
-                                    srf.to_string(),
-                                    index,
-                                    Box::new(data_type),
-                                    Visibility::Visible,
-                                )
-                                .build()
-                            };
-
-                        let eval_scalar = EvalScalar {
-                            items: vec![ScalarItem {
-                                scalar: srf_result,
-                                index: column_binding.index,
-                            }],
-                        };
-                        // Add srf result column
-                        bind_context.add_column_binding(column_binding);
-
-                        let flatten_expr =
-                            SExpr::create_unary(Arc::new(eval_scalar.into()), Arc::new(srf_expr));
-
-                        let (new_expr, mut bind_context) = self
-                            .extract_srf_table_function_columns(
-                                &mut bind_context,
-                                span,
-                                &func_name,
-                                flatten_expr,
-                                alias,
-                            )
-                            .await?;
-
-                        // add left table columns.
-                        let mut new_columns = parent_context.columns.clone();
-                        new_columns.extend_from_slice(&bind_context.columns);
-                        bind_context.columns = new_columns;
-
-                        return Ok((new_expr, bind_context));
-                    } else {
-                        return Err(ErrorCode::Internal("Failed to bind project_set for lateral join. This may indicate an issue with the SRF (Set Returning Function) processing or an internal logic error.")
-                            .set_span(*span));
-                    }
-                } else {
-                    return Err(ErrorCode::InvalidArgument(format!(
-                        "The function '{}' is not supported for lateral joins. Lateral joins currently support only Set Returning Functions (SRFs).",
-                        func_name
-                    ))
-                    .set_span(*span));
-                }
-            }
-            _ => unreachable!(),
-        }
-    }
-
-    /// Bind a table function.
-    #[async_backtrace::framed]
-    async fn bind_table_function(
-        &mut self,
-        bind_context: &mut BindContext,
-        span: &Span,
-        name: &Identifier,
-        params: &[Expr],
-        named_params: &[(Identifier, Expr)],
-        alias: &Option<TableAlias>,
-    ) -> Result<(SExpr, BindContext)> {
-        let func_name = normalize_identifier(name, &self.name_resolution_ctx);
-
-        if BUILTIN_FUNCTIONS
-            .get_property(&func_name.name)
-            .map(|p| p.kind == FunctionKind::SRF)
-            .unwrap_or(false)
-        {
-            // If it is a set-returning function, we bind it as a subquery.
-            let args = parse_table_function_args(span, &func_name, params, named_params)?;
-
-            let select_stmt = SelectStmt {
-                span: *span,
-                hints: None,
-                distinct: false,
-                top_n: None,
-                select_list: vec![SelectTarget::AliasedExpr {
-                    expr: Box::new(databend_common_ast::ast::Expr::FunctionCall {
-                        span: *span,
-                        func: ASTFunctionCall {
-                            distinct: false,
-                            name: databend_common_ast::ast::Identifier::from_name(
-                                *span,
-                                &func_name.name,
-                            ),
-                            params: vec![],
-                            args,
-                            window: None,
-                            lambda: None,
-                        },
-                    }),
-                    alias: None,
-                }],
-                from: vec![],
-                selection: None,
-                group_by: None,
-                having: None,
-                window_list: None,
-                qualify: None,
-            };
-            let (srf_expr, mut bind_context) = self
-                .bind_select_stmt(bind_context, &select_stmt, &[], 0)
-                .await?;
-
-            return self
-                .extract_srf_table_function_columns(
-                    &mut bind_context,
-                    span,
-                    &func_name,
-                    srf_expr,
-                    alias,
-                )
-                .await;
-        }
-
-        let mut scalar_binder = ScalarBinder::new(
-            bind_context,
-            self.ctx.clone(),
-            &self.name_resolution_ctx,
-            self.metadata.clone(),
-            &[],
-            self.m_cte_bound_ctx.clone(),
-            self.ctes_map.clone(),
-        );
-        let table_args = bind_table_args(&mut scalar_binder, params, named_params).await?;
-
-        if func_name.name.eq_ignore_ascii_case("result_scan") {
-            let query_id = parse_result_scan_args(&table_args)?;
-            if query_id.is_empty() {
-                return Err(ErrorCode::InvalidArgument("The `RESULT_SCAN` function requires a 'query_id' parameter. Please specify a valid query ID.")
-                    .set_span(*span));
-            }
-            let kv_store = UserApiProvider::instance().get_meta_store_client();
-            let meta_key = self.ctx.get_result_cache_key(&query_id);
-            if meta_key.is_none() {
-                return Err(ErrorCode::EmptyData(format!(
-                    "`RESULT_SCAN` failed: No cache key found in current session for query ID '{}'.",
-                    query_id
-                )).set_span(*span));
-            }
-            let result_cache_mgr = ResultCacheMetaManager::create(kv_store, 0);
-            let meta_key = meta_key.unwrap();
-            let (table_schema, block_raw_data) = match result_cache_mgr
-                .get(meta_key.clone())
-                .await?
-            {
-                Some(value) => {
-                    let op = DataOperator::instance().operator();
-                    ResultCacheReader::read_table_schema_and_data(op, &value.location).await?
-                }
-                None => {
-                    return Err(ErrorCode::EmptyData(format!(
-                        "`RESULT_SCAN` failed: Unable to fetch cached data for query ID '{}'. The data may have exceeded its TTL or been cleaned up. Cache key: '{}'",
-                        query_id, meta_key
-                    )).set_span(*span));
-                }
-            };
-            let table = ResultScan::try_create(table_schema, query_id, block_raw_data)?;
-
-            let table_alias_name = if let Some(table_alias) = alias {
-                Some(normalize_identifier(&table_alias.name, &self.name_resolution_ctx).name)
-            } else {
-                None
-            };
-
-            let table_index = self.metadata.write().add_table(
-                CATALOG_DEFAULT.to_string(),
-                "system".to_string(),
-                table.clone(),
-                table_alias_name,
-                false,
-                false,
-                false,
-                false,
-            );
-
-            let (s_expr, mut bind_context) = self
-                .bind_base_table(bind_context, "system", table_index, None)
-                .await?;
-            if let Some(alias) = alias {
-                bind_context.apply_table_alias(alias, &self.name_resolution_ctx)?;
-            }
-            Ok((s_expr, bind_context))
-        } else {
-            // Other table functions always reside is default catalog
-            let table_meta: Arc<dyn TableFunction> = self
-                .catalogs
-                .get_default_catalog(self.ctx.txn_mgr())?
-                .get_table_function(&func_name.name, table_args)?;
-            let table = table_meta.as_table();
-            let table_alias_name = if let Some(table_alias) = alias {
-                Some(normalize_identifier(&table_alias.name, &self.name_resolution_ctx).name)
-            } else {
-                None
-            };
-            let table_index = self.metadata.write().add_table(
-                CATALOG_DEFAULT.to_string(),
-                "system".to_string(),
-                table.clone(),
-                table_alias_name,
-                false,
-                false,
-                false,
-                false,
-            );
-
-            let (s_expr, mut bind_context) = self
-                .bind_base_table(bind_context, "system", table_index, None)
-                .await?;
-            if let Some(alias) = alias {
-                bind_context.apply_table_alias(alias, &self.name_resolution_ctx)?;
-            }
-            Ok((s_expr, bind_context))
-        }
-    }
-
-    /// Bind a subquery.
-    #[async_backtrace::framed]
-    async fn bind_subquery(
-        &mut self,
-        bind_context: &mut BindContext,
-        lateral: bool,
-        subquery: &Query,
-        alias: &Option<TableAlias>,
-    ) -> Result<(SExpr, BindContext)> {
-        // If the subquery is a lateral subquery, we need to let it see the columns
-        // from the previous queries.
-        let (result, mut result_bind_context) = if lateral {
-            let mut new_bind_context = BindContext::with_parent(Box::new(bind_context.clone()));
-            self.bind_query(&mut new_bind_context, subquery).await?
-        } else {
-            let mut new_bind_context = BindContext::with_parent(
-                bind_context
-                    .parent
-                    .clone()
-                    .unwrap_or_else(|| Box::new(BindContext::new())),
-            );
-            self.bind_query(&mut new_bind_context, subquery).await?
-        };
-
-        if let Some(alias) = alias {
-            result_bind_context.apply_table_alias(alias, &self.name_resolution_ctx)?;
-            // Reset column name as alias column name
-            for i in 0..alias.columns.len() {
-                let column = &result_bind_context.columns[i];
-                self.metadata
-                    .write()
-                    .change_derived_column_alias(column.index, column.column_name.clone());
-            }
-        }
-        Ok((result, result_bind_context))
-    }
-
-    /// Bind a location.
-    #[async_backtrace::framed]
-    async fn bind_location(
-        &mut self,
-        bind_context: &mut BindContext,
-        location: &FileLocation,
-        options: &SelectStageOptions,
-        alias: &Option<TableAlias>,
-    ) -> Result<(SExpr, BindContext)> {
-        let location = match location {
-            FileLocation::Uri(uri) => FileLocation::Uri(UriLocation {
-                connection: Connection::new(options.connection.clone()),
-                ..uri.clone()
-            }),
-            _ => location.clone(),
-        };
-        let (mut stage_info, path) = resolve_file_location(self.ctx.as_ref(), &location).await?;
-        if let Some(f) = &options.file_format {
-            stage_info.file_format_params = match StageFileFormatType::from_str(f) {
-                Ok(t) => FileFormatParams::default_by_type(t)?,
-                _ => self.ctx.get_file_format(f).await?,
-            }
-        }
-        let files_info = StageFilesInfo {
-            path,
-            pattern: options.pattern.clone(),
-            files: options.files.clone(),
-        };
-        let table_ctx = self.ctx.clone();
-        self.bind_stage_table(table_ctx, bind_context, stage_info, files_info, alias, None)
-            .await
-    }
-
-    #[async_recursion]
-    #[async_backtrace::framed]
-    pub(crate) async fn bind_single_table(
-        &mut self,
-        bind_context: &mut BindContext,
-        table_ref: &TableReference,
-    ) -> Result<(SExpr, BindContext)> {
-        match table_ref {
-            TableReference::Table {
-                span,
-                catalog,
-                database,
-                table,
-                alias,
-                temporal,
-                consume,
-                pivot: _,
-                unpivot: _,
-            } => {
-                self.bind_table(
-                    bind_context,
-                    span,
-                    catalog,
-                    database,
-                    table,
-                    alias,
-                    temporal,
-                    *consume,
-                )
-                .await
-            }
-            TableReference::TableFunction {
-                span,
-                name,
-                params,
-                named_params,
-                alias,
-                ..
-            } => {
-                self.bind_table_function(bind_context, span, name, params, named_params, alias)
-                    .await
-            }
-            TableReference::Subquery {
-                span: _,
-                lateral,
-                subquery,
-                alias,
-            } => {
-                self.bind_subquery(bind_context, *lateral, subquery, alias)
-                    .await
-            }
-            TableReference::Location {
-                span: _,
-                location,
-                options,
-                alias,
-            } => {
-                self.bind_location(bind_context, location, options, alias)
-                    .await
-            }
-            TableReference::Join { join, .. } => {
-                let (left_expr, left_bind_ctx) =
-                    self.bind_table_reference(bind_context, &join.left).await?;
-                let (right_expr, right_bind_ctx) =
-                    self.bind_table_reference(bind_context, &join.right).await?;
-                self.bind_join(
-                    bind_context,
-                    left_bind_ctx,
-                    right_bind_ctx,
-                    left_expr,
-                    right_expr,
-                    join,
-                )
-                .await
-            }
-        }
-    }
-
     #[async_backtrace::framed]
     pub(crate) async fn bind_stage_table(
         &mut self,
@@ -1043,92 +234,6 @@ impl Binder {
         Ok((s_expr, bind_context))
     }
 
-    #[async_backtrace::framed]
-    pub async fn bind_table_reference(
-        &mut self,
-        bind_context: &mut BindContext,
-        table_ref: &TableReference,
-    ) -> Result<(SExpr, BindContext)> {
-        let mut current_ref = table_ref;
-        let current_ctx = bind_context;
-
-        // Stack to keep track of the joins
-        let mut join_stack: Vec<&Join> = Vec::new();
-
-        // Traverse the table reference hierarchy to get to the innermost table
-        while let TableReference::Join { join, .. } = current_ref {
-            join_stack.push(join);
-
-            // Check whether the right-hand side is a Join or a TableReference
-            match &*join.right {
-                TableReference::Join { .. } => {
-                    // Traverse the right-hand side if the right-hand side is a Join
-                    current_ref = &join.right;
-                }
-                _ => {
-                    // Traverse the left-hand side if the right-hand side is a TableReference
-                    current_ref = &join.left;
-                }
-            }
-        }
-
-        // Bind the innermost table
-        // current_ref must be left table in its join
-        let (mut result_expr, mut result_ctx) =
-            self.bind_single_table(current_ctx, current_ref).await?;
-
-        for join in join_stack.iter().rev() {
-            match &*join.right {
-                TableReference::Join { .. } => {
-                    let (left_expr, left_ctx) =
-                        self.bind_single_table(&mut result_ctx, &join.left).await?;
-                    let (join_expr, ctx) = self
-                        .bind_join(
-                            current_ctx,
-                            left_ctx,
-                            result_ctx,
-                            left_expr,
-                            result_expr,
-                            join,
-                        )
-                        .await?;
-                    result_expr = join_expr;
-                    result_ctx = ctx;
-                }
-                _ => {
-                    if join.right.is_lateral_table_function() {
-                        let (expr, ctx) = self
-                            .bind_lateral_table_function(
-                                &mut result_ctx,
-                                result_expr.clone(),
-                                &join.right,
-                            )
-                            .await?;
-                        result_expr = expr;
-                        result_ctx = ctx;
-                    } else {
-                        let (right_expr, right_ctx) =
-                            self.bind_single_table(&mut result_ctx, &join.right).await?;
-                        let (join_expr, ctx) = self
-                            .bind_join(
-                                current_ctx,
-                                result_ctx,
-                                right_ctx,
-                                result_expr,
-                                right_expr,
-                                join,
-                            )
-                            .await?;
-                        result_expr = join_expr;
-                        result_ctx = ctx;
-                    }
-                }
-            }
-        }
-
-        Ok((result_expr, result_ctx))
-    }
-
     fn bind_cte_scan(&mut self, cte_info: &CteInfo) -> Result<SExpr> {
         let blocks = Arc::new(RwLock::new(vec![]));
         self.ctx
@@ -1513,79 +618,3 @@ impl Binder {
         Ok(index_metas)
     }
 }
-
-// copy from common-storages-fuse to avoid cyclic dependency.
-fn string_value(value: &Scalar) -> Result<String> {
-    match value {
-        Scalar::String(val) => Ok(val.clone()),
-        other => Err(ErrorCode::BadArguments(format!(
-            "Expected a string value, but found a '{}'.",
-            other
-        ))),
-    }
-}
-
-#[inline(always)]
-pub fn parse_result_scan_args(table_args: &TableArgs) -> Result<String> {
-    let args = table_args.expect_all_positioned("RESULT_SCAN", Some(1))?;
-    string_value(&args[0])
-}
-
-// parse flatten named params to arguments
-fn parse_table_function_args(
-    span: &Span,
-    func_name: &Identifier,
-    params: &[Expr],
-    named_params: &[(Identifier, Expr)],
-) -> Result<Vec<Expr>> {
-    if func_name.name.eq_ignore_ascii_case("flatten") {
-        // build flatten function arguments.
-        let mut named_args: HashMap<String, Expr> = named_params
-            .iter()
-            .map(|(name, value)| (name.name.to_lowercase(), value.clone()))
-            .collect::<HashMap<_, _>>();
-
-        let mut args = Vec::with_capacity(named_args.len() + params.len());
-        let names = vec!["input", "path", "outer", "recursive", "mode"];
-        for name in names {
-            if named_args.is_empty() {
-                break;
-            }
-            match named_args.remove(name) {
-                Some(val) => args.push(val),
-                None => args.push(Expr::Literal {
-                    span: None,
-                    value: Literal::Null,
-                }),
-            }
-        }
-        if !named_args.is_empty() {
-            let invalid_names = named_args.into_keys().collect::<Vec<String>>().join(", ");
-            return Err(ErrorCode::InvalidArgument(format!(
-                "Invalid named parameters for 'flatten': {}, valid parameters are: [input, path, outer, recursive, mode]",
-                invalid_names,
-            ))
-            .set_span(*span));
-        }
-
-        if !params.is_empty() {
-            args.extend(params.iter().cloned());
-        }
-        Ok(args)
-    } else {
-        if !named_params.is_empty() {
-            let invalid_names = named_params
-                .iter()
-                .map(|(name, _)| name.name.clone())
-                .collect::<Vec<String>>()
-                .join(", ");
-            return Err(ErrorCode::InvalidArgument(format!(
-                "Named parameters are not allowed for '{}'. Invalid parameters provided: {}.",
-                func_name.name, invalid_names
-            ))
-            .set_span(*span));
-        }
-
-        Ok(params.to_vec())
-    }
-}
diff --git a/src/query/sql/src/planner/binder/update.rs b/src/query/sql/src/planner/binder/update.rs
index f73678d58abc6..99d8422a302d7 100644
--- a/src/query/sql/src/planner/binder/update.rs
+++ b/src/query/sql/src/planner/binder/update.rs
@@ -52,9 +52,7 @@ impl Binder {
             ..
         } = stmt;
 
-        if let Some(with) = &with {
-            self.add_cte(with, bind_context)?;
-        }
+        self.init_cte(bind_context, with)?;
 
         let (catalog_name, database_name, table_name) = if let TableReference::Table {
             catalog,
@@ -79,7 +77,7 @@ impl Binder {
             ));
         };
 
-        let (table_expr, mut context) = self.bind_single_table(bind_context, table).await?;
+        let (table_expr, mut context) = self.bind_table_reference(bind_context, table).await?;
 
         let table = self
             .ctx
diff --git a/src/query/sql/src/planner/binder/values.rs b/src/query/sql/src/planner/binder/values.rs
deleted file mode 100644
index df8884e653e25..0000000000000
--- a/src/query/sql/src/planner/binder/values.rs
+++ /dev/null
@@ -1,224 +0,0 @@
-// Copyright 2021 Datafuse Labs
-//
-// Licensed under the Apache License, Version 2.0 (the "License");
-// you may not use this file except in compliance with the License.
-// You may obtain a copy of the License at
-//
-//     http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing, software
-// distributed under the License is distributed on an "AS IS" BASIS,
-// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// See the License for the specific language governing permissions and
-// limitations under the License.
-
-use std::collections::HashMap;
-use std::sync::Arc;
-
-use databend_common_ast::ast::Expr as AExpr;
-use databend_common_ast::Span;
-use databend_common_catalog::table_context::TableContext;
-use databend_common_exception::ErrorCode;
-use databend_common_exception::Result;
-use databend_common_expression::type_check::common_super_type;
-use databend_common_expression::types::DataType;
-use databend_common_expression::ColumnBuilder;
-use databend_common_expression::DataBlock;
-use databend_common_expression::DataField;
-use databend_common_expression::DataSchema;
-use databend_common_expression::DataSchemaRefExt;
-use databend_common_expression::Evaluator;
-use databend_common_functions::BUILTIN_FUNCTIONS;
-use indexmap::IndexMap;
-
-use crate::binder::wrap_cast;
-use crate::optimizer::ColumnSet;
-use crate::optimizer::SExpr;
-use crate::plans::BoundColumnRef;
-use crate::plans::ConstantTableScan;
-use crate::BindContext;
-use crate::Binder;
-use crate::ColumnBindingBuilder;
-use crate::MetadataRef;
-use crate::NameResolutionContext;
-use crate::ScalarBinder;
-use crate::ScalarExpr;
-use crate::Visibility;
-
-impl Binder {
-    #[async_backtrace::framed]
-    pub(crate) async fn bind_values(
-        &mut self,
-        bind_context: &mut BindContext,
-        span: Span,
-        values: &[Vec<AExpr>],
-    ) -> Result<(SExpr, BindContext)> {
-        bind_values(
-            self.ctx.clone(),
-            &self.name_resolution_ctx,
-            self.metadata.clone(),
-            bind_context,
-            span,
-            values,
-        )
-        .await
-    }
-}
-
-pub async fn bind_values(
-    ctx: Arc<dyn TableContext>,
-    name_resolution_ctx: &NameResolutionContext,
-    metadata: MetadataRef,
-    bind_context: &mut BindContext,
-    span: Span,
-    values: &[Vec<AExpr>],
-) -> Result<(SExpr, BindContext)> {
-    if values.is_empty() {
-        return Err(ErrorCode::SemanticError(
-            "Values lists must have at least one row".to_string(),
-        )
-        .set_span(span));
-    }
-    let same_length = values.windows(2).all(|v| v[0].len() == v[1].len());
-    if !same_length {
-        return Err(ErrorCode::SemanticError(
-            "Values lists must all be the same length".to_string(),
-        )
-        .set_span(span));
-    }
-
-    let num_rows = values.len();
-    let num_cols = values[0].len();
-
-    // assigns default column names col0, col1, etc.
-    let names = (0..num_cols)
-        .map(|i| format!("col{}", i))
-        .collect::<Vec<_>>();
-
-    let mut scalar_binder = ScalarBinder::new(
-        bind_context,
-        ctx.clone(),
-        name_resolution_ctx,
-        metadata.clone(),
-        &[],
-        HashMap::new(),
-        Box::new(IndexMap::new()),
-    );
-
-    let mut col_scalars = vec![Vec::with_capacity(values.len()); num_cols];
-    let mut common_types: Vec<Option<DataType>> = vec![None; num_cols];
-
-    for row_values in values.iter() {
-        for (i, value) in row_values.iter().enumerate() {
-            let (scalar, data_type) = scalar_binder.bind(value).await?;
-            col_scalars[i].push((scalar, data_type.clone()));
-
-            // Get the common data type for each columns.
-            match &common_types[i] {
-                Some(common_type) => {
-                    if common_type != &data_type {
-                        let new_common_type = common_super_type(
-                            common_type.clone(),
-                            data_type.clone(),
-                            &BUILTIN_FUNCTIONS.default_cast_rules,
-                        );
-                        if new_common_type.is_none() {
-                            return Err(ErrorCode::SemanticError(format!(
-                                "{} and {} don't have common data type",
-                                common_type, data_type
-                            ))
-                            .set_span(span));
-                        }
-                        common_types[i] = new_common_type;
-                    }
-                }
-                None => {
-                    common_types[i] = Some(data_type);
-                }
-            }
-        }
-    }
-
-    let mut value_fields = Vec::with_capacity(names.len());
-    for (name, common_type) in names.into_iter().zip(common_types.into_iter()) {
-        let value_field = DataField::new(&name, common_type.unwrap());
-        value_fields.push(value_field);
-    }
-    let value_schema = DataSchema::new(value_fields);
-
-    let input = DataBlock::empty();
-    let func_ctx = ctx.get_function_context()?;
-    let evaluator = Evaluator::new(&input, &func_ctx, &BUILTIN_FUNCTIONS);
-
-    // use values to build columns
-    let mut value_columns = Vec::with_capacity(col_scalars.len());
-    for (scalars, value_field) in col_scalars.iter().zip(value_schema.fields().iter()) {
-        let mut builder = ColumnBuilder::with_capacity(value_field.data_type(), col_scalars.len());
-        for (scalar, value_type) in scalars {
-            let scalar = if value_type != value_field.data_type() {
-                wrap_cast(scalar, value_field.data_type())
-            } else {
-                scalar.clone()
-            };
-            let expr = scalar
-                .as_expr()?
-                .project_column_ref(|col| value_schema.index_of(&col.index.to_string()).unwrap());
-            let result = evaluator.run(&expr)?;
-
-            match result.as_scalar() {
-                Some(val) => {
-                    builder.push(val.as_ref());
-                }
-                None => {
-                    return Err(ErrorCode::SemanticError(format!(
-                        "Value must be a scalar, but get {}",
-                        result
-                    ))
-                    .set_span(span));
-                }
-            }
-        }
-        value_columns.push(builder.build());
-    }
-
-    // add column bindings
-    let mut columns = ColumnSet::new();
-    let mut fields = Vec::with_capacity(values.len());
-    for value_field in value_schema.fields() {
-        let index = metadata.read().columns().len();
-        columns.insert(index);
-
-        let column_binding = ColumnBindingBuilder::new(
-            value_field.name().clone(),
-            index,
-            Box::new(value_field.data_type().clone()),
-            Visibility::Visible,
-        )
-        .build();
-        let _ = metadata.write().add_derived_column(
-            value_field.name().clone(),
-            value_field.data_type().clone(),
-            Some(ScalarExpr::BoundColumnRef(BoundColumnRef {
-                span,
-                column: column_binding.clone(),
-            })),
-        );
-        bind_context.add_column_binding(column_binding);
-
-        let field = DataField::new(&index.to_string(), value_field.data_type().clone());
-        fields.push(field);
-    }
-    let schema = DataSchemaRefExt::create(fields);
-
-    let s_expr = SExpr::create_leaf(Arc::new(
-        ConstantTableScan {
-            values: value_columns,
-            num_rows,
-            schema,
-            columns,
-        }
-        .into(),
-    ));
-
-    Ok((s_expr, bind_context.clone()))
-}
diff --git a/src/query/sql/src/planner/dataframe.rs b/src/query/sql/src/planner/dataframe.rs
index 24fb213e41cfa..44d85c98b3041 100644
--- a/src/query/sql/src/planner/dataframe.rs
+++ b/src/query/sql/src/planner/dataframe.rs
@@ -445,28 +445,34 @@ impl Dataframe {
     }
 
     pub async fn except(mut self, dataframe: Dataframe) -> Result<Self> {
-        let (s_expr, bind_context) = self.binder.bind_except(
-            None,
-            None,
-            self.bind_context,
-            dataframe.bind_context,
-            self.s_expr,
-            dataframe.s_expr,
-        )?;
+        let (s_expr, bind_context) = self
+            .binder
+            .bind_except(
+                None,
+                None,
+                self.bind_context,
+                dataframe.bind_context,
+                self.s_expr,
+                dataframe.s_expr,
+            )
+            .await?;
         self.s_expr = s_expr;
         self.bind_context = bind_context;
         Ok(self)
     }
 
     pub async fn intersect(mut self, dataframe: Dataframe) -> Result<Self> {
-        let (s_expr, bind_context) = self.binder.bind_intersect(
-            None,
-            None,
-            self.bind_context,
-            dataframe.bind_context,
-            self.s_expr,
-            dataframe.s_expr,
-        )?;
+        let (s_expr, bind_context) = self
+            .binder
+            .bind_intersect(
+                None,
+                None,
+                self.bind_context,
+                dataframe.bind_context,
+                self.s_expr,
+                dataframe.s_expr,
+            )
+            .await?;
         self.s_expr = s_expr;
         self.bind_context = bind_context;
         Ok(self)
diff --git a/src/query/sql/src/planner/optimizer/decorrelate/decorrelate.rs b/src/query/sql/src/planner/optimizer/decorrelate/decorrelate.rs
index a5975c420281e..d5e20d78b8d53 100644
--- a/src/query/sql/src/planner/optimizer/decorrelate/decorrelate.rs
+++ b/src/query/sql/src/planner/optimizer/decorrelate/decorrelate.rs
@@ -58,7 +58,7 @@ pub fn decorrelate_subquery(
     metadata: MetadataRef,
     s_expr: SExpr,
 ) -> Result<SExpr> {
-    let mut rewriter = SubqueryRewriter::new(ctx, metadata);
+    let mut rewriter = SubqueryRewriter::new(ctx, metadata, None);
     rewriter.rewrite(&s_expr)
 }
 
@@ -201,6 +201,7 @@ impl SubqueryRewriter {
             need_hold_hash_table: false,
             is_lateral: false,
             single_to_inner: None,
+            build_side_cache_info: None,
         };
 
         // Rewrite plan to semi-join.
@@ -291,6 +292,7 @@ impl SubqueryRewriter {
                     need_hold_hash_table: false,
                     is_lateral: false,
                     single_to_inner: None,
+                    build_side_cache_info: None,
                 };
                 let s_expr = SExpr::create_binary(
                     Arc::new(join_plan.into()),
@@ -341,6 +343,7 @@ impl SubqueryRewriter {
                     need_hold_hash_table: false,
                     is_lateral: false,
                     single_to_inner: None,
+                    build_side_cache_info: None,
                 };
                 let s_expr = SExpr::create_binary(
                     Arc::new(join_plan.into()),
@@ -406,6 +409,7 @@ impl SubqueryRewriter {
                     need_hold_hash_table: false,
                     is_lateral: false,
                     single_to_inner: None,
+                    build_side_cache_info: None,
                 }
                 .into();
                 Ok((
@@ -428,6 +432,8 @@ impl SubqueryRewriter {
         left_conditions: &mut Vec<ScalarExpr>,
         right_conditions: &mut Vec<ScalarExpr>,
     ) -> Result<()> {
+        let mut correlated_columns = correlated_columns.clone().into_iter().collect::<Vec<_>>();
+        correlated_columns.sort();
         for correlated_column in correlated_columns.iter() {
             let metadata = self.metadata.read();
             let column_entry = metadata.column(*correlated_column);
diff --git a/src/query/sql/src/planner/optimizer/decorrelate/flatten_plan.rs b/src/query/sql/src/planner/optimizer/decorrelate/flatten_plan.rs
index f57de1eccad30..f0f2b7b8531b5 100644
--- a/src/query/sql/src/planner/optimizer/decorrelate/flatten_plan.rs
+++ b/src/query/sql/src/planner/optimizer/decorrelate/flatten_plan.rs
@@ -32,6 +32,7 @@ use crate::plans::AggregateFunction;
 use crate::plans::AggregateMode;
 use crate::plans::BoundColumnRef;
 use crate::plans::EvalScalar;
+use crate::plans::ExpressionScan;
 use crate::plans::Filter;
 use crate::plans::Join;
 use crate::plans::JoinType;
@@ -168,6 +169,7 @@ impl SubqueryRewriter {
                 need_hold_hash_table: false,
                 is_lateral: false,
                 single_to_inner: None,
+                build_side_cache_info: None,
             }
             .into();
 
@@ -230,6 +232,10 @@ impl SubqueryRewriter {
                 self.flatten_window(plan, op, correlated_columns, flatten_info)
             }
 
+            RelOperator::ExpressionScan(scan) => {
+                self.flatten_expression_scan(plan, scan, correlated_columns)
+            }
+
             _ => Err(ErrorCode::Internal(
                 "Invalid plan type for flattening subquery",
             )),
@@ -501,6 +507,7 @@ impl SubqueryRewriter {
                     need_hold_hash_table: false,
                     is_lateral: false,
                     single_to_inner: None,
+                    build_side_cache_info: None,
                 }
                 .into(),
             ),
@@ -764,4 +771,21 @@ impl SubqueryRewriter {
             Arc::new(right_flatten_plan),
         ))
     }
+
+    fn flatten_expression_scan(
+        &mut self,
+        plan: &SExpr,
+        scan: &ExpressionScan,
+        correlated_columns: &ColumnSet,
+    ) -> Result<SExpr> {
+        let binder = self.binder.as_ref().unwrap();
+        for correlated_column in correlated_columns.iter() {
+            let derived_column_index = binder
+                .expression_scan_context
+                .get_derived_column(scan.expression_scan_index, *correlated_column);
+            self.derived_columns
+                .insert(*correlated_column, derived_column_index);
+        }
+        Ok(plan.clone())
+    }
 }
diff --git a/src/query/sql/src/planner/optimizer/decorrelate/subquery_rewriter.rs b/src/query/sql/src/planner/optimizer/decorrelate/subquery_rewriter.rs
index 7ecfe8214fcf0..900cabba170a9 100644
--- a/src/query/sql/src/planner/optimizer/decorrelate/subquery_rewriter.rs
+++ b/src/query/sql/src/planner/optimizer/decorrelate/subquery_rewriter.rs
@@ -51,6 +51,7 @@ use crate::plans::SubqueryType;
 use crate::plans::UDFCall;
 use crate::plans::UDFLambdaCall;
 use crate::plans::WindowFuncType;
+use crate::Binder;
 use crate::IndexType;
 use crate::MetadataRef;
 
@@ -71,14 +72,16 @@ pub struct SubqueryRewriter {
     pub(crate) ctx: Arc<dyn TableContext>,
     pub(crate) metadata: MetadataRef,
     pub(crate) derived_columns: HashMap<IndexType, IndexType>,
+    pub(crate) binder: Option<Binder>,
 }
 
 impl SubqueryRewriter {
-    pub fn new(ctx: Arc<dyn TableContext>, metadata: MetadataRef) -> Self {
+    pub fn new(ctx: Arc<dyn TableContext>, metadata: MetadataRef, binder: Option<Binder>) -> Self {
         Self {
             ctx,
             metadata,
             derived_columns: Default::default(),
+            binder,
         }
     }
 
@@ -180,6 +183,8 @@ impl SubqueryRewriter {
             | RelOperator::Scan(_)
             | RelOperator::CteScan(_)
             | RelOperator::ConstantTableScan(_)
+            | RelOperator::ExpressionScan(_)
+            | RelOperator::CacheScan(_)
             | RelOperator::AddRowNumber(_)
             | RelOperator::Exchange(_) => Ok(s_expr.clone()),
         }
@@ -522,6 +527,7 @@ impl SubqueryRewriter {
                     need_hold_hash_table: false,
                     is_lateral: false,
                     single_to_inner: None,
+                    build_side_cache_info: None,
                 }
                 .into();
                 Ok((
@@ -593,6 +599,7 @@ impl SubqueryRewriter {
                     need_hold_hash_table: false,
                     is_lateral: false,
                     single_to_inner: None,
+                    build_side_cache_info: None,
                 }
                 .into();
                 let s_expr = SExpr::create_binary(
@@ -624,6 +631,7 @@ impl SubqueryRewriter {
             need_hold_hash_table: false,
             is_lateral: false,
             single_to_inner: None,
+            build_side_cache_info: None,
         }
         .into();
 
diff --git a/src/query/sql/src/planner/optimizer/format.rs b/src/query/sql/src/planner/optimizer/format.rs
index a7ba3cc65b35b..5ec7d6bb9ec5e 100644
--- a/src/query/sql/src/planner/optimizer/format.rs
+++ b/src/query/sql/src/planner/optimizer/format.rs
@@ -68,6 +68,8 @@ pub fn display_rel_op(rel_op: &RelOperator) -> String {
         RelOperator::CteScan(_) => "CteScan".to_string(),
         RelOperator::MaterializedCte(_) => "MaterializedCte".to_string(),
         RelOperator::ConstantTableScan(_) => "ConstantTableScan".to_string(),
+        RelOperator::ExpressionScan(_) => "ExpressionScan".to_string(),
+        RelOperator::CacheScan(_) => "CacheScan".to_string(),
         RelOperator::AddRowNumber(_) => "AddRowNumber".to_string(),
         RelOperator::Udf(_) => "Udf".to_string(),
         RelOperator::AsyncFunction(_) => "AsyncFunction".to_string(),
diff --git a/src/query/sql/src/planner/optimizer/hyper_dp/dphyp.rs b/src/query/sql/src/planner/optimizer/hyper_dp/dphyp.rs
index 136621a0115fb..dc3a320375d27 100644
--- a/src/query/sql/src/planner/optimizer/hyper_dp/dphyp.rs
+++ b/src/query/sql/src/planner/optimizer/hyper_dp/dphyp.rs
@@ -192,7 +192,7 @@ impl DPhpy {
                     };
                     self.filters.insert(filter);
                 }
-                if !is_inner_join {
+                if !is_inner_join || op.build_side_cache_info.is_some() {
                     let new_s_expr = self.new_children(s_expr)?;
                     self.join_relations.push(JoinRelation::new(&new_s_expr));
                     Ok((Arc::new(new_s_expr), true))
@@ -260,6 +260,8 @@ impl DPhpy {
             }
             RelOperator::DummyTableScan(_)
             | RelOperator::ConstantTableScan(_)
+            | RelOperator::ExpressionScan(_)
+            | RelOperator::CacheScan(_)
             | RelOperator::CteScan(_)
             | RelOperator::AsyncFunction(_)
             | RelOperator::MaterializedCte(_) => Ok((Arc::new(s_expr.clone()), true)),
diff --git a/src/query/sql/src/planner/optimizer/hyper_dp/join_node.rs b/src/query/sql/src/planner/optimizer/hyper_dp/join_node.rs
index b42cea78ff6b3..232c284bcc16d 100644
--- a/src/query/sql/src/planner/optimizer/hyper_dp/join_node.rs
+++ b/src/query/sql/src/planner/optimizer/hyper_dp/join_node.rs
@@ -89,6 +89,7 @@ impl JoinNode {
             need_hold_hash_table: false,
             is_lateral: false,
             single_to_inner: None,
+            build_side_cache_info: None,
         });
         let children = self
             .children
diff --git a/src/query/sql/src/planner/optimizer/rule/rewrite/rule_commute_join.rs b/src/query/sql/src/planner/optimizer/rule/rewrite/rule_commute_join.rs
index 5b18bd3e37d05..8fb741e2e6df2 100644
--- a/src/query/sql/src/planner/optimizer/rule/rewrite/rule_commute_join.rs
+++ b/src/query/sql/src/planner/optimizer/rule/rewrite/rule_commute_join.rs
@@ -57,6 +57,11 @@ impl Rule for RuleCommuteJoin {
 
     fn apply(&self, s_expr: &SExpr, state: &mut TransformResult) -> Result<()> {
         let mut join: Join = s_expr.plan().clone().try_into()?;
+
+        if join.build_side_cache_info.is_some() {
+            return Ok(());
+        }
+
         let left_child = s_expr.child(0)?;
         let right_child = s_expr.child(1)?;
         let left_rel_expr = RelExpr::with_s_expr(left_child);
diff --git a/src/query/sql/src/planner/optimizer/rule/rewrite/rule_semi_to_inner_join.rs b/src/query/sql/src/planner/optimizer/rule/rewrite/rule_semi_to_inner_join.rs
index cd5d15a6f1e8a..5c52d0cceaf6b 100644
--- a/src/query/sql/src/planner/optimizer/rule/rewrite/rule_semi_to_inner_join.rs
+++ b/src/query/sql/src/planner/optimizer/rule/rewrite/rule_semi_to_inner_join.rs
@@ -126,6 +126,8 @@ fn find_group_by_keys(child: &SExpr, group_by_keys: &mut HashSet<IndexType>) ->
         | RelOperator::ProjectSet(_)
         | RelOperator::MaterializedCte(_)
         | RelOperator::ConstantTableScan(_)
+        | RelOperator::ExpressionScan(_)
+        | RelOperator::CacheScan(_)
         | RelOperator::Udf(_)
         | RelOperator::Scan(_)
         | RelOperator::CteScan(_)
diff --git a/src/query/sql/src/planner/optimizer/s_expr.rs b/src/query/sql/src/planner/optimizer/s_expr.rs
index 3fa48568e0586..0f63a49a7a1fe 100644
--- a/src/query/sql/src/planner/optimizer/s_expr.rs
+++ b/src/query/sql/src/planner/optimizer/s_expr.rs
@@ -319,8 +319,10 @@ impl SExpr {
             | RelOperator::CteScan(_)
             | RelOperator::AddRowNumber(_)
             | RelOperator::MaterializedCte(_)
-            | RelOperator::AsyncFunction(_)
-            | RelOperator::ConstantTableScan(_) => {}
+            | RelOperator::ConstantTableScan(_)
+            | RelOperator::ExpressionScan(_)
+            | RelOperator::CacheScan(_)
+            | RelOperator::AsyncFunction(_) => {}
         };
         for child in &self.children {
             let udf = child.get_udfs()?;
@@ -405,8 +407,10 @@ fn find_subquery(rel_op: &RelOperator) -> bool {
         | RelOperator::CteScan(_)
         | RelOperator::AddRowNumber(_)
         | RelOperator::MaterializedCte(_)
-        | RelOperator::AsyncFunction(_)
-        | RelOperator::ConstantTableScan(_) => false,
+        | RelOperator::ConstantTableScan(_)
+        | RelOperator::ExpressionScan(_)
+        | RelOperator::CacheScan(_)
+        | RelOperator::AsyncFunction(_) => false,
         RelOperator::Join(op) => {
             op.left_conditions.iter().any(find_subquery_in_expr)
                 || op.right_conditions.iter().any(find_subquery_in_expr)
diff --git a/src/query/sql/src/planner/plans/cache_scan.rs b/src/query/sql/src/planner/plans/cache_scan.rs
new file mode 100644
index 0000000000000..023ef70d60ebb
--- /dev/null
+++ b/src/query/sql/src/planner/plans/cache_scan.rs
@@ -0,0 +1,136 @@
+// Copyright 2021 Datafuse Labs
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+use std::sync::Arc;
+
+use databend_common_catalog::table_context::TableContext;
+use databend_common_exception::Result;
+use databend_common_expression::DataSchemaRef;
+
+use crate::optimizer::ColumnSet;
+use crate::optimizer::Distribution;
+use crate::optimizer::PhysicalProperty;
+use crate::optimizer::RelExpr;
+use crate::optimizer::RelationalProperty;
+use crate::optimizer::RequiredProperty;
+use crate::optimizer::StatInfo;
+use crate::optimizer::Statistics;
+use crate::plans::Operator;
+use crate::plans::RelOp;
+
+#[derive(Clone, Debug, PartialEq, Eq, Hash, serde::Serialize, serde::Deserialize)]
+pub enum CacheSource {
+    HashJoinBuild((usize, Vec<usize>)),
+}
+
+impl CacheSource {
+    pub fn project(&self, projection: &[usize]) -> Self {
+        match self {
+            CacheSource::HashJoinBuild((cache_index, column_indexes)) => {
+                let column_indexes = column_indexes.iter().map(|idx| projection[*idx]).collect();
+                CacheSource::HashJoinBuild((*cache_index, column_indexes))
+            }
+        }
+    }
+}
+
+// Constant table is a table with constant values.
+#[derive(Clone, Debug, PartialEq, Eq)]
+pub struct CacheScan {
+    pub cache_source: CacheSource,
+    pub columns: ColumnSet,
+    pub schema: DataSchemaRef,
+}
+
+impl CacheScan {
+    pub fn prune_columns(&self, columns: ColumnSet) -> Self {
+        let mut projection = columns
+            .iter()
+            .map(|index| self.schema.index_of(&index.to_string()).unwrap())
+            .collect::<Vec<_>>();
+        projection.sort();
+
+        let schema = Arc::new(self.schema.project(&projection));
+        let cache_source = self.cache_source.project(&projection);
+        let columns = self.columns.intersection(&columns).cloned().collect();
+
+        CacheScan {
+            cache_source,
+            schema,
+            columns,
+        }
+    }
+
+    pub fn used_columns(&self) -> Result<ColumnSet> {
+        Ok(ColumnSet::new())
+    }
+}
+
+impl std::hash::Hash for CacheScan {
+    fn hash<H: std::hash::Hasher>(&self, state: &mut H) {
+        self.cache_source.hash(state);
+        let mut column = self.columns.iter().collect::<Vec<_>>();
+        column.sort();
+        for column in column.iter() {
+            column.hash(state);
+        }
+    }
+}
+
+impl Operator for CacheScan {
+    fn rel_op(&self) -> RelOp {
+        RelOp::CacheScan
+    }
+
+    fn arity(&self) -> usize {
+        0
+    }
+
+    fn derive_relational_prop(&self, _rel_expr: &RelExpr) -> Result<Arc<RelationalProperty>> {
+        Ok(Arc::new(RelationalProperty {
+            output_columns: self.columns.clone(),
+            outer_columns: Default::default(),
+            used_columns: Default::default(),
+            orderings: vec![],
+        }))
+    }
+
+    fn derive_physical_prop(&self, _rel_expr: &RelExpr) -> Result<PhysicalProperty> {
+        Ok(PhysicalProperty {
+            distribution: Distribution::Serial,
+        })
+    }
+
+    fn derive_stats(&self, _rel_expr: &RelExpr) -> Result<Arc<StatInfo>> {
+        Ok(Arc::new(StatInfo {
+            cardinality: 0.0,
+            statistics: Statistics {
+                precise_cardinality: None,
+                column_stats: Default::default(),
+            },
+        }))
+    }
+
+    fn compute_required_prop_child(
+        &self,
+        _ctx: Arc<dyn TableContext>,
+        _rel_expr: &RelExpr,
+        _child_index: usize,
+        required: &RequiredProperty,
+    ) -> Result<RequiredProperty> {
+        let mut required = required.clone();
+        required.distribution = Distribution::Serial;
+        Ok(required.clone())
+    }
+}
diff --git a/src/query/sql/src/planner/plans/expression_scan.rs b/src/query/sql/src/planner/plans/expression_scan.rs
new file mode 100644
index 0000000000000..c38d568470b07
--- /dev/null
+++ b/src/query/sql/src/planner/plans/expression_scan.rs
@@ -0,0 +1,122 @@
+// Copyright 2021 Datafuse Labs
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+use std::collections::HashSet;
+use std::sync::Arc;
+
+use databend_common_catalog::table_context::TableContext;
+use databend_common_exception::Result;
+use databend_common_expression::types::DataType;
+use databend_common_expression::DataSchemaRef;
+
+use crate::optimizer::ColumnSet;
+use crate::optimizer::PhysicalProperty;
+use crate::optimizer::RelExpr;
+use crate::optimizer::RelationalProperty;
+use crate::optimizer::RequiredProperty;
+use crate::optimizer::StatInfo;
+use crate::optimizer::Statistics;
+use crate::plans::Operator;
+use crate::plans::RelOp;
+use crate::ScalarExpr;
+
+// Constant table is a table with constant values.
+#[derive(Clone, Debug, PartialEq, Eq)]
+pub struct ExpressionScan {
+    pub expression_scan_index: usize,
+    pub values: Vec<Vec<ScalarExpr>>,
+    pub num_scalar_columns: usize,
+    pub cache_index: usize,
+    pub column_indexes: Vec<usize>,
+    pub data_types: Vec<DataType>,
+    pub schema: DataSchemaRef,
+}
+
+impl ExpressionScan {
+    pub fn used_columns(&self) -> Result<ColumnSet> {
+        let mut columns = HashSet::new();
+        for row in self.values.iter() {
+            for value in row {
+                columns.extend(value.used_columns());
+            }
+        }
+        Ok(columns)
+    }
+
+    pub fn remove_cache_column(&mut self, index: usize) {
+        for row in self.values.iter_mut() {
+            row.remove(index);
+        }
+        self.column_indexes.remove(index);
+        self.data_types.remove(index);
+    }
+}
+
+impl std::hash::Hash for ExpressionScan {
+    fn hash<H: std::hash::Hasher>(&self, state: &mut H) {
+        self.expression_scan_index.hash(state);
+        for row in self.values.iter() {
+            for scalar in row {
+                scalar.hash(state);
+            }
+        }
+        self.num_scalar_columns.hash(state);
+        self.cache_index.hash(state);
+        self.column_indexes.hash(state);
+        self.data_types.hash(state);
+    }
+}
+
+impl Operator for ExpressionScan {
+    fn rel_op(&self) -> RelOp {
+        RelOp::ExpressionScan
+    }
+
+    fn arity(&self) -> usize {
+        1
+    }
+
+    fn derive_relational_prop(&self, _rel_expr: &RelExpr) -> Result<Arc<RelationalProperty>> {
+        Ok(Arc::new(RelationalProperty {
+            output_columns: self.column_indexes.clone().into_iter().collect(),
+            outer_columns: self.used_columns()?,
+            used_columns: self.used_columns()?,
+            orderings: vec![],
+        }))
+    }
+
+    fn derive_physical_prop(&self, rel_expr: &RelExpr) -> Result<PhysicalProperty> {
+        rel_expr.derive_physical_prop_child(0)
+    }
+
+    fn derive_stats(&self, _rel_expr: &RelExpr) -> Result<Arc<StatInfo>> {
+        Ok(Arc::new(StatInfo {
+            cardinality: self.values.len() as f64,
+            statistics: Statistics {
+                precise_cardinality: None,
+                column_stats: Default::default(),
+            },
+        }))
+    }
+
+    fn compute_required_prop_child(
+        &self,
+        _ctx: Arc<dyn TableContext>,
+        _rel_expr: &RelExpr,
+        _child_index: usize,
+        required: &RequiredProperty,
+    ) -> Result<RequiredProperty> {
+        Ok(required.clone())
+    }
+}
diff --git a/src/query/sql/src/planner/plans/join.rs b/src/query/sql/src/planner/plans/join.rs
index 7ef36bcf5540d..43ba5ee331f61 100644
--- a/src/query/sql/src/planner/plans/join.rs
+++ b/src/query/sql/src/planner/plans/join.rs
@@ -142,6 +142,12 @@ impl Display for JoinType {
     }
 }
 
+#[derive(Clone, Debug, PartialEq, Eq, Hash)]
+pub struct HashJoinBuildCacheInfo {
+    pub cache_idx: usize,
+    pub columns: Vec<usize>,
+}
+
 /// Join operator. We will choose hash join by default.
 /// In the case that using hash join, the right child
 /// is always the build side, and the left child is always
@@ -162,6 +168,7 @@ pub struct Join {
     // When left/right single join converted to inner join, record the original join type
     // and do some special processing during runtime.
     pub single_to_inner: Option<JoinType>,
+    pub build_side_cache_info: Option<HashJoinBuildCacheInfo>,
 }
 
 impl Default for Join {
@@ -176,6 +183,7 @@ impl Default for Join {
             need_hold_hash_table: false,
             is_lateral: false,
             single_to_inner: None,
+            build_side_cache_info: None,
         }
     }
 }
diff --git a/src/query/sql/src/planner/plans/mod.rs b/src/query/sql/src/planner/plans/mod.rs
index 9eda56f00cdbf..6e667ca095709 100644
--- a/src/query/sql/src/planner/plans/mod.rs
+++ b/src/query/sql/src/planner/plans/mod.rs
@@ -15,6 +15,7 @@
 mod add_row_number;
 mod aggregate;
 mod async_function;
+mod cache_scan;
 mod call;
 mod constant_table_scan;
 mod copy_into_table;
@@ -25,6 +26,7 @@ mod delete;
 mod dummy_table_scan;
 mod eval_scalar;
 mod exchange;
+mod expression_scan;
 mod filter;
 pub mod insert;
 mod join;
@@ -56,6 +58,7 @@ mod window;
 pub use add_row_number::AddRowNumber;
 pub use aggregate::*;
 pub use async_function::AsyncFunction;
+pub use cache_scan::*;
 pub use call::CallPlan;
 pub use constant_table_scan::ConstantTableScan;
 pub use copy_into_location::*;
@@ -68,6 +71,7 @@ pub use delete::SubqueryDesc;
 pub use dummy_table_scan::DummyTableScan;
 pub use eval_scalar::*;
 pub use exchange::*;
+pub use expression_scan::*;
 pub use filter::*;
 pub use insert::Insert;
 pub use insert::InsertInputSource;
diff --git a/src/query/sql/src/planner/plans/operator.rs b/src/query/sql/src/planner/plans/operator.rs
index 3d49be60cac3a..3008b8a49b9fe 100644
--- a/src/query/sql/src/planner/plans/operator.rs
+++ b/src/query/sql/src/planner/plans/operator.rs
@@ -28,6 +28,7 @@ use super::limit::Limit;
 use super::scan::Scan;
 use super::sort::Sort;
 use super::union_all::UnionAll;
+use super::ExpressionScan;
 use crate::optimizer::PhysicalProperty;
 use crate::optimizer::RelExpr;
 use crate::optimizer::RelationalProperty;
@@ -35,6 +36,7 @@ use crate::optimizer::RequiredProperty;
 use crate::optimizer::StatInfo;
 use crate::plans::materialized_cte::MaterializedCte;
 use crate::plans::AsyncFunction;
+use crate::plans::CacheScan;
 use crate::plans::ConstantTableScan;
 use crate::plans::CteScan;
 use crate::plans::Exchange;
@@ -96,6 +98,8 @@ pub enum RelOp {
     ProjectSet,
     MaterializedCte,
     ConstantTableScan,
+    ExpressionScan,
+    CacheScan,
     AddRowNumber,
     Udf,
     AsyncFunction,
@@ -123,6 +127,8 @@ pub enum RelOperator {
     ProjectSet(ProjectSet),
     MaterializedCte(MaterializedCte),
     ConstantTableScan(ConstantTableScan),
+    ExpressionScan(ExpressionScan),
+    CacheScan(CacheScan),
     Udf(Udf),
     AsyncFunction(AsyncFunction),
 }
@@ -145,6 +151,8 @@ impl Operator for RelOperator {
             RelOperator::CteScan(rel_op) => rel_op.rel_op(),
             RelOperator::MaterializedCte(rel_op) => rel_op.rel_op(),
             RelOperator::ConstantTableScan(rel_op) => rel_op.rel_op(),
+            RelOperator::ExpressionScan(rel_op) => rel_op.rel_op(),
+            RelOperator::CacheScan(rel_op) => rel_op.rel_op(),
             RelOperator::AddRowNumber(rel_op) => rel_op.rel_op(),
             RelOperator::Udf(rel_op) => rel_op.rel_op(),
             RelOperator::AsyncFunction(rel_op) => rel_op.rel_op(),
@@ -169,6 +177,8 @@ impl Operator for RelOperator {
             RelOperator::ProjectSet(rel_op) => rel_op.arity(),
             RelOperator::MaterializedCte(rel_op) => rel_op.arity(),
             RelOperator::ConstantTableScan(rel_op) => rel_op.arity(),
+            RelOperator::ExpressionScan(rel_op) => rel_op.arity(),
+            RelOperator::CacheScan(rel_op) => rel_op.arity(),
             RelOperator::Udf(rel_op) => rel_op.arity(),
             RelOperator::AsyncFunction(rel_op) => rel_op.arity(),
         }
@@ -191,6 +201,8 @@ impl Operator for RelOperator {
             RelOperator::CteScan(rel_op) => rel_op.derive_relational_prop(rel_expr),
             RelOperator::MaterializedCte(rel_op) => rel_op.derive_relational_prop(rel_expr),
             RelOperator::ConstantTableScan(rel_op) => rel_op.derive_relational_prop(rel_expr),
+            RelOperator::ExpressionScan(rel_op) => rel_op.derive_relational_prop(rel_expr),
+            RelOperator::CacheScan(rel_op) => rel_op.derive_relational_prop(rel_expr),
             RelOperator::AddRowNumber(rel_op) => rel_op.derive_relational_prop(rel_expr),
             RelOperator::Udf(rel_op) => rel_op.derive_relational_prop(rel_expr),
             RelOperator::AsyncFunction(rel_op) => rel_op.derive_relational_prop(rel_expr),
@@ -214,6 +226,8 @@ impl Operator for RelOperator {
             RelOperator::CteScan(rel_op) => rel_op.derive_physical_prop(rel_expr),
             RelOperator::MaterializedCte(rel_op) => rel_op.derive_physical_prop(rel_expr),
             RelOperator::ConstantTableScan(rel_op) => rel_op.derive_physical_prop(rel_expr),
+            RelOperator::ExpressionScan(rel_op) => rel_op.derive_physical_prop(rel_expr),
+            RelOperator::CacheScan(rel_op) => rel_op.derive_physical_prop(rel_expr),
             RelOperator::AddRowNumber(rel_op) => rel_op.derive_physical_prop(rel_expr),
             RelOperator::Udf(rel_op) => rel_op.derive_physical_prop(rel_expr),
             RelOperator::AsyncFunction(rel_op) => rel_op.derive_physical_prop(rel_expr),
@@ -237,6 +251,8 @@ impl Operator for RelOperator {
             RelOperator::CteScan(rel_op) => rel_op.derive_stats(rel_expr),
             RelOperator::MaterializedCte(rel_op) => rel_op.derive_stats(rel_expr),
             RelOperator::ConstantTableScan(rel_op) => rel_op.derive_stats(rel_expr),
+            RelOperator::ExpressionScan(rel_op) => rel_op.derive_stats(rel_expr),
+            RelOperator::CacheScan(rel_op) => rel_op.derive_stats(rel_expr),
             RelOperator::AddRowNumber(rel_op) => rel_op.derive_stats(rel_expr),
             RelOperator::Udf(rel_op) => rel_op.derive_stats(rel_expr),
             RelOperator::AsyncFunction(rel_op) => rel_op.derive_stats(rel_expr),
@@ -296,6 +312,12 @@ impl Operator for RelOperator {
             RelOperator::ConstantTableScan(rel_op) => {
                 rel_op.compute_required_prop_child(ctx, rel_expr, child_index, required)
             }
+            RelOperator::ExpressionScan(rel_op) => {
+                rel_op.compute_required_prop_child(ctx, rel_expr, child_index, required)
+            }
+            RelOperator::CacheScan(rel_op) => {
+                rel_op.compute_required_prop_child(ctx, rel_expr, child_index, required)
+            }
             RelOperator::AddRowNumber(rel_op) => {
                 rel_op.compute_required_prop_child(ctx, rel_expr, child_index, required)
             }
@@ -363,6 +385,12 @@ impl Operator for RelOperator {
             RelOperator::ConstantTableScan(rel_op) => {
                 rel_op.compute_required_prop_children(ctx, rel_expr, required)
             }
+            RelOperator::ExpressionScan(rel_op) => {
+                rel_op.compute_required_prop_children(ctx, rel_expr, required)
+            }
+            RelOperator::CacheScan(rel_op) => {
+                rel_op.compute_required_prop_children(ctx, rel_expr, required)
+            }
             RelOperator::Udf(rel_op) => {
                 rel_op.compute_required_prop_children(ctx, rel_expr, required)
             }
@@ -639,6 +667,12 @@ impl From<ConstantTableScan> for RelOperator {
     }
 }
 
+impl From<ExpressionScan> for RelOperator {
+    fn from(value: ExpressionScan) -> Self {
+        Self::ExpressionScan(value)
+    }
+}
+
 impl TryFrom<RelOperator> for ConstantTableScan {
     type Error = ErrorCode;
 
diff --git a/src/query/sql/src/planner/semantic/mod.rs b/src/query/sql/src/planner/semantic/mod.rs
index 83bb966d1e803..a39afafc437f3 100644
--- a/src/query/sql/src/planner/semantic/mod.rs
+++ b/src/query/sql/src/planner/semantic/mod.rs
@@ -29,7 +29,7 @@ pub use aggregate_rewriter::AggregateRewriter;
 pub use aggregating_index_visitor::AggregatingIndexChecker;
 pub use aggregating_index_visitor::AggregatingIndexRewriter;
 pub use aggregating_index_visitor::RefreshAggregatingIndexRewriter;
-pub(crate) use async_function_rewriter::AsyncFunctionRewriter;
+pub use async_function_rewriter::AsyncFunctionRewriter;
 pub use distinct_to_groupby::DistinctToGroupBy;
 pub use grouping_check::GroupingChecker;
 pub use lowering::*;
diff --git a/src/query/sql/src/planner/semantic/type_check.rs b/src/query/sql/src/planner/semantic/type_check.rs
index c06f4005d4b7e..923cf0a3ceb02 100644
--- a/src/query/sql/src/planner/semantic/type_check.rs
+++ b/src/query/sql/src/planner/semantic/type_check.rs
@@ -4000,6 +4000,7 @@ impl<'a> TypeChecker<'a> {
             &mut bind_context,
             None,
             &values,
+            None,
         )
         .await?;
         assert_eq!(ctx.columns.len(), 1);
diff --git a/tests/sqllogictests/suites/mode/standalone/explain/expression_scan.test b/tests/sqllogictests/suites/mode/standalone/explain/expression_scan.test
new file mode 100644
index 0000000000000..9520cf985ac11
--- /dev/null
+++ b/tests/sqllogictests/suites/mode/standalone/explain/expression_scan.test
@@ -0,0 +1,268 @@
+statement ok
+drop table if exists t1;
+
+statement ok
+drop table if exists t2;
+
+statement ok
+create table t1(a int, b int, c int);
+
+statement ok
+create table t2(a int, b int, c int);
+
+statement ok
+insert into t1 values (1, 10, 11), (10, 20, 111), (2, 20, 200);
+
+statement ok
+insert into t2 values (1, 10, 22), (2222, 10, 22), (3, 20, 222);
+
+query T
+explain select t1.a, t1.b from t1 join lateral (values(t1.b)) as v1 ("c1") on t1.b = v1.c1;
+----
+HashJoin
+├── output columns: [t1.a (#0), t1.b (#1)]
+├── join type: INNER
+├── build keys: [t1.b (#1), b (#1)]
+├── probe keys: [v1.c1 (#3), b (#5)]
+├── filters: []
+├── cache index: 0
+├── cache columns: [1]
+├── estimated rows: 3.00
+├── TableScan(Build)
+│   ├── table: default.default.t1
+│   ├── output columns: [a (#0), b (#1)]
+│   ├── read rows: 3
+│   ├── read size: < 1 KiB
+│   ├── partitions total: 1
+│   ├── partitions scanned: 1
+│   ├── pruning stats: [segments: <range pruning: 1 to 1>, blocks: <range pruning: 1 to 1>]
+│   ├── push downs: [filters: [], limit: NONE]
+│   └── estimated rows: 3.00
+└── ExpressionScan(Probe)
+    ├── output columns: [c1 (#3), b (#5)]
+    ├── column 0: [t1.b (#1), t1.b (#1)]
+    └── AggregateFinal
+        ├── output columns: [t1.b (#1)]
+        ├── group by: [b]
+        ├── aggregate functions: []
+        ├── estimated rows: 0.00
+        └── AggregatePartial
+            ├── group by: [b]
+            ├── aggregate functions: []
+            ├── estimated rows: 0.00
+            └── CacheScan
+                ├── output columns: [t1.b (#1)]
+                ├── cache index: 0
+                └── column indexes: [1]
+
+query T
+explain select t1.a, t1.b from t1 join lateral (values(t1.b), (t1.c)) as v1 ("c1") on t1.b = v1.c1;
+----
+HashJoin
+├── output columns: [t1.a (#0), t1.b (#1)]
+├── join type: INNER
+├── build keys: [t1.b (#1), b (#1), c (#2)]
+├── probe keys: [v1.c1 (#3), b (#5), c (#6)]
+├── filters: []
+├── cache index: 0
+├── cache columns: [1, 2]
+├── estimated rows: 6.00
+├── TableScan(Build)
+│   ├── table: default.default.t1
+│   ├── output columns: [a (#0), b (#1), c (#2)]
+│   ├── read rows: 3
+│   ├── read size: < 1 KiB
+│   ├── partitions total: 1
+│   ├── partitions scanned: 1
+│   ├── pruning stats: [segments: <range pruning: 1 to 1>, blocks: <range pruning: 1 to 1>]
+│   ├── push downs: [filters: [], limit: NONE]
+│   └── estimated rows: 3.00
+└── ExpressionScan(Probe)
+    ├── output columns: [c1 (#3), b (#5), c (#6)]
+    ├── column 0: [t1.b (#1), t1.b (#1), t1.c (#2)]
+    ├── column 1: [t1.c (#2), t1.b (#1), t1.c (#2)]
+    └── AggregateFinal
+        ├── output columns: [t1.c (#2), t1.b (#1)]
+        ├── group by: [c, b]
+        ├── aggregate functions: []
+        ├── estimated rows: 0.00
+        └── AggregatePartial
+            ├── group by: [c, b]
+            ├── aggregate functions: []
+            ├── estimated rows: 0.00
+            └── CacheScan
+                ├── output columns: [t1.c (#2), t1.b (#1)]
+                ├── cache index: 0
+                └── column indexes: [1, 2]
+
+query T
+explain select t1.a, v1.c1, v1.c2 from t1 left join lateral (values('t1_b', t1.b), ('t1_c', t1.c)) as v1 ("c1", "c2") on t1.a = v1.c2;
+----
+HashJoin
+├── output columns: [c1 (#3), c2 (#4), t1.a (#0)]
+├── join type: RIGHT OUTER
+├── build keys: [t1.a (#0), b (#1), c (#2)]
+├── probe keys: [v1.c2 (#4), b (#6), c (#7)]
+├── filters: []
+├── cache index: 0
+├── cache columns: [1, 2]
+├── estimated rows: 6.00
+├── TableScan(Build)
+│   ├── table: default.default.t1
+│   ├── output columns: [a (#0), b (#1), c (#2)]
+│   ├── read rows: 3
+│   ├── read size: < 1 KiB
+│   ├── partitions total: 1
+│   ├── partitions scanned: 1
+│   ├── pruning stats: [segments: <range pruning: 1 to 1>, blocks: <range pruning: 1 to 1>]
+│   ├── push downs: [filters: [], limit: NONE]
+│   └── estimated rows: 3.00
+└── ExpressionScan(Probe)
+    ├── output columns: [c1 (#3), c2 (#4), b (#6), c (#7)]
+    ├── column 0: ['t1_b', t1.b (#1), t1.b (#1), t1.c (#2)]
+    ├── column 1: ['t1_c', t1.c (#2), t1.b (#1), t1.c (#2)]
+    └── AggregateFinal
+        ├── output columns: [t1.c (#2), t1.b (#1)]
+        ├── group by: [c, b]
+        ├── aggregate functions: []
+        ├── estimated rows: 0.00
+        └── AggregatePartial
+            ├── group by: [c, b]
+            ├── aggregate functions: []
+            ├── estimated rows: 0.00
+            └── CacheScan
+                ├── output columns: [t1.c (#2), t1.b (#1)]
+                ├── cache index: 0
+                └── column indexes: [1, 2]
+
+query T
+explain select t1.a, v1.c1, v1.c2 from t1 left join lateral (values('t1_b', t1.b), ('t1_c', t1.c)) as v1 ("c1", "c2") on t1.a = v1.c2 left join lateral (values('t1_b', t1.b), ('t1_c', t1.c)) as v2 ("c1", "c2") on t1.a = v2.c2;
+----
+HashJoin
+├── output columns: [c1 (#3), c2 (#4), t1.a (#0)]
+├── join type: RIGHT OUTER
+├── build keys: [t1.a (#0), b (#1), c (#2)]
+├── probe keys: [v2.c2 (#9), b (#11), c (#12)]
+├── filters: []
+├── cache index: 1
+├── cache columns: [1, 2]
+├── estimated rows: 12.00
+├── HashJoin(Build)
+│   ├── output columns: [c1 (#3), c2 (#4), t1.a (#0), t1.b (#1), t1.c (#2)]
+│   ├── join type: RIGHT OUTER
+│   ├── build keys: [t1.a (#0), b (#1), c (#2)]
+│   ├── probe keys: [v1.c2 (#4), b (#6), c (#7)]
+│   ├── filters: []
+│   ├── cache index: 0
+│   ├── cache columns: [1, 2]
+│   ├── estimated rows: 6.00
+│   ├── TableScan(Build)
+│   │   ├── table: default.default.t1
+│   │   ├── output columns: [a (#0), b (#1), c (#2)]
+│   │   ├── read rows: 3
+│   │   ├── read size: < 1 KiB
+│   │   ├── partitions total: 1
+│   │   ├── partitions scanned: 1
+│   │   ├── pruning stats: [segments: <range pruning: 1 to 1>, blocks: <range pruning: 1 to 1>]
+│   │   ├── push downs: [filters: [], limit: NONE]
+│   │   └── estimated rows: 3.00
+│   └── ExpressionScan(Probe)
+│       ├── output columns: [c1 (#3), c2 (#4), b (#6), c (#7)]
+│       ├── column 0: ['t1_b', t1.b (#1), t1.b (#1), t1.c (#2)]
+│       ├── column 1: ['t1_c', t1.c (#2), t1.b (#1), t1.c (#2)]
+│       └── AggregateFinal
+│           ├── output columns: [t1.c (#2), t1.b (#1)]
+│           ├── group by: [c, b]
+│           ├── aggregate functions: []
+│           ├── estimated rows: 0.00
+│           └── AggregatePartial
+│               ├── group by: [c, b]
+│               ├── aggregate functions: []
+│               ├── estimated rows: 0.00
+│               └── CacheScan
+│                   ├── output columns: [t1.c (#2), t1.b (#1)]
+│                   ├── cache index: 0
+│                   └── column indexes: [1, 2]
+└── ExpressionScan(Probe)
+    ├── output columns: [c1 (#8), c2 (#9), b (#11), c (#12)]
+    ├── column 0: ['t1_b', t1.b (#1), t1.b (#1), t1.c (#2)]
+    ├── column 1: ['t1_c', t1.c (#2), t1.b (#1), t1.c (#2)]
+    └── AggregateFinal
+        ├── output columns: [t1.c (#2), t1.b (#1)]
+        ├── group by: [c, b]
+        ├── aggregate functions: []
+        ├── estimated rows: 0.00
+        └── AggregatePartial
+            ├── group by: [c, b]
+            ├── aggregate functions: []
+            ├── estimated rows: 0.00
+            └── CacheScan
+                ├── output columns: [t1.c (#2), t1.b (#1)]
+                ├── cache index: 1
+                └── column indexes: [1, 2]
+
+query T
+explain select t1.a, v1.c1, v1.c2 from t1 inner join lateral (values('t1_b', t1.b), ('t1_c', t1.c)) as v1 ("c1", "c2") on t1.a = v1.c2 inner join lateral (values('t1_b', t1.b), ('t1_c', v1.c1)) as v2 ("c1", "c2") on t1.a = v2.c2;
+----
+HashJoin
+├── output columns: [c1 (#3), c2 (#4), t1.a (#0)]
+├── join type: INNER
+├── build keys: [CAST(t1.a (#0) AS Int64 NULL), b (#1), c1 (#3)]
+├── probe keys: [v2.c2 (#9), b (#11), c1 (#13)]
+├── filters: []
+├── cache index: 1
+├── cache columns: [1, 3]
+├── estimated rows: 12.00
+├── HashJoin(Build)
+│   ├── output columns: [c1 (#3), c2 (#4), t1.a (#0), t1.b (#1)]
+│   ├── join type: INNER
+│   ├── build keys: [t1.a (#0), b (#1), c (#2)]
+│   ├── probe keys: [v1.c2 (#4), b (#6), c (#7)]
+│   ├── filters: []
+│   ├── cache index: 0
+│   ├── cache columns: [1, 2]
+│   ├── estimated rows: 6.00
+│   ├── TableScan(Build)
+│   │   ├── table: default.default.t1
+│   │   ├── output columns: [a (#0), b (#1), c (#2)]
+│   │   ├── read rows: 3
+│   │   ├── read size: < 1 KiB
+│   │   ├── partitions total: 1
+│   │   ├── partitions scanned: 1
+│   │   ├── pruning stats: [segments: <range pruning: 1 to 1>, blocks: <range pruning: 1 to 1>]
+│   │   ├── push downs: [filters: [], limit: NONE]
+│   │   └── estimated rows: 3.00
+│   └── ExpressionScan(Probe)
+│       ├── output columns: [c1 (#3), c2 (#4), b (#6), c (#7)]
+│       ├── column 0: ['t1_b', t1.b (#1), t1.b (#1), t1.c (#2)]
+│       ├── column 1: ['t1_c', t1.c (#2), t1.b (#1), t1.c (#2)]
+│       └── AggregateFinal
+│           ├── output columns: [t1.c (#2), t1.b (#1)]
+│           ├── group by: [c, b]
+│           ├── aggregate functions: []
+│           ├── estimated rows: 0.00
+│           └── AggregatePartial
+│               ├── group by: [c, b]
+│               ├── aggregate functions: []
+│               ├── estimated rows: 0.00
+│               └── CacheScan
+│                   ├── output columns: [t1.c (#2), t1.b (#1)]
+│                   ├── cache index: 0
+│                   └── column indexes: [1, 2]
+└── ExpressionScan(Probe)
+    ├── output columns: [c1 (#8), c2 (#9), b (#11), c1 (#13)]
+    ├── column 0: ['t1_b', CAST(t1.b (#1) AS Int64 NULL), t1.b (#1), v1.c1 (#3)]
+    ├── column 1: ['t1_c', CAST(v1.c1 (#3) AS Int64 NULL), t1.b (#1), v1.c1 (#3)]
+    └── AggregateFinal
+        ├── output columns: [c1 (#3), t1.b (#1)]
+        ├── group by: [c1, b]
+        ├── aggregate functions: []
+        ├── estimated rows: 0.00
+        └── AggregatePartial
+            ├── group by: [c1, b]
+            ├── aggregate functions: []
+            ├── estimated rows: 0.00
+            └── CacheScan
+                ├── output columns: [c1 (#3), t1.b (#1)]
+                ├── cache index: 1
+                └── column indexes: [1, 3]
diff --git a/tests/sqllogictests/suites/mode/standalone/explain_native/expression_scan.test b/tests/sqllogictests/suites/mode/standalone/explain_native/expression_scan.test
new file mode 100644
index 0000000000000..c35e06b9d5384
--- /dev/null
+++ b/tests/sqllogictests/suites/mode/standalone/explain_native/expression_scan.test
@@ -0,0 +1,269 @@
+statement ok
+drop table if exists t1;
+
+statement ok
+drop table if exists t2;
+
+statement ok
+create table t1(a int, b int, c int);
+
+statement ok
+create table t2(a int, b int, c int);
+
+statement ok
+insert into t1 values (1, 10, 11), (10, 20, 111), (2, 20, 200);
+
+statement ok
+insert into t2 values (1, 10, 22), (2222, 10, 22), (3, 20, 222);
+
+query T
+explain select t1.a, t1.b from t1 join lateral (values(t1.b)) as v1 ("c1") on t1.b = v1.c1;
+----
+HashJoin
+├── output columns: [t1.a (#0), t1.b (#1)]
+├── join type: INNER
+├── build keys: [t1.b (#1), b (#1)]
+├── probe keys: [v1.c1 (#3), b (#5)]
+├── filters: []
+├── cache index: 0
+├── cache columns: [1]
+├── estimated rows: 3.00
+├── TableScan(Build)
+│   ├── table: default.default.t1
+│   ├── output columns: [a (#0), b (#1)]
+│   ├── read rows: 3
+│   ├── read size: < 1 KiB
+│   ├── partitions total: 1
+│   ├── partitions scanned: 1
+│   ├── pruning stats: [segments: <range pruning: 1 to 1>, blocks: <range pruning: 1 to 1>]
+│   ├── push downs: [filters: [], limit: NONE]
+│   └── estimated rows: 3.00
+└── ExpressionScan(Probe)
+    ├── output columns: [c1 (#3), b (#5)]
+    ├── column 0: [t1.b (#1), t1.b (#1)]
+    └── AggregateFinal
+        ├── output columns: [t1.b (#1)]
+        ├── group by: [b]
+        ├── aggregate functions: []
+        ├── estimated rows: 0.00
+        └── AggregatePartial
+            ├── group by: [b]
+            ├── aggregate functions: []
+            ├── estimated rows: 0.00
+            └── CacheScan
+                ├── output columns: [t1.b (#1)]
+                ├── cache index: 0
+                └── column indexes: [1]
+
+
+query T
+explain select t1.a, t1.b from t1 join lateral (values(t1.b), (t1.c)) as v1 ("c1") on t1.b = v1.c1;
+----
+HashJoin
+├── output columns: [t1.a (#0), t1.b (#1)]
+├── join type: INNER
+├── build keys: [t1.b (#1), b (#1), c (#2)]
+├── probe keys: [v1.c1 (#3), b (#5), c (#6)]
+├── filters: []
+├── cache index: 0
+├── cache columns: [1, 2]
+├── estimated rows: 6.00
+├── TableScan(Build)
+│   ├── table: default.default.t1
+│   ├── output columns: [a (#0), b (#1), c (#2)]
+│   ├── read rows: 3
+│   ├── read size: < 1 KiB
+│   ├── partitions total: 1
+│   ├── partitions scanned: 1
+│   ├── pruning stats: [segments: <range pruning: 1 to 1>, blocks: <range pruning: 1 to 1>]
+│   ├── push downs: [filters: [], limit: NONE]
+│   └── estimated rows: 3.00
+└── ExpressionScan(Probe)
+    ├── output columns: [c1 (#3), b (#5), c (#6)]
+    ├── column 0: [t1.b (#1), t1.b (#1), t1.c (#2)]
+    ├── column 1: [t1.c (#2), t1.b (#1), t1.c (#2)]
+    └── AggregateFinal
+        ├── output columns: [t1.c (#2), t1.b (#1)]
+        ├── group by: [c, b]
+        ├── aggregate functions: []
+        ├── estimated rows: 0.00
+        └── AggregatePartial
+            ├── group by: [c, b]
+            ├── aggregate functions: []
+            ├── estimated rows: 0.00
+            └── CacheScan
+                ├── output columns: [t1.c (#2), t1.b (#1)]
+                ├── cache index: 0
+                └── column indexes: [1, 2]
+
+query T
+explain select t1.a, v1.c1, v1.c2 from t1 left join lateral (values('t1_b', t1.b), ('t1_c', t1.c)) as v1 ("c1", "c2") on t1.a = v1.c2;
+----
+HashJoin
+├── output columns: [c1 (#3), c2 (#4), t1.a (#0)]
+├── join type: RIGHT OUTER
+├── build keys: [t1.a (#0), b (#1), c (#2)]
+├── probe keys: [v1.c2 (#4), b (#6), c (#7)]
+├── filters: []
+├── cache index: 0
+├── cache columns: [1, 2]
+├── estimated rows: 6.00
+├── TableScan(Build)
+│   ├── table: default.default.t1
+│   ├── output columns: [a (#0), b (#1), c (#2)]
+│   ├── read rows: 3
+│   ├── read size: < 1 KiB
+│   ├── partitions total: 1
+│   ├── partitions scanned: 1
+│   ├── pruning stats: [segments: <range pruning: 1 to 1>, blocks: <range pruning: 1 to 1>]
+│   ├── push downs: [filters: [], limit: NONE]
+│   └── estimated rows: 3.00
+└── ExpressionScan(Probe)
+    ├── output columns: [c1 (#3), c2 (#4), b (#6), c (#7)]
+    ├── column 0: ['t1_b', t1.b (#1), t1.b (#1), t1.c (#2)]
+    ├── column 1: ['t1_c', t1.c (#2), t1.b (#1), t1.c (#2)]
+    └── AggregateFinal
+        ├── output columns: [t1.c (#2), t1.b (#1)]
+        ├── group by: [c, b]
+        ├── aggregate functions: []
+        ├── estimated rows: 0.00
+        └── AggregatePartial
+            ├── group by: [c, b]
+            ├── aggregate functions: []
+            ├── estimated rows: 0.00
+            └── CacheScan
+                ├── output columns: [t1.c (#2), t1.b (#1)]
+                ├── cache index: 0
+                └── column indexes: [1, 2]
+
+query T
+explain select t1.a, v1.c1, v1.c2 from t1 left join lateral (values('t1_b', t1.b), ('t1_c', t1.c)) as v1 ("c1", "c2") on t1.a = v1.c2 left join lateral (values('t1_b', t1.b), ('t1_c', t1.c)) as v2 ("c1", "c2") on t1.a = v2.c2;
+----
+HashJoin
+├── output columns: [c1 (#3), c2 (#4), t1.a (#0)]
+├── join type: RIGHT OUTER
+├── build keys: [t1.a (#0), b (#1), c (#2)]
+├── probe keys: [v2.c2 (#9), b (#11), c (#12)]
+├── filters: []
+├── cache index: 1
+├── cache columns: [1, 2]
+├── estimated rows: 12.00
+├── HashJoin(Build)
+│   ├── output columns: [c1 (#3), c2 (#4), t1.a (#0), t1.b (#1), t1.c (#2)]
+│   ├── join type: RIGHT OUTER
+│   ├── build keys: [t1.a (#0), b (#1), c (#2)]
+│   ├── probe keys: [v1.c2 (#4), b (#6), c (#7)]
+│   ├── filters: []
+│   ├── cache index: 0
+│   ├── cache columns: [1, 2]
+│   ├── estimated rows: 6.00
+│   ├── TableScan(Build)
+│   │   ├── table: default.default.t1
+│   │   ├── output columns: [a (#0), b (#1), c (#2)]
+│   │   ├── read rows: 3
+│   │   ├── read size: < 1 KiB
+│   │   ├── partitions total: 1
+│   │   ├── partitions scanned: 1
+│   │   ├── pruning stats: [segments: <range pruning: 1 to 1>, blocks: <range pruning: 1 to 1>]
+│   │   ├── push downs: [filters: [], limit: NONE]
+│   │   └── estimated rows: 3.00
+│   └── ExpressionScan(Probe)
+│       ├── output columns: [c1 (#3), c2 (#4), b (#6), c (#7)]
+│       ├── column 0: ['t1_b', t1.b (#1), t1.b (#1), t1.c (#2)]
+│       ├── column 1: ['t1_c', t1.c (#2), t1.b (#1), t1.c (#2)]
+│       └── AggregateFinal
+│           ├── output columns: [t1.c (#2), t1.b (#1)]
+│           ├── group by: [c, b]
+│           ├── aggregate functions: []
+│           ├── estimated rows: 0.00
+│           └── AggregatePartial
+│               ├── group by: [c, b]
+│               ├── aggregate functions: []
+│               ├── estimated rows: 0.00
+│               └── CacheScan
+│                   ├── output columns: [t1.c (#2), t1.b (#1)]
+│                   ├── cache index: 0
+│                   └── column indexes: [1, 2]
+└── ExpressionScan(Probe)
+    ├── output columns: [c1 (#8), c2 (#9), b (#11), c (#12)]
+    ├── column 0: ['t1_b', t1.b (#1), t1.b (#1), t1.c (#2)]
+    ├── column 1: ['t1_c', t1.c (#2), t1.b (#1), t1.c (#2)]
+    └── AggregateFinal
+        ├── output columns: [t1.c (#2), t1.b (#1)]
+        ├── group by: [c, b]
+        ├── aggregate functions: []
+        ├── estimated rows: 0.00
+        └── AggregatePartial
+            ├── group by: [c, b]
+            ├── aggregate functions: []
+            ├── estimated rows: 0.00
+            └── CacheScan
+                ├── output columns: [t1.c (#2), t1.b (#1)]
+                ├── cache index: 1
+                └── column indexes: [1, 2]
+
+query T
+explain select t1.a, v1.c1, v1.c2 from t1 inner join lateral (values('t1_b', t1.b), ('t1_c', t1.c)) as v1 ("c1", "c2") on t1.a = v1.c2 inner join lateral (values('t1_b', t1.b), ('t1_c', v1.c1)) as v2 ("c1", "c2") on t1.a = v2.c2;
+----
+HashJoin
+├── output columns: [c1 (#3), c2 (#4), t1.a (#0)]
+├── join type: INNER
+├── build keys: [CAST(t1.a (#0) AS Int64 NULL), b (#1), c1 (#3)]
+├── probe keys: [v2.c2 (#9), b (#11), c1 (#13)]
+├── filters: []
+├── cache index: 1
+├── cache columns: [1, 3]
+├── estimated rows: 12.00
+├── HashJoin(Build)
+│   ├── output columns: [c1 (#3), c2 (#4), t1.a (#0), t1.b (#1)]
+│   ├── join type: INNER
+│   ├── build keys: [t1.a (#0), b (#1), c (#2)]
+│   ├── probe keys: [v1.c2 (#4), b (#6), c (#7)]
+│   ├── filters: []
+│   ├── cache index: 0
+│   ├── cache columns: [1, 2]
+│   ├── estimated rows: 6.00
+│   ├── TableScan(Build)
+│   │   ├── table: default.default.t1
+│   │   ├── output columns: [a (#0), b (#1), c (#2)]
+│   │   ├── read rows: 3
+│   │   ├── read size: < 1 KiB
+│   │   ├── partitions total: 1
+│   │   ├── partitions scanned: 1
+│   │   ├── pruning stats: [segments: <range pruning: 1 to 1>, blocks: <range pruning: 1 to 1>]
+│   │   ├── push downs: [filters: [], limit: NONE]
+│   │   └── estimated rows: 3.00
+│   └── ExpressionScan(Probe)
+│       ├── output columns: [c1 (#3), c2 (#4), b (#6), c (#7)]
+│       ├── column 0: ['t1_b', t1.b (#1), t1.b (#1), t1.c (#2)]
+│       ├── column 1: ['t1_c', t1.c (#2), t1.b (#1), t1.c (#2)]
+│       └── AggregateFinal
+│           ├── output columns: [t1.c (#2), t1.b (#1)]
+│           ├── group by: [c, b]
+│           ├── aggregate functions: []
+│           ├── estimated rows: 0.00
+│           └── AggregatePartial
+│               ├── group by: [c, b]
+│               ├── aggregate functions: []
+│               ├── estimated rows: 0.00
+│               └── CacheScan
+│                   ├── output columns: [t1.c (#2), t1.b (#1)]
+│                   ├── cache index: 0
+│                   └── column indexes: [1, 2]
+└── ExpressionScan(Probe)
+    ├── output columns: [c1 (#8), c2 (#9), b (#11), c1 (#13)]
+    ├── column 0: ['t1_b', CAST(t1.b (#1) AS Int64 NULL), t1.b (#1), v1.c1 (#3)]
+    ├── column 1: ['t1_c', CAST(v1.c1 (#3) AS Int64 NULL), t1.b (#1), v1.c1 (#3)]
+    └── AggregateFinal
+        ├── output columns: [c1 (#3), t1.b (#1)]
+        ├── group by: [c1, b]
+        ├── aggregate functions: []
+        ├── estimated rows: 0.00
+        └── AggregatePartial
+            ├── group by: [c1, b]
+            ├── aggregate functions: []
+            ├── estimated rows: 0.00
+            └── CacheScan
+                ├── output columns: [c1 (#3), t1.b (#1)]
+                ├── cache index: 1
+                └── column indexes: [1, 3]
diff --git a/tests/sqllogictests/suites/query/lateral.test b/tests/sqllogictests/suites/query/lateral.test
index 836cc65ed96ba..31ecc26c22b5c 100644
--- a/tests/sqllogictests/suites/query/lateral.test
+++ b/tests/sqllogictests/suites/query/lateral.test
@@ -170,5 +170,51 @@ SELECT u.user_id, f.value from
 3 "climbing"
 3 "writing"
 
+statement ok
+drop table if exists t1;
+
+statement ok
+drop table if exists t2;
+
+statement ok
+create table t1(a int, b int, c int);
+
+statement ok
+create table t2(a int, b int, c int);
+
+statement ok
+insert into t1 values (1, 10, 11), (10, 20, 111), (2, 20, 200);
+
+statement ok
+insert into t2 values (1, 10, 22), (2222, 10, 22), (3, 20, 222);
+
+query I
+select t1.a, t1.b from t1 join lateral (values(t1.b)) as v1 ("c1") on t1.b = v1.c1 order by t1.a, t1.b;
+----
+1 10
+2 20
+10 20
+
+query I
+select t1.a, t1.b from t1 join lateral (values(t1.b), (t1.c)) as v1 ("c1") on t1.b = v1.c1 order by t1.a, t1.b;
+----
+1 10
+2 20
+10 20
+
+query T
+select t1.a, v1.c1, v1.c2 from t1 left join lateral (values('t1_b', t1.b), ('t1_c', t1.c)) as v1 ("c1", "c2") on t1.a = v1.c2 order by t1.a, v1.c1, v1.c2;
+----
+1 NULL NULL
+2 NULL NULL
+10 NULL NULL
+
+query T
+select t1.a, v1.c1, v1.c2 from t1 left join lateral (values('t1_b', t1.b), ('t1_c', t1.c)) as v1 ("c1", "c2") on t1.a = v1.c2 left join lateral (values('t1_b', t1.b), ('t1_c', t1.c)) as v2 ("c1", "c2") on t1.a = v2.c2 order by t1.a, v1.c1, v1.c2;
+----
+1 NULL NULL
+2 NULL NULL
+10 NULL NULL
+
 statement ok
 drop database test_lateral

From 3109054a4dfea22009026479bd9a21d87eff13e3 Mon Sep 17 00:00:00 2001
From: TCeason <33082201+TCeason@users.noreply.github.com>
Date: Sat, 25 May 2024 20:20:56 +0800
Subject: [PATCH 2/3] fix(query): delete len check in
 mget_database/table_names_by_ids (#15640)

---
 src/meta/api/src/schema_api_impl.rs           | 44 ++++++-------------
 .../base/05_ddl/05_0017_ddl_grant_role.test   | 12 +++++
 2 files changed, 26 insertions(+), 30 deletions(-)

diff --git a/src/meta/api/src/schema_api_impl.rs b/src/meta/api/src/schema_api_impl.rs
index 8191c2e7c5ba8..a1b2a54c23cb7 100644
--- a/src/meta/api/src/schema_api_impl.rs
+++ b/src/meta/api/src/schema_api_impl.rs
@@ -190,7 +190,6 @@ use databend_common_meta_kvapi::kvapi;
 use databend_common_meta_kvapi::kvapi::DirName;
 use databend_common_meta_kvapi::kvapi::Key;
 use databend_common_meta_kvapi::kvapi::UpsertKVReq;
-use databend_common_meta_types::anyerror::AnyError;
 use databend_common_meta_types::protobuf as pb;
 use databend_common_meta_types::txn_op::Request;
 use databend_common_meta_types::txn_op_response::Response;
@@ -198,9 +197,6 @@ use databend_common_meta_types::ConditionResult;
 use databend_common_meta_types::InvalidReply;
 use databend_common_meta_types::MatchSeq;
 use databend_common_meta_types::MatchSeqExt;
-use databend_common_meta_types::MetaAPIError;
-use databend_common_meta_types::MetaDataError;
-use databend_common_meta_types::MetaDataReadError;
 use databend_common_meta_types::MetaError;
 use databend_common_meta_types::MetaId;
 use databend_common_meta_types::MetaNetworkError;
@@ -2288,10 +2284,13 @@ impl<KV: kvapi::KVApi<Error = MetaError> + ?Sized> SchemaApi for KV {
         let seq_names = self.mget_kv(&id_name_kv_keys).await?;
         let mut table_names = Vec::with_capacity(table_ids.len());
 
-        // None means table_name not found, maybe immutable table id. Ignore it
-        for seq_name in seq_names.into_iter().flatten() {
-            let name_ident: DBIdTableName = deserialize_struct(&seq_name.data)?;
-            table_names.push(Some(name_ident.table_name));
+        for seq_name in seq_names {
+            if let Some(seq_name) = seq_name {
+                let name_ident: DBIdTableName = deserialize_struct(&seq_name.data)?;
+                table_names.push(Some(name_ident.table_name));
+            } else {
+                table_names.push(None);
+            }
         }
 
         let mut meta_kv_keys = Vec::with_capacity(table_ids.len());
@@ -2301,15 +2300,6 @@ impl<KV: kvapi::KVApi<Error = MetaError> + ?Sized> SchemaApi for KV {
         }
 
         let seq_metas = self.mget_kv(&meta_kv_keys).await?;
-        if seq_metas.len() != table_names.len() {
-            return Err(KVAppError::MetaError(MetaError::APIError(
-                MetaAPIError::DataError(MetaDataError::ReadError(MetaDataReadError::new(
-                    "mget_table_names_by_ids",
-                    "",
-                    &AnyError::error("The system is experiencing high load, please retry later"),
-                ))),
-            )));
-        }
         for (i, seq_meta_opt) in seq_metas.iter().enumerate() {
             if let Some(seq_meta) = seq_meta_opt {
                 let table_meta: TableMeta = deserialize_struct(&seq_meta.data)?;
@@ -2362,10 +2352,13 @@ impl<KV: kvapi::KVApi<Error = MetaError> + ?Sized> SchemaApi for KV {
         // If multi drop/create db the capacity may not same
         let mut db_names = Vec::with_capacity(db_ids.len());
 
-        // None means db_name not found, maybe immutable database id. Ignore it
-        for seq_name in seq_names.into_iter().flatten() {
-            let name_ident: DatabaseNameIdentRaw = deserialize_struct(&seq_name.data)?;
-            db_names.push(Some(name_ident.database_name().to_string()));
+        for seq_name in seq_names {
+            if let Some(seq_name) = seq_name {
+                let name_ident: DatabaseNameIdentRaw = deserialize_struct(&seq_name.data)?;
+                db_names.push(Some(name_ident.database_name().to_string()));
+            } else {
+                db_names.push(None);
+            }
         }
 
         let mut meta_kv_keys = Vec::with_capacity(db_ids.len());
@@ -2375,15 +2368,6 @@ impl<KV: kvapi::KVApi<Error = MetaError> + ?Sized> SchemaApi for KV {
         }
 
         let seq_metas = self.mget_kv(&meta_kv_keys).await?;
-        if seq_metas.len() != db_names.len() {
-            return Err(KVAppError::MetaError(MetaError::APIError(
-                MetaAPIError::DataError(MetaDataError::ReadError(MetaDataReadError::new(
-                    "mget_table_names_by_ids",
-                    "",
-                    &AnyError::error("The system is experiencing high load, please retry later"),
-                ))),
-            )));
-        }
         for (i, seq_meta_opt) in seq_metas.iter().enumerate() {
             if let Some(seq_meta) = seq_meta_opt {
                 let db_meta: DatabaseMeta = deserialize_struct(&seq_meta.data)?;
diff --git a/tests/sqllogictests/suites/base/05_ddl/05_0017_ddl_grant_role.test b/tests/sqllogictests/suites/base/05_ddl/05_0017_ddl_grant_role.test
index 818024a12bf3f..ebe5fdb8bedf6 100644
--- a/tests/sqllogictests/suites/base/05_ddl/05_0017_ddl_grant_role.test
+++ b/tests/sqllogictests/suites/base/05_ddl/05_0017_ddl_grant_role.test
@@ -181,3 +181,15 @@ drop function if exists isnotempty;
 
 statement ok
 drop user if exists u_05_0017;
+
+statement ok
+drop user if exists 'u@05_0017';
+
+statement ok
+create user 'u@05_0017' identified by '123';
+
+statement ok
+show grants for 'u@05_0017';
+
+statement ok
+drop user if exists 'u@05_0017';

From a7bc773fc5192d695f7d01373ce0f158972fc857 Mon Sep 17 00:00:00 2001
From: zhya <mytesla@live.com>
Date: Sat, 25 May 2024 23:14:46 +0800
Subject: [PATCH 3/3] chore: add setting compact_max_block_selection (#15641)

---
 src/query/settings/src/settings_default.rs            |  6 ++++++
 src/query/settings/src/settings_getter_setter.rs      |  8 ++++++++
 .../storages/fuse/src/operations/mutation/mod.rs      |  1 -
 .../mutation/mutator/block_compact_mutator.rs         | 11 +++++++----
 4 files changed, 21 insertions(+), 5 deletions(-)

diff --git a/src/query/settings/src/settings_default.rs b/src/query/settings/src/settings_default.rs
index c47f9d1555d13..0a7d4e7f07374 100644
--- a/src/query/settings/src/settings_default.rs
+++ b/src/query/settings/src/settings_default.rs
@@ -576,6 +576,12 @@ impl DefaultSettings {
                     mode: SettingMode::Both,
                     range: Some(SettingRange::Numeric(0..=u64::MAX)),
                 }),
+                ("compact_max_block_selection", DefaultSettingValue {
+                    value: UserSettingValue::UInt64(10000),
+                    desc: "Limits the maximum number of blocks that can be selected during a compact operation.",
+                    mode: SettingMode::Both,
+                    range: Some(SettingRange::Numeric(2..=u64::MAX)),
+                }),
                 ("enable_distributed_recluster", DefaultSettingValue {
                     value: UserSettingValue::UInt64(0),
                     desc: "Enable distributed execution of table recluster.",
diff --git a/src/query/settings/src/settings_getter_setter.rs b/src/query/settings/src/settings_getter_setter.rs
index 2e1d002de8f68..63ad26148da59 100644
--- a/src/query/settings/src/settings_getter_setter.rs
+++ b/src/query/settings/src/settings_getter_setter.rs
@@ -508,6 +508,14 @@ impl Settings {
         self.try_get_u64("recluster_block_size")
     }
 
+    pub fn set_compact_max_block_selection(&self, val: u64) -> Result<()> {
+        self.try_set_u64("compact_max_block_selection", val)
+    }
+
+    pub fn get_compact_max_block_selection(&self) -> Result<u64> {
+        self.try_get_u64("compact_max_block_selection")
+    }
+
     pub fn get_enable_distributed_recluster(&self) -> Result<bool> {
         Ok(self.try_get_u64("enable_distributed_recluster")? != 0)
     }
diff --git a/src/query/storages/fuse/src/operations/mutation/mod.rs b/src/query/storages/fuse/src/operations/mutation/mod.rs
index 28db295a87d3a..99205f85d3396 100644
--- a/src/query/storages/fuse/src/operations/mutation/mod.rs
+++ b/src/query/storages/fuse/src/operations/mutation/mod.rs
@@ -20,6 +20,5 @@ pub use meta::*;
 pub use mutator::*;
 pub use processors::*;
 
-pub static MAX_BLOCK_COUNT: usize = 10_000;
 pub type SegmentIndex = usize;
 pub type BlockIndex = usize;
diff --git a/src/query/storages/fuse/src/operations/mutation/mutator/block_compact_mutator.rs b/src/query/storages/fuse/src/operations/mutation/mutator/block_compact_mutator.rs
index bd2acdc77e8e9..dd78404d195f1 100644
--- a/src/query/storages/fuse/src/operations/mutation/mutator/block_compact_mutator.rs
+++ b/src/query/storages/fuse/src/operations/mutation/mutator/block_compact_mutator.rs
@@ -45,7 +45,6 @@ use crate::operations::mutation::CompactExtraInfo;
 use crate::operations::mutation::CompactLazyPartInfo;
 use crate::operations::mutation::CompactTaskInfo;
 use crate::operations::mutation::SegmentIndex;
-use crate::operations::mutation::MAX_BLOCK_COUNT;
 use crate::operations::CompactOptions;
 use crate::statistics::reducers::merge_statistics_mut;
 use crate::statistics::sort_by_cluster_stats;
@@ -84,6 +83,11 @@ impl BlockCompactMutator {
         let snapshot = self.compact_params.base_snapshot.clone();
         let segment_locations = &snapshot.segments;
         let number_segments = segment_locations.len();
+
+        let settings = self.ctx.get_settings();
+        let compact_max_block_selection = settings.get_compact_max_block_selection()? as usize;
+        let max_threads = settings.get_max_threads()? as usize;
+
         let num_segment_limit = self
             .compact_params
             .num_segment_limit
@@ -91,7 +95,7 @@ impl BlockCompactMutator {
         let num_block_limit = self
             .compact_params
             .num_block_limit
-            .unwrap_or(MAX_BLOCK_COUNT);
+            .unwrap_or(compact_max_block_selection);
 
         info!("block compaction limits: seg {num_segment_limit},  block {num_block_limit}");
 
@@ -112,7 +116,7 @@ impl BlockCompactMutator {
         let mut segment_idx = 0;
         let mut is_end = false;
         let mut parts = Vec::new();
-        let chunk_size = self.ctx.get_settings().get_max_threads()? as usize * 4;
+        let chunk_size = max_threads * 4;
         for chunk in segment_locations.chunks(chunk_size) {
             // Read the segments information in parallel.
             let mut segment_infos = segments_io
@@ -194,7 +198,6 @@ impl BlockCompactMutator {
         metrics_inc_compact_block_build_lazy_part_milliseconds(elapsed_time.as_millis() as u64);
 
         let cluster = self.ctx.get_cluster();
-        let max_threads = self.ctx.get_settings().get_max_threads()? as usize;
         let partitions = if cluster.is_empty() || parts.len() < cluster.nodes.len() * max_threads {
             // NOTE: The snapshot schema does not contain the stream column.
             let column_ids = self