github-actions[bot] commented on code in PR #36474: URL: https://github.com/apache/doris/pull/36474#discussion_r1644175061
########## be/src/vec/sink/writer/iceberg/partition_writer.h: ########## @@ -0,0 +1,64 @@ +// 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 + +namespace doris { +namespace io { +class FileSystem; +} + +class ObjectPool; +class RuntimeState; +class RuntimeProfile; + +namespace iceberg { +class Schema; +} + +namespace vectorized { + +class Block; +class VFileFormatTransformer; + +class IPartitionWriter { +public: + struct WriteInfo { + std::string write_path; + std::string original_write_path; + std::string target_path; Review Comment: warning: use of undeclared identifier 'std' [clang-diagnostic-error] ```cpp std::string target_path; ^ ``` ########## be/src/vec/sink/writer/iceberg/partition_writer.h: ########## @@ -0,0 +1,64 @@ +// 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 + +namespace doris { +namespace io { +class FileSystem; +} + +class ObjectPool; +class RuntimeState; +class RuntimeProfile; + +namespace iceberg { +class Schema; +} + +namespace vectorized { + +class Block; +class VFileFormatTransformer; + +class IPartitionWriter { +public: + struct WriteInfo { + std::string write_path; Review Comment: warning: use of undeclared identifier 'std' [clang-diagnostic-error] ```cpp std::string write_path; ^ ``` ########## be/src/vec/sink/writer/iceberg/partition_writer.h: ########## @@ -0,0 +1,64 @@ +// 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 + +namespace doris { +namespace io { +class FileSystem; +} + +class ObjectPool; +class RuntimeState; +class RuntimeProfile; + +namespace iceberg { +class Schema; +} + +namespace vectorized { + +class Block; +class VFileFormatTransformer; + +class IPartitionWriter { +public: + struct WriteInfo { + std::string write_path; + std::string original_write_path; + std::string target_path; + TFileType::type file_type; Review Comment: warning: use of undeclared identifier 'TFileType' [clang-diagnostic-error] ```cpp TFileType::type file_type; ^ ``` ########## be/src/vec/sink/writer/iceberg/partition_writer.h: ########## @@ -0,0 +1,64 @@ +// 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 + +namespace doris { +namespace io { +class FileSystem; +} + +class ObjectPool; +class RuntimeState; +class RuntimeProfile; + +namespace iceberg { +class Schema; +} + +namespace vectorized { + +class Block; +class VFileFormatTransformer; + +class IPartitionWriter { +public: + struct WriteInfo { + std::string write_path; + std::string original_write_path; + std::string target_path; + TFileType::type file_type; + }; + + IPartitionWriter() = default; + virtual ~IPartitionWriter() = default; + + virtual Status open(RuntimeState* state, RuntimeProfile* profile, const RowDescriptor* row_desc, + ObjectPool* pool) = 0; + + virtual Status write(vectorized::Block& block) = 0; Review Comment: warning: unknown type name 'Status' [clang-diagnostic-error] ```cpp virtual Status write(vectorized::Block& block) = 0; ^ ``` ########## be/src/vec/sink/writer/iceberg/partition_writer.h: ########## @@ -0,0 +1,64 @@ +// 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 + +namespace doris { +namespace io { +class FileSystem; +} + +class ObjectPool; +class RuntimeState; +class RuntimeProfile; + +namespace iceberg { +class Schema; +} + +namespace vectorized { + +class Block; +class VFileFormatTransformer; + +class IPartitionWriter { +public: + struct WriteInfo { + std::string write_path; + std::string original_write_path; + std::string target_path; + TFileType::type file_type; + }; + + IPartitionWriter() = default; + virtual ~IPartitionWriter() = default; + + virtual Status open(RuntimeState* state, RuntimeProfile* profile, const RowDescriptor* row_desc, + ObjectPool* pool) = 0; + + virtual Status write(vectorized::Block& block) = 0; + + virtual Status close(const Status& status) = 0; + + virtual const std::string& file_name() const = 0; + + virtual int file_name_index() const = 0; + + virtual size_t written_len() = 0; Review Comment: warning: unknown type name 'size_t' [clang-diagnostic-error] ```cpp virtual size_t written_len() = 0; ^ ``` ########## be/src/vec/sink/writer/iceberg/partition_writer.h: ########## @@ -0,0 +1,64 @@ +// 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 + +namespace doris { +namespace io { +class FileSystem; +} + +class ObjectPool; +class RuntimeState; +class RuntimeProfile; + +namespace iceberg { +class Schema; +} + +namespace vectorized { + +class Block; +class VFileFormatTransformer; + +class IPartitionWriter { +public: + struct WriteInfo { + std::string write_path; + std::string original_write_path; + std::string target_path; + TFileType::type file_type; + }; + + IPartitionWriter() = default; + virtual ~IPartitionWriter() = default; + + virtual Status open(RuntimeState* state, RuntimeProfile* profile, const RowDescriptor* row_desc, + ObjectPool* pool) = 0; + + virtual Status write(vectorized::Block& block) = 0; + + virtual Status close(const Status& status) = 0; + + virtual const std::string& file_name() const = 0; Review Comment: warning: use of undeclared identifier 'std' [clang-diagnostic-error] ```cpp virtual const std::string& file_name() const = 0; ^ ``` ########## be/src/vec/sink/writer/iceberg/sorting_file_writer.h: ########## @@ -0,0 +1,323 @@ +// 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 "runtime/types.h" +//#include "vec/functions/function_string.h" +//#include "vec/data_types/data_type_factory.hpp" +//#include "util/bit_util.h" + +#include <vector> + +#include "runtime/runtime_state.h" +#include "vec/columns/column.h" +#include "vec/common/sort/sorter.h" +#include "vec/core/block.h" +#include "vec/exprs/vslot_ref.h" +#include "vec/sink/writer/iceberg/partition_writer.h" +#include "vec/spill/spill_stream.h" +#include "vec/spill/spill_stream_manager.h" + +namespace doris { + +class RuntimeState; +class RuntimeProfile; + +namespace iceberg {}; // namespace iceberg + +namespace vectorized { +class Block; + +template <typename OutputWriter> +class SortingFileWriter : public IPartitionWriter { +public: + SortingFileWriter(const TDataSink& t_sink, std::vector<std::string> partition_values, + const VExprContextSPtrs& write_output_expr_ctxs, + const doris::iceberg::Schema& schema, const std::string* iceberg_schema_json, + std::vector<std::string> write_column_names, WriteInfo write_info, + std::string file_name, int file_name_index, + TFileFormatType::type file_format_type, TFileCompressType::type compress_type, + const std::map<std::string, std::string>& hadoop_conf) + : _t_sink(t_sink), + _is_asc_order(t_sink.iceberg_table_sink.sort_info.is_asc_order), + _nulls_first(t_sink.iceberg_table_sink.sort_info.nulls_first), + _output_writer(t_sink, std::move(partition_values), write_output_expr_ctxs, schema, + iceberg_schema_json, write_column_names, std::move(write_info), + std::move(file_name), file_name_index, file_format_type, compress_type, + hadoop_conf) {} + + Status open(RuntimeState* state, RuntimeProfile* profile, const RowDescriptor* row_desc, + ObjectPool* pool) override { + DCHECK(row_desc); + _state = state; + _profile = profile; + + _spill_counters = ADD_LABEL_COUNTER_WITH_LEVEL(_profile, "Spill", 1); + _spill_timer = ADD_CHILD_TIMER_WITH_LEVEL(_profile, "SpillTime", "Spill", 1); + _spill_serialize_block_timer = + ADD_CHILD_TIMER_WITH_LEVEL(_profile, "SpillSerializeBlockTime", "Spill", 1); + _spill_write_disk_timer = + ADD_CHILD_TIMER_WITH_LEVEL(_profile, "SpillWriteDiskTime", "Spill", 1); + _spill_data_size = ADD_CHILD_COUNTER_WITH_LEVEL(_profile, "SpillWriteDataSize", + TUnit::BYTES, "Spill", 1); + _spill_block_count = ADD_CHILD_COUNTER_WITH_LEVEL(_profile, "SpillWriteBlockCount", + TUnit::UNIT, "Spill", 1); + _spill_wait_in_queue_timer = + ADD_CHILD_TIMER_WITH_LEVEL(_profile, "SpillWaitInQueueTime", "Spill", 1); + _spill_write_wait_io_timer = + ADD_CHILD_TIMER_WITH_LEVEL(_profile, "SpillWriteWaitIOTime", "Spill", 1); + _spill_read_wait_io_timer = + ADD_CHILD_TIMER_WITH_LEVEL(_profile, "SpillReadWaitIOTime", "Spill", 1); + + _spill_recover_time = ADD_CHILD_TIMER_WITH_LEVEL(_profile, "SpillRecoverTime", "Spill", 1); + _spill_read_data_time = + ADD_CHILD_TIMER_WITH_LEVEL(_profile, "SpillReadDataTime", "Spill", 1); + _spill_deserialize_time = + ADD_CHILD_TIMER_WITH_LEVEL(_profile, "SpillDeserializeTime", "Spill", 1); + _spill_read_bytes = ADD_CHILD_COUNTER_WITH_LEVEL(_profile, "SpillReadDataSize", + TUnit::BYTES, "Spill", 1); + _spill_read_wait_io_timer = + ADD_CHILD_TIMER_WITH_LEVEL(_profile, "SpillReadWaitIOTime", "Spill", 1); + + int limit = -1; + int64_t offset = 0; + auto& iceberg_table_sink = _t_sink.iceberg_table_sink; + RETURN_IF_ERROR(_vsort_exec_exprs.init(iceberg_table_sink.sort_info, pool)); + RETURN_IF_ERROR(_vsort_exec_exprs.prepare(state, *row_desc, *row_desc)); + RETURN_IF_ERROR(_vsort_exec_exprs.open(state)); + + _sorter = vectorized::FullSorter::create_unique(_vsort_exec_exprs, limit, offset, pool, + _is_asc_order, _nulls_first, *row_desc, + state, _profile); + _sorter->set_enable_spill(true); + + _merger = std::make_unique<vectorized::VSortedRunMerger>(_sorter->get_sort_description(), + SPILL_BLOCK_BATCH_ROW_COUNT, limit, + offset, _profile); + RETURN_IF_ERROR(_output_writer.open(state, profile, row_desc, pool)); + return Status::OK(); + } + + Status write(vectorized::Block& block) override { + RETURN_IF_ERROR(_sorter->append_block(&block)); + _update_spill_block_batch_row_count(block); + + if (_sorter->state().get_sorted_block().empty()) { + return Status::OK(); + } + RETURN_IF_ERROR(flush_to_temp_file()); + return Status::OK(); + } + + Status flush_to_temp_file() { + RETURN_IF_ERROR(ExecEnv::GetInstance()->spill_stream_mgr()->register_spill_stream( + _state, _spilling_stream, print_id(_state->query_id()), "sort", 1 /* node_id */, + SPILL_BLOCK_BATCH_ROW_COUNT, SORT_BLOCK_SPILL_BATCH_BYTES, _profile)); + + _sorted_streams.emplace_back(_spilling_stream); + + _spilling_stream->set_write_counters(_spill_serialize_block_timer, _spill_block_count, + _spill_data_size, _spill_write_disk_timer, + _spill_write_wait_io_timer); + + RETURN_IF_ERROR(_spilling_stream->prepare_spill()); + + RETURN_IF_ERROR(_sorter->prepare_for_read()); + bool eos = false; + Block block; + while (!eos) { + RETURN_IF_ERROR(_sorter->merge_sort_read_for_spill(_state, &block, + _spill_block_batch_row_count, &eos)); + RETURN_IF_ERROR(_spilling_stream->spill_block(_state, block, eos)); + block.clear_column_data(); + } + _sorter->reset(); + return Status::OK(); + } + + Status close(const Status& status) override { Review Comment: warning: function 'close' has cognitive complexity of 63 (threshold 50) [readability-function-cognitive-complexity] ```cpp Status close(const Status& status) override { ^ ``` <details> <summary>Additional context</summary> **be/src/vec/sink/writer/iceberg/sorting_file_writer.h:153:** nesting level increased to 1 ```cpp Defer defer {[&]() { ^ ``` **be/src/vec/sink/writer/iceberg/sorting_file_writer.h:155:** +2, including nesting penalty of 1, nesting level increased to 2 ```cpp if (!st.ok()) { ^ ``` **be/src/vec/sink/writer/iceberg/sorting_file_writer.h:163:** +1, including nesting penalty of 0, nesting level increased to 1 ```cpp if (status.ok() && _sorter != nullptr) { ^ ``` **be/src/vec/sink/writer/iceberg/sorting_file_writer.h:164:** +2, including nesting penalty of 1, nesting level increased to 2 ```cpp if (!_sorter->state().unsorted_block_empty()) { ^ ``` **be/src/vec/sink/writer/iceberg/sorting_file_writer.h:165:** +3, including nesting penalty of 2, nesting level increased to 3 ```cpp if (_sorted_streams.empty()) { ^ ``` **be/src/vec/sink/writer/iceberg/sorting_file_writer.h:166:** +4, including nesting penalty of 3, nesting level increased to 4 ```cpp RETURN_IF_ERROR(_sorter->sort()); ^ ``` **be/src/common/status.h:626:** expanded from macro 'RETURN_IF_ERROR' ```cpp do { \ ^ ``` **be/src/vec/sink/writer/iceberg/sorting_file_writer.h:166:** +5, including nesting penalty of 4, nesting level increased to 5 ```cpp RETURN_IF_ERROR(_sorter->sort()); ^ ``` **be/src/common/status.h:628:** expanded from macro 'RETURN_IF_ERROR' ```cpp if (UNLIKELY(!_status_.ok())) { \ ^ ``` **be/src/vec/sink/writer/iceberg/sorting_file_writer.h:168:** +4, including nesting penalty of 3, nesting level increased to 4 ```cpp RETURN_IF_ERROR(_output_writer.write(_sorter->state().get_sorted_block()[0])); ^ ``` **be/src/common/status.h:626:** expanded from macro 'RETURN_IF_ERROR' ```cpp do { \ ^ ``` **be/src/vec/sink/writer/iceberg/sorting_file_writer.h:168:** +5, including nesting penalty of 4, nesting level increased to 5 ```cpp RETURN_IF_ERROR(_output_writer.write(_sorter->state().get_sorted_block()[0])); ^ ``` **be/src/common/status.h:628:** expanded from macro 'RETURN_IF_ERROR' ```cpp if (UNLIKELY(!_status_.ok())) { \ ^ ``` **be/src/vec/sink/writer/iceberg/sorting_file_writer.h:171:** +3, including nesting penalty of 2, nesting level increased to 3 ```cpp RETURN_IF_ERROR(flush_to_temp_file()); ^ ``` **be/src/common/status.h:626:** expanded from macro 'RETURN_IF_ERROR' ```cpp do { \ ^ ``` **be/src/vec/sink/writer/iceberg/sorting_file_writer.h:171:** +4, including nesting penalty of 3, nesting level increased to 4 ```cpp RETURN_IF_ERROR(flush_to_temp_file()); ^ ``` **be/src/common/status.h:628:** expanded from macro 'RETURN_IF_ERROR' ```cpp if (UNLIKELY(!_status_.ok())) { \ ^ ``` **be/src/vec/sink/writer/iceberg/sorting_file_writer.h:173:** +2, including nesting penalty of 1, nesting level increased to 2 ```cpp if (_merger != nullptr) { ^ ``` **be/src/vec/sink/writer/iceberg/sorting_file_writer.h:174:** +3, including nesting penalty of 2, nesting level increased to 3 ```cpp RETURN_IF_ERROR(_combine_files()); ^ ``` **be/src/common/status.h:626:** expanded from macro 'RETURN_IF_ERROR' ```cpp do { \ ^ ``` **be/src/vec/sink/writer/iceberg/sorting_file_writer.h:174:** +4, including nesting penalty of 3, nesting level increased to 4 ```cpp RETURN_IF_ERROR(_combine_files()); ^ ``` **be/src/common/status.h:628:** expanded from macro 'RETURN_IF_ERROR' ```cpp if (UNLIKELY(!_status_.ok())) { \ ^ ``` **be/src/vec/sink/writer/iceberg/sorting_file_writer.h:176:** +3, including nesting penalty of 2, nesting level increased to 3 ```cpp while (!eos) { ^ ``` **be/src/vec/sink/writer/iceberg/sorting_file_writer.h:178:** +4, including nesting penalty of 3, nesting level increased to 4 ```cpp RETURN_IF_ERROR(_merger->get_next(&output_block, &eos)); ^ ``` **be/src/common/status.h:626:** expanded from macro 'RETURN_IF_ERROR' ```cpp do { \ ^ ``` **be/src/vec/sink/writer/iceberg/sorting_file_writer.h:178:** +5, including nesting penalty of 4, nesting level increased to 5 ```cpp RETURN_IF_ERROR(_merger->get_next(&output_block, &eos)); ^ ``` **be/src/common/status.h:628:** expanded from macro 'RETURN_IF_ERROR' ```cpp if (UNLIKELY(!_status_.ok())) { \ ^ ``` **be/src/vec/sink/writer/iceberg/sorting_file_writer.h:179:** +4, including nesting penalty of 3, nesting level increased to 4 ```cpp RETURN_IF_ERROR(_output_writer.write(output_block)); ^ ``` **be/src/common/status.h:626:** expanded from macro 'RETURN_IF_ERROR' ```cpp do { \ ^ ``` **be/src/vec/sink/writer/iceberg/sorting_file_writer.h:179:** +5, including nesting penalty of 4, nesting level increased to 5 ```cpp RETURN_IF_ERROR(_output_writer.write(output_block)); ^ ``` **be/src/common/status.h:628:** expanded from macro 'RETURN_IF_ERROR' ```cpp if (UNLIKELY(!_status_.ok())) { \ ^ ``` </details> ########## be/src/vec/sink/writer/iceberg/partition_writer.h: ########## @@ -0,0 +1,64 @@ +// 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 + +namespace doris { +namespace io { +class FileSystem; +} + +class ObjectPool; +class RuntimeState; +class RuntimeProfile; + +namespace iceberg { +class Schema; +} + +namespace vectorized { + +class Block; +class VFileFormatTransformer; + +class IPartitionWriter { +public: + struct WriteInfo { + std::string write_path; + std::string original_write_path; + std::string target_path; + TFileType::type file_type; + }; + + IPartitionWriter() = default; + virtual ~IPartitionWriter() = default; + + virtual Status open(RuntimeState* state, RuntimeProfile* profile, const RowDescriptor* row_desc, Review Comment: warning: unknown type name 'RowDescriptor' [clang-diagnostic-error] ```cpp virtual Status open(RuntimeState* state, RuntimeProfile* profile, const RowDescriptor* row_desc, ^ ``` ########## be/src/vec/sink/writer/iceberg/partition_writer.h: ########## @@ -0,0 +1,64 @@ +// 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 + +namespace doris { +namespace io { +class FileSystem; +} + +class ObjectPool; +class RuntimeState; +class RuntimeProfile; + +namespace iceberg { +class Schema; +} + +namespace vectorized { + +class Block; +class VFileFormatTransformer; + +class IPartitionWriter { +public: + struct WriteInfo { + std::string write_path; + std::string original_write_path; + std::string target_path; + TFileType::type file_type; + }; + + IPartitionWriter() = default; + virtual ~IPartitionWriter() = default; + + virtual Status open(RuntimeState* state, RuntimeProfile* profile, const RowDescriptor* row_desc, Review Comment: warning: unknown type name 'Status' [clang-diagnostic-error] ```cpp virtual Status open(RuntimeState* state, RuntimeProfile* profile, const RowDescriptor* row_desc, ^ ``` ########## be/src/vec/sink/writer/iceberg/partition_writer.h: ########## @@ -0,0 +1,64 @@ +// 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 + +namespace doris { +namespace io { +class FileSystem; +} + +class ObjectPool; +class RuntimeState; +class RuntimeProfile; + +namespace iceberg { +class Schema; +} + +namespace vectorized { + +class Block; +class VFileFormatTransformer; + +class IPartitionWriter { +public: + struct WriteInfo { + std::string write_path; + std::string original_write_path; Review Comment: warning: use of undeclared identifier 'std' [clang-diagnostic-error] ```cpp std::string original_write_path; ^ ``` ########## be/src/vec/sink/writer/iceberg/partition_writer.h: ########## @@ -0,0 +1,64 @@ +// 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 + +namespace doris { +namespace io { +class FileSystem; +} + +class ObjectPool; +class RuntimeState; +class RuntimeProfile; + +namespace iceberg { +class Schema; +} + +namespace vectorized { + +class Block; +class VFileFormatTransformer; + +class IPartitionWriter { +public: + struct WriteInfo { + std::string write_path; + std::string original_write_path; + std::string target_path; + TFileType::type file_type; + }; + + IPartitionWriter() = default; + virtual ~IPartitionWriter() = default; + + virtual Status open(RuntimeState* state, RuntimeProfile* profile, const RowDescriptor* row_desc, + ObjectPool* pool) = 0; + + virtual Status write(vectorized::Block& block) = 0; + + virtual Status close(const Status& status) = 0; Review Comment: warning: unknown type name 'Status' [clang-diagnostic-error] ```cpp virtual Status close(const Status& status) = 0; ^ ``` ########## be/src/vec/sink/writer/iceberg/partition_writer.h: ########## @@ -0,0 +1,64 @@ +// 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 + +namespace doris { +namespace io { +class FileSystem; +} + +class ObjectPool; +class RuntimeState; +class RuntimeProfile; + +namespace iceberg { +class Schema; +} + +namespace vectorized { + +class Block; +class VFileFormatTransformer; + +class IPartitionWriter { +public: + struct WriteInfo { + std::string write_path; + std::string original_write_path; + std::string target_path; + TFileType::type file_type; + }; + + IPartitionWriter() = default; + virtual ~IPartitionWriter() = default; + + virtual Status open(RuntimeState* state, RuntimeProfile* profile, const RowDescriptor* row_desc, + ObjectPool* pool) = 0; + + virtual Status write(vectorized::Block& block) = 0; + + virtual Status close(const Status& status) = 0; Review Comment: warning: unknown type name 'Status' [clang-diagnostic-error] ```cpp virtual Status close(const Status& status) = 0; ^ ``` -- 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