You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by ga...@apache.org on 2022/12/02 09:11:42 UTC

[doris] branch master updated: [Pipeline](exec) Support pipeline exec engine (#14736)

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

gabriellee pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/doris.git


The following commit(s) were added to refs/heads/master by this push:
     new 12304bc0ee [Pipeline](exec) Support pipeline exec engine (#14736)
12304bc0ee is described below

commit 12304bc0eec9abe0edaa8e0ed3b497f629270ce9
Author: HappenLee <ha...@hotmail.com>
AuthorDate: Fri Dec 2 17:11:34 2022 +0800

    [Pipeline](exec) Support pipeline exec engine (#14736)
    
    Co-authored-by: Lijia Liu <li...@yeah.net>
    Co-authored-by: HappenLee <ha...@hotmail.com>
    Co-authored-by: Jerry Hu <mr...@gmail.com>
    Co-authored-by: Pxl <95...@qq.com>
    Co-authored-by: shee <13...@users.noreply.github.com>
    Co-authored-by: Gabriel <ga...@gmail.com>
    
    ## Problem Summary:
    
    ### 1. Design
    
    DSIP: https://cwiki.apache.org/confluence/display/DORIS/DSIP-027%3A+Support+Pipeline+Exec+Engine
    
    ### 2. How to use:
    
    Set the environment variable `set enable_pipeline_engine = true; `
---
 be/CMakeLists.txt                                  |   2 +
 be/src/common/config.h                             |   2 +
 be/src/exec/data_sink.cpp                          |  24 +-
 be/src/exec/data_sink.h                            |   4 +-
 be/src/exec/exec_node.cpp                          |  69 ++-
 be/src/exec/exec_node.h                            |  52 +-
 be/src/exec/table_function_node.cpp                |  25 +-
 be/src/exec/table_function_node.h                  |  21 +-
 be/src/pipeline/CMakeLists.txt                     |  48 ++
 be/src/pipeline/exec/agg_context.cpp               | 106 ++++
 be/src/pipeline/exec/agg_context.h                 |  72 +++
 be/src/pipeline/exec/aggregation_sink_operator.cpp |  77 +++
 be/src/pipeline/exec/aggregation_sink_operator.h   |  65 +++
 .../pipeline/exec/aggregation_source_operator.cpp  |  67 +++
 .../exec/aggregation_source_operator.h}            |  41 +-
 be/src/pipeline/exec/empty_set_operator.cpp        |  44 ++
 be/src/pipeline/exec/empty_set_operator.h          |  61 +++
 be/src/pipeline/exec/exchange_sink_buffer.cpp      | 236 +++++++++
 be/src/pipeline/exec/exchange_sink_buffer.h        |  86 ++++
 be/src/pipeline/exec/exchange_sink_operator.cpp    |  97 ++++
 be/src/pipeline/exec/exchange_sink_operator.h      |  75 +++
 be/src/pipeline/exec/exchange_source_operator.cpp  |  63 +++
 be/src/pipeline/exec/exchange_source_operator.h    |  55 ++
 .../exec/olap_scan_operator.cpp}                   |  24 +-
 .../exec/olap_scan_operator.h}                     |  39 +-
 be/src/pipeline/exec/operator.cpp                  |  91 ++++
 be/src/pipeline/exec/operator.h                    | 198 +++++++
 be/src/pipeline/exec/repeat_operator.cpp           |  76 +++
 .../exec/repeat_operator.h}                        |  44 +-
 be/src/pipeline/exec/result_sink_operator.cpp      |  69 +++
 be/src/pipeline/exec/result_sink_operator.h        |  69 +++
 be/src/pipeline/exec/scan_operator.cpp             |  68 +++
 be/src/pipeline/exec/scan_operator.h               |  59 +++
 be/src/pipeline/exec/sort_sink_operator.cpp        |  51 ++
 be/src/pipeline/exec/sort_sink_operator.h          |  70 +++
 be/src/pipeline/exec/sort_source_operator.cpp      |  54 ++
 be/src/pipeline/exec/sort_source_operator.h        |  68 +++
 .../exec/streaming_aggregation_sink_operator.cpp   | 101 ++++
 .../exec/streaming_aggregation_sink_operator.h     |  75 +++
 .../exec/streaming_aggregation_source_operator.cpp |  86 ++++
 .../exec/streaming_aggregation_source_operator.h   |  56 ++
 be/src/pipeline/exec/table_function_operator.h     |  92 ++++
 be/src/pipeline/pipeline.cpp                       |  76 +++
 be/src/pipeline/pipeline.h                         |  90 ++++
 be/src/pipeline/pipeline_fragment_context.cpp      | 569 +++++++++++++++++++++
 be/src/pipeline/pipeline_fragment_context.h        | 127 +++++
 be/src/pipeline/pipeline_task.cpp                  | 219 ++++++++
 be/src/pipeline/pipeline_task.h                    | 204 ++++++++
 be/src/pipeline/task_scheduler.cpp                 | 327 ++++++++++++
 be/src/pipeline/task_scheduler.h                   | 270 ++++++++++
 be/src/runtime/buffer_control_block.cpp            |   9 +-
 be/src/runtime/buffer_control_block.h              |  37 +-
 be/src/runtime/exec_env.h                          |   7 +
 be/src/runtime/exec_env_init.cpp                   |  15 +
 be/src/runtime/fragment_mgr.cpp                    | 163 +++++-
 be/src/runtime/fragment_mgr.h                      |  20 +-
 be/src/runtime/plan_fragment_executor.cpp          |   6 +-
 be/src/runtime/query_fragments_ctx.h               |  12 +
 be/src/runtime/result_buffer_mgr.cpp               |  15 +-
 be/src/runtime/result_buffer_mgr.h                 |   6 +-
 be/src/runtime/result_file_sink.cpp                |   3 +-
 be/src/runtime/result_sink.cpp                     |   4 +-
 be/src/runtime/runtime_state.h                     |   7 +-
 be/src/service/backend_service.cpp                 |   2 +-
 be/src/service/internal_service.cpp                |  11 +-
 be/src/util/proto_util.h                           |  66 ++-
 be/src/util/runtime_profile.cpp                    |  14 +
 be/src/util/runtime_profile.h                      |  19 +-
 be/src/vec/exec/scan/new_olap_scan_node.cpp        |   5 +-
 be/src/vec/exec/scan/new_olap_scan_node.h          |   5 +
 be/src/vec/exec/scan/pip_scanner_context.h         |  47 ++
 be/src/vec/exec/scan/scanner_context.cpp           |  33 +-
 be/src/vec/exec/scan/scanner_context.h             |  33 +-
 be/src/vec/exec/scan/vscan_node.cpp                |  20 +-
 be/src/vec/exec/scan/vscan_node.h                  |   7 +-
 be/src/vec/exec/vaggregation_node.cpp              |  95 ++--
 be/src/vec/exec/vaggregation_node.h                |  33 +-
 be/src/vec/exec/vexchange_node.cpp                 |  32 +-
 be/src/vec/exec/vexchange_node.h                   |   6 +
 be/src/vec/exec/vrepeat_node.cpp                   | 122 +++--
 be/src/vec/exec/vrepeat_node.h                     |  33 +-
 be/src/vec/exec/vsort_node.cpp                     |  65 ++-
 be/src/vec/exec/vsort_node.h                       |  10 +-
 be/src/vec/exec/vtable_function_node.cpp           |  34 +-
 be/src/vec/exec/vtable_function_node.h             |  24 +
 be/src/vec/runtime/vdata_stream_mgr.cpp            |   4 +-
 be/src/vec/runtime/vdata_stream_recvr.cpp          |  40 +-
 be/src/vec/runtime/vdata_stream_recvr.h            |  89 +++-
 be/src/vec/runtime/vsorted_run_merger.cpp          |   4 +-
 be/src/vec/sink/vdata_stream_sender.cpp            |  98 ++--
 be/src/vec/sink/vdata_stream_sender.h              | 112 +++-
 be/src/vec/sink/vjdbc_table_sink.cpp               |   2 +-
 be/src/vec/sink/vjdbc_table_sink.h                 |   2 +-
 be/src/vec/sink/vmysql_result_writer.cpp           |   4 +
 be/src/vec/sink/vmysql_result_writer.h             |   2 +
 be/src/vec/sink/vmysql_table_sink.cpp              |   2 +-
 be/src/vec/sink/vmysql_table_sink.h                |   2 +-
 be/src/vec/sink/vodbc_table_sink.cpp               |   2 +-
 be/src/vec/sink/vodbc_table_sink.h                 |   2 +-
 be/src/vec/sink/vresult_file_sink.cpp              |   4 +-
 be/src/vec/sink/vresult_file_sink.h                |   2 +-
 be/src/vec/sink/vresult_sink.cpp                   |   9 +-
 be/src/vec/sink/vresult_sink.h                     |   6 +-
 be/src/vec/sink/vresult_writer.h                   |   2 +
 be/src/vec/sink/vtable_sink.cpp                    |   2 +-
 be/src/vec/sink/vtable_sink.h                      |   2 +-
 be/src/vec/sink/vtablet_sink.cpp                   |   2 +-
 be/src/vec/sink/vtablet_sink.h                     |   2 +-
 be/test/runtime/fragment_mgr_test.cpp              |   3 -
 be/test/runtime/result_buffer_mgr_test.cpp         |  12 +-
 be/test/vec/runtime/vdata_stream_test.cpp          |   5 +-
 .../java/org/apache/doris/qe/SessionVariable.java  |  14 +
 gensrc/thrift/PaloInternalService.thrift           |   2 +
 113 files changed, 5651 insertions(+), 465 deletions(-)

diff --git a/be/CMakeLists.txt b/be/CMakeLists.txt
index 28ef74079d..6113cf2cdb 100644
--- a/be/CMakeLists.txt
+++ b/be/CMakeLists.txt
@@ -680,6 +680,7 @@ set(DORIS_LINK_LIBS
     Webserver
     Geo
     Vec
+    Pipeline
     ${WL_END_GROUP}
 )
 
@@ -910,6 +911,7 @@ endif()
 
 add_subdirectory(${SRC_DIR}/util)
 add_subdirectory(${SRC_DIR}/vec)
+add_subdirectory(${SRC_DIR}/pipeline)
 
 if (${MAKE_TEST} STREQUAL "ON")
     add_subdirectory(${TEST_DIR})
diff --git a/be/src/common/config.h b/be/src/common/config.h
index 9faf33ef00..a5e0c4826c 100644
--- a/be/src/common/config.h
+++ b/be/src/common/config.h
@@ -846,6 +846,8 @@ CONF_Bool(enable_java_support, "true");
 // Set config randomly to check more issues in github workflow
 CONF_Bool(enable_fuzzy_mode, "false");
 
+CONF_Int32(pipeline_executor_size, "0");
+
 #ifdef BE_TEST
 // test s3
 CONF_String(test_s3_resource, "resource");
diff --git a/be/src/exec/data_sink.cpp b/be/src/exec/data_sink.cpp
index 0c8842ad32..8362e89d7b 100644
--- a/be/src/exec/data_sink.cpp
+++ b/be/src/exec/data_sink.cpp
@@ -47,7 +47,7 @@ namespace doris {
 Status DataSink::create_data_sink(ObjectPool* pool, const TDataSink& thrift_sink,
                                   const std::vector<TExpr>& output_exprs,
                                   const TPlanFragmentExecParams& params,
-                                  const RowDescriptor& row_desc, bool is_vec,
+                                  const RowDescriptor& row_desc, RuntimeState* state,
                                   std::unique_ptr<DataSink>* sink, DescriptorTbl& desc_tbl) {
     DataSink* tmp_sink = nullptr;
 
@@ -61,10 +61,10 @@ Status DataSink::create_data_sink(ObjectPool* pool, const TDataSink& thrift_sink
                         ? params.send_query_statistics_with_every_batch
                         : false;
         // TODO: figure out good buffer size based on size of output row
-        if (is_vec) {
-            tmp_sink = new doris::vectorized::VDataStreamSender(
-                    pool, params.sender_id, row_desc, thrift_sink.stream_sink, params.destinations,
-                    16 * 1024, send_query_statistics_with_every_batch);
+        if (state->enable_vectorized_exec()) {
+            tmp_sink = new vectorized::VDataStreamSender(
+                    state, pool, params.sender_id, row_desc, thrift_sink.stream_sink,
+                    params.destinations, 16 * 1024, send_query_statistics_with_every_batch);
         } else {
             tmp_sink = new DataStreamSender(pool, params.sender_id, row_desc,
                                             thrift_sink.stream_sink, params.destinations, 16 * 1024,
@@ -80,7 +80,7 @@ Status DataSink::create_data_sink(ObjectPool* pool, const TDataSink& thrift_sink
         }
 
         // TODO: figure out good buffer size based on size of output row
-        if (is_vec) {
+        if (state->enable_vectorized_exec()) {
             tmp_sink = new doris::vectorized::VResultSink(row_desc, output_exprs,
                                                           thrift_sink.result_sink, 4096);
         } else {
@@ -95,7 +95,7 @@ Status DataSink::create_data_sink(ObjectPool* pool, const TDataSink& thrift_sink
         }
 
         // TODO: figure out good buffer size based on size of output row
-        if (is_vec) {
+        if (state->enable_vectorized_exec()) {
             bool send_query_statistics_with_every_batch =
                     params.__isset.send_query_statistics_with_every_batch
                             ? params.send_query_statistics_with_every_batch
@@ -139,7 +139,7 @@ Status DataSink::create_data_sink(ObjectPool* pool, const TDataSink& thrift_sink
         if (!thrift_sink.__isset.mysql_table_sink) {
             return Status::InternalError("Missing data buffer sink.");
         }
-        if (is_vec) {
+        if (state->enable_vectorized_exec()) {
             doris::vectorized::VMysqlTableSink* vmysql_tbl_sink =
                     new doris::vectorized::VMysqlTableSink(pool, row_desc, output_exprs);
             sink->reset(vmysql_tbl_sink);
@@ -158,7 +158,7 @@ Status DataSink::create_data_sink(ObjectPool* pool, const TDataSink& thrift_sink
         if (!thrift_sink.__isset.odbc_table_sink) {
             return Status::InternalError("Missing data odbc sink.");
         }
-        if (is_vec) {
+        if (state->enable_vectorized_exec()) {
             sink->reset(new vectorized::VOdbcTableSink(pool, row_desc, output_exprs));
         } else {
             sink->reset(new OdbcTableSink(pool, row_desc, output_exprs));
@@ -170,7 +170,7 @@ Status DataSink::create_data_sink(ObjectPool* pool, const TDataSink& thrift_sink
         if (!thrift_sink.__isset.jdbc_table_sink) {
             return Status::InternalError("Missing data jdbc sink.");
         }
-        if (is_vec) {
+        if (state->enable_vectorized_exec()) {
             if (config::enable_java_support) {
                 sink->reset(new vectorized::VJdbcTableSink(pool, row_desc, output_exprs));
             } else {
@@ -196,7 +196,7 @@ Status DataSink::create_data_sink(ObjectPool* pool, const TDataSink& thrift_sink
     case TDataSinkType::OLAP_TABLE_SINK: {
         Status status;
         DCHECK(thrift_sink.__isset.olap_table_sink);
-        if (is_vec) {
+        if (state->enable_vectorized_exec()) {
             sink->reset(new stream_load::VOlapTableSink(pool, row_desc, output_exprs, &status));
         } else {
             sink->reset(new stream_load::OlapTableSink(pool, row_desc, output_exprs, &status));
@@ -220,7 +220,7 @@ Status DataSink::create_data_sink(ObjectPool* pool, const TDataSink& thrift_sink
     }
     }
 
-    if (sink->get() != nullptr) {
+    if (*sink != nullptr) {
         RETURN_IF_ERROR((*sink)->init(thrift_sink));
     }
 
diff --git a/be/src/exec/data_sink.h b/be/src/exec/data_sink.h
index 423e60a74a..9f21bcf4a1 100644
--- a/be/src/exec/data_sink.h
+++ b/be/src/exec/data_sink.h
@@ -62,7 +62,7 @@ public:
     virtual Status send(RuntimeState* state, RowBatch* batch) = 0;
 
     // Send a Block into this sink.
-    virtual Status send(RuntimeState* state, vectorized::Block* block) {
+    virtual Status send(RuntimeState* state, vectorized::Block* block, bool eos = false) {
         return Status::NotSupported("Not support send block");
     };
     // Releases all resources that were allocated in prepare()/send().
@@ -80,7 +80,7 @@ public:
     static Status create_data_sink(ObjectPool* pool, const TDataSink& thrift_sink,
                                    const std::vector<TExpr>& output_exprs,
                                    const TPlanFragmentExecParams& params,
-                                   const RowDescriptor& row_desc, bool is_vec,
+                                   const RowDescriptor& row_desc, RuntimeState* state,
                                    std::unique_ptr<DataSink>* sink, DescriptorTbl& desc_tbl);
 
     // Returns the runtime profile for the sink.
diff --git a/be/src/exec/exec_node.cpp b/be/src/exec/exec_node.cpp
index 29462ac037..14b8690d5d 100644
--- a/be/src/exec/exec_node.cpp
+++ b/be/src/exec/exec_node.cpp
@@ -88,10 +88,6 @@ namespace doris {
 
 const std::string ExecNode::ROW_THROUGHPUT_COUNTER = "RowsReturnedRate";
 
-int ExecNode::get_node_id_from_profile(RuntimeProfile* p) {
-    return p->metadata();
-}
-
 ExecNode::RowBatchQueue::RowBatchQueue(int max_batches) : BlockingQueue<RowBatch*>(max_batches) {}
 
 ExecNode::RowBatchQueue::~RowBatchQueue() {
@@ -150,7 +146,8 @@ ExecNode::ExecNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl
           _rows_returned_rate(nullptr),
           _memory_used_counter(nullptr),
           _get_next_span(),
-          _is_closed(false) {
+          _is_closed(false),
+          _ref(0) {
     if (tnode.__isset.output_tuple_id) {
         _output_row_descriptor.reset(new RowDescriptor(descs, {tnode.output_tuple_id}, {true}));
     }
@@ -251,7 +248,7 @@ Status ExecNode::prepare(RuntimeState* state) {
     return Status::OK();
 }
 
-Status ExecNode::open(RuntimeState* state) {
+Status ExecNode::alloc_resource(doris::RuntimeState* state) {
     SCOPED_CONSUME_MEM_TRACKER(mem_tracker_growh());
     if (_vconjunct_ctx_ptr) {
         RETURN_IF_ERROR((*_vconjunct_ctx_ptr)->open(state));
@@ -264,6 +261,10 @@ Status ExecNode::open(RuntimeState* state) {
     }
 }
 
+Status ExecNode::open(RuntimeState* state) {
+    return alloc_resource(state);
+}
+
 Status ExecNode::reset(RuntimeState* state) {
     _num_rows_returned = 0;
     for (int i = 0; i < _children.size(); ++i) {
@@ -280,16 +281,35 @@ Status ExecNode::collect_query_statistics(QueryStatistics* statistics) {
     return Status::OK();
 }
 
+void ExecNode::release_resource(doris::RuntimeState* state) {
+    if (!_is_resource_released) {
+        if (_rows_returned_counter != nullptr) {
+            COUNTER_SET(_rows_returned_counter, _num_rows_returned);
+        }
+
+        if (_vconjunct_ctx_ptr) {
+            (*_vconjunct_ctx_ptr)->close(state);
+        }
+        if (typeid(*this) != typeid(doris::vectorized::NewOlapScanNode)) {
+            Expr::close(_conjunct_ctxs, state);
+        }
+        vectorized::VExpr::close(_projections, state);
+
+        if (_buffer_pool_client.is_registered()) {
+            state->exec_env()->buffer_pool()->DeregisterClient(&_buffer_pool_client);
+        }
+
+        runtime_profile()->add_to_span();
+        _is_resource_released = true;
+    }
+}
+
 Status ExecNode::close(RuntimeState* state) {
     if (_is_closed) {
         return Status::OK();
     }
     _is_closed = true;
 
-    if (_rows_returned_counter != nullptr) {
-        COUNTER_SET(_rows_returned_counter, _num_rows_returned);
-    }
-
     Status result;
     for (int i = 0; i < _children.size(); ++i) {
         auto st = _children[i]->close(state);
@@ -297,21 +317,7 @@ Status ExecNode::close(RuntimeState* state) {
             result = st;
         }
     }
-
-    if (_vconjunct_ctx_ptr) {
-        (*_vconjunct_ctx_ptr)->close(state);
-    }
-    if (typeid(*this) != typeid(doris::vectorized::NewOlapScanNode)) {
-        Expr::close(_conjunct_ctxs, state);
-    }
-    vectorized::VExpr::close(_projections, state);
-
-    if (_buffer_pool_client.is_registered()) {
-        state->exec_env()->buffer_pool()->DeregisterClient(&_buffer_pool_client);
-    }
-
-    runtime_profile()->add_to_span();
-
+    release_resource(state);
     return result;
 }
 
@@ -855,4 +861,17 @@ Status ExecNode::get_next_after_projects(RuntimeState* state, vectorized::Block*
     return get_next(state, block, eos);
 }
 
+Status ExecNode::execute(RuntimeState* state, vectorized::Block* input_block,
+                         vectorized::Block* output_block, bool* eos) {
+    return Status::NotSupported("{} not implements execute", get_name());
+}
+
+Status ExecNode::pull(RuntimeState* state, vectorized::Block* output_block, bool* eos) {
+    return Status::NotSupported("{} not implements pull", get_name());
+}
+
+Status ExecNode::sink(RuntimeState* state, vectorized::Block* input_block, bool eos) {
+    return Status::NotSupported("{} not implements sink", get_name());
+}
+
 } // namespace doris
diff --git a/be/src/exec/exec_node.h b/be/src/exec/exec_node.h
index 7f5ba74656..4d02c7df45 100644
--- a/be/src/exec/exec_node.h
+++ b/be/src/exec/exec_node.h
@@ -52,6 +52,12 @@ class Block;
 class VExpr;
 } // namespace vectorized
 
+namespace pipeline {
+class PipelineFragmentContext;
+class Pipeline;
+class Operator;
+} // namespace pipeline
+
 using std::string;
 using std::stringstream;
 using std::vector;
@@ -89,6 +95,11 @@ public:
     // Caller must not be holding any io buffers. This will cause deadlock.
     virtual Status open(RuntimeState* state);
 
+    // Alloc and open resource for the node
+    // Only pipeline operator use exec node need to impl the virtual function
+    // so only vectorized exec node need to impl
+    virtual Status alloc_resource(RuntimeState* state);
+
     // Retrieves rows and returns them via row_batch. Sets eos to true
     // if subsequent calls will not retrieve any more rows.
     // Data referenced by any tuples returned in row_batch must not be overwritten
@@ -104,10 +115,29 @@ public:
     // TODO: AggregationNode and HashJoinNode cannot be "re-opened" yet.
     virtual Status get_next(RuntimeState* state, RowBatch* row_batch, bool* eos);
     virtual Status get_next(RuntimeState* state, vectorized::Block* block, bool* eos);
-
     // new interface to compatible new optimizers in FE
     Status get_next_after_projects(RuntimeState* state, vectorized::Block* block, bool* eos);
 
+    // Process data
+    // Eg: Projection, Union All, HashProbe
+    virtual Status execute(RuntimeState* state, vectorized::Block* input_block,
+                           vectorized::Block* output_block, bool* eos);
+
+    // Emit data, both need impl with method: sink
+    // Eg: Aggregation, Sort
+    virtual Status pull(RuntimeState* state, vectorized::Block* output_block, bool* eos);
+
+    virtual Status push(RuntimeState* state, vectorized::Block* input_block, bool eos) {
+        return Status::OK();
+    }
+
+    bool can_read() const { return _can_read; }
+
+    // Sink Data to ExecNode to do some stock work, both need impl with method: get_result
+    // `eos` means source is exhausted, exec node should do some finalize work
+    // Eg: Aggregation, Sort
+    virtual Status sink(RuntimeState* state, vectorized::Block* input_block, bool eos);
+
     // Resets the stream of row batches to be retrieved by subsequent GetNext() calls.
     // Clears all internal state, returning this node to the state it was in after calling
     // Prepare() and before calling Open(). This function must not clear memory
@@ -140,6 +170,14 @@ public:
     // each implementation should start out by calling the default implementation.
     virtual Status close(RuntimeState* state);
 
+    void increase_ref() { ++_ref; }
+    int decrease_ref() { return --_ref; }
+
+    // Release and close resource for the node
+    // Only pipeline operator use exec node need to impl the virtual function
+    // so only vectorized exec node need to impl
+    virtual void release_resource(RuntimeState* state);
+
     // Creates exec node tree from list of nodes contained in plan via depth-first
     // traversal. All nodes are placed in pool.
     // Returns error if 'plan' is corrupted, otherwise success.
@@ -202,14 +240,14 @@ public:
 
     virtual std::string get_name();
 
-    // Extract node id from p->name().
-    static int get_node_id_from_profile(RuntimeProfile* p);
-
     // Names of counters shared by all exec nodes
     static const std::string ROW_THROUGHPUT_COUNTER;
 
+    ExecNode* child(int i) { return _children[i]; }
+
 protected:
     friend class DataSink;
+    friend class doris::pipeline::Operator;
 
     /// Initialize 'buffer_pool_client_' and claim the initial reservation for this
     /// ExecNode. Only needs to be called by ExecNodes that will use the client.
@@ -335,8 +373,6 @@ protected:
     // Set to true if this is a vectorized exec node.
     bool _is_vec = false;
 
-    ExecNode* child(int i) { return _children[i]; }
-
     bool is_closed() const { return _is_closed; }
 
     // TODO(zc)
@@ -372,9 +408,13 @@ protected:
     /// allocations. ExecNodes overriding this function should return
     /// ExecNode::QueryMaintenance().
     virtual Status QueryMaintenance(RuntimeState* state, const std::string& msg) WARN_UNUSED_RESULT;
+    std::atomic<bool> _can_read = false;
 
 private:
+    friend class pipeline::Operator;
     bool _is_closed;
+    bool _is_resource_released = false;
+    std::atomic_int _ref; // used by pipeline operator to release resource.
 };
 
 } // namespace doris
diff --git a/be/src/exec/table_function_node.cpp b/be/src/exec/table_function_node.cpp
index 930e484d2a..3c1a26cf74 100644
--- a/be/src/exec/table_function_node.cpp
+++ b/be/src/exec/table_function_node.cpp
@@ -17,6 +17,7 @@
 
 #include "exec/table_function_node.h"
 
+#include "exec/exec_node.h"
 #include "exprs/expr.h"
 #include "exprs/expr_context.h"
 #include "exprs/table_function/table_function_factory.h"
@@ -33,7 +34,7 @@ TableFunctionNode::TableFunctionNode(ObjectPool* pool, const TPlanNode& tnode,
                                      const DescriptorTbl& descs)
         : ExecNode(pool, tnode, descs) {}
 
-TableFunctionNode::~TableFunctionNode() {}
+TableFunctionNode::~TableFunctionNode() = default;
 
 Status TableFunctionNode::init(const TPlanNode& tnode, RuntimeState* state) {
     RETURN_IF_ERROR(ExecNode::init(tnode, state));
@@ -101,13 +102,13 @@ Status TableFunctionNode::prepare(RuntimeState* state) {
     return Status::OK();
 }
 
-Status TableFunctionNode::open(RuntimeState* state) {
-    START_AND_SCOPE_SPAN(state->get_tracer(), span, "TableFunctionNode::open");
+Status TableFunctionNode::alloc_resource(RuntimeState* state) {
     SCOPED_TIMER(_runtime_profile->total_time_counter());
     RETURN_IF_CANCELLED(state);
-    RETURN_IF_ERROR(ExecNode::open(state));
     SCOPED_CONSUME_MEM_TRACKER(mem_tracker_growh());
 
+    ExecNode::alloc_resource(state);
+
     RETURN_IF_ERROR(Expr::open(_fn_ctxs, state));
     RETURN_IF_ERROR(vectorized::VExpr::open(_vfn_ctxs, state));
 
@@ -115,7 +116,6 @@ Status TableFunctionNode::open(RuntimeState* state) {
         RETURN_IF_ERROR(fn->open());
     }
 
-    RETURN_IF_ERROR(_children[0]->open(state));
     return Status::OK();
 }
 
@@ -381,19 +381,4 @@ Status TableFunctionNode::get_next(RuntimeState* state, RowBatch* row_batch, boo
     return Status::OK();
 }
 
-Status TableFunctionNode::close(RuntimeState* state) {
-    if (is_closed()) {
-        return Status::OK();
-    }
-    START_AND_SCOPE_SPAN(state->get_tracer(), span, "TableFunctionNode::close");
-    Expr::close(_fn_ctxs, state);
-    vectorized::VExpr::close(_vfn_ctxs, state);
-
-    if (_num_rows_filtered_counter != nullptr) {
-        COUNTER_SET(_num_rows_filtered_counter, static_cast<int64_t>(_num_rows_filtered));
-    }
-
-    return ExecNode::close(state);
-}
-
 }; // namespace doris
diff --git a/be/src/exec/table_function_node.h b/be/src/exec/table_function_node.h
index 06de804c3c..ece29bd50e 100644
--- a/be/src/exec/table_function_node.h
+++ b/be/src/exec/table_function_node.h
@@ -18,6 +18,8 @@
 #pragma once
 
 #include "exec/exec_node.h"
+#include "exprs/expr.h"
+#include "vec/exprs/vexpr.h"
 
 namespace doris {
 
@@ -30,13 +32,26 @@ class TupleRow;
 class TableFunctionNode : public ExecNode {
 public:
     TableFunctionNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs);
-    ~TableFunctionNode();
+    ~TableFunctionNode() override;
 
     Status init(const TPlanNode& tnode, RuntimeState* state = nullptr) override;
     Status prepare(RuntimeState* state) override;
-    Status open(RuntimeState* state) override;
+    Status open(RuntimeState* state) override {
+        START_AND_SCOPE_SPAN(state->get_tracer(), span, "TableFunctionNode::open");
+        RETURN_IF_ERROR(alloc_resource(state));
+        return _children[0]->open(state);
+    }
     Status get_next(RuntimeState* state, RowBatch* row_batch, bool* eos) override;
-    Status close(RuntimeState* state) override;
+    Status alloc_resource(RuntimeState* state) override;
+    void release_resource(doris::RuntimeState* state) override {
+        Expr::close(_fn_ctxs, state);
+        vectorized::VExpr::close(_vfn_ctxs, state);
+
+        if (_num_rows_filtered_counter != nullptr) {
+            COUNTER_SET(_num_rows_filtered_counter, static_cast<int64_t>(_num_rows_filtered));
+        }
+        ExecNode::release_resource(state);
+    }
 
 protected:
     Status _prepare_output_slot_ids(const TPlanNode& tnode);
diff --git a/be/src/pipeline/CMakeLists.txt b/be/src/pipeline/CMakeLists.txt
new file mode 100644
index 0000000000..71060233ef
--- /dev/null
+++ b/be/src/pipeline/CMakeLists.txt
@@ -0,0 +1,48 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+# where to put generated libraries
+set(LIBRARY_OUTPUT_PATH "${BUILD_DIR}/src/pipeline")
+
+# where to put generated binaries
+set(EXECUTABLE_OUTPUT_PATH "${BUILD_DIR}/src/pipeline")
+
+set(PIPELINE_FILES
+        pipeline.cpp
+        pipeline_fragment_context.cpp
+        pipeline_task.cpp
+        task_scheduler.cpp
+        exec/operator.cpp
+        exec/scan_operator.cpp
+        exec/olap_scan_operator.cpp
+        exec/empty_set_operator.cpp
+        exec/exchange_source_operator.cpp
+        exec/exchange_sink_operator.cpp
+        exec/exchange_sink_buffer.cpp
+        exec/result_sink_operator.cpp
+        exec/aggregation_sink_operator.cpp
+        exec/aggregation_source_operator.cpp
+        exec/streaming_aggregation_source_operator.cpp
+        exec/streaming_aggregation_sink_operator.cpp
+        exec/agg_context.cpp
+        exec/sort_source_operator.cpp
+        exec/sort_sink_operator.cpp
+        exec/repeat_operator.cpp)
+
+add_library(Pipeline STATIC
+        ${PIPELINE_FILES}
+        )
\ No newline at end of file
diff --git a/be/src/pipeline/exec/agg_context.cpp b/be/src/pipeline/exec/agg_context.cpp
new file mode 100644
index 0000000000..9c6aa8ecd9
--- /dev/null
+++ b/be/src/pipeline/exec/agg_context.cpp
@@ -0,0 +1,106 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "agg_context.h"
+
+#include "runtime/descriptors.h"
+#include "vec/columns/column_nullable.h"
+#include "vec/core/block.h"
+#include "vec/data_types/data_type_nullable.h"
+
+namespace doris {
+namespace pipeline {
+
+std::unique_ptr<vectorized::Block> AggContext::get_free_block() {
+    {
+        std::lock_guard<std::mutex> l(_free_blocks_lock);
+        if (!_free_blocks.empty()) {
+            auto block = std::move(_free_blocks.back());
+            _free_blocks.pop_back();
+            return block;
+        }
+    }
+
+    return std::make_unique<vectorized::Block>();
+}
+
+void AggContext::return_free_block(std::unique_ptr<vectorized::Block> block) {
+    DCHECK(block->rows() == 0);
+    std::lock_guard<std::mutex> l(_free_blocks_lock);
+    _free_blocks.emplace_back(std::move(block));
+}
+
+bool AggContext::has_data_or_finished() {
+    std::unique_lock<std::mutex> l(_transfer_lock);
+    return !_blocks_queue.empty() || _is_finished;
+}
+
+Status AggContext::get_block(std::unique_ptr<vectorized::Block>* block) {
+    std::unique_lock<std::mutex> l(_transfer_lock);
+    if (_is_canceled) {
+        return Status::InternalError("AggContext canceled");
+    }
+    if (!_blocks_queue.empty()) {
+        auto [block_ptr, block_size] = std::move(_blocks_queue.front());
+        *block = std::move(block_ptr);
+        _blocks_queue.pop_front();
+        _cur_bytes_in_queue -= block_size;
+    } else {
+        if (_is_finished) {
+            _data_exhausted = true;
+        }
+    }
+    return Status::OK();
+}
+
+bool AggContext::has_enough_space_to_push() {
+    std::unique_lock<std::mutex> l(_transfer_lock);
+    return _cur_bytes_in_queue < MAX_BYTE_OF_QUEUE / 2;
+}
+
+void AggContext::push_block(std::unique_ptr<vectorized::Block> block) {
+    if (!block) {
+        return;
+    }
+    auto block_size = block->allocated_bytes();
+    std::unique_lock<std::mutex> l(_transfer_lock);
+    _blocks_queue.emplace_back(std::move(block), block_size);
+    _cur_bytes_in_queue += block_size;
+
+    _max_bytes_in_queue = std::max(_max_bytes_in_queue, _cur_bytes_in_queue);
+    _max_size_of_queue = std::max(_max_size_of_queue, (int64)_blocks_queue.size());
+}
+
+void AggContext::set_finish() {
+    std::unique_lock<std::mutex> l(_transfer_lock);
+    _is_finished = true;
+}
+
+void AggContext::set_canceled() {
+    std::unique_lock<std::mutex> l(_transfer_lock);
+    DCHECK(!_is_finished);
+    _is_canceled = true;
+    _is_finished = true;
+}
+
+bool AggContext::is_finish() {
+    std::unique_lock<std::mutex> l(_transfer_lock);
+    return _is_finished;
+}
+
+} // namespace pipeline
+} // namespace doris
\ No newline at end of file
diff --git a/be/src/pipeline/exec/agg_context.h b/be/src/pipeline/exec/agg_context.h
new file mode 100644
index 0000000000..42bfe81a33
--- /dev/null
+++ b/be/src/pipeline/exec/agg_context.h
@@ -0,0 +1,72 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+#pragma once
+
+#include "common/status.h"
+
+namespace doris {
+class TupleDescriptor;
+namespace vectorized {
+class Block;
+}
+namespace pipeline {
+
+class AggContext {
+public:
+    AggContext() = default;
+    ~AggContext() { DCHECK(_is_finished); }
+
+    std::unique_ptr<vectorized::Block> get_free_block();
+
+    void return_free_block(std::unique_ptr<vectorized::Block>);
+
+    bool has_data_or_finished();
+    Status get_block(std::unique_ptr<vectorized::Block>* block);
+
+    bool has_enough_space_to_push();
+    void push_block(std::unique_ptr<vectorized::Block>);
+
+    void set_finish();
+    void set_canceled(); // should set before finish
+    bool is_finish();
+
+    bool data_exhausted() const { return _data_exhausted; }
+
+    int64_t max_bytes_in_queue() const { return _max_bytes_in_queue; }
+
+    int64_t max_size_of_queue() const { return _max_size_of_queue; }
+
+private:
+    std::mutex _free_blocks_lock;
+    std::vector<std::unique_ptr<vectorized::Block>> _free_blocks;
+
+    std::mutex _transfer_lock;
+    std::list<std::pair<std::unique_ptr<vectorized::Block>, size_t>> _blocks_queue;
+
+    bool _data_exhausted = false;
+    bool _is_finished = false;
+    bool _is_canceled = false;
+
+    // int64_t just for counter of profile
+    int64_t _cur_bytes_in_queue = 0;
+    int64_t _max_bytes_in_queue = 0;
+    int64_t _max_size_of_queue = 0;
+    static constexpr int64_t MAX_BYTE_OF_QUEUE = 1024l * 1024 * 1024 / 10;
+};
+
+} // namespace pipeline
+} // namespace doris
\ No newline at end of file
diff --git a/be/src/pipeline/exec/aggregation_sink_operator.cpp b/be/src/pipeline/exec/aggregation_sink_operator.cpp
new file mode 100644
index 0000000000..d9459bf56d
--- /dev/null
+++ b/be/src/pipeline/exec/aggregation_sink_operator.cpp
@@ -0,0 +1,77 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "aggregation_sink_operator.h"
+
+#include "vec/exec/vaggregation_node.h"
+
+namespace doris::pipeline {
+
+AggSinkOperator::AggSinkOperator(AggSinkOperatorBuilder* operator_builder,
+                                 vectorized::AggregationNode* agg_node)
+        : Operator(operator_builder), _agg_node(agg_node) {}
+
+Status AggSinkOperator::prepare(RuntimeState* state) {
+    RETURN_IF_ERROR(Operator::prepare(state));
+    _agg_node->increase_ref();
+    return Status::OK();
+}
+
+Status AggSinkOperator::open(RuntimeState* state) {
+    SCOPED_TIMER(_runtime_profile->total_time_counter());
+    RETURN_IF_ERROR(Operator::open(state));
+    RETURN_IF_ERROR(_agg_node->alloc_resource(state));
+    return Status::OK();
+}
+
+bool AggSinkOperator::can_write() {
+    return true;
+}
+
+Status AggSinkOperator::sink(RuntimeState* state, vectorized::Block* in_block,
+                             SourceState source_state) {
+    SCOPED_TIMER(_runtime_profile->total_time_counter());
+    return _agg_node->sink(state, in_block, source_state == SourceState::FINISHED);
+}
+
+Status AggSinkOperator::close(RuntimeState* state) {
+    _fresh_exec_timer(_agg_node);
+    if (!_agg_node->decrease_ref()) {
+        _agg_node->release_resource(state);
+    }
+    return Status::OK();
+}
+
+///////////////////////////////  operator template  ////////////////////////////////
+
+AggSinkOperatorBuilder::AggSinkOperatorBuilder(int32_t id, const std::string& name,
+                                               vectorized::AggregationNode* exec_node)
+        : OperatorBuilder(id, name, exec_node), _agg_node(exec_node) {}
+
+OperatorPtr AggSinkOperatorBuilder::build_operator() {
+    return std::make_shared<AggSinkOperator>(this, _agg_node);
+}
+
+// use final aggregation source operator
+bool AggSinkOperatorBuilder::is_sink() const {
+    return true;
+}
+
+bool AggSinkOperatorBuilder::is_source() const {
+    return false;
+}
+} // namespace doris::pipeline
\ No newline at end of file
diff --git a/be/src/pipeline/exec/aggregation_sink_operator.h b/be/src/pipeline/exec/aggregation_sink_operator.h
new file mode 100644
index 0000000000..fc74f2366d
--- /dev/null
+++ b/be/src/pipeline/exec/aggregation_sink_operator.h
@@ -0,0 +1,65 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include "agg_context.h"
+#include "operator.h"
+
+namespace doris {
+namespace vectorized {
+class AggregationNode;
+class VExprContext;
+class Block;
+} // namespace vectorized
+
+namespace pipeline {
+class AggSinkOperatorBuilder;
+class AggSinkOperator : public Operator {
+public:
+    AggSinkOperator(AggSinkOperatorBuilder* operator_builder, vectorized::AggregationNode*);
+
+    Status prepare(RuntimeState*) override;
+    Status open(RuntimeState* state) override;
+
+    Status sink(RuntimeState* state, vectorized::Block* block, SourceState source_state) override;
+
+    bool can_write() override;
+
+    Status close(RuntimeState* state) override;
+
+    Status finalize(doris::RuntimeState* state) override { return Status::OK(); }
+
+private:
+    vectorized::AggregationNode* _agg_node;
+};
+
+class AggSinkOperatorBuilder : public OperatorBuilder {
+public:
+    AggSinkOperatorBuilder(int32_t, const std::string&, vectorized::AggregationNode*);
+
+    OperatorPtr build_operator() override;
+
+    bool is_sink() const override;
+    bool is_source() const override;
+
+private:
+    vectorized::AggregationNode* _agg_node;
+};
+
+} // namespace pipeline
+} // namespace doris
\ No newline at end of file
diff --git a/be/src/pipeline/exec/aggregation_source_operator.cpp b/be/src/pipeline/exec/aggregation_source_operator.cpp
new file mode 100644
index 0000000000..a0dc95615f
--- /dev/null
+++ b/be/src/pipeline/exec/aggregation_source_operator.cpp
@@ -0,0 +1,67 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "aggregation_source_operator.h"
+
+#include "vec/exec/vaggregation_node.h"
+
+namespace doris {
+namespace pipeline {
+
+AggregationSourceOperator::AggregationSourceOperator(OperatorBuilder* templ,
+                                                     vectorized::AggregationNode* node)
+        : Operator(templ), _agg_node(node) {}
+
+Status AggregationSourceOperator::prepare(RuntimeState* state) {
+    _agg_node->increase_ref();
+    return Status::OK();
+}
+
+bool AggregationSourceOperator::can_read() {
+    return _agg_node->can_read();
+}
+
+Status AggregationSourceOperator::get_block(RuntimeState* state, vectorized::Block* block,
+                                            SourceState& source_state) {
+    SCOPED_TIMER(_runtime_profile->total_time_counter());
+    bool eos = false;
+    RETURN_IF_ERROR(_agg_node->pull(state, block, &eos));
+    source_state = eos ? SourceState::FINISHED : SourceState::DEPEND_ON_SOURCE;
+    return Status::OK();
+}
+
+Status AggregationSourceOperator::close(RuntimeState* state) {
+    _fresh_exec_timer(_agg_node);
+    if (!_agg_node->decrease_ref()) {
+        _agg_node->release_resource(state);
+    }
+    return Status::OK();
+}
+
+///////////////////////////////  operator template  ////////////////////////////////
+
+AggregationSourceOperatorBuilder::AggregationSourceOperatorBuilder(
+        int32_t id, const std::string& name, vectorized::AggregationNode* exec_node)
+        : OperatorBuilder(id, name, exec_node) {}
+
+OperatorPtr AggregationSourceOperatorBuilder::build_operator() {
+    return std::make_shared<AggregationSourceOperator>(
+            this, assert_cast<vectorized::AggregationNode*>(_related_exec_node));
+}
+
+} // namespace pipeline
+} // namespace doris
\ No newline at end of file
diff --git a/be/src/vec/sink/vmysql_table_sink.h b/be/src/pipeline/exec/aggregation_source_operator.h
similarity index 51%
copy from be/src/vec/sink/vmysql_table_sink.h
copy to be/src/pipeline/exec/aggregation_source_operator.h
index 6a30275a8e..1c611d163d 100644
--- a/be/src/vec/sink/vmysql_table_sink.h
+++ b/be/src/pipeline/exec/aggregation_source_operator.h
@@ -15,32 +15,37 @@
 // specific language governing permissions and limitations
 // under the License.
 #pragma once
-#include <vector>
 
-#include "common/status.h"
-#include "vec/sink/vmysql_table_writer.h"
-#include "vec/sink/vtable_sink.h"
+#include "operator.h"
 
 namespace doris {
 namespace vectorized {
+class AggregationNode;
+}
 
-// This class is a sinker, which put input data to mysql table
-class VMysqlTableSink : public VTableSink {
-public:
-    VMysqlTableSink(ObjectPool* pool, const RowDescriptor& row_desc,
-                    const std::vector<TExpr>& t_exprs);
+namespace pipeline {
 
-    Status init(const TDataSink& thrift_sink) override;
+// For read none streaming agg sink operator's data
+class AggregationSourceOperator : public Operator {
+public:
+    AggregationSourceOperator(OperatorBuilder*, vectorized::AggregationNode*);
+    Status prepare(RuntimeState* state) override;
+    bool can_read() override;
+    Status close(RuntimeState* state) override;
+    Status get_block(RuntimeState*, vectorized::Block*, SourceState&) override;
 
-    Status open(RuntimeState* state) override;
+private:
+    vectorized::AggregationNode* _agg_node;
+};
 
-    Status send(RuntimeState* state, vectorized::Block* block) override;
+class AggregationSourceOperatorBuilder : public OperatorBuilder {
+public:
+    AggregationSourceOperatorBuilder(int32_t, const std::string&, vectorized::AggregationNode*);
 
-    Status close(RuntimeState* state, Status exec_status) override;
+    bool is_source() const override { return true; }
 
-private:
-    MysqlConnInfo _conn_info;
-    std::unique_ptr<VMysqlTableWriter> _writer;
+    OperatorPtr build_operator() override;
 };
-} // namespace vectorized
-} // namespace doris
+
+} // namespace pipeline
+} // namespace doris
\ No newline at end of file
diff --git a/be/src/pipeline/exec/empty_set_operator.cpp b/be/src/pipeline/exec/empty_set_operator.cpp
new file mode 100644
index 0000000000..a0913766cd
--- /dev/null
+++ b/be/src/pipeline/exec/empty_set_operator.cpp
@@ -0,0 +1,44 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "empty_set_operator.h"
+
+#include "vec/exec/vempty_set_node.h"
+
+namespace doris::pipeline {
+
+EmptySetSourceOperator::EmptySetSourceOperator(EmptySetSourceOperatorBuilder* operator_builder,
+                                               vectorized::VEmptySetNode* empty_set_node)
+        : Operator(operator_builder), _empty_set_node(empty_set_node) {}
+
+bool EmptySetSourceOperator::can_read() {
+    return true;
+}
+
+Status EmptySetSourceOperator::get_block(RuntimeState* state, vectorized::Block* block,
+                                         SourceState& source_state) {
+    bool eos = false;
+    RETURN_IF_ERROR(_empty_set_node->get_next(state, block, &eos));
+    source_state = eos ? SourceState::FINISHED : SourceState::DEPEND_ON_SOURCE;
+    return Status::OK();
+}
+
+EmptySetSourceOperatorBuilder::EmptySetSourceOperatorBuilder(
+        int32_t id, const string& name, vectorized::VEmptySetNode* empty_set_node)
+        : OperatorBuilder(id, name, empty_set_node), _empty_set_node(empty_set_node) {}
+
+} // namespace doris::pipeline
diff --git a/be/src/pipeline/exec/empty_set_operator.h b/be/src/pipeline/exec/empty_set_operator.h
new file mode 100644
index 0000000000..90af89dbcb
--- /dev/null
+++ b/be/src/pipeline/exec/empty_set_operator.h
@@ -0,0 +1,61 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include "operator.h"
+
+namespace doris {
+
+namespace vectorized {
+class VEmptySetNode;
+}
+
+namespace pipeline {
+
+class EmptySetSourceOperatorBuilder;
+
+class EmptySetSourceOperator : public Operator {
+public:
+    EmptySetSourceOperator(EmptySetSourceOperatorBuilder* operator_builder,
+                           vectorized::VEmptySetNode* empty_set_node);
+    Status get_block(RuntimeState* state, vectorized::Block* block,
+                     SourceState& source_state) override;
+
+    bool can_read() override;
+
+private:
+    vectorized::VEmptySetNode* _empty_set_node;
+};
+
+class EmptySetSourceOperatorBuilder : public OperatorBuilder {
+public:
+    EmptySetSourceOperatorBuilder(int32_t id, const std::string& name,
+                                  vectorized::VEmptySetNode* empty_set_node);
+
+    bool is_source() const override { return true; }
+
+    OperatorPtr build_operator() override {
+        return std::make_shared<EmptySetSourceOperator>(this, _empty_set_node);
+    }
+
+private:
+    vectorized::VEmptySetNode* _empty_set_node;
+};
+
+} // namespace pipeline
+} // namespace doris
diff --git a/be/src/pipeline/exec/exchange_sink_buffer.cpp b/be/src/pipeline/exec/exchange_sink_buffer.cpp
new file mode 100644
index 0000000000..2ee2db1c9c
--- /dev/null
+++ b/be/src/pipeline/exec/exchange_sink_buffer.cpp
@@ -0,0 +1,236 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "exchange_sink_buffer.h"
+
+#include <google/protobuf/stubs/common.h>
+
+#include <atomic>
+#include <memory>
+
+#include "common/status.h"
+#include "pipeline/pipeline_fragment_context.h"
+#include "service/brpc.h"
+#include "util/proto_util.h"
+#include "util/time.h"
+#include "vec/sink/vdata_stream_sender.h"
+
+namespace doris::pipeline {
+template <typename T>
+class SelfDeleteClosure : public google::protobuf::Closure {
+public:
+    SelfDeleteClosure(InstanceLoId id, bool eos) : _id(id), _eos(eos) {}
+    ~SelfDeleteClosure() override = default;
+    SelfDeleteClosure(const SelfDeleteClosure& other) = delete;
+    SelfDeleteClosure& operator=(const SelfDeleteClosure& other) = delete;
+    void addFailedHandler(std::function<void(const InstanceLoId&, const std::string&)> fail_fn) {
+        _fail_fn = std::move(fail_fn);
+    }
+    void addSuccessHandler(std::function<void(const InstanceLoId&, const bool&, const T&)> suc_fn) {
+        _suc_fn = suc_fn;
+    }
+
+    void Run() noexcept override {
+        std::unique_ptr<SelfDeleteClosure> self_guard(this);
+        try {
+            if (cntl.Failed()) {
+                std::string err = fmt::format(
+                        "failed to send brpc when exchange, error={}, error_text={}, client: {}, "
+                        "latency = {}",
+                        berror(cntl.ErrorCode()), cntl.ErrorText(), BackendOptions::get_localhost(),
+                        cntl.latency_us());
+                _fail_fn(_id, err);
+            } else {
+                _suc_fn(_id, _eos, result);
+            }
+        } catch (const std::exception& exp) {
+            LOG(FATAL) << "brpc callback error: " << exp.what();
+        } catch (...) {
+            LOG(FATAL) << "brpc callback error.";
+        }
+    }
+
+public:
+    brpc::Controller cntl;
+    T result;
+
+private:
+    std::function<void(const InstanceLoId&, const std::string&)> _fail_fn;
+    std::function<void(const InstanceLoId&, const bool&, const T&)> _suc_fn;
+    InstanceLoId _id;
+    bool _eos;
+};
+
+ExchangeSinkBuffer::ExchangeSinkBuffer(PUniqueId query_id, PlanNodeId dest_node_id, int send_id,
+                                       int be_number, PipelineFragmentContext* context)
+        : _is_finishing(false),
+          _query_id(query_id),
+          _dest_node_id(dest_node_id),
+          _sender_id(send_id),
+          _be_number(be_number),
+          _context(context) {}
+
+ExchangeSinkBuffer::~ExchangeSinkBuffer() = default;
+
+void ExchangeSinkBuffer::close() {
+    for (const auto& pair : _instance_to_request) {
+        if (pair.second) {
+            pair.second->release_finst_id();
+            pair.second->release_query_id();
+        }
+    }
+}
+
+bool ExchangeSinkBuffer::can_write() const {
+    size_t max_package_size = 64 * _instance_to_package_queue.size();
+    size_t total_package_size = 0;
+    for (auto& [_, q] : _instance_to_package_queue) {
+        total_package_size += q.size();
+    }
+    return total_package_size <= max_package_size;
+}
+
+bool ExchangeSinkBuffer::is_pending_finish() const {
+    for (auto& pair : _instance_to_package_queue_mutex) {
+        std::unique_lock<std::mutex> lock(*(pair.second));
+        auto& id = pair.first;
+        if (!_instance_to_sending_by_pipeline.at(id)) {
+            return true;
+        }
+    }
+    return false;
+}
+
+void ExchangeSinkBuffer::register_sink(TUniqueId fragment_instance_id) {
+    if (_is_finishing) {
+        return;
+    }
+    auto low_id = fragment_instance_id.lo;
+    if (_instance_to_package_queue_mutex.count(low_id)) {
+        return;
+    }
+    _instance_to_package_queue_mutex[low_id] = std::make_unique<std::mutex>();
+    _instance_to_seq[low_id] = 0;
+    _instance_to_package_queue[low_id] = std::queue<TransmitInfo, std::list<TransmitInfo>>();
+    PUniqueId finst_id;
+    finst_id.set_hi(fragment_instance_id.hi);
+    finst_id.set_lo(fragment_instance_id.lo);
+    _instance_to_finst_id[low_id] = finst_id;
+    _instance_to_sending_by_pipeline[low_id] = true;
+}
+
+Status ExchangeSinkBuffer::add_block(TransmitInfo&& request) {
+    if (_is_finishing) {
+        return Status::OK();
+    }
+    TUniqueId ins_id = request.channel->_fragment_instance_id;
+    bool send_now = false;
+    {
+        std::unique_lock<std::mutex> lock(*_instance_to_package_queue_mutex[ins_id.lo]);
+        // Do not have in process rpc, directly send
+        if (_instance_to_sending_by_pipeline[ins_id.lo]) {
+            send_now = true;
+            _instance_to_sending_by_pipeline[ins_id.lo] = false;
+        }
+        _instance_to_package_queue[ins_id.lo].emplace(std::move(request));
+    }
+    if (send_now) {
+        RETURN_IF_ERROR(_send_rpc(ins_id.lo));
+    }
+
+    return Status::OK();
+}
+
+Status ExchangeSinkBuffer::_send_rpc(InstanceLoId id) {
+    std::unique_lock<std::mutex> lock(*_instance_to_package_queue_mutex[id]);
+
+    std::queue<TransmitInfo, std::list<TransmitInfo>>& q = _instance_to_package_queue[id];
+    if (q.empty() || _is_finishing) {
+        _instance_to_sending_by_pipeline[id] = true;
+        return Status::OK();
+    }
+
+    TransmitInfo& request = q.front();
+
+    if (!_instance_to_request[id]) {
+        _construct_request(id);
+    }
+
+    auto& brpc_request = _instance_to_request[id];
+    brpc_request->set_eos(request.eos);
+    brpc_request->set_packet_seq(_instance_to_seq[id]++);
+    if (request.block) {
+        brpc_request->set_allocated_block(request.block.get());
+    }
+
+    auto* _closure = new SelfDeleteClosure<PTransmitDataResult>(id, request.eos);
+    _closure->cntl.set_timeout_ms(request.channel->_brpc_timeout_ms);
+    _closure->addFailedHandler(
+            [&](const InstanceLoId& id, const std::string& err) { _failed(id, err); });
+    _closure->addSuccessHandler([&](const InstanceLoId& id, const bool& eos,
+                                    const PTransmitDataResult& result) {
+        Status s = Status(result.status());
+        if (!s.ok()) {
+            _failed(id,
+                    fmt::format("exchange req success but status isn't ok: {}", s.get_error_msg()));
+        } else if (eos) {
+            _ended(id);
+        } else {
+            _send_rpc(id);
+        }
+    });
+
+    {
+        SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(ExecEnv::GetInstance()->orphan_mem_tracker());
+        if (enable_http_send_block(*brpc_request)) {
+            RETURN_IF_ERROR(transmit_block_http(_context->get_runtime_state(), _closure,
+                                                *brpc_request, request.channel->_brpc_dest_addr));
+        } else {
+            transmit_block(*request.channel->_brpc_stub, _closure, *brpc_request);
+        }
+    }
+
+    if (request.block) {
+        brpc_request->release_block();
+    }
+    q.pop();
+
+    return Status::OK();
+}
+
+void ExchangeSinkBuffer::_construct_request(InstanceLoId id) {
+    _instance_to_request[id] = std::make_unique<PTransmitDataParams>();
+    _instance_to_request[id]->set_allocated_finst_id(&_instance_to_finst_id[id]);
+    _instance_to_request[id]->set_allocated_query_id(&_query_id);
+
+    _instance_to_request[id]->set_node_id(_dest_node_id);
+    _instance_to_request[id]->set_sender_id(_sender_id);
+    _instance_to_request[id]->set_be_number(_be_number);
+}
+
+void ExchangeSinkBuffer::_ended(InstanceLoId id) {
+    std::unique_lock<std::mutex> lock(*_instance_to_package_queue_mutex[id]);
+    _instance_to_sending_by_pipeline[id] = true;
+}
+
+void ExchangeSinkBuffer::_failed(InstanceLoId id, const std::string& err) {
+    _is_finishing = true;
+    _context->cancel(PPlanFragmentCancelReason::INTERNAL_ERROR, err);
+    _ended(id);
+};
+
+} // namespace doris::pipeline
diff --git a/be/src/pipeline/exec/exchange_sink_buffer.h b/be/src/pipeline/exec/exchange_sink_buffer.h
new file mode 100644
index 0000000000..1eba3cc25b
--- /dev/null
+++ b/be/src/pipeline/exec/exchange_sink_buffer.h
@@ -0,0 +1,86 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include <parallel_hashmap/phmap.h>
+
+#include <list>
+#include <queue>
+#include <shared_mutex>
+
+#include "gen_cpp/Types_types.h"
+#include "gen_cpp/internal_service.pb.h"
+#include "runtime/runtime_state.h"
+
+namespace doris {
+namespace vectorized {
+class PipChannel;
+}
+
+namespace pipeline {
+using InstanceLoId = int64_t;
+struct TransmitInfo {
+    vectorized::PipChannel* channel;
+    std::unique_ptr<PBlock> block;
+    bool eos;
+};
+
+class PipelineFragmentContext;
+
+// Each ExchangeSinkOperator have one ExchangeSinkBuffer
+class ExchangeSinkBuffer {
+public:
+    ExchangeSinkBuffer(PUniqueId, int, PlanNodeId, int, PipelineFragmentContext*);
+    ~ExchangeSinkBuffer();
+    void register_sink(TUniqueId);
+    Status add_block(TransmitInfo&& request);
+    bool can_write() const;
+    bool is_pending_finish() const;
+    void close();
+
+private:
+    phmap::flat_hash_map<InstanceLoId, std::unique_ptr<std::mutex>>
+            _instance_to_package_queue_mutex;
+    phmap::flat_hash_map<InstanceLoId, std::queue<TransmitInfo, std::list<TransmitInfo>>>
+            _instance_to_package_queue;
+    using PackageSeq = int64_t;
+    // must init zero
+    phmap::flat_hash_map<InstanceLoId, PackageSeq> _instance_to_seq;
+    phmap::flat_hash_map<InstanceLoId, std::unique_ptr<PTransmitDataParams>> _instance_to_request;
+    phmap::flat_hash_map<InstanceLoId, PUniqueId> _instance_to_finst_id;
+    phmap::flat_hash_map<InstanceLoId, bool> _instance_to_sending_by_pipeline;
+
+    std::atomic<bool> _is_finishing;
+    PUniqueId _query_id;
+    PlanNodeId _dest_node_id;
+    // Sender instance id, unique within a fragment. StreamSender save the variable
+    int _sender_id;
+    int _be_number;
+
+    PipelineFragmentContext* _context;
+
+private:
+    Status _send_rpc(InstanceLoId);
+    // must hold the _instance_to_package_queue_mutex[id] mutex to opera
+    void _construct_request(InstanceLoId id);
+    inline void _ended(InstanceLoId id);
+    inline void _failed(InstanceLoId id, const std::string& err);
+};
+
+} // namespace pipeline
+} // namespace doris
\ No newline at end of file
diff --git a/be/src/pipeline/exec/exchange_sink_operator.cpp b/be/src/pipeline/exec/exchange_sink_operator.cpp
new file mode 100644
index 0000000000..7179f11d30
--- /dev/null
+++ b/be/src/pipeline/exec/exchange_sink_operator.cpp
@@ -0,0 +1,97 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "exchange_sink_operator.h"
+
+#include "common/status.h"
+#include "exchange_sink_buffer.h"
+#include "gen_cpp/internal_service.pb.h"
+#include "util/brpc_client_cache.h"
+#include "vec/exprs/vexpr.h"
+#include "vec/runtime/vpartition_info.h"
+#include "vec/sink/vdata_stream_sender.h"
+
+namespace doris::pipeline {
+
+ExchangeSinkOperator::ExchangeSinkOperator(OperatorBuilder* operator_builder,
+                                           vectorized::VDataStreamSender* sink,
+                                           PipelineFragmentContext* context)
+        : Operator(operator_builder), _sink(sink), _context(context) {}
+
+ExchangeSinkOperator::~ExchangeSinkOperator() = default;
+
+Status ExchangeSinkOperator::init(ExecNode* exec_node, RuntimeState* state) {
+    RETURN_IF_ERROR(Operator::init(exec_node, state));
+    _state = state;
+    return Status::OK();
+}
+
+Status ExchangeSinkOperator::init(const TDataSink& tsink) {
+    RETURN_IF_ERROR(_sink->init(tsink));
+
+    PUniqueId query_id;
+    query_id.set_hi(_state->query_id().hi);
+    query_id.set_lo(_state->query_id().lo);
+    _sink_buffer =
+            std::make_unique<ExchangeSinkBuffer>(query_id, tsink.stream_sink.dest_node_id,
+                                                 _sink->_sender_id, _state->be_number(), _context);
+    return Status::OK();
+}
+
+Status ExchangeSinkOperator::prepare(RuntimeState* state) {
+    RETURN_IF_ERROR(Operator::prepare(state));
+    RETURN_IF_ERROR(_sink->prepare(state));
+    _sink->profile()->add_child(_runtime_profile.get(), true, nullptr);
+
+    _sink->registe_channels(_sink_buffer.get());
+    return Status::OK();
+}
+
+Status ExchangeSinkOperator::open(RuntimeState* state) {
+    SCOPED_TIMER(_runtime_profile->total_time_counter());
+    RETURN_IF_ERROR(_sink->open(state));
+    return Status::OK();
+}
+
+bool ExchangeSinkOperator::can_write() {
+    return _sink_buffer->can_write() && _sink->channel_all_can_write();
+}
+
+Status ExchangeSinkOperator::finalize(RuntimeState* state) {
+    Status result = Status::OK();
+    RETURN_IF_ERROR(_sink->close(state, result));
+    return result;
+}
+
+Status ExchangeSinkOperator::sink(RuntimeState* state, vectorized::Block* block,
+                                  SourceState source_state) {
+    SCOPED_TIMER(_runtime_profile->total_time_counter());
+    RETURN_IF_ERROR(_sink->send(state, block, source_state == SourceState::FINISHED));
+    return Status::OK();
+}
+
+bool ExchangeSinkOperator::is_pending_finish() const {
+    return _sink_buffer->is_pending_finish();
+}
+
+Status ExchangeSinkOperator::close(RuntimeState* state) {
+    _sink_buffer->close();
+    RETURN_IF_ERROR(Operator::close(state));
+    return Status::OK();
+}
+
+} // namespace doris::pipeline
diff --git a/be/src/pipeline/exec/exchange_sink_operator.h b/be/src/pipeline/exec/exchange_sink_operator.h
new file mode 100644
index 0000000000..481baf1868
--- /dev/null
+++ b/be/src/pipeline/exec/exchange_sink_operator.h
@@ -0,0 +1,75 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include "exchange_sink_buffer.h"
+#include "operator.h"
+#include "vec/sink/vdata_stream_sender.h"
+
+namespace doris {
+
+namespace pipeline {
+class PipelineFragmentContext;
+
+// Now local exchange is not supported since VDataStreamRecvr is considered as a pipeline broker.
+class ExchangeSinkOperator : public Operator {
+public:
+    ExchangeSinkOperator(OperatorBuilder* operator_builder, vectorized::VDataStreamSender* sink,
+                         PipelineFragmentContext* context);
+    ~ExchangeSinkOperator() override;
+    Status init(ExecNode* exec_node, RuntimeState* state = nullptr) override;
+    Status init(const TDataSink& tsink) override;
+
+    Status prepare(RuntimeState* state) override;
+    Status open(RuntimeState* state) override;
+    bool can_write() override;
+    Status sink(RuntimeState* state, vectorized::Block* block, SourceState source_state) override;
+    bool is_pending_finish() const override;
+    Status finalize(RuntimeState* state) override;
+
+    Status close(RuntimeState* state) override;
+
+    RuntimeState* state() { return _state; }
+
+private:
+    std::unique_ptr<ExchangeSinkBuffer> _sink_buffer;
+    vectorized::VDataStreamSender* _sink;
+    RuntimeState* _state = nullptr;
+    PipelineFragmentContext* _context;
+};
+
+class ExchangeSinkOperatorBuilder : public OperatorBuilder {
+public:
+    ExchangeSinkOperatorBuilder(int32_t id, const std::string& name, ExecNode* exec_node,
+                                vectorized::VDataStreamSender* sink,
+                                PipelineFragmentContext* context)
+            : OperatorBuilder(id, name, exec_node), _sink(sink), _context(context) {}
+
+    bool is_sink() const override { return true; }
+
+    OperatorPtr build_operator() override {
+        return std::make_shared<ExchangeSinkOperator>(this, _sink, _context);
+    }
+
+private:
+    vectorized::VDataStreamSender* _sink;
+    PipelineFragmentContext* _context;
+};
+
+} // namespace pipeline
+} // namespace doris
\ No newline at end of file
diff --git a/be/src/pipeline/exec/exchange_source_operator.cpp b/be/src/pipeline/exec/exchange_source_operator.cpp
new file mode 100644
index 0000000000..746d87e107
--- /dev/null
+++ b/be/src/pipeline/exec/exchange_source_operator.cpp
@@ -0,0 +1,63 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "exchange_source_operator.h"
+
+#include "common/status.h"
+#include "vec/exec/vexchange_node.h"
+#include "vec/runtime/vdata_stream_recvr.h"
+
+namespace doris::pipeline {
+
+ExchangeSourceOperator::ExchangeSourceOperator(OperatorBuilder* operator_builder,
+                                               vectorized::VExchangeNode* node)
+        : Operator(operator_builder), _exchange_node(node) {}
+
+Status ExchangeSourceOperator::open(RuntimeState* state) {
+    SCOPED_TIMER(_runtime_profile->total_time_counter());
+    return _exchange_node->alloc_resource(state);
+}
+
+bool ExchangeSourceOperator::can_read() {
+    return _exchange_node->_stream_recvr->ready_to_read();
+}
+
+Status ExchangeSourceOperator::get_block(RuntimeState* state, vectorized::Block* block,
+                                         SourceState& source_state) {
+    SCOPED_TIMER(_runtime_profile->total_time_counter());
+    bool eos = false;
+    auto st = _exchange_node->get_next(state, block, &eos);
+    source_state = eos ? SourceState::FINISHED : SourceState::DEPEND_ON_SOURCE;
+    return st;
+}
+
+bool ExchangeSourceOperator::is_pending_finish() const {
+    // TODO HappenLee
+    return false;
+}
+
+Status ExchangeSourceOperator::close(RuntimeState* state) {
+    if (is_closed()) {
+        return Status::OK();
+    }
+    _fresh_exec_timer(_exchange_node);
+    _exchange_node->release_resource(state);
+
+    return Operator::close(state);
+}
+
+} // namespace doris::pipeline
diff --git a/be/src/pipeline/exec/exchange_source_operator.h b/be/src/pipeline/exec/exchange_source_operator.h
new file mode 100644
index 0000000000..8d1707a1e0
--- /dev/null
+++ b/be/src/pipeline/exec/exchange_source_operator.h
@@ -0,0 +1,55 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include "operator.h"
+
+namespace doris::vectorized {
+class VExchangeNode;
+}
+
+namespace doris::pipeline {
+
+class ExchangeSourceOperator : public Operator {
+public:
+    explicit ExchangeSourceOperator(OperatorBuilder*, vectorized::VExchangeNode*);
+    Status open(RuntimeState* state) override;
+    bool can_read() override;
+    Status get_block(RuntimeState* state, vectorized::Block* block,
+                     SourceState& source_state) override;
+    bool is_pending_finish() const override;
+    Status close(RuntimeState* state) override;
+
+private:
+    vectorized::VExchangeNode* _exchange_node;
+};
+
+class ExchangeSourceOperatorBuilder : public OperatorBuilder {
+public:
+    ExchangeSourceOperatorBuilder(int32_t id, const std::string& name, ExecNode* exec_node)
+            : OperatorBuilder(id, name, exec_node) {}
+
+    bool is_source() const override { return true; }
+
+    OperatorPtr build_operator() override {
+        return std::make_shared<ExchangeSourceOperator>(
+                this, reinterpret_cast<vectorized::VExchangeNode*>(_related_exec_node));
+    }
+};
+
+} // namespace doris::pipeline
\ No newline at end of file
diff --git a/be/src/vec/sink/vresult_writer.h b/be/src/pipeline/exec/olap_scan_operator.cpp
similarity index 55%
copy from be/src/vec/sink/vresult_writer.h
copy to be/src/pipeline/exec/olap_scan_operator.cpp
index a1fb6ecce3..2e4fdc2570 100644
--- a/be/src/vec/sink/vresult_writer.h
+++ b/be/src/pipeline/exec/olap_scan_operator.cpp
@@ -15,17 +15,19 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#pragma once
+#include "olap_scan_operator.h"
 
-#include "runtime/result_writer.h"
+#include "vec/exec/scan/new_olap_scan_node.h"
 
-namespace doris {
-namespace vectorized {
-class VResultWriter : public ResultWriter {
-public:
-    VResultWriter() : ResultWriter() {}
+namespace doris::pipeline {
 
-    virtual Status append_block(Block& block) = 0;
-};
-} // namespace vectorized
-} // namespace doris
\ No newline at end of file
+OlapScanOperator::OlapScanOperator(OperatorBuilder* operator_builder,
+                                   vectorized::NewOlapScanNode* scan_node)
+        : ScanOperator(operator_builder, scan_node) {}
+
+OlapScanOperatorBuilder::OlapScanOperatorBuilder(uint32_t id, const std::string& name,
+                                                 vectorized::NewOlapScanNode* new_olap_scan_node)
+        : ScanOperatorBuilder(id, name, new_olap_scan_node),
+          _new_olap_scan_node(new_olap_scan_node) {}
+
+} // namespace doris::pipeline
\ No newline at end of file
diff --git a/be/src/vec/sink/vodbc_table_sink.h b/be/src/pipeline/exec/olap_scan_operator.h
similarity index 53%
copy from be/src/vec/sink/vodbc_table_sink.h
copy to be/src/pipeline/exec/olap_scan_operator.h
index dc3d38efee..26631c6390 100644
--- a/be/src/vec/sink/vodbc_table_sink.h
+++ b/be/src/pipeline/exec/olap_scan_operator.h
@@ -14,32 +14,33 @@
 // KIND, either express or implied.  See the License for the
 // specific language governing permissions and limitations
 // under the License.
-#pragma once
 
-#include "common/status.h"
-#include "exec/odbc_connector.h"
-#include "vec/sink/vtable_sink.h"
+#pragma once
 
-namespace doris {
-namespace vectorized {
+#include <utility>
 
-// This class is a sinker, which put input data to odbc table
-class VOdbcTableSink : public VTableSink {
-public:
-    VOdbcTableSink(ObjectPool* pool, const RowDescriptor& row_desc,
-                   const std::vector<TExpr>& t_exprs);
+#include "scan_operator.h"
+#include "vec/exec/scan/new_olap_scan_node.h"
 
-    Status init(const TDataSink& thrift_sink) override;
+namespace doris::pipeline {
 
-    Status open(RuntimeState* state) override;
+class OlapScanOperatorBuilder;
+class OlapScanOperator : public ScanOperator {
+public:
+    OlapScanOperator(OperatorBuilder* operator_builder, vectorized::NewOlapScanNode* scan_node);
+};
 
-    Status send(RuntimeState* state, vectorized::Block* block) override;
+class OlapScanOperatorBuilder : public ScanOperatorBuilder {
+public:
+    OlapScanOperatorBuilder(uint32_t id, const std::string& name,
+                            vectorized::NewOlapScanNode* new_olap_scan_node);
 
-    Status close(RuntimeState* state, Status exec_status) override;
+    OperatorPtr build_operator() override {
+        return std::make_shared<OlapScanOperator>(this, _new_olap_scan_node);
+    }
 
 private:
-    ODBCConnectorParam _odbc_param;
-    std::unique_ptr<ODBCConnector> _writer;
+    vectorized::NewOlapScanNode* _new_olap_scan_node;
 };
-} // namespace vectorized
-} // namespace doris
+
+} // namespace doris::pipeline
\ No newline at end of file
diff --git a/be/src/pipeline/exec/operator.cpp b/be/src/pipeline/exec/operator.cpp
new file mode 100644
index 0000000000..572cd1ffcc
--- /dev/null
+++ b/be/src/pipeline/exec/operator.cpp
@@ -0,0 +1,91 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "operator.h"
+
+namespace doris::pipeline {
+
+Operator::Operator(OperatorBuilder* operator_builder)
+        : _operator_builder(operator_builder), _is_closed(false) {}
+
+bool Operator::is_sink() const {
+    return _operator_builder->is_sink();
+}
+
+bool Operator::is_source() const {
+    return _operator_builder->is_source();
+}
+
+Status Operator::init(ExecNode* exec_node, RuntimeState* state) {
+    _runtime_profile.reset(new RuntimeProfile(_operator_builder->get_name()));
+    if (exec_node) {
+        exec_node->runtime_profile()->insert_child_head(_runtime_profile.get(), true);
+    }
+    return Status::OK();
+}
+
+Status Operator::prepare(RuntimeState* state) {
+    _mem_tracker = std::make_unique<MemTracker>("Operator:" + _runtime_profile->name(),
+                                                _runtime_profile.get());
+    return Status::OK();
+}
+
+Status Operator::open(RuntimeState* state) {
+    return Status::OK();
+}
+
+Status Operator::close(RuntimeState* state) {
+    if (_is_closed) {
+        return Status::OK();
+    }
+    _is_closed = true;
+    return Status::OK();
+}
+
+const RowDescriptor& Operator::row_desc() {
+    return _operator_builder->row_desc();
+}
+
+void Operator::_fresh_exec_timer(doris::ExecNode* node) {
+    node->_runtime_profile->total_time_counter()->update(
+            _runtime_profile->total_time_counter()->value());
+}
+
+std::string Operator::debug_string() const {
+    std::stringstream ss;
+    ss << _operator_builder->get_name() << ", source: " << is_source();
+    ss << ", sink: " << is_sink() << ", is closed: " << _is_closed;
+    ss << ", is pending finish: " << is_pending_finish();
+    return ss.str();
+}
+
+/////////////////////////////////////// OperatorBuilder ////////////////////////////////////////////////////////////
+
+Status OperatorBuilder::prepare(doris::RuntimeState* state) {
+    _state = state;
+    // runtime filter, now dispose by NewOlapScanNode
+    return Status::OK();
+}
+
+void OperatorBuilder::close(doris::RuntimeState* state) {
+    if (_is_closed) {
+        return;
+    }
+    _is_closed = true;
+}
+
+} // namespace doris::pipeline
diff --git a/be/src/pipeline/exec/operator.h b/be/src/pipeline/exec/operator.h
new file mode 100644
index 0000000000..d65d0291b3
--- /dev/null
+++ b/be/src/pipeline/exec/operator.h
@@ -0,0 +1,198 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include <utility>
+
+#include "common/status.h"
+#include "exec/exec_node.h"
+#include "runtime/runtime_state.h"
+#include "vec/core/block.h"
+
+namespace doris::pipeline {
+
+// Result of source pull data, init state is DEPEND_ON_SOURCE
+enum class SourceState : uint8_t {
+    DEPEND_ON_SOURCE = 0, // Operator has no more data in itself, needs to read from source.
+    MORE_DATA = 1,        // Still have data can read
+    FINISHED = 2
+};
+
+//
+enum class SinkState : uint8_t {
+    SINK_IDLE = 0, // can send block to sink
+    SINK_BUSY = 1, // sink buffer is full, should wait sink to send some block
+    FINISHED = 2
+};
+////////////////       DO NOT USE THE UP State     ////////////////
+
+class OperatorBuilder;
+class Operator;
+
+using OperatorPtr = std::shared_ptr<Operator>;
+using Operators = std::vector<OperatorPtr>;
+
+class Operator {
+public:
+    explicit Operator(OperatorBuilder* operator_builder);
+    virtual ~Operator() = default;
+
+    // After both sink and source need to know the cancel state.
+    // do cancel work
+    bool is_sink() const;
+
+    bool is_source() const;
+
+    // Should be call after ExecNode is constructed
+    virtual Status init(ExecNode* exec_node, RuntimeState* state = nullptr);
+
+    // Only result sink and data stream sink need to impl the virtual function
+    virtual Status init(const TDataSink& tsink) { return Status::OK(); };
+
+    // Do prepare some state of Operator
+    virtual Status prepare(RuntimeState* state);
+
+    // Like ExecNode,when pipeline task first time be scheduled, can't block
+    // the pipeline should be open after dependencies is finish
+    // Eg a -> c, b-> c, after a, b pipeline finish, c pipeline should call open
+    // Now the pipeline only have one task, so the there is no performance bottleneck for the mechanism,
+    // but if one pipeline have multi task to parallel work, need to rethink the logic
+    //
+    // Each operator should call open_self() to prepare resource to do data compute.
+    // if ExecNode split to sink and source operator, open_self() should be called in sink operator
+    virtual Status open(RuntimeState* state);
+
+    // Release the resource, should not block the thread
+    //
+    // Each operator should call close_self() to release resource
+    // if ExecNode split to sink and source operator, close_self() should be called in source operator
+    virtual Status close(RuntimeState* state);
+
+    Status set_child(OperatorPtr child) {
+        if (is_source()) {
+            return Status::InternalError("source can not has child.");
+        }
+        _child = std::move(child);
+        return Status::OK();
+    }
+
+    virtual bool can_read() { return false; } // for source
+
+    virtual bool can_write() { return false; } // for sink
+
+    // for pipeline
+    virtual Status get_block([[maybe_unused]] RuntimeState* runtime_state,
+                             [[maybe_unused]] vectorized::Block* block,
+                             [[maybe_unused]] SourceState& result_state) {
+        std::stringstream error_msg;
+        error_msg << " has not implements get_block";
+        return Status::NotSupported(error_msg.str());
+    }
+
+    // return can write continue
+    virtual Status sink(RuntimeState* state, vectorized::Block* block, SourceState source_state) {
+        std::stringstream error_msg;
+        error_msg << " not a sink ";
+        return Status::NotSupported(error_msg.str());
+    }
+
+    virtual Status finalize(RuntimeState* state) {
+        std::stringstream error_msg;
+        error_msg << " not a sink, can not finalize";
+        return Status::NotSupported(error_msg.str());
+    }
+
+    // close be called
+    // - Source: scan thread do not exist
+    // - Sink: RPC do not be disposed
+    // - else return false
+    virtual bool is_pending_finish() const { return false; }
+
+    // TODO: should we keep the function
+    // virtual bool is_finished() = 0;
+
+    bool is_closed() const { return _is_closed; }
+
+    MemTracker* mem_tracker() const { return _mem_tracker.get(); }
+
+    const OperatorBuilder* operator_builder() const { return _operator_builder; }
+
+    const RowDescriptor& row_desc();
+
+    RuntimeProfile* runtime_profile() { return _runtime_profile.get(); }
+    std::string debug_string() const;
+
+protected:
+    void _fresh_exec_timer(ExecNode* node);
+
+    std::unique_ptr<MemTracker> _mem_tracker;
+
+    OperatorBuilder* _operator_builder;
+    // source has no child
+    // if an operator is not source, it will get data from its child.
+    OperatorPtr _child;
+
+    std::unique_ptr<RuntimeProfile> _runtime_profile;
+    // TODO pipeline Account for peak memory used by this operator
+    RuntimeProfile::Counter* _memory_used_counter = nullptr;
+
+private:
+    bool _is_closed = false;
+};
+
+class OperatorBuilder {
+public:
+    OperatorBuilder(int32_t id, const std::string& name, ExecNode* exec_node = nullptr)
+            : _id(id), _name(name), _related_exec_node(exec_node) {}
+
+    virtual ~OperatorBuilder() = default;
+
+    virtual OperatorPtr build_operator() = 0;
+
+    virtual bool is_sink() const { return false; }
+    virtual bool is_source() const { return false; }
+
+    // create the object used by all operator
+    virtual Status prepare(RuntimeState* state);
+
+    // destory the object used by all operator
+    virtual void close(RuntimeState* state);
+
+    std::string get_name() const { return _name; }
+
+    RuntimeState* runtime_state() { return _state; }
+
+    const RowDescriptor& row_desc() { return _related_exec_node->row_desc(); }
+
+    ExecNode* exec_node() const { return _related_exec_node; }
+
+    int32_t id() const { return _id; }
+
+protected:
+    const int32_t _id;
+    const std::string _name;
+    ExecNode* _related_exec_node;
+
+    RuntimeState* _state = nullptr;
+    bool _is_closed = false;
+};
+
+using OperatorBuilderPtr = std::shared_ptr<OperatorBuilder>;
+using OperatorBuilders = std::vector<OperatorBuilderPtr>;
+
+} // namespace doris::pipeline
diff --git a/be/src/pipeline/exec/repeat_operator.cpp b/be/src/pipeline/exec/repeat_operator.cpp
new file mode 100644
index 0000000000..467057e034
--- /dev/null
+++ b/be/src/pipeline/exec/repeat_operator.cpp
@@ -0,0 +1,76 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "repeat_operator.h"
+
+#include "vec/exec/vrepeat_node.h"
+
+namespace doris {
+namespace pipeline {
+
+RepeatOperator::RepeatOperator(RepeatOperatorBuilder* operator_builder,
+                               vectorized::VRepeatNode* repeat_node)
+        : Operator(operator_builder), _repeat_node(repeat_node) {}
+
+Status RepeatOperator::open(RuntimeState* state) {
+    SCOPED_TIMER(_runtime_profile->total_time_counter());
+    RETURN_IF_ERROR(Operator::open(state));
+    _child_block.reset(new vectorized::Block);
+    return _repeat_node->alloc_resource(state);
+}
+
+Status RepeatOperator::close(RuntimeState* state) {
+    _fresh_exec_timer(_repeat_node);
+    _repeat_node->release_resource(state);
+    Operator::close(state);
+    return Status::OK();
+}
+
+Status RepeatOperator::get_block(RuntimeState* state, vectorized::Block* block,
+                                 SourceState& source_state) {
+    SCOPED_TIMER(_runtime_profile->total_time_counter());
+    if (_repeat_node->need_more_input_data()) {
+        RETURN_IF_ERROR(_child->get_block(state, _child_block.get(), _child_source_state));
+        source_state = _child_source_state;
+        if (_child_block->rows() == 0) {
+            return Status::OK();
+        }
+
+        _repeat_node->push(state, _child_block.get(), source_state == SourceState::FINISHED);
+    }
+
+    bool eos = false;
+    RETURN_IF_ERROR(_repeat_node->pull(state, block, &eos));
+    if (eos) {
+        source_state = SourceState::FINISHED;
+        _child_block->clear_column_data();
+    } else if (!_repeat_node->need_more_input_data()) {
+        source_state = SourceState::MORE_DATA;
+    } else {
+        _child_block->clear_column_data();
+    }
+    return Status::OK();
+}
+
+RepeatOperatorBuilder::RepeatOperatorBuilder(int32_t id, vectorized::VRepeatNode* repeat_node)
+        : OperatorBuilder(id, "RepeatOperatorBuilder", repeat_node), _repeat_node(repeat_node) {}
+
+OperatorPtr RepeatOperatorBuilder::build_operator() {
+    return std::make_shared<RepeatOperator>(this, _repeat_node);
+}
+} // namespace pipeline
+} // namespace doris
diff --git a/be/src/vec/sink/vmysql_table_sink.h b/be/src/pipeline/exec/repeat_operator.h
similarity index 53%
copy from be/src/vec/sink/vmysql_table_sink.h
copy to be/src/pipeline/exec/repeat_operator.h
index 6a30275a8e..e2aaaeab02 100644
--- a/be/src/vec/sink/vmysql_table_sink.h
+++ b/be/src/pipeline/exec/repeat_operator.h
@@ -14,33 +14,45 @@
 // KIND, either express or implied.  See the License for the
 // specific language governing permissions and limitations
 // under the License.
+
 #pragma once
-#include <vector>
 
-#include "common/status.h"
-#include "vec/sink/vmysql_table_writer.h"
-#include "vec/sink/vtable_sink.h"
+#include "operator.h"
 
 namespace doris {
 namespace vectorized {
-
-// This class is a sinker, which put input data to mysql table
-class VMysqlTableSink : public VTableSink {
+class VRepeatNode;
+class VExprContext;
+class Block;
+} // namespace vectorized
+namespace pipeline {
+class RepeatOperatorBuilder;
+class RepeatOperator : public Operator {
 public:
-    VMysqlTableSink(ObjectPool* pool, const RowDescriptor& row_desc,
-                    const std::vector<TExpr>& t_exprs);
-
-    Status init(const TDataSink& thrift_sink) override;
+    RepeatOperator(RepeatOperatorBuilder* operator_builder, vectorized::VRepeatNode* repeat_node);
 
     Status open(RuntimeState* state) override;
 
-    Status send(RuntimeState* state, vectorized::Block* block) override;
+    Status close(RuntimeState* state) override;
 
-    Status close(RuntimeState* state, Status exec_status) override;
+    Status get_block(RuntimeState* state, vectorized::Block* block,
+                     SourceState& source_state) override;
 
 private:
-    MysqlConnInfo _conn_info;
-    std::unique_ptr<VMysqlTableWriter> _writer;
+    vectorized::VRepeatNode* _repeat_node;
+    std::unique_ptr<vectorized::Block> _child_block;
+    SourceState _child_source_state;
 };
-} // namespace vectorized
+
+class RepeatOperatorBuilder : public OperatorBuilder {
+public:
+    RepeatOperatorBuilder(int32_t id, vectorized::VRepeatNode* repeat_node);
+
+    OperatorPtr build_operator() override;
+
+private:
+    vectorized::VRepeatNode* _repeat_node;
+};
+
+} // namespace pipeline
 } // namespace doris
diff --git a/be/src/pipeline/exec/result_sink_operator.cpp b/be/src/pipeline/exec/result_sink_operator.cpp
new file mode 100644
index 0000000000..fb2a14355e
--- /dev/null
+++ b/be/src/pipeline/exec/result_sink_operator.cpp
@@ -0,0 +1,69 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "result_sink_operator.h"
+
+#include "runtime/buffer_control_block.h"
+#include "vec/sink/vresult_sink.h"
+
+namespace doris::pipeline {
+ResultSinkOperator::ResultSinkOperator(OperatorBuilder* operator_builder,
+                                       vectorized::VResultSink* sink)
+        : Operator(operator_builder), _sink(sink) {}
+
+Status ResultSinkOperator::init(const TDataSink& tsink) {
+    return Status::OK();
+}
+
+Status ResultSinkOperator::prepare(RuntimeState* state) {
+    RETURN_IF_ERROR(Operator::prepare(state));
+    return _sink->prepare(state);
+}
+
+Status ResultSinkOperator::open(RuntimeState* state) {
+    SCOPED_TIMER(_runtime_profile->total_time_counter());
+    return _sink->open(state);
+}
+
+bool ResultSinkOperator::can_write() {
+    return _sink->_sender->can_sink();
+}
+
+Status ResultSinkOperator::sink(RuntimeState* state, vectorized::Block* block,
+                                SourceState source_state) {
+    SCOPED_TIMER(_runtime_profile->total_time_counter());
+    if (!block) {
+        DCHECK(source_state == SourceState::FINISHED)
+                << "block is null, eos should invoke in finalize.";
+        return Status::OK();
+    }
+    return _sink->send(state, block);
+}
+
+Status ResultSinkOperator::finalize(RuntimeState* state) {
+    _finalized = true;
+    return _sink->close(state, Status::OK());
+}
+
+// TODO: Support fresh exec time for sink
+Status ResultSinkOperator::close(RuntimeState* state) {
+    if (!_finalized) {
+        RETURN_IF_ERROR(_sink->close(state, Status::InternalError("Not finalized")));
+    }
+    return Status::OK();
+}
+} // namespace doris::pipeline
\ No newline at end of file
diff --git a/be/src/pipeline/exec/result_sink_operator.h b/be/src/pipeline/exec/result_sink_operator.h
new file mode 100644
index 0000000000..190cdfb570
--- /dev/null
+++ b/be/src/pipeline/exec/result_sink_operator.h
@@ -0,0 +1,69 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include "operator.h"
+
+namespace doris {
+namespace vectorized {
+class VResultSink;
+}
+
+namespace pipeline {
+
+class ResultSinkOperator : public Operator {
+public:
+    ResultSinkOperator(OperatorBuilder* operator_builder, vectorized::VResultSink* sink);
+
+    Status init(const TDataSink& tsink) override;
+
+    Status prepare(RuntimeState* state) override;
+
+    Status open(RuntimeState* state) override;
+
+    bool can_write() override;
+
+    Status sink(RuntimeState* state, vectorized::Block* block, SourceState source_state) override;
+
+    Status finalize(RuntimeState* state) override;
+
+    Status close(RuntimeState* state) override;
+
+private:
+    vectorized::VResultSink* _sink;
+    bool _finalized = false;
+};
+
+class ResultSinkOperatorBuilder : public OperatorBuilder {
+public:
+    ResultSinkOperatorBuilder(int32_t id, const std::string& name, ExecNode* exec_node,
+                              vectorized::VResultSink* sink)
+            : OperatorBuilder(id, name, exec_node), _sink(sink) {}
+
+    bool is_sink() const override { return true; }
+
+    OperatorPtr build_operator() override {
+        return std::make_shared<ResultSinkOperator>(this, _sink);
+    }
+
+private:
+    vectorized::VResultSink* _sink;
+};
+
+} // namespace pipeline
+} // namespace doris
\ No newline at end of file
diff --git a/be/src/pipeline/exec/scan_operator.cpp b/be/src/pipeline/exec/scan_operator.cpp
new file mode 100644
index 0000000000..4a06a80c95
--- /dev/null
+++ b/be/src/pipeline/exec/scan_operator.cpp
@@ -0,0 +1,68 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "scan_operator.h"
+
+#include "vec/exec/scan/scanner_context.h"
+#include "vec/exec/scan/vscan_node.h"
+
+namespace doris::pipeline {
+
+ScanOperator::ScanOperator(OperatorBuilder* operator_builder, vectorized::VScanNode* scan_node)
+        : Operator(operator_builder), _scan_node(scan_node) {}
+
+Status ScanOperator::open(RuntimeState* state) {
+    SCOPED_TIMER(_runtime_profile->total_time_counter());
+    RETURN_IF_ERROR(Operator::open(state));
+    return _scan_node->open(state);
+}
+
+bool ScanOperator::can_read() {
+    if (_scan_node->_eos || !_scan_node->_scanner_ctx || _scan_node->_scanner_ctx->done() ||
+        _scan_node->_scanner_ctx->can_finish()) {
+        // _eos: need eos
+        // !_scanner_ctx: need call open
+        // _scanner_ctx->done(): need finish
+        // _scanner_ctx->can_finish(): should be scheduled
+        return true;
+    } else {
+        return !_scan_node->_scanner_ctx->empty_in_queue(); // have block to process
+    }
+}
+
+Status ScanOperator::get_block(RuntimeState* state, vectorized::Block* block,
+                               SourceState& result_state) {
+    SCOPED_TIMER(_runtime_profile->total_time_counter());
+    bool eos = false;
+    RETURN_IF_ERROR(_scan_node->get_next(state, block, &eos));
+    result_state = eos ? SourceState::FINISHED : SourceState::DEPEND_ON_SOURCE;
+    return Status::OK();
+}
+
+bool ScanOperator::is_pending_finish() const {
+    return _scan_node->_scanner_ctx && !_scan_node->_scanner_ctx->can_finish();
+}
+
+Status ScanOperator::close(RuntimeState* state) {
+    if (!is_closed()) {
+        RETURN_IF_ERROR(_scan_node->close(state));
+    }
+    _fresh_exec_timer(_scan_node);
+    return Operator::close(state);
+}
+
+} // namespace doris::pipeline
diff --git a/be/src/pipeline/exec/scan_operator.h b/be/src/pipeline/exec/scan_operator.h
new file mode 100644
index 0000000000..1f32f1c899
--- /dev/null
+++ b/be/src/pipeline/exec/scan_operator.h
@@ -0,0 +1,59 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include <utility>
+
+#include "operator.h"
+
+namespace doris::vectorized {
+class VScanNode;
+class VScanner;
+class ScannerContext;
+} // namespace doris::vectorized
+
+namespace doris::pipeline {
+
+class ScanOperator : public Operator {
+public:
+    ScanOperator(OperatorBuilder* operator_builder, vectorized::VScanNode* scan_node);
+
+    bool can_read() override; // for source
+
+    Status get_block(RuntimeState* state, vectorized::Block* block,
+                     SourceState& result_state) override;
+
+    bool is_pending_finish() const override;
+
+    Status open(RuntimeState* state) override;
+
+    Status close(RuntimeState* state) override;
+
+private:
+    vectorized::VScanNode* _scan_node;
+};
+
+class ScanOperatorBuilder : public OperatorBuilder {
+public:
+    ScanOperatorBuilder(int32_t id, const std::string& name, ExecNode* exec_node)
+            : OperatorBuilder(id, name, exec_node) {}
+
+    bool is_source() const override { return true; }
+};
+
+} // namespace doris::pipeline
\ No newline at end of file
diff --git a/be/src/pipeline/exec/sort_sink_operator.cpp b/be/src/pipeline/exec/sort_sink_operator.cpp
new file mode 100644
index 0000000000..273f20198c
--- /dev/null
+++ b/be/src/pipeline/exec/sort_sink_operator.cpp
@@ -0,0 +1,51 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "sort_sink_operator.h"
+
+#include "vec/exec/vsort_node.h"
+
+namespace doris::pipeline {
+
+SortSinkOperatorBuilder::SortSinkOperatorBuilder(int32_t id, const string& name,
+                                                 vectorized::VSortNode* sort_node)
+        : OperatorBuilder(id, name, sort_node), _sort_node(sort_node) {}
+
+SortSinkOperator::SortSinkOperator(SortSinkOperatorBuilder* operator_builder,
+                                   vectorized::VSortNode* sort_node)
+        : Operator(operator_builder), _sort_node(sort_node) {}
+
+Status SortSinkOperator::open(doris::RuntimeState* state) {
+    SCOPED_TIMER(_runtime_profile->total_time_counter());
+    RETURN_IF_ERROR(Operator::open(state));
+    RETURN_IF_ERROR(_sort_node->alloc_resource(state));
+    return Status::OK();
+}
+
+Status SortSinkOperator::close(doris::RuntimeState* /*state*/) {
+    _fresh_exec_timer(_sort_node);
+    return Status::OK();
+}
+
+Status SortSinkOperator::sink(doris::RuntimeState* state, vectorized::Block* block,
+                              SourceState source_state) {
+    SCOPED_TIMER(_runtime_profile->total_time_counter());
+    // TODO pipeline when sort node's _reuse_mem is false, we should pass a new block to it.
+    RETURN_IF_ERROR(_sort_node->sink(state, block, source_state == SourceState::FINISHED));
+    return Status::OK();
+}
+} // namespace doris::pipeline
diff --git a/be/src/pipeline/exec/sort_sink_operator.h b/be/src/pipeline/exec/sort_sink_operator.h
new file mode 100644
index 0000000000..96adbfa31e
--- /dev/null
+++ b/be/src/pipeline/exec/sort_sink_operator.h
@@ -0,0 +1,70 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include <utility>
+
+#include "operator.h"
+
+namespace doris {
+
+namespace vectorized {
+class VSortNode;
+}
+
+namespace pipeline {
+
+class SortSinkOperatorBuilder;
+
+class SortSinkOperator : public Operator {
+public:
+    SortSinkOperator(SortSinkOperatorBuilder* operator_builder, vectorized::VSortNode* sort_node);
+
+    Status open(RuntimeState* state) override;
+
+    Status close(RuntimeState* state) override;
+
+    // return can write continue
+    Status sink(RuntimeState* state, vectorized::Block* block, SourceState source_state) override;
+
+    Status finalize(RuntimeState* /*state*/) override { return Status::OK(); }
+
+    bool can_write() override { return true; };
+
+private:
+    vectorized::VSortNode* _sort_node;
+};
+
+class SortSinkOperatorBuilder : public OperatorBuilder {
+public:
+    SortSinkOperatorBuilder(int32_t id, const std::string& name, vectorized::VSortNode* sort_node);
+
+    bool is_sink() const override { return true; }
+
+    bool is_source() const override { return false; }
+
+    OperatorPtr build_operator() override {
+        return std::make_shared<SortSinkOperator>(this, _sort_node);
+    }
+
+private:
+    vectorized::VSortNode* _sort_node;
+};
+
+} // namespace pipeline
+} // namespace doris
diff --git a/be/src/pipeline/exec/sort_source_operator.cpp b/be/src/pipeline/exec/sort_source_operator.cpp
new file mode 100644
index 0000000000..4f25b216c3
--- /dev/null
+++ b/be/src/pipeline/exec/sort_source_operator.cpp
@@ -0,0 +1,54 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "sort_source_operator.h"
+
+#include "vec/exec/vsort_node.h"
+
+namespace doris::pipeline {
+
+SortSourceOperatorBuilder::SortSourceOperatorBuilder(int32_t id, const string& name,
+                                                     vectorized::VSortNode* sort_node)
+        : OperatorBuilder(id, name, sort_node), _sort_node(sort_node) {}
+
+SortSourceOperator::SortSourceOperator(SortSourceOperatorBuilder* operator_builder,
+                                       vectorized::VSortNode* sort_node)
+        : Operator(operator_builder), _sort_node(sort_node) {}
+
+Status SortSourceOperator::close(doris::RuntimeState* state) {
+    if (is_closed()) {
+        return Status::OK();
+    }
+    _fresh_exec_timer(_sort_node);
+    _sort_node->release_resource(state);
+    return Operator::close(state);
+}
+
+bool SortSourceOperator::can_read() {
+    return _sort_node->can_read();
+}
+
+Status SortSourceOperator::get_block(RuntimeState* state, vectorized::Block* block,
+                                     SourceState& source_state) {
+    SCOPED_TIMER(_runtime_profile->total_time_counter());
+    bool eos = false;
+    RETURN_IF_ERROR(_sort_node->pull(state, block, &eos));
+    source_state = eos ? SourceState::FINISHED : SourceState::DEPEND_ON_SOURCE;
+    return Status::OK();
+}
+
+} // namespace doris::pipeline
diff --git a/be/src/pipeline/exec/sort_source_operator.h b/be/src/pipeline/exec/sort_source_operator.h
new file mode 100644
index 0000000000..9b83ce06fb
--- /dev/null
+++ b/be/src/pipeline/exec/sort_source_operator.h
@@ -0,0 +1,68 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include <utility>
+
+#include "operator.h"
+
+namespace doris {
+
+namespace vectorized {
+class VSortNode;
+}
+
+namespace pipeline {
+
+class SortSourceOperatorBuilder;
+
+class SortSourceOperator : public Operator {
+public:
+    SortSourceOperator(SortSourceOperatorBuilder* operator_builder,
+                       vectorized::VSortNode* sort_node);
+
+    Status close(RuntimeState* state) override;
+
+    Status get_block(RuntimeState* state, vectorized::Block* block,
+                     SourceState& source_state) override;
+
+    bool can_read() override;
+
+private:
+    vectorized::VSortNode* _sort_node;
+};
+
+class SortSourceOperatorBuilder : public OperatorBuilder {
+public:
+    SortSourceOperatorBuilder(int32_t id, const std::string& name,
+                              vectorized::VSortNode* sort_node);
+
+    bool is_sink() const override { return false; }
+
+    bool is_source() const override { return true; }
+
+    OperatorPtr build_operator() override {
+        return std::make_shared<SortSourceOperator>(this, _sort_node);
+    }
+
+private:
+    vectorized::VSortNode* _sort_node;
+};
+
+} // namespace pipeline
+} // namespace doris
\ No newline at end of file
diff --git a/be/src/pipeline/exec/streaming_aggregation_sink_operator.cpp b/be/src/pipeline/exec/streaming_aggregation_sink_operator.cpp
new file mode 100644
index 0000000000..9c1a2a57e8
--- /dev/null
+++ b/be/src/pipeline/exec/streaming_aggregation_sink_operator.cpp
@@ -0,0 +1,101 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "streaming_aggregation_sink_operator.h"
+
+#include "vec/exec/vaggregation_node.h"
+
+namespace doris::pipeline {
+
+StreamingAggSinkOperator::StreamingAggSinkOperator(
+        StreamingAggSinkOperatorBuilder* operator_builder, vectorized::AggregationNode* agg_node,
+        std::shared_ptr<AggContext> agg_context)
+        : Operator(operator_builder), _agg_node(agg_node), _agg_context(std::move(agg_context)) {}
+
+Status StreamingAggSinkOperator::prepare(RuntimeState* state) {
+    RETURN_IF_ERROR(Operator::prepare(state));
+    _queue_byte_size_counter =
+            ADD_COUNTER(_runtime_profile.get(), "MaxSizeInBlockQueue", TUnit::BYTES);
+    _queue_size_counter = ADD_COUNTER(_runtime_profile.get(), "MaxSizeOfBlockQueue", TUnit::UNIT);
+    return Status::OK();
+}
+
+Status StreamingAggSinkOperator::open(RuntimeState* state) {
+    SCOPED_TIMER(_runtime_profile->total_time_counter());
+    RETURN_IF_ERROR(Operator::open(state));
+    RETURN_IF_ERROR(_agg_node->alloc_resource(state));
+    return Status::OK();
+}
+
+bool StreamingAggSinkOperator::can_write() {
+    // sink and source in diff threads
+    return _agg_context->has_enough_space_to_push();
+}
+
+Status StreamingAggSinkOperator::sink(RuntimeState* state, vectorized::Block* in_block,
+                                      SourceState source_state) {
+    SCOPED_TIMER(_runtime_profile->total_time_counter());
+    Status ret = Status::OK();
+    if (in_block && in_block->rows() > 0) {
+        auto bock_from_ctx = _agg_context->get_free_block();
+        RETURN_IF_ERROR(_agg_node->do_pre_agg(in_block, bock_from_ctx.get()));
+        if (bock_from_ctx->rows() == 0) {
+            _agg_context->return_free_block(std::move(bock_from_ctx));
+        } else {
+            _agg_context->push_block(std::move(bock_from_ctx));
+        }
+    }
+
+    if (UNLIKELY(source_state == SourceState::FINISHED)) {
+        _agg_context->set_finish();
+    }
+    return Status::OK();
+}
+
+Status StreamingAggSinkOperator::close(RuntimeState* state) {
+    _fresh_exec_timer(_agg_node);
+    if (_agg_context && !_agg_context->is_finish()) {
+        // finish should be set, if not set here means error.
+        _agg_context->set_canceled();
+    }
+    COUNTER_SET(_queue_size_counter, _agg_context->max_size_of_queue());
+    COUNTER_SET(_queue_byte_size_counter, _agg_context->max_bytes_in_queue());
+    return Status::OK();
+}
+
+///////////////////////////////  operator template  ////////////////////////////////
+
+StreamingAggSinkOperatorBuilder::StreamingAggSinkOperatorBuilder(
+        int32_t id, const std::string& name, vectorized::AggregationNode* exec_node,
+        std::shared_ptr<AggContext> agg_context)
+        : OperatorBuilder(id, name, exec_node),
+          _agg_node(exec_node),
+          _agg_context(std::move(agg_context)) {}
+
+OperatorPtr StreamingAggSinkOperatorBuilder::build_operator() {
+    return std::make_shared<StreamingAggSinkOperator>(this, _agg_node, _agg_context);
+}
+
+// use final aggregation source operator
+bool StreamingAggSinkOperatorBuilder::is_sink() const {
+    return true;
+}
+
+bool StreamingAggSinkOperatorBuilder::is_source() const {
+    return false;
+}
+} // namespace doris::pipeline
diff --git a/be/src/pipeline/exec/streaming_aggregation_sink_operator.h b/be/src/pipeline/exec/streaming_aggregation_sink_operator.h
new file mode 100644
index 0000000000..691521ee9b
--- /dev/null
+++ b/be/src/pipeline/exec/streaming_aggregation_sink_operator.h
@@ -0,0 +1,75 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include "agg_context.h"
+#include "operator.h"
+
+namespace doris {
+namespace vectorized {
+class AggregationNode;
+class VExprContext;
+class Block;
+} // namespace vectorized
+
+namespace pipeline {
+class StreamingAggSinkOperatorBuilder;
+class StreamingAggSinkOperator : public Operator {
+public:
+    StreamingAggSinkOperator(StreamingAggSinkOperatorBuilder* operator_builder,
+                             vectorized::AggregationNode*, std::shared_ptr<AggContext>);
+
+    Status prepare(RuntimeState*) override;
+
+    Status open(RuntimeState* state) override;
+
+    Status sink(RuntimeState* state, vectorized::Block* block, SourceState source_state) override;
+
+    bool can_write() override;
+
+    Status close(RuntimeState* state) override;
+
+    Status finalize(doris::RuntimeState* state) override { return Status::OK(); }
+
+private:
+    vectorized::AggregationNode* _agg_node;
+    vectorized::Block _preagg_block = vectorized::Block();
+
+    RuntimeProfile::Counter* _queue_byte_size_counter;
+    RuntimeProfile::Counter* _queue_size_counter;
+
+    std::shared_ptr<AggContext> _agg_context;
+};
+
+class StreamingAggSinkOperatorBuilder : public OperatorBuilder {
+public:
+    StreamingAggSinkOperatorBuilder(int32_t, const std::string&, vectorized::AggregationNode*,
+                                    std::shared_ptr<AggContext>);
+
+    OperatorPtr build_operator() override;
+
+    bool is_sink() const override;
+    bool is_source() const override;
+
+private:
+    vectorized::AggregationNode* _agg_node;
+    std::shared_ptr<AggContext> _agg_context;
+};
+
+} // namespace pipeline
+} // namespace doris
\ No newline at end of file
diff --git a/be/src/pipeline/exec/streaming_aggregation_source_operator.cpp b/be/src/pipeline/exec/streaming_aggregation_source_operator.cpp
new file mode 100644
index 0000000000..3270b553fd
--- /dev/null
+++ b/be/src/pipeline/exec/streaming_aggregation_source_operator.cpp
@@ -0,0 +1,86 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "streaming_aggregation_source_operator.h"
+
+#include "vec/exec/vaggregation_node.h"
+
+namespace doris {
+namespace pipeline {
+StreamingAggSourceOperator::StreamingAggSourceOperator(OperatorBuilder* templ,
+                                                       vectorized::AggregationNode* node,
+                                                       std::shared_ptr<AggContext> agg_context)
+        : Operator(templ), _agg_node(node), _agg_context(std::move(agg_context)) {}
+
+Status StreamingAggSourceOperator::prepare(RuntimeState* state) {
+    _agg_node->increase_ref();
+    return Status::OK();
+}
+
+bool StreamingAggSourceOperator::can_read() {
+    return _agg_context->has_data_or_finished();
+}
+
+Status StreamingAggSourceOperator::get_block(RuntimeState* state, vectorized::Block* block,
+                                             SourceState& source_state) {
+    SCOPED_TIMER(_runtime_profile->total_time_counter());
+    bool eos = false;
+    if (!_agg_context->data_exhausted()) {
+        std::unique_ptr<vectorized::Block> agg_block;
+        RETURN_IF_ERROR(_agg_context->get_block(&agg_block));
+
+        if (_agg_context->data_exhausted()) {
+            RETURN_IF_ERROR(_agg_node->pull(state, block, &eos));
+        } else {
+            block->swap(*agg_block);
+            agg_block->clear_column_data(_agg_node->row_desc().num_materialized_slots());
+            _agg_context->return_free_block(std::move(agg_block));
+        }
+    } else {
+        RETURN_IF_ERROR(_agg_node->pull(state, block, &eos));
+    }
+
+    source_state = eos ? SourceState::FINISHED : SourceState::DEPEND_ON_SOURCE;
+
+    return Status::OK();
+}
+
+Status StreamingAggSourceOperator::close(RuntimeState* state) {
+    if (is_closed()) {
+        return Status::OK();
+    }
+    _fresh_exec_timer(_agg_node);
+    if (!_agg_node->decrease_ref()) {
+        _agg_node->release_resource(state);
+    }
+    return Operator::close(state);
+}
+
+///////////////////////////////  operator template  ////////////////////////////////
+
+StreamingAggSourceOperatorBuilder::StreamingAggSourceOperatorBuilder(
+        int32_t id, const std::string& name, vectorized::AggregationNode* exec_node,
+        std::shared_ptr<AggContext> agg_context)
+        : OperatorBuilder(id, name, exec_node), _agg_context(std::move(agg_context)) {}
+
+OperatorPtr StreamingAggSourceOperatorBuilder::build_operator() {
+    return std::make_shared<StreamingAggSourceOperator>(
+            this, assert_cast<vectorized::AggregationNode*>(_related_exec_node), _agg_context);
+}
+
+} // namespace pipeline
+} // namespace doris
diff --git a/be/src/pipeline/exec/streaming_aggregation_source_operator.h b/be/src/pipeline/exec/streaming_aggregation_source_operator.h
new file mode 100644
index 0000000000..106f1cf99d
--- /dev/null
+++ b/be/src/pipeline/exec/streaming_aggregation_source_operator.h
@@ -0,0 +1,56 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+#pragma once
+
+#include "agg_context.h"
+#include "operator.h"
+
+namespace doris {
+namespace vectorized {
+class AggregationNode;
+}
+namespace pipeline {
+
+class StreamingAggSourceOperator : public Operator {
+public:
+    StreamingAggSourceOperator(OperatorBuilder*, vectorized::AggregationNode*,
+                               std::shared_ptr<AggContext>);
+    Status prepare(RuntimeState* state) override;
+    bool can_read() override;
+    Status close(RuntimeState* state) override;
+    Status get_block(RuntimeState*, vectorized::Block*, SourceState& source_state) override;
+
+private:
+    vectorized::AggregationNode* _agg_node;
+    std::shared_ptr<AggContext> _agg_context;
+};
+
+class StreamingAggSourceOperatorBuilder : public OperatorBuilder {
+public:
+    StreamingAggSourceOperatorBuilder(int32_t, const std::string&, vectorized::AggregationNode*,
+                                      std::shared_ptr<AggContext>);
+
+    bool is_source() const override { return true; }
+
+    OperatorPtr build_operator() override;
+
+private:
+    std::shared_ptr<AggContext> _agg_context;
+};
+
+} // namespace pipeline
+} // namespace doris
\ No newline at end of file
diff --git a/be/src/pipeline/exec/table_function_operator.h b/be/src/pipeline/exec/table_function_operator.h
new file mode 100644
index 0000000000..0735b981cc
--- /dev/null
+++ b/be/src/pipeline/exec/table_function_operator.h
@@ -0,0 +1,92 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include "operator.h"
+#include "vec/exec/vtable_function_node.h"
+
+namespace doris {
+
+namespace pipeline {
+class TableFunctionOperator;
+
+class TableFunctionOperatorBuilder : public OperatorBuilder {
+public:
+    TableFunctionOperatorBuilder(int32_t id, vectorized::VTableFunctionNode* node)
+            : OperatorBuilder(id, "TableFunctionOperatorBuilder", node), _node(node) {}
+
+    OperatorPtr build_operator() override;
+
+private:
+    vectorized::VTableFunctionNode* _node;
+};
+
+class TableFunctionOperator : public Operator {
+public:
+    TableFunctionOperator(TableFunctionOperatorBuilder* operator_builder,
+                          vectorized::VTableFunctionNode* node)
+            : Operator(operator_builder), _node(node) {}
+
+    Status open(RuntimeState* state) override {
+        RETURN_IF_ERROR(Operator::open(state));
+        _child_block.reset(new vectorized::Block);
+        return _node->alloc_resource(state);
+    }
+
+    Status close(RuntimeState* state) override {
+        _node->release_resource(state);
+        _fresh_exec_timer(_node);
+        return Operator::close(state);
+    }
+
+    Status get_block(RuntimeState* state, vectorized::Block* block,
+                     SourceState& source_state) override {
+        if (_node->need_more_input_data()) {
+            RETURN_IF_ERROR(_child->get_block(state, _child_block.get(), _child_source_state));
+            source_state = _child_source_state;
+            if (_child_block->rows() == 0) {
+                return Status::OK();
+            }
+            _node->push(state, _child_block.get(), source_state == SourceState::FINISHED);
+        }
+
+        bool eos = false;
+        RETURN_IF_ERROR(_node->pull(state, block, &eos));
+        if (eos) {
+            source_state = SourceState::FINISHED;
+            _child_block->clear_column_data();
+        } else if (!_node->need_more_input_data()) {
+            source_state = SourceState::MORE_DATA;
+        } else {
+            _child_block->clear_column_data();
+        }
+        return Status::OK();
+    }
+
+private:
+    vectorized::VTableFunctionNode* _node;
+    std::unique_ptr<vectorized::Block> _child_block;
+    SourceState _child_source_state;
+};
+
+OperatorPtr TableFunctionOperatorBuilder::build_operator() {
+    return std::make_shared<TableFunctionOperator>(this, _node);
+}
+
+} // namespace pipeline
+} // namespace doris
diff --git a/be/src/pipeline/pipeline.cpp b/be/src/pipeline/pipeline.cpp
new file mode 100644
index 0000000000..7342d49850
--- /dev/null
+++ b/be/src/pipeline/pipeline.cpp
@@ -0,0 +1,76 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "pipeline.h"
+
+#include "pipeline_fragment_context.h"
+
+namespace doris::pipeline {
+
+Status Pipeline::prepare(RuntimeState* state) {
+    std::stringstream ss;
+    ss << "Pipeline"
+       << " (pipeline id=" << _pipeline_id << ")";
+    _pipeline_profile.reset(new RuntimeProfile(ss.str()));
+    for (auto& op : _operator_builders) {
+        RETURN_IF_ERROR(op->prepare(state));
+    }
+    RETURN_IF_ERROR(_sink->prepare(state));
+    return Status::OK();
+}
+
+Status Pipeline::build_operators(Operators& operators) {
+    OperatorPtr pre;
+    for (auto& operator_t : _operator_builders) {
+        auto o = operator_t->build_operator();
+        RETURN_IF_ERROR(o->init(operator_t->exec_node(), _context->get_runtime_state()));
+        if (pre) {
+            o->set_child(pre);
+        }
+        operators.emplace_back(o);
+        pre = std::move(o);
+    }
+    return Status::OK();
+}
+
+void Pipeline::close(RuntimeState* state) {
+    for (auto& op : _operator_builders) {
+        op->close(state);
+    }
+    _sink->close(state);
+}
+
+Status Pipeline::add_operator(OperatorBuilderPtr& op) {
+    if (_operator_builders.empty() && !op->is_source()) {
+        return Status::InternalError("Should set source before other operator");
+    }
+    _operator_builders.emplace_back(op);
+    return Status::OK();
+}
+
+Status Pipeline::set_sink(OperatorBuilderPtr& sink_) {
+    if (_sink) {
+        return Status::InternalError("set sink twice");
+    }
+    if (!sink_->is_sink()) {
+        return Status::InternalError("should set a sink operator but {}", typeid(sink_).name());
+    }
+    _sink = sink_;
+    return Status::OK();
+}
+
+} // namespace doris::pipeline
\ No newline at end of file
diff --git a/be/src/pipeline/pipeline.h b/be/src/pipeline/pipeline.h
new file mode 100644
index 0000000000..865b7b2a04
--- /dev/null
+++ b/be/src/pipeline/pipeline.h
@@ -0,0 +1,90 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include <atomic>
+#include <vector>
+
+#include "common/status.h"
+#include "exec/operator.h"
+#include "vec/core/block.h"
+
+namespace doris::pipeline {
+
+class Pipeline;
+using PipelinePtr = std::shared_ptr<Pipeline>;
+using Pipelines = std::vector<PipelinePtr>;
+using PipelineId = uint32_t;
+
+class PipelineTask;
+class PipelineFragmentContext;
+
+class Pipeline : public std::enable_shared_from_this<Pipeline> {
+    friend class PipelineTask;
+
+public:
+    Pipeline() = delete;
+    explicit Pipeline(PipelineId pipeline_id, std::shared_ptr<PipelineFragmentContext> context)
+            : _complete_dependency(0), _pipeline_id(pipeline_id), _context(std::move(context)) {}
+
+    Status prepare(RuntimeState* state);
+
+    void close(RuntimeState*);
+
+    void add_dependency(std::shared_ptr<Pipeline>& pipeline) {
+        pipeline->_parents.push_back(shared_from_this());
+        _dependencies.push_back(pipeline);
+    }
+
+    // If all dependency be finished, the pipeline task shoule be scheduled
+    // pipeline is finish must call the parents `finish_one_dependency`
+    // like the condition variables.
+    // Eg: hash build finish must call the hash probe the method
+    bool finish_one_dependency() {
+        DCHECK(_complete_dependency < _dependencies.size());
+        return _complete_dependency.fetch_add(1) == _dependencies.size() - 1;
+    }
+
+    bool has_dependency() { return _complete_dependency.load() < _dependencies.size(); }
+
+    Status add_operator(OperatorBuilderPtr& op);
+
+    Status set_sink(OperatorBuilderPtr& sink_operator);
+
+    OperatorBuilder* sink() { return _sink.get(); }
+
+    Status build_operators(Operators&);
+
+    RuntimeProfile* runtime_profile() { return _pipeline_profile.get(); }
+
+private:
+    std::atomic<uint32_t> _complete_dependency;
+
+    OperatorBuilders _operator_builders; // left is _source, right is _root
+    OperatorBuilderPtr _sink;            // put block to sink
+
+    std::vector<std::shared_ptr<Pipeline>> _parents;
+    std::vector<std::shared_ptr<Pipeline>> _dependencies;
+
+    PipelineId _pipeline_id;
+    std::shared_ptr<PipelineFragmentContext> _context;
+
+    std::unique_ptr<RuntimeProfile> _pipeline_profile;
+};
+
+} // namespace doris::pipeline
\ No newline at end of file
diff --git a/be/src/pipeline/pipeline_fragment_context.cpp b/be/src/pipeline/pipeline_fragment_context.cpp
new file mode 100644
index 0000000000..d1913a1534
--- /dev/null
+++ b/be/src/pipeline/pipeline_fragment_context.cpp
@@ -0,0 +1,569 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "pipeline_fragment_context.h"
+
+#include <thrift/protocol/TDebugProtocol.h>
+
+#include "exec/agg_context.h"
+#include "exec/aggregation_sink_operator.h"
+#include "exec/aggregation_source_operator.h"
+#include "exec/data_sink.h"
+#include "exec/empty_set_operator.h"
+#include "exec/exchange_sink_operator.h"
+#include "exec/exchange_source_operator.h"
+#include "exec/olap_scan_operator.h"
+#include "exec/repeat_operator.h"
+#include "exec/result_sink_operator.h"
+#include "exec/scan_node.h"
+#include "exec/sort_sink_operator.h"
+#include "exec/sort_source_operator.h"
+#include "exec/streaming_aggregation_sink_operator.h"
+#include "exec/streaming_aggregation_source_operator.h"
+#include "gen_cpp/FrontendService.h"
+#include "gen_cpp/HeartbeatService_types.h"
+#include "pipeline/exec/table_function_operator.h"
+#include "pipeline_task.h"
+#include "runtime/client_cache.h"
+#include "runtime/fragment_mgr.h"
+#include "runtime/runtime_state.h"
+#include "task_scheduler.h"
+#include "util/container_util.hpp"
+#include "vec/exec/scan/new_file_scan_node.h"
+#include "vec/exec/scan/new_olap_scan_node.h"
+#include "vec/exec/scan/vscan_node.h"
+#include "vec/exec/vaggregation_node.h"
+#include "vec/exec/vempty_set_node.h"
+#include "vec/exec/vexchange_node.h"
+#include "vec/exec/vrepeat_node.h"
+#include "vec/exec/vsort_node.h"
+#include "vec/runtime/vdata_stream_mgr.h"
+#include "vec/sink/vdata_stream_sender.h"
+#include "vec/sink/vresult_sink.h"
+
+using apache::thrift::transport::TTransportException;
+using apache::thrift::TException;
+
+namespace doris::pipeline {
+
+PipelineFragmentContext::PipelineFragmentContext(const TUniqueId& query_id,
+                                                 const TUniqueId& instance_id, int backend_num,
+                                                 std::shared_ptr<QueryFragmentsCtx> query_ctx,
+                                                 ExecEnv* exec_env)
+        : _query_id(query_id),
+          _fragment_instance_id(instance_id),
+          _backend_num(backend_num),
+          _exec_env(exec_env),
+          _cancel_reason(PPlanFragmentCancelReason::INTERNAL_ERROR),
+          _closed_pipeline_cnt(0),
+          _query_ctx(std::move(query_ctx)) {
+    _fragment_watcher.start();
+}
+
+PipelineFragmentContext::~PipelineFragmentContext() = default;
+
+void PipelineFragmentContext::cancel(const PPlanFragmentCancelReason& reason,
+                                     const std::string& msg) {
+    if (!_runtime_state->is_cancelled()) {
+        std::lock_guard<std::mutex> l(_status_lock);
+        if (_runtime_state->is_cancelled()) {
+            return;
+        }
+        if (reason != PPlanFragmentCancelReason::LIMIT_REACH) {
+            _exec_status = Status::Cancelled(msg);
+        }
+        _runtime_state->set_is_cancelled(true);
+        _cancel_reason = reason;
+        _cancel_msg = msg;
+        // To notify wait_for_start()
+        _query_ctx->set_ready_to_execute(true);
+
+        // must close stream_mgr to avoid dead lock in Exchange Node
+        _exec_env->vstream_mgr()->cancel(_fragment_instance_id);
+        // Cancel the result queue manager used by spark doris connector
+        // TODO pipeline incomp
+        // _exec_env->result_queue_mgr()->update_queue_status(id, Status::Aborted(msg));
+    }
+}
+
+PipelinePtr PipelineFragmentContext::add_pipeline() {
+    // _prepared、_submitted, _canceled should do not add pipeline
+    PipelineId id = _next_pipeline_id++;
+    auto pipeline = std::make_shared<Pipeline>(id, shared_from_this());
+    _pipelines.emplace_back(pipeline);
+    return pipeline;
+}
+
+Status PipelineFragmentContext::prepare(const doris::TExecPlanFragmentParams& request) {
+    if (_prepared) {
+        return Status::InternalError("Already prepared");
+    }
+    //    _runtime_profile.reset(new RuntimeProfile("PipelineContext"));
+    //    _start_timer = ADD_TIMER(_runtime_profile, "StartTime");
+    //    COUNTER_UPDATE(_start_timer, _fragment_watcher.elapsed_time());
+    //    _prepare_timer = ADD_TIMER(_runtime_profile, "PrepareTime");
+    //    SCOPED_TIMER(_prepare_timer);
+
+    auto* fragment_context = this;
+    OpentelemetryTracer tracer = telemetry::get_noop_tracer();
+    if (opentelemetry::trace::Tracer::GetCurrentSpan()->GetContext().IsValid()) {
+        tracer = telemetry::get_tracer(print_id(_query_id));
+    }
+    START_AND_SCOPE_SPAN(tracer, span, "PipelineFragmentExecutor::prepare");
+
+    const TPlanFragmentExecParams& params = request.params;
+
+    LOG_INFO("PipelineFragmentContext::prepare")
+            .tag("query_id", _query_id)
+            .tag("instance_id", params.fragment_instance_id)
+            .tag("backend_num", request.backend_num)
+            .tag("pthread_id", (uintptr_t)pthread_self());
+
+    // Must be vec exec engine
+    if (!request.query_options.__isset.enable_vectorized_engine ||
+        !request.query_options.enable_vectorized_engine) {
+        return Status::InternalError("should set enable_vectorized_engine to true");
+    }
+
+    // 1. init _runtime_state
+    _runtime_state = std::make_unique<RuntimeState>(params, request.query_options,
+                                                    _query_ctx->query_globals, _exec_env);
+    _runtime_state->set_query_fragments_ctx(_query_ctx.get());
+    _runtime_state->set_query_mem_tracker(_query_ctx->query_mem_tracker);
+    _runtime_state->set_tracer(std::move(tracer));
+
+    // TODO should be combine with plan_fragment_executor.prepare funciton
+    SCOPED_ATTACH_TASK(get_runtime_state());
+    _runtime_state->init_scanner_mem_trackers();
+    _runtime_state->runtime_filter_mgr()->init();
+    _runtime_state->set_be_number(request.backend_num);
+
+    if (request.__isset.backend_id) {
+        _runtime_state->set_backend_id(request.backend_id);
+    }
+    if (request.__isset.import_label) {
+        _runtime_state->set_import_label(request.import_label);
+    }
+    if (request.__isset.db_name) {
+        _runtime_state->set_db_name(request.db_name);
+    }
+    if (request.__isset.load_job_id) {
+        _runtime_state->set_load_job_id(request.load_job_id);
+    }
+    if (request.__isset.load_error_hub_info) {
+        _runtime_state->set_load_error_hub_info(request.load_error_hub_info);
+    }
+
+    if (request.query_options.__isset.is_report_success) {
+        fragment_context->set_is_report_success(request.query_options.is_report_success);
+    }
+
+    auto* desc_tbl = _query_ctx->desc_tbl;
+    _runtime_state->set_desc_tbl(desc_tbl);
+
+    // 2. Create ExecNode to build pipeline with PipelineFragmentContext
+    RETURN_IF_ERROR(ExecNode::create_tree(_runtime_state.get(), _runtime_state->obj_pool(),
+                                          request.fragment.plan, *desc_tbl, &_root_plan));
+    _runtime_state->set_fragment_root_id(_root_plan->id());
+
+    // Set senders of exchange nodes before pipeline build
+    std::vector<ExecNode*> exch_nodes;
+    _root_plan->collect_nodes(TPlanNodeType::EXCHANGE_NODE, &exch_nodes);
+    for (ExecNode* exch_node : exch_nodes) {
+        DCHECK_EQ(exch_node->type(), TPlanNodeType::EXCHANGE_NODE);
+        int num_senders = find_with_default(params.per_exch_num_senders, exch_node->id(), 0);
+        DCHECK_GT(num_senders, 0);
+        static_cast<vectorized::VExchangeNode*>(exch_node)->set_num_senders(num_senders);
+    }
+
+    // All prepare work do in exec node tree
+    RETURN_IF_ERROR(_root_plan->prepare(_runtime_state.get()));
+    // set scan ranges
+    std::vector<ExecNode*> scan_nodes;
+    std::vector<TScanRangeParams> no_scan_ranges;
+    _root_plan->collect_scan_nodes(&scan_nodes);
+    VLOG_CRITICAL << "scan_nodes.size()=" << scan_nodes.size();
+    VLOG_CRITICAL << "params.per_node_scan_ranges.size()=" << params.per_node_scan_ranges.size();
+
+    _root_plan->try_do_aggregate_serde_improve();
+    // set scan range in ScanNode
+    for (int i = 0; i < scan_nodes.size(); ++i) {
+        // TODO(cmy): this "if...else" should be removed once all ScanNode are derived from VScanNode.
+        ExecNode* node = scan_nodes[i];
+        if (typeid(*node) == typeid(vectorized::NewOlapScanNode) ||
+            typeid(*node) == typeid(vectorized::NewFileScanNode) // ||
+//            typeid(*node) == typeid(vectorized::NewOdbcScanNode) ||
+//            typeid(*node) == typeid(vectorized::NewEsScanNode)
+#ifdef LIBJVM
+//            || typeid(*node) == typeid(vectorized::NewJdbcScanNode)
+#endif
+        ) {
+            auto* scan_node = static_cast<vectorized::VScanNode*>(scan_nodes[i]);
+            const std::vector<TScanRangeParams>& scan_ranges =
+                    find_with_default(params.per_node_scan_ranges, scan_node->id(), no_scan_ranges);
+            scan_node->set_scan_ranges(scan_ranges);
+        } else {
+            ScanNode* scan_node = static_cast<ScanNode*>(scan_nodes[i]);
+            const std::vector<TScanRangeParams>& scan_ranges =
+                    find_with_default(params.per_node_scan_ranges, scan_node->id(), no_scan_ranges);
+            scan_node->set_scan_ranges(scan_ranges);
+            VLOG_CRITICAL << "scan_node_Id=" << scan_node->id() << " size=" << scan_ranges.size();
+        }
+    }
+
+    _runtime_state->set_per_fragment_instance_idx(params.sender_id);
+    _runtime_state->set_num_per_fragment_instances(params.num_senders);
+
+    if (request.fragment.__isset.output_sink) {
+        RETURN_IF_ERROR(DataSink::create_data_sink(
+                _runtime_state->obj_pool(), request.fragment.output_sink,
+                request.fragment.output_exprs, params, _root_plan->row_desc(), _runtime_state.get(),
+                &_sink, *desc_tbl));
+    }
+
+    _root_pipeline = fragment_context->add_pipeline();
+    RETURN_IF_ERROR(_build_pipelines(_root_plan, _root_pipeline));
+    if (_sink) {
+        RETURN_IF_ERROR(_create_sink(request.fragment.output_sink));
+    }
+    RETURN_IF_ERROR(_build_pipeline_tasks(request));
+    _runtime_state->runtime_profile()->add_child(_sink->profile(), true, nullptr);
+    _runtime_state->runtime_profile()->add_child(_root_plan->runtime_profile(), true, nullptr);
+
+    _prepared = true;
+    return Status::OK();
+}
+
+Status PipelineFragmentContext::_build_pipeline_tasks(
+        const doris::TExecPlanFragmentParams& request) {
+    for (auto& pipeline : _pipelines) {
+        RETURN_IF_ERROR(pipeline->prepare(_runtime_state.get()));
+    }
+
+    for (PipelinePtr& pipeline : _pipelines) {
+        // if sink
+        auto sink = pipeline->sink()->build_operator();
+        RETURN_IF_ERROR(sink->init(pipeline->sink()->exec_node(), _runtime_state.get()));
+        // TODO pipeline 1 need to add new interface for exec node and operator
+        sink->init(request.fragment.output_sink);
+
+        Operators operators;
+        RETURN_IF_ERROR(pipeline->build_operators(operators));
+        auto task = std::make_unique<PipelineTask>(pipeline, 0, _runtime_state.get(), operators,
+                                                   sink, this, pipeline->runtime_profile());
+        sink->set_child(task->get_root());
+        _tasks.emplace_back(std::move(task));
+    }
+
+    for (auto& task : _tasks) {
+        RETURN_IF_ERROR(task->prepare(_runtime_state.get()));
+    }
+    return Status::OK();
+}
+
+// TODO: use virtual function to do abstruct
+Status PipelineFragmentContext::_build_pipelines(ExecNode* node, PipelinePtr cur_pipe) {
+    auto* fragment_context = this;
+    auto node_type = node->type();
+    switch (node_type) {
+    // for source
+    case TPlanNodeType::OLAP_SCAN_NODE: {
+        auto* new_olap_scan_node = assert_cast<vectorized::NewOlapScanNode*>(node);
+        OperatorBuilderPtr operator_t = std::make_shared<OlapScanOperatorBuilder>(
+                fragment_context->next_operator_builder_id(), "OlapScanOperator",
+                new_olap_scan_node);
+        RETURN_IF_ERROR(cur_pipe->add_operator(operator_t));
+        break;
+    }
+    case TPlanNodeType::EXCHANGE_NODE: {
+        OperatorBuilderPtr operator_t = std::make_shared<ExchangeSourceOperatorBuilder>(
+                next_operator_builder_id(), "ExchangeSourceOperator", node);
+        RETURN_IF_ERROR(cur_pipe->add_operator(operator_t));
+        break;
+    }
+    case TPlanNodeType::EMPTY_SET_NODE: {
+        auto* empty_set_node = assert_cast<vectorized::VEmptySetNode*>(node);
+        OperatorBuilderPtr operator_t = std::make_shared<EmptySetSourceOperatorBuilder>(
+                next_operator_builder_id(), "EmptySetSourceOperator", empty_set_node);
+        RETURN_IF_ERROR(cur_pipe->add_operator(operator_t));
+        break;
+    }
+    case TPlanNodeType::AGGREGATION_NODE: {
+        auto* agg_node = assert_cast<vectorized::AggregationNode*>(node);
+        auto new_pipe = add_pipeline();
+        RETURN_IF_ERROR(_build_pipelines(node->child(0), new_pipe));
+        if (agg_node->is_streaming_preagg()) {
+            auto agg_ctx = std::make_shared<AggContext>();
+            OperatorBuilderPtr pre_agg_sink = std::make_shared<StreamingAggSinkOperatorBuilder>(
+                    next_operator_builder_id(), "StreamingAggSinkOperator", agg_node, agg_ctx);
+            RETURN_IF_ERROR(new_pipe->set_sink(pre_agg_sink));
+
+            OperatorBuilderPtr pre_agg_source = std::make_shared<StreamingAggSourceOperatorBuilder>(
+                    next_operator_builder_id(), "StreamingAggSourceOperator", agg_node, agg_ctx);
+            RETURN_IF_ERROR(cur_pipe->add_operator(pre_agg_source));
+        } else {
+            OperatorBuilderPtr agg_sink = std::make_shared<AggSinkOperatorBuilder>(
+                    next_operator_builder_id(), "AggSinkOperator", agg_node);
+            RETURN_IF_ERROR(new_pipe->set_sink(agg_sink));
+
+            OperatorBuilderPtr agg_source = std::make_shared<AggregationSourceOperatorBuilder>(
+                    next_operator_builder_id(), "AggSourceOperator", agg_node);
+            RETURN_IF_ERROR(cur_pipe->add_operator(agg_source));
+        }
+        break;
+    }
+    case TPlanNodeType::SORT_NODE: {
+        auto* sort_node = assert_cast<vectorized::VSortNode*>(node);
+        auto new_pipeline = add_pipeline();
+        RETURN_IF_ERROR(_build_pipelines(node->child(0), new_pipeline));
+
+        OperatorBuilderPtr sort_sink = std::make_shared<SortSinkOperatorBuilder>(
+                next_operator_builder_id(), "SortSinkOperator", sort_node);
+        RETURN_IF_ERROR(new_pipeline->set_sink(sort_sink));
+
+        OperatorBuilderPtr sort_source = std::make_shared<SortSourceOperatorBuilder>(
+                next_operator_builder_id(), "SortSourceOperator", sort_node);
+        RETURN_IF_ERROR(cur_pipe->add_operator(sort_source));
+        break;
+    }
+    case TPlanNodeType::REPEAT_NODE: {
+        auto* repeat_node = assert_cast<vectorized::VRepeatNode*>(node);
+        RETURN_IF_ERROR(_build_pipelines(node->child(0), cur_pipe));
+        OperatorBuilderPtr builder =
+                std::make_shared<RepeatOperatorBuilder>(next_operator_builder_id(), repeat_node);
+        RETURN_IF_ERROR(cur_pipe->add_operator(builder));
+        break;
+    }
+    case TPlanNodeType::TABLE_FUNCTION_NODE: {
+        auto* repeat_node = assert_cast<vectorized::VTableFunctionNode*>(node);
+        RETURN_IF_ERROR(_build_pipelines(node->child(0), cur_pipe));
+        OperatorBuilderPtr builder = std::make_shared<TableFunctionOperatorBuilder>(
+                next_operator_builder_id(), repeat_node);
+        RETURN_IF_ERROR(cur_pipe->add_operator(builder));
+        break;
+    }
+    default:
+        return Status::InternalError("Unsupported exec type in pipeline: {}",
+                                     print_plan_node_type(node_type));
+    }
+    return Status::OK();
+}
+
+Status PipelineFragmentContext::submit() {
+    if (_submitted) {
+        return Status::InternalError("submitted");
+    }
+
+    for (auto& task : _tasks) {
+        RETURN_IF_ERROR(_exec_env->pipeline_task_scheduler()->schedule_task(task.get()));
+    }
+    _submitted = true;
+    return Status::OK();
+}
+
+// construct sink operator
+Status PipelineFragmentContext::_create_sink(const TDataSink& thrift_sink) {
+    OperatorBuilderPtr sink_;
+    switch (thrift_sink.type) {
+    case TDataSinkType::DATA_STREAM_SINK: {
+        auto* exchange_sink = assert_cast<doris::vectorized::VDataStreamSender*>(_sink.get());
+        sink_ = std::make_shared<ExchangeSinkOperatorBuilder>(
+                next_operator_builder_id(), "ExchangeSinkOperator", nullptr, exchange_sink, this);
+        break;
+    }
+    case TDataSinkType::RESULT_SINK: {
+        auto* result_sink = assert_cast<doris::vectorized::VResultSink*>(_sink.get());
+        sink_ = std::make_shared<ResultSinkOperatorBuilder>(
+                next_operator_builder_id(), "ResultSinkOperator", nullptr, result_sink);
+        break;
+    }
+    default:
+        return Status::InternalError("Unsuported sink type in pipeline: {}", thrift_sink.type);
+    }
+    return _root_pipeline->set_sink(sink_);
+}
+
+void PipelineFragmentContext::close_a_pipeline() {
+    ++_closed_pipeline_cnt;
+    if (_closed_pipeline_cnt == _pipelines.size()) {
+        //        _runtime_profile->total_time_counter()->update(_fragment_watcher.elapsed_time());
+        send_report(true);
+        _exec_env->fragment_mgr()->remove_pipeline_context(shared_from_this());
+    }
+}
+
+// TODO pipeline dump copy from FragmentExecState::to_http_path
+std::string PipelineFragmentContext::to_http_path(const std::string& file_name) {
+    std::stringstream url;
+    url << "http://" << BackendOptions::get_localhost() << ":" << config::webserver_port
+        << "/api/_download_load?"
+        << "token=" << _exec_env->token() << "&file=" << file_name;
+    return url.str();
+}
+
+// TODO pipeline dump copy from FragmentExecState::coordinator_callback
+// TODO pipeline this callback should be placed in a thread pool
+void PipelineFragmentContext::send_report(bool done) {
+    DCHECK(_closed_pipeline_cnt == _pipelines.size());
+
+    Status exec_status = Status::OK();
+    {
+        std::lock_guard<std::mutex> l(_status_lock);
+        if (!_exec_status.ok()) {
+            exec_status = _exec_status;
+        }
+    }
+
+    // If plan is done successfully, but _is_report_success is false,
+    // no need to send report.
+    if (!_is_report_success && done && exec_status.ok()) {
+        return;
+    }
+
+    Status coord_status;
+    auto coord_addr = _query_ctx->coord_addr;
+    FrontendServiceConnection coord(_exec_env->frontend_client_cache(), coord_addr, &coord_status);
+    if (!coord_status.ok()) {
+        std::stringstream ss;
+        ss << "couldn't get a client for " << coord_addr << ", reason: " << coord_status;
+        LOG(WARNING) << "query_id: " << print_id(_query_id) << ", " << ss.str();
+        {
+            std::lock_guard<std::mutex> l(_status_lock);
+            if (_exec_status.ok()) {
+                _exec_status = Status::InternalError(ss.str());
+            }
+        }
+        return;
+    }
+    auto* profile = _is_report_success ? _runtime_state->runtime_profile() : nullptr;
+
+    TReportExecStatusParams params;
+    params.protocol_version = FrontendServiceVersion::V1;
+    params.__set_query_id(_query_id);
+    params.__set_backend_num(_backend_num);
+    params.__set_fragment_instance_id(_fragment_instance_id);
+    exec_status.set_t_status(&params);
+    params.__set_done(true);
+
+    auto* runtime_state = _runtime_state.get();
+    DCHECK(runtime_state != nullptr);
+    if (runtime_state->query_type() == TQueryType::LOAD && !done && exec_status.ok()) {
+        // this is a load plan, and load is not finished, just make a brief report
+        params.__set_loaded_rows(runtime_state->num_rows_load_total());
+        params.__set_loaded_bytes(runtime_state->num_bytes_load_total());
+    } else {
+        if (runtime_state->query_type() == TQueryType::LOAD) {
+            params.__set_loaded_rows(runtime_state->num_rows_load_total());
+            params.__set_loaded_bytes(runtime_state->num_bytes_load_total());
+        }
+        if (profile == nullptr) {
+            params.__isset.profile = false;
+        } else {
+            profile->to_thrift(&params.profile);
+            params.__isset.profile = true;
+        }
+
+        if (!runtime_state->output_files().empty()) {
+            params.__isset.delta_urls = true;
+            for (auto& it : runtime_state->output_files()) {
+                params.delta_urls.push_back(to_http_path(it));
+            }
+        }
+        if (runtime_state->num_rows_load_total() > 0 ||
+            runtime_state->num_rows_load_filtered() > 0) {
+            params.__isset.load_counters = true;
+
+            static std::string s_dpp_normal_all = "dpp.norm.ALL";
+            static std::string s_dpp_abnormal_all = "dpp.abnorm.ALL";
+            static std::string s_unselected_rows = "unselected.rows";
+
+            params.load_counters.emplace(s_dpp_normal_all,
+                                         std::to_string(runtime_state->num_rows_load_success()));
+            params.load_counters.emplace(s_dpp_abnormal_all,
+                                         std::to_string(runtime_state->num_rows_load_filtered()));
+            params.load_counters.emplace(s_unselected_rows,
+                                         std::to_string(runtime_state->num_rows_load_unselected()));
+        }
+        if (!runtime_state->get_error_log_file_path().empty()) {
+            params.__set_tracking_url(
+                    to_load_error_http_path(runtime_state->get_error_log_file_path()));
+        }
+        if (!runtime_state->export_output_files().empty()) {
+            params.__isset.export_files = true;
+            params.export_files = runtime_state->export_output_files();
+        }
+        if (!runtime_state->tablet_commit_infos().empty()) {
+            params.__isset.commitInfos = true;
+            params.commitInfos.reserve(runtime_state->tablet_commit_infos().size());
+            for (auto& info : runtime_state->tablet_commit_infos()) {
+                params.commitInfos.push_back(info);
+            }
+        }
+        if (!runtime_state->error_tablet_infos().empty()) {
+            params.__isset.errorTabletInfos = true;
+            params.errorTabletInfos.reserve(runtime_state->error_tablet_infos().size());
+            for (auto& info : runtime_state->error_tablet_infos()) {
+                params.errorTabletInfos.push_back(info);
+            }
+        }
+
+        // Send new errors to coordinator
+        runtime_state->get_unreported_errors(&(params.error_log));
+        params.__isset.error_log = (params.error_log.size() > 0);
+    }
+
+    if (_exec_env->master_info()->__isset.backend_id) {
+        params.__set_backend_id(_exec_env->master_info()->backend_id);
+    }
+
+    TReportExecStatusResult res;
+    Status rpc_status;
+
+    VLOG_DEBUG << "reportExecStatus params is "
+               << apache::thrift::ThriftDebugString(params).c_str();
+    try {
+        try {
+            coord->reportExecStatus(res, params);
+        } catch (TTransportException& e) {
+            LOG(WARNING) << "Retrying ReportExecStatus. query id: " << print_id(_query_id)
+                         << ", instance id: " << print_id(_fragment_instance_id) << " to "
+                         << coord_addr << ", err: " << e.what();
+            rpc_status = coord.reopen();
+
+            if (!rpc_status.ok()) {
+                // we need to cancel the execution of this fragment
+                cancel(PPlanFragmentCancelReason::INTERNAL_ERROR, "report rpc fail");
+                return;
+            }
+            coord->reportExecStatus(res, params);
+        }
+
+        rpc_status = Status(res.status);
+    } catch (TException& e) {
+        std::stringstream msg;
+        msg << "ReportExecStatus() to " << coord_addr << " failed:\n" << e.what();
+        LOG(WARNING) << msg.str();
+        rpc_status = Status::InternalError(msg.str());
+    }
+
+    if (!rpc_status.ok()) {
+        // we need to cancel the execution of this fragment
+        cancel(PPlanFragmentCancelReason::INTERNAL_ERROR, "rpc fail 2");
+    }
+}
+
+} // namespace doris::pipeline
\ No newline at end of file
diff --git a/be/src/pipeline/pipeline_fragment_context.h b/be/src/pipeline/pipeline_fragment_context.h
new file mode 100644
index 0000000000..f35c6cecd1
--- /dev/null
+++ b/be/src/pipeline/pipeline_fragment_context.h
@@ -0,0 +1,127 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include "pipeline.h"
+#include "runtime/runtime_state.h"
+
+namespace doris {
+class ExecNode;
+class DataSink;
+
+namespace pipeline {
+
+class PipelineTask;
+
+class PipelineFragmentContext : public std::enable_shared_from_this<PipelineFragmentContext> {
+public:
+    PipelineFragmentContext(const TUniqueId& query_id, const TUniqueId& instance_id,
+                            int backend_num, std::shared_ptr<QueryFragmentsCtx> query_ctx,
+                            ExecEnv* exec_env);
+
+    virtual ~PipelineFragmentContext();
+
+    PipelinePtr add_pipeline();
+
+    TUniqueId get_fragment_id() { return _fragment_id; }
+
+    RuntimeState* get_runtime_state() { return _runtime_state.get(); }
+
+    // should be protected by lock?
+    bool is_canceled() const { return _runtime_state->is_cancelled(); }
+
+    int32_t next_operator_builder_id() { return _next_operator_builder_id++; }
+
+    Status prepare(const doris::TExecPlanFragmentParams& request);
+
+    Status submit();
+
+    void set_is_report_success(bool is_report_success) { _is_report_success = is_report_success; }
+
+    ExecNode*& plan() { return _root_plan; }
+
+    void set_need_wait_execution_trigger() { _need_wait_execution_trigger = true; }
+
+    void cancel(const PPlanFragmentCancelReason& reason = PPlanFragmentCancelReason::INTERNAL_ERROR,
+                const std::string& msg = "");
+
+    // TODO: Support pipeline runtime filter
+
+    QueryFragmentsCtx* get_query_context() { return _query_ctx.get(); }
+
+    TUniqueId get_query_id() const { return _query_id; }
+
+    void close_a_pipeline();
+
+    std::string to_http_path(const std::string& file_name);
+
+    void send_report(bool);
+
+private:
+    // Id of this query
+    TUniqueId _query_id;
+    // Id of this instance
+    TUniqueId _fragment_instance_id;
+
+    int _backend_num;
+
+    ExecEnv* _exec_env;
+    TUniqueId _fragment_id;
+
+    bool _prepared = false;
+    bool _submitted = false;
+
+    std::mutex _status_lock;
+    Status _exec_status;
+    PPlanFragmentCancelReason _cancel_reason;
+    std::string _cancel_msg;
+
+    Pipelines _pipelines;
+    PipelineId _next_pipeline_id = 0;
+    std::atomic<int> _closed_pipeline_cnt;
+
+    int32_t _next_operator_builder_id = 10000;
+
+    std::vector<std::unique_ptr<PipelineTask>> _tasks;
+
+    PipelinePtr _root_pipeline;
+
+    std::unique_ptr<RuntimeProfile> _runtime_profile;
+    bool _is_report_success = false;
+
+    std::unique_ptr<RuntimeState> _runtime_state;
+
+    ExecNode* _root_plan = nullptr; // lives in _runtime_state->obj_pool()
+    std::unique_ptr<DataSink> _sink;
+
+    std::shared_ptr<QueryFragmentsCtx> _query_ctx;
+
+    // If set the true, this plan fragment will be executed only after FE send execution start rpc.
+    bool _need_wait_execution_trigger = false;
+
+    MonotonicStopWatch _fragment_watcher;
+    //    RuntimeProfile::Counter* _start_timer;
+    //    RuntimeProfile::Counter* _prepare_timer;
+
+private:
+    Status _create_sink(const TDataSink& t_data_sink);
+    Status _build_pipelines(ExecNode*, PipelinePtr);
+    Status _build_pipeline_tasks(const doris::TExecPlanFragmentParams& request);
+};
+} // namespace pipeline
+} // namespace doris
\ No newline at end of file
diff --git a/be/src/pipeline/pipeline_task.cpp b/be/src/pipeline/pipeline_task.cpp
new file mode 100644
index 0000000000..fbe1d6b819
--- /dev/null
+++ b/be/src/pipeline/pipeline_task.cpp
@@ -0,0 +1,219 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "pipeline_task.h"
+
+#include "pipeline/pipeline_fragment_context.h"
+
+namespace doris::pipeline {
+
+void PipelineTask::_init_profile() {
+    std::stringstream ss;
+    ss << "PipelineTask"
+       << " (index=" << _index << ")";
+    auto* task_profile = new RuntimeProfile(ss.str());
+    _parent_profile->add_child(task_profile, true, nullptr);
+    _task_profile.reset(task_profile);
+    _sink_timer = ADD_TIMER(_task_profile, "SinkTime");
+    _get_block_timer = ADD_TIMER(_task_profile, "GetBlockTime");
+    _wait_source_timer = ADD_TIMER(_task_profile, "WaitSourceTime");
+    _wait_sink_timer = ADD_TIMER(_task_profile, "WaitSinkTime");
+    _wait_worker_timer = ADD_TIMER(_task_profile, "WaitWorkerTime");
+    _wait_schedule_timer = ADD_TIMER(_task_profile, "WaitScheduleTime");
+    _block_counts = ADD_COUNTER(_task_profile, "NumBlockedTimes", TUnit::UNIT);
+    _yield_counts = ADD_COUNTER(_task_profile, "NumYieldTimes", TUnit::UNIT);
+}
+
+Status PipelineTask::prepare(RuntimeState* state) {
+    DCHECK(_sink);
+    DCHECK(_cur_state == NOT_READY);
+    _init_profile();
+    RETURN_IF_ERROR(_sink->prepare(state));
+    for (auto& o : _operators) {
+        RETURN_IF_ERROR(o->prepare(state));
+    }
+    _block.reset(new doris::vectorized::Block());
+    _init_state();
+    _prepared = true;
+    return Status::OK();
+}
+
+void PipelineTask::_init_state() {
+    if (has_dependency()) {
+        set_state(BLOCKED_FOR_DEPENDENCY);
+    } else if (!(_source->can_read())) {
+        set_state(BLOCKED_FOR_SOURCE);
+    } else if (!(_sink->can_write())) {
+        set_state(BLOCKED_FOR_SINK);
+    } else {
+        set_state(RUNNABLE);
+    }
+}
+
+bool PipelineTask::has_dependency() {
+    if (_dependency_finish) {
+        return false;
+    }
+    if (_fragment_context->is_canceled()) {
+        _dependency_finish = true;
+        return false;
+    }
+    if (_pipeline->has_dependency()) {
+        return true;
+    }
+    // FE do not call execute
+    if (!_state->get_query_fragments_ctx()
+                 ->is_ready_to_execute()) { // TODO pipeline config::s_ready_to_execute
+        return true;
+    }
+
+    // runtime filter is a dependency
+    _dependency_finish = true;
+    return false;
+}
+
+Status PipelineTask::open() {
+    if (_sink) {
+        RETURN_IF_ERROR(_sink->open(_state));
+    }
+    for (auto& o : _operators) {
+        RETURN_IF_ERROR(o->open(_state));
+    }
+    _opened = true;
+    return Status::OK();
+}
+
+Status PipelineTask::execute(bool* eos) {
+    SCOPED_ATTACH_TASK(runtime_state());
+    SCOPED_TIMER(_task_profile->total_time_counter());
+    int64_t time_spent = 0;
+    // The status must be runnable
+    *eos = false;
+    if (!_opened) {
+        if (!_source->can_read()) {
+            set_state(BLOCKED_FOR_SOURCE);
+            return Status::OK();
+        }
+        if (!_sink->can_write()) {
+            set_state(BLOCKED_FOR_SINK);
+            return Status::OK();
+        }
+        SCOPED_RAW_TIMER(&time_spent);
+        RETURN_IF_ERROR(open());
+    }
+
+    while (!_fragment_context->is_canceled()) {
+        if (!_source->can_read() && _data_state != SourceState::MORE_DATA) {
+            set_state(BLOCKED_FOR_SOURCE);
+            break;
+        }
+        if (!_sink->can_write()) {
+            set_state(BLOCKED_FOR_SINK);
+            break;
+        }
+        if (time_spent > THREAD_TIME_SLICE) {
+            COUNTER_UPDATE(_yield_counts, 1);
+            break;
+        }
+        SCOPED_RAW_TIMER(&time_spent);
+        _block->clear_column_data(_root->row_desc().num_materialized_slots());
+        auto* block = _block.get();
+
+        // Pull block from operator chain
+        {
+            SCOPED_TIMER(_get_block_timer);
+            RETURN_IF_ERROR(_root->get_block(_state, block, _data_state));
+        }
+        *eos = _data_state == SourceState::FINISHED;
+        if (_block->rows() != 0 || *eos) {
+            SCOPED_TIMER(_sink_timer);
+            RETURN_IF_ERROR(_sink->sink(_state, block, _data_state));
+            if (*eos) { // just return, the scheduler will do finish work
+                break;
+            }
+        }
+        *eos = false;
+    }
+
+    return Status::OK();
+}
+
+Status PipelineTask::finalize() {
+    return _sink->finalize(_state);
+}
+
+Status PipelineTask::close() {
+    auto s = _sink->close(_state);
+    for (auto& op : _operators) {
+        auto tem = op->close(_state);
+        if (!tem.ok() && s.ok()) {
+            s = tem;
+        }
+    }
+    if (_opened) {
+        COUNTER_UPDATE(_wait_source_timer, _wait_source_watcher.elapsed_time());
+        COUNTER_UPDATE(_wait_sink_timer, _wait_sink_watcher.elapsed_time());
+        COUNTER_UPDATE(_wait_worker_timer, _wait_worker_watcher.elapsed_time());
+        COUNTER_UPDATE(_wait_schedule_timer, _wait_schedule_watcher.elapsed_time());
+    }
+    _pipeline->close(_state);
+    return s;
+}
+
+QueryFragmentsCtx* PipelineTask::query_fragments_context() {
+    return _fragment_context->get_query_context();
+}
+
+// The FSM see PipelineTaskState's comment
+void PipelineTask::set_state(PipelineTaskState state) {
+    if (_cur_state == state) {
+        return;
+    }
+    if (_cur_state == BLOCKED_FOR_SOURCE) {
+        if (state == RUNNABLE) {
+            _wait_source_watcher.stop();
+        }
+    } else if (_cur_state == BLOCKED_FOR_SINK) {
+        if (state == RUNNABLE) {
+            _wait_sink_watcher.stop();
+        }
+    } else if (_cur_state == RUNNABLE) {
+        if (state == BLOCKED_FOR_SOURCE) {
+            _wait_source_watcher.start();
+            COUNTER_UPDATE(_block_counts, 1);
+        } else if (state == BLOCKED_FOR_SINK) {
+            _wait_sink_watcher.start();
+            COUNTER_UPDATE(_block_counts, 1);
+        } else if (state == BLOCKED_FOR_DEPENDENCY) {
+            COUNTER_UPDATE(_block_counts, 1);
+        }
+    }
+    _cur_state = state;
+}
+
+std::string PipelineTask::debug_string() const {
+    std::stringstream ss;
+    ss << "PipelineTask(" << _index << ")" << get_state_name(_cur_state) << "\nsink: ";
+    ss << _sink->debug_string();
+    ss << "\n operators(from source to root)";
+    for (auto operatr : _operators) {
+        ss << "\n" << operatr->debug_string();
+    }
+    return ss.str();
+}
+
+} // namespace doris::pipeline
\ No newline at end of file
diff --git a/be/src/pipeline/pipeline_task.h b/be/src/pipeline/pipeline_task.h
new file mode 100644
index 0000000000..a7f0dffec0
--- /dev/null
+++ b/be/src/pipeline/pipeline_task.h
@@ -0,0 +1,204 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include "exec/operator.h"
+#include "pipeline.h"
+#include "util/stopwatch.hpp"
+
+namespace doris::pipeline {
+
+/**
+ * PipelineTaskState indicates all possible states of a pipeline task.
+ * A FSM is described as below:
+ *
+ *                 |-----------------------------------------------------|
+ *                 |---|                  transfer 2    transfer 3       |   transfer 4
+ *                     |-------> BLOCKED ------------|                   |---------------------------------------> CANCELED
+ *              |------|                             |                   | transfer 5           transfer 6|
+ * NOT_READY ---| transfer 0                         |-----> RUNNABLE ---|---------> PENDING_FINISH ------|
+ *              |                                    |          ^        |                      transfer 7|
+ *              |------------------------------------|          |--------|---------------------------------------> FINISHED
+ *                transfer 1                                   transfer 9          transfer 8
+ * BLOCKED include BLOCKED_FOR_DEPENDENCY, BLOCKED_FOR_SOURCE and BLOCKED_FOR_SINK.
+ *
+ * transfer 0 (NOT_READY -> BLOCKED): this pipeline task has some incomplete dependencies
+ * transfer 1 (NOT_READY -> RUNNABLE): this pipeline task has no incomplete dependencies
+ * transfer 2 (BLOCKED -> RUNNABLE): runnable condition for this pipeline task is met (e.g. get a new block from rpc)
+ * transfer 3 (RUNNABLE -> BLOCKED): runnable condition for this pipeline task is not met (e.g. sink operator send a block by RPC and wait for a response)
+ * transfer 4 (RUNNABLE -> CANCELED): current fragment is cancelled
+ * transfer 5 (RUNNABLE -> PENDING_FINISH): this pipeline task completed but wait for releasing resources hold by itself
+ * transfer 6 (PENDING_FINISH -> CANCELED): current fragment is cancelled
+ * transfer 7 (PENDING_FINISH -> FINISHED): this pipeline task completed and resources hold by itself have been released already
+ * transfer 8 (RUNNABLE -> FINISHED): this pipeline task completed and no resource need to be released
+ * transfer 9 (RUNNABLE -> RUNNABLE): this pipeline task yields CPU and re-enters the runnable queue if it is runnable and has occupied CPU for a max time slice
+ */
+enum PipelineTaskState : uint8_t {
+    NOT_READY = 0, // do not prepare
+    BLOCKED_FOR_DEPENDENCY = 1,
+    BLOCKED_FOR_SOURCE = 2,
+    BLOCKED_FOR_SINK = 3,
+    RUNNABLE = 4, // can execute
+    PENDING_FINISH =
+            5, // compute task is over, but still hold resource. like some scan and sink task
+    FINISHED = 6,
+    CANCELED = 7
+};
+
+inline const char* get_state_name(PipelineTaskState idx) {
+    switch (idx) {
+    case PipelineTaskState::NOT_READY:
+        return "NOT_READY";
+    case PipelineTaskState::BLOCKED_FOR_DEPENDENCY:
+        return "BLOCKED_FOR_DEPENDENCY";
+    case PipelineTaskState::BLOCKED_FOR_SOURCE:
+        return "BLOCKED_FOR_SOURCE";
+    case PipelineTaskState::BLOCKED_FOR_SINK:
+        return "BLOCKED_FOR_SINK";
+    case PipelineTaskState::RUNNABLE:
+        return "RUNNABLE";
+    case PipelineTaskState::PENDING_FINISH:
+        return "PENDING_FINISH";
+    case PipelineTaskState::FINISHED:
+        return "FINISHED";
+    case PipelineTaskState::CANCELED:
+        return "CANCELED";
+    }
+    __builtin_unreachable();
+}
+
+// The class do the pipeline task. Minest schdule union by task scheduler
+class PipelineTask {
+public:
+    PipelineTask(PipelinePtr& pipeline, uint32_t index, RuntimeState* state, Operators& operators,
+                 OperatorPtr& sink, PipelineFragmentContext* fragment_context,
+                 RuntimeProfile* parent_profile)
+            : _index(index),
+              _pipeline(pipeline),
+              _operators(operators),
+              _source(_operators.front()),
+              _root(_operators.back()),
+              _sink(sink),
+              _prepared(false),
+              _opened(false),
+              _state(state),
+              _cur_state(NOT_READY),
+              _data_state(SourceState::DEPEND_ON_SOURCE),
+              _fragment_context(fragment_context),
+              _parent_profile(parent_profile) {}
+
+    Status prepare(RuntimeState* state);
+
+    Status execute(bool* eos);
+
+    // if the pipeline create a bunch of pipeline task
+    // must be call after all pipeline task is finish to release resource
+    Status close();
+
+    void start_worker_watcher() { _wait_worker_watcher.start(); }
+    void stop_worker_watcher() { _wait_worker_watcher.stop(); }
+    void start_schedule_watcher() { _wait_schedule_watcher.start(); }
+    void stop_schedule_watcher() { _wait_schedule_watcher.stop(); }
+
+    PipelineTaskState get_state() { return _cur_state; }
+    void set_state(PipelineTaskState state);
+    bool is_blocking_state() {
+        switch (_cur_state) {
+        case BLOCKED_FOR_DEPENDENCY:
+        case BLOCKED_FOR_SOURCE:
+        case BLOCKED_FOR_SINK:
+            return true;
+        default:
+            return false;
+        }
+    }
+
+    bool is_pending_finish() { return _source->is_pending_finish() || _sink->is_pending_finish(); }
+
+    bool source_can_read() { return _source->can_read(); }
+
+    bool sink_can_write() { return _sink->can_write(); }
+
+    Status finalize();
+
+    void finish_p_dependency() {
+        for (const auto& p : _pipeline->_parents) {
+            p->finish_one_dependency();
+        }
+    }
+
+    PipelineFragmentContext* fragment_context() { return _fragment_context; }
+
+    QueryFragmentsCtx* query_fragments_context();
+
+    int get_previous_core_id() const { return _previous_schedule_id; }
+
+    void set_previous_core_id(int id) { _previous_schedule_id = id; }
+
+    bool has_dependency();
+
+    uint32_t index() const { return _index; }
+
+    OperatorPtr get_root() { return _root; }
+
+    std::string debug_string() const;
+
+    RuntimeState* runtime_state() { return _state; }
+
+    static constexpr auto THREAD_TIME_SLICE = 100'000'000L;
+
+private:
+    Status open();
+    void _init_profile();
+    void _init_state();
+
+private:
+    uint32_t _index;
+    PipelinePtr _pipeline;
+    bool _dependency_finish = false;
+    Operators _operators; // left is _source, right is _root
+    OperatorPtr _source;
+    OperatorPtr _root;
+    OperatorPtr _sink;
+
+    bool _prepared;
+    bool _opened;
+    RuntimeState* _state;
+    int _previous_schedule_id = -1;
+    PipelineTaskState _cur_state;
+    SourceState _data_state;
+    std::unique_ptr<doris::vectorized::Block> _block;
+    PipelineFragmentContext* _fragment_context;
+
+    RuntimeProfile* _parent_profile;
+    std::unique_ptr<RuntimeProfile> _task_profile;
+    RuntimeProfile::Counter* _sink_timer;
+    RuntimeProfile::Counter* _get_block_timer;
+    RuntimeProfile::Counter* _block_counts;
+    MonotonicStopWatch _wait_source_watcher;
+    RuntimeProfile::Counter* _wait_source_timer;
+    MonotonicStopWatch _wait_sink_watcher;
+    RuntimeProfile::Counter* _wait_sink_timer;
+    MonotonicStopWatch _wait_worker_watcher;
+    RuntimeProfile::Counter* _wait_worker_timer;
+    // TODO we should calculate the time between when really runnable and runnable
+    MonotonicStopWatch _wait_schedule_watcher;
+    RuntimeProfile::Counter* _wait_schedule_timer;
+    RuntimeProfile::Counter* _yield_counts;
+};
+} // namespace doris::pipeline
\ No newline at end of file
diff --git a/be/src/pipeline/task_scheduler.cpp b/be/src/pipeline/task_scheduler.cpp
new file mode 100644
index 0000000000..f8d7e28f91
--- /dev/null
+++ b/be/src/pipeline/task_scheduler.cpp
@@ -0,0 +1,327 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "task_scheduler.h"
+
+#include "common/signal_handler.h"
+#include "pipeline_fragment_context.h"
+#include "util/thread.h"
+
+namespace doris::pipeline {
+
+Status BlockedTaskScheduler::start() {
+    LOG(INFO) << "BlockedTaskScheduler start";
+    RETURN_IF_ERROR(Thread::create(
+            "BlockedTaskScheduler", "schedule_blocked_pipeline", [this]() { this->_schedule(); },
+            &_thread));
+    while (!this->_started.load()) {
+        std::this_thread::sleep_for(std::chrono::milliseconds(5));
+    }
+    return Status::OK();
+}
+
+void BlockedTaskScheduler::shutdown() {
+    LOG(INFO) << "Start shutdown BlockedTaskScheduler";
+    if (!this->_shutdown) {
+        this->_shutdown = true;
+        if (_thread) {
+            _task_cond.notify_one();
+            _thread->join();
+        }
+    }
+}
+
+void BlockedTaskScheduler::add_blocked_task(PipelineTask* task) {
+    std::unique_lock<std::mutex> lock(_task_mutex);
+    _blocked_tasks.push_back(task);
+    _task_cond.notify_one();
+}
+
+void BlockedTaskScheduler::_schedule() {
+    LOG(INFO) << "BlockedTaskScheduler schedule thread start";
+    _started.store(true);
+    std::list<PipelineTask*> local_blocked_tasks;
+    int empty_times = 0;
+    std::vector<PipelineTask*> ready_tasks;
+
+    while (!_shutdown.load()) {
+        {
+            std::unique_lock<std::mutex> lock(this->_task_mutex);
+            local_blocked_tasks.splice(local_blocked_tasks.end(), _blocked_tasks);
+            if (local_blocked_tasks.empty()) {
+                while (!_shutdown.load() && _blocked_tasks.empty()) {
+                    _task_cond.wait_for(lock, std::chrono::milliseconds(10));
+                }
+
+                if (_shutdown.load()) {
+                    break;
+                }
+
+                DCHECK(!_blocked_tasks.empty());
+                local_blocked_tasks.splice(local_blocked_tasks.end(), _blocked_tasks);
+            }
+        }
+
+        auto iter = local_blocked_tasks.begin();
+        DateTimeValue now = DateTimeValue::local_time();
+        while (iter != local_blocked_tasks.end()) {
+            auto* task = *iter;
+            auto state = task->get_state();
+            if (state == PENDING_FINISH) {
+                // should cancel or should finish
+                if (task->is_pending_finish()) {
+                    iter++;
+                } else {
+                    _make_task_run(local_blocked_tasks, iter, ready_tasks, PENDING_FINISH);
+                }
+            } else if (task->fragment_context()->is_canceled()) {
+                if (task->is_pending_finish()) {
+                    task->set_state(PENDING_FINISH);
+                    iter++;
+                } else {
+                    _make_task_run(local_blocked_tasks, iter, ready_tasks);
+                }
+            } else if (task->query_fragments_context()->is_timeout(now)) {
+                LOG(WARNING) << "Timeout, query_id="
+                             << print_id(task->query_fragments_context()->query_id)
+                             << ", instance_id="
+                             << print_id(task->fragment_context()->get_fragment_id());
+
+                task->fragment_context()->cancel(PPlanFragmentCancelReason::TIMEOUT);
+
+                if (task->is_pending_finish()) {
+                    task->set_state(PENDING_FINISH);
+                    iter++;
+                } else {
+                    _make_task_run(local_blocked_tasks, iter, ready_tasks);
+                }
+            } else if (state == BLOCKED_FOR_DEPENDENCY) {
+                if (task->has_dependency()) {
+                    iter++;
+                } else {
+                    _make_task_run(local_blocked_tasks, iter, ready_tasks);
+                }
+            } else if (state == BLOCKED_FOR_SOURCE) {
+                if (task->source_can_read()) {
+                    _make_task_run(local_blocked_tasks, iter, ready_tasks);
+                } else {
+                    iter++;
+                }
+            } else if (state == BLOCKED_FOR_SINK) {
+                if (task->sink_can_write()) {
+                    _make_task_run(local_blocked_tasks, iter, ready_tasks);
+                } else {
+                    iter++;
+                }
+            } else {
+                // TODO: DCHECK the state
+                _make_task_run(local_blocked_tasks, iter, ready_tasks);
+            }
+        }
+
+        if (ready_tasks.empty()) {
+            empty_times += 1;
+        } else {
+            empty_times = 0;
+            for (auto& task : ready_tasks) {
+                task->stop_schedule_watcher();
+                _task_queue->push_back(task);
+            }
+            ready_tasks.clear();
+        }
+
+        if (empty_times != 0 && (empty_times & (EMPTY_TIMES_TO_YIELD - 1)) == 0) {
+#ifdef __x86_64__
+            _mm_pause();
+#else
+            sched_yield();
+#endif
+        }
+        if (empty_times == EMPTY_TIMES_TO_YIELD * 10) {
+            empty_times = 0;
+            sched_yield();
+        }
+    }
+    LOG(INFO) << "BlockedTaskScheduler schedule thread stop";
+}
+
+void BlockedTaskScheduler::_make_task_run(std::list<PipelineTask*>& local_tasks,
+                                          std::list<PipelineTask*>::iterator& task_itr,
+                                          std::vector<PipelineTask*>& ready_tasks,
+                                          PipelineTaskState t_state) {
+    auto task = *task_itr;
+    task->start_schedule_watcher();
+    task->set_state(t_state);
+    local_tasks.erase(task_itr++);
+    ready_tasks.emplace_back(task);
+}
+
+/////////////////////////  TaskScheduler  ///////////////////////////////////////////////////////////////////////////
+
+TaskScheduler::~TaskScheduler() {
+    shutdown();
+}
+
+Status TaskScheduler::start() {
+    int cores = _task_queue->cores();
+    // Must be mutil number of cpu cores
+    ThreadPoolBuilder("TaskSchedulerThreadPool")
+            .set_min_threads(cores)
+            .set_max_threads(cores)
+            .set_max_queue_size(0)
+            .build(&_fix_thread_pool);
+    _markers.reserve(cores);
+    for (size_t i = 0; i < cores; ++i) {
+        LOG(INFO) << "Start TaskScheduler thread " << i;
+        _markers.push_back(std::make_unique<std::atomic<bool>>(true));
+        RETURN_IF_ERROR(
+                _fix_thread_pool->submit_func(std::bind(&TaskScheduler::_do_work, this, i)));
+    }
+    return _blocked_task_scheduler->start();
+}
+
+Status TaskScheduler::schedule_task(PipelineTask* task) {
+    if (task->is_blocking_state()) {
+        _blocked_task_scheduler->add_blocked_task(task);
+    } else {
+        _task_queue->push_back(task);
+    }
+    // TODO control num of task
+    return Status::OK();
+}
+
+void TaskScheduler::_do_work(size_t index) {
+    LOG(INFO) << "Start TaskScheduler worker " << index;
+    auto queue = _task_queue;
+    const auto& marker = _markers[index];
+    while (*marker) {
+        auto task = queue->try_take(index);
+        if (!task) {
+            task = queue->steal_take(index);
+            if (!task) {
+                // TODO: The take is a stock method, rethink the logic
+                task = queue->take(index);
+                if (!task) {
+                    continue;
+                }
+            }
+        }
+        task->stop_worker_watcher();
+        auto* fragment_ctx = task->fragment_context();
+        doris::signal::query_id_hi = fragment_ctx->get_query_id().hi;
+        doris::signal::query_id_lo = fragment_ctx->get_query_id().lo;
+        bool canceled = fragment_ctx->is_canceled();
+
+        auto check_state = task->get_state();
+        if (check_state == PENDING_FINISH) {
+            bool is_pending = task->is_pending_finish();
+            DCHECK(!is_pending) << "must not pending close " << task->debug_string();
+            _try_close_task(task, canceled ? CANCELED : FINISHED);
+            continue;
+        }
+        DCHECK(check_state != FINISHED && check_state != CANCELED) << "task already finish";
+
+        if (canceled) {
+            // may change from pending FINISH,should called cancel
+            // also may change form BLOCK, other task called cancel
+            _try_close_task(task, CANCELED);
+            continue;
+        }
+
+        DCHECK(check_state == RUNNABLE);
+        // task exec
+        bool eos = false;
+        auto status = task->execute(&eos);
+        task->set_previous_core_id(index);
+        if (!status.ok()) {
+            LOG(WARNING) << "Pipeline taks execute task fail " << task->debug_string();
+            // exec failed,cancel all fragment instance
+            fragment_ctx->cancel(PPlanFragmentCancelReason::INTERNAL_ERROR, "execute fail");
+            _try_close_task(task, CANCELED);
+            continue;
+        }
+
+        if (eos) {
+            // TODO: pipeline parallel need to wait the last task finish to call finalize
+            //  and find_p_dependency
+            status = task->finalize();
+            if (!status.ok()) {
+                // execute failed,cancel all fragment
+                fragment_ctx->cancel(PPlanFragmentCancelReason::INTERNAL_ERROR,
+                                     "finalize fail:" + status.to_string());
+                _try_close_task(task, CANCELED);
+            } else {
+                task->finish_p_dependency();
+                _try_close_task(task, FINISHED);
+            }
+            continue;
+        }
+
+        auto pipeline_state = task->get_state();
+        switch (pipeline_state) {
+        case BLOCKED_FOR_SOURCE:
+        case BLOCKED_FOR_SINK:
+            _blocked_task_scheduler->add_blocked_task(task);
+            break;
+        case RUNNABLE:
+            queue->push_back(task, index);
+            break;
+        default:
+            DCHECK(false) << "error state after run task, " << get_state_name(pipeline_state);
+            break;
+        }
+    }
+    LOG(INFO) << "Stop TaskScheduler worker " << index;
+}
+
+void TaskScheduler::_try_close_task(PipelineTask* task, PipelineTaskState state) {
+    // state only should be CANCELED or FINISHED
+    if (task->is_pending_finish()) {
+        task->set_state(PENDING_FINISH);
+        _blocked_task_scheduler->add_blocked_task(task);
+    } else {
+        auto status = task->close();
+        if (!status.ok()) {
+            // TODO: LOG warning
+        }
+        task->set_state(state);
+        // TODO: rethink the logic
+        if (state == CANCELED) {
+            task->finish_p_dependency();
+        }
+        task->fragment_context()->close_a_pipeline();
+    }
+}
+
+void TaskScheduler::shutdown() {
+    if (!this->_shutdown.load()) {
+        this->_shutdown.store(true);
+        _blocked_task_scheduler->shutdown();
+        if (_task_queue) {
+            _task_queue->close();
+        }
+        if (_fix_thread_pool) {
+            for (const auto& marker : _markers) {
+                marker->store(false);
+            }
+            _fix_thread_pool->shutdown();
+            _fix_thread_pool->wait();
+        }
+    }
+}
+
+} // namespace doris::pipeline
\ No newline at end of file
diff --git a/be/src/pipeline/task_scheduler.h b/be/src/pipeline/task_scheduler.h
new file mode 100644
index 0000000000..77e2ee8952
--- /dev/null
+++ b/be/src/pipeline/task_scheduler.h
@@ -0,0 +1,270 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include <queue>
+
+#include "common/status.h"
+#include "pipeline.h"
+#include "pipeline_task.h"
+#include "util/threadpool.h"
+
+namespace doris::pipeline {
+
+class SubWorkTaskQueue {
+    friend class WorkTaskQueue;
+
+public:
+    void add_total_time(const uint64_t duration) { _total_consume_time.fetch_add(duration); }
+
+    void push_back(PipelineTask* task) { _queue.emplace(task); }
+
+    PipelineTask* try_take() {
+        if (_queue.empty()) {
+            return nullptr;
+        }
+        auto task = _queue.front();
+        _queue.pop();
+        return task;
+    }
+
+    void set_factor_for_normal(double factor_for_normal) { _factor_for_normal = factor_for_normal; }
+
+    // TODO pipeline 1 may overflow here ?
+    double total_consume_time() { return _total_consume_time.load() / _factor_for_normal; }
+
+    bool empty() { return _queue.empty(); }
+
+private:
+    std::queue<PipelineTask*> _queue;
+    // factor for normalization
+    double _factor_for_normal = 1;
+    // TODO pipeline whether need to set to zero
+    // the value cal the queue task time consume, the WorkTaskQueue
+    // use it to find the min queue to take task work
+    std::atomic<uint64_t> _total_consume_time = 0;
+};
+
+// Each thread have private muti level queue
+class WorkTaskQueue {
+public:
+    explicit WorkTaskQueue() : _closed(false) {
+        double factor = 1;
+        for (int i = SUB_QUEUE_LEVEL - 1; i >= 0; --i) {
+            _sub_queues[i].set_factor_for_normal(factor);
+            factor *= LEVEL_QUEUE_TIME_FACTOR;
+        }
+    }
+
+    void close() {
+        std::unique_lock<std::mutex> lock(_work_size_mutex);
+        _closed = true;
+        _wait_task.notify_all();
+    }
+
+    PipelineTask* try_take_unprotected() {
+        if (_total_task_size == 0 || _closed) {
+            return nullptr;
+        }
+        double min_consume_time = _sub_queues[0].total_consume_time();
+        int idx = 0;
+        for (int i = 1; i < SUB_QUEUE_LEVEL; ++i) {
+            if (!_sub_queues[i].empty()) {
+                double consume_time = _sub_queues[i].total_consume_time();
+                if (idx == -1 || consume_time < min_consume_time) {
+                    idx = i;
+                    min_consume_time = consume_time;
+                }
+            }
+        }
+        auto task = _sub_queues[idx].try_take();
+        if (task) {
+            _total_task_size--;
+        }
+        return task;
+    }
+
+    PipelineTask* try_take() {
+        std::unique_lock<std::mutex> lock(_work_size_mutex);
+        return try_take_unprotected();
+    }
+
+    PipelineTask* take() {
+        std::unique_lock<std::mutex> lock(_work_size_mutex);
+        while (!_closed) {
+            auto task = try_take_unprotected();
+            if (task) {
+                return task;
+            } else {
+                _wait_task.wait(lock);
+            }
+        }
+        DCHECK(_closed);
+        return nullptr;
+    }
+
+    void push(PipelineTask* task) {
+        size_t level = _compute_level(task);
+        std::unique_lock<std::mutex> lock(_work_size_mutex);
+        _sub_queues[level].push_back(task);
+        _total_task_size++;
+        _wait_task.notify_one();
+    }
+
+    // Get the each thread task size to do
+    size_t size() { return _total_task_size; }
+
+private:
+    static constexpr auto LEVEL_QUEUE_TIME_FACTOR = 1.2;
+    static constexpr size_t SUB_QUEUE_LEVEL = 5;
+    SubWorkTaskQueue _sub_queues[SUB_QUEUE_LEVEL];
+    std::mutex _work_size_mutex;
+    std::condition_variable _wait_task;
+    std::atomic<size_t> _total_task_size = 0;
+    bool _closed;
+
+private:
+    size_t _compute_level(PipelineTask* task) { return 0; }
+};
+
+// Need consider NUMA architecture
+class TaskQueue {
+public:
+    explicit TaskQueue(size_t core_size) : _core_size(core_size) {
+        _async_queue.reset(new WorkTaskQueue[core_size]);
+    }
+
+    ~TaskQueue() = default;
+
+    void close() {
+        for (int i = 0; i < _core_size; ++i) {
+            _async_queue[i].close();
+        }
+    }
+
+    // Get the task by core id. TODO: To think the logic is useful?
+    PipelineTask* try_take(size_t core_id) { return _async_queue[core_id].try_take(); }
+
+    // not block, steal task by other core queue
+    PipelineTask* steal_take(size_t core_id) {
+        DCHECK(core_id < _core_size);
+        size_t next_id = core_id;
+        for (size_t i = 1; i < _core_size; ++i) {
+            ++next_id;
+            if (next_id == _core_size) {
+                next_id = 0;
+            }
+            DCHECK(next_id < _core_size);
+            auto task = try_take(next_id);
+            if (task) {
+                return task;
+            }
+        }
+        return nullptr;
+    }
+
+    // TODO pipeline 1 add timeout interface, other queue may have new task
+    PipelineTask* take(size_t core_id) { return _async_queue[core_id].take(); }
+
+    void push_back(PipelineTask* task) {
+        int core_id = task->get_previous_core_id();
+        if (core_id < 0) {
+            core_id = _next_core.fetch_add(1) % _core_size;
+        }
+        push_back(task, core_id);
+    }
+
+    void push_back(PipelineTask* task, size_t core_id) {
+        DCHECK(core_id < _core_size);
+        task->start_worker_watcher();
+        _async_queue[core_id].push(task);
+    }
+
+    int cores() const { return _core_size; }
+
+private:
+    std::unique_ptr<WorkTaskQueue[]> _async_queue;
+    size_t _core_size;
+    std::atomic<size_t> _next_core = 0;
+};
+
+// TODO pipeline sr
+class BlockedTaskScheduler {
+public:
+    explicit BlockedTaskScheduler(std::shared_ptr<TaskQueue> task_queue)
+            : _task_queue(std::move(task_queue)), _started(false), _shutdown(false) {}
+
+    ~BlockedTaskScheduler() = default;
+
+    Status start();
+    void shutdown();
+    void add_blocked_task(PipelineTask* task);
+
+private:
+    std::shared_ptr<TaskQueue> _task_queue;
+
+    std::mutex _task_mutex;
+    std::condition_variable _task_cond;
+    std::list<PipelineTask*> _blocked_tasks;
+
+    scoped_refptr<Thread> _thread;
+    std::atomic<bool> _started;
+    std::atomic<bool> _shutdown;
+
+    static constexpr auto EMPTY_TIMES_TO_YIELD = 64;
+
+private:
+    void _schedule();
+    void _make_task_run(std::list<PipelineTask*>& local_tasks,
+                        std::list<PipelineTask*>::iterator& task_itr,
+                        std::vector<PipelineTask*>& ready_tasks,
+                        PipelineTaskState state = PipelineTaskState::RUNNABLE);
+};
+
+class TaskScheduler {
+public:
+    TaskScheduler(ExecEnv* exec_env, std::shared_ptr<BlockedTaskScheduler> b_scheduler,
+                  std::shared_ptr<TaskQueue> task_queue)
+            : _task_queue(std::move(task_queue)),
+              _exec_env(exec_env),
+              _blocked_task_scheduler(std::move(b_scheduler)),
+              _shutdown(false) {}
+
+    ~TaskScheduler();
+
+    Status schedule_task(PipelineTask* task);
+
+    Status start();
+
+    void shutdown();
+
+    ExecEnv* exec_env() { return _exec_env; }
+
+private:
+    std::unique_ptr<ThreadPool> _fix_thread_pool;
+    std::shared_ptr<TaskQueue> _task_queue;
+    std::vector<std::unique_ptr<std::atomic<bool>>> _markers;
+    ExecEnv* _exec_env;
+    std::shared_ptr<BlockedTaskScheduler> _blocked_task_scheduler;
+    std::atomic<bool> _shutdown;
+
+private:
+    void _do_work(size_t index);
+    void _try_close_task(PipelineTask* task, PipelineTaskState state);
+};
+} // namespace doris::pipeline
\ No newline at end of file
diff --git a/be/src/runtime/buffer_control_block.cpp b/be/src/runtime/buffer_control_block.cpp
index 03bd6d3466..a3d765f3c2 100644
--- a/be/src/runtime/buffer_control_block.cpp
+++ b/be/src/runtime/buffer_control_block.cpp
@@ -95,6 +95,11 @@ Status BufferControlBlock::init() {
     return Status::OK();
 }
 
+bool BufferControlBlock::can_sink() {
+    std::unique_lock<std::mutex> l(_lock);
+    return _get_batch_queue_empty() || _buffer_rows < _buffer_limit || _is_cancelled;
+}
+
 Status BufferControlBlock::add_batch(std::unique_ptr<TFetchDataResult>& result) {
     std::unique_lock<std::mutex> l(_lock);
 
@@ -104,7 +109,7 @@ Status BufferControlBlock::add_batch(std::unique_ptr<TFetchDataResult>& result)
 
     int num_rows = result->result_batch.rows.size();
 
-    while ((!_batch_queue.empty() && (num_rows + _buffer_rows) > _buffer_limit) && !_is_cancelled) {
+    while ((!_batch_queue.empty() && _buffer_rows > _buffer_limit) && !_is_cancelled) {
         _data_removal.wait(l);
     }
 
@@ -158,7 +163,7 @@ Status BufferControlBlock::get_batch(TFetchDataResult* result) {
     _batch_queue.pop_front();
     _buffer_rows -= item->result_batch.rows.size();
     _data_removal.notify_one();
-    *result = *(item.get());
+    *result = *item;
     result->__set_packet_num(_packet_num);
     _packet_num++;
     return Status::OK();
diff --git a/be/src/runtime/buffer_control_block.h b/be/src/runtime/buffer_control_block.h
index 8528f74164..e1dba35762 100644
--- a/be/src/runtime/buffer_control_block.h
+++ b/be/src/runtime/buffer_control_block.h
@@ -17,6 +17,7 @@
 
 #pragma once
 
+#include <atomic>
 #include <condition_variable>
 #include <deque>
 #include <list>
@@ -60,9 +61,10 @@ struct GetResultBatchCtx {
 class BufferControlBlock {
 public:
     BufferControlBlock(const TUniqueId& id, int buffer_size);
-    ~BufferControlBlock();
+    virtual ~BufferControlBlock();
 
     Status init();
+    virtual bool can_sink(); // 只有一个fragment写入,因此can_sink返回true,则一定可以执行sink
     Status add_batch(std::unique_ptr<TFetchDataResult>& result);
 
     // get result from batch, use timeout?
@@ -86,7 +88,7 @@ public:
         // _query_statistics may be null when the result sink init failed
         // or some other failure.
         // and the number of written rows is only needed when all things go well.
-        if (_query_statistics.get() != nullptr) {
+        if (_query_statistics != nullptr) {
             _query_statistics->set_returned_rows(num_rows);
         }
     }
@@ -94,22 +96,25 @@ public:
     // TODO: The value of query peak mem usage in fe.audit.log comes from a random BE,
     // not the BE with the largest peak mem usage
     void update_max_peak_memory_bytes() {
-        if (_query_statistics.get() != nullptr) {
+        if (_query_statistics != nullptr) {
             int64_t max_peak_memory_bytes = _query_statistics->calculate_max_peak_memory_bytes();
             _query_statistics->set_max_peak_memory_bytes(max_peak_memory_bytes);
         }
     }
 
-private:
-    typedef std::list<std::unique_ptr<TFetchDataResult>> ResultQueue;
+protected:
+    virtual bool _get_batch_queue_empty() { return _batch_queue.empty(); }
+    virtual void _update_batch_queue_empty() {}
+
+    using ResultQueue = std::list<std::unique_ptr<TFetchDataResult>>;
 
     // result's query id
     TUniqueId _fragment_id;
     bool _is_close;
-    bool _is_cancelled;
+    std::atomic_bool _is_cancelled;
     Status _status;
-    int _buffer_rows;
-    int _buffer_limit;
+    std::atomic_int _buffer_rows;
+    const int _buffer_limit;
     int64_t _packet_num;
 
     // blocking queue for batch
@@ -129,4 +134,20 @@ private:
     std::shared_ptr<QueryStatistics> _query_statistics;
 };
 
+class PipBufferControlBlock : public BufferControlBlock {
+public:
+    PipBufferControlBlock(const TUniqueId& id, int buffer_size)
+            : BufferControlBlock(id, buffer_size) {}
+
+    bool can_sink() override {
+        return _get_batch_queue_empty() || _buffer_rows < _buffer_limit || _is_cancelled;
+    }
+
+private:
+    bool _get_batch_queue_empty() override { return _batch_queue_empty; }
+    void _update_batch_queue_empty() override { _batch_queue_empty = _batch_queue.empty(); }
+
+    std::atomic_bool _batch_queue_empty = false;
+};
+
 } // namespace doris
diff --git a/be/src/runtime/exec_env.h b/be/src/runtime/exec_env.h
index 4bd9fd73af..6ce903f9fd 100644
--- a/be/src/runtime/exec_env.h
+++ b/be/src/runtime/exec_env.h
@@ -29,6 +29,9 @@ namespace vectorized {
 class VDataStreamMgr;
 class ScannerScheduler;
 } // namespace vectorized
+namespace pipeline {
+class TaskScheduler;
+}
 class BfdParser;
 class BrokerMgr;
 
@@ -110,6 +113,8 @@ public:
     ClientCache<FrontendServiceClient>* frontend_client_cache() { return _frontend_client_cache; }
     ClientCache<TPaloBrokerServiceClient>* broker_client_cache() { return _broker_client_cache; }
 
+    pipeline::TaskScheduler* pipeline_task_scheduler() { return _pipeline_task_scheduler; }
+
     // using template to simplify client cache management
     template <typename T>
     ClientCache<T>* get_client_cache() {
@@ -137,6 +142,7 @@ public:
     }
     void init_download_cache_buf();
     void init_download_cache_required_components();
+    Status init_pipeline_task_scheduler();
     char* get_download_cache_buf(ThreadPoolToken* token) {
         if (_download_cache_buf_map.find(token) == _download_cache_buf_map.end()) {
             return nullptr;
@@ -233,6 +239,7 @@ private:
     std::unordered_map<ThreadPoolToken*, std::unique_ptr<char[]>> _download_cache_buf_map;
     CgroupsMgr* _cgroups_mgr = nullptr;
     FragmentMgr* _fragment_mgr = nullptr;
+    pipeline::TaskScheduler* _pipeline_task_scheduler = nullptr;
     ResultCache* _result_cache = nullptr;
     TMasterInfo* _master_info = nullptr;
     LoadPathMgr* _load_path_mgr = nullptr;
diff --git a/be/src/runtime/exec_env_init.cpp b/be/src/runtime/exec_env_init.cpp
index 735d8091ed..d84b40c670 100644
--- a/be/src/runtime/exec_env_init.cpp
+++ b/be/src/runtime/exec_env_init.cpp
@@ -25,6 +25,7 @@
 #include "olap/segment_loader.h"
 #include "olap/storage_engine.h"
 #include "olap/storage_policy_mgr.h"
+#include "pipeline/task_scheduler.h"
 #include "runtime/broker_mgr.h"
 #include "runtime/bufferpool/buffer_pool.h"
 #include "runtime/cache/result_cache.h"
@@ -128,6 +129,7 @@ Status ExecEnv::_init(const std::vector<StorePath>& store_paths) {
 
     init_download_cache_required_components();
 
+    RETURN_IF_ERROR(init_pipeline_task_scheduler());
     _scanner_scheduler = new doris::vectorized::ScannerScheduler();
 
     _cgroups_mgr = new CgroupsMgr(this, config::doris_cgroups);
@@ -172,6 +174,18 @@ Status ExecEnv::_init(const std::vector<StorePath>& store_paths) {
     return Status::OK();
 }
 
+Status ExecEnv::init_pipeline_task_scheduler() {
+    auto executors_size = config::pipeline_executor_size;
+    if (executors_size <= 0) {
+        executors_size = CpuInfo::num_cores();
+    }
+    auto t_queue = std::make_shared<pipeline::TaskQueue>(executors_size);
+    auto b_scheduler = std::make_shared<pipeline::BlockedTaskScheduler>(t_queue);
+    _pipeline_task_scheduler = new pipeline::TaskScheduler(this, b_scheduler, t_queue);
+    RETURN_IF_ERROR(_pipeline_task_scheduler->start());
+    return Status::OK();
+}
+
 Status ExecEnv::_init_mem_env() {
     bool is_percent = false;
     std::stringstream ss;
@@ -348,6 +362,7 @@ void ExecEnv::_destroy() {
     SAFE_DELETE(_load_path_mgr);
     SAFE_DELETE(_master_info);
     SAFE_DELETE(_fragment_mgr);
+    SAFE_DELETE(_pipeline_task_scheduler);
     SAFE_DELETE(_cgroups_mgr);
     SAFE_DELETE(_scan_thread_pool);
     SAFE_DELETE(_remote_scan_thread_pool);
diff --git a/be/src/runtime/fragment_mgr.cpp b/be/src/runtime/fragment_mgr.cpp
index b4e0595451..b7ae9cf735 100644
--- a/be/src/runtime/fragment_mgr.cpp
+++ b/be/src/runtime/fragment_mgr.cpp
@@ -34,6 +34,7 @@
 #include "gen_cpp/Types_types.h"
 #include "gutil/strings/substitute.h"
 #include "opentelemetry/trace/scope.h"
+#include "pipeline/pipeline_fragment_context.h"
 #include "runtime/client_cache.h"
 #include "runtime/datetime_value.h"
 #include "runtime/descriptors.h"
@@ -548,7 +549,13 @@ Status FragmentMgr::exec_plan_fragment(const TExecPlanFragmentParams& params) {
         set_pipe(params.params.fragment_instance_id, pipe);
         return Status::OK();
     } else {
-        return exec_plan_fragment(params, std::bind<void>(&empty_function, std::placeholders::_1));
+        if (params.query_options.__isset.enable_pipeline_engine &&
+            params.query_options.enable_pipeline_engine) {
+            return exec_pipeline(params);
+        } else {
+            return exec_plan_fragment(params,
+                                      std::bind<void>(&empty_function, std::placeholders::_1));
+        }
     }
 }
 
@@ -591,6 +598,132 @@ std::shared_ptr<StreamLoadPipe> FragmentMgr::get_pipe(const TUniqueId& fragment_
     }
 }
 
+Status FragmentMgr::exec_pipeline(const TExecPlanFragmentParams& params) {
+    auto tracer = telemetry::is_current_span_valid() ? telemetry::get_tracer("tracer")
+                                                     : telemetry::get_noop_tracer();
+    START_AND_SCOPE_SPAN(tracer, span, "FragmentMgr::exec_plan_fragment");
+    const TUniqueId& fragment_instance_id = params.params.fragment_instance_id;
+    {
+        std::lock_guard<std::mutex> lock(_lock);
+        auto iter = _pipeline_map.find(fragment_instance_id);
+        if (iter != _pipeline_map.end()) {
+            // Duplicated
+            return Status::OK();
+        }
+    }
+
+    std::shared_ptr<QueryFragmentsCtx> fragments_ctx;
+    if (params.is_simplified_param) {
+        // Get common components from _fragments_ctx_map
+        std::lock_guard<std::mutex> lock(_lock);
+        auto search = _fragments_ctx_map.find(params.params.query_id);
+        if (search == _fragments_ctx_map.end()) {
+            return Status::InternalError(
+                    "Failed to get query fragments context. Query may be "
+                    "timeout or be cancelled. host: {}",
+                    BackendOptions::get_localhost());
+        }
+        fragments_ctx = search->second;
+        _set_scan_concurrency(params, fragments_ctx.get());
+    } else {
+        // This may be a first fragment request of the query.
+        // Create the query fragments context.
+        fragments_ctx = std::make_shared<QueryFragmentsCtx>(params.fragment_num_on_host, _exec_env);
+        fragments_ctx->query_id = params.params.query_id;
+        RETURN_IF_ERROR(DescriptorTbl::create(&(fragments_ctx->obj_pool), params.desc_tbl,
+                                              &(fragments_ctx->desc_tbl)));
+        fragments_ctx->coord_addr = params.coord;
+        LOG(INFO) << "query_id: "
+                  << UniqueId(fragments_ctx->query_id.hi, fragments_ctx->query_id.lo)
+                  << " coord_addr " << fragments_ctx->coord_addr;
+        fragments_ctx->query_globals = params.query_globals;
+
+        if (params.__isset.resource_info) {
+            fragments_ctx->user = params.resource_info.user;
+            fragments_ctx->group = params.resource_info.group;
+            fragments_ctx->set_rsc_info = true;
+        }
+
+        fragments_ctx->timeout_second = params.query_options.query_timeout;
+        _set_scan_concurrency(params, fragments_ctx.get());
+
+        bool has_query_mem_tracker =
+                params.query_options.__isset.mem_limit && (params.query_options.mem_limit > 0);
+        int64_t bytes_limit = has_query_mem_tracker ? params.query_options.mem_limit : -1;
+        if (bytes_limit > MemInfo::mem_limit()) {
+            VLOG_NOTICE << "Query memory limit " << PrettyPrinter::print(bytes_limit, TUnit::BYTES)
+                        << " exceeds process memory limit of "
+                        << PrettyPrinter::print(MemInfo::mem_limit(), TUnit::BYTES)
+                        << ". Using process memory limit instead";
+            bytes_limit = MemInfo::mem_limit();
+        }
+        if (params.query_options.query_type == TQueryType::SELECT) {
+            fragments_ctx->query_mem_tracker = std::make_shared<MemTrackerLimiter>(
+                    MemTrackerLimiter::Type::QUERY,
+                    fmt::format("Query#Id={}", print_id(fragments_ctx->query_id)), bytes_limit);
+        } else if (params.query_options.query_type == TQueryType::LOAD) {
+            fragments_ctx->query_mem_tracker = std::make_shared<MemTrackerLimiter>(
+                    MemTrackerLimiter::Type::LOAD,
+                    fmt::format("Load#Id={}", print_id(fragments_ctx->query_id)), bytes_limit);
+        }
+        if (params.query_options.__isset.is_report_success &&
+            params.query_options.is_report_success) {
+            fragments_ctx->query_mem_tracker->enable_print_log_usage();
+        }
+
+        if (!params.__isset.need_wait_execution_trigger || !params.need_wait_execution_trigger) {
+            // 马上运行pipeline task
+            fragments_ctx->set_ready_to_execute_only();
+        }
+
+        {
+            // Find _fragments_ctx_map again, in case some other request has already
+            // create the query fragments context.
+            std::lock_guard<std::mutex> lock(_lock);
+            auto search = _fragments_ctx_map.find(params.params.query_id);
+            if (search == _fragments_ctx_map.end()) {
+                _fragments_ctx_map.insert(std::make_pair(fragments_ctx->query_id, fragments_ctx));
+            } else {
+                // Already has a query fragmentscontext, use it
+                fragments_ctx = search->second;
+            }
+        }
+    }
+
+    std::shared_ptr<pipeline::PipelineFragmentContext> context =
+            std::make_shared<pipeline::PipelineFragmentContext>(
+                    fragments_ctx->query_id, fragment_instance_id, params.backend_num,
+                    fragments_ctx, _exec_env);
+    RETURN_IF_ERROR(context->prepare(params));
+    {
+        std::lock_guard<std::mutex> lock(_lock);
+        _pipeline_map.insert(std::make_pair(fragment_instance_id, context));
+        _cv.notify_all();
+    }
+
+    auto st = context->submit();
+    if (!st.ok()) {
+        // TODO pipeline 如果一个task都没有提交成功,则要让timeout checker线程去移除
+        // 提交失败也不能移出,可能有些pipeline task提交成功,有些失败,要等所有task都结束才能移除。
+        context->cancel(PPlanFragmentCancelReason::INTERNAL_ERROR, "submit context fail");
+        remove_pipeline_context(context);
+        return Status::InternalError("Submit pipeline failed. err = {}, BE: {}", st.get_error_msg(),
+                                     BackendOptions::get_localhost());
+    }
+    return Status::OK();
+}
+void FragmentMgr::remove_pipeline_context(
+        std::shared_ptr<pipeline::PipelineFragmentContext> f_context) {
+    std::lock_guard<std::mutex> lock(_lock);
+    auto query_id = f_context->get_query_id();
+    auto* q_context = f_context->get_query_context();
+    bool all_done = q_context->countdown();
+    _pipeline_map.erase(f_context->get_fragment_id());
+    if (all_done) {
+        _fragments_ctx_map.erase(query_id);
+    }
+}
+
 Status FragmentMgr::exec_plan_fragment(const TExecPlanFragmentParams& params, FinishCallback cb) {
     auto tracer = telemetry::is_current_span_valid() ? telemetry::get_tracer("tracer")
                                                      : telemetry::get_noop_tracer();
@@ -780,21 +913,32 @@ bool FragmentMgr::_is_scan_node(const TPlanNodeType::type& type) {
            type == TPlanNodeType::JDBC_SCAN_NODE;
 }
 
-Status FragmentMgr::cancel(const TUniqueId& fragment_id, const PPlanFragmentCancelReason& reason,
-                           const std::string& msg) {
+void FragmentMgr::cancel(const TUniqueId& fragment_id, const PPlanFragmentCancelReason& reason,
+                         const std::string& msg) {
     std::shared_ptr<FragmentExecState> exec_state;
     {
         std::lock_guard<std::mutex> lock(_lock);
         auto iter = _fragment_map.find(fragment_id);
-        if (iter == _fragment_map.end()) {
-            // No match
-            return Status::OK();
+        if (iter != _fragment_map.end()) {
+            exec_state = iter->second;
         }
-        exec_state = iter->second;
     }
-    exec_state->cancel(reason, msg);
+    if (exec_state) {
+        exec_state->cancel(reason, msg);
+        return;
+    }
 
-    return Status::OK();
+    std::shared_ptr<pipeline::PipelineFragmentContext> pipeline_fragment_ctx;
+    {
+        std::lock_guard<std::mutex> lock(_lock);
+        auto iter = _pipeline_map.find(fragment_id);
+        if (iter != _pipeline_map.end()) {
+            pipeline_fragment_ctx = iter->second;
+        }
+    }
+    if (pipeline_fragment_ctx) {
+        pipeline_fragment_ctx->cancel(reason, msg);
+    }
 }
 
 void FragmentMgr::cancel_worker() {
@@ -962,6 +1106,7 @@ Status FragmentMgr::exec_external_plan_fragment(const TScanOpenParams& params,
 
 Status FragmentMgr::apply_filter(const PPublishFilterRequest* request,
                                  butil::IOBufAsZeroCopyInputStream* attach_data) {
+    // TODO pipeline runtime filter
     UniqueId fragment_instance_id = request->fragment_id();
     TUniqueId tfragment_instance_id = fragment_instance_id.to_thrift();
     std::shared_ptr<FragmentExecState> fragment_state;
diff --git a/be/src/runtime/fragment_mgr.h b/be/src/runtime/fragment_mgr.h
index 2246a42ac8..4894ccddbf 100644
--- a/be/src/runtime/fragment_mgr.h
+++ b/be/src/runtime/fragment_mgr.h
@@ -42,6 +42,10 @@ class IOBufAsZeroCopyInputStream;
 
 namespace doris {
 
+namespace pipeline {
+class PipelineFragmentContext;
+}
+
 class QueryFragmentsCtx;
 class ExecEnv;
 class FragmentExecState;
@@ -66,17 +70,22 @@ public:
     // execute one plan fragment
     Status exec_plan_fragment(const TExecPlanFragmentParams& params);
 
+    Status exec_pipeline(const TExecPlanFragmentParams& params);
+
+    void remove_pipeline_context(
+            std::shared_ptr<pipeline::PipelineFragmentContext> pipeline_context);
+
     // TODO(zc): report this is over
     Status exec_plan_fragment(const TExecPlanFragmentParams& params, FinishCallback cb);
 
     Status start_query_execution(const PExecPlanFragmentStartRequest* request);
 
-    Status cancel(const TUniqueId& fragment_id) {
-        return cancel(fragment_id, PPlanFragmentCancelReason::INTERNAL_ERROR);
+    void cancel(const TUniqueId& fragment_id) {
+        cancel(fragment_id, PPlanFragmentCancelReason::INTERNAL_ERROR);
     }
 
-    Status cancel(const TUniqueId& fragment_id, const PPlanFragmentCancelReason& reason,
-                  const std::string& msg = "");
+    void cancel(const TUniqueId& fragment_id, const PPlanFragmentCancelReason& reason,
+                const std::string& msg = "");
 
     void cancel_worker();
 
@@ -119,6 +128,9 @@ private:
 
     // Make sure that remove this before no data reference FragmentExecState
     std::unordered_map<TUniqueId, std::shared_ptr<FragmentExecState>> _fragment_map;
+
+    std::unordered_map<TUniqueId, std::shared_ptr<pipeline::PipelineFragmentContext>> _pipeline_map;
+
     // query id -> QueryFragmentsCtx
     std::unordered_map<TUniqueId, std::shared_ptr<QueryFragmentsCtx>> _fragments_ctx_map;
     std::unordered_map<TUniqueId, std::unordered_map<int, int64_t>> _bf_size_map;
diff --git a/be/src/runtime/plan_fragment_executor.cpp b/be/src/runtime/plan_fragment_executor.cpp
index 3a9206f1fa..f386da763d 100644
--- a/be/src/runtime/plan_fragment_executor.cpp
+++ b/be/src/runtime/plan_fragment_executor.cpp
@@ -190,9 +190,9 @@ Status PlanFragmentExecutor::prepare(const TExecPlanFragmentParams& request,
 
     // set up sink, if required
     if (request.fragment.__isset.output_sink) {
-        RETURN_IF_ERROR(DataSink::create_data_sink(
-                obj_pool(), request.fragment.output_sink, request.fragment.output_exprs, params,
-                row_desc(), runtime_state()->enable_vectorized_exec(), &_sink, *desc_tbl));
+        RETURN_IF_ERROR(DataSink::create_data_sink(obj_pool(), request.fragment.output_sink,
+                                                   request.fragment.output_exprs, params,
+                                                   row_desc(), runtime_state(), &_sink, *desc_tbl));
         RETURN_IF_ERROR(_sink->prepare(runtime_state()));
 
         RuntimeProfile* sink_profile = _sink->profile();
diff --git a/be/src/runtime/query_fragments_ctx.h b/be/src/runtime/query_fragments_ctx.h
index 4bfff8c4da..8513e99b88 100644
--- a/be/src/runtime/query_fragments_ctx.h
+++ b/be/src/runtime/query_fragments_ctx.h
@@ -90,6 +90,18 @@ public:
         }
         _start_cond.notify_all();
     }
+    void set_ready_to_execute_only() {
+        {
+            std::lock_guard<std::mutex> l(_start_lock);
+            _ready_to_execute = true;
+        }
+        _start_cond.notify_all();
+    }
+
+    bool is_ready_to_execute() {
+        std::lock_guard<std::mutex> l(_start_lock);
+        return _ready_to_execute;
+    }
 
     bool wait_for_start() {
         int wait_time = config::max_fragment_start_wait_time_seconds;
diff --git a/be/src/runtime/result_buffer_mgr.cpp b/be/src/runtime/result_buffer_mgr.cpp
index ce589dd745..e1d3069459 100644
--- a/be/src/runtime/result_buffer_mgr.cpp
+++ b/be/src/runtime/result_buffer_mgr.cpp
@@ -17,6 +17,8 @@
 
 #include "runtime/result_buffer_mgr.h"
 
+#include <memory>
+
 #include "gen_cpp/PaloInternalService_types.h"
 #include "gen_cpp/types.pb.h"
 #include "runtime/buffer_control_block.h"
@@ -58,15 +60,22 @@ Status ResultBufferMgr::init() {
 }
 
 Status ResultBufferMgr::create_sender(const TUniqueId& query_id, int buffer_size,
-                                      std::shared_ptr<BufferControlBlock>* sender) {
+                                      std::shared_ptr<BufferControlBlock>* sender,
+                                      bool enable_pipeline) {
     *sender = find_control_block(query_id);
     if (*sender != nullptr) {
         LOG(WARNING) << "already have buffer control block for this instance " << query_id;
         return Status::OK();
     }
 
-    std::shared_ptr<BufferControlBlock> control_block(
-            new BufferControlBlock(query_id, buffer_size));
+    std::shared_ptr<BufferControlBlock> control_block = nullptr;
+
+    if (enable_pipeline) {
+        control_block = std::make_shared<PipBufferControlBlock>(query_id, buffer_size);
+    } else {
+        control_block = std::make_shared<BufferControlBlock>(query_id, buffer_size);
+    }
+
     {
         std::lock_guard<std::mutex> l(_lock);
         _buffer_map.insert(std::make_pair(query_id, control_block));
diff --git a/be/src/runtime/result_buffer_mgr.h b/be/src/runtime/result_buffer_mgr.h
index 26a07bd90c..c4722bcef7 100644
--- a/be/src/runtime/result_buffer_mgr.h
+++ b/be/src/runtime/result_buffer_mgr.h
@@ -47,7 +47,7 @@ public:
     // the returned sender do not need release
     // sender is not used when call cancel or unregister
     Status create_sender(const TUniqueId& query_id, int buffer_size,
-                         std::shared_ptr<BufferControlBlock>* sender);
+                         std::shared_ptr<BufferControlBlock>* sender, bool enable_pipeline);
     // fetch data, used by RPC
     Status fetch_data(const TUniqueId& fragment_id, TFetchDataResult* result);
 
@@ -60,8 +60,8 @@ public:
     Status cancel_at_time(time_t cancel_time, const TUniqueId& query_id);
 
 private:
-    typedef std::unordered_map<TUniqueId, std::shared_ptr<BufferControlBlock>> BufferMap;
-    typedef std::map<time_t, std::vector<TUniqueId>> TimeoutMap;
+    using BufferMap = std::unordered_map<TUniqueId, std::shared_ptr<BufferControlBlock>>;
+    using TimeoutMap = std::map<time_t, std::vector<TUniqueId>>;
 
     std::shared_ptr<BufferControlBlock> find_control_block(const TUniqueId& query_id);
 
diff --git a/be/src/runtime/result_file_sink.cpp b/be/src/runtime/result_file_sink.cpp
index cd3e61659a..91143980cd 100644
--- a/be/src/runtime/result_file_sink.cpp
+++ b/be/src/runtime/result_file_sink.cpp
@@ -100,7 +100,8 @@ Status ResultFileSink::prepare(RuntimeState* state) {
     if (_is_top_sink) {
         // create sender
         RETURN_IF_ERROR(state->exec_env()->result_mgr()->create_sender(
-                state->fragment_instance_id(), _buf_size, &_sender));
+                state->fragment_instance_id(), _buf_size, &_sender,
+                _state->enable_pipeline_exec()));
         // create writer
         _writer.reset(new (std::nothrow) FileResultWriter(
                 _file_opts.get(), _storage_type, state->fragment_instance_id(), _output_expr_ctxs,
diff --git a/be/src/runtime/result_sink.cpp b/be/src/runtime/result_sink.cpp
index de393e9e11..625e495ffa 100644
--- a/be/src/runtime/result_sink.cpp
+++ b/be/src/runtime/result_sink.cpp
@@ -66,8 +66,8 @@ Status ResultSink::prepare(RuntimeState* state) {
     RETURN_IF_ERROR(prepare_exprs(state));
 
     // create sender
-    RETURN_IF_ERROR(state->exec_env()->result_mgr()->create_sender(state->fragment_instance_id(),
-                                                                   _buf_size, &_sender));
+    RETURN_IF_ERROR(state->exec_env()->result_mgr()->create_sender(
+            state->fragment_instance_id(), _buf_size, &_sender, state->enable_pipeline_exec()));
 
     // create writer based on sink type
     switch (_sink_type) {
diff --git a/be/src/runtime/runtime_state.h b/be/src/runtime/runtime_state.h
index 7b8ae4d89d..016f0089fd 100644
--- a/be/src/runtime/runtime_state.h
+++ b/be/src/runtime/runtime_state.h
@@ -176,9 +176,9 @@ public:
     // _unreported_error_idx to _errors_log.size()
     void get_unreported_errors(std::vector<std::string>* new_errors);
 
-    bool is_cancelled() const { return _is_cancelled; }
+    bool is_cancelled() const { return _is_cancelled.load(); }
     int codegen_level() const { return _query_options.codegen_level; }
-    void set_is_cancelled(bool v) { _is_cancelled = v; }
+    void set_is_cancelled(bool v) { _is_cancelled.store(v); }
 
     void set_backend_id(int64_t backend_id) { _backend_id = backend_id; }
     int64_t backend_id() const { return _backend_id; }
@@ -326,6 +326,7 @@ public:
         }
         return _query_options.be_exec_version;
     }
+    bool enable_pipeline_exec() const { return _query_options.enable_pipeline_engine; }
 
     bool trim_tailing_spaces_for_external_table_query() const {
         return _query_options.trim_tailing_spaces_for_external_table_query;
@@ -464,7 +465,7 @@ private:
     ThreadResourceMgr::ResourcePool* _resource_pool;
 
     // if true, execution should stop with a CANCELLED status
-    bool _is_cancelled;
+    std::atomic<bool> _is_cancelled;
 
     int _per_fragment_instance_idx;
     int _num_per_fragment_instances = 0;
diff --git a/be/src/service/backend_service.cpp b/be/src/service/backend_service.cpp
index fb1b0851f9..b584668f58 100644
--- a/be/src/service/backend_service.cpp
+++ b/be/src/service/backend_service.cpp
@@ -103,7 +103,7 @@ Status BackendService::start_plan_fragment_execution(const TExecPlanFragmentPara
 void BackendService::cancel_plan_fragment(TCancelPlanFragmentResult& return_val,
                                           const TCancelPlanFragmentParams& params) {
     LOG(INFO) << "cancel_plan_fragment(): instance_id=" << params.fragment_instance_id;
-    _exec_env->fragment_mgr()->cancel(params.fragment_instance_id).set_t_status(&return_val);
+    _exec_env->fragment_mgr()->cancel(params.fragment_instance_id);
 }
 
 void BackendService::transmit_data(TTransmitDataResult& return_val,
diff --git a/be/src/service/internal_service.cpp b/be/src/service/internal_service.cpp
index 28b9b8f4ca..c33db5a884 100644
--- a/be/src/service/internal_service.cpp
+++ b/be/src/service/internal_service.cpp
@@ -388,18 +388,17 @@ void PInternalServiceImpl::cancel_plan_fragment(google::protobuf::RpcController*
     tid.__set_hi(request->finst_id().hi());
     tid.__set_lo(request->finst_id().lo());
 
-    Status st;
+    Status st = Status::OK();
     if (request->has_cancel_reason()) {
         LOG(INFO) << "cancel fragment, fragment_instance_id=" << print_id(tid)
                   << ", reason: " << request->cancel_reason();
-        st = _exec_env->fragment_mgr()->cancel(tid, request->cancel_reason());
+        _exec_env->fragment_mgr()->cancel(tid, request->cancel_reason());
     } else {
         LOG(INFO) << "cancel fragment, fragment_instance_id=" << print_id(tid);
-        st = _exec_env->fragment_mgr()->cancel(tid);
-    }
-    if (!st.ok()) {
-        LOG(WARNING) << "cancel plan fragment failed, errmsg=" << st.get_error_msg();
+        _exec_env->fragment_mgr()->cancel(tid);
     }
+
+    // TODO: the logic seems useless, cancel only return Status::OK. remove it
     st.to_protobuf(result->mutable_status());
 }
 
diff --git a/be/src/util/proto_util.h b/be/src/util/proto_util.h
index 1ad4796691..5828b8eeec 100644
--- a/be/src/util/proto_util.h
+++ b/be/src/util/proto_util.h
@@ -17,7 +17,15 @@
 
 #pragma once
 
+#include <brpc/http_method.h>
+#include <gen_cpp/internal_service.pb.h>
+
+#include "common/config.h"
+#include "common/status.h"
 #include "exception.h"
+#include "runtime/exec_env.h"
+#include "runtime/runtime_state.h"
+#include "util/brpc_client_cache.h"
 
 namespace doris {
 
@@ -28,6 +36,55 @@ namespace doris {
 // 2G: In the default "baidu_std" brpcd, upper limit of the request and attachment length is 2G.
 constexpr size_t MIN_HTTP_BRPC_SIZE = (1ULL << 31);
 
+// Embed column_values and brpc request serialization string in controller attachment.
+template <typename Params, typename Closure>
+inline Status request_embed_attachment_contain_block(Params* brpc_request, Closure* closure) {
+    auto block = brpc_request->block();
+    Status st = request_embed_attachment(brpc_request, block.column_values(), closure);
+    block.set_column_values("");
+    return st;
+}
+
+inline bool enable_http_send_block(
+        const PTransmitDataParams& request,
+        bool transfer_large_data_by_brpc = config::transfer_large_data_by_brpc) {
+    if (!config::transfer_large_data_by_brpc) {
+        return false;
+    }
+    if (!request.has_block() || !request.block().has_column_values()) {
+        return false;
+    }
+    if (request.ByteSizeLong() < MIN_HTTP_BRPC_SIZE) {
+        return false;
+    }
+    return true;
+}
+
+template <typename Closure>
+inline void transmit_block(PBackendService_Stub& stub, Closure* closure,
+                           const PTransmitDataParams& params) {
+    closure->cntl.http_request().Clear();
+    stub.transmit_block(&closure->cntl, &params, &closure->result, closure);
+}
+
+template <typename Closure>
+inline Status transmit_block_http(RuntimeState* state, Closure* closure,
+                                  PTransmitDataParams& params, TNetworkAddress brpc_dest_addr) {
+    RETURN_IF_ERROR(request_embed_attachment_contain_block(&params, closure));
+
+    std::string brpc_url =
+            fmt::format("http://{}:{}", brpc_dest_addr.hostname, brpc_dest_addr.port);
+    std::shared_ptr<PBackendService_Stub> brpc_http_stub =
+            state->exec_env()->brpc_internal_client_cache()->get_new_client_no_cache(brpc_url,
+                                                                                     "http");
+    closure->cntl.http_request().uri() = brpc_url + "/PInternalServiceImpl/transmit_block_by_http";
+    closure->cntl.http_request().set_method(brpc::HTTP_METHOD_POST);
+    closure->cntl.http_request().set_content_type("application/json");
+    brpc_http_stub->transmit_block_by_http(&closure->cntl, nullptr, &closure->result, closure);
+
+    return Status::OK();
+}
+
 // TODO(zxy) delete in v1.3 version
 // Transfer RowBatch in ProtoBuf Request to Controller Attachment.
 // This can avoid reaching the upper limit of the ProtoBuf Request length (2G),
@@ -94,15 +151,6 @@ inline Status request_embed_attachment_contain_tuple(Params* brpc_request, Closu
     return st;
 }
 
-// Embed column_values and brpc request serialization string in controller attachment.
-template <typename Params, typename Closure>
-inline Status request_embed_attachment_contain_block(Params* brpc_request, Closure* closure) {
-    auto block = brpc_request->block();
-    Status st = request_embed_attachment(brpc_request, block.column_values(), closure);
-    block.set_column_values("");
-    return st;
-}
-
 template <typename Params, typename Closure>
 inline Status request_embed_attachment(Params* brpc_request, const std::string& data,
                                        Closure* closure) {
diff --git a/be/src/util/runtime_profile.cpp b/be/src/util/runtime_profile.cpp
index 604d98bfcc..32e76c94eb 100644
--- a/be/src/util/runtime_profile.cpp
+++ b/be/src/util/runtime_profile.cpp
@@ -235,6 +235,11 @@ void RuntimeProfile::divide(int n) {
     }
 }
 
+void RuntimeProfile::clear_children() {
+    std::lock_guard<std::mutex> l(_children_lock);
+    _children.clear();
+}
+
 void RuntimeProfile::compute_time_in_profile() {
     compute_time_in_profile(total_time_counter()->value());
 }
@@ -277,6 +282,15 @@ RuntimeProfile* RuntimeProfile::create_child(const std::string& name, bool inden
     return child;
 }
 
+void RuntimeProfile::insert_child_head(doris::RuntimeProfile* child, bool indent) {
+    std::lock_guard<std::mutex> l(_children_lock);
+    DCHECK(child != nullptr);
+    _child_map[child->_name] = child;
+
+    auto it = _children.begin();
+    _children.insert(it, std::make_pair(child, indent));
+}
+
 void RuntimeProfile::add_child_unlock(RuntimeProfile* child, bool indent, RuntimeProfile* loc) {
     DCHECK(child != nullptr);
     _child_map[child->_name] = child;
diff --git a/be/src/util/runtime_profile.h b/be/src/util/runtime_profile.h
index 62f0365afc..606e127274 100644
--- a/be/src/util/runtime_profile.h
+++ b/be/src/util/runtime_profile.h
@@ -48,10 +48,15 @@ namespace doris {
 #define ADD_CHILD_COUNTER(profile, name, type, parent) (profile)->add_counter(name, type, parent)
 #define ADD_CHILD_TIMER(profile, name, parent) (profile)->add_counter(name, TUnit::TIME_NS, parent)
 #define SCOPED_TIMER(c) ScopedTimer<MonotonicStopWatch> MACRO_CONCAT(SCOPED_TIMER, __COUNTER__)(c)
+#define SCOPED_TIMER_ATOMIC(c) \
+    ScopedTimer<MonotonicStopWatch, std::atomic_bool> MACRO_CONCAT(SCOPED_TIMER, __COUNTER__)(c)
 #define SCOPED_CPU_TIMER(c) \
     ScopedTimer<ThreadCpuStopWatch> MACRO_CONCAT(SCOPED_TIMER, __COUNTER__)(c)
 #define CANCEL_SAFE_SCOPED_TIMER(c, is_cancelled) \
     ScopedTimer<MonotonicStopWatch> MACRO_CONCAT(SCOPED_TIMER, __COUNTER__)(c, is_cancelled)
+#define CANCEL_SAFE_SCOPED_TIMER_ATOMIC(c, is_cancelled)                                       \
+    ScopedTimer<MonotonicStopWatch, std::atomic_bool> MACRO_CONCAT(SCOPED_TIMER, __COUNTER__)( \
+            c, is_cancelled)
 #define SCOPED_RAW_TIMER(c)                                                                  \
     doris::ScopedRawTimer<doris::MonotonicStopWatch, int64_t> MACRO_CONCAT(SCOPED_RAW_TIMER, \
                                                                            __COUNTER__)(c)
@@ -233,6 +238,8 @@ public:
     // already be added to the profile.
     void add_child(RuntimeProfile* child, bool indent, RuntimeProfile* location);
 
+    void insert_child_head(RuntimeProfile* child, bool indent);
+
     void add_child_unlock(RuntimeProfile* child, bool indent, RuntimeProfile* loc);
 
     /// Creates a new child profile with the given 'name'. A child profile with that name
@@ -383,6 +390,8 @@ public:
     // This function updates _local_time_percent for each profile.
     void compute_time_in_profile();
 
+    void clear_children();
+
 private:
     // Pool for allocated counters. Usually owned by the creator of this
     // object, but occasionally allocated in the constructor.
@@ -531,10 +540,10 @@ private:
 // Utility class to update time elapsed when the object goes out of scope.
 // 'T' must implement the stopWatch "interface" (start,stop,elapsed_time) but
 // we use templates not to pay for virtual function overhead.
-template <class T>
+template <class T, typename Bool = bool>
 class ScopedTimer {
 public:
-    ScopedTimer(RuntimeProfile::Counter* counter, const bool* is_cancelled = nullptr)
+    ScopedTimer(RuntimeProfile::Counter* counter, const Bool* is_cancelled = nullptr)
             : _counter(counter), _is_cancelled(is_cancelled) {
         if (counter == nullptr) {
             return;
@@ -557,7 +566,9 @@ public:
 
     // Update counter when object is destroyed
     ~ScopedTimer() {
-        if (_counter == nullptr) return;
+        if (_counter == nullptr) {
+            return;
+        }
         _sw.stop();
         UpdateCounter();
     }
@@ -569,7 +580,7 @@ public:
 private:
     T _sw;
     RuntimeProfile::Counter* _counter;
-    const bool* _is_cancelled;
+    const Bool* _is_cancelled;
 };
 
 // Utility class to update time elapsed when the object goes out of scope.
diff --git a/be/src/vec/exec/scan/new_olap_scan_node.cpp b/be/src/vec/exec/scan/new_olap_scan_node.cpp
index 996d2b2ac4..c72f5b289d 100644
--- a/be/src/vec/exec/scan/new_olap_scan_node.cpp
+++ b/be/src/vec/exec/scan/new_olap_scan_node.cpp
@@ -20,6 +20,9 @@
 #include "common/status.h"
 #include "olap/storage_engine.h"
 #include "olap/tablet.h"
+#include "pipeline/exec/olap_scan_operator.h"
+#include "pipeline/pipeline.h"
+#include "pipeline/pipeline_fragment_context.h"
 #include "util/to_string.h"
 #include "vec/columns/column_const.h"
 #include "vec/exec/scan/new_olap_scanner.h"
@@ -42,7 +45,7 @@ Status NewOlapScanNode::prepare(RuntimeState* state) {
 }
 
 Status NewOlapScanNode::_init_profile() {
-    VScanNode::_init_profile();
+    RETURN_IF_ERROR(VScanNode::_init_profile());
 
     _num_disks_accessed_counter = ADD_COUNTER(_runtime_profile, "NumDiskAccess", TUnit::UNIT);
     _tablet_counter = ADD_COUNTER(_runtime_profile, "TabletNum", TUnit::UNIT);
diff --git a/be/src/vec/exec/scan/new_olap_scan_node.h b/be/src/vec/exec/scan/new_olap_scan_node.h
index dc882aa6a9..e72ed82a4a 100644
--- a/be/src/vec/exec/scan/new_olap_scan_node.h
+++ b/be/src/vec/exec/scan/new_olap_scan_node.h
@@ -19,6 +19,10 @@
 
 #include "vec/exec/scan/vscan_node.h"
 
+namespace doris::pipeline {
+class OlapScanOperator;
+}
+
 namespace doris::vectorized {
 
 class NewOlapScanner;
@@ -26,6 +30,7 @@ class NewOlapScanNode : public VScanNode {
 public:
     NewOlapScanNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs);
     friend class NewOlapScanner;
+    friend class doris::pipeline::OlapScanOperator;
 
     Status prepare(RuntimeState* state) override;
 
diff --git a/be/src/vec/exec/scan/pip_scanner_context.h b/be/src/vec/exec/scan/pip_scanner_context.h
new file mode 100644
index 0000000000..d595396c72
--- /dev/null
+++ b/be/src/vec/exec/scan/pip_scanner_context.h
@@ -0,0 +1,47 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include "scanner_context.h"
+
+namespace doris {
+
+namespace pipeline {
+
+class PipScannerContext : public vectorized::ScannerContext {
+public:
+    PipScannerContext(RuntimeState* state, vectorized::VScanNode* parent,
+                      const TupleDescriptor* input_tuple_desc,
+                      const TupleDescriptor* output_tuple_desc,
+                      const std::list<vectorized::VScanner*>& scanners, int64_t limit,
+                      int64_t max_bytes_in_blocks_queue)
+            : vectorized::ScannerContext(state, parent, input_tuple_desc, output_tuple_desc,
+                                         scanners, limit, max_bytes_in_blocks_queue) {}
+
+    void _update_block_queue_empty() override { _blocks_queue_empty = _blocks_queue.empty(); }
+
+    // We should make those method lock free.
+    bool done() override { return _is_finished || _should_stop || _status_error; }
+    bool can_finish() override { return _num_running_scanners == 0 && _num_scheduling_ctx == 0; }
+    bool empty_in_queue() override { return _blocks_queue_empty; }
+
+private:
+    std::atomic_bool _blocks_queue_empty = true;
+};
+} // namespace pipeline
+} // namespace doris
diff --git a/be/src/vec/exec/scan/scanner_context.cpp b/be/src/vec/exec/scan/scanner_context.cpp
index 7f6cd847bb..203ef8ed68 100644
--- a/be/src/vec/exec/scan/scanner_context.cpp
+++ b/be/src/vec/exec/scan/scanner_context.cpp
@@ -17,6 +17,8 @@
 
 #include "scanner_context.h"
 
+#include <mutex>
+
 #include "common/config.h"
 #include "runtime/runtime_state.h"
 #include "util/threadpool.h"
@@ -96,14 +98,20 @@ void ScannerContext::return_free_block(vectorized::Block* block) {
 
 void ScannerContext::append_blocks_to_queue(const std::vector<vectorized::Block*>& blocks) {
     std::lock_guard<std::mutex> l(_transfer_lock);
-    blocks_queue.insert(blocks_queue.end(), blocks.begin(), blocks.end());
+    _blocks_queue.insert(_blocks_queue.end(), blocks.begin(), blocks.end());
+    _update_block_queue_empty();
     for (auto b : blocks) {
         _cur_bytes_in_queue += b->allocated_bytes();
     }
     _blocks_queue_added_cv.notify_one();
 }
 
-Status ScannerContext::get_block_from_queue(vectorized::Block** block, bool* eos) {
+bool ScannerContext::empty_in_queue() {
+    std::unique_lock<std::mutex> l(_transfer_lock);
+    return _blocks_queue.empty();
+}
+
+Status ScannerContext::get_block_from_queue(vectorized::Block** block, bool* eos, bool wait) {
     std::unique_lock<std::mutex> l(_transfer_lock);
     // Normally, the scanner scheduler will schedule ctx.
     // But when the amount of data in the blocks queue exceeds the upper limit,
@@ -119,7 +127,7 @@ Status ScannerContext::get_block_from_queue(vectorized::Block** block, bool* eos
     {
         SCOPED_TIMER(_parent->_scanner_wait_batch_timer);
         _blocks_queue_added_cv.wait(l, [this]() {
-            return !blocks_queue.empty() || _is_finished || !_process_status.ok() ||
+            return !_blocks_queue.empty() || _is_finished || !_process_status.ok() ||
                    _state->is_cancelled();
         });
     }
@@ -132,9 +140,10 @@ Status ScannerContext::get_block_from_queue(vectorized::Block** block, bool* eos
         return _process_status;
     }
 
-    if (!blocks_queue.empty()) {
-        *block = blocks_queue.front();
-        blocks_queue.pop_front();
+    if (!_blocks_queue.empty()) {
+        *block = _blocks_queue.front();
+        _blocks_queue.pop_front();
+        _update_block_queue_empty();
         _cur_bytes_in_queue -= (*block)->allocated_bytes();
         return Status::OK();
     } else {
@@ -147,6 +156,7 @@ bool ScannerContext::set_status_on_error(const Status& status) {
     std::lock_guard<std::mutex> l(_transfer_lock);
     if (_process_status.ok()) {
         _process_status = status;
+        _status_error = true;
         _blocks_queue_added_cv.notify_one();
         return true;
     }
@@ -183,7 +193,7 @@ void ScannerContext::clear_and_join() {
     COUNTER_SET(_parent->_scanner_sched_counter, _num_scanner_scheduling);
     COUNTER_SET(_parent->_scanner_ctx_sched_counter, _num_ctx_scheduling);
 
-    std::for_each(blocks_queue.begin(), blocks_queue.end(),
+    std::for_each(_blocks_queue.begin(), _blocks_queue.end(),
                   std::default_delete<vectorized::Block>());
     std::for_each(_free_blocks.begin(), _free_blocks.end(),
                   std::default_delete<vectorized::Block>());
@@ -191,13 +201,18 @@ void ScannerContext::clear_and_join() {
     return;
 }
 
+bool ScannerContext::can_finish() {
+    std::unique_lock<std::mutex> l(_transfer_lock);
+    return _num_running_scanners == 0 && _num_scheduling_ctx == 0;
+}
+
 std::string ScannerContext::debug_string() {
     return fmt::format(
             "id: {}, sacnners: {}, blocks in queue: {},"
             " status: {}, _should_stop: {}, _is_finished: {}, free blocks: {},"
             " limit: {}, _num_running_scanners: {}, _num_scheduling_ctx: {}, _max_thread_num: {},"
-            " _block_per_scanner: {}, _cur_bytes_in_queue: {}, _max_bytes_in_queue: {}",
-            ctx_id, _scanners.size(), blocks_queue.size(), _process_status.ok(), _should_stop,
+            " _block_per_scanner: {}, _cur_bytes_in_queue: {}, MAX_BYTE_OF_QUEUE: {}",
+            ctx_id, _scanners.size(), _blocks_queue.size(), _process_status.ok(), _should_stop,
             _is_finished, _free_blocks.size(), limit, _num_running_scanners, _num_scheduling_ctx,
             _max_thread_num, _block_per_scanner, _cur_bytes_in_queue, _max_bytes_in_queue);
 }
diff --git a/be/src/vec/exec/scan/scanner_context.h b/be/src/vec/exec/scan/scanner_context.h
index 24a490673d..41bd0b4d14 100644
--- a/be/src/vec/exec/scan/scanner_context.h
+++ b/be/src/vec/exec/scan/scanner_context.h
@@ -17,7 +17,9 @@
 
 #pragma once
 
+#include <atomic>
 #include <condition_variable>
+#include <mutex>
 
 #include "common/status.h"
 #include "runtime/descriptors.h"
@@ -63,6 +65,8 @@ public:
         }
     }
 
+    virtual ~ScannerContext() = default;
+
     Status init();
 
     vectorized::Block* get_free_block(bool* get_free_block);
@@ -74,7 +78,7 @@ public:
     // Get next block from blocks queue. Called by ScanNode
     // Set eos to true if there is no more data to read.
     // And if eos is true, the block returned must be nullptr.
-    Status get_block_from_queue(vectorized::Block** block, bool* eos);
+    Status get_block_from_queue(vectorized::Block** block, bool* eos, bool wait = true);
 
     // When a scanner complete a scan, this method will be called
     // to return the scanner to the list for next scheduling.
@@ -96,8 +100,8 @@ public:
     }
 
     // Return true if this ScannerContext need no more process
-    bool done() {
-        std::lock_guard<std::mutex> l(_transfer_lock);
+    virtual bool done() {
+        std::unique_lock<std::mutex> l(_transfer_lock);
         return _is_finished || _should_stop || !_process_status.ok();
     }
 
@@ -114,6 +118,8 @@ public:
 
     void clear_and_join();
 
+    virtual bool can_finish();
+
     std::string debug_string();
 
     RuntimeState* state() { return _state; }
@@ -123,7 +129,8 @@ public:
 
     VScanNode* parent() { return _parent; }
 
-public:
+    virtual bool empty_in_queue();
+
     // the unique id of this context
     std::string ctx_id;
     int32_t queue_idx = -1;
@@ -132,11 +139,14 @@ public:
 private:
     Status _close_and_clear_scanners();
 
-    inline bool _has_enough_space_in_blocks_queue() {
+    inline bool _has_enough_space_in_blocks_queue() const {
         return _cur_bytes_in_queue < _max_bytes_in_queue / 2;
     }
 
-private:
+    // do nothing here, we only do update on pip_scanner_context
+    virtual void _update_block_queue_empty() {}
+
+protected:
     RuntimeState* _state;
     VScanNode* _parent;
 
@@ -154,7 +164,7 @@ private:
     // The blocks got from scanners will be added to the "blocks_queue".
     // And the upper scan node will be as a consumer to fetch blocks from this queue.
     // Should be protected by "_transfer_lock"
-    std::list<vectorized::Block*> blocks_queue;
+    std::list<vectorized::Block*> _blocks_queue;
     // Wait in get_block_from_queue(), by ScanNode.
     std::condition_variable _blocks_queue_added_cv;
     // Wait in clear_and_join(), by ScanNode.
@@ -174,8 +184,9 @@ private:
     //      Always be set by ScannerScheduler.
     //      True means all scanners are finished to scan.
     Status _process_status;
-    bool _should_stop = false;
-    bool _is_finished = false;
+    std::atomic_bool _status_error = false;
+    std::atomic_bool _should_stop = false;
+    std::atomic_bool _is_finished = false;
 
     // Pre-allocated blocks for all scanners to share, for memory reuse.
     std::mutex _free_blocks_lock;
@@ -185,13 +196,13 @@ private:
     int64_t limit;
 
     // Current number of running scanners.
-    int32_t _num_running_scanners = 0;
+    std::atomic_int32_t _num_running_scanners = 0;
     // Current number of ctx being scheduled.
     // After each Scanner finishes a task, it will put the corresponding ctx
     // back into the scheduling queue.
     // Therefore, there will be multiple pointer of same ctx in the scheduling queue.
     // Here we record the number of ctx in the scheduling  queue to clean up at the end.
-    int32_t _num_scheduling_ctx = 0;
+    std::atomic_int32_t _num_scheduling_ctx = 0;
     // Num of unfinished scanners. Should be set in init()
     int32_t _num_unfinished_scanners = 0;
     // Max number of scan thread for this scanner context.
diff --git a/be/src/vec/exec/scan/vscan_node.cpp b/be/src/vec/exec/scan/vscan_node.cpp
index b1511c47a3..c83669532c 100644
--- a/be/src/vec/exec/scan/vscan_node.cpp
+++ b/be/src/vec/exec/scan/vscan_node.cpp
@@ -22,6 +22,7 @@
 #include "runtime/runtime_filter_mgr.h"
 #include "util/runtime_profile.h"
 #include "vec/columns/column_const.h"
+#include "vec/exec/scan/pip_scanner_context.h"
 #include "vec/exec/scan/scanner_scheduler.h"
 #include "vec/exec/scan/vscanner.h"
 #include "vec/exprs/vcompound_pred.h"
@@ -65,6 +66,7 @@ Status VScanNode::init(const TPlanNode& tnode, RuntimeState* state) {
     }
 
     RETURN_IF_ERROR(_register_runtime_filter());
+    RETURN_IF_ERROR(_init_profile());
 
     return Status::OK();
 }
@@ -73,8 +75,6 @@ Status VScanNode::prepare(RuntimeState* state) {
     RETURN_IF_ERROR(ExecNode::prepare(state));
     SCOPED_CONSUME_MEM_TRACKER(mem_tracker_growh());
 
-    RETURN_IF_ERROR(_init_profile());
-
     // init profile for runtime filter
     for (auto& rf_ctx : _runtime_filter_ctxs) {
         rf_ctx.runtime_filter->init_profile(_runtime_profile.get());
@@ -175,9 +175,15 @@ Status VScanNode::_init_profile() {
 }
 
 Status VScanNode::_start_scanners(const std::list<VScanner*>& scanners) {
-    _scanner_ctx.reset(new ScannerContext(_state, this, _input_tuple_desc, _output_tuple_desc,
-                                          scanners, limit(),
-                                          _state->query_options().mem_limit / 20));
+    if (_state->enable_pipeline_exec()) {
+        _scanner_ctx.reset(new pipeline::PipScannerContext(_state, this, _input_tuple_desc,
+                                                           _output_tuple_desc, scanners, limit(),
+                                                           _state->query_options().mem_limit / 20));
+    } else {
+        _scanner_ctx.reset(new ScannerContext(_state, this, _input_tuple_desc, _output_tuple_desc,
+                                              scanners, limit(),
+                                              _state->query_options().mem_limit / 20));
+    }
     RETURN_IF_ERROR(_scanner_ctx->init());
     RETURN_IF_ERROR(_state->exec_env()->scanner_scheduler()->submit(_scanner_ctx.get()));
     return Status::OK();
@@ -200,7 +206,7 @@ Status VScanNode::_register_runtime_filter() {
     return Status::OK();
 }
 
-Status VScanNode::_acquire_runtime_filter() {
+Status VScanNode::_acquire_runtime_filter(bool wait) {
     SCOPED_TIMER(_acquire_runtime_filter_timer);
     std::vector<VExpr*> vexprs;
     for (size_t i = 0; i < _runtime_filter_descs.size(); ++i) {
@@ -213,7 +219,7 @@ Status VScanNode::_acquire_runtime_filter() {
             }
         }
         bool ready = runtime_filter->is_ready();
-        if (!ready) {
+        if (!ready && wait) {
             ready = runtime_filter->await();
         }
         if (ready) {
diff --git a/be/src/vec/exec/scan/vscan_node.h b/be/src/vec/exec/scan/vscan_node.h
index fbe800ccd6..7e99cf3cdc 100644
--- a/be/src/vec/exec/scan/vscan_node.h
+++ b/be/src/vec/exec/scan/vscan_node.h
@@ -26,6 +26,10 @@
 #include "vec/exprs/vexpr.h"
 #include "vec/exprs/vin_predicate.h"
 
+namespace doris::pipeline {
+class ScanOperator;
+}
+
 namespace doris::vectorized {
 
 class VScanner;
@@ -49,6 +53,7 @@ public:
     friend class NewOlapScanner;
     friend class VFileScanner;
     friend class ScannerContext;
+    friend class doris::pipeline::ScanOperator;
 
     Status init(const TPlanNode& tnode, RuntimeState* state = nullptr) override;
 
@@ -257,7 +262,7 @@ private:
     // Register and get all runtime filters at Init phase.
     Status _register_runtime_filter();
     // Get all arrived runtime filters at Open phase.
-    Status _acquire_runtime_filter();
+    Status _acquire_runtime_filter(bool wait = true);
     // Append late-arrival runtime filters to the vconjunct_ctx.
     Status _append_rf_into_conjuncts(std::vector<VExpr*>& vexprs);
 
diff --git a/be/src/vec/exec/vaggregation_node.cpp b/be/src/vec/exec/vaggregation_node.cpp
index 0d86bfc546..92ed1f2336 100644
--- a/be/src/vec/exec/vaggregation_node.cpp
+++ b/be/src/vec/exec/vaggregation_node.cpp
@@ -288,12 +288,9 @@ void AggregationNode::_init_hash_method(std::vector<VExprContext*>& probe_exprs)
             _agg_data->init(AggregatedDataVariants::Type::serialized);
         }
     }
-} // namespace doris::vectorized
+}
 
-Status AggregationNode::prepare(RuntimeState* state) {
-    SCOPED_TIMER(_runtime_profile->total_time_counter());
-    RETURN_IF_ERROR(ExecNode::prepare(state));
-    SCOPED_CONSUME_MEM_TRACKER(mem_tracker_growh());
+Status AggregationNode::prepare_profile(RuntimeState* state) {
     _build_timer = ADD_TIMER(runtime_profile(), "BuildTime");
     _serialize_key_timer = ADD_TIMER(runtime_profile(), "SerializeKeyTime");
     _exec_timer = ADD_TIMER(runtime_profile(), "ExecTime");
@@ -445,10 +442,14 @@ Status AggregationNode::prepare(RuntimeState* state) {
     return Status::OK();
 }
 
-Status AggregationNode::open(RuntimeState* state) {
-    START_AND_SCOPE_SPAN(state->get_tracer(), span, "AggregationNode::open");
-    SCOPED_TIMER(_runtime_profile->total_time_counter());
-    RETURN_IF_ERROR(ExecNode::open(state));
+Status AggregationNode::prepare(RuntimeState* state) {
+    RETURN_IF_ERROR(ExecNode::prepare(state));
+    RETURN_IF_ERROR(prepare_profile(state));
+    return Status::OK();
+}
+
+Status AggregationNode::alloc_resource(doris::RuntimeState* state) {
+    RETURN_IF_ERROR(ExecNode::alloc_resource(state));
     SCOPED_CONSUME_MEM_TRACKER(mem_tracker_growh());
 
     RETURN_IF_ERROR(VExpr::open(_probe_expr_ctxs, state));
@@ -457,10 +458,6 @@ Status AggregationNode::open(RuntimeState* state) {
         RETURN_IF_ERROR(_aggregate_evaluators[i]->open(state));
     }
 
-    RETURN_IF_ERROR(_children[0]->open(state));
-
-    // Streaming preaggregations do all processing in GetNext().
-    if (_is_streaming_preagg) return Status::OK();
     // move _create_agg_status to open not in during prepare,
     // because during prepare and open thread is not the same one,
     // this could cause unable to get JVM
@@ -468,6 +465,18 @@ Status AggregationNode::open(RuntimeState* state) {
         _create_agg_status(_agg_data->without_key);
         _agg_data_created_without_key = true;
     }
+
+    return Status::OK();
+}
+
+Status AggregationNode::open(RuntimeState* state) {
+    START_AND_SCOPE_SPAN(state->get_tracer(), span, "AggregationNode::open");
+    SCOPED_TIMER(_runtime_profile->total_time_counter());
+    RETURN_IF_ERROR(ExecNode::open(state));
+    RETURN_IF_ERROR(_children[0]->open(state));
+
+    // Streaming preaggregations do all processing in GetNext().
+    if (_is_streaming_preagg) return Status::OK();
     bool eos = false;
     Block block;
     while (!eos) {
@@ -475,11 +484,7 @@ Status AggregationNode::open(RuntimeState* state) {
         release_block_memory(block);
         RETURN_IF_ERROR_AND_CHECK_SPAN(_children[0]->get_next_after_projects(state, &block, &eos),
                                        _children[0]->get_next_span(), eos);
-        if (block.rows() == 0) {
-            continue;
-        }
-        RETURN_IF_ERROR(_executor.execute(&block));
-        _executor.update_memusage();
+        RETURN_IF_ERROR(sink(state, &block, eos));
     }
     _children[0]->close(state);
 
@@ -490,6 +495,18 @@ Status AggregationNode::get_next(RuntimeState* state, RowBatch* row_batch, bool*
     return Status::NotSupported("Not Implemented Aggregation Node::get_next scalar");
 }
 
+Status AggregationNode::do_pre_agg(vectorized::Block* input_block,
+                                   vectorized::Block* output_block) {
+    RETURN_IF_ERROR(_executor.pre_agg(input_block, output_block));
+
+    // pre stream agg need use _num_row_return to decide whether to do pre stream agg
+    _num_rows_returned += output_block->rows();
+    _make_nullable_output_key(output_block);
+    COUNTER_SET(_rows_returned_counter, _num_rows_returned);
+    _executor.update_memusage();
+    return Status::OK();
+}
+
 Status AggregationNode::get_next(RuntimeState* state, Block* block, bool* eos) {
     INIT_AND_SCOPE_GET_NEXT_SPAN(state->get_tracer(), _get_next_span, "AggregationNode::get_next");
     SCOPED_TIMER(_runtime_profile->total_time_counter());
@@ -505,32 +522,37 @@ Status AggregationNode::get_next(RuntimeState* state, Block* block, bool* eos) {
                     _children[0]->get_next_after_projects(state, &_preagg_block, &child_eos),
                     _children[0]->get_next_span(), child_eos);
         } while (_preagg_block.rows() == 0 && !child_eos);
-
         if (_preagg_block.rows() != 0) {
-            RETURN_IF_ERROR(_executor.pre_agg(&_preagg_block, block));
+            RETURN_IF_ERROR(do_pre_agg(&_preagg_block, block));
         } else {
-            RETURN_IF_ERROR(_executor.get_result(state, block, eos));
+            RETURN_IF_ERROR(pull(state, block, eos));
         }
-        // pre stream agg need use _num_row_return to decide whether to do pre stream agg
-        _num_rows_returned += block->rows();
-        _make_nullable_output_key(block);
-        COUNTER_SET(_rows_returned_counter, _num_rows_returned);
     } else {
-        RETURN_IF_ERROR(_executor.get_result(state, block, eos));
-        _make_nullable_output_key(block);
-        // dispose the having clause, should not be execute in prestreaming agg
-        RETURN_IF_ERROR(VExprContext::filter_block(_vconjunct_ctx_ptr, block, block->columns()));
-        reached_limit(block, eos);
+        RETURN_IF_ERROR(pull(state, block, eos));
     }
+    return Status::OK();
+}
+
+Status AggregationNode::pull(doris::RuntimeState* state, vectorized::Block* block, bool* eos) {
+    RETURN_IF_ERROR(_executor.get_result(state, block, eos));
+    _make_nullable_output_key(block);
+    // dispose the having clause, should not be execute in prestreaming agg
+    RETURN_IF_ERROR(VExprContext::filter_block(_vconjunct_ctx_ptr, block, block->columns()));
+    reached_limit(block, eos);
 
-    _executor.update_memusage();
     return Status::OK();
 }
 
-Status AggregationNode::close(RuntimeState* state) {
-    if (is_closed()) return Status::OK();
-    START_AND_SCOPE_SPAN(state->get_tracer(), span, "AggregationNode::close");
+Status AggregationNode::sink(doris::RuntimeState* state, vectorized::Block* in_block, bool eos) {
+    if (in_block->rows() > 0) {
+        RETURN_IF_ERROR(_executor.execute(in_block));
+        _executor.update_memusage();
+    }
+    if (eos) _can_read = true;
+    return Status::OK();
+}
 
+void AggregationNode::release_resource(RuntimeState* state) {
     for (auto* aggregate_evaluator : _aggregate_evaluators) aggregate_evaluator->close(state);
     VExpr::close(_probe_expr_ctxs, state);
     if (_executor.close) _executor.close();
@@ -544,7 +566,12 @@ Status AggregationNode::close(RuntimeState* state) {
                 _agg_data->_aggregated_method_variant);
     }
     _release_mem();
+    ExecNode::release_resource(state);
+}
 
+Status AggregationNode::close(RuntimeState* state) {
+    if (is_closed()) return Status::OK();
+    START_AND_SCOPE_SPAN(state->get_tracer(), span, "AggregationNode::close");
     return ExecNode::close(state);
 }
 
diff --git a/be/src/vec/exec/vaggregation_node.h b/be/src/vec/exec/vaggregation_node.h
index ccb28902d9..1ef8518fee 100644
--- a/be/src/vec/exec/vaggregation_node.h
+++ b/be/src/vec/exec/vaggregation_node.h
@@ -34,6 +34,13 @@ class TPlanNode;
 class DescriptorTbl;
 class MemPool;
 
+namespace pipeline {
+class AggSinkOperator;
+class AggregationSourceOperator;
+class StreamingAggSinkOperator;
+class StreamingAggSourceOperator;
+} // namespace pipeline
+
 namespace vectorized {
 class VExprContext;
 
@@ -754,20 +761,31 @@ private:
 };
 
 // not support spill
-class AggregationNode : public ::doris::ExecNode {
+class AggregationNode final : public ::doris::ExecNode {
 public:
     using Sizes = std::vector<size_t>;
 
     AggregationNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs);
     ~AggregationNode();
-    virtual Status init(const TPlanNode& tnode, RuntimeState* state = nullptr);
-    virtual Status prepare(RuntimeState* state);
-    virtual Status open(RuntimeState* state);
-    virtual Status get_next(RuntimeState* state, RowBatch* row_batch, bool* eos);
-    virtual Status get_next(RuntimeState* state, Block* block, bool* eos);
-    virtual Status close(RuntimeState* state);
+    virtual Status init(const TPlanNode& tnode, RuntimeState* state = nullptr) override;
+    Status prepare_profile(RuntimeState* state);
+    virtual Status prepare(RuntimeState* state) override;
+    virtual Status open(RuntimeState* state) override;
+    virtual Status alloc_resource(RuntimeState* state) override;
+    virtual Status get_next(RuntimeState* state, RowBatch* row_batch, bool* eos) override;
+    virtual Status get_next(RuntimeState* state, Block* block, bool* eos) override;
+    virtual Status close(RuntimeState* state) override;
+    virtual void release_resource(RuntimeState* state) override;
+    Status pull(doris::RuntimeState* state, vectorized::Block* output_block, bool* eos) override;
+    Status sink(doris::RuntimeState* state, vectorized::Block* input_block, bool eos) override;
+    Status do_pre_agg(vectorized::Block* input_block, vectorized::Block* output_block);
+    bool is_streaming_preagg() { return _is_streaming_preagg; }
 
 private:
+    friend class pipeline::AggSinkOperator;
+    friend class pipeline::StreamingAggSinkOperator;
+    friend class pipeline::AggregationSourceOperator;
+    friend class pipeline::StreamingAggSourceOperator;
     // group by k1,k2
     std::vector<VExprContext*> _probe_expr_ctxs;
     // left / full join will change the key nullable make output/input solt
@@ -832,6 +850,7 @@ private:
     std::unique_ptr<AggregateDataContainer> _aggregate_data_container;
 
 private:
+    void _release_self_resource(RuntimeState* state);
     /// Return true if we should keep expanding hash tables in the preagg. If false,
     /// the preagg should pass through any rows it can't fit in its tables.
     bool _should_expand_preagg_hash_tables();
diff --git a/be/src/vec/exec/vexchange_node.cpp b/be/src/vec/exec/vexchange_node.cpp
index 59073c879a..13a75ed990 100644
--- a/be/src/vec/exec/vexchange_node.cpp
+++ b/be/src/vec/exec/vexchange_node.cpp
@@ -17,6 +17,9 @@
 
 #include "vec/exec/vexchange_node.h"
 
+#include "pipeline/exec/exchange_source_operator.h"
+#include "pipeline/pipeline.h"
+#include "pipeline/pipeline_fragment_context.h"
 #include "runtime/exec_env.h"
 #include "runtime/runtime_state.h"
 #include "runtime/thread_context.h"
@@ -62,18 +65,23 @@ Status VExchangeNode::prepare(RuntimeState* state) {
     }
     return Status::OK();
 }
-Status VExchangeNode::open(RuntimeState* state) {
-    START_AND_SCOPE_SPAN(state->get_tracer(), span, "VExchangeNode::open");
-    SCOPED_TIMER(_runtime_profile->total_time_counter());
-    RETURN_IF_ERROR(ExecNode::open(state));
-    SCOPED_CONSUME_MEM_TRACKER(mem_tracker_growh());
 
+Status VExchangeNode::alloc_resource(RuntimeState* state) {
+    RETURN_IF_ERROR(ExecNode::alloc_resource(state));
     if (_is_merging) {
         RETURN_IF_ERROR(_vsort_exec_exprs.open(state));
         RETURN_IF_ERROR(_stream_recvr->create_merger(_vsort_exec_exprs.lhs_ordering_expr_ctxs(),
                                                      _is_asc_order, _nulls_first,
                                                      state->batch_size(), _limit, _offset));
     }
+    return Status::OK();
+}
+
+Status VExchangeNode::open(RuntimeState* state) {
+    START_AND_SCOPE_SPAN(state->get_tracer(), span, "VExchangeNode::open");
+    SCOPED_TIMER(_runtime_profile->total_time_counter());
+    SCOPED_CONSUME_MEM_TRACKER(mem_tracker_growh());
+    RETURN_IF_ERROR(ExecNode::open(state));
 
     return Status::OK();
 }
@@ -99,17 +107,21 @@ Status VExchangeNode::get_next(RuntimeState* state, Block* block, bool* eos) {
     return status;
 }
 
+void VExchangeNode::release_resource(RuntimeState* state) {
+    if (_stream_recvr != nullptr) {
+        _stream_recvr->close();
+    }
+    if (_is_merging) {
+        _vsort_exec_exprs.close(state);
+    }
+}
+
 Status VExchangeNode::close(RuntimeState* state) {
     if (is_closed()) {
         return Status::OK();
     }
     START_AND_SCOPE_SPAN(state->get_tracer(), span, "VExchangeNode::close");
 
-    if (_stream_recvr != nullptr) {
-        _stream_recvr->close();
-    }
-    if (_is_merging) _vsort_exec_exprs.close(state);
-
     return ExecNode::close(state);
 }
 
diff --git a/be/src/vec/exec/vexchange_node.h b/be/src/vec/exec/vexchange_node.h
index 5e2d4533af..190b625e3d 100644
--- a/be/src/vec/exec/vexchange_node.h
+++ b/be/src/vec/exec/vexchange_node.h
@@ -23,19 +23,25 @@
 #include "vec/common/sort/vsort_exec_exprs.h"
 
 namespace doris {
+namespace pipeline {
+class ExchangeSourceOperator;
+}
 namespace vectorized {
 class VDataStreamRecvr;
 
 class VExchangeNode : public ExecNode {
 public:
+    friend class doris::pipeline::ExchangeSourceOperator;
     VExchangeNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs);
     virtual ~VExchangeNode() {}
 
     virtual Status init(const TPlanNode& tnode, RuntimeState* state = nullptr) override;
     virtual Status prepare(RuntimeState* state) override;
+    virtual Status alloc_resource(RuntimeState* state) override;
     virtual Status open(RuntimeState* state) override;
     virtual Status get_next(RuntimeState* state, RowBatch* row_batch, bool* eos) override;
     virtual Status get_next(RuntimeState* state, Block* row_batch, bool* eos) override;
+    virtual void release_resource(RuntimeState* state) override;
     virtual Status close(RuntimeState* state) override;
 
     // Status collect_query_statistics(QueryStatistics* statistics) override;
diff --git a/be/src/vec/exec/vrepeat_node.cpp b/be/src/vec/exec/vrepeat_node.cpp
index cb79fcb317..9c77b21bfc 100644
--- a/be/src/vec/exec/vrepeat_node.cpp
+++ b/be/src/vec/exec/vrepeat_node.cpp
@@ -24,7 +24,14 @@
 
 namespace doris::vectorized {
 VRepeatNode::VRepeatNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs)
-        : RepeatNode(pool, tnode, descs) {}
+        : ExecNode(pool, tnode, descs),
+          _slot_id_set_list(tnode.repeat_node.slot_id_set_list),
+          _all_slot_ids(tnode.repeat_node.all_slot_ids),
+          _repeat_id_list(tnode.repeat_node.repeat_id_list),
+          _grouping_list(tnode.repeat_node.grouping_list),
+          _output_tuple_id(tnode.repeat_node.output_tuple_id),
+          _child_eos(false),
+          _repeat_id_idx(0) {}
 
 Status VRepeatNode::init(const TPlanNode& tnode, RuntimeState* state) {
     RETURN_IF_ERROR(ExecNode::init(tnode, state));
@@ -35,7 +42,14 @@ Status VRepeatNode::init(const TPlanNode& tnode, RuntimeState* state) {
 Status VRepeatNode::prepare(RuntimeState* state) {
     VLOG_CRITICAL << "VRepeatNode::prepare";
     SCOPED_TIMER(_runtime_profile->total_time_counter());
-    RETURN_IF_ERROR(RepeatNode::prepare(state));
+
+    RETURN_IF_ERROR(ExecNode::prepare(state));
+    SCOPED_CONSUME_MEM_TRACKER(mem_tracker_growh());
+    _output_tuple_desc = state->desc_tbl().get_tuple_descriptor(_output_tuple_id);
+    if (_output_tuple_desc == nullptr) {
+        return Status::InternalError("Failed to get tuple descriptor.");
+    }
+
     RETURN_IF_ERROR(VExpr::prepare(_expr_ctxs, state, child(0)->row_desc()));
 
     for (const auto& slot_desc : _output_tuple_desc->slots()) {
@@ -50,7 +64,16 @@ Status VRepeatNode::open(RuntimeState* state) {
     START_AND_SCOPE_SPAN(state->get_tracer(), span, "VRepeatNode::open");
     VLOG_CRITICAL << "VRepeatNode::open";
     SCOPED_TIMER(_runtime_profile->total_time_counter());
-    RETURN_IF_ERROR(RepeatNode::open(state));
+    RETURN_IF_ERROR(ExecNode::open(state));
+    RETURN_IF_ERROR(child(0)->open(state));
+    return Status::OK();
+}
+
+Status VRepeatNode::alloc_resource(RuntimeState* state) {
+    START_AND_SCOPE_SPAN(state->get_tracer(), span, "VRepeatNode::open");
+    SCOPED_TIMER(_runtime_profile->total_time_counter());
+    RETURN_IF_ERROR(ExecNode::alloc_resource(state));
+    SCOPED_CONSUME_MEM_TRACKER(mem_tracker_growh());
     RETURN_IF_ERROR(VExpr::open(_expr_ctxs, state));
     return Status::OK();
 }
@@ -148,6 +171,58 @@ Status VRepeatNode::get_repeated_block(Block* child_block, int repeat_id_idx, Bl
     return Status::OK();
 }
 
+Status VRepeatNode::pull(doris::RuntimeState* state, vectorized::Block* output_block, bool* eos) {
+    SCOPED_TIMER(_runtime_profile->total_time_counter());
+
+    RETURN_IF_CANCELLED(state);
+    DCHECK(_repeat_id_idx >= 0);
+    for (const std::vector<int64_t>& v : _grouping_list) {
+        DCHECK(_repeat_id_idx <= (int)v.size());
+    }
+    DCHECK(output_block->rows() == 0);
+
+    DCHECK(_intermediate_block);
+    DCHECK_NE(_intermediate_block->rows(), 0);
+
+    RETURN_IF_ERROR(get_repeated_block(_intermediate_block.get(), _repeat_id_idx, output_block));
+
+    _repeat_id_idx++;
+
+    int size = _repeat_id_list.size();
+    if (_repeat_id_idx >= size) {
+        _intermediate_block->clear();
+        release_block_memory(*_child_block);
+        _repeat_id_idx = 0;
+    }
+
+    reached_limit(output_block, eos);
+    COUNTER_SET(_rows_returned_counter, _num_rows_returned);
+    return Status::OK();
+}
+
+Status VRepeatNode::push(RuntimeState* state, vectorized::Block* input_block, bool eos) {
+    DCHECK(!_intermediate_block || _intermediate_block->rows() == 0);
+    DCHECK(!_expr_ctxs.empty());
+    _intermediate_block.reset(new Block());
+
+    for (auto expr : _expr_ctxs) {
+        int result_column_id = -1;
+        RETURN_IF_ERROR(expr->execute(input_block, &result_column_id));
+        DCHECK(result_column_id != -1);
+        input_block->get_by_position(result_column_id).column =
+                input_block->get_by_position(result_column_id)
+                        .column->convert_to_full_column_if_const();
+        _intermediate_block->insert(input_block->get_by_position(result_column_id));
+    }
+    DCHECK_EQ(_expr_ctxs.size(), _intermediate_block->columns());
+
+    return Status::OK();
+}
+
+bool VRepeatNode::need_more_input_data() {
+    return !_intermediate_block || _intermediate_block->rows() == 0;
+}
+
 Status VRepeatNode::get_next(RuntimeState* state, Block* block, bool* eos) {
     INIT_AND_SCOPE_GET_NEXT_SPAN(state->get_tracer(), _get_next_span, "VRepeatNode::get_next");
     VLOG_CRITICAL << "VRepeatNode::get_next";
@@ -163,8 +238,7 @@ Status VRepeatNode::get_next(RuntimeState* state, Block* block, bool* eos) {
     }
     DCHECK(block->rows() == 0);
 
-    // current child block has finished its repeat, get child's next block
-    if (_child_block->rows() == 0) {
+    if (need_more_input_data()) {
         while (_child_block->rows() == 0 && !_child_eos) {
             RETURN_IF_ERROR_AND_CHECK_SPAN(
                     child(0)->get_next_after_projects(state, _child_block.get(), &_child_eos),
@@ -176,35 +250,10 @@ Status VRepeatNode::get_next(RuntimeState* state, Block* block, bool* eos) {
             return Status::OK();
         }
 
-        DCHECK(!_expr_ctxs.empty());
-        _intermediate_block.reset(new Block());
-        for (auto vexpr_ctx : _expr_ctxs) {
-            int result_column_id = -1;
-            RETURN_IF_ERROR(vexpr_ctx->execute(_child_block.get(), &result_column_id));
-            DCHECK(result_column_id != -1);
-            _child_block->get_by_position(result_column_id).column =
-                    _child_block->get_by_position(result_column_id)
-                            .column->convert_to_full_column_if_const();
-            _intermediate_block->insert(_child_block->get_by_position(result_column_id));
-        }
-        DCHECK_EQ(_expr_ctxs.size(), _intermediate_block->columns());
-    }
-
-    RETURN_IF_ERROR(get_repeated_block(_intermediate_block.get(), _repeat_id_idx, block));
-
-    _repeat_id_idx++;
-
-    int size = _repeat_id_list.size();
-    if (_repeat_id_idx >= size) {
-        _intermediate_block->clear();
-        release_block_memory(*_child_block);
-        _repeat_id_idx = 0;
+        push(state, _child_block.get(), *eos);
     }
 
-    reached_limit(block, eos);
-    COUNTER_SET(_rows_returned_counter, _num_rows_returned);
-    VLOG_ROW << "VRepeatNode output rows: " << block->rows();
-    return Status::OK();
+    return pull(state, block, eos);
 }
 
 Status VRepeatNode::close(RuntimeState* state) {
@@ -212,12 +261,15 @@ Status VRepeatNode::close(RuntimeState* state) {
     if (is_closed()) {
         return Status::OK();
     }
-    START_AND_SCOPE_SPAN(state->get_tracer(), span, "VRepeatNode::close");
-    VExpr::close(_expr_ctxs, state);
-    _release_mem();
     return ExecNode::close(state);
 }
 
+void VRepeatNode::release_resource(RuntimeState* state) {
+    START_AND_SCOPE_SPAN(state->get_tracer(), span, "VSortNode::close");
+    VExpr::close(_expr_ctxs, state);
+    ExecNode::release_resource(state);
+}
+
 void VRepeatNode::debug_string(int indentation_level, std::stringstream* out) const {
     *out << string(indentation_level * 2, ' ');
     *out << "VRepeatNode(";
diff --git a/be/src/vec/exec/vrepeat_node.h b/be/src/vec/exec/vrepeat_node.h
index 1bf047a196..53eb025cc1 100644
--- a/be/src/vec/exec/vrepeat_node.h
+++ b/be/src/vec/exec/vrepeat_node.h
@@ -17,7 +17,7 @@
 
 #pragma once
 
-#include "exec/repeat_node.h"
+#include "exec/exec_node.h"
 
 namespace doris {
 
@@ -30,32 +30,49 @@ class Status;
 namespace vectorized {
 class VExprContext;
 
-class VRepeatNode : public RepeatNode {
+class VRepeatNode : public ExecNode {
 public:
     VRepeatNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs);
     ~VRepeatNode() override = default;
 
-    virtual Status init(const TPlanNode& tnode, RuntimeState* state = nullptr) override;
-    virtual Status prepare(RuntimeState* state) override;
-    virtual Status open(RuntimeState* state) override;
-    virtual Status get_next(RuntimeState* state, Block* block, bool* eos) override;
-    virtual Status close(RuntimeState* state) override;
+    Status init(const TPlanNode& tnode, RuntimeState* state = nullptr) override;
+    Status prepare(RuntimeState* state) override;
+    Status alloc_resource(RuntimeState* state) override;
+    Status open(RuntimeState* state) override;
+    void release_resource(RuntimeState* state) override;
+    Status get_next(RuntimeState* state, Block* block, bool* eos) override;
+    Status close(RuntimeState* state) override;
+
+    Status pull(RuntimeState* state, vectorized::Block* output_block, bool* eos) override;
+    Status push(RuntimeState* state, vectorized::Block* input_block, bool eos) override;
+    bool need_more_input_data();
 
 protected:
     virtual void debug_string(int indentation_level, std::stringstream* out) const override;
 
 private:
-    using RepeatNode::get_next;
     Status get_repeated_block(Block* child_block, int repeat_id_idx, Block* output_block);
 
     void _release_mem();
 
+    // Slot id set used to indicate those slots need to set to null.
+    std::vector<std::set<SlotId>> _slot_id_set_list;
+    // all slot id
+    std::set<SlotId> _all_slot_ids;
+    // An integer bitmap list, it indicates the bit position of the exprs not null.
+    std::vector<int64_t> _repeat_id_list;
+    std::vector<std::vector<int64_t>> _grouping_list;
+    TupleId _output_tuple_id;
+    const TupleDescriptor* _output_tuple_desc;
+
     std::unique_ptr<Block> _child_block {};
     std::unique_ptr<Block> _intermediate_block {};
 
     std::vector<SlotDescriptor*> _output_slots;
 
     std::vector<VExprContext*> _expr_ctxs;
+    bool _child_eos;
+    int _repeat_id_idx;
 };
 } // namespace vectorized
 } // namespace doris
diff --git a/be/src/vec/exec/vsort_node.cpp b/be/src/vec/exec/vsort_node.cpp
index ed37e5cc24..c633a08113 100644
--- a/be/src/vec/exec/vsort_node.cpp
+++ b/be/src/vec/exec/vsort_node.cpp
@@ -18,14 +18,13 @@
 #include "vec/exec/vsort_node.h"
 
 #include "common/config.h"
-#include "exec/sort_exec_exprs.h"
+#include "pipeline/pipeline.h"
 #include "runtime/row_batch.h"
 #include "runtime/runtime_state.h"
 #include "util/debug_util.h"
 #include "vec/common/sort/heap_sorter.h"
 #include "vec/common/sort/topn_sorter.h"
 #include "vec/core/sort_block.h"
-#include "vec/utils/util.hpp"
 
 namespace doris::vectorized {
 
@@ -66,21 +65,41 @@ Status VSortNode::init(const TPlanNode& tnode, RuntimeState* state) {
 
 Status VSortNode::prepare(RuntimeState* state) {
     SCOPED_TIMER(_runtime_profile->total_time_counter());
-    _runtime_profile->add_info_string("TOP-N", _limit == -1 ? "false" : "true");
     RETURN_IF_ERROR(ExecNode::prepare(state));
     SCOPED_CONSUME_MEM_TRACKER(mem_tracker_growh());
+    _runtime_profile->add_info_string("TOP-N", _limit == -1 ? "false" : "true");
     RETURN_IF_ERROR(_vsort_exec_exprs.prepare(state, child(0)->row_desc(), _row_descriptor));
     return Status::OK();
 }
 
-Status VSortNode::open(RuntimeState* state) {
-    START_AND_SCOPE_SPAN(state->get_tracer(), span, "VSortNode::open");
-    SCOPED_TIMER(_runtime_profile->total_time_counter());
-    RETURN_IF_ERROR(ExecNode::open(state));
+Status VSortNode::alloc_resource(doris::RuntimeState* state) {
+    RETURN_IF_ERROR(ExecNode::alloc_resource(state));
     SCOPED_CONSUME_MEM_TRACKER(mem_tracker_growh());
     RETURN_IF_ERROR(_vsort_exec_exprs.open(state));
     RETURN_IF_CANCELLED(state);
     RETURN_IF_ERROR(state->check_query_state("vsort, while open."));
+
+    return Status::OK();
+}
+
+Status VSortNode::sink(RuntimeState* state, vectorized::Block* input_block, bool eos) {
+    if (input_block->rows() > 0) {
+        RETURN_IF_ERROR(_sorter->append_block(input_block));
+        RETURN_IF_CANCELLED(state);
+        RETURN_IF_ERROR(state->check_query_state("vsort, while sorting input."));
+    }
+
+    if (eos) {
+        RETURN_IF_ERROR(_sorter->prepare_for_read());
+        _can_read = true;
+    }
+    return Status::OK();
+}
+
+Status VSortNode::open(RuntimeState* state) {
+    START_AND_SCOPE_SPAN(state->get_tracer(), span, "VSortNode::open");
+    SCOPED_TIMER(_runtime_profile->total_time_counter());
+    RETURN_IF_ERROR(ExecNode::open(state));
     RETURN_IF_ERROR(child(0)->open(state));
 
     // The child has been opened and the sorter created. Sort the input.
@@ -91,17 +110,13 @@ Status VSortNode::open(RuntimeState* state) {
         RETURN_IF_ERROR_AND_CHECK_SPAN(
                 child(0)->get_next_after_projects(state, upstream_block.get(), &eos),
                 child(0)->get_next_span(), eos);
-        if (upstream_block->rows() != 0) {
-            RETURN_IF_ERROR(_sorter->append_block(upstream_block.get()));
-            RETURN_IF_CANCELLED(state);
-            RETURN_IF_ERROR(state->check_query_state("vsort, while sorting input."));
-            if (!_reuse_mem) {
-                upstream_block.reset(new Block());
-            }
+        RETURN_IF_ERROR(sink(state, upstream_block.get(), eos));
+        if (!_reuse_mem) {
+            upstream_block.reset(new Block());
         }
     } while (!eos);
+
     child(0)->close(state);
-    RETURN_IF_ERROR(_sorter->prepare_for_read());
     return Status::OK();
 }
 
@@ -110,27 +125,35 @@ Status VSortNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* eos)
     return Status::NotSupported("Not Implemented VSortNode::get_next scalar");
 }
 
+Status VSortNode::pull(doris::RuntimeState* state, vectorized::Block* output_block, bool* eos) {
+    RETURN_IF_ERROR(_sorter->get_next(state, output_block, eos));
+    reached_limit(output_block, eos);
+    return Status::OK();
+}
+
 Status VSortNode::get_next(RuntimeState* state, Block* block, bool* eos) {
     INIT_AND_SCOPE_GET_NEXT_SPAN(state->get_tracer(), _get_next_span, "VSortNode::get_next");
     SCOPED_TIMER(_runtime_profile->total_time_counter());
     SCOPED_CONSUME_MEM_TRACKER(mem_tracker_growh());
 
-    RETURN_IF_ERROR(_sorter->get_next(state, block, eos));
-    reached_limit(block, eos);
-    return Status::OK();
+    return pull(state, block, eos);
 }
 
 Status VSortNode::reset(RuntimeState* state) {
     return Status::OK();
 }
 
+void VSortNode::release_resource(doris::RuntimeState* state) {
+    START_AND_SCOPE_SPAN(state->get_tracer(), span, "VSortNode::close");
+    _vsort_exec_exprs.close(state);
+    _sorter = nullptr;
+    ExecNode::release_resource(state);
+}
+
 Status VSortNode::close(RuntimeState* state) {
     if (is_closed()) {
         return Status::OK();
     }
-    START_AND_SCOPE_SPAN(state->get_tracer(), span, "VSortNode::close");
-    _vsort_exec_exprs.close(state);
-    _sorter = nullptr;
     return ExecNode::close(state);
 }
 
diff --git a/be/src/vec/exec/vsort_node.h b/be/src/vec/exec/vsort_node.h
index 7b2834e4b4..6da8e52e77 100644
--- a/be/src/vec/exec/vsort_node.h
+++ b/be/src/vec/exec/vsort_node.h
@@ -31,7 +31,7 @@ namespace doris::vectorized {
 // In get_next(), VSortNode do the merge sort to gather data to a new block
 
 // support spill to disk in the future
-class VSortNode : public doris::ExecNode {
+class VSortNode final : public doris::ExecNode {
 public:
     VSortNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs);
 
@@ -41,6 +41,8 @@ public:
 
     virtual Status prepare(RuntimeState* state) override;
 
+    virtual Status alloc_resource(RuntimeState* state) override;
+
     virtual Status open(RuntimeState* state) override;
 
     virtual Status get_next(RuntimeState* state, RowBatch* row_batch, bool* eos) override;
@@ -51,6 +53,12 @@ public:
 
     virtual Status close(RuntimeState* state) override;
 
+    virtual void release_resource(RuntimeState* state) override;
+
+    virtual Status pull(RuntimeState* state, vectorized::Block* output_block, bool* eos) override;
+
+    virtual Status sink(RuntimeState* state, vectorized::Block* input_block, bool eos) override;
+
 protected:
     virtual void debug_string(int indentation_level, std::stringstream* out) const override;
 
diff --git a/be/src/vec/exec/vtable_function_node.cpp b/be/src/vec/exec/vtable_function_node.cpp
index a741bd3553..b283988589 100644
--- a/be/src/vec/exec/vtable_function_node.cpp
+++ b/be/src/vec/exec/vtable_function_node.cpp
@@ -83,11 +83,22 @@ Status VTableFunctionNode::get_next(RuntimeState* state, Block* block, bool* eos
 
     RETURN_IF_CANCELLED(state);
 
-    RETURN_IF_ERROR(get_expanded_block(state, block, eos));
+    // if child_block is empty, get data from child.
+    if (need_more_input_data()) {
+        while (_child_block->rows() == 0 && !_child_eos) {
+            RETURN_IF_ERROR_AND_CHECK_SPAN(
+                    child(0)->get_next_after_projects(state, _child_block.get(), &_child_eos),
+                    child(0)->get_next_span(), _child_eos);
+        }
+        if (_child_eos && _child_block->rows() == 0) {
+            *eos = true;
+            return Status::OK();
+        }
 
-    reached_limit(block, eos);
+        push(state, _child_block.get(), *eos);
+    }
 
-    COUNTER_SET(_rows_returned_counter, _num_rows_returned);
+    pull(state, block, eos);
 
     return Status::OK();
 }
@@ -111,23 +122,8 @@ Status VTableFunctionNode::get_expanded_block(RuntimeState* state, Block* output
         RETURN_IF_CANCELLED(state);
         RETURN_IF_ERROR(state->check_query_state("VTableFunctionNode, while getting next batch."));
 
-        // if child_block is empty, get data from child.
         if (_child_block->rows() == 0) {
-            while (_child_block->rows() == 0 && !_child_eos) {
-                RETURN_IF_ERROR_AND_CHECK_SPAN(
-                        child(0)->get_next_after_projects(state, _child_block.get(), &_child_eos),
-                        child(0)->get_next_span(), _child_eos);
-            }
-            if (_child_eos && _child_block->rows() == 0) {
-                *eos = true;
-                break;
-            }
-
-            for (TableFunction* fn : _fns) {
-                RETURN_IF_ERROR(fn->process_init(_child_block.get()));
-            }
-
-            RETURN_IF_ERROR(_process_next_child_row());
+            break;
         }
 
         bool skip_child_row = false;
diff --git a/be/src/vec/exec/vtable_function_node.h b/be/src/vec/exec/vtable_function_node.h
index 8e572f55e3..0f44e78e5c 100644
--- a/be/src/vec/exec/vtable_function_node.h
+++ b/be/src/vec/exec/vtable_function_node.h
@@ -18,6 +18,7 @@
 #pragma once
 
 #include "exec/table_function_node.h"
+#include "exprs/table_function/table_function.h"
 
 namespace doris::vectorized {
 
@@ -30,6 +31,29 @@ public:
     Status prepare(RuntimeState* state) override;
     Status get_next(RuntimeState* state, Block* block, bool* eos) override;
 
+    bool need_more_input_data() { return !_child_block || !_child_block->rows(); }
+
+    Status push(RuntimeState*, vectorized::Block* input_block, bool eos) override {
+        if (eos) {
+            return Status::OK();
+        }
+
+        if (input_block != _child_block.get()) {
+            _child_block.reset(input_block);
+        }
+        for (TableFunction* fn : _fns) {
+            RETURN_IF_ERROR(fn->process_init(_child_block.get()));
+        }
+        RETURN_IF_ERROR(_process_next_child_row());
+        return Status::OK();
+    }
+
+    Status pull(RuntimeState* state, vectorized::Block* output_block, bool* eos) override {
+        RETURN_IF_ERROR(get_expanded_block(state, output_block, eos));
+        reached_limit(output_block, eos);
+        return Status::OK();
+    }
+
 private:
     Status _process_next_child_row() override;
 
diff --git a/be/src/vec/runtime/vdata_stream_mgr.cpp b/be/src/vec/runtime/vdata_stream_mgr.cpp
index 4743d0933f..ea11f6ed00 100644
--- a/be/src/vec/runtime/vdata_stream_mgr.cpp
+++ b/be/src/vec/runtime/vdata_stream_mgr.cpp
@@ -49,7 +49,7 @@ std::shared_ptr<VDataStreamRecvr> VDataStreamMgr::create_recvr(
         RuntimeState* state, const RowDescriptor& row_desc, const TUniqueId& fragment_instance_id,
         PlanNodeId dest_node_id, int num_senders, int buffer_size, RuntimeProfile* profile,
         bool is_merging, std::shared_ptr<QueryStatisticsRecvr> sub_plan_query_statistics_recvr) {
-    DCHECK(profile != NULL);
+    DCHECK(profile != nullptr);
     VLOG_FILE << "creating receiver for fragment=" << fragment_instance_id
               << ", node=" << dest_node_id;
     std::shared_ptr<VDataStreamRecvr> recvr(new VDataStreamRecvr(
@@ -171,7 +171,7 @@ void VDataStreamMgr::cancel(const TUniqueId& fragment_instance_id) {
                 _fragment_stream_set.lower_bound(std::make_pair(fragment_instance_id, 0));
         while (i != _fragment_stream_set.end() && i->first == fragment_instance_id) {
             std::shared_ptr<VDataStreamRecvr> recvr = find_recvr(i->first, i->second, false);
-            if (recvr == NULL) {
+            if (recvr == nullptr) {
                 // keep going but at least log it
                 std::stringstream err;
                 err << "cancel(): missing in stream_map: fragment=" << i->first
diff --git a/be/src/vec/runtime/vdata_stream_recvr.cpp b/be/src/vec/runtime/vdata_stream_recvr.cpp
index a9c741969a..ec9814a735 100644
--- a/be/src/vec/runtime/vdata_stream_recvr.cpp
+++ b/be/src/vec/runtime/vdata_stream_recvr.cpp
@@ -38,6 +38,12 @@ VDataStreamRecvr::SenderQueue::SenderQueue(VDataStreamRecvr* parent_recvr, int n
 
 VDataStreamRecvr::SenderQueue::~SenderQueue() = default;
 
+bool VDataStreamRecvr::SenderQueue::should_wait() {
+    DCHECK(false) << "VDataStreamRecvr::SenderQueue::should_wait execute";
+    std::unique_lock<std::mutex> l(_lock);
+    return !_is_cancelled && _block_queue.empty() && _num_remaining_senders > 0;
+}
+
 Status VDataStreamRecvr::SenderQueue::get_batch(Block** next_block) {
     std::unique_lock<std::mutex> l(_lock);
     // wait until something shows up or we know we're done
@@ -45,13 +51,16 @@ Status VDataStreamRecvr::SenderQueue::get_batch(Block** next_block) {
         VLOG_ROW << "wait arrival fragment_instance_id=" << _recvr->fragment_instance_id()
                  << " node=" << _recvr->dest_node_id();
         // Don't count time spent waiting on the sender as active time.
-        CANCEL_SAFE_SCOPED_TIMER(_recvr->_data_arrival_timer, &_is_cancelled);
-        CANCEL_SAFE_SCOPED_TIMER(
-                _received_first_batch ? NULL : _recvr->_first_batch_wait_total_timer,
+        CANCEL_SAFE_SCOPED_TIMER_ATOMIC(_recvr->_data_arrival_timer, &_is_cancelled);
+        CANCEL_SAFE_SCOPED_TIMER_ATOMIC(
+                _received_first_batch ? nullptr : _recvr->_first_batch_wait_total_timer,
                 &_is_cancelled);
         _data_arrival_cv.wait(l);
     }
+    return _inner_get_batch(next_block);
+}
 
+Status VDataStreamRecvr::SenderQueue::_inner_get_batch(Block** next_block) {
     // _cur_batch must be replaced with the returned batch.
     _current_block.reset();
     *next_block = nullptr;
@@ -71,6 +80,7 @@ Status VDataStreamRecvr::SenderQueue::get_batch(Block** next_block) {
     _recvr->_num_buffered_bytes -= _block_queue.front().first;
     VLOG_ROW << "fetched #rows=" << result->rows();
     _block_queue.pop_front();
+    _update_block_queue_empty();
 
     _current_block.reset(result);
     *next_block = _current_block.get();
@@ -131,6 +141,7 @@ void VDataStreamRecvr::SenderQueue::add_block(const PBlock& pblock, int be_numbe
 
     VLOG_ROW << "added #rows=" << block->rows() << " batch_size=" << block_byte_size << "\n";
     _block_queue.emplace_back(block_byte_size, block);
+    _update_block_queue_empty();
     // if done is nullptr, this function can't delay this response
     if (done != nullptr && _recvr->exceeds_limit(block_byte_size)) {
         MonotonicStopWatch monotonicStopWatch;
@@ -148,7 +159,7 @@ void VDataStreamRecvr::SenderQueue::add_block(Block* block, bool use_move) {
     // limit memory via DataStreamRecvr::exceeds_limit.
     STOP_CHECK_THREAD_MEM_TRACKER_LIMIT();
     std::unique_lock<std::mutex> l(_lock);
-    if (_is_cancelled) {
+    if (_is_cancelled || !block->rows()) {
         return;
     }
     Block* nblock = new Block(block->get_columns_with_type_and_name());
@@ -167,6 +178,7 @@ void VDataStreamRecvr::SenderQueue::add_block(Block* block, bool use_move) {
 
     size_t block_size = nblock->bytes();
     _block_queue.emplace_back(block_size, nblock);
+    _update_block_queue_empty();
     _data_arrival_cv.notify_one();
 
     if (_recvr->exceeds_limit(block_size)) {
@@ -267,7 +279,8 @@ VDataStreamRecvr::VDataStreamRecvr(
           _is_closed(false),
           _num_buffered_bytes(0),
           _profile(profile),
-          _sub_plan_query_statistics_recvr(sub_plan_query_statistics_recvr) {
+          _sub_plan_query_statistics_recvr(sub_plan_query_statistics_recvr),
+          _enable_pipeline(state->enable_pipeline_exec()) {
     // DataStreamRecvr may be destructed after the instance execution thread ends.
     _mem_tracker = std::make_unique<MemTracker>(
             "VDataStreamRecvr:" + print_id(_fragment_instance_id), _profile);
@@ -278,8 +291,12 @@ VDataStreamRecvr::VDataStreamRecvr(
     _sender_queues.reserve(num_queues);
     int num_sender_per_queue = is_merging ? 1 : num_senders;
     for (int i = 0; i < num_queues; ++i) {
-        SenderQueue* queue =
-                _sender_queue_pool.add(new SenderQueue(this, num_sender_per_queue, profile));
+        SenderQueue* queue = nullptr;
+        if (_enable_pipeline) {
+            queue = _sender_queue_pool.add(new PipSenderQueue(this, num_sender_per_queue, profile));
+        } else {
+            queue = _sender_queue_pool.add(new SenderQueue(this, num_sender_per_queue, profile));
+        }
         _sender_queues.push_back(queue);
     }
 
@@ -332,6 +349,15 @@ void VDataStreamRecvr::add_block(Block* block, int sender_id, bool use_move) {
     _sender_queues[use_sender_id]->add_block(block, use_move);
 }
 
+bool VDataStreamRecvr::ready_to_read() {
+    for (size_t i = 0; i < _sender_queues.size(); i++) {
+        if (_sender_queues[i]->should_wait()) {
+            return false;
+        }
+    }
+    return true;
+}
+
 Status VDataStreamRecvr::get_next(Block* block, bool* eos) {
     SCOPED_CONSUME_MEM_TRACKER(_mem_tracker.get());
     if (!_is_merging) {
diff --git a/be/src/vec/runtime/vdata_stream_recvr.h b/be/src/vec/runtime/vdata_stream_recvr.h
index 6d38ecb955..45af88cd50 100644
--- a/be/src/vec/runtime/vdata_stream_recvr.h
+++ b/be/src/vec/runtime/vdata_stream_recvr.h
@@ -28,9 +28,9 @@
 #include "common/status.h"
 #include "gen_cpp/Types_types.h"
 #include "runtime/descriptors.h"
-#include "runtime/query_fragments_ctx.h"
 #include "runtime/query_statistics.h"
 #include "util/runtime_profile.h"
+#include "vec/core/materialize_block.h"
 
 namespace google {
 namespace protobuf {
@@ -57,7 +57,7 @@ public:
                      RuntimeProfile* profile,
                      std::shared_ptr<QueryStatisticsRecvr> sub_plan_query_statistics_recvr);
 
-    ~VDataStreamRecvr();
+    virtual ~VDataStreamRecvr();
 
     Status create_merger(const std::vector<VExprContext*>& ordering_expr,
                          const std::vector<bool>& is_asc_order,
@@ -69,6 +69,8 @@ public:
 
     void add_block(Block* block, int sender_id, bool use_move);
 
+    bool ready_to_read();
+
     Status get_next(Block* block, bool* eos);
 
     const TUniqueId& fragment_instance_id() const { return _fragment_instance_id; }
@@ -87,14 +89,15 @@ public:
 
     void close();
 
-private:
-    class SenderQueue;
-    friend struct ReceiveQueueSortCursorImpl;
-
     bool exceeds_limit(int batch_size) {
         return _num_buffered_bytes + batch_size > _total_buffer_limit;
     }
 
+private:
+    class SenderQueue;
+    class PipSenderQueue;
+    friend struct ReceiveQueueSortCursorImpl;
+
     // DataStreamMgr instance used to create this recvr. (Not owned)
     VDataStreamMgr* _mgr;
 
@@ -137,11 +140,13 @@ private:
     RuntimeProfile::Counter* _decompress_bytes;
 
     std::shared_ptr<QueryStatisticsRecvr> _sub_plan_query_statistics_recvr;
+
+    bool _enable_pipeline;
 };
 
 class ThreadClosure : public google::protobuf::Closure {
 public:
-    void Run() { _cv.notify_one(); }
+    void Run() override { _cv.notify_one(); }
     void wait(std::unique_lock<std::mutex>& lock) { _cv.wait(lock); }
 
 private:
@@ -152,14 +157,16 @@ class VDataStreamRecvr::SenderQueue {
 public:
     SenderQueue(VDataStreamRecvr* parent_recvr, int num_senders, RuntimeProfile* profile);
 
-    ~SenderQueue();
+    virtual ~SenderQueue();
+
+    virtual bool should_wait();
 
-    Status get_batch(Block** next_block);
+    virtual Status get_batch(Block** next_block);
 
     void add_block(const PBlock& pblock, int be_number, int64_t packet_seq,
                    ::google::protobuf::Closure** done);
 
-    void add_block(Block* block, bool use_move);
+    virtual void add_block(Block* block, bool use_move);
 
     void decrement_senders(int sender_id);
 
@@ -169,17 +176,22 @@ public:
 
     Block* current_block() const { return _current_block.get(); }
 
-private:
+protected:
+    virtual void _update_block_queue_empty() {}
+    Status _inner_get_batch(Block** next_block);
+
     VDataStreamRecvr* _recvr;
     std::mutex _lock;
-    bool _is_cancelled;
-    int _num_remaining_senders;
+    std::atomic_bool _is_cancelled;
+    std::atomic_int _num_remaining_senders;
     std::condition_variable _data_arrival_cv;
     std::condition_variable _data_removal_cv;
 
     using VecBlockQueue = std::list<std::pair<int, Block*>>;
     VecBlockQueue _block_queue;
 
+    std::atomic_bool _block_queue_empty = true;
+
     std::unique_ptr<Block> _current_block;
 
     bool _received_first_batch;
@@ -190,5 +202,56 @@ private:
     std::deque<std::pair<google::protobuf::Closure*, MonotonicStopWatch>> _pending_closures;
     std::unordered_map<std::thread::id, std::unique_ptr<ThreadClosure>> _local_closure;
 };
+
+class VDataStreamRecvr::PipSenderQueue : public SenderQueue {
+public:
+    PipSenderQueue(VDataStreamRecvr* parent_recvr, int num_senders, RuntimeProfile* profile)
+            : SenderQueue(parent_recvr, num_senders, profile) {}
+
+    bool should_wait() override {
+        return !_is_cancelled && _block_queue_empty && _num_remaining_senders > 0;
+    }
+
+    void _update_block_queue_empty() override { _block_queue_empty = _block_queue.empty(); }
+
+    Status get_batch(Block** next_block) override {
+        CHECK(!should_wait());
+        std::lock_guard<std::mutex> l(_lock); // protect _block_queue
+        return _inner_get_batch(next_block);
+    }
+
+    void add_block(Block* block, bool use_move) override {
+        // Avoid deadlock when calling SenderQueue::cancel() in tcmalloc hook,
+        // limit memory via DataStreamRecvr::exceeds_limit.
+        STOP_CHECK_THREAD_MEM_TRACKER_LIMIT();
+
+        if (_is_cancelled || !block->rows()) {
+            return;
+        }
+        Block* nblock = new Block(block->get_columns_with_type_and_name());
+
+        // local exchange should copy the block contented if use move == false
+        if (use_move) {
+            block->clear();
+        } else {
+            auto rows = block->rows();
+            for (int i = 0; i < nblock->columns(); ++i) {
+                nblock->get_by_position(i).column =
+                        nblock->get_by_position(i).column->clone_resized(rows);
+            }
+        }
+        materialize_block_inplace(*nblock);
+
+        size_t block_size = nblock->bytes();
+        {
+            std::unique_lock<std::mutex> l(_lock);
+            _block_queue.emplace_back(block_size, nblock);
+        }
+        _update_block_queue_empty();
+        _data_arrival_cv.notify_one();
+
+        _recvr->_num_buffered_bytes += block_size;
+    }
+};
 } // namespace vectorized
 } // namespace doris
diff --git a/be/src/vec/runtime/vsorted_run_merger.cpp b/be/src/vec/runtime/vsorted_run_merger.cpp
index 937d743780..e2654e6b60 100644
--- a/be/src/vec/runtime/vsorted_run_merger.cpp
+++ b/be/src/vec/runtime/vsorted_run_merger.cpp
@@ -50,7 +50,9 @@ Status VSortedRunMerger::prepare(const vector<BlockSupplier>& input_runs, bool p
     }
 
     for (auto& _cursor : _cursors) {
-        if (!_cursor._is_eof) _priority_queue.push(MergeSortCursor(&_cursor));
+        if (!_cursor._is_eof) {
+            _priority_queue.push(MergeSortCursor(&_cursor));
+        }
     }
 
     for (const auto& cursor : _cursors) {
diff --git a/be/src/vec/sink/vdata_stream_sender.cpp b/be/src/vec/sink/vdata_stream_sender.cpp
index 100341a197..db1d8c82d2 100644
--- a/be/src/vec/sink/vdata_stream_sender.cpp
+++ b/be/src/vec/sink/vdata_stream_sender.cpp
@@ -22,6 +22,7 @@
 
 #include <random>
 
+#include "common/status.h"
 #include "runtime/dpp_sink_internal.h"
 #include "runtime/exec_env.h"
 #include "runtime/memory/mem_tracker.h"
@@ -36,7 +37,7 @@
 
 namespace doris::vectorized {
 
-Status VDataStreamSender::Channel::init(RuntimeState* state) {
+Status Channel::init(RuntimeState* state) {
     _be_number = state->be_number();
 
     _capacity = std::max(1, _buffer_size / std::max(_row_desc.get_row_size(), 1));
@@ -88,7 +89,7 @@ Status VDataStreamSender::Channel::init(RuntimeState* state) {
     return Status::OK();
 }
 
-Status VDataStreamSender::Channel::send_current_block(bool eos) {
+Status Channel::send_current_block(bool eos) {
     // FIXME: Now, local exchange will cause the performance problem is in a multi-threaded scenario
     // so this feature is turned off here by default. We need to re-examine this logic
     if (_enable_local_exchange && is_local()) {
@@ -103,7 +104,7 @@ Status VDataStreamSender::Channel::send_current_block(bool eos) {
     return Status::OK();
 }
 
-Status VDataStreamSender::Channel::send_local_block(bool eos) {
+Status Channel::send_local_block(bool eos) {
     std::shared_ptr<VDataStreamRecvr> recvr =
             _parent->state()->exec_env()->vstream_mgr()->find_recvr(_fragment_instance_id,
                                                                     _dest_node_id);
@@ -120,7 +121,7 @@ Status VDataStreamSender::Channel::send_local_block(bool eos) {
     return Status::OK();
 }
 
-Status VDataStreamSender::Channel::send_local_block(Block* block) {
+Status Channel::send_local_block(Block* block) {
     std::shared_ptr<VDataStreamRecvr> recvr =
             _parent->state()->exec_env()->vstream_mgr()->find_recvr(_fragment_instance_id,
                                                                     _dest_node_id);
@@ -132,7 +133,7 @@ Status VDataStreamSender::Channel::send_local_block(Block* block) {
     return Status::OK();
 }
 
-Status VDataStreamSender::Channel::send_block(PBlock* block, bool eos) {
+Status Channel::send_block(PBlock* block, bool eos) {
     SCOPED_TIMER(_parent->_brpc_send_timer);
     if (_closure == nullptr) {
         _closure = new RefCountClosure<PTransmitDataResult>();
@@ -159,37 +160,22 @@ Status VDataStreamSender::Channel::send_block(PBlock* block, bool eos) {
     _closure->ref();
     _closure->cntl.set_timeout_ms(_brpc_timeout_ms);
 
-    if (_parent->_transfer_large_data_by_brpc && _brpc_request.has_block() &&
-        _brpc_request.block().has_column_values() &&
-        _brpc_request.ByteSizeLong() > MIN_HTTP_BRPC_SIZE) {
-        SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(ExecEnv::GetInstance()->orphan_mem_tracker());
-        Status st = request_embed_attachment_contain_block<PTransmitDataParams,
-                                                           RefCountClosure<PTransmitDataResult>>(
-                &_brpc_request, _closure);
-        RETURN_IF_ERROR(st);
-        std::string brpc_url =
-                fmt::format("http://{}:{}", _brpc_dest_addr.hostname, _brpc_dest_addr.port);
-        std::shared_ptr<PBackendService_Stub> _brpc_http_stub =
-                _state->exec_env()->brpc_internal_client_cache()->get_new_client_no_cache(brpc_url,
-                                                                                          "http");
-        _closure->cntl.http_request().uri() =
-                brpc_url + "/PInternalServiceImpl/transmit_block_by_http";
-        _closure->cntl.http_request().set_method(brpc::HTTP_METHOD_POST);
-        _closure->cntl.http_request().set_content_type("application/json");
-        _brpc_http_stub->transmit_block_by_http(&_closure->cntl, nullptr, &_closure->result,
-                                                _closure);
-    } else {
+    {
         SCOPED_SWITCH_THREAD_MEM_TRACKER_LIMITER(ExecEnv::GetInstance()->orphan_mem_tracker());
-        _closure->cntl.http_request().Clear();
-        _brpc_stub->transmit_block(&_closure->cntl, &_brpc_request, &_closure->result, _closure);
+        if (enable_http_send_block(_brpc_request, _parent->_transfer_large_data_by_brpc)) {
+            RETURN_IF_ERROR(transmit_block_http(_state, _closure, _brpc_request, _brpc_dest_addr));
+        } else {
+            transmit_block(*_brpc_stub, _closure, _brpc_request);
+        }
     }
+
     if (block != nullptr) {
         _brpc_request.release_block();
     }
     return Status::OK();
 }
 
-Status VDataStreamSender::Channel::add_rows(Block* block, const std::vector<int>& rows) {
+Status Channel::add_rows(Block* block, const std::vector<int>& rows) {
     if (_fragment_instance_id.lo == -1) {
         return Status::OK();
     }
@@ -224,7 +210,7 @@ Status VDataStreamSender::Channel::add_rows(Block* block, const std::vector<int>
     return Status::OK();
 }
 
-Status VDataStreamSender::Channel::close_wait(RuntimeState* state) {
+Status Channel::close_wait(RuntimeState* state) {
     if (_need_close) {
         Status st = _wait_last_brpc();
         if (!st.ok()) {
@@ -237,7 +223,7 @@ Status VDataStreamSender::Channel::close_wait(RuntimeState* state) {
     return Status::OK();
 }
 
-Status VDataStreamSender::Channel::close_internal() {
+Status Channel::close_internal() {
     if (!_need_close) {
         return Status::OK();
     }
@@ -253,7 +239,7 @@ Status VDataStreamSender::Channel::close_internal() {
     return Status::OK();
 }
 
-Status VDataStreamSender::Channel::close(RuntimeState* state) {
+Status Channel::close(RuntimeState* state) {
     Status st = close_internal();
     if (!st.ok()) {
         state->log_error(st.get_error_msg());
@@ -261,12 +247,12 @@ Status VDataStreamSender::Channel::close(RuntimeState* state) {
     return st;
 }
 
-void VDataStreamSender::Channel::ch_roll_pb_block() {
+void Channel::ch_roll_pb_block() {
     _ch_cur_pb_block = (_ch_cur_pb_block == &_ch_pb_block1 ? &_ch_pb_block2 : &_ch_pb_block1);
 }
 
-VDataStreamSender::VDataStreamSender(ObjectPool* pool, int sender_id, const RowDescriptor& row_desc,
-                                     const TDataStreamSink& sink,
+VDataStreamSender::VDataStreamSender(RuntimeState* state, ObjectPool* pool, int sender_id,
+                                     const RowDescriptor& row_desc, const TDataStreamSink& sink,
                                      const std::vector<TPlanFragmentDestination>& destinations,
                                      int per_channel_buffer_size,
                                      bool send_query_statistics_with_every_batch)
@@ -276,7 +262,6 @@ VDataStreamSender::VDataStreamSender(ObjectPool* pool, int sender_id, const RowD
           _current_channel_idx(0),
           _part_type(sink.output_partition.type),
           _ignore_not_found(sink.__isset.ignore_not_found ? sink.ignore_not_found : true),
-          _cur_pb_block(&_pb_block1),
           _profile(nullptr),
           _serialize_batch_timer(nullptr),
           _bytes_sent_counter(nullptr),
@@ -289,7 +274,8 @@ VDataStreamSender::VDataStreamSender(ObjectPool* pool, int sender_id, const RowD
            sink.output_partition.type == TPartitionType::RANDOM ||
            sink.output_partition.type == TPartitionType::RANGE_PARTITIONED ||
            sink.output_partition.type == TPartitionType::BUCKET_SHFFULE_HASH_PARTITIONED);
-    //
+    _cur_pb_block = &_pb_block1;
+
     std::map<int64_t, int64_t> fragment_id_to_channel_index;
 
     for (int i = 0; i < destinations.size(); ++i) {
@@ -298,10 +284,17 @@ VDataStreamSender::VDataStreamSender(ObjectPool* pool, int sender_id, const RowD
         const auto& fragment_instance_id = destinations[i].fragment_instance_id;
         if (fragment_id_to_channel_index.find(fragment_instance_id.lo) ==
             fragment_id_to_channel_index.end()) {
-            _channel_shared_ptrs.emplace_back(
-                    new Channel(this, row_desc, destinations[i].brpc_server, fragment_instance_id,
-                                sink.dest_node_id, per_channel_buffer_size, is_transfer_chain,
-                                send_query_statistics_with_every_batch));
+            if (state->enable_pipeline_exec()) {
+                _channel_shared_ptrs.emplace_back(new PipChannel(
+                        this, row_desc, destinations[i].brpc_server, fragment_instance_id,
+                        sink.dest_node_id, per_channel_buffer_size, is_transfer_chain,
+                        send_query_statistics_with_every_batch));
+            } else {
+                _channel_shared_ptrs.emplace_back(new Channel(
+                        this, row_desc, destinations[i].brpc_server, fragment_instance_id,
+                        sink.dest_node_id, per_channel_buffer_size, is_transfer_chain,
+                        send_query_statistics_with_every_batch));
+            }
             fragment_id_to_channel_index.emplace(fragment_instance_id.lo,
                                                  _channel_shared_ptrs.size() - 1);
             _channels.push_back(_channel_shared_ptrs.back().get());
@@ -322,7 +315,6 @@ VDataStreamSender::VDataStreamSender(ObjectPool* pool, int sender_id, const RowD
           _row_desc(row_desc),
           _current_channel_idx(0),
           _ignore_not_found(true),
-          _cur_pb_block(&_pb_block1),
           _profile(nullptr),
           _serialize_batch_timer(nullptr),
           _compress_timer(nullptr),
@@ -331,6 +323,7 @@ VDataStreamSender::VDataStreamSender(ObjectPool* pool, int sender_id, const RowD
           _bytes_sent_counter(nullptr),
           _local_bytes_send_counter(nullptr),
           _dest_node_id(0) {
+    _cur_pb_block = &_pb_block1;
     _name = "VDataStreamSender";
 }
 
@@ -342,7 +335,6 @@ VDataStreamSender::VDataStreamSender(ObjectPool* pool, const RowDescriptor& row_
           _row_desc(row_desc),
           _current_channel_idx(0),
           _ignore_not_found(true),
-          _cur_pb_block(&_pb_block1),
           _profile(nullptr),
           _serialize_batch_timer(nullptr),
           _compress_timer(nullptr),
@@ -351,6 +343,7 @@ VDataStreamSender::VDataStreamSender(ObjectPool* pool, const RowDescriptor& row_
           _bytes_sent_counter(nullptr),
           _local_bytes_send_counter(nullptr),
           _dest_node_id(0) {
+    _cur_pb_block = &_pb_block1;
     _name = "VDataStreamSender";
 }
 
@@ -462,7 +455,7 @@ Status VDataStreamSender::send(RuntimeState* state, RowBatch* batch) {
     return Status::NotSupported("Not Implemented VOlapScanNode Node::get_next scalar");
 }
 
-Status VDataStreamSender::send(RuntimeState* state, Block* block) {
+Status VDataStreamSender::send(RuntimeState* state, Block* block, bool eos) {
     INIT_AND_SCOPE_SEND_SPAN(state->get_tracer(), _send_span, "VDataStreamSender::send")
     SCOPED_TIMER(_profile->total_time_counter());
     SCOPED_CONSUME_MEM_TRACKER(_mem_tracker.get());
@@ -486,7 +479,7 @@ Status VDataStreamSender::send(RuntimeState* state, Block* block) {
                 if (channel->is_local()) {
                     RETURN_IF_ERROR(channel->send_local_block(block));
                 } else {
-                    RETURN_IF_ERROR(channel->send_block(_cur_pb_block));
+                    RETURN_IF_ERROR(channel->send_block(_cur_pb_block, eos));
                 }
             }
             // rollover
@@ -500,7 +493,7 @@ Status VDataStreamSender::send(RuntimeState* state, Block* block) {
             RETURN_IF_ERROR(current_channel->send_local_block(block));
         } else {
             RETURN_IF_ERROR(serialize_block(block, current_channel->ch_cur_pb_block()));
-            RETURN_IF_ERROR(current_channel->send_block(current_channel->ch_cur_pb_block()));
+            RETURN_IF_ERROR(current_channel->send_block(current_channel->ch_cur_pb_block(), eos));
             current_channel->ch_roll_pb_block();
         }
         _current_channel_idx = (_current_channel_idx + 1) % _channels.size();
@@ -615,4 +608,19 @@ void VDataStreamSender::_roll_pb_block() {
     _cur_pb_block = (_cur_pb_block == &_pb_block1 ? &_pb_block2 : &_pb_block1);
 }
 
+void VDataStreamSender::registe_channels(pipeline::ExchangeSinkBuffer* buffer) {
+    for (auto channel : _channels) {
+        ((PipChannel*)channel)->registe(buffer);
+    }
+}
+
+bool VDataStreamSender::channel_all_can_write() {
+    for (auto channel : _channels) {
+        if (!channel->can_write()) {
+            return false;
+        }
+    }
+    return true;
+}
+
 } // namespace doris::vectorized
diff --git a/be/src/vec/sink/vdata_stream_sender.h b/be/src/vec/sink/vdata_stream_sender.h
index 049de75d0e..9a36731b8d 100644
--- a/be/src/vec/sink/vdata_stream_sender.h
+++ b/be/src/vec/sink/vdata_stream_sender.h
@@ -17,16 +17,21 @@
 
 #pragma once
 
+#include <memory>
+
 #include "common/global_types.h"
 #include "exec/data_sink.h"
 #include "gen_cpp/PaloInternalService_types.h"
 #include "gen_cpp/data.pb.h"
 #include "gen_cpp/internal_service.pb.h"
+#include "pipeline/exec/exchange_sink_buffer.h"
 #include "runtime/descriptors.h"
 #include "service/backend_options.h"
 #include "util/ref_count_closure.h"
 #include "util/uid_util.h"
 #include "vec/exprs/vexpr_context.h"
+#include "vec/runtime/vdata_stream_mgr.h"
+#include "vec/runtime/vdata_stream_recvr.h"
 
 namespace doris {
 class ObjectPool;
@@ -38,14 +43,20 @@ class ExprContext;
 class MemTracker;
 class PartRangeKey;
 
+namespace pipeline {
+class ExchangeSinkOperator;
+}
+
 namespace vectorized {
 class VExprContext;
 class VPartitionInfo;
+class Channel;
 
 class VDataStreamSender : public DataSink {
 public:
-    VDataStreamSender(ObjectPool* pool, int sender_id, const RowDescriptor& row_desc,
-                      const TDataStreamSink& sink,
+    friend class pipeline::ExchangeSinkOperator;
+    VDataStreamSender(RuntimeState* state, ObjectPool* pool, int sender_id,
+                      const RowDescriptor& row_desc, const TDataStreamSink& sink,
                       const std::vector<TPlanFragmentDestination>& destinations,
                       int per_channel_buffer_size, bool send_query_statistics_with_every_batch);
 
@@ -64,7 +75,7 @@ public:
     Status open(RuntimeState* state) override;
 
     Status send(RuntimeState* state, RowBatch* batch) override;
-    Status send(RuntimeState* state, Block* block) override;
+    Status send(RuntimeState* state, Block* block, bool eos = false) override;
 
     Status close(RuntimeState* state, Status exec_status) override;
     RuntimeProfile* profile() override { return _profile; }
@@ -73,9 +84,15 @@ public:
 
     Status serialize_block(Block* src, PBlock* dest, int num_receivers = 1);
 
+    void registe_channels(pipeline::ExchangeSinkBuffer* buffer);
+
+    bool channel_all_can_write();
+
 protected:
+    friend class Channel;
+    friend class pipeline::ExchangeSinkBuffer;
+
     void _roll_pb_block();
-    class Channel;
 
     Status get_partition_column_result(Block* block, int* result) const {
         int counter = 0;
@@ -114,7 +131,7 @@ protected:
     // one while the other one is still being sent
     PBlock _pb_block1;
     PBlock _pb_block2;
-    PBlock* _cur_pb_block = nullptr;
+    PBlock* _cur_pb_block;
 
     // compute per-row partition values
     std::vector<VExprContext*> _partition_expr_ctxs;
@@ -153,10 +170,10 @@ protected:
     bool _new_shuffle_hash_method = false;
 };
 
-// TODO: support local exechange
-
-class VDataStreamSender::Channel {
+class Channel {
 public:
+    friend class VDataStreamSender;
+    friend class pipeline::ExchangeSinkBuffer;
     // Create channel to send data to particular ipaddress/port/query/node
     // combination. buffer_size is specified in bytes and a soft limit on
     // how much tuple data is getting accumulated before being sent; it only applies
@@ -175,14 +192,14 @@ public:
               _need_close(false),
               _brpc_dest_addr(brpc_dest),
               _is_transfer_chain(is_transfer_chain),
-              _send_query_statistics_with_every_batch(send_query_statistics_with_every_batch),
-              _ch_cur_pb_block(&_ch_pb_block1) {
+              _send_query_statistics_with_every_batch(send_query_statistics_with_every_batch) {
         std::string localhost = BackendOptions::get_localhost();
         _is_local = (_brpc_dest_addr.hostname == localhost) &&
                     (_brpc_dest_addr.port == config::brpc_port);
         if (_is_local) {
             VLOG_NOTICE << "will use local Exchange, dest_node_id is : " << _dest_node_id;
         }
+        _ch_cur_pb_block = &_ch_pb_block1;
     }
 
     virtual ~Channel() {
@@ -207,11 +224,11 @@ public:
     // Returns the status of the most recently finished transmit_data
     // rpc (or OK if there wasn't one that hasn't been reported yet).
     // if batch is nullptr, send the eof packet
-    Status send_block(PBlock* block, bool eos = false);
+    virtual Status send_block(PBlock* block, bool eos = false);
 
     Status add_rows(Block* block, const std::vector<int>& row);
 
-    Status send_current_block(bool eos = false);
+    virtual Status send_current_block(bool eos = false);
 
     Status send_local_block(bool eos = false);
 
@@ -240,7 +257,17 @@ public:
 
     void ch_roll_pb_block();
 
-private:
+    bool can_write() {
+        if (!_enable_local_exchange || !is_local()) {
+            return true;
+        }
+        std::shared_ptr<VDataStreamRecvr> recvr =
+                _parent->state()->exec_env()->vstream_mgr()->find_recvr(_fragment_instance_id,
+                                                                        _dest_node_id);
+        return recvr == nullptr || !recvr->exceeds_limit(0);
+    }
+
+protected:
     Status _wait_last_brpc() {
         SCOPED_TIMER(_parent->_brpc_wait_timer);
         if (_closure == nullptr) {
@@ -261,7 +288,6 @@ private:
         return Status::OK();
     }
 
-private:
     // Serialize _batch into _thrift_batch and send via send_batch().
     // Returns send_batch() status.
     Status send_current_batch(bool eos = false);
@@ -331,5 +357,63 @@ Status VDataStreamSender::channel_add_rows(Channels& channels, int num_channels,
     return Status::OK();
 }
 
+class PipChannel : public Channel {
+public:
+    PipChannel(VDataStreamSender* parent, const RowDescriptor& row_desc,
+               const TNetworkAddress& brpc_dest, const TUniqueId& fragment_instance_id,
+               PlanNodeId dest_node_id, int buffer_size, bool is_transfer_chain,
+               bool send_query_statistics_with_every_batch)
+            : Channel(parent, row_desc, brpc_dest, fragment_instance_id, dest_node_id, buffer_size,
+                      is_transfer_chain, send_query_statistics_with_every_batch) {}
+
+    // Asynchronously sends a block
+    // Returns the status of the most recently finished transmit_data
+    // rpc (or OK if there wasn't one that hasn't been reported yet).
+    // if batch is nullptr, send the eof packet
+    Status send_block(PBlock* block, bool eos = false) override {
+        if (eos) {
+            if (_eos_send) {
+                return Status::OK();
+            } else {
+                _eos_send = true;
+            }
+        }
+        if (eos || block->column_metas_size()) {
+            RETURN_IF_ERROR(_buffer->add_block(
+                    {this, block ? std::make_unique<PBlock>(std::move(*block)) : nullptr, eos}));
+        }
+        return Status::OK();
+    }
+
+    // send _mutable_block
+    Status send_current_block(bool eos = false) override {
+        if (_enable_local_exchange && is_local()) {
+            return send_local_block(eos);
+        }
+
+        PBlock* block_ptr = nullptr;
+        if (_mutable_block) {
+            block_ptr = new PBlock(); // TODO: need a pool of PBlock()
+            auto block = _mutable_block->to_block();
+            RETURN_IF_ERROR(_parent->serialize_block(&block, block_ptr));
+            block.clear_column_data();
+            _mutable_block->set_muatable_columns(block.mutate_columns());
+        }
+        RETURN_IF_ERROR(send_block(block_ptr, eos));
+        return Status::OK();
+    }
+
+    void registe(pipeline::ExchangeSinkBuffer* buffer) {
+        _buffer = buffer;
+        _buffer->register_sink(_fragment_instance_id);
+    }
+
+private:
+    friend class VDataStreamSender;
+
+    pipeline::ExchangeSinkBuffer* _buffer = nullptr;
+    bool _eos_send = false;
+};
+
 } // namespace vectorized
 } // namespace doris
diff --git a/be/src/vec/sink/vjdbc_table_sink.cpp b/be/src/vec/sink/vjdbc_table_sink.cpp
index 0d9e457cff..0007a8b9b7 100644
--- a/be/src/vec/sink/vjdbc_table_sink.cpp
+++ b/be/src/vec/sink/vjdbc_table_sink.cpp
@@ -66,7 +66,7 @@ Status VJdbcTableSink::open(RuntimeState* state) {
     return Status::OK();
 }
 
-Status VJdbcTableSink::send(RuntimeState* state, Block* block) {
+Status VJdbcTableSink::send(RuntimeState* state, Block* block, bool eos) {
     INIT_AND_SCOPE_SEND_SPAN(state->get_tracer(), _send_span, "VJdbcTableSink::send");
     Status status = Status::OK();
     if (block == nullptr || block->rows() == 0) {
diff --git a/be/src/vec/sink/vjdbc_table_sink.h b/be/src/vec/sink/vjdbc_table_sink.h
index 52238942a2..a3de3ff62b 100644
--- a/be/src/vec/sink/vjdbc_table_sink.h
+++ b/be/src/vec/sink/vjdbc_table_sink.h
@@ -32,7 +32,7 @@ public:
 
     Status open(RuntimeState* state) override;
 
-    Status send(RuntimeState* state, vectorized::Block* block) override;
+    Status send(RuntimeState* state, vectorized::Block* block, bool eos = false) override;
 
     Status close(RuntimeState* state, Status exec_status) override;
 
diff --git a/be/src/vec/sink/vmysql_result_writer.cpp b/be/src/vec/sink/vmysql_result_writer.cpp
index 155e4c3f2e..e5f2c258e3 100644
--- a/be/src/vec/sink/vmysql_result_writer.cpp
+++ b/be/src/vec/sink/vmysql_result_writer.cpp
@@ -664,6 +664,10 @@ Status VMysqlResultWriter::append_block(Block& input_block) {
     return status;
 }
 
+bool VMysqlResultWriter::can_sink() {
+    return _sinker->can_sink();
+}
+
 Status VMysqlResultWriter::close() {
     COUNTER_SET(_sent_rows_counter, _written_rows);
     return Status::OK();
diff --git a/be/src/vec/sink/vmysql_result_writer.h b/be/src/vec/sink/vmysql_result_writer.h
index 7b8df80306..e17d41b7fa 100644
--- a/be/src/vec/sink/vmysql_result_writer.h
+++ b/be/src/vec/sink/vmysql_result_writer.h
@@ -43,6 +43,8 @@ public:
 
     virtual Status append_block(Block& block) override;
 
+    virtual bool can_sink() override;
+
     virtual Status close() override;
 
 private:
diff --git a/be/src/vec/sink/vmysql_table_sink.cpp b/be/src/vec/sink/vmysql_table_sink.cpp
index bdb2374b3a..ee9bc8095b 100644
--- a/be/src/vec/sink/vmysql_table_sink.cpp
+++ b/be/src/vec/sink/vmysql_table_sink.cpp
@@ -53,7 +53,7 @@ Status VMysqlTableSink::open(RuntimeState* state) {
     return Status::OK();
 }
 
-Status VMysqlTableSink::send(RuntimeState* state, Block* block) {
+Status VMysqlTableSink::send(RuntimeState* state, Block* block, bool eos) {
     INIT_AND_SCOPE_SEND_SPAN(state->get_tracer(), _send_span, "VMysqlTableSink::send");
     return _writer->append(block);
 }
diff --git a/be/src/vec/sink/vmysql_table_sink.h b/be/src/vec/sink/vmysql_table_sink.h
index 6a30275a8e..9d41fe8a84 100644
--- a/be/src/vec/sink/vmysql_table_sink.h
+++ b/be/src/vec/sink/vmysql_table_sink.h
@@ -34,7 +34,7 @@ public:
 
     Status open(RuntimeState* state) override;
 
-    Status send(RuntimeState* state, vectorized::Block* block) override;
+    Status send(RuntimeState* state, vectorized::Block* block, bool eos = false) override;
 
     Status close(RuntimeState* state, Status exec_status) override;
 
diff --git a/be/src/vec/sink/vodbc_table_sink.cpp b/be/src/vec/sink/vodbc_table_sink.cpp
index 8d6a0596b2..b2012cb911 100644
--- a/be/src/vec/sink/vodbc_table_sink.cpp
+++ b/be/src/vec/sink/vodbc_table_sink.cpp
@@ -56,7 +56,7 @@ Status VOdbcTableSink::open(RuntimeState* state) {
     return Status::OK();
 }
 
-Status VOdbcTableSink::send(RuntimeState* state, Block* block) {
+Status VOdbcTableSink::send(RuntimeState* state, Block* block, bool eos) {
     INIT_AND_SCOPE_SEND_SPAN(state->get_tracer(), _send_span, "VOdbcTableSink::send");
     Status status = Status::OK();
     if (block == nullptr || block->rows() == 0) {
diff --git a/be/src/vec/sink/vodbc_table_sink.h b/be/src/vec/sink/vodbc_table_sink.h
index dc3d38efee..7218edf10f 100644
--- a/be/src/vec/sink/vodbc_table_sink.h
+++ b/be/src/vec/sink/vodbc_table_sink.h
@@ -33,7 +33,7 @@ public:
 
     Status open(RuntimeState* state) override;
 
-    Status send(RuntimeState* state, vectorized::Block* block) override;
+    Status send(RuntimeState* state, vectorized::Block* block, bool eos = false) override;
 
     Status close(RuntimeState* state, Status exec_status) override;
 
diff --git a/be/src/vec/sink/vresult_file_sink.cpp b/be/src/vec/sink/vresult_file_sink.cpp
index 7bfc8b4c8a..51c6673fd0 100644
--- a/be/src/vec/sink/vresult_file_sink.cpp
+++ b/be/src/vec/sink/vresult_file_sink.cpp
@@ -103,7 +103,7 @@ Status VResultFileSink::prepare(RuntimeState* state) {
     if (_is_top_sink) {
         // create sender
         RETURN_IF_ERROR(state->exec_env()->result_mgr()->create_sender(
-                state->fragment_instance_id(), _buf_size, &_sender));
+                state->fragment_instance_id(), _buf_size, &_sender, state->enable_pipeline_exec()));
         // create writer
         _writer.reset(new (std::nothrow) VFileResultWriter(
                 _file_opts.get(), _storage_type, state->fragment_instance_id(), _output_vexpr_ctxs,
@@ -142,7 +142,7 @@ Status VResultFileSink::send(RuntimeState* state, RowBatch* batch) {
     return Status::NotSupported("Not Implemented VResultFileSink Node::get_next scalar");
 }
 
-Status VResultFileSink::send(RuntimeState* state, Block* block) {
+Status VResultFileSink::send(RuntimeState* state, Block* block, bool eos) {
     INIT_AND_SCOPE_SEND_SPAN(state->get_tracer(), _send_span, "VResultFileSink::send");
     RETURN_IF_ERROR(_writer->append_block(*block));
     return Status::OK();
diff --git a/be/src/vec/sink/vresult_file_sink.h b/be/src/vec/sink/vresult_file_sink.h
index c5c509caf7..a5ce85bf4b 100644
--- a/be/src/vec/sink/vresult_file_sink.h
+++ b/be/src/vec/sink/vresult_file_sink.h
@@ -41,7 +41,7 @@ public:
     // send data in 'batch' to this backend stream mgr
     // Blocks until all rows in batch are placed in the buffer
     Status send(RuntimeState* state, RowBatch* batch) override;
-    Status send(RuntimeState* state, Block* block) override;
+    Status send(RuntimeState* state, Block* block, bool eos = false) override;
     // Flush all buffered data and close all existing channels to destination
     // hosts. Further send() calls are illegal after calling close().
     Status close(RuntimeState* state, Status exec_status) override;
diff --git a/be/src/vec/sink/vresult_sink.cpp b/be/src/vec/sink/vresult_sink.cpp
index 77e64a8959..c1da47760a 100644
--- a/be/src/vec/sink/vresult_sink.cpp
+++ b/be/src/vec/sink/vresult_sink.cpp
@@ -50,6 +50,7 @@ Status VResultSink::prepare_exprs(RuntimeState* state) {
     RETURN_IF_ERROR(VExpr::prepare(_output_vexpr_ctxs, state, _row_desc));
     return Status::OK();
 }
+
 Status VResultSink::prepare(RuntimeState* state) {
     RETURN_IF_ERROR(DataSink::prepare(state));
     auto fragment_instance_id = state->fragment_instance_id();
@@ -61,8 +62,8 @@ Status VResultSink::prepare(RuntimeState* state) {
     RETURN_IF_ERROR(prepare_exprs(state));
 
     // create sender
-    RETURN_IF_ERROR(state->exec_env()->result_mgr()->create_sender(state->fragment_instance_id(),
-                                                                   _buf_size, &_sender));
+    RETURN_IF_ERROR(state->exec_env()->result_mgr()->create_sender(
+            state->fragment_instance_id(), _buf_size, &_sender, state->enable_pipeline_exec()));
 
     // create writer based on sink type
     switch (_sink_type) {
@@ -87,7 +88,7 @@ Status VResultSink::send(RuntimeState* state, RowBatch* batch) {
     return Status::NotSupported("Not Implemented Result Sink::send scalar");
 }
 
-Status VResultSink::send(RuntimeState* state, Block* block) {
+Status VResultSink::send(RuntimeState* state, Block* block, bool eos) {
     INIT_AND_SCOPE_SEND_SPAN(state->get_tracer(), _send_span, "VResultSink::send");
     // The memory consumption in the process of sending the results is not check query memory limit.
     // Avoid the query being cancelled when the memory limit is reached after the query result comes out.
@@ -119,7 +120,7 @@ Status VResultSink::close(RuntimeState* state, Status exec_status) {
         _sender->close(final_status);
     }
     state->exec_env()->result_mgr()->cancel_at_time(
-            time(NULL) + config::result_buffer_cancelled_interval_time,
+            time(nullptr) + config::result_buffer_cancelled_interval_time,
             state->fragment_instance_id());
 
     VExpr::close(_output_vexpr_ctxs, state);
diff --git a/be/src/vec/sink/vresult_sink.h b/be/src/vec/sink/vresult_sink.h
index 0ccc8ca593..b71cf122fe 100644
--- a/be/src/vec/sink/vresult_sink.h
+++ b/be/src/vec/sink/vresult_sink.h
@@ -29,11 +29,15 @@ class ExprContext;
 class ResultWriter;
 class MemTracker;
 struct ResultFileOptions;
+namespace pipeline {
+class ResultSinkOperator;
+}
 namespace vectorized {
 class VExprContext;
 
 class VResultSink : public DataSink {
 public:
+    friend class pipeline::ResultSinkOperator;
     VResultSink(const RowDescriptor& row_desc, const std::vector<TExpr>& select_exprs,
                 const TResultSink& sink, int buffer_size);
 
@@ -44,7 +48,7 @@ public:
 
     // not implement
     virtual Status send(RuntimeState* state, RowBatch* batch) override;
-    virtual Status send(RuntimeState* state, Block* block) override;
+    virtual Status send(RuntimeState* state, Block* block, bool eos = false) override;
     // Flush all buffered data and close all existing channels to destination
     // hosts. Further send() calls are illegal after calling close().
     virtual Status close(RuntimeState* state, Status exec_status) override;
diff --git a/be/src/vec/sink/vresult_writer.h b/be/src/vec/sink/vresult_writer.h
index a1fb6ecce3..5bd590f14d 100644
--- a/be/src/vec/sink/vresult_writer.h
+++ b/be/src/vec/sink/vresult_writer.h
@@ -26,6 +26,8 @@ public:
     VResultWriter() : ResultWriter() {}
 
     virtual Status append_block(Block& block) = 0;
+
+    virtual bool can_sink() { return true; }
 };
 } // namespace vectorized
 } // namespace doris
\ No newline at end of file
diff --git a/be/src/vec/sink/vtable_sink.cpp b/be/src/vec/sink/vtable_sink.cpp
index c67a9a933a..4bf4d64147 100644
--- a/be/src/vec/sink/vtable_sink.cpp
+++ b/be/src/vec/sink/vtable_sink.cpp
@@ -59,7 +59,7 @@ Status VTableSink::send(RuntimeState* state, RowBatch* batch) {
             "Not Implemented VTableSink::send(RuntimeState* state, RowBatch* batch)");
 }
 
-Status VTableSink::send(RuntimeState* state, Block* block) {
+Status VTableSink::send(RuntimeState* state, Block* block, bool eos) {
     INIT_AND_SCOPE_SEND_SPAN(state->get_tracer(), _send_span, "VTableSink::send");
     return Status::OK();
 }
diff --git a/be/src/vec/sink/vtable_sink.h b/be/src/vec/sink/vtable_sink.h
index ccad0cb3aa..cf2423dacf 100644
--- a/be/src/vec/sink/vtable_sink.h
+++ b/be/src/vec/sink/vtable_sink.h
@@ -40,7 +40,7 @@ public:
 
     Status send(RuntimeState* state, RowBatch* batch) override;
 
-    Status send(RuntimeState* state, vectorized::Block* block) override;
+    Status send(RuntimeState* state, vectorized::Block* block, bool eos = false) override;
     // Flush all buffered data and close all existing channels to destination
     // hosts. Further send() calls are illegal after calling close().
     Status close(RuntimeState* state, Status exec_status) override;
diff --git a/be/src/vec/sink/vtablet_sink.cpp b/be/src/vec/sink/vtablet_sink.cpp
index a9461e1659..d46dfe956b 100644
--- a/be/src/vec/sink/vtablet_sink.cpp
+++ b/be/src/vec/sink/vtablet_sink.cpp
@@ -494,7 +494,7 @@ Status VOlapTableSink::find_tablet(RuntimeState* state, vectorized::Block* block
     return status;
 }
 
-Status VOlapTableSink::send(RuntimeState* state, vectorized::Block* input_block) {
+Status VOlapTableSink::send(RuntimeState* state, vectorized::Block* input_block, bool eos) {
     INIT_AND_SCOPE_SEND_SPAN(state->get_tracer(), _send_span, "VOlapTableSink::send");
     SCOPED_CONSUME_MEM_TRACKER(_mem_tracker.get());
     Status status = Status::OK();
diff --git a/be/src/vec/sink/vtablet_sink.h b/be/src/vec/sink/vtablet_sink.h
index 857c43722b..151873cbf7 100644
--- a/be/src/vec/sink/vtablet_sink.h
+++ b/be/src/vec/sink/vtablet_sink.h
@@ -90,7 +90,7 @@ public:
 
     Status close(RuntimeState* state, Status close_status) override;
     using OlapTableSink::send;
-    Status send(RuntimeState* state, vectorized::Block* block) override;
+    Status send(RuntimeState* state, vectorized::Block* block, bool eos = false) override;
 
     size_t get_pending_bytes() const;
 
diff --git a/be/test/runtime/fragment_mgr_test.cpp b/be/test/runtime/fragment_mgr_test.cpp
index 060b3a501a..55939cc04f 100644
--- a/be/test/runtime/fragment_mgr_test.cpp
+++ b/be/test/runtime/fragment_mgr_test.cpp
@@ -95,8 +95,6 @@ TEST_F(FragmentMgrTest, CancelNormal) {
     params.params.fragment_instance_id.__set_hi(100);
     params.params.fragment_instance_id.__set_lo(200);
     EXPECT_TRUE(mgr.exec_plan_fragment(params).ok());
-    // Cancel after add
-    EXPECT_TRUE(mgr.cancel(params.params.fragment_instance_id).ok());
 }
 
 TEST_F(FragmentMgrTest, CancelWithoutAdd) {
@@ -105,7 +103,6 @@ TEST_F(FragmentMgrTest, CancelWithoutAdd) {
     params.params.fragment_instance_id = TUniqueId();
     params.params.fragment_instance_id.__set_hi(100);
     params.params.fragment_instance_id.__set_lo(200);
-    EXPECT_TRUE(mgr.cancel(params.params.fragment_instance_id).ok());
 }
 
 TEST_F(FragmentMgrTest, PrepareFailed) {
diff --git a/be/test/runtime/result_buffer_mgr_test.cpp b/be/test/runtime/result_buffer_mgr_test.cpp
index 492efe0f4b..152c155ef0 100644
--- a/be/test/runtime/result_buffer_mgr_test.cpp
+++ b/be/test/runtime/result_buffer_mgr_test.cpp
@@ -43,7 +43,7 @@ TEST_F(ResultBufferMgrTest, create_normal) {
     query_id.hi = 100;
 
     std::shared_ptr<BufferControlBlock> control_block1;
-    EXPECT_TRUE(buffer_mgr.create_sender(query_id, 1024, &control_block1).ok());
+    EXPECT_TRUE(buffer_mgr.create_sender(query_id, 1024, &control_block1, false).ok());
 }
 
 TEST_F(ResultBufferMgrTest, create_same_buffer) {
@@ -53,9 +53,9 @@ TEST_F(ResultBufferMgrTest, create_same_buffer) {
     query_id.hi = 100;
 
     std::shared_ptr<BufferControlBlock> control_block1;
-    EXPECT_TRUE(buffer_mgr.create_sender(query_id, 1024, &control_block1).ok());
+    EXPECT_TRUE(buffer_mgr.create_sender(query_id, 1024, &control_block1, false).ok());
     std::shared_ptr<BufferControlBlock> control_block2;
-    EXPECT_TRUE(buffer_mgr.create_sender(query_id, 1024, &control_block2).ok());
+    EXPECT_TRUE(buffer_mgr.create_sender(query_id, 1024, &control_block2, false).ok());
 
     EXPECT_EQ(control_block1.get(), control_block1.get());
 }
@@ -67,7 +67,7 @@ TEST_F(ResultBufferMgrTest, fetch_data_normal) {
     query_id.hi = 100;
 
     std::shared_ptr<BufferControlBlock> control_block1;
-    EXPECT_TRUE(buffer_mgr.create_sender(query_id, 1024, &control_block1).ok());
+    EXPECT_TRUE(buffer_mgr.create_sender(query_id, 1024, &control_block1, false).ok());
 
     TFetchDataResult* result = new TFetchDataResult();
     result->result_batch.rows.push_back("hello test");
@@ -85,7 +85,7 @@ TEST_F(ResultBufferMgrTest, fetch_data_no_block) {
     query_id.hi = 100;
 
     std::shared_ptr<BufferControlBlock> control_block1;
-    EXPECT_TRUE(buffer_mgr.create_sender(query_id, 1024, &control_block1).ok());
+    EXPECT_TRUE(buffer_mgr.create_sender(query_id, 1024, &control_block1, false).ok());
 
     TFetchDataResult* result = new TFetchDataResult();
     query_id.lo = 11;
@@ -101,7 +101,7 @@ TEST_F(ResultBufferMgrTest, normal_cancel) {
     query_id.hi = 100;
 
     std::shared_ptr<BufferControlBlock> control_block1;
-    EXPECT_TRUE(buffer_mgr.create_sender(query_id, 1024, &control_block1).ok());
+    EXPECT_TRUE(buffer_mgr.create_sender(query_id, 1024, &control_block1, false).ok());
 
     EXPECT_TRUE(buffer_mgr.cancel(query_id).ok());
 }
diff --git a/be/test/vec/runtime/vdata_stream_test.cpp b/be/test/vec/runtime/vdata_stream_test.cpp
index 539dd51d74..cfc99d86a4 100644
--- a/be/test/vec/runtime/vdata_stream_test.cpp
+++ b/be/test/vec/runtime/vdata_stream_test.cpp
@@ -158,8 +158,9 @@ TEST_F(VDataStreamTest, BasicTest) {
     }
     int per_channel_buffer_size = 1024 * 1024;
     bool send_query_statistics_with_every_batch = false;
-    VDataStreamSender sender(&_object_pool, sender_id, row_desc, tsink.stream_sink, dests,
-                             per_channel_buffer_size, send_query_statistics_with_every_batch);
+    VDataStreamSender sender(&runtime_stat, &_object_pool, sender_id, row_desc, tsink.stream_sink,
+                             dests, per_channel_buffer_size,
+                             send_query_statistics_with_every_batch);
     sender.set_query_statistics(std::make_shared<QueryStatistics>());
     sender.init(tsink);
     sender.prepare(&runtime_stat);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java
index a46e2a2999..23bba29085 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java
@@ -169,6 +169,8 @@ public class SessionVariable implements Serializable, Writable {
 
     public static final String ENABLE_VECTORIZED_ENGINE = "enable_vectorized_engine";
 
+    public static final String ENABLE_PIPELINE_ENGINE = "enable_pipeline_engine";
+
     public static final String ENABLE_SINGLE_DISTINCT_COLUMN_OPT = "enable_single_distinct_column_opt";
 
     public static final String CPU_RESOURCE_LIMIT = "cpu_resource_limit";
@@ -469,6 +471,9 @@ public class SessionVariable implements Serializable, Writable {
     @VariableMgr.VarAttr(name = ENABLE_VECTORIZED_ENGINE)
     public boolean enableVectorizedEngine = true;
 
+    @VariableMgr.VarAttr(name = ENABLE_PIPELINE_ENGINE)
+    public boolean enablePipelineEngine = false;
+
     @VariableMgr.VarAttr(name = ENABLE_PARALLEL_OUTFILE)
     public boolean enableParallelOutfile = false;
 
@@ -1105,6 +1110,14 @@ public class SessionVariable implements Serializable, Writable {
         this.enableVectorizedEngine = enableVectorizedEngine;
     }
 
+    public boolean enablePipelineEngine() {
+        return enablePipelineEngine && enableVectorizedEngine;
+    }
+
+    public void setEnablePipelineEngine(boolean enablePipelineEngine) {
+        this.enablePipelineEngine = enablePipelineEngine;
+    }
+
     public boolean enablePushDownNoGroupAgg() {
         return enablePushDownNoGroupAgg;
     }
@@ -1311,6 +1324,7 @@ public class SessionVariable implements Serializable, Writable {
         tResult.setCodegenLevel(codegenLevel);
         tResult.setEnableVectorizedEngine(enableVectorizedEngine);
         tResult.setBeExecVersion(Config.be_exec_version);
+        tResult.setEnablePipelineEngine(enablePipelineEngine);
         tResult.setReturnObjectDataAsBinary(returnObjectDataAsBinary);
         tResult.setTrimTailingSpacesForExternalTableQuery(trimTailingSpacesForExternalTableQuery);
         tResult.setEnableShareHashTableForBroadcastJoin(enableShareHashTableForBroadcastJoin);
diff --git a/gensrc/thrift/PaloInternalService.thrift b/gensrc/thrift/PaloInternalService.thrift
index 47a9144269..cb343ae29b 100644
--- a/gensrc/thrift/PaloInternalService.thrift
+++ b/gensrc/thrift/PaloInternalService.thrift
@@ -183,6 +183,8 @@ struct TQueryOptions {
   53: optional i32 partitioned_hash_join_rows_threshold = 0
 
   54: optional bool enable_share_hash_table_for_broadcast_join
+
+  55: optional bool enable_pipeline_engine = false
 }
     
 


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org