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

yiguolei 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 abaa042cf06 [refactor](thrift) remove useless thrift definition and 
remove related code (#56102)
abaa042cf06 is described below

commit abaa042cf06eef14ccc0cc909a66b109d0e7714b
Author: yiguolei <[email protected]>
AuthorDate: Wed Sep 17 21:01:08 2025 +0800

    [refactor](thrift) remove useless thrift definition and remove related code 
(#56102)
---
 be/src/http/action/http_stream.cpp                 |   6 +-
 be/src/http/action/stream_load.cpp                 |   9 +-
 be/src/io/fs/multi_table_pipe.cpp                  |   6 +-
 be/src/olap/push_handler.cpp                       |   3 -
 be/src/runtime/fold_constant_executor.cpp          |   8 +-
 be/src/runtime/fragment_mgr.cpp                    |  14 +-
 be/src/runtime/fragment_mgr.h                      |   6 -
 .../routine_load/routine_load_task_executor.cpp    |  11 +-
 be/src/runtime/stream_load/stream_load_context.cpp |   6 +-
 .../runtime/stream_load/stream_load_executor.cpp   |  10 +-
 be/src/service/backend_service.cpp                 |  63 -------
 be/src/service/backend_service.h                   |  23 ---
 be/src/service/internal_service.cpp                |  49 +-----
 .../org/apache/doris/rpc/BackendServiceProxy.java  |  33 ----
 .../apache/doris/service/FrontendServiceImpl.java  |  10 --
 .../java/org/apache/doris/task/AgentClient.java    |  61 -------
 .../org/apache/doris/common/GenericPoolTest.java   |  35 ----
 .../apache/doris/utframe/MockedBackendFactory.java |  33 ----
 gensrc/thrift/BackendService.thrift                |  24 +--
 gensrc/thrift/FrontendService.thrift               |  12 +-
 gensrc/thrift/PaloInternalService.thrift           | 183 +--------------------
 gensrc/thrift/Types.thrift                         |   7 -
 22 files changed, 26 insertions(+), 586 deletions(-)

diff --git a/be/src/http/action/http_stream.cpp 
b/be/src/http/action/http_stream.cpp
index 86827415106..495ddf38bd9 100644
--- a/be/src/http/action/http_stream.cpp
+++ b/be/src/http/action/http_stream.cpp
@@ -382,11 +382,7 @@ Status HttpStreamAction::process_put(HttpRequest* http_req,
                 content_length *= 3;
             }
         }
-        if (ctx->put_result.__isset.params) {
-            ctx->put_result.params.__set_content_length(content_length);
-        } else {
-            
ctx->put_result.pipeline_params.__set_content_length(content_length);
-        }
+        ctx->put_result.pipeline_params.__set_content_length(content_length);
     }
     TPipelineFragmentParamsList mocked;
     return _exec_env->stream_load_executor()->execute_plan_fragment(ctx, 
mocked);
diff --git a/be/src/http/action/stream_load.cpp 
b/be/src/http/action/stream_load.cpp
index 1f7b1986315..b4a3a74fa9b 100644
--- a/be/src/http/action/stream_load.cpp
+++ b/be/src/http/action/stream_load.cpp
@@ -792,14 +792,11 @@ Status StreamLoadAction::_process_put(HttpRequest* 
http_req,
                 content_length *= 3;
             }
         }
-        if (ctx->put_result.__isset.params) {
-            ctx->put_result.params.__set_content_length(content_length);
-        } else {
-            
ctx->put_result.pipeline_params.__set_content_length(content_length);
-        }
+        ctx->put_result.pipeline_params.__set_content_length(content_length);
     }
 
