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


##########
be/src/olap/rowset/segment_v2/vertical_segment_writer.h:
##########
@@ -0,0 +1,203 @@
+// 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 <butil/macros.h>

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



##########
be/src/olap/rowset/segment_v2/vertical_segment_writer.h:
##########
@@ -0,0 +1,203 @@
+// 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 <butil/macros.h>
+#include <gen_cpp/olap_file.pb.h>
+#include <gen_cpp/segment_v2.pb.h>
+#include <stddef.h>
+
+#include <cstdint>
+#include <functional>
+#include <map>
+#include <memory> // unique_ptr
+#include <string>
+#include <unordered_set>
+#include <vector>
+
+#include "common/status.h" // Status
+#include "gen_cpp/segment_v2.pb.h"
+#include "gutil/macros.h"
+#include "gutil/strings/substitute.h"
+#include "olap/olap_define.h"
+#include "olap/rowset/segment_v2/column_writer.h"
+#include "olap/tablet.h"
+#include "olap/tablet_schema.h"
+#include "util/faststring.h"
+#include "util/slice.h"
+
+namespace doris {
+namespace vectorized {
+class Block;
+class IOlapColumnDataAccessor;
+class OlapBlockDataConvertor;
+} // namespace vectorized
+
+class DataDir;
+class MemTracker;
+class ShortKeyIndexBuilder;
+class PrimaryKeyIndexBuilder;
+class KeyCoder;
+struct RowsetWriterContext;
+
+namespace io {
+class FileWriter;
+} // namespace io
+
+namespace segment_v2 {
+
+struct VerticalSegmentWriterOptions {
+    uint32_t num_rows_per_block = 1024;
+    bool enable_unique_key_merge_on_write = false;
+    CompressionTypePB compression_type = UNKNOWN_COMPRESSION;
+
+    RowsetWriterContext* rowset_ctx = nullptr;
+    DataWriteType write_type = DataWriteType::TYPE_DEFAULT;
+};
+
+struct RowsInBlock {
+    const vectorized::Block* block;
+    size_t row_pos;
+    size_t num_rows;
+};
+
+using TabletSharedPtr = std::shared_ptr<Tablet>;
+
+class VerticalSegmentWriter {
+public:
+    explicit VerticalSegmentWriter(io::FileWriter* file_writer, uint32_t 
segment_id,
+                                   TabletSchemaSPtr tablet_schema, 
TabletSharedPtr tablet,
+                                   DataDir* data_dir, uint32_t 
max_row_per_segment,
+                                   const VerticalSegmentWriterOptions& opts,
+                                   std::shared_ptr<MowContext> mow_context);
+    ~VerticalSegmentWriter();
+
+    Status init();
+
+    // Add one block to batch, memory is owned by the caller.
+    // The batched blocks will be flushed in write_batch.
+    // Once write_batch is called, no more blocks shoud be added.
+    Status batch_block(const vectorized::Block* block, size_t row_pos, size_t 
num_rows);
+    Status write_batch();
+
+    size_t inverted_index_file_size() const { return 
_inverted_index_file_size; }
+    uint32_t num_rows_written() const { return _num_rows_written; }

Review Comment:
   warning: function 'num_rows_written' should be marked [[nodiscard]] 
[modernize-use-nodiscard]
   
   ```suggestion
       [[nodiscard]] uint32_t num_rows_written() const { return 
_num_rows_written; }
   ```
   



##########
be/src/olap/rowset/segment_v2/vertical_segment_writer.h:
##########
@@ -0,0 +1,203 @@
+// 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 <butil/macros.h>
+#include <gen_cpp/olap_file.pb.h>
+#include <gen_cpp/segment_v2.pb.h>
+#include <stddef.h>
+
+#include <cstdint>
+#include <functional>
+#include <map>
+#include <memory> // unique_ptr
+#include <string>
+#include <unordered_set>
+#include <vector>
+
+#include "common/status.h" // Status
+#include "gen_cpp/segment_v2.pb.h"
+#include "gutil/macros.h"
+#include "gutil/strings/substitute.h"
+#include "olap/olap_define.h"
+#include "olap/rowset/segment_v2/column_writer.h"
+#include "olap/tablet.h"
+#include "olap/tablet_schema.h"
+#include "util/faststring.h"
+#include "util/slice.h"
+
+namespace doris {
+namespace vectorized {
+class Block;
+class IOlapColumnDataAccessor;
+class OlapBlockDataConvertor;
+} // namespace vectorized
+
+class DataDir;
+class MemTracker;
+class ShortKeyIndexBuilder;
+class PrimaryKeyIndexBuilder;
+class KeyCoder;
+struct RowsetWriterContext;
+
+namespace io {
+class FileWriter;
+} // namespace io
+
+namespace segment_v2 {
+
+struct VerticalSegmentWriterOptions {
+    uint32_t num_rows_per_block = 1024;
+    bool enable_unique_key_merge_on_write = false;
+    CompressionTypePB compression_type = UNKNOWN_COMPRESSION;
+
+    RowsetWriterContext* rowset_ctx = nullptr;
+    DataWriteType write_type = DataWriteType::TYPE_DEFAULT;
+};
+
+struct RowsInBlock {
+    const vectorized::Block* block;
+    size_t row_pos;
+    size_t num_rows;
+};
+
+using TabletSharedPtr = std::shared_ptr<Tablet>;
+
+class VerticalSegmentWriter {
+public:
+    explicit VerticalSegmentWriter(io::FileWriter* file_writer, uint32_t 
segment_id,
+                                   TabletSchemaSPtr tablet_schema, 
TabletSharedPtr tablet,
+                                   DataDir* data_dir, uint32_t 
max_row_per_segment,
+                                   const VerticalSegmentWriterOptions& opts,
+                                   std::shared_ptr<MowContext> mow_context);
+    ~VerticalSegmentWriter();
+
+    Status init();
+
+    // Add one block to batch, memory is owned by the caller.
+    // The batched blocks will be flushed in write_batch.
+    // Once write_batch is called, no more blocks shoud be added.
+    Status batch_block(const vectorized::Block* block, size_t row_pos, size_t 
num_rows);
+    Status write_batch();
+
+    size_t inverted_index_file_size() const { return 
_inverted_index_file_size; }
+    uint32_t num_rows_written() const { return _num_rows_written; }
+    int64_t num_rows_filtered() const { return _num_rows_filtered; }
+    uint32_t row_count() const { return _row_count; }
+
+    Status finalize(uint64_t* segment_file_size, uint64_t* index_size);
+
+    uint32_t segment_id() { return _segment_id; }
+
+    Status finalize_columns_index(uint64_t* index_size);
+    Status finalize_footer(uint64_t* segment_file_size);
+
+    Slice min_encoded_key();
+    Slice max_encoded_key();
+
+    std::string data_dir_path() const { return _data_dir == nullptr ? "" : 
_data_dir->path(); }
+
+    void clear();
+
+private:
+    DISALLOW_COPY_AND_ASSIGN(VerticalSegmentWriter);
+    void _init_column_meta(ColumnMetaPB* meta, uint32_t column_id, const 
TabletColumn& column);
+    Status _create_column_writer(uint32_t cid, const TabletColumn& column);
+    void _handle_delete_sign_col(const vectorized::Block* block, size_t 
row_pos, size_t num_rows,
+                                 size_t segment_start_pos);
+    size_t _calculate_inverted_index_file_size();
+    uint64_t _estimated_remaining_size();
+    Status _write_ordinal_index();
+    Status _write_zone_map();
+    Status _write_bitmap_index();
+    Status _write_inverted_index();
+    Status _write_bloom_filter_index();
+    Status _write_short_key_index();
+    Status _write_primary_key_index();
+    Status _write_footer();
+    Status _write_raw_data(const std::vector<Slice>& slices);
+    void _maybe_invalid_row_cache(const std::string& key);
+    std::string _encode_keys(const 
std::vector<vectorized::IOlapColumnDataAccessor*>& key_columns,
+                             size_t pos, bool null_first = true);
+    // used for unique-key with merge on write and segment min_max key
+    std::string _full_encode_keys(
+            const std::vector<vectorized::IOlapColumnDataAccessor*>& 
key_columns, size_t pos,
+            bool null_first = true);
+    // used for unique-key with merge on write
+    void _encode_seq_column(const vectorized::IOlapColumnDataAccessor* 
seq_column, size_t pos,
+                            string* encoded_keys);
+    void _set_min_max_key(const Slice& key);
+    void _set_min_key(const Slice& key);
+    void _set_max_key(const Slice& key);
+    void _serialize_block_to_row_column(vectorized::Block& block);
+    Status _append_block_with_partial_content(RowsInBlock& data);
+    Status _fill_missing_columns(vectorized::MutableColumns& 
mutable_full_columns,
+                                 const std::vector<bool>& 
use_default_or_null_flag,
+                                 bool has_default_or_nullable, const size_t& 
segment_start_pos);
+
+private:

Review Comment:
   warning: redundant access specifier has the same accessibility as the 
previous access specifier [readability-redundant-access-specifiers]
   
   ```suggestion
   
   ```
   <details>
   <summary>Additional context</summary>
   
   **be/src/olap/rowset/segment_v2/vertical_segment_writer.h:116:** previously 
declared here
   ```cpp
   private:
   ^
   ```
   
   </details>
   



##########
be/src/olap/rowset/segment_v2/vertical_segment_writer.h:
##########
@@ -0,0 +1,203 @@
+// 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 <butil/macros.h>
+#include <gen_cpp/olap_file.pb.h>
+#include <gen_cpp/segment_v2.pb.h>
+#include <stddef.h>
+
+#include <cstdint>
+#include <functional>
+#include <map>
+#include <memory> // unique_ptr
+#include <string>
+#include <unordered_set>
+#include <vector>
+
+#include "common/status.h" // Status
+#include "gen_cpp/segment_v2.pb.h"
+#include "gutil/macros.h"
+#include "gutil/strings/substitute.h"
+#include "olap/olap_define.h"
+#include "olap/rowset/segment_v2/column_writer.h"
+#include "olap/tablet.h"
+#include "olap/tablet_schema.h"
+#include "util/faststring.h"
+#include "util/slice.h"
+
+namespace doris {
+namespace vectorized {
+class Block;
+class IOlapColumnDataAccessor;
+class OlapBlockDataConvertor;
+} // namespace vectorized
+
+class DataDir;
+class MemTracker;
+class ShortKeyIndexBuilder;
+class PrimaryKeyIndexBuilder;
+class KeyCoder;
+struct RowsetWriterContext;
+
+namespace io {
+class FileWriter;
+} // namespace io
+
+namespace segment_v2 {
+
+struct VerticalSegmentWriterOptions {
+    uint32_t num_rows_per_block = 1024;
+    bool enable_unique_key_merge_on_write = false;
+    CompressionTypePB compression_type = UNKNOWN_COMPRESSION;
+
+    RowsetWriterContext* rowset_ctx = nullptr;
+    DataWriteType write_type = DataWriteType::TYPE_DEFAULT;
+};
+
+struct RowsInBlock {
+    const vectorized::Block* block;
+    size_t row_pos;
+    size_t num_rows;
+};
+
+using TabletSharedPtr = std::shared_ptr<Tablet>;
+
+class VerticalSegmentWriter {
+public:
+    explicit VerticalSegmentWriter(io::FileWriter* file_writer, uint32_t 
segment_id,
+                                   TabletSchemaSPtr tablet_schema, 
TabletSharedPtr tablet,
+                                   DataDir* data_dir, uint32_t 
max_row_per_segment,
+                                   const VerticalSegmentWriterOptions& opts,
+                                   std::shared_ptr<MowContext> mow_context);
+    ~VerticalSegmentWriter();
+
+    Status init();
+
+    // Add one block to batch, memory is owned by the caller.
+    // The batched blocks will be flushed in write_batch.
+    // Once write_batch is called, no more blocks shoud be added.
+    Status batch_block(const vectorized::Block* block, size_t row_pos, size_t 
num_rows);
+    Status write_batch();
+
+    size_t inverted_index_file_size() const { return 
_inverted_index_file_size; }
+    uint32_t num_rows_written() const { return _num_rows_written; }
+    int64_t num_rows_filtered() const { return _num_rows_filtered; }
+    uint32_t row_count() const { return _row_count; }
+
+    Status finalize(uint64_t* segment_file_size, uint64_t* index_size);
+
+    uint32_t segment_id() { return _segment_id; }

Review Comment:
   warning: method 'segment_id' can be made const 
[readability-make-member-function-const]
   
   ```suggestion
       uint32_t segment_id() const { return _segment_id; }
   ```
   



##########
be/src/olap/rowset/segment_v2/vertical_segment_writer.h:
##########
@@ -0,0 +1,203 @@
+// 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 <butil/macros.h>
+#include <gen_cpp/olap_file.pb.h>
+#include <gen_cpp/segment_v2.pb.h>
+#include <stddef.h>
+
+#include <cstdint>
+#include <functional>
+#include <map>
+#include <memory> // unique_ptr
+#include <string>
+#include <unordered_set>
+#include <vector>
+
+#include "common/status.h" // Status
+#include "gen_cpp/segment_v2.pb.h"
+#include "gutil/macros.h"
+#include "gutil/strings/substitute.h"
+#include "olap/olap_define.h"
+#include "olap/rowset/segment_v2/column_writer.h"
+#include "olap/tablet.h"
+#include "olap/tablet_schema.h"
+#include "util/faststring.h"
+#include "util/slice.h"
+
+namespace doris {
+namespace vectorized {
+class Block;
+class IOlapColumnDataAccessor;
+class OlapBlockDataConvertor;
+} // namespace vectorized
+
+class DataDir;
+class MemTracker;
+class ShortKeyIndexBuilder;
+class PrimaryKeyIndexBuilder;
+class KeyCoder;
+struct RowsetWriterContext;
+
+namespace io {
+class FileWriter;
+} // namespace io
+
+namespace segment_v2 {
+
+struct VerticalSegmentWriterOptions {
+    uint32_t num_rows_per_block = 1024;
+    bool enable_unique_key_merge_on_write = false;
+    CompressionTypePB compression_type = UNKNOWN_COMPRESSION;
+
+    RowsetWriterContext* rowset_ctx = nullptr;
+    DataWriteType write_type = DataWriteType::TYPE_DEFAULT;
+};
+
+struct RowsInBlock {
+    const vectorized::Block* block;
+    size_t row_pos;
+    size_t num_rows;
+};
+
+using TabletSharedPtr = std::shared_ptr<Tablet>;
+
+class VerticalSegmentWriter {
+public:
+    explicit VerticalSegmentWriter(io::FileWriter* file_writer, uint32_t 
segment_id,
+                                   TabletSchemaSPtr tablet_schema, 
TabletSharedPtr tablet,
+                                   DataDir* data_dir, uint32_t 
max_row_per_segment,
+                                   const VerticalSegmentWriterOptions& opts,
+                                   std::shared_ptr<MowContext> mow_context);
+    ~VerticalSegmentWriter();
+
+    Status init();
+
+    // Add one block to batch, memory is owned by the caller.
+    // The batched blocks will be flushed in write_batch.
+    // Once write_batch is called, no more blocks shoud be added.
+    Status batch_block(const vectorized::Block* block, size_t row_pos, size_t 
num_rows);
+    Status write_batch();
+
+    size_t inverted_index_file_size() const { return 
_inverted_index_file_size; }
+    uint32_t num_rows_written() const { return _num_rows_written; }
+    int64_t num_rows_filtered() const { return _num_rows_filtered; }
+    uint32_t row_count() const { return _row_count; }
+
+    Status finalize(uint64_t* segment_file_size, uint64_t* index_size);
+
+    uint32_t segment_id() { return _segment_id; }
+
+    Status finalize_columns_index(uint64_t* index_size);
+    Status finalize_footer(uint64_t* segment_file_size);
+
+    Slice min_encoded_key();
+    Slice max_encoded_key();
+
+    std::string data_dir_path() const { return _data_dir == nullptr ? "" : 
_data_dir->path(); }

Review Comment:
   warning: function 'data_dir_path' should be marked [[nodiscard]] 
[modernize-use-nodiscard]
   
   ```suggestion
       [[nodiscard]] std::string data_dir_path() const { return _data_dir == 
nullptr ? "" : _data_dir->path(); }
   ```
   



##########
be/src/olap/rowset/segment_v2/vertical_segment_writer.h:
##########
@@ -0,0 +1,203 @@
+// 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 <butil/macros.h>
+#include <gen_cpp/olap_file.pb.h>
+#include <gen_cpp/segment_v2.pb.h>
+#include <stddef.h>
+
+#include <cstdint>
+#include <functional>
+#include <map>
+#include <memory> // unique_ptr
+#include <string>
+#include <unordered_set>
+#include <vector>
+
+#include "common/status.h" // Status
+#include "gen_cpp/segment_v2.pb.h"
+#include "gutil/macros.h"
+#include "gutil/strings/substitute.h"
+#include "olap/olap_define.h"
+#include "olap/rowset/segment_v2/column_writer.h"
+#include "olap/tablet.h"
+#include "olap/tablet_schema.h"
+#include "util/faststring.h"
+#include "util/slice.h"
+
+namespace doris {
+namespace vectorized {
+class Block;
+class IOlapColumnDataAccessor;
+class OlapBlockDataConvertor;
+} // namespace vectorized
+
+class DataDir;
+class MemTracker;
+class ShortKeyIndexBuilder;
+class PrimaryKeyIndexBuilder;
+class KeyCoder;
+struct RowsetWriterContext;
+
+namespace io {
+class FileWriter;
+} // namespace io
+
+namespace segment_v2 {
+
+struct VerticalSegmentWriterOptions {
+    uint32_t num_rows_per_block = 1024;
+    bool enable_unique_key_merge_on_write = false;
+    CompressionTypePB compression_type = UNKNOWN_COMPRESSION;
+
+    RowsetWriterContext* rowset_ctx = nullptr;
+    DataWriteType write_type = DataWriteType::TYPE_DEFAULT;
+};
+
+struct RowsInBlock {
+    const vectorized::Block* block;
+    size_t row_pos;
+    size_t num_rows;
+};
+
+using TabletSharedPtr = std::shared_ptr<Tablet>;
+
+class VerticalSegmentWriter {
+public:
+    explicit VerticalSegmentWriter(io::FileWriter* file_writer, uint32_t 
segment_id,
+                                   TabletSchemaSPtr tablet_schema, 
TabletSharedPtr tablet,
+                                   DataDir* data_dir, uint32_t 
max_row_per_segment,
+                                   const VerticalSegmentWriterOptions& opts,
+                                   std::shared_ptr<MowContext> mow_context);
+    ~VerticalSegmentWriter();
+
+    Status init();
+
+    // Add one block to batch, memory is owned by the caller.
+    // The batched blocks will be flushed in write_batch.
+    // Once write_batch is called, no more blocks shoud be added.
+    Status batch_block(const vectorized::Block* block, size_t row_pos, size_t 
num_rows);
+    Status write_batch();
+
+    size_t inverted_index_file_size() const { return 
_inverted_index_file_size; }

Review Comment:
   warning: function 'inverted_index_file_size' should be marked [[nodiscard]] 
[modernize-use-nodiscard]
   
   ```suggestion
       [[nodiscard]] size_t inverted_index_file_size() const { return 
_inverted_index_file_size; }
   ```
   



##########
be/src/olap/rowset/segment_v2/vertical_segment_writer.h:
##########
@@ -0,0 +1,203 @@
+// 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 <butil/macros.h>
+#include <gen_cpp/olap_file.pb.h>
+#include <gen_cpp/segment_v2.pb.h>
+#include <stddef.h>
+
+#include <cstdint>
+#include <functional>
+#include <map>
+#include <memory> // unique_ptr
+#include <string>
+#include <unordered_set>
+#include <vector>
+
+#include "common/status.h" // Status
+#include "gen_cpp/segment_v2.pb.h"
+#include "gutil/macros.h"
+#include "gutil/strings/substitute.h"
+#include "olap/olap_define.h"
+#include "olap/rowset/segment_v2/column_writer.h"
+#include "olap/tablet.h"
+#include "olap/tablet_schema.h"
+#include "util/faststring.h"
+#include "util/slice.h"
+
+namespace doris {
+namespace vectorized {
+class Block;
+class IOlapColumnDataAccessor;
+class OlapBlockDataConvertor;
+} // namespace vectorized
+
+class DataDir;
+class MemTracker;
+class ShortKeyIndexBuilder;
+class PrimaryKeyIndexBuilder;
+class KeyCoder;
+struct RowsetWriterContext;
+
+namespace io {
+class FileWriter;
+} // namespace io
+
+namespace segment_v2 {
+
+struct VerticalSegmentWriterOptions {
+    uint32_t num_rows_per_block = 1024;
+    bool enable_unique_key_merge_on_write = false;
+    CompressionTypePB compression_type = UNKNOWN_COMPRESSION;
+
+    RowsetWriterContext* rowset_ctx = nullptr;
+    DataWriteType write_type = DataWriteType::TYPE_DEFAULT;
+};
+
+struct RowsInBlock {
+    const vectorized::Block* block;
+    size_t row_pos;
+    size_t num_rows;
+};
+
+using TabletSharedPtr = std::shared_ptr<Tablet>;
+
+class VerticalSegmentWriter {
+public:
+    explicit VerticalSegmentWriter(io::FileWriter* file_writer, uint32_t 
segment_id,
+                                   TabletSchemaSPtr tablet_schema, 
TabletSharedPtr tablet,
+                                   DataDir* data_dir, uint32_t 
max_row_per_segment,
+                                   const VerticalSegmentWriterOptions& opts,
+                                   std::shared_ptr<MowContext> mow_context);
+    ~VerticalSegmentWriter();
+
+    Status init();
+
+    // Add one block to batch, memory is owned by the caller.
+    // The batched blocks will be flushed in write_batch.
+    // Once write_batch is called, no more blocks shoud be added.
+    Status batch_block(const vectorized::Block* block, size_t row_pos, size_t 
num_rows);
+    Status write_batch();
+
+    size_t inverted_index_file_size() const { return 
_inverted_index_file_size; }
+    uint32_t num_rows_written() const { return _num_rows_written; }
+    int64_t num_rows_filtered() const { return _num_rows_filtered; }

Review Comment:
   warning: function 'num_rows_filtered' should be marked [[nodiscard]] 
[modernize-use-nodiscard]
   
   ```suggestion
       [[nodiscard]] int64_t num_rows_filtered() const { return 
_num_rows_filtered; }
   ```
   



##########
be/src/olap/rowset/segment_v2/vertical_segment_writer.cpp:
##########
@@ -0,0 +1,1025 @@
+// 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/rowset/segment_v2/vertical_segment_writer.h"
+
+#include <assert.h>
+#include <gen_cpp/segment_v2.pb.h>
+#include <parallel_hashmap/phmap.h>
+
+#include <algorithm>
+#include <ostream>
+#include <unordered_map>
+#include <utility>
+
+// IWYU pragma: no_include <opentelemetry/common/threadlocal.h>
+#include "common/compiler_util.h" // IWYU pragma: keep
+#include "common/config.h"
+#include "common/logging.h" // LOG
+#include "gutil/port.h"
+#include "io/fs/file_writer.h"
+#include "olap/data_dir.h"
+#include "olap/key_coder.h"
+#include "olap/olap_common.h"
+#include "olap/primary_key_index.h"
+#include "olap/row_cursor.h"                      // RowCursor // IWYU pragma: 
keep
+#include "olap/rowset/rowset_writer_context.h"    // RowsetWriterContext
+#include "olap/rowset/segment_v2/column_writer.h" // ColumnWriter
+#include "olap/rowset/segment_v2/page_io.h"
+#include "olap/rowset/segment_v2/page_pointer.h"
+#include "olap/segment_loader.h"
+#include "olap/short_key_index.h"
+#include "olap/tablet_schema.h"
+#include "olap/utils.h"
+#include "runtime/memory/mem_tracker.h"
+#include "service/point_query_executor.h"
+#include "util/coding.h"
+#include "util/crc32c.h"
+#include "util/faststring.h"
+#include "util/key_util.h"
+#include "vec/columns/column_nullable.h"
+#include "vec/common/schema_util.h"
+#include "vec/core/block.h"
+#include "vec/core/column_with_type_and_name.h"
+#include "vec/core/types.h"
+#include "vec/io/reader_buffer.h"
+#include "vec/jsonb/serialize.h"
+#include "vec/olap/olap_data_convertor.h"
+
+namespace doris {
+namespace segment_v2 {
+
+using namespace ErrorCode;
+
+static const char* k_segment_magic = "D0R1";
+static const uint32_t k_segment_magic_length = 4;
+
+VerticalSegmentWriter::VerticalSegmentWriter(io::FileWriter* file_writer, 
uint32_t segment_id,
+                                             TabletSchemaSPtr tablet_schema, 
TabletSharedPtr tablet,
+                                             DataDir* data_dir, uint32_t 
max_row_per_segment,
+                                             const 
VerticalSegmentWriterOptions& opts,
+                                             std::shared_ptr<MowContext> 
mow_context)
+        : _segment_id(segment_id),
+          _tablet_schema(tablet_schema),
+          _tablet(tablet),
+          _data_dir(data_dir),
+          _max_row_per_segment(max_row_per_segment),
+          _opts(opts),
+          _file_writer(file_writer),
+          
_mem_tracker(std::make_unique<MemTracker>("VerticalSegmentWriter:Segment-" +
+                                                    
std::to_string(segment_id))),
+          _mow_context(mow_context) {
+    CHECK_NOTNULL(file_writer);
+    _num_key_columns = _tablet_schema->num_key_columns();
+    _num_short_key_columns = _tablet_schema->num_short_key_columns();
+    DCHECK(_num_key_columns >= _num_short_key_columns);
+    for (size_t cid = 0; cid < _num_key_columns; ++cid) {
+        const auto& column = _tablet_schema->column(cid);
+        _key_coders.push_back(get_key_coder(column.type()));
+        _key_index_size.push_back(column.index_length());
+    }
+    // encode the sequence id into the primary key index
+    if (_tablet_schema->has_sequence_col() && _tablet_schema->keys_type() == 
UNIQUE_KEYS &&
+        _opts.enable_unique_key_merge_on_write) {
+        const auto& column = 
_tablet_schema->column(_tablet_schema->sequence_col_idx());
+        _seq_coder = get_key_coder(column.type());
+    }
+}
+
+VerticalSegmentWriter::~VerticalSegmentWriter() {
+    _mem_tracker->release(_mem_tracker->consumption());
+}
+
+void VerticalSegmentWriter::_init_column_meta(ColumnMetaPB* meta, uint32_t 
column_id,
+                                              const TabletColumn& column) {
+    meta->set_column_id(column_id);
+    meta->set_unique_id(column.unique_id());
+    meta->set_type(int(column.type()));
+    meta->set_length(column.length());
+    meta->set_encoding(DEFAULT_ENCODING);
+    meta->set_compression(_opts.compression_type);
+    meta->set_is_nullable(column.is_nullable());
+    for (uint32_t i = 0; i < column.get_subtype_count(); ++i) {
+        _init_column_meta(meta->add_children_columns(), column_id, 
column.get_sub_column(i));
+    }
+}
+
+Status VerticalSegmentWriter::_create_column_writer(uint32_t cid, const 
TabletColumn& column) {
+    ColumnWriterOptions opts;
+    opts.meta = _footer.add_columns();
+
+    _init_column_meta(opts.meta, cid, column);
+
+    // now we create zone map for key columns in AGG_KEYS or all column in 
UNIQUE_KEYS or DUP_KEYS
+    // and not support zone map for array type and jsonb type.
+    opts.need_zone_map = (column.is_key() || _tablet_schema->keys_type() != 
KeysType::AGG_KEYS) &&
+                         column.type() != FieldType::OLAP_FIELD_TYPE_OBJECT;
+    opts.need_bloom_filter = column.is_bf_column();
+    auto* tablet_index = 
_tablet_schema->get_ngram_bf_index(column.unique_id());
+    if (tablet_index) {
+        opts.need_bloom_filter = true;
+        opts.is_ngram_bf_index = true;
+        opts.gram_size = tablet_index->get_gram_size();
+        opts.gram_bf_size = tablet_index->get_gram_bf_size();
+    }
+
+    opts.need_bitmap_index = column.has_bitmap_index();
+    bool skip_inverted_index = false;
+    if (_opts.rowset_ctx != nullptr) {
+        // skip write inverted index for index compaction
+        skip_inverted_index = 
_opts.rowset_ctx->skip_inverted_index.count(column.unique_id()) > 0;
+    }
+    // skip write inverted index on load if skip_write_index_on_load is true
+    if (_opts.write_type == DataWriteType::TYPE_DIRECT &&
+        _tablet_schema->skip_write_index_on_load()) {
+        skip_inverted_index = true;
+    }
+    // indexes for this column
+    opts.indexes = _tablet_schema->get_indexes_for_column(column.unique_id());
+    for (auto index : opts.indexes) {
+        if (!skip_inverted_index && index && index->index_type() == 
IndexType::INVERTED) {
+            opts.inverted_index = index;
+            // TODO support multiple inverted index
+            break;
+        }
+    }
+    if (column.type() == FieldType::OLAP_FIELD_TYPE_STRUCT) {
+        opts.need_zone_map = false;
+        if (opts.need_bloom_filter) {
+            return Status::NotSupported("Do not support bloom filter for 
struct type");
+        }
+        if (opts.need_bitmap_index) {
+            return Status::NotSupported("Do not support bitmap index for 
struct type");
+        }
+    }
+    if (column.type() == FieldType::OLAP_FIELD_TYPE_ARRAY) {
+        opts.need_zone_map = false;
+        if (opts.need_bloom_filter) {
+            return Status::NotSupported("Do not support bloom filter for array 
type");
+        }
+        if (opts.need_bitmap_index) {
+            return Status::NotSupported("Do not support bitmap index for array 
type");
+        }
+    }
+    if (column.type() == FieldType::OLAP_FIELD_TYPE_JSONB) {
+        opts.need_zone_map = false;
+        if (opts.need_bloom_filter) {
+            return Status::NotSupported("Do not support bloom filter for jsonb 
type");
+        }
+        if (opts.need_bitmap_index) {
+            return Status::NotSupported("Do not support bitmap index for jsonb 
type");
+        }
+    }
+    if (column.type() == FieldType::OLAP_FIELD_TYPE_AGG_STATE) {
+        opts.need_zone_map = false;
+        if (opts.need_bloom_filter) {
+            return Status::NotSupported("Do not support bloom filter for 
agg_state type");
+        }
+        if (opts.need_bitmap_index) {
+            return Status::NotSupported("Do not support bitmap index for 
agg_state type");
+        }
+    }
+    if (column.type() == FieldType::OLAP_FIELD_TYPE_MAP) {
+        opts.need_zone_map = false;
+        if (opts.need_bloom_filter) {
+            return Status::NotSupported("Do not support bloom filter for map 
type");
+        }
+        if (opts.need_bitmap_index) {
+            return Status::NotSupported("Do not support bitmap index for map 
type");
+        }
+    }
+
+    if (column.is_row_store_column()) {
+        // smaller page size for row store column
+        opts.data_page_size = config::row_column_page_size;
+    }
+
+    std::unique_ptr<ColumnWriter> writer;
+    RETURN_IF_ERROR(ColumnWriter::create(opts, &column, _file_writer, 
&writer));
+    RETURN_IF_ERROR(writer->init());
+    _column_writers.push_back(std::move(writer));
+
+    _olap_data_convertor->add_column_data_convertor(column);
+    return Status::OK();
+};
+
+Status VerticalSegmentWriter::init() {
+    DCHECK(_column_writers.empty());
+    if (_opts.compression_type == UNKNOWN_COMPRESSION) {
+        _opts.compression_type = _tablet_schema->compression_type();
+    }
+    _olap_data_convertor = 
std::make_unique<vectorized::OlapBlockDataConvertor>();
+    _olap_data_convertor->reserve(_tablet_schema->num_columns());
+    _column_writers.reserve(_tablet_schema->columns().size());
+    // we don't need the short key index for unique key merge on write table.
+    if (_tablet_schema->keys_type() == UNIQUE_KEYS && 
_opts.enable_unique_key_merge_on_write) {
+        size_t seq_col_length = 0;
+        if (_tablet_schema->has_sequence_col()) {
+            seq_col_length =
+                    
_tablet_schema->column(_tablet_schema->sequence_col_idx()).length() + 1;
+        }
+        _primary_key_index_builder.reset(new 
PrimaryKeyIndexBuilder(_file_writer, seq_col_length));
+        RETURN_IF_ERROR(_primary_key_index_builder->init());
+    } else {
+        _short_key_index_builder.reset(
+                new ShortKeyIndexBuilder(_segment_id, 
_opts.num_rows_per_block));
+    }
+    return Status::OK();
+}
+
+void VerticalSegmentWriter::_maybe_invalid_row_cache(const std::string& key) {

Review Comment:
   warning: method '_maybe_invalid_row_cache' can be made const 
[readability-make-member-function-const]
   
   ```suggestion
   void VerticalSegmentWriter::_maybe_invalid_row_cache(const std::string& key) 
const {
   ```
   



##########
be/src/olap/rowset/segment_v2/vertical_segment_writer.h:
##########
@@ -0,0 +1,203 @@
+// 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 <butil/macros.h>
+#include <gen_cpp/olap_file.pb.h>
+#include <gen_cpp/segment_v2.pb.h>
+#include <stddef.h>
+
+#include <cstdint>
+#include <functional>
+#include <map>
+#include <memory> // unique_ptr
+#include <string>
+#include <unordered_set>
+#include <vector>
+
+#include "common/status.h" // Status
+#include "gen_cpp/segment_v2.pb.h"
+#include "gutil/macros.h"
+#include "gutil/strings/substitute.h"
+#include "olap/olap_define.h"
+#include "olap/rowset/segment_v2/column_writer.h"
+#include "olap/tablet.h"
+#include "olap/tablet_schema.h"
+#include "util/faststring.h"
+#include "util/slice.h"
+
+namespace doris {
+namespace vectorized {
+class Block;
+class IOlapColumnDataAccessor;
+class OlapBlockDataConvertor;
+} // namespace vectorized
+
+class DataDir;
+class MemTracker;
+class ShortKeyIndexBuilder;
+class PrimaryKeyIndexBuilder;
+class KeyCoder;
+struct RowsetWriterContext;
+
+namespace io {
+class FileWriter;
+} // namespace io
+
+namespace segment_v2 {
+
+struct VerticalSegmentWriterOptions {
+    uint32_t num_rows_per_block = 1024;
+    bool enable_unique_key_merge_on_write = false;
+    CompressionTypePB compression_type = UNKNOWN_COMPRESSION;
+
+    RowsetWriterContext* rowset_ctx = nullptr;
+    DataWriteType write_type = DataWriteType::TYPE_DEFAULT;
+};
+
+struct RowsInBlock {
+    const vectorized::Block* block;
+    size_t row_pos;
+    size_t num_rows;
+};
+
+using TabletSharedPtr = std::shared_ptr<Tablet>;
+
+class VerticalSegmentWriter {
+public:
+    explicit VerticalSegmentWriter(io::FileWriter* file_writer, uint32_t 
segment_id,
+                                   TabletSchemaSPtr tablet_schema, 
TabletSharedPtr tablet,
+                                   DataDir* data_dir, uint32_t 
max_row_per_segment,
+                                   const VerticalSegmentWriterOptions& opts,
+                                   std::shared_ptr<MowContext> mow_context);
+    ~VerticalSegmentWriter();
+
+    Status init();
+
+    // Add one block to batch, memory is owned by the caller.
+    // The batched blocks will be flushed in write_batch.
+    // Once write_batch is called, no more blocks shoud be added.
+    Status batch_block(const vectorized::Block* block, size_t row_pos, size_t 
num_rows);
+    Status write_batch();
+
+    size_t inverted_index_file_size() const { return 
_inverted_index_file_size; }
+    uint32_t num_rows_written() const { return _num_rows_written; }
+    int64_t num_rows_filtered() const { return _num_rows_filtered; }
+    uint32_t row_count() const { return _row_count; }

Review Comment:
   warning: function 'row_count' should be marked [[nodiscard]] 
[modernize-use-nodiscard]
   
   ```suggestion
       [[nodiscard]] uint32_t row_count() const { return _row_count; }
   ```
   



##########
be/src/olap/rowset/segment_v2/vertical_segment_writer.cpp:
##########
@@ -0,0 +1,1025 @@
+// 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/rowset/segment_v2/vertical_segment_writer.h"
+
+#include <assert.h>
+#include <gen_cpp/segment_v2.pb.h>
+#include <parallel_hashmap/phmap.h>
+
+#include <algorithm>
+#include <ostream>
+#include <unordered_map>
+#include <utility>
+
+// IWYU pragma: no_include <opentelemetry/common/threadlocal.h>
+#include "common/compiler_util.h" // IWYU pragma: keep
+#include "common/config.h"
+#include "common/logging.h" // LOG
+#include "gutil/port.h"
+#include "io/fs/file_writer.h"
+#include "olap/data_dir.h"
+#include "olap/key_coder.h"
+#include "olap/olap_common.h"
+#include "olap/primary_key_index.h"
+#include "olap/row_cursor.h"                      // RowCursor // IWYU pragma: 
keep
+#include "olap/rowset/rowset_writer_context.h"    // RowsetWriterContext
+#include "olap/rowset/segment_v2/column_writer.h" // ColumnWriter
+#include "olap/rowset/segment_v2/page_io.h"
+#include "olap/rowset/segment_v2/page_pointer.h"
+#include "olap/segment_loader.h"
+#include "olap/short_key_index.h"
+#include "olap/tablet_schema.h"
+#include "olap/utils.h"
+#include "runtime/memory/mem_tracker.h"
+#include "service/point_query_executor.h"
+#include "util/coding.h"
+#include "util/crc32c.h"
+#include "util/faststring.h"
+#include "util/key_util.h"
+#include "vec/columns/column_nullable.h"
+#include "vec/common/schema_util.h"
+#include "vec/core/block.h"
+#include "vec/core/column_with_type_and_name.h"
+#include "vec/core/types.h"
+#include "vec/io/reader_buffer.h"
+#include "vec/jsonb/serialize.h"
+#include "vec/olap/olap_data_convertor.h"
+
+namespace doris {
+namespace segment_v2 {
+
+using namespace ErrorCode;
+
+static const char* k_segment_magic = "D0R1";
+static const uint32_t k_segment_magic_length = 4;
+
+VerticalSegmentWriter::VerticalSegmentWriter(io::FileWriter* file_writer, 
uint32_t segment_id,
+                                             TabletSchemaSPtr tablet_schema, 
TabletSharedPtr tablet,
+                                             DataDir* data_dir, uint32_t 
max_row_per_segment,
+                                             const 
VerticalSegmentWriterOptions& opts,
+                                             std::shared_ptr<MowContext> 
mow_context)
+        : _segment_id(segment_id),
+          _tablet_schema(tablet_schema),
+          _tablet(tablet),
+          _data_dir(data_dir),
+          _max_row_per_segment(max_row_per_segment),
+          _opts(opts),
+          _file_writer(file_writer),
+          
_mem_tracker(std::make_unique<MemTracker>("VerticalSegmentWriter:Segment-" +
+                                                    
std::to_string(segment_id))),
+          _mow_context(mow_context) {
+    CHECK_NOTNULL(file_writer);
+    _num_key_columns = _tablet_schema->num_key_columns();
+    _num_short_key_columns = _tablet_schema->num_short_key_columns();
+    DCHECK(_num_key_columns >= _num_short_key_columns);
+    for (size_t cid = 0; cid < _num_key_columns; ++cid) {
+        const auto& column = _tablet_schema->column(cid);
+        _key_coders.push_back(get_key_coder(column.type()));
+        _key_index_size.push_back(column.index_length());
+    }
+    // encode the sequence id into the primary key index
+    if (_tablet_schema->has_sequence_col() && _tablet_schema->keys_type() == 
UNIQUE_KEYS &&
+        _opts.enable_unique_key_merge_on_write) {
+        const auto& column = 
_tablet_schema->column(_tablet_schema->sequence_col_idx());
+        _seq_coder = get_key_coder(column.type());
+    }
+}
+
+VerticalSegmentWriter::~VerticalSegmentWriter() {

Review Comment:
   warning: use '= default' to define a trivial destructor 
[modernize-use-equals-default]
   ```cpp
   VerticalSegmentWriter::~VerticalSegmentWriter() {
                          ^
   ```
   



-- 
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