github-actions[bot] commented on code in PR #27067:
URL: https://github.com/apache/doris/pull/27067#discussion_r1394185939


##########
be/src/pipeline/exec/olap_table_sink_v2_operator.cpp:
##########
@@ -0,0 +1,54 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "olap_table_sink_v2_operator.h"
+
+#include "common/status.h"
+
+namespace doris {
+class DataSink;
+} // namespace doris
+
+namespace doris::pipeline {
+
+OperatorPtr OlapTableSinkV2OperatorBuilder::build_operator() {
+    return std::make_shared<OlapTableSinkV2Operator>(this, _sink);
+}
+
+Status OlapTableSinkV2LocalState::init(RuntimeState* state, 
LocalSinkStateInfo& info) {

Review Comment:
   warning: method 'init' can be made static 
[readability-convert-member-functions-to-static]
   
   be/src/pipeline/exec/olap_table_sink_v2_operator.h:54:
   ```diff
   -     Status init(RuntimeState* state, LocalSinkStateInfo& info) override;
   +     static Status init(RuntimeState* state, LocalSinkStateInfo& info) 
override;
   ```
   



##########
be/src/pipeline/exec/olap_table_sink_v2_operator.cpp:
##########
@@ -0,0 +1,54 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "olap_table_sink_v2_operator.h"
+
+#include "common/status.h"
+
+namespace doris {
+class DataSink;
+} // namespace doris
+
+namespace doris::pipeline {
+
+OperatorPtr OlapTableSinkV2OperatorBuilder::build_operator() {
+    return std::make_shared<OlapTableSinkV2Operator>(this, _sink);
+}
+
+Status OlapTableSinkV2LocalState::init(RuntimeState* state, 
LocalSinkStateInfo& info) {
+    RETURN_IF_ERROR(Base::init(state, info));
+    SCOPED_TIMER(exec_time_counter());
+    SCOPED_TIMER(_open_timer);
+    auto& p = _parent->cast<Parent>();
+    RETURN_IF_ERROR(_writer->init_properties(p._pool, p._group_commit));
+    return Status::OK();
+}
+
+Status OlapTableSinkV2LocalState::close(RuntimeState* state, Status 
exec_status) {

Review Comment:
   warning: method 'close' can be made static 
[readability-convert-member-functions-to-static]
   
   be/src/pipeline/exec/olap_table_sink_v2_operator.h:61:
   ```diff
   -     Status close(RuntimeState* state, Status exec_status) override;
   +     static Status close(RuntimeState* state, Status exec_status) override;
   ```
   



##########
be/src/pipeline/exec/olap_table_sink_v2_operator.h:
##########
@@ -41,9 +42,76 @@ class OlapTableSinkV2Operator final : public 
DataSinkOperator<OlapTableSinkV2Ope
     bool can_write() override { return true; } // TODO: need use mem_limit
 };
 
-OperatorPtr OlapTableSinkV2OperatorBuilder::build_operator() {
-    return std::make_shared<OlapTableSinkV2Operator>(this, _sink);
-}
+class OlapTableSinkV2OperatorX;
+
+class OlapTableSinkV2LocalState final
+        : public AsyncWriterSink<vectorized::VTabletWriterV2, 
OlapTableSinkV2OperatorX> {
+public:
+    using Base = AsyncWriterSink<vectorized::VTabletWriterV2, 
OlapTableSinkV2OperatorX>;
+    using Parent = OlapTableSinkV2OperatorX;
+    ENABLE_FACTORY_CREATOR(OlapTableSinkV2LocalState);
+    OlapTableSinkV2LocalState(DataSinkOperatorXBase* parent, RuntimeState* 
state)
+            : Base(parent, state) {};
+    Status init(RuntimeState* state, LocalSinkStateInfo& info) override;
+    Status open(RuntimeState* state) override {

Review Comment:
   warning: method 'open' can be made static 
[readability-convert-member-functions-to-static]
   
   ```suggestion
       static Status open(RuntimeState* state) override {
   ```
   



##########
be/src/pipeline/exec/olap_table_sink_v2_operator.h:
##########
@@ -41,9 +42,76 @@
     bool can_write() override { return true; } // TODO: need use mem_limit
 };
 
-OperatorPtr OlapTableSinkV2OperatorBuilder::build_operator() {
-    return std::make_shared<OlapTableSinkV2Operator>(this, _sink);
-}
+class OlapTableSinkV2OperatorX;
+
+class OlapTableSinkV2LocalState final
+        : public AsyncWriterSink<vectorized::VTabletWriterV2, 
OlapTableSinkV2OperatorX> {
+public:
+    using Base = AsyncWriterSink<vectorized::VTabletWriterV2, 
OlapTableSinkV2OperatorX>;
+    using Parent = OlapTableSinkV2OperatorX;
+    ENABLE_FACTORY_CREATOR(OlapTableSinkV2LocalState);
+    OlapTableSinkV2LocalState(DataSinkOperatorXBase* parent, RuntimeState* 
state)
+            : Base(parent, state) {};
+    Status init(RuntimeState* state, LocalSinkStateInfo& info) override;
+    Status open(RuntimeState* state) override {
+        SCOPED_TIMER(exec_time_counter());
+        SCOPED_TIMER(_open_timer);
+        return Base::open(state);
+    }
+
+    Status close(RuntimeState* state, Status exec_status) override;
+    friend class OlapTableSinkV2OperatorX;
+
+private:
+    Status _close_status = Status::OK();
+};
+
+class OlapTableSinkV2OperatorX final : public 
DataSinkOperatorX<OlapTableSinkV2LocalState> {
+public:
+    using Base = DataSinkOperatorX<OlapTableSinkV2LocalState>;
+    OlapTableSinkV2OperatorX(ObjectPool* pool, int operator_id, const 
RowDescriptor& row_desc,
+                             const std::vector<TExpr>& t_output_expr, bool 
group_commit)
+            : Base(operator_id, 0),
+              _row_desc(row_desc),
+              _t_output_expr(t_output_expr),
+              _group_commit(group_commit),
+              _pool(pool) {};
+
+    Status init(const TDataSink& thrift_sink) override {

Review Comment:
   warning: method 'init' can be made static 
[readability-convert-member-functions-to-static]
   
   ```suggestion
       static Status init(const TDataSink& thrift_sink) override {
   ```
   



##########
be/src/pipeline/exec/olap_table_sink_v2_operator.h:
##########
@@ -41,9 +42,76 @@
     bool can_write() override { return true; } // TODO: need use mem_limit
 };
 
-OperatorPtr OlapTableSinkV2OperatorBuilder::build_operator() {
-    return std::make_shared<OlapTableSinkV2Operator>(this, _sink);
-}
+class OlapTableSinkV2OperatorX;
+
+class OlapTableSinkV2LocalState final
+        : public AsyncWriterSink<vectorized::VTabletWriterV2, 
OlapTableSinkV2OperatorX> {
+public:
+    using Base = AsyncWriterSink<vectorized::VTabletWriterV2, 
OlapTableSinkV2OperatorX>;
+    using Parent = OlapTableSinkV2OperatorX;
+    ENABLE_FACTORY_CREATOR(OlapTableSinkV2LocalState);
+    OlapTableSinkV2LocalState(DataSinkOperatorXBase* parent, RuntimeState* 
state)
+            : Base(parent, state) {};
+    Status init(RuntimeState* state, LocalSinkStateInfo& info) override;
+    Status open(RuntimeState* state) override {
+        SCOPED_TIMER(exec_time_counter());
+        SCOPED_TIMER(_open_timer);
+        return Base::open(state);
+    }
+
+    Status close(RuntimeState* state, Status exec_status) override;
+    friend class OlapTableSinkV2OperatorX;
+
+private:
+    Status _close_status = Status::OK();
+};
+
+class OlapTableSinkV2OperatorX final : public 
DataSinkOperatorX<OlapTableSinkV2LocalState> {
+public:
+    using Base = DataSinkOperatorX<OlapTableSinkV2LocalState>;
+    OlapTableSinkV2OperatorX(ObjectPool* pool, int operator_id, const 
RowDescriptor& row_desc,
+                             const std::vector<TExpr>& t_output_expr, bool 
group_commit)
+            : Base(operator_id, 0),
+              _row_desc(row_desc),
+              _t_output_expr(t_output_expr),
+              _group_commit(group_commit),
+              _pool(pool) {};
+
+    Status init(const TDataSink& thrift_sink) override {
+        RETURN_IF_ERROR(Base::init(thrift_sink));
+        // From the thrift expressions create the real exprs.
+        RETURN_IF_ERROR(vectorized::VExpr::create_expr_trees(_t_output_expr, 
_output_vexpr_ctxs));
+        return Status::OK();
+    }
+
+    Status prepare(RuntimeState* state) override {
+        RETURN_IF_ERROR(Base::prepare(state));
+        return vectorized::VExpr::prepare(_output_vexpr_ctxs, state, 
_row_desc);
+    }
+
+    Status open(RuntimeState* state) override {

Review Comment:
   warning: method 'open' can be made static 
[readability-convert-member-functions-to-static]
   
   ```suggestion
       static Status open(RuntimeState* state) override {
   ```
   



##########
be/src/pipeline/exec/olap_table_sink_v2_operator.h:
##########
@@ -41,9 +42,76 @@
     bool can_write() override { return true; } // TODO: need use mem_limit
 };
 
-OperatorPtr OlapTableSinkV2OperatorBuilder::build_operator() {
-    return std::make_shared<OlapTableSinkV2Operator>(this, _sink);
-}
+class OlapTableSinkV2OperatorX;
+
+class OlapTableSinkV2LocalState final
+        : public AsyncWriterSink<vectorized::VTabletWriterV2, 
OlapTableSinkV2OperatorX> {
+public:
+    using Base = AsyncWriterSink<vectorized::VTabletWriterV2, 
OlapTableSinkV2OperatorX>;
+    using Parent = OlapTableSinkV2OperatorX;
+    ENABLE_FACTORY_CREATOR(OlapTableSinkV2LocalState);
+    OlapTableSinkV2LocalState(DataSinkOperatorXBase* parent, RuntimeState* 
state)
+            : Base(parent, state) {};
+    Status init(RuntimeState* state, LocalSinkStateInfo& info) override;
+    Status open(RuntimeState* state) override {
+        SCOPED_TIMER(exec_time_counter());
+        SCOPED_TIMER(_open_timer);
+        return Base::open(state);
+    }
+
+    Status close(RuntimeState* state, Status exec_status) override;
+    friend class OlapTableSinkV2OperatorX;
+
+private:
+    Status _close_status = Status::OK();
+};
+
+class OlapTableSinkV2OperatorX final : public 
DataSinkOperatorX<OlapTableSinkV2LocalState> {
+public:
+    using Base = DataSinkOperatorX<OlapTableSinkV2LocalState>;
+    OlapTableSinkV2OperatorX(ObjectPool* pool, int operator_id, const 
RowDescriptor& row_desc,
+                             const std::vector<TExpr>& t_output_expr, bool 
group_commit)
+            : Base(operator_id, 0),
+              _row_desc(row_desc),
+              _t_output_expr(t_output_expr),
+              _group_commit(group_commit),
+              _pool(pool) {};
+
+    Status init(const TDataSink& thrift_sink) override {
+        RETURN_IF_ERROR(Base::init(thrift_sink));
+        // From the thrift expressions create the real exprs.
+        RETURN_IF_ERROR(vectorized::VExpr::create_expr_trees(_t_output_expr, 
_output_vexpr_ctxs));
+        return Status::OK();
+    }
+
+    Status prepare(RuntimeState* state) override {
+        RETURN_IF_ERROR(Base::prepare(state));
+        return vectorized::VExpr::prepare(_output_vexpr_ctxs, state, 
_row_desc);
+    }
+
+    Status open(RuntimeState* state) override {
+        RETURN_IF_ERROR(Base::open(state));
+        return vectorized::VExpr::open(_output_vexpr_ctxs, state);
+    }
+
+    Status sink(RuntimeState* state, vectorized::Block* in_block,

Review Comment:
   warning: method 'sink' can be made static 
[readability-convert-member-functions-to-static]
   
   ```suggestion
       static Status sink(RuntimeState* state, vectorized::Block* in_block,
   ```
   



##########
be/src/vec/sink/writer/vtablet_writer_v2.cpp:
##########
@@ -0,0 +1,547 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "vec/sink/writer/vtablet_writer_v2.h"
+
+#include <brpc/uri.h>
+#include <bthread/bthread.h>
+#include <fmt/format.h>
+#include <gen_cpp/DataSinks_types.h>
+#include <gen_cpp/Descriptors_types.h>
+#include <gen_cpp/Metrics_types.h>
+#include <gen_cpp/Types_types.h>
+#include <gen_cpp/internal_service.pb.h>
+
+#include <algorithm>
+#include <execution>
+#include <mutex>
+#include <ranges>
+#include <string>
+#include <unordered_map>
+
+#include "common/compiler_util.h" // IWYU pragma: keep
+#include "common/logging.h"
+#include "common/object_pool.h"
+#include "common/signal_handler.h"
+#include "common/status.h"
+#include "exec/tablet_info.h"
+#include "olap/delta_writer_v2.h"
+#include "runtime/descriptors.h"
+#include "runtime/exec_env.h"
+#include "runtime/runtime_state.h"
+#include "runtime/thread_context.h"
+#include "service/brpc.h"
+#include "util/brpc_client_cache.h"
+#include "util/doris_metrics.h"
+#include "util/network_util.h"
+#include "util/threadpool.h"
+#include "util/thrift_util.h"
+#include "util/uid_util.h"
+#include "vec/core/block.h"
+#include "vec/exprs/vexpr.h"
+#include "vec/sink/delta_writer_v2_pool.h"
+#include "vec/sink/load_stream_stub.h"
+#include "vec/sink/load_stream_stub_pool.h"
+#include "vec/sink/vtablet_block_convertor.h"
+#include "vec/sink/vtablet_finder.h"
+
+namespace doris {
+
+namespace vectorized {
+
+VTabletWriterV2::VTabletWriterV2(const TDataSink& t_sink, const 
VExprContextSPtrs& output_exprs)
+        : AsyncResultWriter(output_exprs), _t_sink(t_sink) {
+    DCHECK(t_sink.__isset.olap_table_sink);
+}
+
+VTabletWriterV2::~VTabletWriterV2() = default;
+
+Status VTabletWriterV2::on_partitions_created(TCreatePartitionResult* result) {
+    // add new tablet locations. it will use by address. so add to pool
+    auto* new_locations = _pool->add(new 
std::vector<TTabletLocation>(result->tablets));
+    _location->add_locations(*new_locations);
+
+    // update new node info
+    _nodes_info->add_nodes(result->nodes);
+
+    // incremental open stream
+    RETURN_IF_ERROR(_incremental_open_streams(result->partitions));
+
+    return Status::OK();
+}
+
+static Status on_partitions_created(void* writer, TCreatePartitionResult* 
result) {
+    return 
static_cast<VTabletWriterV2*>(writer)->on_partitions_created(result);
+}
+
+Status VTabletWriterV2::_incremental_open_streams(
+        const std::vector<TOlapTablePartition>& partitions) {
+    // do what we did in prepare() for partitions. indexes which don't change 
when we create new partition is orthogonal to partitions.
+    std::unordered_set<int64_t> known_indexes;
+    std::unordered_set<int64_t> new_backends;
+    for (const auto& t_partition : partitions) {
+        VOlapTablePartition* partition = nullptr;
+        RETURN_IF_ERROR(_vpartition->generate_partition_from(t_partition, 
partition));
+        for (const auto& index : partition->indexes) {
+            for (const auto& tablet_id : index.tablets) {
+                auto nodes = _location->find_tablet(tablet_id)->node_ids;
+                for (auto& node : nodes) {
+                    PTabletID tablet;
+                    tablet.set_partition_id(partition->id);
+                    tablet.set_index_id(index.index_id);
+                    tablet.set_tablet_id(tablet_id);
+                    if (!_streams_for_node.contains(node)) {
+                        new_backends.insert(node);
+                    }
+                    _tablets_for_node[node].emplace(tablet_id, tablet);
+                    if (known_indexes.contains(index.index_id)) [[likely]] {
+                        continue;
+                    }
+                    _indexes_from_node[node].emplace_back(tablet);
+                    known_indexes.insert(index.index_id);
+                }
+            }
+        }
+    }
+    for (int64_t node_id : new_backends) {
+        auto load_streams = 
ExecEnv::GetInstance()->load_stream_stub_pool()->get_or_create(
+                _load_id, _backend_id, node_id, _stream_per_node, 
_num_local_sink);
+        RETURN_IF_ERROR(_open_streams_to_backend(node_id, *load_streams));
+        _streams_for_node[node_id] = load_streams;
+    }
+    return Status::OK();
+}
+
+Status VTabletWriterV2::_init_row_distribution() {
+    VRowDistributionContext ctx;
+
+    ctx.state = _state;
+    ctx.block_convertor = _block_convertor.get();
+    ctx.tablet_finder = _tablet_finder.get();
+    ctx.vpartition = _vpartition;
+    ctx.add_partition_request_timer = _add_partition_request_timer;
+    ctx.txn_id = _txn_id;
+    ctx.pool = _pool;
+    ctx.location = _location;
+    ctx.vec_output_expr_ctxs = &_vec_output_expr_ctxs;
+    ctx.on_partitions_created = &vectorized::on_partitions_created;
+    ctx.caller = (void*)this;
+    ctx.schema = _schema;
+
+    _row_distribution.init(&ctx);
+
+    RETURN_IF_ERROR(_row_distribution.open(_output_row_desc));
+
+    return Status::OK();
+}
+
+Status VTabletWriterV2::init_properties(ObjectPool* pool, bool group_commit) {
+    _pool = pool;
+    _group_commit = group_commit;
+    return Status::OK();
+}
+
+Status VTabletWriterV2::_init(RuntimeState* state, RuntimeProfile* profile) {
+    auto& table_sink = _t_sink.olap_table_sink;
+    _load_id.set_hi(table_sink.load_id.hi);
+    _load_id.set_lo(table_sink.load_id.lo);
+    _txn_id = table_sink.txn_id;
+    _num_replicas = table_sink.num_replicas;
+    _tuple_desc_id = table_sink.tuple_id;
+    _write_file_cache = table_sink.write_file_cache;
+    _schema.reset(new OlapTableSchemaParam());
+    RETURN_IF_ERROR(_schema->init(table_sink.schema));
+    _location = _pool->add(new OlapTableLocationParam(table_sink.location));
+    _nodes_info = _pool->add(new DorisNodesInfo(table_sink.nodes_info));
+
+    // if distributed column list is empty, we can ensure that tablet is with 
random distribution info
+    // and if load_to_single_tablet is set and set to true, we should find 
only one tablet in one partition
+    // for the whole olap table sink
+    auto find_tablet_mode = 
OlapTabletFinder::FindTabletMode::FIND_TABLET_EVERY_ROW;
+    if (table_sink.partition.distributed_columns.empty()) {
+        if (table_sink.__isset.load_to_single_tablet && 
table_sink.load_to_single_tablet) {
+            find_tablet_mode = 
OlapTabletFinder::FindTabletMode::FIND_TABLET_EVERY_SINK;
+        } else {
+            find_tablet_mode = 
OlapTabletFinder::FindTabletMode::FIND_TABLET_EVERY_BATCH;
+        }
+    }
+    _vpartition = _pool->add(new doris::VOlapTablePartitionParam(_schema, 
table_sink.partition));
+    _tablet_finder = std::make_unique<OlapTabletFinder>(_vpartition, 
find_tablet_mode);
+    RETURN_IF_ERROR(_vpartition->init());
+
+    _state = state;
+    _profile = profile;
+
+    _sender_id = state->per_fragment_instance_idx();
+    _num_senders = state->num_per_fragment_instances();
+    _backend_id = state->backend_id();
+    _stream_per_node = state->load_stream_per_node();
+    _total_streams = state->total_load_streams();
+    _num_local_sink = state->num_local_sink();
+    DCHECK(_stream_per_node > 0) << "load stream per node should be greator 
than 0";
+    DCHECK(_total_streams > 0) << "total load streams should be greator than 
0";
+    DCHECK(_num_local_sink > 0) << "num local sink should be greator than 0";
+    LOG(INFO) << "num senders: " << _num_senders << ", stream per node: " << 
_stream_per_node
+              << ", total_streams " << _total_streams << ", num_local_sink: " 
<< _num_local_sink;
+    _is_high_priority =
+            (state->execution_timeout() <= 
config::load_task_high_priority_threshold_second);
+
+    // profile must add to state's object pool
+    _profile = state->obj_pool()->add(new RuntimeProfile("VTabletWriterV2"));
+    _mem_tracker =
+            std::make_shared<MemTracker>("VTabletWriterV2:" + 
std::to_string(state->load_job_id()));
+    SCOPED_TIMER(_profile->total_time_counter());
+    SCOPED_CONSUME_MEM_TRACKER(_mem_tracker.get());
+
+    // get table's tuple descriptor
+    _output_tuple_desc = 
state->desc_tbl().get_tuple_descriptor(_tuple_desc_id);
+    if (_output_tuple_desc == nullptr) {
+        return Status::InternalError("unknown destination tuple descriptor, id 
= {}",
+                                     _tuple_desc_id);
+    }
+    _block_convertor = 
std::make_unique<OlapTableBlockConvertor>(_output_tuple_desc);
+    _block_convertor->init_autoinc_info(_schema->db_id(), _schema->table_id(),
+                                        _state->batch_size());
+    _output_row_desc = _pool->add(new RowDescriptor(_output_tuple_desc, 
false));
+
+    // add all counter
+    _input_rows_counter = ADD_COUNTER(_profile, "RowsRead", TUnit::UNIT);
+    _output_rows_counter = ADD_COUNTER(_profile, "RowsReturned", TUnit::UNIT);
+    _filtered_rows_counter = ADD_COUNTER(_profile, "RowsFiltered", 
TUnit::UNIT);
+    _send_data_timer = ADD_TIMER(_profile, "SendDataTime");
+    _wait_mem_limit_timer = ADD_CHILD_TIMER(_profile, "WaitMemLimitTime", 
"SendDataTime");
+    _row_distribution_timer = ADD_CHILD_TIMER(_profile, "RowDistributionTime", 
"SendDataTime");
+    _write_memtable_timer = ADD_CHILD_TIMER(_profile, "WriteMemTableTime", 
"SendDataTime");
+    _validate_data_timer = ADD_TIMER(_profile, "ValidateDataTime");
+    _open_timer = ADD_TIMER(_profile, "OpenTime");
+    _close_timer = ADD_TIMER(_profile, "CloseWaitTime");
+    _close_writer_timer = ADD_CHILD_TIMER(_profile, "CloseWriterTime", 
"CloseWaitTime");
+    _close_load_timer = ADD_CHILD_TIMER(_profile, "CloseLoadTime", 
"CloseWaitTime");
+
+    if (config::share_delta_writers) {
+        _delta_writer_for_tablet = 
ExecEnv::GetInstance()->delta_writer_v2_pool()->get_or_create(
+                _load_id, _num_local_sink);
+    } else {
+        _delta_writer_for_tablet = 
std::make_shared<DeltaWriterV2Map>(_load_id);
+    }
+    return Status::OK();
+}
+
+Status VTabletWriterV2::open(RuntimeState* state, RuntimeProfile* profile) {
+    RETURN_IF_ERROR(_init(state, profile));
+    SCOPED_TIMER(_profile->total_time_counter());
+    SCOPED_TIMER(_open_timer);
+    SCOPED_CONSUME_MEM_TRACKER(_mem_tracker.get());
+    signal::set_signal_task_id(_load_id);
+
+    _build_tablet_node_mapping();
+    RETURN_IF_ERROR(_open_streams(_backend_id));
+    RETURN_IF_ERROR(_init_row_distribution());
+
+    return Status::OK();
+}
+
+Status VTabletWriterV2::_open_streams(int64_t src_id) {
+    for (auto& [dst_id, _] : _tablets_for_node) {
+        auto streams = 
ExecEnv::GetInstance()->load_stream_stub_pool()->get_or_create(
+                _load_id, src_id, dst_id, _stream_per_node, _num_local_sink);
+        RETURN_IF_ERROR(_open_streams_to_backend(dst_id, *streams));
+        _streams_for_node[dst_id] = streams;
+    }
+    return Status::OK();
+}
+
+Status VTabletWriterV2::_open_streams_to_backend(int64_t dst_id,
+                                                 
::doris::stream_load::LoadStreams& streams) {
+    auto node_info = _nodes_info->find_node(dst_id);
+    if (node_info == nullptr) {
+        return Status::InternalError("Unknown node {} in tablet location", 
dst_id);
+    }
+    // get tablet schema from each backend only in the 1st stream
+    for (auto& stream : streams.streams() | std::ranges::views::take(1)) {
+        const std::vector<PTabletID>& tablets_for_schema = 
_indexes_from_node[node_info->id];
+        
RETURN_IF_ERROR(stream->open(_state->exec_env()->brpc_internal_client_cache(), 
*node_info,
+                                     _txn_id, *_schema, tablets_for_schema, 
_total_streams,
+                                     _state->enable_profile()));
+    }
+    // for the rest streams, open without getting tablet schema
+    for (auto& stream : streams.streams() | std::ranges::views::drop(1)) {
+        
RETURN_IF_ERROR(stream->open(_state->exec_env()->brpc_internal_client_cache(), 
*node_info,
+                                     _txn_id, *_schema, {}, _total_streams,
+                                     _state->enable_profile()));
+    }
+    return Status::OK();
+}
+
+void VTabletWriterV2::_build_tablet_node_mapping() {

Review Comment:
   warning: method '_build_tablet_node_mapping' can be made static 
[readability-convert-member-functions-to-static]
   
   be/src/vec/sink/writer/vtablet_writer_v2.h:132:
   ```diff
   -     void _build_tablet_node_mapping();
   +     static void _build_tablet_node_mapping();
   ```
   



##########
be/src/vec/sink/writer/vtablet_writer_v2.cpp:
##########
@@ -0,0 +1,547 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "vec/sink/writer/vtablet_writer_v2.h"
+
+#include <brpc/uri.h>
+#include <bthread/bthread.h>
+#include <fmt/format.h>
+#include <gen_cpp/DataSinks_types.h>
+#include <gen_cpp/Descriptors_types.h>
+#include <gen_cpp/Metrics_types.h>
+#include <gen_cpp/Types_types.h>
+#include <gen_cpp/internal_service.pb.h>
+
+#include <algorithm>
+#include <execution>
+#include <mutex>
+#include <ranges>
+#include <string>
+#include <unordered_map>
+
+#include "common/compiler_util.h" // IWYU pragma: keep
+#include "common/logging.h"
+#include "common/object_pool.h"
+#include "common/signal_handler.h"
+#include "common/status.h"
+#include "exec/tablet_info.h"
+#include "olap/delta_writer_v2.h"
+#include "runtime/descriptors.h"
+#include "runtime/exec_env.h"
+#include "runtime/runtime_state.h"
+#include "runtime/thread_context.h"
+#include "service/brpc.h"
+#include "util/brpc_client_cache.h"
+#include "util/doris_metrics.h"
+#include "util/network_util.h"
+#include "util/threadpool.h"
+#include "util/thrift_util.h"
+#include "util/uid_util.h"
+#include "vec/core/block.h"
+#include "vec/exprs/vexpr.h"
+#include "vec/sink/delta_writer_v2_pool.h"
+#include "vec/sink/load_stream_stub.h"
+#include "vec/sink/load_stream_stub_pool.h"
+#include "vec/sink/vtablet_block_convertor.h"
+#include "vec/sink/vtablet_finder.h"
+
+namespace doris {
+
+namespace vectorized {
+
+VTabletWriterV2::VTabletWriterV2(const TDataSink& t_sink, const 
VExprContextSPtrs& output_exprs)
+        : AsyncResultWriter(output_exprs), _t_sink(t_sink) {
+    DCHECK(t_sink.__isset.olap_table_sink);
+}
+
+VTabletWriterV2::~VTabletWriterV2() = default;
+
+Status VTabletWriterV2::on_partitions_created(TCreatePartitionResult* result) {
+    // add new tablet locations. it will use by address. so add to pool
+    auto* new_locations = _pool->add(new 
std::vector<TTabletLocation>(result->tablets));
+    _location->add_locations(*new_locations);
+
+    // update new node info
+    _nodes_info->add_nodes(result->nodes);
+
+    // incremental open stream
+    RETURN_IF_ERROR(_incremental_open_streams(result->partitions));
+
+    return Status::OK();
+}
+
+static Status on_partitions_created(void* writer, TCreatePartitionResult* 
result) {
+    return 
static_cast<VTabletWriterV2*>(writer)->on_partitions_created(result);
+}
+
+Status VTabletWriterV2::_incremental_open_streams(
+        const std::vector<TOlapTablePartition>& partitions) {
+    // do what we did in prepare() for partitions. indexes which don't change 
when we create new partition is orthogonal to partitions.
+    std::unordered_set<int64_t> known_indexes;
+    std::unordered_set<int64_t> new_backends;
+    for (const auto& t_partition : partitions) {
+        VOlapTablePartition* partition = nullptr;
+        RETURN_IF_ERROR(_vpartition->generate_partition_from(t_partition, 
partition));
+        for (const auto& index : partition->indexes) {
+            for (const auto& tablet_id : index.tablets) {
+                auto nodes = _location->find_tablet(tablet_id)->node_ids;
+                for (auto& node : nodes) {
+                    PTabletID tablet;
+                    tablet.set_partition_id(partition->id);
+                    tablet.set_index_id(index.index_id);
+                    tablet.set_tablet_id(tablet_id);
+                    if (!_streams_for_node.contains(node)) {
+                        new_backends.insert(node);
+                    }
+                    _tablets_for_node[node].emplace(tablet_id, tablet);
+                    if (known_indexes.contains(index.index_id)) [[likely]] {
+                        continue;
+                    }
+                    _indexes_from_node[node].emplace_back(tablet);
+                    known_indexes.insert(index.index_id);
+                }
+            }
+        }
+    }
+    for (int64_t node_id : new_backends) {
+        auto load_streams = 
ExecEnv::GetInstance()->load_stream_stub_pool()->get_or_create(
+                _load_id, _backend_id, node_id, _stream_per_node, 
_num_local_sink);
+        RETURN_IF_ERROR(_open_streams_to_backend(node_id, *load_streams));
+        _streams_for_node[node_id] = load_streams;
+    }
+    return Status::OK();
+}
+
+Status VTabletWriterV2::_init_row_distribution() {
+    VRowDistributionContext ctx;
+
+    ctx.state = _state;
+    ctx.block_convertor = _block_convertor.get();
+    ctx.tablet_finder = _tablet_finder.get();
+    ctx.vpartition = _vpartition;
+    ctx.add_partition_request_timer = _add_partition_request_timer;
+    ctx.txn_id = _txn_id;
+    ctx.pool = _pool;
+    ctx.location = _location;
+    ctx.vec_output_expr_ctxs = &_vec_output_expr_ctxs;
+    ctx.on_partitions_created = &vectorized::on_partitions_created;
+    ctx.caller = (void*)this;
+    ctx.schema = _schema;
+
+    _row_distribution.init(&ctx);
+
+    RETURN_IF_ERROR(_row_distribution.open(_output_row_desc));
+
+    return Status::OK();
+}
+
+Status VTabletWriterV2::init_properties(ObjectPool* pool, bool group_commit) {
+    _pool = pool;
+    _group_commit = group_commit;
+    return Status::OK();
+}
+
+Status VTabletWriterV2::_init(RuntimeState* state, RuntimeProfile* profile) {
+    auto& table_sink = _t_sink.olap_table_sink;
+    _load_id.set_hi(table_sink.load_id.hi);
+    _load_id.set_lo(table_sink.load_id.lo);
+    _txn_id = table_sink.txn_id;
+    _num_replicas = table_sink.num_replicas;
+    _tuple_desc_id = table_sink.tuple_id;
+    _write_file_cache = table_sink.write_file_cache;
+    _schema.reset(new OlapTableSchemaParam());
+    RETURN_IF_ERROR(_schema->init(table_sink.schema));
+    _location = _pool->add(new OlapTableLocationParam(table_sink.location));
+    _nodes_info = _pool->add(new DorisNodesInfo(table_sink.nodes_info));
+
+    // if distributed column list is empty, we can ensure that tablet is with 
random distribution info
+    // and if load_to_single_tablet is set and set to true, we should find 
only one tablet in one partition
+    // for the whole olap table sink
+    auto find_tablet_mode = 
OlapTabletFinder::FindTabletMode::FIND_TABLET_EVERY_ROW;
+    if (table_sink.partition.distributed_columns.empty()) {
+        if (table_sink.__isset.load_to_single_tablet && 
table_sink.load_to_single_tablet) {
+            find_tablet_mode = 
OlapTabletFinder::FindTabletMode::FIND_TABLET_EVERY_SINK;
+        } else {
+            find_tablet_mode = 
OlapTabletFinder::FindTabletMode::FIND_TABLET_EVERY_BATCH;
+        }
+    }
+    _vpartition = _pool->add(new doris::VOlapTablePartitionParam(_schema, 
table_sink.partition));
+    _tablet_finder = std::make_unique<OlapTabletFinder>(_vpartition, 
find_tablet_mode);
+    RETURN_IF_ERROR(_vpartition->init());
+
+    _state = state;
+    _profile = profile;
+
+    _sender_id = state->per_fragment_instance_idx();
+    _num_senders = state->num_per_fragment_instances();
+    _backend_id = state->backend_id();
+    _stream_per_node = state->load_stream_per_node();
+    _total_streams = state->total_load_streams();
+    _num_local_sink = state->num_local_sink();
+    DCHECK(_stream_per_node > 0) << "load stream per node should be greator 
than 0";
+    DCHECK(_total_streams > 0) << "total load streams should be greator than 
0";
+    DCHECK(_num_local_sink > 0) << "num local sink should be greator than 0";
+    LOG(INFO) << "num senders: " << _num_senders << ", stream per node: " << 
_stream_per_node
+              << ", total_streams " << _total_streams << ", num_local_sink: " 
<< _num_local_sink;
+    _is_high_priority =
+            (state->execution_timeout() <= 
config::load_task_high_priority_threshold_second);
+
+    // profile must add to state's object pool
+    _profile = state->obj_pool()->add(new RuntimeProfile("VTabletWriterV2"));
+    _mem_tracker =
+            std::make_shared<MemTracker>("VTabletWriterV2:" + 
std::to_string(state->load_job_id()));
+    SCOPED_TIMER(_profile->total_time_counter());
+    SCOPED_CONSUME_MEM_TRACKER(_mem_tracker.get());
+
+    // get table's tuple descriptor
+    _output_tuple_desc = 
state->desc_tbl().get_tuple_descriptor(_tuple_desc_id);
+    if (_output_tuple_desc == nullptr) {
+        return Status::InternalError("unknown destination tuple descriptor, id 
= {}",
+                                     _tuple_desc_id);
+    }
+    _block_convertor = 
std::make_unique<OlapTableBlockConvertor>(_output_tuple_desc);
+    _block_convertor->init_autoinc_info(_schema->db_id(), _schema->table_id(),
+                                        _state->batch_size());
+    _output_row_desc = _pool->add(new RowDescriptor(_output_tuple_desc, 
false));
+
+    // add all counter
+    _input_rows_counter = ADD_COUNTER(_profile, "RowsRead", TUnit::UNIT);
+    _output_rows_counter = ADD_COUNTER(_profile, "RowsReturned", TUnit::UNIT);
+    _filtered_rows_counter = ADD_COUNTER(_profile, "RowsFiltered", 
TUnit::UNIT);
+    _send_data_timer = ADD_TIMER(_profile, "SendDataTime");
+    _wait_mem_limit_timer = ADD_CHILD_TIMER(_profile, "WaitMemLimitTime", 
"SendDataTime");
+    _row_distribution_timer = ADD_CHILD_TIMER(_profile, "RowDistributionTime", 
"SendDataTime");
+    _write_memtable_timer = ADD_CHILD_TIMER(_profile, "WriteMemTableTime", 
"SendDataTime");
+    _validate_data_timer = ADD_TIMER(_profile, "ValidateDataTime");
+    _open_timer = ADD_TIMER(_profile, "OpenTime");
+    _close_timer = ADD_TIMER(_profile, "CloseWaitTime");
+    _close_writer_timer = ADD_CHILD_TIMER(_profile, "CloseWriterTime", 
"CloseWaitTime");
+    _close_load_timer = ADD_CHILD_TIMER(_profile, "CloseLoadTime", 
"CloseWaitTime");
+
+    if (config::share_delta_writers) {
+        _delta_writer_for_tablet = 
ExecEnv::GetInstance()->delta_writer_v2_pool()->get_or_create(
+                _load_id, _num_local_sink);
+    } else {
+        _delta_writer_for_tablet = 
std::make_shared<DeltaWriterV2Map>(_load_id);
+    }
+    return Status::OK();
+}
+
+Status VTabletWriterV2::open(RuntimeState* state, RuntimeProfile* profile) {
+    RETURN_IF_ERROR(_init(state, profile));
+    SCOPED_TIMER(_profile->total_time_counter());
+    SCOPED_TIMER(_open_timer);
+    SCOPED_CONSUME_MEM_TRACKER(_mem_tracker.get());
+    signal::set_signal_task_id(_load_id);
+
+    _build_tablet_node_mapping();
+    RETURN_IF_ERROR(_open_streams(_backend_id));
+    RETURN_IF_ERROR(_init_row_distribution());
+
+    return Status::OK();
+}
+
+Status VTabletWriterV2::_open_streams(int64_t src_id) {

Review Comment:
   warning: method '_open_streams' can be made static 
[readability-convert-member-functions-to-static]
   
   be/src/vec/sink/writer/vtablet_writer_v2.h:126:
   ```diff
   -     Status _open_streams(int64_t src_id);
   +     static Status _open_streams(int64_t src_id);
   ```
   



##########
be/src/pipeline/exec/olap_table_sink_v2_operator.h:
##########
@@ -41,9 +42,76 @@
     bool can_write() override { return true; } // TODO: need use mem_limit
 };
 
-OperatorPtr OlapTableSinkV2OperatorBuilder::build_operator() {
-    return std::make_shared<OlapTableSinkV2Operator>(this, _sink);
-}
+class OlapTableSinkV2OperatorX;
+
+class OlapTableSinkV2LocalState final
+        : public AsyncWriterSink<vectorized::VTabletWriterV2, 
OlapTableSinkV2OperatorX> {
+public:
+    using Base = AsyncWriterSink<vectorized::VTabletWriterV2, 
OlapTableSinkV2OperatorX>;
+    using Parent = OlapTableSinkV2OperatorX;
+    ENABLE_FACTORY_CREATOR(OlapTableSinkV2LocalState);
+    OlapTableSinkV2LocalState(DataSinkOperatorXBase* parent, RuntimeState* 
state)
+            : Base(parent, state) {};
+    Status init(RuntimeState* state, LocalSinkStateInfo& info) override;
+    Status open(RuntimeState* state) override {
+        SCOPED_TIMER(exec_time_counter());
+        SCOPED_TIMER(_open_timer);
+        return Base::open(state);
+    }
+
+    Status close(RuntimeState* state, Status exec_status) override;
+    friend class OlapTableSinkV2OperatorX;
+
+private:
+    Status _close_status = Status::OK();
+};
+
+class OlapTableSinkV2OperatorX final : public 
DataSinkOperatorX<OlapTableSinkV2LocalState> {
+public:
+    using Base = DataSinkOperatorX<OlapTableSinkV2LocalState>;
+    OlapTableSinkV2OperatorX(ObjectPool* pool, int operator_id, const 
RowDescriptor& row_desc,
+                             const std::vector<TExpr>& t_output_expr, bool 
group_commit)
+            : Base(operator_id, 0),
+              _row_desc(row_desc),
+              _t_output_expr(t_output_expr),
+              _group_commit(group_commit),
+              _pool(pool) {};
+
+    Status init(const TDataSink& thrift_sink) override {
+        RETURN_IF_ERROR(Base::init(thrift_sink));
+        // From the thrift expressions create the real exprs.
+        RETURN_IF_ERROR(vectorized::VExpr::create_expr_trees(_t_output_expr, 
_output_vexpr_ctxs));
+        return Status::OK();
+    }
+
+    Status prepare(RuntimeState* state) override {

Review Comment:
   warning: method 'prepare' can be made static 
[readability-convert-member-functions-to-static]
   
   ```suggestion
       static Status prepare(RuntimeState* state) override {
   ```
   



##########
be/src/vec/sink/writer/vtablet_writer_v2.cpp:
##########
@@ -0,0 +1,547 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "vec/sink/writer/vtablet_writer_v2.h"
+
+#include <brpc/uri.h>
+#include <bthread/bthread.h>
+#include <fmt/format.h>
+#include <gen_cpp/DataSinks_types.h>
+#include <gen_cpp/Descriptors_types.h>
+#include <gen_cpp/Metrics_types.h>
+#include <gen_cpp/Types_types.h>
+#include <gen_cpp/internal_service.pb.h>
+
+#include <algorithm>
+#include <execution>
+#include <mutex>
+#include <ranges>
+#include <string>
+#include <unordered_map>
+
+#include "common/compiler_util.h" // IWYU pragma: keep
+#include "common/logging.h"
+#include "common/object_pool.h"
+#include "common/signal_handler.h"
+#include "common/status.h"
+#include "exec/tablet_info.h"
+#include "olap/delta_writer_v2.h"
+#include "runtime/descriptors.h"
+#include "runtime/exec_env.h"
+#include "runtime/runtime_state.h"
+#include "runtime/thread_context.h"
+#include "service/brpc.h"
+#include "util/brpc_client_cache.h"
+#include "util/doris_metrics.h"
+#include "util/network_util.h"
+#include "util/threadpool.h"
+#include "util/thrift_util.h"
+#include "util/uid_util.h"
+#include "vec/core/block.h"
+#include "vec/exprs/vexpr.h"
+#include "vec/sink/delta_writer_v2_pool.h"
+#include "vec/sink/load_stream_stub.h"
+#include "vec/sink/load_stream_stub_pool.h"
+#include "vec/sink/vtablet_block_convertor.h"
+#include "vec/sink/vtablet_finder.h"
+
+namespace doris {
+
+namespace vectorized {
+
+VTabletWriterV2::VTabletWriterV2(const TDataSink& t_sink, const 
VExprContextSPtrs& output_exprs)
+        : AsyncResultWriter(output_exprs), _t_sink(t_sink) {
+    DCHECK(t_sink.__isset.olap_table_sink);
+}
+
+VTabletWriterV2::~VTabletWriterV2() = default;
+
+Status VTabletWriterV2::on_partitions_created(TCreatePartitionResult* result) {
+    // add new tablet locations. it will use by address. so add to pool
+    auto* new_locations = _pool->add(new 
std::vector<TTabletLocation>(result->tablets));
+    _location->add_locations(*new_locations);
+
+    // update new node info
+    _nodes_info->add_nodes(result->nodes);
+
+    // incremental open stream
+    RETURN_IF_ERROR(_incremental_open_streams(result->partitions));
+
+    return Status::OK();
+}
+
+static Status on_partitions_created(void* writer, TCreatePartitionResult* 
result) {
+    return 
static_cast<VTabletWriterV2*>(writer)->on_partitions_created(result);
+}
+
+Status VTabletWriterV2::_incremental_open_streams(

Review Comment:
   warning: method '_incremental_open_streams' can be made static 
[readability-convert-member-functions-to-static]
   
   ```suggestion
   static Status VTabletWriterV2::_incremental_open_streams(
   ```
   



##########
be/src/vec/sink/writer/vtablet_writer_v2.h:
##########
@@ -0,0 +1,233 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+#include <brpc/controller.h>
+#include <bthread/types.h>
+#include <butil/errno.h>
+#include <fmt/format.h>
+#include <gen_cpp/PaloInternalService_types.h>
+#include <gen_cpp/Types_types.h>
+#include <gen_cpp/internal_service.pb.h>
+#include <gen_cpp/types.pb.h>
+#include <glog/logging.h>
+#include <google/protobuf/stubs/callback.h>
+#include <stddef.h>

Review Comment:
   warning: inclusion of deprecated C++ header 'stddef.h'; consider using 
'cstddef' instead [modernize-deprecated-headers]
   
   ```suggestion
   #include <cstddef>
   ```
   



##########
be/src/vec/sink/writer/vtablet_writer_v2.cpp:
##########
@@ -0,0 +1,547 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "vec/sink/writer/vtablet_writer_v2.h"
+
+#include <brpc/uri.h>
+#include <bthread/bthread.h>
+#include <fmt/format.h>
+#include <gen_cpp/DataSinks_types.h>
+#include <gen_cpp/Descriptors_types.h>
+#include <gen_cpp/Metrics_types.h>
+#include <gen_cpp/Types_types.h>
+#include <gen_cpp/internal_service.pb.h>
+
+#include <algorithm>
+#include <execution>
+#include <mutex>
+#include <ranges>
+#include <string>
+#include <unordered_map>
+
+#include "common/compiler_util.h" // IWYU pragma: keep
+#include "common/logging.h"
+#include "common/object_pool.h"
+#include "common/signal_handler.h"
+#include "common/status.h"
+#include "exec/tablet_info.h"
+#include "olap/delta_writer_v2.h"
+#include "runtime/descriptors.h"
+#include "runtime/exec_env.h"
+#include "runtime/runtime_state.h"
+#include "runtime/thread_context.h"
+#include "service/brpc.h"
+#include "util/brpc_client_cache.h"
+#include "util/doris_metrics.h"
+#include "util/network_util.h"
+#include "util/threadpool.h"
+#include "util/thrift_util.h"
+#include "util/uid_util.h"
+#include "vec/core/block.h"
+#include "vec/exprs/vexpr.h"
+#include "vec/sink/delta_writer_v2_pool.h"
+#include "vec/sink/load_stream_stub.h"
+#include "vec/sink/load_stream_stub_pool.h"
+#include "vec/sink/vtablet_block_convertor.h"
+#include "vec/sink/vtablet_finder.h"
+
+namespace doris {
+
+namespace vectorized {
+
+VTabletWriterV2::VTabletWriterV2(const TDataSink& t_sink, const 
VExprContextSPtrs& output_exprs)
+        : AsyncResultWriter(output_exprs), _t_sink(t_sink) {
+    DCHECK(t_sink.__isset.olap_table_sink);
+}
+
+VTabletWriterV2::~VTabletWriterV2() = default;
+
+Status VTabletWriterV2::on_partitions_created(TCreatePartitionResult* result) {
+    // add new tablet locations. it will use by address. so add to pool
+    auto* new_locations = _pool->add(new 
std::vector<TTabletLocation>(result->tablets));
+    _location->add_locations(*new_locations);
+
+    // update new node info
+    _nodes_info->add_nodes(result->nodes);
+
+    // incremental open stream
+    RETURN_IF_ERROR(_incremental_open_streams(result->partitions));
+
+    return Status::OK();
+}
+
+static Status on_partitions_created(void* writer, TCreatePartitionResult* 
result) {
+    return 
static_cast<VTabletWriterV2*>(writer)->on_partitions_created(result);
+}
+
+Status VTabletWriterV2::_incremental_open_streams(
+        const std::vector<TOlapTablePartition>& partitions) {
+    // do what we did in prepare() for partitions. indexes which don't change 
when we create new partition is orthogonal to partitions.
+    std::unordered_set<int64_t> known_indexes;
+    std::unordered_set<int64_t> new_backends;
+    for (const auto& t_partition : partitions) {
+        VOlapTablePartition* partition = nullptr;
+        RETURN_IF_ERROR(_vpartition->generate_partition_from(t_partition, 
partition));
+        for (const auto& index : partition->indexes) {
+            for (const auto& tablet_id : index.tablets) {
+                auto nodes = _location->find_tablet(tablet_id)->node_ids;
+                for (auto& node : nodes) {
+                    PTabletID tablet;
+                    tablet.set_partition_id(partition->id);
+                    tablet.set_index_id(index.index_id);
+                    tablet.set_tablet_id(tablet_id);
+                    if (!_streams_for_node.contains(node)) {
+                        new_backends.insert(node);
+                    }
+                    _tablets_for_node[node].emplace(tablet_id, tablet);
+                    if (known_indexes.contains(index.index_id)) [[likely]] {
+                        continue;
+                    }
+                    _indexes_from_node[node].emplace_back(tablet);
+                    known_indexes.insert(index.index_id);
+                }
+            }
+        }
+    }
+    for (int64_t node_id : new_backends) {
+        auto load_streams = 
ExecEnv::GetInstance()->load_stream_stub_pool()->get_or_create(
+                _load_id, _backend_id, node_id, _stream_per_node, 
_num_local_sink);
+        RETURN_IF_ERROR(_open_streams_to_backend(node_id, *load_streams));
+        _streams_for_node[node_id] = load_streams;
+    }
+    return Status::OK();
+}
+
+Status VTabletWriterV2::_init_row_distribution() {
+    VRowDistributionContext ctx;
+
+    ctx.state = _state;
+    ctx.block_convertor = _block_convertor.get();
+    ctx.tablet_finder = _tablet_finder.get();
+    ctx.vpartition = _vpartition;
+    ctx.add_partition_request_timer = _add_partition_request_timer;
+    ctx.txn_id = _txn_id;
+    ctx.pool = _pool;
+    ctx.location = _location;
+    ctx.vec_output_expr_ctxs = &_vec_output_expr_ctxs;
+    ctx.on_partitions_created = &vectorized::on_partitions_created;
+    ctx.caller = (void*)this;
+    ctx.schema = _schema;
+
+    _row_distribution.init(&ctx);
+
+    RETURN_IF_ERROR(_row_distribution.open(_output_row_desc));
+
+    return Status::OK();
+}
+
+Status VTabletWriterV2::init_properties(ObjectPool* pool, bool group_commit) {
+    _pool = pool;
+    _group_commit = group_commit;
+    return Status::OK();
+}
+
+Status VTabletWriterV2::_init(RuntimeState* state, RuntimeProfile* profile) {
+    auto& table_sink = _t_sink.olap_table_sink;
+    _load_id.set_hi(table_sink.load_id.hi);
+    _load_id.set_lo(table_sink.load_id.lo);
+    _txn_id = table_sink.txn_id;
+    _num_replicas = table_sink.num_replicas;
+    _tuple_desc_id = table_sink.tuple_id;
+    _write_file_cache = table_sink.write_file_cache;
+    _schema.reset(new OlapTableSchemaParam());
+    RETURN_IF_ERROR(_schema->init(table_sink.schema));
+    _location = _pool->add(new OlapTableLocationParam(table_sink.location));
+    _nodes_info = _pool->add(new DorisNodesInfo(table_sink.nodes_info));
+
+    // if distributed column list is empty, we can ensure that tablet is with 
random distribution info
+    // and if load_to_single_tablet is set and set to true, we should find 
only one tablet in one partition
+    // for the whole olap table sink
+    auto find_tablet_mode = 
OlapTabletFinder::FindTabletMode::FIND_TABLET_EVERY_ROW;
+    if (table_sink.partition.distributed_columns.empty()) {
+        if (table_sink.__isset.load_to_single_tablet && 
table_sink.load_to_single_tablet) {
+            find_tablet_mode = 
OlapTabletFinder::FindTabletMode::FIND_TABLET_EVERY_SINK;
+        } else {
+            find_tablet_mode = 
OlapTabletFinder::FindTabletMode::FIND_TABLET_EVERY_BATCH;
+        }
+    }
+    _vpartition = _pool->add(new doris::VOlapTablePartitionParam(_schema, 
table_sink.partition));
+    _tablet_finder = std::make_unique<OlapTabletFinder>(_vpartition, 
find_tablet_mode);
+    RETURN_IF_ERROR(_vpartition->init());
+
+    _state = state;
+    _profile = profile;
+
+    _sender_id = state->per_fragment_instance_idx();
+    _num_senders = state->num_per_fragment_instances();
+    _backend_id = state->backend_id();
+    _stream_per_node = state->load_stream_per_node();
+    _total_streams = state->total_load_streams();
+    _num_local_sink = state->num_local_sink();
+    DCHECK(_stream_per_node > 0) << "load stream per node should be greator 
than 0";
+    DCHECK(_total_streams > 0) << "total load streams should be greator than 
0";
+    DCHECK(_num_local_sink > 0) << "num local sink should be greator than 0";
+    LOG(INFO) << "num senders: " << _num_senders << ", stream per node: " << 
_stream_per_node
+              << ", total_streams " << _total_streams << ", num_local_sink: " 
<< _num_local_sink;
+    _is_high_priority =
+            (state->execution_timeout() <= 
config::load_task_high_priority_threshold_second);
+
+    // profile must add to state's object pool
+    _profile = state->obj_pool()->add(new RuntimeProfile("VTabletWriterV2"));
+    _mem_tracker =
+            std::make_shared<MemTracker>("VTabletWriterV2:" + 
std::to_string(state->load_job_id()));
+    SCOPED_TIMER(_profile->total_time_counter());
+    SCOPED_CONSUME_MEM_TRACKER(_mem_tracker.get());
+
+    // get table's tuple descriptor
+    _output_tuple_desc = 
state->desc_tbl().get_tuple_descriptor(_tuple_desc_id);
+    if (_output_tuple_desc == nullptr) {
+        return Status::InternalError("unknown destination tuple descriptor, id 
= {}",
+                                     _tuple_desc_id);
+    }
+    _block_convertor = 
std::make_unique<OlapTableBlockConvertor>(_output_tuple_desc);
+    _block_convertor->init_autoinc_info(_schema->db_id(), _schema->table_id(),
+                                        _state->batch_size());
+    _output_row_desc = _pool->add(new RowDescriptor(_output_tuple_desc, 
false));
+
+    // add all counter
+    _input_rows_counter = ADD_COUNTER(_profile, "RowsRead", TUnit::UNIT);
+    _output_rows_counter = ADD_COUNTER(_profile, "RowsReturned", TUnit::UNIT);
+    _filtered_rows_counter = ADD_COUNTER(_profile, "RowsFiltered", 
TUnit::UNIT);
+    _send_data_timer = ADD_TIMER(_profile, "SendDataTime");
+    _wait_mem_limit_timer = ADD_CHILD_TIMER(_profile, "WaitMemLimitTime", 
"SendDataTime");
+    _row_distribution_timer = ADD_CHILD_TIMER(_profile, "RowDistributionTime", 
"SendDataTime");
+    _write_memtable_timer = ADD_CHILD_TIMER(_profile, "WriteMemTableTime", 
"SendDataTime");
+    _validate_data_timer = ADD_TIMER(_profile, "ValidateDataTime");
+    _open_timer = ADD_TIMER(_profile, "OpenTime");
+    _close_timer = ADD_TIMER(_profile, "CloseWaitTime");
+    _close_writer_timer = ADD_CHILD_TIMER(_profile, "CloseWriterTime", 
"CloseWaitTime");
+    _close_load_timer = ADD_CHILD_TIMER(_profile, "CloseLoadTime", 
"CloseWaitTime");
+
+    if (config::share_delta_writers) {
+        _delta_writer_for_tablet = 
ExecEnv::GetInstance()->delta_writer_v2_pool()->get_or_create(
+                _load_id, _num_local_sink);
+    } else {
+        _delta_writer_for_tablet = 
std::make_shared<DeltaWriterV2Map>(_load_id);
+    }
+    return Status::OK();
+}
+
+Status VTabletWriterV2::open(RuntimeState* state, RuntimeProfile* profile) {
+    RETURN_IF_ERROR(_init(state, profile));
+    SCOPED_TIMER(_profile->total_time_counter());
+    SCOPED_TIMER(_open_timer);
+    SCOPED_CONSUME_MEM_TRACKER(_mem_tracker.get());
+    signal::set_signal_task_id(_load_id);
+
+    _build_tablet_node_mapping();
+    RETURN_IF_ERROR(_open_streams(_backend_id));
+    RETURN_IF_ERROR(_init_row_distribution());
+
+    return Status::OK();
+}
+
+Status VTabletWriterV2::_open_streams(int64_t src_id) {
+    for (auto& [dst_id, _] : _tablets_for_node) {
+        auto streams = 
ExecEnv::GetInstance()->load_stream_stub_pool()->get_or_create(
+                _load_id, src_id, dst_id, _stream_per_node, _num_local_sink);
+        RETURN_IF_ERROR(_open_streams_to_backend(dst_id, *streams));
+        _streams_for_node[dst_id] = streams;
+    }
+    return Status::OK();
+}
+
+Status VTabletWriterV2::_open_streams_to_backend(int64_t dst_id,
+                                                 
::doris::stream_load::LoadStreams& streams) {
+    auto node_info = _nodes_info->find_node(dst_id);

Review Comment:
   warning: 'auto node_info' can be declared as 'const auto *node_info' 
[readability-qualified-auto]
   
   ```suggestion
       const auto *node_info = _nodes_info->find_node(dst_id);
   ```
   



##########
be/src/vec/sink/writer/vtablet_writer_v2.cpp:
##########
@@ -0,0 +1,547 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "vec/sink/writer/vtablet_writer_v2.h"
+
+#include <brpc/uri.h>
+#include <bthread/bthread.h>
+#include <fmt/format.h>
+#include <gen_cpp/DataSinks_types.h>
+#include <gen_cpp/Descriptors_types.h>
+#include <gen_cpp/Metrics_types.h>
+#include <gen_cpp/Types_types.h>
+#include <gen_cpp/internal_service.pb.h>
+
+#include <algorithm>
+#include <execution>
+#include <mutex>
+#include <ranges>
+#include <string>
+#include <unordered_map>
+
+#include "common/compiler_util.h" // IWYU pragma: keep
+#include "common/logging.h"
+#include "common/object_pool.h"
+#include "common/signal_handler.h"
+#include "common/status.h"
+#include "exec/tablet_info.h"
+#include "olap/delta_writer_v2.h"
+#include "runtime/descriptors.h"
+#include "runtime/exec_env.h"
+#include "runtime/runtime_state.h"
+#include "runtime/thread_context.h"
+#include "service/brpc.h"
+#include "util/brpc_client_cache.h"
+#include "util/doris_metrics.h"
+#include "util/network_util.h"
+#include "util/threadpool.h"
+#include "util/thrift_util.h"
+#include "util/uid_util.h"
+#include "vec/core/block.h"
+#include "vec/exprs/vexpr.h"
+#include "vec/sink/delta_writer_v2_pool.h"
+#include "vec/sink/load_stream_stub.h"
+#include "vec/sink/load_stream_stub_pool.h"
+#include "vec/sink/vtablet_block_convertor.h"
+#include "vec/sink/vtablet_finder.h"
+
+namespace doris {
+
+namespace vectorized {

Review Comment:
   warning: nested namespaces can be concatenated 
[modernize-concat-nested-namespaces]
   
   ```suggestion
   namespace doris::vectorized {
   ```
   
   be/src/vec/sink/writer/vtablet_writer_v2.cpp:545:
   ```diff
   - } // namespace vectorized
   - } // namespace doris
   + } // namespace doris
   ```
   



##########
be/src/vec/sink/writer/vtablet_writer_v2.h:
##########
@@ -0,0 +1,233 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+#include <brpc/controller.h>

Review Comment:
   warning: 'brpc/controller.h' file not found [clang-diagnostic-error]
   ```cpp
   #include <brpc/controller.h>
            ^
   ```
   



##########
be/src/vec/sink/writer/vtablet_writer_v2.h:
##########
@@ -0,0 +1,233 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+#include <brpc/controller.h>
+#include <bthread/types.h>
+#include <butil/errno.h>
+#include <fmt/format.h>
+#include <gen_cpp/PaloInternalService_types.h>
+#include <gen_cpp/Types_types.h>
+#include <gen_cpp/internal_service.pb.h>
+#include <gen_cpp/types.pb.h>
+#include <glog/logging.h>
+#include <google/protobuf/stubs/callback.h>
+#include <stddef.h>
+#include <stdint.h>

Review Comment:
   warning: inclusion of deprecated C++ header 'stdint.h'; consider using 
'cstdint' instead [modernize-deprecated-headers]
   
   ```suggestion
   #include <cstdint>
   ```
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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

Reply via email to