-    VLOG_NOTICE << "params is " << 
apache::thrift::ThriftDebugString(ctx->put_result.params);
+    VLOG_NOTICE << "params is "
+                << 
apache::thrift::ThriftDebugString(ctx->put_result.pipeline_params);
     // if we not use streaming, we must download total content before we begin
     // to process this load
     if (!ctx->use_streaming) {
diff --git a/be/src/io/fs/multi_table_pipe.cpp 
b/be/src/io/fs/multi_table_pipe.cpp
index 0f52bfde950..224925dc6a0 100644
--- a/be/src/io/fs/multi_table_pipe.cpp
+++ b/be/src/io/fs/multi_table_pipe.cpp
@@ -213,11 +213,7 @@ Status MultiTablePipe::request_and_exec_plans() {
             return plan_status;
         }
 
-        if (_ctx->multi_table_put_result.__isset.params &&
-            !_ctx->multi_table_put_result.__isset.pipeline_params) {
-            return Status::Aborted("only support pipeline engine");
-        } else if (!_ctx->multi_table_put_result.__isset.params &&
-                   _ctx->multi_table_put_result.__isset.pipeline_params) {
+        if (_ctx->multi_table_put_result.__isset.pipeline_params) {
             st = exec_plans(exec_env, 
_ctx->multi_table_put_result.pipeline_params);
         } else {
             return Status::Aborted("too many or too few params are set in 
multi_table_put_result.");
diff --git a/be/src/olap/push_handler.cpp b/be/src/olap/push_handler.cpp
index 1ef770345d7..07586acc5a2 100644
--- a/be/src/olap/push_handler.cpp
+++ b/be/src/olap/push_handler.cpp
@@ -383,9 +383,6 @@ Status PushBrokerReader::init() {
     TPlanFragmentExecParams params;
     params.fragment_instance_id = dummy_id;
     params.query_id = dummy_id;
-    TExecPlanFragmentParams fragment_params;
-    fragment_params.params = params;
-    fragment_params.protocol_version = PaloInternalServiceVersion::V1;
     TQueryOptions query_options;
     TQueryGlobals query_globals;
     std::shared_ptr<MemTrackerLimiter> tracker = 
MemTrackerLimiter::create_shared(
diff --git a/be/src/runtime/fold_constant_executor.cpp 
b/be/src/runtime/fold_constant_executor.cpp
index f6945090a84..6449b0218ab 100644
--- a/be/src/runtime/fold_constant_executor.cpp
+++ b/be/src/runtime/fold_constant_executor.cpp
@@ -162,15 +162,11 @@ Status FoldConstantExecutor::_init(const TQueryGlobals& 
query_globals,
     TPlanFragmentExecParams params;
     params.fragment_instance_id = _query_id;
     params.query_id = _query_id;
-    TExecPlanFragmentParams fragment_params;
-    fragment_params.params = params;
-    fragment_params.protocol_version = PaloInternalServiceVersion::V1;
     _mem_tracker = MemTrackerLimiter::create_shared(
             MemTrackerLimiter::Type::OTHER,
             fmt::format("FoldConstant:query_id={}", print_id(_query_id)));
-    _runtime_state =
-            RuntimeState::create_unique(fragment_params.params, query_options, 
query_globals,
-                                        ExecEnv::GetInstance(), nullptr, 
_mem_tracker);
+    _runtime_state = RuntimeState::create_unique(params, query_options, 
query_globals,
+                                                 ExecEnv::GetInstance(), 
nullptr, _mem_tracker);
     DescriptorTbl* desc_tbl = nullptr;
     Status status =
             DescriptorTbl::create(_runtime_state->obj_pool(), 
TDescriptorTable(), &desc_tbl);
diff --git a/be/src/runtime/fragment_mgr.cpp b/be/src/runtime/fragment_mgr.cpp
index 508dd24825a..f1e04796397 100644
--- a/be/src/runtime/fragment_mgr.cpp
+++ b/be/src/runtime/fragment_mgr.cpp
@@ -592,11 +592,6 @@ void FragmentMgr::coordinator_callback(const 
ReportStatusRequest& req) {
 
 static void empty_function(RuntimeState*, Status*) {}
 
-Status FragmentMgr::exec_plan_fragment(const TExecPlanFragmentParams& params,
-                                       const QuerySource query_source) {
-    return Status::InternalError("Non-pipeline is disabled!");
-}
-
 Status FragmentMgr::exec_plan_fragment(const TPipelineFragmentParams& params,
                                        const QuerySource query_source,
                                        const TPipelineFragmentParamsList& 
parent) {
@@ -775,11 +770,6 @@ Status FragmentMgr::_get_or_create_query_ctx(const 
TPipelineFragmentParams& para
     return Status::OK();
 }
 
-Status FragmentMgr::exec_plan_fragment(const TExecPlanFragmentParams& params,
-                                       QuerySource query_source, const 
FinishCallback& cb) {
-    return Status::InternalError("Non-pipeline is disabled!");
-}
-
 std::string FragmentMgr::dump_pipeline_tasks(int64_t duration) {
     fmt::memory_buffer debug_string_buffer;
     size_t i = 0;
@@ -829,7 +819,7 @@ Status FragmentMgr::exec_plan_fragment(const 
TPipelineFragmentParams& params,
                                        const TPipelineFragmentParamsList& 
parent) {
     VLOG_ROW << "Query: " << print_id(params.query_id) << " exec_plan_fragment 
params is "
              << apache::thrift::ThriftDebugString(params).c_str();
-    // sometimes TExecPlanFragmentParams debug string is too long and glog
+    // sometimes TPipelineFragmentParams debug string is too long and glog
     // will truncate the log line, so print query options seperately for 
debuggin purpose
     VLOG_ROW << "Query: " << print_id(params.query_id) << "query options is "
              << 
apache::thrift::ThriftDebugString(params.query_options).c_str();
@@ -1159,7 +1149,7 @@ void FragmentMgr::_check_brpc_available(const 
std::shared_ptr<PBackendService_St
 void FragmentMgr::debug(std::stringstream& ss) {}
 /*
  * 1. resolve opaqued_query_plan to thrift structure
- * 2. build TExecPlanFragmentParams
+ * 2. build TPipelineFragmentParams
  */
 Status FragmentMgr::exec_external_plan_fragment(const TScanOpenParams& params,
                                                 const TQueryPlanInfo& 
t_query_plan_info,
diff --git a/be/src/runtime/fragment_mgr.h b/be/src/runtime/fragment_mgr.h
index c23c4b768ec..aca118dedaf 100644
--- a/be/src/runtime/fragment_mgr.h
+++ b/be/src/runtime/fragment_mgr.h
@@ -55,7 +55,6 @@ class PipelineFragmentContext;
 class QueryContext;
 class ExecEnv;
 class ThreadPool;
-class TExecPlanFragmentParams;
 class PExecPlanFragmentStartRequest;
 class PMergeFilterRequest;
 class RuntimeProfile;
@@ -120,17 +119,12 @@ public:
     void stop();
 
     // execute one plan fragment
-    Status exec_plan_fragment(const TExecPlanFragmentParams& params, const 
QuerySource query_type);
 
     Status exec_plan_fragment(const TPipelineFragmentParams& params, const 
QuerySource query_type,
                               const TPipelineFragmentParamsList& parent);
 
     void remove_pipeline_context(std::pair<TUniqueId, int> key);
 
-    // TODO(zc): report this is over
-    Status exec_plan_fragment(const TExecPlanFragmentParams& params, const 
QuerySource query_type,
-                              const FinishCallback& cb);
-
     Status exec_plan_fragment(const TPipelineFragmentParams& params, const 
QuerySource query_type,
                               const FinishCallback& cb, const 
TPipelineFragmentParamsList& parent);
 
diff --git a/be/src/runtime/routine_load/routine_load_task_executor.cpp 
b/be/src/runtime/routine_load/routine_load_task_executor.cpp
index e51a91913a6..29743921c6c 100644
--- a/be/src/runtime/routine_load/routine_load_task_executor.cpp
+++ b/be/src/runtime/routine_load/routine_load_task_executor.cpp
@@ -278,13 +278,10 @@ Status RoutineLoadTaskExecutor::submit_task(const 
TRoutineLoadTask& task) {
     TStatus tstatus;
     tstatus.status_code = TStatusCode::OK;
     put_result.status = tstatus;
-    if (task.__isset.params) {
-        put_result.params = task.params;
-        put_result.__isset.params = true;
-    } else {
-        put_result.pipeline_params = task.pipeline_params;
-        put_result.__isset.pipeline_params = true;
-    }
+
+    put_result.pipeline_params = task.pipeline_params;
+    put_result.__isset.pipeline_params = true;
+
     ctx->put_result = put_result;
     if (task.__isset.format) {
         ctx->format = task.format;
diff --git a/be/src/runtime/stream_load/stream_load_context.cpp 
b/be/src/runtime/stream_load/stream_load_context.cpp
index 176846fb5f3..62ae3db68d7 100644
--- a/be/src/runtime/stream_load/stream_load_context.cpp
+++ b/be/src/runtime/stream_load/stream_load_context.cpp
@@ -357,10 +357,8 @@ std::string StreamLoadContext::brief(bool detail) const {
 }
 
 bool StreamLoadContext::is_mow_table() const {
-    return (put_result.__isset.params && 
put_result.params.__isset.is_mow_table &&
-            put_result.params.is_mow_table) ||
-           (put_result.__isset.pipeline_params && 
put_result.pipeline_params.__isset.is_mow_table &&
-            put_result.pipeline_params.is_mow_table);
+    return put_result.__isset.pipeline_params && 
put_result.pipeline_params.__isset.is_mow_table &&
+           put_result.pipeline_params.is_mow_table;
 }
 
 #include "common/compile_check_end.h"
diff --git a/be/src/runtime/stream_load/stream_load_executor.cpp 
b/be/src/runtime/stream_load/stream_load_executor.cpp
index 7ae8217c76d..4a2fca5ad57 100644
--- a/be/src/runtime/stream_load/stream_load_executor.cpp
+++ b/be/src/runtime/stream_load/stream_load_executor.cpp
@@ -142,14 +142,8 @@ Status 
StreamLoadExecutor::execute_plan_fragment(std::shared_ptr<StreamLoadConte
             }
         }
     };
-
-    if (ctx->put_result.__isset.params) {
-        st = 
_exec_env->fragment_mgr()->exec_plan_fragment(ctx->put_result.params,
-                                                           
QuerySource::STREAM_LOAD, exec_fragment);
-    } else {
-        st = _exec_env->fragment_mgr()->exec_plan_fragment(
-                ctx->put_result.pipeline_params, QuerySource::STREAM_LOAD, 
exec_fragment, parent);
-    }
+    st = _exec_env->fragment_mgr()->exec_plan_fragment(
+            ctx->put_result.pipeline_params, QuerySource::STREAM_LOAD, 
exec_fragment, parent);
 
     if (!st.ok()) {
         // no need to check unref's return value
diff --git a/be/src/service/backend_service.cpp 
b/be/src/service/backend_service.cpp
index b7bd4de1a18..12c091d2c69 100644
--- a/be/src/service/backend_service.cpp
+++ b/be/src/service/backend_service.cpp
@@ -707,69 +707,6 @@ Status BackendService::create_service(StorageEngine& 
engine, ExecEnv* exec_env,
     return Status::OK();
 }
 
-void BaseBackendService::exec_plan_fragment(TExecPlanFragmentResult& 
return_val,
-                                            const TExecPlanFragmentParams& 
params) {
-    LOG(INFO) << "exec_plan_fragment() instance_id=" << 
print_id(params.params.fragment_instance_id)
-              << " coord=" << params.coord << " backend#=" << 
params.backend_num;
-    return_val.__set_status(start_plan_fragment_execution(params).to_thrift());
-}
-
-Status BaseBackendService::start_plan_fragment_execution(
-        const TExecPlanFragmentParams& exec_params) {
-    if (!exec_params.fragment.__isset.output_sink) {
-        return Status::InternalError("missing sink in plan fragment");
-    }
-    return _exec_env->fragment_mgr()->exec_plan_fragment(exec_params,
-                                                         
QuerySource::INTERNAL_FRONTEND);
-}
-
-void BaseBackendService::submit_export_task(TStatus& t_status, const 
TExportTaskRequest& request) {
-    //    VLOG_ROW << "submit_export_task. request  is "
-    //            << apache::thrift::ThriftDebugString(request).c_str();
-    //
-    //    Status status = _exec_env->export_task_mgr()->start_task(request);
-    //    if (status.ok()) {
-    //        VLOG_RPC << "start export task successful id="
-    //            << request.params.params.fragment_instance_id;
-    //    } else {
-    //        VLOG_RPC << "start export task failed id="
-    //            << request.params.params.fragment_instance_id
-    //            << " and err_msg=" << status;
-    //    }
-    //    status.to_thrift(&t_status);
-}
-
-void BaseBackendService::get_export_status(TExportStatusResult& result, const 
TUniqueId& task_id) {
-    //    VLOG_ROW << "get_export_status. task_id  is " << task_id;
-    //    Status status = 
_exec_env->export_task_mgr()->get_task_state(task_id, &result);
-    //    if (!status.ok()) {
-    //        LOG(WARNING) << "get export task state failed. [id=" << task_id 
<< "]";
-    //    } else {
-    //        VLOG_RPC << "get export task state successful. [id=" << task_id
-    //            << ",status=" << result.status.status_code
-    //            << ",state=" << result.state
-    //            << ",files=";
-    //        for (auto& item : result.files) {
-    //            VLOG_RPC << item << ", ";
-    //        }
-    //        VLOG_RPC << "]";
-    //    }
-    //    status.to_thrift(&result.status);
-    //    result.__set_state(TExportState::RUNNING);
-}
-
-void BaseBackendService::erase_export_task(TStatus& t_status, const TUniqueId& 
task_id) {
-    //    VLOG_ROW << "erase_export_task. task_id  is " << task_id;
-    //    Status status = _exec_env->export_task_mgr()->erase_task(task_id);
-    //    if (!status.ok()) {
-    //        LOG(WARNING) << "delete export task failed. because "
-    //            << status << " with task_id " << task_id;
-    //    } else {
-    //        VLOG_RPC << "delete export task successful with task_id " << 
task_id;
-    //    }
-    //    status.to_thrift(&t_status);
-}
-
 void BackendService::get_tablet_stat(TTabletStatResult& result) {
     _engine.tablet_manager()->get_tablet_stat(&result);
 }
diff --git a/be/src/service/backend_service.h b/be/src/service/backend_service.h
index 1469bc2faa9..75aff2d34eb 100644
--- a/be/src/service/backend_service.h
+++ b/be/src/service/backend_service.h
@@ -36,15 +36,8 @@ class ThriftServer;
 class TAgentResult;
 class TAgentTaskRequest;
 class TAgentPublishRequest;
-class TExecPlanFragmentParams;
-class TExecPlanFragmentResult;
-class TCancelPlanFragmentResult;
-class TTransmitDataResult;
-class TExportTaskRequest;
-class TExportStatusResult;
 class TStreamLoadRecordResult;
 class TDiskTrashInfo;
-class TCancelPlanFragmentParams;
 class TCheckStorageFormatResult;
 class TRoutineLoadTask;
 class TScanBatchResult;
@@ -56,7 +49,6 @@ class TScanOpenResult;
 class TSnapshotRequest;
 class TStatus;
 class TTabletStatResult;
-class TTransmitDataParams;
 class TUniqueId;
 class TIngestBinlogRequest;
 class TIngestBinlogResult;
@@ -85,19 +77,6 @@ public:
         
_agent_server->get_topic_subscriber()->handle_topic_info(topic_request);
     }
 
-    // DorisServer service
-    void exec_plan_fragment(TExecPlanFragmentResult& return_val,
-                            const TExecPlanFragmentParams& params) override;
-
-    void cancel_plan_fragment(TCancelPlanFragmentResult& return_val,
-                              const TCancelPlanFragmentParams& params) 
override {};
-
-    void submit_export_task(TStatus& t_status, const TExportTaskRequest& 
request) override;
-
-    void get_export_status(TExportStatusResult& result, const TUniqueId& 
task_id) override;
-
-    void erase_export_task(TStatus& t_status, const TUniqueId& task_id) 
override;
-
     void submit_routine_load_task(TStatus& t_status,
                                   const std::vector<TRoutineLoadTask>& tasks) 
override;
 
@@ -162,8 +141,6 @@ public:
     void stop_works() { _agent_server->stop_report_workers(); }
 
 protected:
-    Status start_plan_fragment_execution(const TExecPlanFragmentParams& 
exec_params);
-
     void get_stream_load_record(TStreamLoadRecordResult& result, int64_t 
last_stream_record_time,
                                 std::shared_ptr<StreamLoadRecorder> 
stream_load_recorder);
 
diff --git a/be/src/service/internal_service.cpp 
b/be/src/service/internal_service.cpp
index 1c49013197e..ab88e389473 100644
--- a/be/src/service/internal_service.cpp
+++ b/be/src/service/internal_service.cpp
@@ -532,52 +532,9 @@ Status PInternalService::_exec_plan_fragment_impl(
                 "Have not receive the first heartbeat message from master, not 
ready to provide "
                 "service");
     }
-    if (version == PFragmentRequestVersion::VERSION_1) {
-        // VERSION_1 should be removed in v1.2
-        TExecPlanFragmentParams t_request;
-        {
-            const uint8_t* buf = (const uint8_t*)ser_request.data();
-            uint32_t len = ser_request.size();
-            RETURN_IF_ERROR(deserialize_thrift_msg(buf, &len, compact, 
&t_request));
-        }
-        if (cb) {
-            return _exec_env->fragment_mgr()->exec_plan_fragment(
-                    t_request, QuerySource::INTERNAL_FRONTEND, cb);
-        } else {
-            return _exec_env->fragment_mgr()->exec_plan_fragment(t_request,
-                                                                 
QuerySource::INTERNAL_FRONTEND);
-        }
-    } else if (version == PFragmentRequestVersion::VERSION_2) {
-        TExecPlanFragmentParamsList t_request;
-        {
-            const uint8_t* buf = (const uint8_t*)ser_request.data();
-            uint32_t len = ser_request.size();
-            RETURN_IF_ERROR(deserialize_thrift_msg(buf, &len, compact, 
&t_request));
-        }
-        const auto& fragment_list = t_request.paramsList;
-        MonotonicStopWatch timer;
-        timer.start();
-
-        for (const TExecPlanFragmentParams& params : t_request.paramsList) {
-            if (cb) {
-                RETURN_IF_ERROR(_exec_env->fragment_mgr()->exec_plan_fragment(
-                        params, QuerySource::INTERNAL_FRONTEND, cb));
-            } else {
-                RETURN_IF_ERROR(_exec_env->fragment_mgr()->exec_plan_fragment(
-                        params, QuerySource::INTERNAL_FRONTEND));
-            }
-        }
-
-        timer.stop();
-        double cost_secs = static_cast<double>(timer.elapsed_time()) / 
1000000000ULL;
-        if (cost_secs > 5) {
-            LOG_WARNING("Prepare {} fragments of query {} costs {} seconds, it 
costs too much",
-                        fragment_list.size(), 
print_id(fragment_list.front().params.query_id),
-                        cost_secs);
-        }
-
-        return Status::OK();
-    } else if (version == PFragmentRequestVersion::VERSION_3) {
+    CHECK(version == PFragmentRequestVersion::VERSION_3)
+            << "only support version 3, received " << version;
+    if (version == PFragmentRequestVersion::VERSION_3) {
         TPipelineFragmentParamsList t_request;
         {
             const uint8_t* buf = (const uint8_t*)ser_request.data();
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/rpc/BackendServiceProxy.java 
b/fe/fe-core/src/main/java/org/apache/doris/rpc/BackendServiceProxy.java
index a3af39bd28d..7e15a050393 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/rpc/BackendServiceProxy.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/rpc/BackendServiceProxy.java
@@ -31,7 +31,6 @@ import 
org.apache.doris.proto.InternalService.PGetWalQueueSizeResponse;
 import org.apache.doris.proto.InternalService.PGroupCommitInsertRequest;
 import org.apache.doris.proto.InternalService.PGroupCommitInsertResponse;
 import org.apache.doris.proto.Types;
-import org.apache.doris.thrift.TExecPlanFragmentParamsList;
 import org.apache.doris.thrift.TFoldConstantParams;
 import org.apache.doris.thrift.TNetworkAddress;
 import org.apache.doris.thrift.TPipelineFragmentParamsList;
@@ -158,38 +157,6 @@ public class BackendServiceProxy {
         }
     }
 
-    public Future<InternalService.PExecPlanFragmentResult> 
execPlanFragmentsAsync(TNetworkAddress address,
-            TExecPlanFragmentParamsList paramsList, boolean twoPhaseExecution) 
throws TException, RpcException {
-        InternalService.PExecPlanFragmentRequest.Builder builder =
-                InternalService.PExecPlanFragmentRequest.newBuilder();
-        if (Config.use_compact_thrift_rpc) {
-            builder.setRequest(
-                    ByteString.copyFrom(new TSerializer(new 
TCompactProtocol.Factory()).serialize(paramsList)));
-            builder.setCompact(true);
-        } else {
-            builder.setRequest(ByteString.copyFrom(new 
TSerializer().serialize(paramsList))).build();
-            builder.setCompact(false);
-        }
-        // VERSION 2 means we send TExecPlanFragmentParamsList, not single 
TExecPlanFragmentParams
-        builder.setVersion(InternalService.PFragmentRequestVersion.VERSION_2);
-
-        final InternalService.PExecPlanFragmentRequest pRequest = 
builder.build();
-        
MetricRepo.BE_COUNTER_QUERY_RPC_ALL.getOrAdd(address.hostname).increase(1L);
-        
MetricRepo.BE_COUNTER_QUERY_RPC_SIZE.getOrAdd(address.hostname).increase((long) 
pRequest.getSerializedSize());
-        try {
-            final BackendServiceClient client = getProxy(address);
-            if (twoPhaseExecution) {
-                return client.execPlanFragmentPrepareAsync(pRequest);
-            } else {
-                return client.execPlanFragmentAsync(pRequest);
-            }
-        } catch (Throwable e) {
-            LOG.warn("Execute plan fragment catch a exception, address={}:{}", 
address.getHostname(), address.getPort(),
-                    e);
-            throw new RpcException(address.hostname, e.getMessage());
-        }
-    }
-
     public Future<InternalService.PExecPlanFragmentResult> 
execPlanFragmentsAsync(TNetworkAddress address,
             TPipelineFragmentParamsList params, boolean twoPhaseExecution) 
throws TException, RpcException {
         InternalService.PExecPlanFragmentRequest.Builder builder =
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java 
b/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java
index 5f5af95deb1..d8cee22b962 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java
@@ -134,7 +134,6 @@ import org.apache.doris.system.Frontend;
 import org.apache.doris.system.SystemInfoService;
 import org.apache.doris.tablefunction.MetadataGenerator;
 import org.apache.doris.thrift.FrontendService;
-import org.apache.doris.thrift.FrontendServiceVersion;
 import org.apache.doris.thrift.TAddPlsqlPackageRequest;
 import org.apache.doris.thrift.TAddPlsqlStoredProcedureRequest;
 import org.apache.doris.thrift.TAutoIncrementRangeRequest;
@@ -160,7 +159,6 @@ import org.apache.doris.thrift.TDropPlsqlPackageRequest;
 import org.apache.doris.thrift.TDropPlsqlStoredProcedureRequest;
 import org.apache.doris.thrift.TEncryptionAlgorithm;
 import org.apache.doris.thrift.TEncryptionKey;
-import org.apache.doris.thrift.TFeResult;
 import org.apache.doris.thrift.TFetchResourceResult;
 import org.apache.doris.thrift.TFetchRoutineLoadJobRequest;
 import org.apache.doris.thrift.TFetchRoutineLoadJobResult;
@@ -275,7 +273,6 @@ import org.apache.doris.thrift.TTableStatus;
 import org.apache.doris.thrift.TTabletLocation;
 import org.apache.doris.thrift.TTxnParams;
 import org.apache.doris.thrift.TUniqueId;
-import org.apache.doris.thrift.TUpdateExportTaskStatusRequest;
 import org.apache.doris.thrift.TUpdateFollowerPartitionStatsCacheRequest;
 import org.apache.doris.thrift.TUpdateFollowerStatsCacheRequest;
 import org.apache.doris.thrift.TUpdatePlanStatsCacheRequest;
@@ -862,13 +859,6 @@ public class FrontendServiceImpl implements 
FrontendService.Iface {
         return result;
     }
 
-    @Override
-    public TFeResult updateExportTaskStatus(TUpdateExportTaskStatusRequest 
request) throws TException {
-        TStatus status = new TStatus(TStatusCode.OK);
-        TFeResult result = new TFeResult(FrontendServiceVersion.V1, status);
-        return result;
-    }
-
     @Override
     public TDescribeTablesResult describeTables(TDescribeTablesParams params) 
throws TException {
         if (LOG.isDebugEnabled()) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/task/AgentClient.java 
b/fe/fe-core/src/main/java/org/apache/doris/task/AgentClient.java
index d364b9fcbe8..0ae1bd8b541 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/task/AgentClient.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/task/AgentClient.java
@@ -18,16 +18,11 @@
 package org.apache.doris.task;
 
 import org.apache.doris.common.ClientPool;
-import org.apache.doris.common.Status;
 import org.apache.doris.thrift.BackendService;
 import org.apache.doris.thrift.TAgentResult;
 import org.apache.doris.thrift.TCheckStorageFormatResult;
-import org.apache.doris.thrift.TExportStatusResult;
-import org.apache.doris.thrift.TExportTaskRequest;
 import org.apache.doris.thrift.TNetworkAddress;
 import org.apache.doris.thrift.TSnapshotRequest;
-import org.apache.doris.thrift.TStatus;
-import org.apache.doris.thrift.TUniqueId;
 
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
@@ -83,62 +78,6 @@ public class AgentClient {
         return result;
     }
 
-    public Status submitExportTask(TExportTaskRequest request) {
-        Status result = Status.CANCELLED;
-        if (LOG.isDebugEnabled()) {
-            LOG.debug("submit export task. request: {}", request);
-        }
-        try {
-            borrowClient();
-            // submit export task
-            TStatus status = client.submitExportTask(request);
-            result = new Status(status);
-        } catch (Exception e) {
-            LOG.warn("submit export task error", e);
-        } finally {
-            returnClient();
-        }
-        return result;
-    }
-
-    public TExportStatusResult getExportStatus(long jobId, long taskId) {
-        TExportStatusResult result = null;
-        TUniqueId request = new TUniqueId(jobId, taskId);
-        if (LOG.isDebugEnabled()) {
-            LOG.debug("get export task status. request: {}", request);
-        }
-        try {
-            borrowClient();
-            // get export status
-            result = client.getExportStatus(request);
-            ok = true;
-        } catch (Exception e) {
-            LOG.warn("get export status error", e);
-        } finally {
-            returnClient();
-        }
-        return result;
-    }
-
-    public Status eraseExportTask(long jobId, long taskId) {
-        Status result = Status.CANCELLED;
-        TUniqueId request = new TUniqueId(jobId, taskId);
-        if (LOG.isDebugEnabled()) {
-            LOG.debug("erase export task. request: {}", request);
-        }
-        try {
-            borrowClient();
-            // erase export task
-            TStatus status = client.eraseExportTask(request);
-            result = new Status(status);
-        } catch (Exception e) {
-            LOG.warn("submit export task error", e);
-        } finally {
-            returnClient();
-        }
-        return result;
-    }
-
     public TCheckStorageFormatResult checkStorageFormat() {
         TCheckStorageFormatResult result = null;
         if (LOG.isDebugEnabled()) {
diff --git 
a/fe/fe-core/src/test/java/org/apache/doris/common/GenericPoolTest.java 
b/fe/fe-core/src/test/java/org/apache/doris/common/GenericPoolTest.java
index bb32e591426..c5361faa517 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/common/GenericPoolTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/common/GenericPoolTest.java
@@ -21,17 +21,11 @@ import org.apache.doris.thrift.BackendService;
 import org.apache.doris.thrift.TAgentPublishRequest;
 import org.apache.doris.thrift.TAgentResult;
 import org.apache.doris.thrift.TAgentTaskRequest;
-import org.apache.doris.thrift.TCancelPlanFragmentParams;
-import org.apache.doris.thrift.TCancelPlanFragmentResult;
 import org.apache.doris.thrift.TCheckStorageFormatResult;
 import org.apache.doris.thrift.TCheckWarmUpCacheAsyncRequest;
 import org.apache.doris.thrift.TCheckWarmUpCacheAsyncResponse;
 import org.apache.doris.thrift.TDictionaryStatusList;
 import org.apache.doris.thrift.TDiskTrashInfo;
-import org.apache.doris.thrift.TExecPlanFragmentParams;
-import org.apache.doris.thrift.TExecPlanFragmentResult;
-import org.apache.doris.thrift.TExportStatusResult;
-import org.apache.doris.thrift.TExportTaskRequest;
 import org.apache.doris.thrift.TGetRealtimeExecStatusRequest;
 import org.apache.doris.thrift.TGetRealtimeExecStatusResponse;
 import org.apache.doris.thrift.TGetTopNHotPartitionsRequest;
@@ -56,7 +50,6 @@ import org.apache.doris.thrift.TStreamLoadRecordResult;
 import org.apache.doris.thrift.TSyncLoadForTabletsRequest;
 import org.apache.doris.thrift.TSyncLoadForTabletsResponse;
 import org.apache.doris.thrift.TTabletStatResult;
-import org.apache.doris.thrift.TUniqueId;
 import org.apache.doris.thrift.TWarmUpCacheAsyncRequest;
 import org.apache.doris.thrift.TWarmUpCacheAsyncResponse;
 import org.apache.doris.thrift.TWarmUpTabletsRequest;
@@ -124,16 +117,6 @@ public class GenericPoolTest {
             //
         }
 
-        @Override
-        public TExecPlanFragmentResult 
execPlanFragment(TExecPlanFragmentParams params) {
-            return new TExecPlanFragmentResult();
-        }
-
-        @Override
-        public TCancelPlanFragmentResult 
cancelPlanFragment(TCancelPlanFragmentParams params) {
-            return new TCancelPlanFragmentResult();
-        }
-
         @Override
         public TAgentResult submitTasks(List<TAgentTaskRequest> tasks) throws 
TException {
             return null;
@@ -160,24 +143,6 @@ public class GenericPoolTest {
             return null;
         }
 
-        @Override
-        public TStatus submitExportTask(TExportTaskRequest request) throws 
TException {
-            // TODO Auto-generated method stub
-            return null;
-        }
-
-        @Override
-        public TExportStatusResult getExportStatus(TUniqueId taskId) throws 
TException {
-            // TODO Auto-generated method stub
-            return null;
-        }
-
-        @Override
-        public TStatus eraseExportTask(TUniqueId taskId) throws TException {
-            // TODO Auto-generated method stub
-            return null;
-        }
-
         @Override
         public long getTrashUsedCapacity() throws TException {
             // TODO Auto-generated method stub
diff --git 
a/fe/fe-core/src/test/java/org/apache/doris/utframe/MockedBackendFactory.java 
b/fe/fe-core/src/test/java/org/apache/doris/utframe/MockedBackendFactory.java
index be2090f2309..eab542842a2 100644
--- 
a/fe/fe-core/src/test/java/org/apache/doris/utframe/MockedBackendFactory.java
+++ 
b/fe/fe-core/src/test/java/org/apache/doris/utframe/MockedBackendFactory.java
@@ -35,8 +35,6 @@ import org.apache.doris.thrift.TAgentResult;
 import org.apache.doris.thrift.TAgentTaskRequest;
 import org.apache.doris.thrift.TBackend;
 import org.apache.doris.thrift.TBackendInfo;
-import org.apache.doris.thrift.TCancelPlanFragmentParams;
-import org.apache.doris.thrift.TCancelPlanFragmentResult;
 import org.apache.doris.thrift.TCheckStorageFormatResult;
 import org.apache.doris.thrift.TCheckWarmUpCacheAsyncRequest;
 import org.apache.doris.thrift.TCheckWarmUpCacheAsyncResponse;
@@ -45,11 +43,6 @@ import org.apache.doris.thrift.TCreateTabletReq;
 import org.apache.doris.thrift.TDictionaryStatusList;
 import org.apache.doris.thrift.TDiskTrashInfo;
 import org.apache.doris.thrift.TDropTabletReq;
-import org.apache.doris.thrift.TExecPlanFragmentParams;
-import org.apache.doris.thrift.TExecPlanFragmentResult;
-import org.apache.doris.thrift.TExportState;
-import org.apache.doris.thrift.TExportStatusResult;
-import org.apache.doris.thrift.TExportTaskRequest;
 import org.apache.doris.thrift.TFinishTaskRequest;
 import org.apache.doris.thrift.TGetRealtimeExecStatusRequest;
 import org.apache.doris.thrift.TGetRealtimeExecStatusResponse;
@@ -81,7 +74,6 @@ import org.apache.doris.thrift.TSyncLoadForTabletsResponse;
 import org.apache.doris.thrift.TTabletInfo;
 import org.apache.doris.thrift.TTabletStatResult;
 import org.apache.doris.thrift.TTaskType;
-import org.apache.doris.thrift.TUniqueId;
 import org.apache.doris.thrift.TWarmUpCacheAsyncRequest;
 import org.apache.doris.thrift.TWarmUpCacheAsyncResponse;
 import org.apache.doris.thrift.TWarmUpTabletsRequest;
@@ -355,16 +347,6 @@ public class MockedBackendFactory {
             }).start();
         }
 
-        @Override
-        public TExecPlanFragmentResult 
execPlanFragment(TExecPlanFragmentParams params) throws TException {
-            return null;
-        }
-
-        @Override
-        public TCancelPlanFragmentResult 
cancelPlanFragment(TCancelPlanFragmentParams params) throws TException {
-            return null;
-        }
-
         @Override
         public TAgentResult submitTasks(List<TAgentTaskRequest> tasks) throws 
TException {
             for (TAgentTaskRequest request : tasks) {
@@ -396,21 +378,6 @@ public class MockedBackendFactory {
             return new TPublishTopicResult(new TStatus(TStatusCode.OK));
         }
 
-        @Override
-        public TStatus submitExportTask(TExportTaskRequest request) throws 
TException {
-            return new TStatus(TStatusCode.OK);
-        }
-
-        @Override
-        public TExportStatusResult getExportStatus(TUniqueId taskId) throws 
TException {
-            return new TExportStatusResult(new TStatus(TStatusCode.OK), 
TExportState.FINISHED);
-        }
-
-        @Override
-        public TStatus eraseExportTask(TUniqueId taskId) throws TException {
-            return new TStatus(TStatusCode.OK);
-        }
-
         @Override
         public long getTrashUsedCapacity() throws TException {
             return 0L;
diff --git a/gensrc/thrift/BackendService.thrift 
b/gensrc/thrift/BackendService.thrift
index fccd7569868..6d02e0c6920 100644
--- a/gensrc/thrift/BackendService.thrift
+++ b/gensrc/thrift/BackendService.thrift
@@ -26,10 +26,6 @@ include "PaloInternalService.thrift"
 include "DorisExternalService.thrift"
 include "FrontendService.thrift"
 
-struct TExportTaskRequest {
-    1: required PaloInternalService.TExecPlanFragmentParams params
-}
-
 struct TTabletStat {
     1: required i64 tablet_id
     // local data size = local inverted index file size + local segment file 
size
@@ -71,9 +67,9 @@ struct TRoutineLoadTask {
     10: optional i64 max_batch_rows
     11: optional i64 max_batch_size
     12: optional TKafkaLoadInfo kafka_load_info
-    13: optional PaloInternalService.TExecPlanFragmentParams params
+    // 13: optional PaloInternalService.TExecPlanFragmentParams params # 
deprecated
     14: optional PlanNodes.TFileFormatType format
-    15: optional PaloInternalService.TPipelineFragmentParams pipeline_params
+    15: optional PaloInternalService.TPipelineFragmentParams pipeline_params 
     16: optional bool is_multi_table
     17: optional bool memtable_on_sink_node;
     18: optional string qualified_user
@@ -373,16 +369,6 @@ struct TDictionaryStatusList {
 }
 
 service BackendService {
-    // Called by coord to start asynchronous execution of plan fragment in 
backend.
-    // Returns as soon as all incoming data streams have been set up.
-    PaloInternalService.TExecPlanFragmentResult 
exec_plan_fragment(1:PaloInternalService.TExecPlanFragmentParams params);
-
-    // Called by coord to cancel execution of a single plan fragment, which 
this
-    // coordinator initiated with a prior call to ExecPlanFragment.
-    // Cancellation is asynchronous.
-    PaloInternalService.TCancelPlanFragmentResult cancel_plan_fragment(
-        1:PaloInternalService.TCancelPlanFragmentParams params);
-
     AgentService.TAgentResult 
submit_tasks(1:list<AgentService.TAgentTaskRequest> tasks);
 
     AgentService.TAgentResult make_snapshot(1:AgentService.TSnapshotRequest 
snapshot_request);
@@ -391,12 +377,6 @@ service BackendService {
 
     AgentService.TAgentResult 
publish_cluster_state(1:AgentService.TAgentPublishRequest request);
 
-    Status.TStatus submit_export_task(1:TExportTaskRequest request);
-
-    PaloInternalService.TExportStatusResult 
get_export_status(1:Types.TUniqueId task_id);
-
-    Status.TStatus erase_export_task(1:Types.TUniqueId task_id);
-
     TTabletStatResult get_tablet_stat();
 
     i64 get_trash_used_capacity();
diff --git a/gensrc/thrift/FrontendService.thrift 
b/gensrc/thrift/FrontendService.thrift
index f52008c53a8..2c735340de8 100644
--- a/gensrc/thrift/FrontendService.thrift
+++ b/gensrc/thrift/FrontendService.thrift
@@ -424,12 +424,6 @@ struct TMasterOpResult {
     11: optional i64 affectedRows;
 }
 
-struct TUpdateExportTaskStatusRequest {
-    1: required FrontendServiceVersion protocolVersion
-    2: required Types.TUniqueId taskId
-    3: required PaloInternalService.TExportStatusResult taskStatus
-}
-
 struct TLoadTxnBeginRequest {
     1: optional string cluster
     2: required string user
@@ -567,7 +561,7 @@ struct TStreamLoadPutRequest {
 struct TStreamLoadPutResult {
     1: required Status.TStatus status
     // valid when status is OK
-    2: optional PaloInternalService.TExecPlanFragmentParams params
+    //2: optional PaloInternalService.TExecPlanFragmentParams params # 
deprecated
     3: optional PaloInternalService.TPipelineFragmentParams pipeline_params
     // used for group commit
     4: optional i64 base_schema_version
@@ -581,7 +575,7 @@ struct TStreamLoadPutResult {
 struct TStreamLoadMultiTablePutResult {
     1: required Status.TStatus status
     // valid when status is OK
-    2: optional list<PaloInternalService.TExecPlanFragmentParams> params
+    // 2: optional list<PaloInternalService.TExecPlanFragmentParams> params # 
deprecated
     3: optional list<PaloInternalService.TPipelineFragmentParams> 
pipeline_params
 }
 
@@ -1665,8 +1659,6 @@ service FrontendService {
     TListPrivilegesResult listSchemaPrivilegeStatus(1: TGetTablesParams params)
     TListPrivilegesResult listUserPrivilegeStatus(1: TGetTablesParams params)
 
-    TFeResult updateExportTaskStatus(1: TUpdateExportTaskStatusRequest request)
-
     TLoadTxnBeginResult loadTxnBegin(1: TLoadTxnBeginRequest request)
     TLoadTxnCommitResult loadTxnPreCommit(1: TLoadTxnCommitRequest request)
     TLoadTxn2PCResult loadTxn2PC(1: TLoadTxn2PCRequest request)
diff --git a/gensrc/thrift/PaloInternalService.thrift 
b/gensrc/thrift/PaloInternalService.thrift
index b0456913037..6e749c266c8 100644
--- a/gensrc/thrift/PaloInternalService.thrift
+++ b/gensrc/thrift/PaloInternalService.thrift
@@ -459,33 +459,8 @@ struct TPlanFragmentExecParams {
   // a globally unique id assigned to this particular execution instance of
   // a TPlanFragment
   2: required Types.TUniqueId fragment_instance_id
-
-  // initial scan ranges for each scan node in TPlanFragment.plan_tree
-  3: required map<Types.TPlanNodeId, list<TScanRangeParams>> 
per_node_scan_ranges
-
-  // number of senders for ExchangeNodes contained in TPlanFragment.plan_tree;
-  // needed to create a DataStreamRecvr
-  4: required map<Types.TPlanNodeId, i32> per_exch_num_senders
-
-  // Output destinations, one per output partition.
-  // The partitioning of the output is specified by
-  // TPlanFragment.output_sink.output_partition.
-  // The number of output partitions is destinations.size().
-  5: list<DataSinks.TPlanFragmentDestination> destinations
-
-  // Debug options: perform some action in a particular phase of a particular 
node
-  // 6: optional Types.TPlanNodeId debug_node_id // Never used
-  // 7: optional PlanNodes.TExecNodePhase debug_phase // Never used
-  // 8: optional PlanNodes.TDebugAction debug_action // Never used
-
-  // Id of this fragment in its role as a sender.
-  9: optional i32 sender_id
-  10: optional i32 num_senders
-  11: optional bool send_query_statistics_with_every_batch
-  // Used to merge and send runtime filter
-  12: optional TRuntimeFilterParams runtime_filter_params //deprecated
-  13: optional bool group_commit // deprecated
-  14: optional list<i32> topn_filter_source_node_ids //deprecated
+  // all fields before 14 are deleted
+  // 14: optional list<i32> topn_filter_source_node_ids //deprecated
 }
 
 // Global query parameters assigned by the coordinator.
@@ -549,122 +524,6 @@ struct TPipelineWorkloadGroup {
   4: optional i64 version
 }
 
-// ExecPlanFragment
-struct TExecPlanFragmentParams {
-  1: required PaloInternalServiceVersion protocol_version
-
-  // required in V1
-  2: optional Planner.TPlanFragment fragment
-
-  // required in V1
-  // @Common components
-  3: optional Descriptors.TDescriptorTable desc_tbl
-
-  // required in V1
-  4: optional TPlanFragmentExecParams params
-
-  // Initiating coordinator.
-  // TODO: determine whether we can get this somehow via the Thrift rpc 
mechanism.
-  // required in V1
-  // @Common components
-  5: optional Types.TNetworkAddress coord
-
-  // backend number assigned by coord to identify backend
-  // required in V1
-  6: optional i32 backend_num
-
-  // Global query parameters assigned by coordinator.
-  // required in V1
-  // @Common components
-  7: optional TQueryGlobals query_globals
-
-  // options for the query
-  // required in V1
-  8: optional TQueryOptions query_options
-
-  // Whether reportd when the backend fails
-  // required in V1
-  9: optional bool is_report_success
-
-  // required in V1
-  // @Common components
-  // Deprecated
-  10: optional Types.TResourceInfo resource_info
-
-  // load job related
-  11: optional string import_label
-  12: optional string db_name
-  13: optional i64 load_job_id
-  14: optional TLoadErrorHubInfo load_error_hub_info
-
-  // The total number of fragments on same BE host
-  15: optional i32 fragment_num_on_host
-
-  // If true, all @Common components is unset and should be got from BE's cache
-  // If this field is unset or it set to false, all @Common components is set.
-  16: optional bool is_simplified_param = false;
-  17: optional TTxnParams txn_conf
-  18: optional i64 backend_id
-  19: optional TGlobalDict global_dict  // scan node could use the global dict 
to encode the string value to an integer
-
-  // If it is true, after this fragment is prepared on the BE side,
-  // it will wait for the FE to send the "start execution" command before it 
is actually executed.
-  // Otherwise, the fragment will start executing directly on the BE side.
-  20: optional bool need_wait_execution_trigger = false;
-
-  // deprecated
-  21: optional bool build_hash_table_for_broadcast_join = false;
-
-  22: optional list<Types.TUniqueId> instances_sharing_hash_table;
-  23: optional string table_name;
-
-  // scan node id -> scan range params, only for external file scan
-  24: optional map<Types.TPlanNodeId, PlanNodes.TFileScanRangeParams> 
file_scan_params
-
-  25: optional i64 wal_id
-
-  // num load stream for each sink backend
-  26: optional i32 load_stream_per_node
-
-  // total num of load streams the downstream backend will see
-  27: optional i32 total_load_streams
-
-  28: optional i32 num_local_sink
-
-  29: optional i64 content_length
-
-  30: optional list<TPipelineWorkloadGroup> workload_groups
-
-  31: optional bool is_nereids = true;
-
-  32: optional Types.TNetworkAddress current_connect_fe
-
-  // For cloud
-  1000: optional bool is_mow_table;
-}
-
-struct TExecPlanFragmentParamsList {
-    1: optional list<TExecPlanFragmentParams> paramsList;
-}
-
-struct TExecPlanFragmentResult {
-  // required in V1
-  1: optional Status.TStatus status
-}
-
-// CancelPlanFragment
-struct TCancelPlanFragmentParams {
-  1: required PaloInternalServiceVersion protocol_version
-
-  // required in V1
-  2: optional Types.TUniqueId fragment_instance_id
-}
-
-struct TCancelPlanFragmentResult {
-  // required in V1
-  1: optional Status.TStatus status
-}
-
 struct TFoldConstantParams {
   1: required map<string, map<string, Exprs.TExpr>> expr_map
   2: required TQueryGlobals query_globals
@@ -674,38 +533,6 @@ struct TFoldConstantParams {
   6: optional bool is_nereids
 }
 
-// TransmitData
-struct TTransmitDataParams {
-  1: required PaloInternalServiceVersion protocol_version
-
-  // required in V1
-  2: optional Types.TUniqueId dest_fragment_instance_id
-
-  // for debugging purposes; currently ignored
-  //3: optional Types.TUniqueId src_fragment_instance_id
-
-  // required in V1
-  4: optional Types.TPlanNodeId dest_node_id
-
-  // if set to true, indicates that no more row batches will be sent
-  // for this dest_node_id
-  6: optional bool eos
-
-  7: optional i32 be_number
-  8: optional i64 packet_seq
-
-  // Id of this fragment in its role as a sender.
-  9: optional i32 sender_id
-}
-
-struct TTransmitDataResult {
-  // required in V1
-  1: optional Status.TStatus status
-  2: optional i64 packet_seq
-  3: optional Types.TUniqueId dest_fragment_instance_id
-  4: optional Types.TPlanNodeId dest_node_id
-}
-
 struct TTabletWithPartition {
     1: required i64 partition_id
     2: required i64 tablet_id
@@ -756,12 +583,6 @@ struct TCondition {
     1000: optional TCompoundType compound_type = TCompoundType.UNKNOWN
 }
 
-struct TExportStatusResult {
-    1: required Status.TStatus status
-    2: required Types.TExportState state
-    3: optional list<string> files
-}
-
 struct TPipelineInstanceParams {
   1: required Types.TUniqueId fragment_instance_id
   // deprecated
diff --git a/gensrc/thrift/Types.thrift b/gensrc/thrift/Types.thrift
index 0600caeadec..34d222087af 100644
--- a/gensrc/thrift/Types.thrift
+++ b/gensrc/thrift/Types.thrift
@@ -685,13 +685,6 @@ struct TResourceInfo {
     2: required string group
 }
 
-enum TExportState {
-    RUNNING = 0,
-    FINISHED = 1,
-    CANCELLED = 2,
-    UNKNOWN = 3
-}
-
 enum TFileType {
     FILE_LOCAL = 0,
     FILE_BROKER = 1,


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]


Reply via email to