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]