From a201957a74d7b6b19acefa79064bc23ccd84506d Mon Sep 17 00:00:00 2001 From: "caojunhui.sticey" Date: Thu, 16 Apr 2026 15:18:32 +0800 Subject: [PATCH 1/5] support binlog meta(fe/be) --- be/src/cloud/pb_convert.cpp | 66 +++ be/src/load/channel/tablets_channel.cpp | 2 + be/src/load/delta_writer/delta_writer.cpp | 6 +- be/src/load/delta_writer/delta_writer.h | 3 - .../load/delta_writer/delta_writer_context.h | 12 + be/src/storage/binlog.h | 3 + be/src/storage/binlog_config.cpp | 28 +- be/src/storage/binlog_config.h | 25 +- be/src/storage/data_dir.cpp | 28 +- be/src/storage/rowset/beta_rowset_writer.cpp | 9 + be/src/storage/rowset/beta_rowset_writer.h | 5 + be/src/storage/rowset/group_rowset_writer.cpp | 53 ++ be/src/storage/rowset/group_rowset_writer.h | 128 +++++ .../storage/rowset/pending_rowset_helper.cpp | 43 +- be/src/storage/rowset/pending_rowset_helper.h | 7 +- be/src/storage/rowset/rowset_factory.cpp | 18 + be/src/storage/rowset/rowset_factory.h | 5 + be/src/storage/rowset/rowset_fwd.h | 4 + be/src/storage/rowset/rowset_meta.h | 6 + be/src/storage/rowset/rowset_writer_context.h | 32 ++ .../rowset/vertical_beta_rowset_writer.h | 27 +- be/src/storage/rowset_builder.cpp | 191 ++++++- be/src/storage/rowset_builder.h | 87 ++- be/src/storage/snapshot/snapshot_manager.cpp | 3 +- be/src/storage/tablet/base_tablet.cpp | 9 +- be/src/storage/tablet/base_tablet.h | 4 + be/src/storage/tablet/tablet.cpp | 108 +++- be/src/storage/tablet/tablet.h | 34 +- be/src/storage/tablet/tablet_manager.cpp | 4 + be/src/storage/tablet/tablet_meta.cpp | 540 +++++++++++------- be/src/storage/tablet/tablet_meta.h | 55 +- be/src/storage/txn/txn_manager.cpp | 2 +- .../olap/rowset/group_rowset_builder_test.cpp | 219 +++++++ .../java/org/apache/doris/catalog/Column.java | 39 ++ .../java/org/apache/doris/alter/Alter.java | 5 + .../apache/doris/alter/AlterOperations.java | 9 + .../org/apache/doris/alter/RollupJobV2.java | 2 +- .../doris/alter/SchemaChangeHandler.java | 236 ++++++-- .../apache/doris/alter/SchemaChangeJobV2.java | 9 +- .../apache/doris/backup/BackupHandler.java | 4 + .../org/apache/doris/backup/RestoreJob.java | 13 +- .../doris/binlog/BinlogConfigCache.java | 6 +- .../apache/doris/binlog/BinlogManager.java | 2 +- .../org/apache/doris/binlog/BinlogUtils.java | 6 + .../org/apache/doris/binlog/DBBinlog.java | 2 +- .../org/apache/doris/binlog/TableBinlog.java | 2 +- .../apache/doris/catalog/BinlogConfig.java | 167 ++++-- .../doris/catalog/CloudTabletStatMgr.java | 2 +- .../org/apache/doris/catalog/Database.java | 4 +- .../java/org/apache/doris/catalog/Env.java | 8 +- .../catalog/LocalTabletInvertedIndex.java | 5 + .../doris/catalog/MaterializedIndex.java | 8 + .../doris/catalog/MaterializedIndexMeta.java | 19 + .../org/apache/doris/catalog/OlapTable.java | 183 +++++- .../doris/catalog/OlapTableWrapper.java | 140 +++++ .../org/apache/doris/catalog/Partition.java | 8 + .../org/apache/doris/catalog/Replica.java | 22 + .../doris/catalog/RowBinlogTableWrapper.java | 40 ++ .../apache/doris/catalog/TableProperty.java | 15 +- .../java/org/apache/doris/catalog/Tablet.java | 10 + .../apache/doris/catalog/TabletStatMgr.java | 20 +- .../doris/common/proc/ReplicasProcNode.java | 5 +- .../doris/common/proc/TabletsProcDir.java | 9 +- .../doris/common/util/BufferSizeUtil.java | 4 + .../doris/common/util/PropertyAnalyzer.java | 30 +- .../doris/datasource/InternalCatalog.java | 33 +- .../apache/doris/master/ReportHandler.java | 7 +- .../trees/plans/commands/ShowDataCommand.java | 98 +++- .../plans/commands/info/AddColumnOp.java | 5 + .../plans/commands/info/AddColumnsOp.java | 6 + .../commands/info/AddPartitionLikeOp.java | 6 + .../plans/commands/info/AddPartitionOp.java | 6 + .../plans/commands/info/AddRollupOp.java | 6 + .../trees/plans/commands/info/AlterOp.java | 7 + .../plans/commands/info/DropColumnOp.java | 7 + .../plans/commands/info/DropPartitionOp.java | 6 + .../plans/commands/info/DropRollupOp.java | 6 + .../commands/info/ModifyColumnCommentOp.java | 6 + .../commands/info/ModifyDistributionOp.java | 6 + .../commands/info/ModifyPartitionOp.java | 6 + .../commands/info/ModifyTableCommentOp.java | 6 + .../info/ModifyTablePropertiesOp.java | 25 +- .../commands/info/RenamePartitionOp.java | 6 + .../plans/commands/info/RenameRollupOp.java | 6 + .../plans/commands/info/RenameTableOp.java | 6 + .../commands/info/ReplacePartitionOp.java | 6 + .../plans/commands/info/ReplaceTableOp.java | 5 + .../apache/doris/task/CreateReplicaTask.java | 33 +- .../doris/alter/SchemaChangeHandlerTest.java | 234 ++++++++ .../doris/binlog/BinlogManagerTest.java | 39 +- .../apache/doris/binlog/BinlogTestUtils.java | 14 +- .../doris/binlog/MockBinlogConfigCache.java | 6 +- .../apache/doris/catalog/CreateTableTest.java | 42 ++ .../catalog/OlapTableRowBinlogSchemaTest.java | 92 +++ .../doris/catalog/OlapTableWrapperTest.java | 120 ++++ .../doris/common/proc/ProcServiceTest.java | 13 + .../plans/commands/ShowDataCommandTest.java | 96 +++- .../commands/TruncateTableCommandTest.java | 17 + .../org/apache/doris/task/AgentTaskTest.java | 29 +- gensrc/proto/olap_file.proto | 16 + gensrc/thrift/AgentService.thrift | 8 + gensrc/thrift/BackendService.thrift | 2 + gensrc/thrift/MasterService.thrift | 4 +- ...test_group_commit_interval_ms_property.out | 9 +- .../test_nereids_alter_database_set_quota.out | 37 +- .../query_p0/show/test_show_create_table.out | 5 +- .../query_p0/system/test_table_properties.out | 15 +- ...st_show_create_table_and_views_nereids.out | 9 +- .../suites/show_p0/test_show_data.groovy | 4 +- 109 files changed, 3421 insertions(+), 494 deletions(-) create mode 100644 be/src/storage/rowset/group_rowset_writer.cpp create mode 100644 be/src/storage/rowset/group_rowset_writer.h create mode 100644 be/test/olap/rowset/group_rowset_builder_test.cpp create mode 100644 fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTableWrapper.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/catalog/RowBinlogTableWrapper.java create mode 100755 fe/fe-core/src/test/java/org/apache/doris/catalog/OlapTableRowBinlogSchemaTest.java create mode 100644 fe/fe-core/src/test/java/org/apache/doris/catalog/OlapTableWrapperTest.java diff --git a/be/src/cloud/pb_convert.cpp b/be/src/cloud/pb_convert.cpp index 8e4eb50f048204..724dde723f6c82 100644 --- a/be/src/cloud/pb_convert.cpp +++ b/be/src/cloud/pb_convert.cpp @@ -120,6 +120,9 @@ void doris_rowset_meta_to_cloud(RowsetMetaCloudPB* out, const RowsetMetaPB& in) if (in.has_commit_tso()) { out->set_commit_tso(in.commit_tso()); } + if (in.has_is_row_binlog()) { + out->set_is_row_binlog(in.is_row_binlog()); + } } void doris_rowset_meta_to_cloud(RowsetMetaCloudPB* out, RowsetMetaPB&& in) { @@ -207,6 +210,9 @@ void doris_rowset_meta_to_cloud(RowsetMetaCloudPB* out, RowsetMetaPB&& in) { if (in.has_commit_tso()) { out->set_commit_tso(in.commit_tso()); } + if (in.has_is_row_binlog()) { + out->set_is_row_binlog(in.is_row_binlog()); + } } RowsetMetaPB cloud_rowset_meta_to_doris(const RowsetMetaCloudPB& in) { @@ -304,6 +310,9 @@ void cloud_rowset_meta_to_doris(RowsetMetaPB* out, const RowsetMetaCloudPB& in) if (in.has_commit_tso()) { out->set_commit_tso(in.commit_tso()); } + if (in.has_is_row_binlog()) { + out->set_is_row_binlog(in.is_row_binlog()); + } } void cloud_rowset_meta_to_doris(RowsetMetaPB* out, RowsetMetaCloudPB&& in) { @@ -390,6 +399,9 @@ void cloud_rowset_meta_to_doris(RowsetMetaPB* out, RowsetMetaCloudPB&& in) { if (in.has_commit_tso()) { out->set_commit_tso(in.commit_tso()); } + if (in.has_is_row_binlog()) { + out->set_is_row_binlog(in.is_row_binlog()); + } } TabletSchemaCloudPB doris_tablet_schema_to_cloud(const TabletSchemaPB& in) { @@ -670,6 +682,18 @@ void doris_tablet_meta_to_cloud(TabletMetaCloudPB* out, const TabletMetaPB& in) if (in.has_binlog_config()) { out->mutable_binlog_config()->CopyFrom(in.binlog_config()); } + if (in.has_row_binlog_schema()) { + doris_tablet_schema_to_cloud(out->mutable_row_binlog_schema(), in.row_binlog_schema()); + } + if (in.row_binlog_rs_metas_size()) { + out->mutable_row_binlog_rs_metas()->Reserve(in.row_binlog_rs_metas_size()); + for (const auto& rs_meta : in.row_binlog_rs_metas()) { + doris_rowset_meta_to_cloud(out->add_row_binlog_rs_metas(), rs_meta); + } + } + if (in.has_row_binlog_schema_hash()) { + out->set_row_binlog_schema_hash(in.row_binlog_schema_hash()); + } out->set_compaction_policy(in.compaction_policy()); out->set_time_series_compaction_goal_size_mbytes(in.time_series_compaction_goal_size_mbytes()); out->set_time_series_compaction_file_count_threshold( @@ -749,6 +773,21 @@ void doris_tablet_meta_to_cloud(TabletMetaCloudPB* out, TabletMetaPB&& in) { if (in.has_binlog_config()) { out->mutable_binlog_config()->Swap(in.mutable_binlog_config()); } + if (in.has_row_binlog_schema()) { + doris_tablet_schema_to_cloud(out->mutable_row_binlog_schema(), + std::move(*in.mutable_row_binlog_schema())); + } + if (in.row_binlog_rs_metas_size()) { + int row_binlog_rs_metas_size = in.row_binlog_rs_metas_size(); + out->mutable_row_binlog_rs_metas()->Reserve(row_binlog_rs_metas_size); + for (int i = 0; i < row_binlog_rs_metas_size; ++i) { + doris_rowset_meta_to_cloud(out->add_row_binlog_rs_metas(), + std::move(*in.mutable_row_binlog_rs_metas(i))); + } + } + if (in.has_row_binlog_schema_hash()) { + out->set_row_binlog_schema_hash(in.row_binlog_schema_hash()); + } out->set_compaction_policy(in.compaction_policy()); out->set_time_series_compaction_goal_size_mbytes(in.time_series_compaction_goal_size_mbytes()); out->set_time_series_compaction_file_count_threshold( @@ -835,6 +874,18 @@ void cloud_tablet_meta_to_doris(TabletMetaPB* out, const TabletMetaCloudPB& in) if (in.has_binlog_config()) { out->mutable_binlog_config()->CopyFrom(in.binlog_config()); } + if (in.has_row_binlog_schema()) { + cloud_tablet_schema_to_doris(out->mutable_row_binlog_schema(), in.row_binlog_schema()); + } + if (in.row_binlog_rs_metas_size()) { + out->mutable_row_binlog_rs_metas()->Reserve(in.row_binlog_rs_metas_size()); + for (const auto& rs_meta : in.row_binlog_rs_metas()) { + cloud_rowset_meta_to_doris(out->add_row_binlog_rs_metas(), rs_meta); + } + } + if (in.has_row_binlog_schema_hash()) { + out->set_row_binlog_schema_hash(in.row_binlog_schema_hash()); + } out->set_compaction_policy(in.compaction_policy()); out->set_time_series_compaction_goal_size_mbytes(in.time_series_compaction_goal_size_mbytes()); out->set_time_series_compaction_file_count_threshold( @@ -914,6 +965,21 @@ void cloud_tablet_meta_to_doris(TabletMetaPB* out, TabletMetaCloudPB&& in) { if (in.has_binlog_config()) { out->mutable_binlog_config()->Swap(in.mutable_binlog_config()); } + if (in.has_row_binlog_schema()) { + cloud_tablet_schema_to_doris(out->mutable_row_binlog_schema(), + std::move(*in.mutable_row_binlog_schema())); + } + if (in.row_binlog_rs_metas_size()) { + int row_binlog_rs_metas_size = in.row_binlog_rs_metas_size(); + out->mutable_row_binlog_rs_metas()->Reserve(row_binlog_rs_metas_size); + for (int i = 0; i < row_binlog_rs_metas_size; ++i) { + cloud_rowset_meta_to_doris(out->add_row_binlog_rs_metas(), + std::move(*in.mutable_row_binlog_rs_metas(i))); + } + } + if (in.has_row_binlog_schema_hash()) { + out->set_row_binlog_schema_hash(in.row_binlog_schema_hash()); + } out->set_compaction_policy(in.compaction_policy()); out->set_time_series_compaction_goal_size_mbytes(in.time_series_compaction_goal_size_mbytes()); out->set_time_series_compaction_file_count_threshold( diff --git a/be/src/load/channel/tablets_channel.cpp b/be/src/load/channel/tablets_channel.cpp index 12d535e9e087f3..9b730e8874eaa1 100644 --- a/be/src/load/channel/tablets_channel.cpp +++ b/be/src/load/channel/tablets_channel.cpp @@ -29,6 +29,7 @@ // IWYU pragma: no_include #include // IWYU pragma: keep #include +#include #include #include #include @@ -203,6 +204,7 @@ Status BaseTabletsChannel::incremental_open(const PTabletWriterOpenRequest& para std::vector* index_slots = nullptr; int32_t schema_hash = 0; + for (const auto& index : _schema->indexes()) { if (index->index_id == _index_id) { index_slots = &index->slots; diff --git a/be/src/load/delta_writer/delta_writer.cpp b/be/src/load/delta_writer/delta_writer.cpp index 33d3c3065ffb01..2fd053f765ebe9 100644 --- a/be/src/load/delta_writer/delta_writer.cpp +++ b/be/src/load/delta_writer/delta_writer.cpp @@ -203,14 +203,10 @@ Status BaseDeltaWriter::wait_calc_delete_bitmap() { return _rowset_builder->wait_calc_delete_bitmap(); } -RowsetBuilder* DeltaWriter::rowset_builder() { - return static_cast(_rowset_builder.get()); -} - Status DeltaWriter::commit_txn(const PSlaveTabletNodes& slave_tablet_nodes) { std::lock_guard l(_lock); SCOPED_TIMER(_commit_txn_timer); - RETURN_IF_ERROR(rowset_builder()->commit_txn()); + RETURN_IF_ERROR(_rowset_builder->commit_txn()); for (auto&& node_info : slave_tablet_nodes.slave_nodes()) { _request_slave_tablet_pull_rowset(node_info); diff --git a/be/src/load/delta_writer/delta_writer.h b/be/src/load/delta_writer/delta_writer.h index 6907e72b57ad2f..ab9715b74b4d52 100644 --- a/be/src/load/delta_writer/delta_writer.h +++ b/be/src/load/delta_writer/delta_writer.h @@ -150,9 +150,6 @@ class DeltaWriter final : public BaseDeltaWriter { void _request_slave_tablet_pull_rowset(const PNodeInfo& node_info); - // Convert `_rowset_builder` from `BaseRowsetBuilder` to `RowsetBuilder` - RowsetBuilder* rowset_builder(); - std::mutex _lock; StorageEngine& _engine; diff --git a/be/src/load/delta_writer/delta_writer_context.h b/be/src/load/delta_writer/delta_writer_context.h index ea5c3ae886d56d..21d436c86358a1 100644 --- a/be/src/load/delta_writer/delta_writer_context.h +++ b/be/src/load/delta_writer/delta_writer_context.h @@ -29,6 +29,12 @@ class TupleDescriptor; class SlotDescriptor; class OlapTableSchemaParam; +enum class WriteRequestType { + DATA = 0, + ROW_BINLOG = 1, + GROUP = 2, +}; + struct WriteRequest { int64_t tablet_id = 0; int32_t schema_hash = 0; @@ -43,7 +49,13 @@ struct WriteRequest { std::shared_ptr table_schema_param = nullptr; bool is_high_priority = false; bool write_file_cache = false; + WriteRequestType write_req_type = WriteRequestType::DATA; std::string storage_vault_id; }; +struct GroupWriteRequest : public WriteRequest { + WriteRequest data_req; + WriteRequest row_binlog_req; +}; + } // namespace doris diff --git a/be/src/storage/binlog.h b/be/src/storage/binlog.h index 3076d4abda8e54..35fef1db444bb3 100644 --- a/be/src/storage/binlog.h +++ b/be/src/storage/binlog.h @@ -28,6 +28,9 @@ namespace doris { constexpr std::string_view kBinlogPrefix = "binlog_"; constexpr std::string_view kBinlogMetaPrefix = "binlog_meta_"; constexpr std::string_view kBinlogDataPrefix = "binlog_data_"; +constexpr std::string_view kRowBinlogPrefix = "binlog_row_"; +// used in file directory +constexpr std::string_view FDRowBinlogSuffix = "_row_binlog"; inline auto make_binlog_meta_key(const std::string_view tablet, int64_t version, const std::string_view rowset) { diff --git a/be/src/storage/binlog_config.cpp b/be/src/storage/binlog_config.cpp index c03ae743fbb43b..595475383a21a2 100644 --- a/be/src/storage/binlog_config.cpp +++ b/be/src/storage/binlog_config.cpp @@ -21,6 +21,8 @@ #include #include +#include "common/logging.h" + namespace doris { BinlogConfig& BinlogConfig::operator=(const TBinlogConfig& config) { if (config.__isset.enable) { @@ -35,6 +37,18 @@ BinlogConfig& BinlogConfig::operator=(const TBinlogConfig& config) { if (config.__isset.max_history_nums) { _max_history_nums = config.max_history_nums; } + if (config.__isset.binlog_format) { + if (config.binlog_format == TBinlogFormat::ROW) { + _binlog_format = BinlogFormatPB::ROW; + } else if (config.binlog_format == TBinlogFormat::STATEMENT_AND_SNAPSHOT) { + _binlog_format = BinlogFormatPB::STATEMENT_AND_SNAPSHOT; + } else { + DCHECK(false) << "can not identify the binlog format " << config.binlog_format; + } + } + if (config.__isset.need_historical_value) { + _need_historical_value = config.need_historical_value; + } return *this; } @@ -51,6 +65,12 @@ BinlogConfig& BinlogConfig::operator=(const BinlogConfigPB& config) { if (config.has_max_history_nums()) { _max_history_nums = config.max_history_nums(); } + if (config.has_binlog_format()) { + _binlog_format = config.binlog_format(); + } + if (config.has_need_historical_value()) { + _need_historical_value = config.need_historical_value(); + } return *this; } @@ -59,12 +79,16 @@ void BinlogConfig::to_pb(BinlogConfigPB* config_pb) const { config_pb->set_ttl_seconds(_ttl_seconds); config_pb->set_max_bytes(_max_bytes); config_pb->set_max_history_nums(_max_history_nums); + config_pb->set_binlog_format(_binlog_format); + config_pb->set_need_historical_value(_need_historical_value); } std::string BinlogConfig::to_string() const { return fmt::format( - "BinlogConfig enable: {}, ttl_seconds: {}, max_bytes: {}, max_history_nums: {}", - _enable, _ttl_seconds, _max_bytes, _max_history_nums); + "BinlogConfig enable: {}, ttl_seconds: {}, max_bytes: {}, max_history_nums: {}, " + "binlog_format: {}, need_historical_value: {}", + _enable, _ttl_seconds, _max_bytes, _max_history_nums, _binlog_format, + _need_historical_value); } } // namespace doris diff --git a/be/src/storage/binlog_config.h b/be/src/storage/binlog_config.h index e20a8adb116cfe..e0b4651db345e8 100644 --- a/be/src/storage/binlog_config.h +++ b/be/src/storage/binlog_config.h @@ -17,6 +17,9 @@ #pragma once +#include +#include + #include #include #include @@ -29,11 +32,14 @@ class BinlogConfigPB; class BinlogConfig { public: BinlogConfig() = default; - BinlogConfig(bool enable, int64_t ttl_seconds, int64_t max_bytes, int64_t max_history_nums) + BinlogConfig(bool enable, int64_t ttl_seconds, int64_t max_bytes, int64_t max_history_nums, + BinlogFormatPB binlog_format, bool need_historical_value) : _enable(enable), _ttl_seconds(ttl_seconds), _max_bytes(max_bytes), - _max_history_nums(max_history_nums) {} + _max_history_nums(max_history_nums), + _binlog_format(binlog_format), + _need_historical_value(need_historical_value) {} BinlogConfig(const BinlogConfig&) = default; BinlogConfig& operator=(const BinlogConfig&) = default; BinlogConfig(BinlogConfig&&) = default; @@ -52,6 +58,19 @@ class BinlogConfig { int64_t max_history_nums() const { return _max_history_nums; } void set_max_history_nums(int64_t max_history_nums) { _max_history_nums = max_history_nums; } + int32_t binlog_format() const { return _binlog_format; } + void set_binlog_format(BinlogFormatPB binlog_format) { _binlog_format = binlog_format; } + + bool need_historical_value() const { return _need_historical_value; } + void set_need_historical_value(bool need_historical_value) { + _need_historical_value = need_historical_value; + } + + bool isCCRBinlogFormat() const { + return _binlog_format == BinlogFormatPB::STATEMENT_AND_SNAPSHOT; + } + bool isRowBinlogFormat() const { return _binlog_format == BinlogFormatPB::ROW; } + BinlogConfig& operator=(const TBinlogConfig& config); BinlogConfig& operator=(const BinlogConfigPB& config); @@ -63,6 +82,8 @@ class BinlogConfig { int64_t _ttl_seconds {std::numeric_limits::max()}; int64_t _max_bytes {std::numeric_limits::max()}; int64_t _max_history_nums {std::numeric_limits::max()}; + BinlogFormatPB _binlog_format = BinlogFormatPB::STATEMENT_AND_SNAPSHOT; + bool _need_historical_value {false}; }; } // namespace doris diff --git a/be/src/storage/data_dir.cpp b/be/src/storage/data_dir.cpp index c61f4c42d7208c..db53a2d741292c 100644 --- a/be/src/storage/data_dir.cpp +++ b/be/src/storage/data_dir.cpp @@ -615,9 +615,13 @@ Status DataDir::load() { if (!tablet) { return true; } - const auto& all_rowsets = tablet->tablet_meta()->all_rs_metas(); + const auto& all_data_rowsets = tablet->tablet_meta()->all_rs_metas(); + const auto& all_row_binlogs = tablet->tablet_meta()->all_row_binlog_rs_metas(); RowsetIdUnorderedSet rowset_ids; - for (const auto& [_, rowset_meta] : all_rowsets) { + for (const auto& [_, rowset_meta] : all_data_rowsets) { + rowset_ids.insert(rowset_meta->rowset_id()); + } + for (auto& [_, rowset_meta] : all_row_binlogs) { rowset_ids.insert(rowset_meta->rowset_id()); } @@ -626,12 +630,14 @@ Status DataDir::load() { int rst_ids_size = delete_bitmap_pb.rowset_ids_size(); int seg_ids_size = delete_bitmap_pb.segment_ids_size(); int seg_maps_size = delete_bitmap_pb.segment_delete_bitmaps_size(); + int binlog_mark_size = delete_bitmap_pb.is_binlog_delvec_size(); CHECK(rst_ids_size == seg_ids_size && seg_ids_size == seg_maps_size); + CHECK(binlog_mark_size == 0 || binlog_mark_size == rst_ids_size); for (int i = 0; i < rst_ids_size; ++i) { RowsetId rst_id; rst_id.init(delete_bitmap_pb.rowset_ids(i)); - // only process the rowset in _rs_metas + // only process the rowset in _rs_metas and _row_binlog_rs_metas if (rowset_ids.find(rst_id) == rowset_ids.end()) { ++unknown_dbm_cnt; continue; @@ -645,8 +651,20 @@ Status DataDir::load() { continue; } auto bitmap = delete_bitmap_pb.segment_delete_bitmaps(i).data(); - tablet->tablet_meta()->delete_bitmap().delete_bitmap[{rst_id, seg_id, version}] = - roaring::Roaring::read(bitmap); + + bool from_binlog = delete_bitmap_pb.is_binlog_delvec_size() > 0 + ? delete_bitmap_pb.is_binlog_delvec(i) + : false; + if (!from_binlog) { + tablet->tablet_meta()->delete_bitmap().delete_bitmap[{rst_id, seg_id, version}] = + roaring::Roaring::read(bitmap); + } else { + tablet->tablet_meta()->binlog_delvec().delete_bitmap[{rst_id, seg_id, version}] = + roaring::Roaring::read(bitmap); + } + VLOG_ROW << "successfully to add delete_bitmap, tablet_id=" << tablet->tablet_id() + << ", rowset_id=" << rst_id << ", seg_id=" << seg_id << ", version=" << version + << ", from_binlog=" << from_binlog; } return true; }; diff --git a/be/src/storage/rowset/beta_rowset_writer.cpp b/be/src/storage/rowset/beta_rowset_writer.cpp index 9648d6fc4216ac..8088a6ee04e8d6 100644 --- a/be/src/storage/rowset/beta_rowset_writer.cpp +++ b/be/src/storage/rowset/beta_rowset_writer.cpp @@ -103,6 +103,9 @@ void build_rowset_meta_with_spec_field(RowsetMeta& rowset_meta, std::vector num_segment_rows; spec_rowset_meta.get_num_segment_rows(&num_segment_rows); rowset_meta.set_num_segment_rows(num_segment_rows); + if (spec_rowset_meta.is_row_binlog()) { + rowset_meta.mark_row_binlog(); + } } } // namespace @@ -272,6 +275,8 @@ BaseBetaRowsetWriter::BaseBetaRowsetWriter() BetaRowsetWriter::BetaRowsetWriter(StorageEngine& engine) : _engine(engine), _segcompaction_worker(std::make_shared(this)) {} +RowBinlogRowsetWriter::RowBinlogRowsetWriter(StorageEngine& engine) : BetaRowsetWriter(engine) {} + BaseBetaRowsetWriter::~BaseBetaRowsetWriter() { if (!_already_built && _rowset_meta->is_local()) { // abnormal exit, remove all files generated @@ -308,6 +313,7 @@ Status BaseBetaRowsetWriter::init(const RowsetWriterContext& rowset_writer_conte _rowset_meta->set_rowset_id(_context.rowset_id); _rowset_meta->set_partition_id(_context.partition_id); _rowset_meta->set_tablet_id(_context.tablet_id); + _rowset_meta->set_index_id(_context.index_id); _rowset_meta->set_tablet_schema_hash(_context.tablet_schema_hash); _rowset_meta->set_rowset_type(_context.rowset_type); _rowset_meta->set_rowset_state(_context.rowset_state); @@ -322,6 +328,9 @@ Status BaseBetaRowsetWriter::init(const RowsetWriterContext& rowset_writer_conte } _rowset_meta->set_tablet_uid(_context.tablet_uid); _rowset_meta->set_tablet_schema(_context.tablet_schema); + if (_context.write_binlog_opt().is_binlog_writer()) { + _rowset_meta->mark_row_binlog(); + } _context.segment_collector = std::make_shared>(this); _context.file_writer_creator = std::make_shared>(this); return Status::OK(); diff --git a/be/src/storage/rowset/beta_rowset_writer.h b/be/src/storage/rowset/beta_rowset_writer.h index 8011864cb58a07..4da74560bcad52 100644 --- a/be/src/storage/rowset/beta_rowset_writer.h +++ b/be/src/storage/rowset/beta_rowset_writer.h @@ -319,4 +319,9 @@ class BetaRowsetWriter : public BaseBetaRowsetWriter { std::atomic _segcompaction_status {ErrorCode::OK}; }; +class RowBinlogRowsetWriter : public BetaRowsetWriter { +public: + RowBinlogRowsetWriter(StorageEngine& engine); +}; + } // namespace doris diff --git a/be/src/storage/rowset/group_rowset_writer.cpp b/be/src/storage/rowset/group_rowset_writer.cpp new file mode 100644 index 00000000000000..4d6b7f2e8c836f --- /dev/null +++ b/be/src/storage/rowset/group_rowset_writer.cpp @@ -0,0 +1,53 @@ +// 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 "storage/rowset/group_rowset_writer.h" + +#include "storage/rowset/beta_rowset_writer.h" + +namespace doris { + +void GroupRowsetWriter::set_data_writer(const RowsetWriterSharedPtr& txn_rowset_writer) { + _txn_rowset_writer = std::dynamic_pointer_cast(txn_rowset_writer); +} + +void GroupRowsetWriter::set_row_binlog_writer( + const RowsetWriterSharedPtr& row_binlog_rowset_writer) { + _row_binlog_rowset_writer = row_binlog_rowset_writer; +} + +Status GroupRowsetWriter::flush_rowsets() { + RETURN_IF_ERROR(_txn_rowset_writer->flush()); + if (_row_binlog_rowset_writer) { + RETURN_IF_ERROR(_row_binlog_rowset_writer->flush()); + } + return Status::OK(); +} + +Status GroupRowsetWriter::build_rowsets(std::vector& rowsets) { + if (rowsets.size() < 2) { + return Status::InvalidArgument( + "GroupRowsetWriter::build_rowsets expects at least 2 rowset slots"); + } + RETURN_IF_ERROR(_txn_rowset_writer->build(rowsets[0])); + if (_row_binlog_rowset_writer) { + RETURN_IF_ERROR(_row_binlog_rowset_writer->build(rowsets[1])); + } + return Status::OK(); +} + +} // namespace doris diff --git a/be/src/storage/rowset/group_rowset_writer.h b/be/src/storage/rowset/group_rowset_writer.h new file mode 100644 index 00000000000000..a48f8aa29bc9f3 --- /dev/null +++ b/be/src/storage/rowset/group_rowset_writer.h @@ -0,0 +1,128 @@ +// 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 "storage/rowset/rowset_writer.h" + +namespace doris { +class GroupRowsetWriter : public RowsetWriter { +public: + GroupRowsetWriter() = default; + + void set_data_writer(const RowsetWriterSharedPtr& txn_rowset_writer); + + void set_row_binlog_writer(const RowsetWriterSharedPtr& row_binlog_rowset_writer); + + ~GroupRowsetWriter() = default; + + Status flush_rowsets(); + + Status build_rowsets(std::vector& rowsets); + + RowsetWriterSharedPtr row_binlog_writer() { return _row_binlog_rowset_writer; } + + RowsetWriterSharedPtr data_writer() { return _txn_rowset_writer; } + + Status init(const RowsetWriterContext& rowset_writer_context) override { + _context = rowset_writer_context; + return Status::OK(); + } + + Status add_block(const Block* block) override { + return Status::Error( + "GroupRowsetWriter::add_block is not implemented"); + } + + // add rowset by create hard link + Status add_rowset(RowsetSharedPtr rowset) override { + return Status::Error( + "GroupRowsetWriter::add_rowset is not implemented"); + } + + // Precondition: the input `rowset` should have the same type of the rowset we're building + Status add_rowset_for_linked_schema_change(RowsetSharedPtr rowset) override { + return Status::Error( + "GroupRowsetWriter::add_rowset_for_linked_schema_change is not implemented"); + } + + // explicit flush all buffered rows into segment file. + // note that `add_row` could also trigger flush when certain conditions are met + Status flush() override { return flush_rowsets(); } + + // GroupRowsetWriter does not support build a single rowset; its build is + // delegated to underlying writers. + Status build(RowsetSharedPtr& rowset) override { + return Status::NotSupported("GroupRowsetWriter::build is not supported"); + } + + RowsetSharedPtr manual_build(const RowsetMetaSharedPtr& rowset_meta) override { + LOG(FATAL) << "GroupRowsetWriter::manual_build not implemented"; + return nullptr; + } + + PUniqueId load_id() override { return _context.load_id; } + + Version version() override { return _context.version; } + + int64_t num_rows() const override { return _txn_rowset_writer->num_rows(); } + + int64_t num_rows_updated() const override { return _txn_rowset_writer->num_rows_updated(); } + + int64_t num_rows_deleted() const override { return _txn_rowset_writer->num_rows_deleted(); } + + int64_t num_rows_new_added() const override { return _txn_rowset_writer->num_rows_new_added(); } + + int64_t num_rows_filtered() const override { return _txn_rowset_writer->num_rows_filtered(); } + + RowsetId rowset_id() override { + LOG(FATAL) << "GroupRowsetWriter::rowset_id not implemented"; + RowsetId res; + return res; + } + + RowsetTypePB type() const override { return BETA_ROWSET; } + + Status get_segment_num_rows(std::vector* segment_num_rows) const override { + return Status::NotSupported("GroupRowsetWriter::get_segment_num_rows to be implemented"); + } + + int32_t allocate_segment_id() override { + LOG(FATAL) << "GroupRowsetWriter::allocate_segment_id is not supported"; + return -1; + } + + void set_segment_start_id(int num_segment) override { + LOG(FATAL) << "GroupRowsetWriter::set_segment_start_id not supported"; + } + + int64_t delete_bitmap_ns() override { return 0; } + + int64_t segment_writer_ns() override { return 0; } + + bool is_partial_update() override { return _txn_rowset_writer->is_partial_update(); } + + std::shared_ptr get_partial_update_info() override { + return _txn_rowset_writer->get_partial_update_info(); + } + +private: + RowsetWriterSharedPtr _txn_rowset_writer; + RowsetWriterSharedPtr _row_binlog_rowset_writer; +}; + +} // namespace doris diff --git a/be/src/storage/rowset/pending_rowset_helper.cpp b/be/src/storage/rowset/pending_rowset_helper.cpp index a68df6a1f0fe9c..553f6aeb83f991 100644 --- a/be/src/storage/rowset/pending_rowset_helper.cpp +++ b/be/src/storage/rowset/pending_rowset_helper.cpp @@ -23,29 +23,32 @@ namespace doris { PendingRowsetGuard::~PendingRowsetGuard() { if (_pending_rowset_set) { - _pending_rowset_set->remove(_rowset_id); + for (const auto& rowset_id : _rowset_ids) { + _pending_rowset_set->remove(rowset_id); + } } } -PendingRowsetGuard::PendingRowsetGuard(const RowsetId& rowset_id, PendingRowsetSet* set) - : _rowset_id(rowset_id), _pending_rowset_set(set) {} +PendingRowsetGuard::PendingRowsetGuard(const std::vector& rowset_ids, + PendingRowsetSet* set) + : _rowset_ids(rowset_ids), _pending_rowset_set(set) {} PendingRowsetGuard::PendingRowsetGuard(PendingRowsetGuard&& other) noexcept { CHECK(!_pending_rowset_set || - (_rowset_id == other._rowset_id && _pending_rowset_set == other._pending_rowset_set)) - << _rowset_id << ' ' << other._rowset_id << ' ' << _pending_rowset_set << ' ' - << other._pending_rowset_set; - _rowset_id = other._rowset_id; + (_rowset_ids == other._rowset_ids && _pending_rowset_set == other._pending_rowset_set)) + << _rowset_ids.size() << ' ' << other._rowset_ids.size() << ' ' << _pending_rowset_set + << ' ' << other._pending_rowset_set; + _rowset_ids = other._rowset_ids; _pending_rowset_set = other._pending_rowset_set; other._pending_rowset_set = nullptr; } PendingRowsetGuard& PendingRowsetGuard::operator=(PendingRowsetGuard&& other) noexcept { CHECK(!_pending_rowset_set || - (_rowset_id == other._rowset_id && _pending_rowset_set == other._pending_rowset_set)) - << _rowset_id << ' ' << other._rowset_id << ' ' << _pending_rowset_set << ' ' - << other._pending_rowset_set; - _rowset_id = other._rowset_id; + (_rowset_ids == other._rowset_ids && _pending_rowset_set == other._pending_rowset_set)) + << _rowset_ids.size() << ' ' << other._rowset_ids.size() << ' ' << _pending_rowset_set + << ' ' << other._pending_rowset_set; + _rowset_ids = other._rowset_ids; _pending_rowset_set = other._pending_rowset_set; other._pending_rowset_set = nullptr; return *this; @@ -53,10 +56,12 @@ PendingRowsetGuard& PendingRowsetGuard::operator=(PendingRowsetGuard&& other) no void PendingRowsetGuard::drop() { if (_pending_rowset_set) { - _pending_rowset_set->remove(_rowset_id); + for (const auto& rowset_id : _rowset_ids) { + _pending_rowset_set->remove(rowset_id); + } } _pending_rowset_set = nullptr; - _rowset_id = RowsetId {}; + _rowset_ids = std::vector {RowsetId {}}; } bool PendingRowsetSet::contains(const RowsetId& rowset_id) { @@ -69,7 +74,17 @@ PendingRowsetGuard PendingRowsetSet::add(const RowsetId& rowset_id) { std::lock_guard lock(_mtx); _set.insert(rowset_id); } - return PendingRowsetGuard {rowset_id, this}; + return PendingRowsetGuard {std::vector {rowset_id}, this}; +} + +PendingRowsetGuard PendingRowsetSet::add(const std::vector& rowset_ids) { + { + std::lock_guard lock(_mtx); + for (const auto& rowset_id : rowset_ids) { + _set.insert(rowset_id); + } + } + return PendingRowsetGuard {rowset_ids, this}; } void PendingRowsetSet::remove(const RowsetId& rowset_id) { diff --git a/be/src/storage/rowset/pending_rowset_helper.h b/be/src/storage/rowset/pending_rowset_helper.h index 013e0792f3c3c2..5e2b45aeed1ecb 100644 --- a/be/src/storage/rowset/pending_rowset_helper.h +++ b/be/src/storage/rowset/pending_rowset_helper.h @@ -45,9 +45,9 @@ class [[nodiscard]] PendingRowsetGuard { private: friend class PendingRowsetSet; - explicit PendingRowsetGuard(const RowsetId& rowset_id, PendingRowsetSet* set); + explicit PendingRowsetGuard(const std::vector& rowset_ids, PendingRowsetSet* set); - RowsetId _rowset_id; + std::vector _rowset_ids; PendingRowsetSet* _pending_rowset_set = nullptr; }; @@ -63,6 +63,9 @@ class PendingRowsetSet { // when `PendingRowsetGuard` is destroyed. PendingRowsetGuard add(const RowsetId& rowset_id); + // for GroupRowsetBuilder + PendingRowsetGuard add(const std::vector& rowset_ids); + private: friend class PendingRowsetGuard; void remove(const RowsetId& rowset_id); diff --git a/be/src/storage/rowset/rowset_factory.cpp b/be/src/storage/rowset/rowset_factory.cpp index 88e1977271517d..eabd48bba54ae2 100644 --- a/be/src/storage/rowset/rowset_factory.cpp +++ b/be/src/storage/rowset/rowset_factory.cpp @@ -27,6 +27,7 @@ #include "runtime/exec_env.h" #include "storage/rowset/beta_rowset.h" #include "storage/rowset/beta_rowset_writer.h" +#include "storage/rowset/group_rowset_writer.h" #include "storage/rowset/rowset_writer.h" #include "storage/rowset/rowset_writer_context.h" #include "storage/rowset/vertical_beta_rowset_writer.h" @@ -54,6 +55,18 @@ Result> RowsetFactory::create_rowset_writer( return ResultError(Status::Error("invalid rowset_type")); } + if (context.write_binlog_opt().is_binlog_writer()) { + std::unique_ptr writer; + if (is_vertical) { + writer = std::make_unique>(engine); + RETURN_IF_ERROR_RESULT(writer->init(context)); + return writer; + } else { + writer = std::make_unique(engine); + RETURN_IF_ERROR_RESULT(writer->init(context)); + return writer; + } + } if (context.rowset_type == BETA_ROWSET) { std::unique_ptr writer; if (is_vertical) { @@ -83,4 +96,9 @@ Result> RowsetFactory::create_rowset_writer( return writer; } +Status RowsetFactory::create_empty_group_rowset_writer(std::unique_ptr* output) { + output->reset(new GroupRowsetWriter()); + return Status::OK(); +} + } // namespace doris diff --git a/be/src/storage/rowset/rowset_factory.h b/be/src/storage/rowset/rowset_factory.h index 347002703f15d7..7a6018239f2888 100644 --- a/be/src/storage/rowset/rowset_factory.h +++ b/be/src/storage/rowset/rowset_factory.h @@ -27,6 +27,7 @@ namespace doris { class RowsetWriter; +class GroupRowsetWriter; struct RowsetWriterContext; class StorageEngine; class CloudStorageEngine; @@ -47,6 +48,10 @@ class RowsetFactory { // create and init cloud rowset writer. static Result> create_rowset_writer( CloudStorageEngine& engine, const RowsetWriterContext& context, bool is_vertical); + + // group rowset writer is used to flush and build synchronously + // and reuse multiple rowset writers internally. + static Status create_empty_group_rowset_writer(std::unique_ptr* output); }; } // namespace doris diff --git a/be/src/storage/rowset/rowset_fwd.h b/be/src/storage/rowset/rowset_fwd.h index 4e24225c2ab78b..2e741f4e1f5fae 100644 --- a/be/src/storage/rowset/rowset_fwd.h +++ b/be/src/storage/rowset/rowset_fwd.h @@ -27,5 +27,9 @@ class RowsetMeta; using RowsetMetaSharedPtr = std::shared_ptr; class RowsetReader; using RowsetReaderSharedPtr = std::shared_ptr; +class RowsetWriter; +using RowsetWriterSharedPtr = std::shared_ptr; +class RowsetBuilder; +using RowsetBuilderSharedPtr = std::shared_ptr; } // namespace doris diff --git a/be/src/storage/rowset/rowset_meta.h b/be/src/storage/rowset/rowset_meta.h index 78dbd7a2b7a35f..39b2ea2fead726 100644 --- a/be/src/storage/rowset/rowset_meta.h +++ b/be/src/storage/rowset/rowset_meta.h @@ -114,6 +114,12 @@ class RowsetMeta : public MetadataAdder { _rowset_meta_pb.set_tablet_schema_hash(tablet_schema_hash); } + void mark_row_binlog() { _rowset_meta_pb.set_is_row_binlog(true); } + + bool is_row_binlog() const { + return _rowset_meta_pb.has_is_row_binlog() && _rowset_meta_pb.is_row_binlog(); + } + RowsetTypePB rowset_type() const { return _rowset_meta_pb.rowset_type(); } void set_rowset_type(RowsetTypePB rowset_type) { _rowset_meta_pb.set_rowset_type(rowset_type); } diff --git a/be/src/storage/rowset/rowset_writer_context.h b/be/src/storage/rowset/rowset_writer_context.h index ba803a9a118839..b594e756a18f17 100644 --- a/be/src/storage/rowset/rowset_writer_context.h +++ b/be/src/storage/rowset/rowset_writer_context.h @@ -61,6 +61,7 @@ struct RowsetWriterContext { RowsetTypePB rowset_type {BETA_ROWSET}; TabletSchemaSPtr tablet_schema; + // PREPARED/COMMITTED for pending rowset // VISIBLE for non-pending rowset RowsetStatePB rowset_state {PREPARED}; @@ -248,6 +249,37 @@ struct RowsetWriterContext { .file_cache_expiration_time = file_cache_ttl_sec, .approximate_bytes_to_write = approximate_bytes_to_write}; } + + struct BinlogOptions { + public: + void mark_primary_writer() { binlog_write_type = BinlogWriteType::PrimaryWriter; } + + void mark_binlog_writer() { binlog_write_type = BinlogWriteType::BinlogWriter; } + + bool is_primary_writer() const { + return binlog_write_type == BinlogWriteType::PrimaryWriter; + } + + bool is_binlog_writer() const { return binlog_write_type == BinlogWriteType::BinlogWriter; } + + bool need_build_binlog() const { return binlog_write_type != BinlogWriteType::Unknown; } + + void set_need_before(bool need_before) { this->_need_before = need_before; } + + private: + // if you don't need to build row_binlog, `PrimaryWriter` and `BinlogWriter` are both false + // if you need to build row_binlog, the `is_primary_writer` of normal rowset writer is true + enum BinlogWriteType { + PrimaryWriter, + BinlogWriter, + Unknown + } binlog_write_type = BinlogWriteType::Unknown; + bool _need_before = false; + } _write_binlog_opt; + + BinlogOptions& write_binlog_opt() { return _write_binlog_opt; } + + const BinlogOptions& write_binlog_opt() const { return _write_binlog_opt; } }; } // namespace doris diff --git a/be/src/storage/rowset/vertical_beta_rowset_writer.h b/be/src/storage/rowset/vertical_beta_rowset_writer.h index 5552fc5598545d..a45952503509c7 100644 --- a/be/src/storage/rowset/vertical_beta_rowset_writer.h +++ b/be/src/storage/rowset/vertical_beta_rowset_writer.h @@ -31,7 +31,7 @@ class Block; // for vertical compaction template requires std::is_base_of_v -class VerticalBetaRowsetWriter final : public T { +class VerticalBetaRowsetWriter : public T { public: template explicit VerticalBetaRowsetWriter(Args&&... args) : T(std::forward(args)...) {} @@ -61,4 +61,27 @@ class VerticalBetaRowsetWriter final : public T { size_t _total_key_group_rows = 0; }; -} // namespace doris \ No newline at end of file +template + requires std::is_base_of_v +class VerticalRowBinlogRowsetWriter : public VerticalBetaRowsetWriter { +public: + explicit VerticalRowBinlogRowsetWriter(StorageEngine& engine) + : VerticalBetaRowsetWriter(engine) {} + + Status add_columns(const Block* block, const std::vector& col_ids, bool is_key, + uint32_t max_rows_per_segment, bool has_cluster_key) override { + return Status::NotSupported("VerticalRowBinlogRowsetWriter::add_columns not implemented"); + } + + // flush last segment's column + Status flush_columns(bool is_key) override { + return Status::NotSupported("VerticalRowBinlogRowsetWriter::flush_columns not implemented"); + } + + // flush when all column finished, flush column footer + Status final_flush() override { + return Status::NotSupported("VerticalRowBinlogRowsetWriter::final_flush not implemented"); + } +}; + +} // namespace doris diff --git a/be/src/storage/rowset_builder.cpp b/be/src/storage/rowset_builder.cpp index 91a888e4cbb4bb..4d258a0b8d75b3 100644 --- a/be/src/storage/rowset_builder.cpp +++ b/be/src/storage/rowset_builder.cpp @@ -39,7 +39,9 @@ #include "storage/partial_update_info.h" #include "storage/rowset/beta_rowset.h" #include "storage/rowset/beta_rowset_writer.h" +#include "storage/rowset/group_rowset_writer.h" #include "storage/rowset/pending_rowset_helper.h" +#include "storage/rowset/rowset_factory.h" #include "storage/rowset/rowset_meta.h" #include "storage/rowset/rowset_meta_manager.h" #include "storage/rowset/rowset_writer.h" @@ -73,9 +75,23 @@ RowsetBuilder::RowsetBuilder(StorageEngine& engine, const WriteRequest& req, RuntimeProfile* profile) : BaseRowsetBuilder(req, profile), _engine(engine) {} +RowBinlogRowsetBuilder::RowBinlogRowsetBuilder(StorageEngine& engine, const WriteRequest& req, + RuntimeProfile* profile) + : RowsetBuilder(engine, req, profile) {} + void BaseRowsetBuilder::_init_profile(RuntimeProfile* profile) { DCHECK(profile != nullptr); - _profile = profile->create_child(fmt::format("RowsetBuilder {}", _req.tablet_id), true, true); + if (_req.write_req_type == WriteRequestType::GROUP) { + _profile = profile->create_child(fmt::format("GroupRowsetBuilder {}", _req.tablet_id), true, + true); + return; + } + + _profile = profile->create_child( + fmt::format( + "RowsetBuilder {} {}", _req.tablet_id, + _req.write_req_type == WriteRequestType::ROW_BINLOG ? "row_binlog" : "data"), + true, true); _build_rowset_timer = ADD_TIMER(_profile, "BuildRowsetTime"); _submit_delete_bitmap_timer = ADD_TIMER(_profile, "DeleteBitmapSubmitTime"); _wait_delete_bitmap_timer = ADD_TIMER(_profile, "DeleteBitmapWaitTime"); @@ -99,7 +115,8 @@ BaseRowsetBuilder::~BaseRowsetBuilder() { RowsetBuilder::~RowsetBuilder() { if (_is_init && !_is_committed) { - _garbage_collection(); + // For txn rowset builders, we need to rollback txn when necessary. + _garbage_collection(is_data_builder()); } } @@ -111,21 +128,28 @@ TabletSharedPtr RowsetBuilder::tablet_sptr() { return std::static_pointer_cast(_tablet); } -void RowsetBuilder::_garbage_collection() { +void RowsetBuilder::_garbage_collection(bool cancel_txn) { Status rollback_status; - TxnManager* txn_mgr = _engine.txn_manager(); - if (tablet() != nullptr) { + bool need_clean = true; + if (tablet() != nullptr && cancel_txn) { + TxnManager* txn_mgr = _engine.txn_manager(); rollback_status = txn_mgr->rollback_txn(_req.partition_id, *tablet(), _req.txn_id); + need_clean = rollback_status.ok(); } // has to check rollback status, because the rowset maybe committed in this thread and // published in another thread, then rollback will fail. // when rollback failed should not delete rowset - if (rollback_status.ok()) { + if (need_clean) { _engine.add_unused_rowset(_rowset); + for (auto& rs : _attach_rowsets) { + _engine.add_unused_rowset(rs); + } } } Status BaseRowsetBuilder::init_mow_context(std::shared_ptr& mow_context) { + DCHECK(is_data_builder()); + std::lock_guard lck(tablet()->get_header_lock()); _max_version_in_flush_phase = tablet()->max_version_unlocked(); std::vector rowset_ptrs; @@ -150,6 +174,8 @@ Status BaseRowsetBuilder::init_mow_context(std::shared_ptr& mow_cont } Status RowsetBuilder::check_tablet_version_count() { + DCHECK(is_data_builder()); + auto max_version_config = _tablet->max_version_config(); auto version_count = tablet()->version_count(); DBUG_EXECUTE_IF("RowsetBuilder.check_tablet_version_count.too_many_version", @@ -181,11 +207,19 @@ Status RowsetBuilder::check_tablet_version_count() { } Status RowsetBuilder::prepare_txn() { + DCHECK(is_data_builder()); return tablet()->prepare_txn(_req.partition_id, _req.txn_id, _req.load_id, false); } Status RowsetBuilder::init() { - _tablet = DORIS_TRY(_engine.get_tablet(_req.tablet_id)); + RowsetWriterContext context; + + RETURN_IF_ERROR(_init_context_common_fields(context)); + + if (tablet()->enable_row_binlog()) { + context.write_binlog_opt().mark_primary_writer(); + } + std::shared_ptr mow_context; if (_tablet->enable_unique_key_merge_on_write()) { RETURN_IF_ERROR(init_mow_context(mow_context)); @@ -215,7 +249,27 @@ Status RowsetBuilder::init() { // build tablet schema in request level RETURN_IF_ERROR(_build_current_tablet_schema(_req.index_id, _req.table_schema_param.get(), *_tablet->tablet_schema())); - RowsetWriterContext context; + + context.mow_context = mow_context; + + context.partial_update_info = _partial_update_info; + _rowset_writer = DORIS_TRY(_tablet->create_rowset_writer(context, false)); + _rowset_id = context.rowset_id; + std::vector tmp_pending_rowset_ids = {_rowset_id}; + tmp_pending_rowset_ids.resize(1 + _attach_rowset_ids.size()); + std::copy(_attach_rowset_ids.begin(), _attach_rowset_ids.end(), + tmp_pending_rowset_ids.begin() + 1); + _pending_rs_guard = _engine.pending_local_rowsets().add(tmp_pending_rowset_ids); + + _calc_delete_bitmap_token = _engine.calc_delete_bitmap_executor()->create_token(); + + _is_init = true; + return Status::OK(); +} + +Status BaseRowsetBuilder::_init_context_common_fields(RowsetWriterContext& context) { + _tablet = DORIS_TRY(ExecEnv::get_tablet(_req.tablet_id)); + context.txn_id = _req.txn_id; context.load_id = _req.load_id; context.rowset_state = PREPARED; @@ -227,15 +281,8 @@ Status RowsetBuilder::init() { context.tablet = _tablet; context.enable_segcompaction = true; context.write_type = DataWriteType::TYPE_DIRECT; - context.mow_context = mow_context; context.write_file_cache = _req.write_file_cache; - context.partial_update_info = _partial_update_info; - _rowset_writer = DORIS_TRY(_tablet->create_rowset_writer(context, false)); - _pending_rs_guard = _engine.pending_local_rowsets().add(context.rowset_id); - _calc_delete_bitmap_token = _engine.calc_delete_bitmap_executor()->create_token(); - - _is_init = true; return Status::OK(); } @@ -250,7 +297,14 @@ Status BaseRowsetBuilder::build_rowset() { return Status::OK(); } +Status GroupRowsetBuilder::build_rowset() { + // build binlog rowset first, then data rowset + RETURN_IF_ERROR(_row_binlog_rowset_builder->build_rowset()); + return _txn_rs_builder->build_rowset(); +} + Status BaseRowsetBuilder::submit_calc_delete_bitmap_task() { + DCHECK(is_data_builder()); if (!_tablet->enable_unique_key_merge_on_write() || _rowset->num_segments() == 0) { return Status::OK(); } @@ -307,6 +361,7 @@ Status BaseRowsetBuilder::submit_calc_delete_bitmap_task() { } Status BaseRowsetBuilder::wait_calc_delete_bitmap() { + DCHECK(is_data_builder()); if (!_tablet->enable_unique_key_merge_on_write() || _partial_update_info->is_partial_update()) { return Status::OK(); } @@ -317,6 +372,7 @@ Status BaseRowsetBuilder::wait_calc_delete_bitmap() { } Status RowsetBuilder::commit_txn() { + DCHECK(is_data_builder()); if (tablet()->enable_unique_key_merge_on_write() && config::enable_merge_on_write_correctness_check && _rowset->num_rows() != 0 && tablet()->tablet_state() != TABLET_NOTREADY) { @@ -392,13 +448,15 @@ Status BaseRowsetBuilder::_build_current_tablet_schema( // After adding a column v2, the schema version increases, max_version_schema needs to be updated. // _tablet_schema includes k, v, and v2 // if v is a variant, need to add the columns decomposed from the v to the _tablet_schema. - if (_tablet_schema->num_variant_columns() > 0) { - TabletSchemaSPtr max_version_schema = std::make_shared(); - max_version_schema->copy_from(*_tablet_schema); - max_version_schema->copy_extracted_columns(ori_tablet_schema); - _tablet->update_max_version_schema(max_version_schema); - } else { - _tablet->update_max_version_schema(_tablet_schema); + if (is_data_builder()) { + if (_tablet_schema->num_variant_columns() > 0) { + TabletSchemaSPtr max_version_schema = std::make_shared(); + max_version_schema->copy_from(*_tablet_schema); + max_version_schema->copy_extracted_columns(ori_tablet_schema); + _tablet->update_max_version_schema(max_version_schema); + } else { + _tablet->update_max_version_schema(_tablet_schema); + } } } @@ -420,4 +478,93 @@ Status BaseRowsetBuilder::_build_current_tablet_schema( table_schema_param->sequence_map_col_uid(), _max_version_in_flush_phase)); return Status::OK(); } + +GroupRowsetBuilder::GroupRowsetBuilder(StorageEngine& engine, const WriteRequest& req, + const WriteRequest& row_binlog_req, RuntimeProfile* profile) + : BaseRowsetBuilder( + [](int64_t tablet_id) { + WriteRequest group_req; + group_req.tablet_id = tablet_id; + group_req.write_req_type = WriteRequestType::GROUP; + return group_req; + }(req.tablet_id), + profile) { + _row_binlog_rowset_builder = + std::make_shared(engine, row_binlog_req, profile); + _txn_rs_builder = std::make_shared(engine, req, profile); +} + +Status GroupRowsetBuilder::init() { + // init binlog builder first so that its rowset id can be added into + // PendingLocalRowsets before txn builder init. + RETURN_IF_ERROR(_row_binlog_rowset_builder->init()); + // before init txn, need to add all rowset_ids into PendingLocalRowsets. + // see https://github.com/apache/doris/pull/25921 + RETURN_IF_ERROR(_txn_rs_builder->attach_pending_rs_guard_to_txn( + _row_binlog_rowset_builder->rowset_id())); + RETURN_IF_ERROR(_txn_rs_builder->init()); + + // Create a GroupRowsetWriter that forwards flush to both underlying + // RowsetWriters. + std::unique_ptr group_writer; + RETURN_IF_ERROR(RowsetFactory::create_empty_group_rowset_writer(&group_writer)); + group_writer->set_data_writer(_txn_rs_builder->rowset_writer()); + group_writer->set_row_binlog_writer(_row_binlog_rowset_builder->rowset_writer()); + + _rowset_writer = std::move(group_writer); + _is_init = true; + return Status::OK(); +} + +Status GroupRowsetBuilder::submit_calc_delete_bitmap_task() { + return _txn_rs_builder->submit_calc_delete_bitmap_task(); +} + +Status GroupRowsetBuilder::wait_calc_delete_bitmap() { + return _txn_rs_builder->wait_calc_delete_bitmap(); +} + +Status GroupRowsetBuilder::commit_txn() { + // Attach binlog rowset to txn rowset, so that commit/rollback and + // clean-up are all handled by txn rowset builder. + RETURN_IF_ERROR(_txn_rs_builder->attach_rowset_to_txn(_row_binlog_rowset_builder->rowset())); + return _txn_rs_builder->commit_txn(); +} + +Status RowBinlogRowsetBuilder::init() { + RowsetWriterContext context; + + RETURN_IF_ERROR(_init_context_common_fields(context)); + + // build tablet schema in request level + RETURN_IF_ERROR(_build_current_tablet_schema( + _req.index_id, _req.table_schema_param.get(), + *std::dynamic_pointer_cast(_tablet)->row_binlog_tablet_schema())); + context.write_binlog_opt().mark_binlog_writer(); + + _rowset_writer = DORIS_TRY(_tablet->create_rowset_writer(context, false)); + // need to attach PendingRowsetGuard after txn_rs_builder init + _rowset_id = context.rowset_id; + + _is_init = true; + return Status::OK(); +} + +Status BaseRowsetBuilder::attach_rowset_to_txn(const RowsetSharedPtr& rowset) { + if (!is_data_builder()) { + return Status::RuntimeError("the rowset isn't allowed to manage txn"); + } + _attach_rowsets.push_back(rowset); + return Status::OK(); +} + +Status BaseRowsetBuilder::attach_pending_rs_guard_to_txn(const RowsetId& rowset_id) { + if (!is_data_builder()) { + return Status::RuntimeError("the rowset isn't allowed to manage txn"); + } + _attach_rowset_ids.push_back(rowset_id); + return Status::OK(); +} + +#include "common/compile_check_end.h" } // namespace doris diff --git a/be/src/storage/rowset_builder.h b/be/src/storage/rowset_builder.h index 205447d8faaeb1..edc97432eaf9c9 100644 --- a/be/src/storage/rowset_builder.h +++ b/be/src/storage/rowset_builder.h @@ -56,11 +56,15 @@ class BaseRowsetBuilder { virtual Status init() = 0; - Status build_rowset(); + virtual Status build_rowset(); - Status submit_calc_delete_bitmap_task(); + virtual Status submit_calc_delete_bitmap_task(); - Status wait_calc_delete_bitmap(); + virtual Status wait_calc_delete_bitmap(); + + virtual Status commit_txn() { + return Status::NotSupported("BaseRowsetBuilder::commit_txn not implemented"); + } Status cancel(); @@ -79,6 +83,17 @@ class BaseRowsetBuilder { return _partial_update_info; } + bool is_data_builder() const { return _req.write_req_type == WriteRequestType::DATA; } + + // Attach an extra rowset (e.g. binlog rowset) to the same txn. + Status attach_rowset_to_txn(const RowsetSharedPtr& rowset); + + // Attach an extra pending rowset id so that PendingLocalRowsets can be + // cleaned up together with the primary rowset. + Status attach_pending_rs_guard_to_txn(const RowsetId& rowset_id); + + RowsetId rowset_id() const { return _rowset_id; } + Status init_mow_context(std::shared_ptr& mow_context); protected: @@ -88,13 +103,20 @@ class BaseRowsetBuilder { virtual void _init_profile(RuntimeProfile* profile); + Status _init_context_common_fields(RowsetWriterContext& context); + bool _is_init = false; bool _is_cancelled = false; + bool _is_committed = false; WriteRequest _req; BaseTabletSPtr _tablet; RowsetSharedPtr _rowset; + // Extra rowsets attached to the same txn (e.g. binlog rowsets). + std::vector _attach_rowsets; std::shared_ptr _rowset_writer; PendingRowsetGuard _pending_rs_guard; + // Extra rowset ids that share the same PendingRowsetGuard. + std::vector _attach_rowset_ids; TabletSchemaSPtr _tablet_schema; std::mutex _lock; @@ -111,10 +133,12 @@ class BaseRowsetBuilder { RuntimeProfile::Counter* _build_rowset_timer = nullptr; RuntimeProfile::Counter* _submit_delete_bitmap_timer = nullptr; RuntimeProfile::Counter* _wait_delete_bitmap_timer = nullptr; + + RowsetId _rowset_id; }; // `StorageEngine` mixin for `BaseRowsetBuilder` -class RowsetBuilder final : public BaseRowsetBuilder { +class RowsetBuilder : public BaseRowsetBuilder { public: RowsetBuilder(StorageEngine& engine, const WriteRequest& req, RuntimeProfile* profile); @@ -122,7 +146,7 @@ class RowsetBuilder final : public BaseRowsetBuilder { Status init() override; - Status commit_txn(); + Status commit_txn() override; // Cast `BaseTablet` to `Tablet` Tablet* tablet(); @@ -134,13 +158,62 @@ class RowsetBuilder final : public BaseRowsetBuilder { Status prepare_txn(); - void _garbage_collection(); + void _garbage_collection(bool cancel_txn); TabletSharedPtr tablet_sptr(); StorageEngine& _engine; RuntimeProfile::Counter* _commit_txn_timer = nullptr; - bool _is_committed = false; +}; + +// Rowset builder dedicated for row_binlog rowset, it shares the same tablet +// but uses an independent row_binlog tablet schema. +class RowBinlogRowsetBuilder : public RowsetBuilder { +public: + RowBinlogRowsetBuilder(StorageEngine& engine, const WriteRequest& req, RuntimeProfile* profile); + + // just attach rowset to txn_rs_builder in GroupRowsetBuilder, then rely on + // txn_rs_builder's clean logic. + ~RowBinlogRowsetBuilder() override = default; + + Status init() override; + + // before commit, binlog rowset builder is responsible for cleaning rowset. + // after commit, rowset will be attached to data(txn) rowset builder, and + // the owner of rowset will be changed, so cleaning rowset is handed to the + // data(txn) rowset builder. + Status commit_txn() override { + _is_committed = true; + return Status::OK(); + } +}; + +// manage one transaction with multiple rowset_builder +// eg. normal data rowset + row_binlog rowset +// Now only support one tablet +class GroupRowsetBuilder : public BaseRowsetBuilder { +public: + GroupRowsetBuilder(StorageEngine& engine, const WriteRequest& req, + const WriteRequest& row_binlog_req, RuntimeProfile* profile); + + Status init() override; + + Status build_rowset() override; + + Status submit_calc_delete_bitmap_task() override; + + Status wait_calc_delete_bitmap() override; + + Status commit_txn() override; + + RowsetBuilder* txn_rowset_builder() { return _txn_rs_builder.get(); } + RowsetBuilder* row_binlog_builder() { return _row_binlog_rowset_builder.get(); } + +private: + // txn rowset builder will manage txn; other builders will add their + // rowsets into here. + std::shared_ptr _txn_rs_builder; + std::shared_ptr _row_binlog_rowset_builder; }; } // namespace doris diff --git a/be/src/storage/snapshot/snapshot_manager.cpp b/be/src/storage/snapshot/snapshot_manager.cpp index 8c3131612dfa2c..f5d16c9a1581af 100644 --- a/be/src/storage/snapshot/snapshot_manager.cpp +++ b/be/src/storage/snapshot/snapshot_manager.cpp @@ -201,6 +201,7 @@ Result> SnapshotManager::convert_rowset_ids( // keep this just for safety new_tablet_meta_pb.clear_inc_rs_metas(); new_tablet_meta_pb.clear_stale_rs_metas(); + // should modify tablet id and schema hash because in restore process the tablet id is not // equal to tablet id in meta new_tablet_meta_pb.set_tablet_id(tablet_id); @@ -715,7 +716,7 @@ Status SnapshotManager::_create_snapshot_files(const TabletSharedPtr& ref_tablet break; } - if (!is_copy_binlog || !target_tablet->is_enable_binlog()) { + if (!is_copy_binlog || !target_tablet->enable_ccr_binlog()) { break; } diff --git a/be/src/storage/tablet/base_tablet.cpp b/be/src/storage/tablet/base_tablet.cpp index e59ee50f1f12c7..6bacf1541b1eeb 100644 --- a/be/src/storage/tablet/base_tablet.cpp +++ b/be/src/storage/tablet/base_tablet.cpp @@ -318,11 +318,16 @@ void BaseTablet::_print_missed_versions(const Versions& missed_versions) const { } bool BaseTablet::_reconstruct_version_tracker_if_necessary() { - double orphan_vertex_ratio = _timestamped_version_tracker.get_orphan_vertex_ratio(); - if (orphan_vertex_ratio >= config::tablet_version_graph_orphan_vertex_ratio) { + double data_orphan_vertex_ratio = _timestamped_version_tracker.get_orphan_vertex_ratio(); + double row_binlog_orphan_vertex_ratio = _row_binlog_version_tracker.get_orphan_vertex_ratio(); + if (data_orphan_vertex_ratio >= config::tablet_version_graph_orphan_vertex_ratio) { _timestamped_version_tracker.construct_versioned_tracker( _tablet_meta->all_rs_metas(), _tablet_meta->all_stale_rs_metas()); return true; + } else if (row_binlog_orphan_vertex_ratio >= config::tablet_version_graph_orphan_vertex_ratio) { + _row_binlog_version_tracker.construct_versioned_tracker( + _tablet_meta->all_row_binlog_rs_metas()); + return true; } return false; } diff --git a/be/src/storage/tablet/base_tablet.h b/be/src/storage/tablet/base_tablet.h index b98a89eb734f2c..4459a62d12ab77 100644 --- a/be/src/storage/tablet/base_tablet.h +++ b/be/src/storage/tablet/base_tablet.h @@ -367,6 +367,8 @@ class BaseTablet : public std::enable_shared_from_this { mutable std::shared_mutex _meta_lock; TimestampedVersionTracker _timestamped_version_tracker; + TimestampedVersionTracker _row_binlog_version_tracker; + // After version 0.13, all newly created rowsets are saved in _rs_version_map. // And if rowset being compacted, the old rowsets will be saved in _stale_rs_version_map; std::unordered_map _rs_version_map; @@ -374,6 +376,8 @@ class BaseTablet : public std::enable_shared_from_this { // These _stale rowsets are been removed when rowsets' pathVersion is expired, // this policy is judged and computed by TimestampedVersionTracker. std::unordered_map _stale_rs_version_map; + // for row_binlog + std::unordered_map _row_binlog_rs_version_map; const TabletMetaSharedPtr _tablet_meta; TabletSchemaSPtr _max_version_schema; diff --git a/be/src/storage/tablet/tablet.cpp b/be/src/storage/tablet/tablet.cpp index 5e75dc66db4cdb..9f0ba06696a3b9 100644 --- a/be/src/storage/tablet/tablet.cpp +++ b/be/src/storage/tablet/tablet.cpp @@ -93,6 +93,7 @@ #include "storage/olap_define.h" #include "storage/olap_meta.h" #include "storage/rowset/beta_rowset.h" +#include "storage/rowset/group_rowset_writer.h" #include "storage/rowset/rowset.h" #include "storage/rowset/rowset_factory.h" #include "storage/rowset/rowset_fwd.h" @@ -323,6 +324,20 @@ Status Tablet::_init_once_action() { _stale_rs_version_map[version] = std::move(rowset); } + // init row_binlog rowset + for (const auto& [_, row_binlog_rs_meta] : _tablet_meta->all_row_binlog_rs_metas()) { + Version version = row_binlog_rs_meta->version(); + RowsetSharedPtr rowset; + res = create_rowset(row_binlog_rs_meta, &rowset); + if (!res.ok()) { + LOG(WARNING) << "fail to init row_binlog rowset. tablet_id:" << tablet_id() + << ", schema_hash:" << schema_hash() << ", version=" << version + << ", res:" << res; + return res; + } + _row_binlog_rs_version_map[version] = std::move(rowset); + } + return res; } @@ -479,7 +494,7 @@ Status Tablet::revise_tablet_meta(const std::vector& to_add, return Status::OK(); } -Status Tablet::add_rowset(RowsetSharedPtr rowset) { +Status Tablet::add_rowset(RowsetSharedPtr rowset, RowsetSharedPtr row_binlog_rowset) { DCHECK(rowset != nullptr); std::lock_guard wrlock(_meta_lock); SCOPED_SIMPLE_TRACE_IF_TIMEOUT(TRACE_TABLET_LOCK_THRESHOLD); @@ -496,6 +511,12 @@ Status Tablet::add_rowset(RowsetSharedPtr rowset) { _timestamped_version_tracker.add_version(rowset->version()); add_compaction_score(rowset->rowset_meta()->get_compaction_score()); + if (row_binlog_rowset != nullptr) { + RETURN_IF_ERROR(_tablet_meta->add_row_binlog_rs_meta(row_binlog_rowset->rowset_meta())); + _row_binlog_rs_version_map[rowset->version()] = row_binlog_rowset; + _row_binlog_version_tracker.add_version(row_binlog_rowset->version()); + } + std::vector rowsets_to_delete; // yiguolei: temp code, should remove the rowset contains by this rowset // but it should be removed in multi path version @@ -1719,6 +1740,12 @@ void Tablet::build_tablet_report_info(TTabletInfo* tablet_info, tablet_info->__set_local_segment_size(_tablet_meta->tablet_local_segment_size()); tablet_info->__set_remote_index_size(_tablet_meta->tablet_remote_index_size()); tablet_info->__set_remote_segment_size(_tablet_meta->tablet_remote_segment_size()); + if (enable_row_binlog()) { + int64_t total_binlog_size = _tablet_meta->binlog_size(); + int64_t total_binlog_file_num = _tablet_meta->binlog_file_num(); + tablet_info->__set_binlog_file_num(total_binlog_file_num); + tablet_info->__set_binlog_size(total_binlog_size); + } } void Tablet::report_error(const Status& st) { @@ -1917,18 +1944,57 @@ Status Tablet::create_initial_rowset(const int64_t req_version) { "init version of tablet should at least 1. req.ver={}", req_version); } Version version(0, req_version); - RowsetSharedPtr new_rowset; - // there is no data in init rowset, so overlapping info is unknown. - RowsetWriterContext context; - context.version = version; - context.rowset_state = VISIBLE; - context.segments_overlap = OVERLAP_UNKNOWN; - context.tablet_schema = tablet_schema(); - context.newest_write_timestamp = UnixSeconds(); - auto rs_writer = DORIS_TRY(create_rowset_writer(context, false)); - RETURN_IF_ERROR(rs_writer->flush()); - RETURN_IF_ERROR(rs_writer->build(new_rowset)); - RETURN_IF_ERROR(add_rowset(std::move(new_rowset))); + + auto get_rowset_writer_context = [&](RowsetWriterContext& context, TabletSchemaSPtr schema) { + // there is no data in init rowset, so overlapping info is unknown. + context.version = version; + context.rowset_state = VISIBLE; + context.segments_overlap = OVERLAP_UNKNOWN; + context.tablet_schema = schema; + context.newest_write_timestamp = UnixSeconds(); + + return Status::OK(); + }; + + if (!enable_row_binlog()) { + RowsetWriterContext context; + RowsetSharedPtr new_rowset; + RETURN_IF_ERROR(get_rowset_writer_context(context, tablet_schema())); + auto rs_writer = DORIS_TRY(create_rowset_writer(context, false)); + + RETURN_IF_ERROR(rs_writer->flush()); + RETURN_IF_ERROR(rs_writer->build(new_rowset)); + RETURN_IF_ERROR(add_rowset(std::move(new_rowset), nullptr)); + } else { + std::unique_ptr group_rowset_writer; + RETURN_IF_ERROR(RowsetFactory::create_empty_group_rowset_writer(&group_rowset_writer)); + + RowsetWriterContext data_context; + data_context.write_binlog_opt().mark_primary_writer(); + RETURN_IF_ERROR(get_rowset_writer_context(data_context, tablet_schema())); + auto data_writer = DORIS_TRY(create_rowset_writer(data_context, false)); + group_rowset_writer->set_data_writer(std::move(data_writer)); + + RowsetWriterContext row_binlog_context; + row_binlog_context.write_binlog_opt().mark_binlog_writer(); + RETURN_IF_ERROR(get_rowset_writer_context(row_binlog_context, row_binlog_tablet_schema())); + auto row_binlog_writer = DORIS_TRY(create_rowset_writer(row_binlog_context, false)); + group_rowset_writer->set_row_binlog_writer(std::move(row_binlog_writer)); + + RETURN_IF_ERROR(group_rowset_writer->flush_rowsets()); + + RowsetSharedPtr new_data_rowset; + RowsetSharedPtr new_row_binlog_rowset; + std::vector waited_build_rowsets; + waited_build_rowsets.push_back(std::move(new_data_rowset)); + waited_build_rowsets.push_back(std::move(new_row_binlog_rowset)); + + RETURN_IF_ERROR(group_rowset_writer->build_rowsets(waited_build_rowsets)); + // don't need to think rollback when only one rowset build success becuase they had not been persisted. + RETURN_IF_ERROR( + add_rowset(std::move(waited_build_rowsets.at(0)), waited_build_rowsets.at(1))); + } + set_cumulative_layer_point(req_version + 1); return Status::OK(); } @@ -2002,12 +2068,18 @@ void Tablet::_init_context_common_fields(RowsetWriterContext& context) { context.enable_unique_key_merge_on_write = enable_unique_key_merge_on_write(); context.encrypt_algorithm = tablet_meta()->encryption_algorithm(); + + if (context.write_binlog_opt().is_binlog_writer()) { + context.tablet_schema_hash = row_binlog_schema_hash(); + bool need_before = tablet_meta()->binlog_config().need_historical_value(); + context.write_binlog_opt().set_need_before(need_before); + context.tablet_path = row_binlog_path(); + } } Status Tablet::create_rowset(const RowsetMetaSharedPtr& rowset_meta, RowsetSharedPtr* rowset) { - return RowsetFactory::create_rowset(_tablet_meta->tablet_schema(), - rowset_meta->is_local() ? _tablet_path : "", rowset_meta, - rowset); + return RowsetFactory::create_rowset(_tablet_meta->tablet_schema(), get_rowset_path(rowset_meta), + rowset_meta, rowset); } Status Tablet::cooldown(RowsetSharedPtr rowset) { @@ -2657,10 +2729,6 @@ bool Tablet::can_add_binlog(uint64_t total_binlog_size) const { return !_data_dir->reach_capacity_limit(total_binlog_size); } -bool Tablet::is_enable_binlog() { - return config::enable_feature_binlog && tablet_meta()->binlog_config().is_enable(); -} - void Tablet::set_binlog_config(BinlogConfig binlog_config) { tablet_meta()->set_binlog_config(binlog_config); } diff --git a/be/src/storage/tablet/tablet.h b/be/src/storage/tablet/tablet.h index 855b827323e363..3c715cea7e48d0 100644 --- a/be/src/storage/tablet/tablet.h +++ b/be/src/storage/tablet/tablet.h @@ -38,6 +38,7 @@ #include "common/config.h" #include "common/metrics/metrics.h" #include "common/status.h" +#include "storage/binlog.h" #include "storage/binlog_config.h" #include "storage/data_dir.h" #include "storage/olap_common.h" @@ -116,6 +117,15 @@ class Tablet final : public BaseTablet { int64_t replica_id() const { return _tablet_meta->replica_id(); } std::string tablet_path() const override { return _tablet_path; } + std::string row_binlog_path() const { + return fmt::format("{}/{}", _tablet_path, FDRowBinlogSuffix); + } + std::string get_rowset_path(const RowsetMetaSharedPtr& rowset_meta) const { + if (!rowset_meta->is_local()) { + return ""; + } + return rowset_meta->is_row_binlog() ? row_binlog_path() : tablet_path(); + } bool set_tablet_schema_into_rowset_meta(); Status init(); @@ -164,7 +174,7 @@ class Tablet final : public BaseTablet { int64_t avg_rs_meta_serialize_size() const; // operation in rowsets - Status add_rowset(RowsetSharedPtr rowset); + Status add_rowset(RowsetSharedPtr rowset, RowsetSharedPtr row_binlog_rowset = nullptr); Status create_initial_rowset(const int64_t version); // MUST hold EXCLUSIVE `_meta_lock`. @@ -377,6 +387,8 @@ class Tablet final : public BaseTablet { const auto& rowset_map() const { return _rs_version_map; } // MUST hold SHARED `_meta_lock` const auto& stale_rowset_map() const { return _stale_rs_version_map; } + // MUST hold SHARED `_meta_lock` + const auto& row_binlog_rowset_map() const { return _row_binlog_rs_version_map; } //////////////////////////////////////////////////////////////////////////// // begin cooldown functions @@ -467,13 +479,29 @@ class Tablet final : public BaseTablet { int64_t get_table_id() { return _tablet_meta->table_id(); } // binlog related functions - bool is_enable_binlog(); - bool is_binlog_enabled() { return _tablet_meta->binlog_config().is_enable(); } + bool enable_binlog() const { + return config::enable_feature_binlog && _tablet_meta->binlog_config().is_enable(); + } + bool enable_ccr_binlog() const { + return enable_binlog() && _tablet_meta->binlog_config().isCCRBinlogFormat(); + } + bool enable_row_binlog() const { + return _tablet_meta->binlog_config().is_enable() && + _tablet_meta->binlog_config().isRowBinlogFormat(); + } + int64_t binlog_ttl_ms() const { return _tablet_meta->binlog_config().ttl_seconds(); } int64_t binlog_max_bytes() const { return _tablet_meta->binlog_config().max_bytes(); } void set_binlog_config(BinlogConfig binlog_config); + // row_binlog + int32_t row_binlog_schema_hash() const { return _tablet_meta->row_binlog_schema_hash(); } + TabletSchemaSPtr row_binlog_tablet_schema() { + std::shared_lock rdlock(_meta_lock); + return _tablet_meta->row_binlog_schema(); + } + void set_is_full_compaction_running(bool is_full_compaction_running) { _is_full_compaction_running = is_full_compaction_running; } diff --git a/be/src/storage/tablet/tablet_manager.cpp b/be/src/storage/tablet/tablet_manager.cpp index 1f8bbabdd2d36e..4c3b54e3cf7f20 100644 --- a/be/src/storage/tablet/tablet_manager.cpp +++ b/be/src/storage/tablet/tablet_manager.cpp @@ -1114,6 +1114,10 @@ void TabletManager::build_all_report_tablets_info(std::map* t_tablet_stat.__set_local_segment_size(tablet_info.local_segment_size); t_tablet_stat.__set_remote_index_size(tablet_info.remote_index_size); t_tablet_stat.__set_remote_segment_size(tablet_info.remote_segment_size); + if (tablet_info.__isset.binlog_size) { + t_tablet_stat.__set_binlog_size(tablet_info.binlog_size); + t_tablet_stat.__set_binlog_file_num(tablet_info.binlog_file_num); + } }; for_each_tablet(handler, filter_all_tablets); diff --git a/be/src/storage/tablet/tablet_meta.cpp b/be/src/storage/tablet/tablet_meta.cpp index 68a4720dece643..d11b69f59f308b 100644 --- a/be/src/storage/tablet/tablet_meta.cpp +++ b/be/src/storage/tablet/tablet_meta.cpp @@ -127,7 +127,8 @@ TabletMetaSharedPtr TabletMeta::create( request.tde_algorithm, storage_format, request.__isset.vertical_compaction_num_columns_per_group ? request.vertical_compaction_num_columns_per_group - : 5); + : 5, + request.__isset.row_binlog_schema ? &request.row_binlog_schema : nullptr); } TabletMeta::~TabletMeta() { @@ -139,7 +140,8 @@ TabletMeta::~TabletMeta() { TabletMeta::TabletMeta() : _tablet_uid(0, 0), _schema(new TabletSchema), - _delete_bitmap(new DeleteBitmap(_tablet_id)) {} + _delete_bitmap(new DeleteBitmap(_tablet_id)), + _binlog_delvec(new DeleteBitmap(_tablet_id)) {} TabletMeta::TabletMeta(int64_t table_id, int64_t partition_id, int64_t tablet_id, int64_t replica_id, int32_t schema_hash, int32_t shard_id, @@ -157,10 +159,12 @@ TabletMeta::TabletMeta(int64_t table_id, int64_t partition_id, int64_t tablet_id TInvertedIndexFileStorageFormat::type inverted_index_file_storage_format, TEncryptionAlgorithm::type tde_algorithm, TStorageFormat::type storage_format, - int32_t vertical_compaction_num_columns_per_group) + int32_t vertical_compaction_num_columns_per_group, + const TTabletSchema* row_binlog_schema) : _tablet_uid(0, 0), _schema(new TabletSchema), _delete_bitmap(new DeleteBitmap(tablet_id)), + _binlog_delvec(new DeleteBitmap(tablet_id)), _storage_format(storage_format) { TabletMetaPB tablet_meta_pb; tablet_meta_pb.set_table_id(table_id); @@ -192,208 +196,47 @@ TabletMeta::TabletMeta(int64_t table_id, int64_t partition_id, int64_t tablet_id time_series_compaction_level_threshold); tablet_meta_pb.set_vertical_compaction_num_columns_per_group( vertical_compaction_num_columns_per_group); - TabletSchemaPB* schema = tablet_meta_pb.mutable_schema(); - schema->set_num_short_key_columns(tablet_schema.short_key_column_count); - schema->set_num_rows_per_row_block(config::default_num_rows_per_column_file_block); - schema->set_sequence_col_idx(tablet_schema.sequence_col_idx); - auto p_seq_map = schema->mutable_seq_map(); // ColumnGroupsPB + SchemaCreateOptions schema_create_options_for_data = { + .col_ordinal_to_unique_id = col_ordinal_to_unique_id, + .compression_type = compression_type, + .inverted_index_file_storage_format = inverted_index_file_storage_format, + .next_unique_id = next_unique_id}; + TabletSchemaPB* schema_pb_for_data = tablet_meta_pb.mutable_schema(); + init_schema_from_thrift(tablet_schema, schema_create_options_for_data, schema_pb_for_data); - for (auto& it : tablet_schema.seq_map) { // std::vector< ::doris::TColumnGroup> - uint32_t key = it.sequence_column; - ColumnGroupPB* cg_pb = p_seq_map->add_cg(); // ColumnGroupPB {key: {v1, v2, v3}} - cg_pb->set_sequence_column(key); - for (auto v : it.columns_in_group) { - cg_pb->add_columns_in_group(v); - } - } - switch (tablet_schema.keys_type) { - case TKeysType::DUP_KEYS: - schema->set_keys_type(KeysType::DUP_KEYS); - break; - case TKeysType::UNIQUE_KEYS: - schema->set_keys_type(KeysType::UNIQUE_KEYS); - break; - case TKeysType::AGG_KEYS: - schema->set_keys_type(KeysType::AGG_KEYS); - break; - default: - LOG(WARNING) << "unknown tablet keys type"; - break; - } - // compress_kind used to compress segment files - schema->set_compress_kind(COMPRESS_LZ4); - - // compression_type used to compress segment page - switch (compression_type) { - case TCompressionType::NO_COMPRESSION: - schema->set_compression_type(segment_v2::NO_COMPRESSION); - break; - case TCompressionType::SNAPPY: - schema->set_compression_type(segment_v2::SNAPPY); - break; - case TCompressionType::LZ4: - schema->set_compression_type(segment_v2::LZ4); - break; - case TCompressionType::LZ4F: - schema->set_compression_type(segment_v2::LZ4F); - break; - case TCompressionType::ZLIB: - schema->set_compression_type(segment_v2::ZLIB); - break; - case TCompressionType::ZSTD: - schema->set_compression_type(segment_v2::ZSTD); - break; - default: - schema->set_compression_type(segment_v2::LZ4F); - break; - } - - switch (inverted_index_file_storage_format) { - case TInvertedIndexFileStorageFormat::V1: - schema->set_inverted_index_storage_format(InvertedIndexStorageFormatPB::V1); - break; - case TInvertedIndexFileStorageFormat::V2: - schema->set_inverted_index_storage_format(InvertedIndexStorageFormatPB::V2); - break; - case TInvertedIndexFileStorageFormat::V3: - schema->set_inverted_index_storage_format(InvertedIndexStorageFormatPB::V3); - break; - default: - schema->set_inverted_index_storage_format(InvertedIndexStorageFormatPB::V3); - break; - } - - switch (tablet_schema.sort_type) { - case TSortType::type::ZORDER: - schema->set_sort_type(SortType::ZORDER); - break; - default: - schema->set_sort_type(SortType::LEXICAL); - } - schema->set_sort_col_num(tablet_schema.sort_col_num); - for (const auto& i : tablet_schema.cluster_key_uids) { - schema->add_cluster_key_uids(i); - } tablet_meta_pb.set_in_restore_mode(false); - // set column information - uint32_t col_ordinal = 0; - bool has_bf_columns = false; - for (TColumn tcolumn : tablet_schema.columns) { - ColumnPB* column = schema->add_column(); - uint32_t unique_id = -1; - if (tcolumn.col_unique_id >= 0) { - unique_id = tcolumn.col_unique_id; - } else { - unique_id = col_ordinal_to_unique_id.at(col_ordinal); - } - col_ordinal++; - init_column_from_tcolumn(unique_id, tcolumn, column); - - if (column->is_bf_column()) { - has_bf_columns = true; - } - - if (tablet_schema.__isset.indexes) { - for (auto& index : tablet_schema.indexes) { - if (index.index_type == TIndexType::type::BLOOMFILTER || - index.index_type == TIndexType::type::NGRAM_BF) { - DCHECK_EQ(index.columns.size(), 1); - if (iequal(tcolumn.column_name, index.columns[0])) { - column->set_is_bf_column(true); - break; - } - } - } - } - } - - // copy index meta - if (tablet_schema.__isset.indexes) { - for (auto& index : tablet_schema.indexes) { - TabletIndexPB* index_pb = schema->add_index(); - index_pb->set_index_id(index.index_id); - index_pb->set_index_name(index.index_name); - // init col_unique_id in index at be side, since col_unique_id may be -1 at fe side - // get column unique id by name - for (auto column_name : index.columns) { - for (auto column : schema->column()) { - if (iequal(column.name(), column_name)) { - index_pb->add_col_unique_id(column.unique_id()); - } - } - } - switch (index.index_type) { - case TIndexType::BITMAP: - index_pb->set_index_type(IndexType::BITMAP); - break; - case TIndexType::INVERTED: - index_pb->set_index_type(IndexType::INVERTED); - break; - case TIndexType::ANN: - index_pb->set_index_type(IndexType::ANN); - break; - case TIndexType::BLOOMFILTER: - index_pb->set_index_type(IndexType::BLOOMFILTER); - break; - case TIndexType::NGRAM_BF: - index_pb->set_index_type(IndexType::NGRAM_BF); - break; + TabletSchemaPB* schema_pb_for_row_binlog = nullptr; + if (row_binlog_schema != nullptr) { + tablet_meta_pb.set_row_binlog_schema_hash(row_binlog_schema->schema_hash); + DCHECK(binlog_config.has_value()); + DCHECK(binlog_config->enable && binlog_config->binlog_format == TBinlogFormat::ROW); + + std::unordered_map row_binlog_col_ordinal_to_unique_id; + uint32_t row_binlog_next_unique_id = 0; + for (uint32_t col_ordinal = 0; col_ordinal < row_binlog_schema->columns.size(); + ++col_ordinal) { + const auto& tcolumn = row_binlog_schema->columns[col_ordinal]; + uint32_t unique_id = 0; + if (tcolumn.col_unique_id >= 0) { + unique_id = tcolumn.col_unique_id; + } else { + unique_id = col_ordinal; } - - if (index.__isset.properties) { - auto properties = index_pb->mutable_properties(); - for (auto kv : index.properties) { - (*properties)[kv.first] = kv.second; - } + row_binlog_col_ordinal_to_unique_id[col_ordinal] = unique_id; + if (row_binlog_next_unique_id <= unique_id) { + row_binlog_next_unique_id = unique_id + 1; } } - } - - schema->set_next_column_unique_id(next_unique_id); - if (has_bf_columns && tablet_schema.__isset.bloom_filter_fpp) { - schema->set_bf_fpp(tablet_schema.bloom_filter_fpp); - } - - if (tablet_schema.__isset.is_in_memory) { - schema->set_is_in_memory(tablet_schema.is_in_memory); - } - - if (tablet_schema.__isset.disable_auto_compaction) { - schema->set_disable_auto_compaction(tablet_schema.disable_auto_compaction); - } - // Deprecated legacy flatten-nested switch. Distinct from variant_enable_nested_group. - if (tablet_schema.__isset.variant_enable_flatten_nested) { - schema->set_enable_variant_flatten_nested(tablet_schema.variant_enable_flatten_nested); - } - - if (tablet_schema.__isset.enable_single_replica_compaction) { - schema->set_enable_single_replica_compaction( - tablet_schema.enable_single_replica_compaction); - } - - if (tablet_schema.__isset.delete_sign_idx) { - schema->set_delete_sign_idx(tablet_schema.delete_sign_idx); - } - if (tablet_schema.__isset.store_row_column) { - schema->set_store_row_column(tablet_schema.store_row_column); - } - if (tablet_schema.__isset.row_store_page_size) { - schema->set_row_store_page_size(tablet_schema.row_store_page_size); - } - if (tablet_schema.__isset.storage_page_size) { - schema->set_storage_page_size(tablet_schema.storage_page_size); - } - if (tablet_schema.__isset.storage_dict_page_size) { - schema->set_storage_dict_page_size(tablet_schema.storage_dict_page_size); - } - if (tablet_schema.__isset.skip_write_index_on_load) { - schema->set_skip_write_index_on_load(tablet_schema.skip_write_index_on_load); - } - if (tablet_schema.__isset.row_store_col_cids) { - schema->mutable_row_store_column_unique_ids()->Add(tablet_schema.row_store_col_cids.begin(), - tablet_schema.row_store_col_cids.end()); + SchemaCreateOptions schema_create_options_for_row_binlog = { + .col_ordinal_to_unique_id = row_binlog_col_ordinal_to_unique_id, + .compression_type = compression_type, + .inverted_index_file_storage_format = inverted_index_file_storage_format, + .next_unique_id = row_binlog_next_unique_id}; + schema_pb_for_row_binlog = tablet_meta_pb.mutable_row_binlog_schema(); + init_schema_from_thrift(*row_binlog_schema, schema_create_options_for_row_binlog, + schema_pb_for_row_binlog); } if (binlog_config.has_value()) { BinlogConfig tmp_binlog_config; @@ -421,15 +264,22 @@ TabletMeta::TabletMeta(int64_t table_id, int64_t partition_id, int64_t tablet_id case TStorageFormat::V1: break; case TStorageFormat::V3: - schema->set_is_external_segment_column_meta_used(true); + schema_pb_for_data->set_is_external_segment_column_meta_used(true); _schema->set_external_segment_meta_used_default(true); - schema->set_integer_type_default_use_plain_encoding(true); + schema_pb_for_data->set_integer_type_default_use_plain_encoding(true); _schema->set_integer_type_default_use_plain_encoding(true); - schema->set_binary_plain_encoding_default_impl( + schema_pb_for_data->set_binary_plain_encoding_default_impl( BinaryPlainEncodingTypePB::BINARY_PLAIN_ENCODING_V2); _schema->set_binary_plain_encoding_default_impl( BinaryPlainEncodingTypePB::BINARY_PLAIN_ENCODING_V2); + + if (schema_pb_for_row_binlog != nullptr) { + schema_pb_for_row_binlog->set_is_external_segment_column_meta_used(true); + schema_pb_for_row_binlog->set_integer_type_default_use_plain_encoding(true); + schema_pb_for_row_binlog->set_binary_plain_encoding_default_impl( + BinaryPlainEncodingTypePB::BINARY_PLAIN_ENCODING_V2); + } break; default: break; @@ -461,6 +311,10 @@ TabletMeta::TabletMeta(const TabletMeta& b) _cooldown_meta_id(b._cooldown_meta_id), _enable_unique_key_merge_on_write(b._enable_unique_key_merge_on_write), _delete_bitmap(b._delete_bitmap), + _binlog_delvec(b._binlog_delvec), + _row_binlog_schema_hash(b._row_binlog_schema_hash), + _row_binlog_schema(b._row_binlog_schema), + _row_binlog_rs_metas(b._row_binlog_rs_metas), _binlog_config(b._binlog_config), _compaction_policy(b._compaction_policy), _time_series_compaction_goal_size_mbytes(b._time_series_compaction_goal_size_mbytes), @@ -575,6 +429,221 @@ void TabletMeta::init_column_from_tcolumn(uint32_t unique_id, const TColumn& tco } } +void TabletMeta::init_schema_from_thrift(const TTabletSchema& tablet_schema, + const SchemaCreateOptions& schema_create_options, + TabletSchemaPB* tablet_schema_pb) { + const std::unordered_map& col_ordinal_to_unique_id = + schema_create_options.col_ordinal_to_unique_id; + TCompressionType::type compression_type = schema_create_options.compression_type; + TInvertedIndexFileStorageFormat::type inverted_index_file_storage_format = + schema_create_options.inverted_index_file_storage_format; + uint32_t next_unique_id = schema_create_options.next_unique_id; + + tablet_schema_pb->set_num_short_key_columns(tablet_schema.short_key_column_count); + tablet_schema_pb->set_num_rows_per_row_block(config::default_num_rows_per_column_file_block); + tablet_schema_pb->set_sequence_col_idx(tablet_schema.sequence_col_idx); + auto p_seq_map = tablet_schema_pb->mutable_seq_map(); // ColumnGroupsPB + for (auto& it : tablet_schema.seq_map) { // std::vector< ::doris::TColumnGroup> + uint32_t key = it.sequence_column; + ColumnGroupPB* cg_pb = p_seq_map->add_cg(); // ColumnGroupPB {key: {v1, v2, v3}} + cg_pb->set_sequence_column(key); + for (auto v : it.columns_in_group) { + cg_pb->add_columns_in_group(v); + } + } + + switch (tablet_schema.keys_type) { + case TKeysType::DUP_KEYS: + tablet_schema_pb->set_keys_type(KeysType::DUP_KEYS); + break; + case TKeysType::UNIQUE_KEYS: + tablet_schema_pb->set_keys_type(KeysType::UNIQUE_KEYS); + break; + case TKeysType::AGG_KEYS: + tablet_schema_pb->set_keys_type(KeysType::AGG_KEYS); + break; + default: + LOG(WARNING) << "unknown tablet keys type"; + break; + } + + // compress_kind used to compress segment files + tablet_schema_pb->set_compress_kind(COMPRESS_LZ4); + + // compression_type used to compress segment page + switch (compression_type) { + case TCompressionType::NO_COMPRESSION: + tablet_schema_pb->set_compression_type(segment_v2::NO_COMPRESSION); + break; + case TCompressionType::SNAPPY: + tablet_schema_pb->set_compression_type(segment_v2::SNAPPY); + break; + case TCompressionType::LZ4: + tablet_schema_pb->set_compression_type(segment_v2::LZ4); + break; + case TCompressionType::LZ4F: + tablet_schema_pb->set_compression_type(segment_v2::LZ4F); + break; + case TCompressionType::ZLIB: + tablet_schema_pb->set_compression_type(segment_v2::ZLIB); + break; + case TCompressionType::ZSTD: + tablet_schema_pb->set_compression_type(segment_v2::ZSTD); + break; + default: + tablet_schema_pb->set_compression_type(segment_v2::LZ4F); + break; + } + + switch (inverted_index_file_storage_format) { + case TInvertedIndexFileStorageFormat::V1: + tablet_schema_pb->set_inverted_index_storage_format(InvertedIndexStorageFormatPB::V1); + break; + case TInvertedIndexFileStorageFormat::V2: + tablet_schema_pb->set_inverted_index_storage_format(InvertedIndexStorageFormatPB::V2); + break; + case TInvertedIndexFileStorageFormat::V3: + tablet_schema_pb->set_inverted_index_storage_format(InvertedIndexStorageFormatPB::V3); + break; + default: + tablet_schema_pb->set_inverted_index_storage_format(InvertedIndexStorageFormatPB::V3); + break; + } + + switch (tablet_schema.sort_type) { + case TSortType::type::ZORDER: + tablet_schema_pb->set_sort_type(SortType::ZORDER); + break; + default: + tablet_schema_pb->set_sort_type(SortType::LEXICAL); + } + tablet_schema_pb->set_sort_col_num(tablet_schema.sort_col_num); + for (const auto& i : tablet_schema.cluster_key_uids) { + tablet_schema_pb->add_cluster_key_uids(i); + } + + // set column information + uint32_t col_ordinal = 0; + bool has_bf_columns = false; + for (TColumn tcolumn : tablet_schema.columns) { + ColumnPB* column = tablet_schema_pb->add_column(); + uint32_t unique_id = -1; + if (tcolumn.col_unique_id >= 0) { + unique_id = tcolumn.col_unique_id; + } else { + unique_id = col_ordinal_to_unique_id.at(col_ordinal); + } + col_ordinal++; + init_column_from_tcolumn(unique_id, tcolumn, column); + + if (column->is_bf_column()) { + has_bf_columns = true; + } + + if (tablet_schema.__isset.indexes) { + for (auto& index : tablet_schema.indexes) { + if (index.index_type == TIndexType::type::BLOOMFILTER || + index.index_type == TIndexType::type::NGRAM_BF) { + DCHECK_EQ(index.columns.size(), 1); + if (iequal(tcolumn.column_name, index.columns[0])) { + column->set_is_bf_column(true); + break; + } + } + } + } + } + + // copy index meta + if (tablet_schema.__isset.indexes) { + for (auto& index : tablet_schema.indexes) { + TabletIndexPB* index_pb = tablet_schema_pb->add_index(); + index_pb->set_index_id(index.index_id); + index_pb->set_index_name(index.index_name); + // init col_unique_id in index at be side, since col_unique_id may be -1 at fe side + // get column unique id by name + for (auto column_name : index.columns) { + for (auto column : tablet_schema_pb->column()) { + if (iequal(column.name(), column_name)) { + index_pb->add_col_unique_id(column.unique_id()); + } + } + } + switch (index.index_type) { + case TIndexType::BITMAP: + index_pb->set_index_type(IndexType::BITMAP); + break; + case TIndexType::INVERTED: + index_pb->set_index_type(IndexType::INVERTED); + break; + case TIndexType::ANN: + index_pb->set_index_type(IndexType::ANN); + break; + case TIndexType::BLOOMFILTER: + index_pb->set_index_type(IndexType::BLOOMFILTER); + break; + case TIndexType::NGRAM_BF: + index_pb->set_index_type(IndexType::NGRAM_BF); + break; + } + + if (index.__isset.properties) { + auto properties = index_pb->mutable_properties(); + for (auto kv : index.properties) { + (*properties)[kv.first] = kv.second; + } + } + } + } + + tablet_schema_pb->set_next_column_unique_id(next_unique_id); + if (has_bf_columns && tablet_schema.__isset.bloom_filter_fpp) { + tablet_schema_pb->set_bf_fpp(tablet_schema.bloom_filter_fpp); + } + + if (tablet_schema.__isset.is_in_memory) { + tablet_schema_pb->set_is_in_memory(tablet_schema.is_in_memory); + } + + if (tablet_schema.__isset.disable_auto_compaction) { + tablet_schema_pb->set_disable_auto_compaction(tablet_schema.disable_auto_compaction); + } + + // Deprecated legacy flatten-nested switch. Distinct from variant_enable_nested_group. + if (tablet_schema.__isset.variant_enable_flatten_nested) { + tablet_schema_pb->set_enable_variant_flatten_nested( + tablet_schema.variant_enable_flatten_nested); + } + + if (tablet_schema.__isset.enable_single_replica_compaction) { + tablet_schema_pb->set_enable_single_replica_compaction( + tablet_schema.enable_single_replica_compaction); + } + + if (tablet_schema.__isset.delete_sign_idx) { + tablet_schema_pb->set_delete_sign_idx(tablet_schema.delete_sign_idx); + } + if (tablet_schema.__isset.store_row_column) { + tablet_schema_pb->set_store_row_column(tablet_schema.store_row_column); + } + if (tablet_schema.__isset.row_store_page_size) { + tablet_schema_pb->set_row_store_page_size(tablet_schema.row_store_page_size); + } + if (tablet_schema.__isset.storage_page_size) { + tablet_schema_pb->set_storage_page_size(tablet_schema.storage_page_size); + } + if (tablet_schema.__isset.storage_dict_page_size) { + tablet_schema_pb->set_storage_dict_page_size(tablet_schema.storage_dict_page_size); + } + if (tablet_schema.__isset.skip_write_index_on_load) { + tablet_schema_pb->set_skip_write_index_on_load(tablet_schema.skip_write_index_on_load); + } + if (tablet_schema.__isset.row_store_col_cids) { + tablet_schema_pb->mutable_row_store_column_unique_ids()->Add( + tablet_schema.row_store_col_cids.begin(), tablet_schema.row_store_col_cids.end()); + } +} + void TabletMeta::remove_rowset_delete_bitmap(const RowsetId& rowset_id, const Version& version) { if (_enable_unique_key_merge_on_write) { delete_bitmap().remove({rowset_id, 0, 0}, {rowset_id, UINT32_MAX, 0}); @@ -798,9 +867,17 @@ void TabletMeta::init_from_pb(const TabletMetaPB& tablet_meta_pb) { _handle = pair.first; _schema = pair.second; + if (tablet_meta_pb.has_row_binlog_schema()) { + TabletSchemaSPtr row_binlog_schema = std::make_shared(); + row_binlog_schema->init_from_pb(tablet_meta_pb.row_binlog_schema()); + _row_binlog_schema = std::move(row_binlog_schema); + _row_binlog_schema_hash = tablet_meta_pb.row_binlog_schema_hash(); + } + if (tablet_meta_pb.has_enable_unique_key_merge_on_write()) { _enable_unique_key_merge_on_write = tablet_meta_pb.enable_unique_key_merge_on_write(); _delete_bitmap->set_tablet_id(_tablet_id); + _binlog_delvec->set_tablet_id(_tablet_id); } // init _rs_metas @@ -821,6 +898,12 @@ void TabletMeta::init_from_pb(const TabletMetaPB& tablet_meta_pb) { } } + for (auto& it : tablet_meta_pb.row_binlog_rs_metas()) { + RowsetMetaSharedPtr rs_meta(new RowsetMeta()); + rs_meta->init_from_pb(it); + _row_binlog_rs_metas.emplace(rs_meta->version(), rs_meta); + } + if (tablet_meta_pb.has_in_restore_mode()) { _in_restore_mode = tablet_meta_pb.in_restore_mode(); } @@ -839,15 +922,26 @@ void TabletMeta::init_from_pb(const TabletMetaPB& tablet_meta_pb) { int seg_ids_size = tablet_meta_pb.delete_bitmap().segment_ids_size(); int versions_size = tablet_meta_pb.delete_bitmap().versions_size(); int seg_maps_size = tablet_meta_pb.delete_bitmap().segment_delete_bitmaps_size(); + int binlog_mark_size = tablet_meta_pb.delete_bitmap().is_binlog_delvec_size(); CHECK(rst_ids_size == seg_ids_size && seg_ids_size == seg_maps_size && seg_maps_size == versions_size); + CHECK(binlog_mark_size == 0 || binlog_mark_size == rst_ids_size); for (int i = 0; i < rst_ids_size; ++i) { RowsetId rst_id; rst_id.init(tablet_meta_pb.delete_bitmap().rowset_ids(i)); auto seg_id = tablet_meta_pb.delete_bitmap().segment_ids(i); auto ver = tablet_meta_pb.delete_bitmap().versions(i); auto bitmap = tablet_meta_pb.delete_bitmap().segment_delete_bitmaps(i).data(); - delete_bitmap().delete_bitmap[{rst_id, seg_id, ver}] = roaring::Roaring::read(bitmap); + bool from_binlog = tablet_meta_pb.delete_bitmap().is_binlog_delvec_size() > 0 + ? tablet_meta_pb.delete_bitmap().is_binlog_delvec(i) + : false; + if (!from_binlog) { + delete_bitmap().delete_bitmap[{rst_id, seg_id, ver}] = + roaring::Roaring::read(bitmap); + } else { + binlog_delvec().delete_bitmap[{rst_id, seg_id, ver}] = + roaring::Roaring::read(bitmap); + } } } @@ -871,6 +965,10 @@ void TabletMeta::init_from_pb(const TabletMetaPB& tablet_meta_pb) { if (tablet_meta_pb.has_encryption_algorithm()) { _encryption_algorithm = tablet_meta_pb.encryption_algorithm(); } + + if (tablet_meta_pb.has_row_binlog_schema_hash()) { + _row_binlog_schema_hash = tablet_meta_pb.row_binlog_schema_hash(); + } } void TabletMeta::to_meta_pb(TabletMetaPB* tablet_meta_pb, bool cloud_get_rowset_meta) { @@ -912,10 +1010,18 @@ void TabletMeta::to_meta_pb(TabletMetaPB* tablet_meta_pb, bool cloud_get_rowset_ for (const auto& [_, rs] : _stale_rs_metas) { rs->to_rowset_pb(tablet_meta_pb->add_stale_rs_metas()); } + for (const auto& [_, rs] : _row_binlog_rs_metas) { + rs->to_rowset_pb(tablet_meta_pb->add_row_binlog_rs_metas()); + } } _schema->to_schema_pb(tablet_meta_pb->mutable_schema()); + if (_row_binlog_schema != nullptr) { + _row_binlog_schema->to_schema_pb(tablet_meta_pb->mutable_row_binlog_schema()); + tablet_meta_pb->set_row_binlog_schema_hash(_row_binlog_schema_hash); + } + tablet_meta_pb->set_in_restore_mode(in_restore_mode()); // to avoid modify tablet meta to the greatest extend @@ -945,6 +1051,18 @@ void TabletMeta::to_meta_pb(TabletMetaPB* tablet_meta_pb, bool cloud_get_rowset_ delete_bitmap_pb->add_rowset_ids(rowset_id.to_string()); delete_bitmap_pb->add_segment_ids(segment_id); delete_bitmap_pb->add_versions(ver); + delete_bitmap_pb->add_is_binlog_delvec(false); + std::string bitmap_data(bitmap.getSizeInBytes(), '\0'); + bitmap.write(bitmap_data.data()); + *(delete_bitmap_pb->add_segment_delete_bitmaps()) = std::move(bitmap_data); + } + + for (auto& [id, bitmap] : binlog_delvec().snapshot().delete_bitmap) { + auto& [rowset_id, segment_id, ver] = id; + delete_bitmap_pb->add_rowset_ids(rowset_id.to_string()); + delete_bitmap_pb->add_segment_ids(segment_id); + delete_bitmap_pb->add_versions(ver); + delete_bitmap_pb->add_is_binlog_delvec(true); std::string bitmap_data(bitmap.getSizeInBytes(), '\0'); bitmap.write(bitmap_data.data()); *(delete_bitmap_pb->add_segment_delete_bitmaps()) = std::move(bitmap_data); @@ -1012,6 +1130,24 @@ Status TabletMeta::add_rs_meta(const RowsetMetaSharedPtr& rs_meta) { return Status::OK(); } +Status TabletMeta::add_row_binlog_rs_meta(const RowsetMetaSharedPtr& row_binlog_meta) { + // check RowsetMeta is valid + for (auto& [_, rs] : _row_binlog_rs_metas) { + if (rs->version() == row_binlog_meta->version()) { + if (rs->rowset_id() != row_binlog_meta->rowset_id()) { + return Status::Error( + "binlog version already exist. binlog_rowset_id={}, version={}, tablet={}", + rs->rowset_id().to_string(), rs->version().to_string(), tablet_id()); + } else { + // rowsetid,version is equal, it is a duplicate req, skip it + return Status::OK(); + } + } + } + _row_binlog_rs_metas.emplace(row_binlog_meta->version(), row_binlog_meta); + return Status::OK(); +} + void TabletMeta::add_rowsets_unchecked(const std::vector& to_add) { for (const auto& rs : to_add) { _rs_metas.emplace(rs->rowset_meta()->version(), rs->rowset_meta()); diff --git a/be/src/storage/tablet/tablet_meta.h b/be/src/storage/tablet/tablet_meta.h index 0e52bd8429bb0b..e9c46ef007e554 100644 --- a/be/src/storage/tablet/tablet_meta.h +++ b/be/src/storage/tablet/tablet_meta.h @@ -118,7 +118,8 @@ class TabletMeta : public MetadataAdder { TInvertedIndexFileStorageFormat::V2, TEncryptionAlgorithm::type tde_algorithm = TEncryptionAlgorithm::PLAINTEXT, TStorageFormat::type storage_format = TStorageFormat::V2, - int32_t vertical_compaction_num_columns_per_group = 5); + int32_t vertical_compaction_num_columns_per_group = 5, + const TTabletSchema* row_binlog_schema = nullptr); // If need add a filed in TableMeta, filed init copy in copy construct function TabletMeta(const TabletMeta& tablet_meta); TabletMeta(TabletMeta&& tablet_meta) = delete; @@ -173,6 +174,7 @@ class TabletMeta : public MetadataAdder { size_t tablet_local_size() const; // Remote disk space occupied by tablet. size_t tablet_remote_size() const; + size_t binlog_size() const; size_t tablet_local_index_size() const; size_t tablet_local_segment_size() const; @@ -181,6 +183,7 @@ class TabletMeta : public MetadataAdder { size_t version_count() const; size_t stale_version_count() const; + size_t binlog_file_num() const; size_t version_count_cross_with_range(const Version& range) const; Version max_version() const; @@ -245,8 +248,23 @@ class TabletMeta : public MetadataAdder { static void init_column_from_tcolumn(uint32_t unique_id, const TColumn& tcolumn, ColumnPB* column); + struct SchemaCreateOptions { + const std::unordered_map& col_ordinal_to_unique_id; + TCompressionType::type compression_type; + TInvertedIndexFileStorageFormat::type inverted_index_file_storage_format; + uint32_t next_unique_id; + }; + + static void init_schema_from_thrift(const TTabletSchema& tablet_schema, + const SchemaCreateOptions& schema_create_options, + TabletSchemaPB* tablet_schema_pb); + DeleteBitmapPtr delete_bitmap_ptr() { return _delete_bitmap; } DeleteBitmap& delete_bitmap() { return *_delete_bitmap; } + + DeleteBitmapPtr binlog_delvec_ptr() { return _binlog_delvec; } + DeleteBitmap& binlog_delvec() { return *_binlog_delvec; } + void remove_rowset_delete_bitmap(const RowsetId& rowset_id, const Version& version); bool enable_unique_key_merge_on_write() const { return _enable_unique_key_merge_on_write; } @@ -261,6 +279,11 @@ class TabletMeta : public MetadataAdder { _binlog_config = std::move(binlog_config); } + const TabletSchemaSPtr& row_binlog_schema() const { return _row_binlog_schema; } + int32_t row_binlog_schema_hash() const { return _row_binlog_schema_hash; } + const RowsetMetaMapContainer& all_row_binlog_rs_metas() const; + Status add_row_binlog_rs_meta(const RowsetMetaSharedPtr& rs_meta); + void set_compaction_policy(std::string compaction_policy) { _compaction_policy = compaction_policy; } @@ -362,6 +385,11 @@ class TabletMeta : public MetadataAdder { // query performance significantly. bool _enable_unique_key_merge_on_write = false; std::shared_ptr _delete_bitmap; + std::shared_ptr _binlog_delvec; + + int32_t _row_binlog_schema_hash = 0; + TabletSchemaSPtr _row_binlog_schema; + RowsetMetaMapContainer _row_binlog_rs_metas; // binlog config BinlogConfig _binlog_config {}; @@ -716,6 +744,8 @@ inline size_t TabletMeta::tablet_local_size() const { total_size += rs->total_disk_size(); } } + // if we need to split data and binlog or not + total_size += binlog_size(); return total_size; } @@ -729,6 +759,16 @@ inline size_t TabletMeta::tablet_remote_size() const { return total_size; } +inline size_t TabletMeta::binlog_size() const { + size_t total_size = 0; + for (auto& [_, rs] : _row_binlog_rs_metas) { + if (rs->is_local()) { + total_size += rs->data_disk_size(); + } + } + return total_size; +} + inline size_t TabletMeta::tablet_local_index_size() const { size_t total_size = 0; for (const auto& [_, rs] : _rs_metas) { @@ -777,6 +817,10 @@ inline size_t TabletMeta::stale_version_count() const { return _rs_metas.size(); } +inline size_t TabletMeta::binlog_file_num() const { + return _row_binlog_rs_metas.size(); +} + inline TabletState TabletMeta::tablet_state() const { return _tablet_state; } @@ -813,6 +857,10 @@ inline const RowsetMetaMapContainer& TabletMeta::all_stale_rs_metas() const { return _stale_rs_metas; } +inline const RowsetMetaMapContainer& TabletMeta::all_row_binlog_rs_metas() const { + return _row_binlog_rs_metas; +} + inline bool TabletMeta::all_beta() const { for (const auto& [_, rs] : _rs_metas) { if (rs->rowset_type() != RowsetTypePB::BETA_ROWSET) { @@ -824,6 +872,11 @@ inline bool TabletMeta::all_beta() const { return false; } } + for (const auto& [_, rs] : _row_binlog_rs_metas) { + if (rs->rowset_type() != RowsetTypePB::BETA_ROWSET) { + return false; + } + } return true; } diff --git a/be/src/storage/txn/txn_manager.cpp b/be/src/storage/txn/txn_manager.cpp index f42dbcf5de7609..d70141d24d55b7 100644 --- a/be/src/storage/txn/txn_manager.cpp +++ b/be/src/storage/txn/txn_manager.cpp @@ -565,7 +565,7 @@ Status TxnManager::publish_txn(OlapMeta* meta, TPartitionId partition_id, } /// Step 3: add to binlog - auto enable_binlog = tablet->is_enable_binlog(); + auto enable_binlog = tablet->enable_ccr_binlog(); if (enable_binlog) { auto status = rowset->add_to_binlog(); if (!status.ok()) { diff --git a/be/test/olap/rowset/group_rowset_builder_test.cpp b/be/test/olap/rowset/group_rowset_builder_test.cpp new file mode 100644 index 00000000000000..a55278782b6def --- /dev/null +++ b/be/test/olap/rowset/group_rowset_builder_test.cpp @@ -0,0 +1,219 @@ +// 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 +#include +#include +#include +#include +#include + +#include +#include +#include + +#include "common/config.h" +#include "io/fs/local_file_system.h" +#include "load/memtable/memtable_memory_limiter.h" +#include "runtime/descriptor_helper.h" +#include "runtime/descriptors.h" +#include "runtime/exec_env.h" +#include "storage/data_dir.h" +#include "storage/rowset_builder.h" +#include "storage/storage_engine.h" +#include "storage/tablet/tablet.h" +#include "storage/tablet/tablet_manager.h" +#include "storage/tablet_info.h" + +namespace doris { + +static const uint32_t MAX_PATH_LEN = 1024; +static StorageEngine* engine_ref = nullptr; + +static void set_up() { + char buffer[MAX_PATH_LEN]; + EXPECT_NE(getcwd(buffer, MAX_PATH_LEN), nullptr); + config::storage_root_path = std::string(buffer) + "/data_test"; + auto st = io::global_local_filesystem()->delete_directory(config::storage_root_path); + ASSERT_TRUE(st.ok()) << st; + st = io::global_local_filesystem()->create_directory(config::storage_root_path); + ASSERT_TRUE(st.ok()) << st; + std::vector paths; + paths.emplace_back(config::storage_root_path, -1); + + EngineOptions options; + options.store_paths = paths; + auto engine = std::make_unique(options); + engine_ref = engine.get(); + Status s = engine->open(); + ASSERT_TRUE(s.ok()) << s; + + ExecEnv* exec_env = doris::ExecEnv::GetInstance(); + exec_env->set_memtable_memory_limiter(new MemTableMemoryLimiter()); + exec_env->set_storage_engine(std::move(engine)); +} + +static void tear_down() { + ExecEnv* exec_env = doris::ExecEnv::GetInstance(); + exec_env->set_memtable_memory_limiter(nullptr); + engine_ref = nullptr; + exec_env->set_storage_engine(nullptr); + EXPECT_EQ(system("rm -rf ./data_test"), 0); + static_cast(io::global_local_filesystem()->delete_directory( + std::string(getenv("DORIS_HOME")) + "/" + UNUSED_PREFIX)); +} + +static void create_tablet_request(int64_t tablet_id, int32_t schema_hash, + TCreateTabletReq* request) { + request->tablet_id = tablet_id; + request->__set_version(1); + request->partition_id = 10001; + request->tablet_schema.schema_hash = schema_hash; + request->tablet_schema.short_key_column_count = 1; + request->tablet_schema.keys_type = TKeysType::AGG_KEYS; + request->tablet_schema.storage_type = TStorageType::COLUMN; + request->__set_storage_format(TStorageFormat::V2); + + TColumn k1; + k1.column_name = "k1"; + k1.__set_is_key(true); + k1.column_type.type = TPrimitiveType::INT; + request->tablet_schema.columns.push_back(k1); + + TColumn v1; + v1.column_name = "v1"; + v1.__set_is_key(false); + v1.column_type.type = TPrimitiveType::INT; + v1.__set_aggregation_type(TAggregationType::SUM); + request->tablet_schema.columns.push_back(v1); +} + +static void create_tablet_request_with_row_binlog(int64_t tablet_id, int32_t schema_hash, + TCreateTabletReq* request) { + create_tablet_request(tablet_id, schema_hash, request); + TBinlogConfig binlog_config; + binlog_config.__set_enable(true); + binlog_config.__set_binlog_format(TBinlogFormat::ROW); + request->__set_binlog_config(binlog_config); + TTabletSchema row_binlog_schema = request->tablet_schema; + row_binlog_schema.schema_hash = schema_hash + 1; + request->__set_row_binlog_schema(row_binlog_schema); +} + +static TDescriptorTable create_descriptor_tablet() { + TDescriptorTableBuilder dtb; + TTupleDescriptorBuilder tuple_builder; + + tuple_builder.add_slot(TSlotDescriptorBuilder() + .type(TYPE_INT) + .column_name("k1") + .column_pos(0) + .nullable(false) + .build()); + tuple_builder.add_slot(TSlotDescriptorBuilder() + .type(TYPE_INT) + .column_name("v1") + .column_pos(1) + .nullable(false) + .build()); + tuple_builder.build(&dtb); + + return dtb.desc_tbl(); +} + +static std::shared_ptr create_table_schema_param( + const TDescriptorTable& tdesc_tbl, int64_t index_id, int32_t schema_hash, + const std::vector& columns) { + auto param = std::make_shared(); + TOlapTableSchemaParam tschema; + tschema.db_id = 1; + tschema.table_id = 2; + tschema.version = 0; + tschema.slot_descs = tdesc_tbl.slotDescriptors; + tschema.tuple_desc = tdesc_tbl.tupleDescriptors[0]; + tschema.indexes.resize(1); + tschema.indexes[0].id = index_id; + tschema.indexes[0].schema_hash = schema_hash; + tschema.indexes[0].columns_desc = columns; + for (const auto& col : columns) { + tschema.indexes[0].columns.push_back(col.column_name); + } + Status st = param->init(tschema); + EXPECT_TRUE(st.ok()) << st; + return param; +} + +class GroupRowsetBuilderTest : public ::testing::Test { +public: + static void SetUpTestSuite() { set_up(); } + static void TearDownTestSuite() { tear_down(); } +}; + +TEST_F(GroupRowsetBuilderTest, buildWithRowBinlogMeta) { + std::unique_ptr profile = std::make_unique("CreateTablet"); + TCreateTabletReq request; + create_tablet_request_with_row_binlog(10010, 270068390, &request); + Status res = engine_ref->create_tablet(request, profile.get()); + ASSERT_TRUE(res.ok()); + + TabletSharedPtr tablet = engine_ref->tablet_manager()->get_tablet(request.tablet_id); + ASSERT_TRUE(tablet != nullptr); + auto st = io::global_local_filesystem()->create_directory(tablet->row_binlog_path()); + ASSERT_TRUE(st.ok()) << st; + + PUniqueId load_id; + load_id.set_hi(0); + load_id.set_lo(0); + const int64_t index_id = 10001; + + TDescriptorTable tdesc_tbl = create_descriptor_tablet(); + auto param = create_table_schema_param(tdesc_tbl, index_id, request.tablet_schema.schema_hash, + request.tablet_schema.columns); + + WriteRequest data_req; + data_req.tablet_id = request.tablet_id; + data_req.schema_hash = request.tablet_schema.schema_hash; + data_req.txn_id = 20010; + data_req.partition_id = request.partition_id; + data_req.index_id = index_id; + data_req.load_id = load_id; + data_req.table_schema_param = param; + data_req.write_req_type = WriteRequestType::DATA; + + WriteRequest row_binlog_req = data_req; + row_binlog_req.schema_hash = request.row_binlog_schema.schema_hash; + row_binlog_req.write_req_type = WriteRequestType::ROW_BINLOG; + + GroupRowsetBuilder builder(*engine_ref, data_req, row_binlog_req, profile.get()); + ASSERT_TRUE(builder.init().ok()); + ASSERT_TRUE(builder.rowset_writer()->flush().ok()); + ASSERT_TRUE(builder.build_rowset().ok()); + + auto row_binlog_meta = builder.row_binlog_builder()->rowset()->rowset_meta(); + auto data_meta = builder.txn_rowset_builder()->rowset()->rowset_meta(); + ASSERT_TRUE(row_binlog_meta->is_row_binlog()); + ASSERT_FALSE(data_meta->is_row_binlog()); + ASSERT_EQ(request.row_binlog_schema.schema_hash, row_binlog_meta->tablet_schema_hash()); + ASSERT_EQ(request.tablet_schema.schema_hash, data_meta->tablet_schema_hash()); + ASSERT_EQ(index_id, row_binlog_meta->index_id()); + ASSERT_EQ(index_id, data_meta->index_id()); + + res = engine_ref->tablet_manager()->drop_tablet(request.tablet_id, request.replica_id, false); + ASSERT_TRUE(res.ok()); +} + +} // namespace doris diff --git a/fe/fe-catalog/src/main/java/org/apache/doris/catalog/Column.java b/fe/fe-catalog/src/main/java/org/apache/doris/catalog/Column.java index eb1789f52535dc..f9da5424a2bcce 100644 --- a/fe/fe-catalog/src/main/java/org/apache/doris/catalog/Column.java +++ b/fe/fe-catalog/src/main/java/org/apache/doris/catalog/Column.java @@ -72,6 +72,45 @@ public class Column implements GsonPostProcessable { private static final String COLUMN_MAP_KEY = "key"; private static final String COLUMN_MAP_VALUE = "value"; + // columns for binlog schema + // explicit columns + public static final String BINLOG_LSN_COL = "__DORIS_BINLOG_LSN__"; + public static final String BINLOG_TIMESTAMP_COL = "__DORIS_BINLOG_TIMESTAMP__"; + // implicit columns + public static final String BINLOG_OPERATION_COL = "__DORIS_BINLOG_OP__"; + public static final String BINLOG_BEFORE_PREFIX = "__BEFORE__"; + + public static String generateBeforeColName(String colName) { + return BINLOG_BEFORE_PREFIX + colName + "__"; + } + + public static Column generateRowBinlogKeyColumn(Column column) { + Column keyColumn = new Column(column); + keyColumn.setComment("key (" + column.getName() + ")"); + return keyColumn; + } + + public static Column generateAfterValueColumn(Column column) { + Column afterValueColumn = new Column(column); + afterValueColumn.setComment("after value (" + column.getName() + ")"); + afterValueColumn.setAggregationType(AggregateType.NONE, true); + return afterValueColumn; + } + + public static Column generateBeforeValueColumn(Column column) { + Column beforeValueColumn = new Column(column); + beforeValueColumn.setName(Column.generateBeforeColName(column.getName())); + beforeValueColumn.setComment("before value (" + column.getName() + ")"); + beforeValueColumn.setIsVisible(true); + beforeValueColumn.setAggregationType(AggregateType.NONE, true); + beforeValueColumn.setIsAllowNull(true); + // clear default value + beforeValueColumn.defaultValue = null; + beforeValueColumn.defaultValueExprDef = null; + beforeValueColumn.realDefaultValue = null; + return beforeValueColumn; + } + @SerializedName(value = "name") private String name; @SerializedName(value = "type") diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java b/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java index 3a2b95f6a3f17c..1825170b3200a8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java @@ -212,6 +212,11 @@ private boolean processAlterOlapTableInternal(List alterOps, OlapTable currentAlterOps.checkMTMVAllow(alterOps); } + // For row binlog tables, only allow operations explicitly marked as safe. + if (olapTable.needRowBinlog()) { + currentAlterOps.checkRowBinlogAllow(alterOps); + } + // check cluster capacity and db quota, only need to check once. if (currentAlterOps.needCheckCapacity()) { Env.getCurrentInternalCatalog().checkAvailableCapacity(db); diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/AlterOperations.java b/fe/fe-core/src/main/java/org/apache/doris/alter/AlterOperations.java index 371d04c464c7d6..c07aa3b1b794ed 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/AlterOperations.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/AlterOperations.java @@ -56,6 +56,15 @@ public void checkMTMVAllow(List alterOps) throws DdlException { } } + // Check whether current alter operations are allowed on tables with row binlog enabled. + public void checkRowBinlogAllow(List alterOps) throws DdlException { + for (AlterOp alterOp : alterOps) { + if (!alterOp.allowOpRowBinlog()) { + throw new DdlException("Not allowed to perform current operation on Table With Row Binlog"); + } + } + } + // some operations take up disk space. so we need to check the disk capacity before processing. // return true if we see these kind of operations. public boolean needCheckCapacity() { diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java b/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java index 27a1fdcc8df136..5b32e7578a51e5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java @@ -273,7 +273,7 @@ protected void createRollupReplica() throws AlterCancelException { tbl.variantEnableFlattenNested(), tbl.storagePageSize(), tbl.getTDEAlgorithm(), tbl.storageDictPageSize(), null, - tbl.getVerticalCompactionNumColumnsPerGroup()); + tbl.getVerticalCompactionNumColumnsPerGroup(), null); createReplicaTask.setBaseTablet(tabletIdMap.get(rollupTabletId), baseSchemaHash); if (this.storageFormat != null) { createReplicaTask.setStorageFormat(this.storageFormat); diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java index 7f8bb6f8a85a75..1142b1645a978b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java @@ -200,8 +200,32 @@ private boolean processAddColumn(AddColumnOp addColumnOp, OlapTable olapTable, Set newColNameSet = Sets.newHashSet(column.getName()); - return addColumnInternal(olapTable, column, columnPos, targetIndexId, baseIndexId, indexSchemaMap, - newColNameSet, false, colUniqueIdSupplierMap); + boolean lightSchemaChange = addColumnInternal(olapTable, column, columnPos, targetIndexId, baseIndexId, + indexSchemaMap, newColNameSet, false, colUniqueIdSupplierMap); + + // add column to binlog schema + long rowBinlogIndexId = olapTable.getBaseIndexMeta().getRowBinlogIndexId(); + if (rowBinlogIndexId > 0) { + if (column.getType().isVariantType()) { + throw new DdlException( + "table with binlog does not support VARIANT column: " + column.getName()); + } + if (!lightSchemaChange) { + throw new DdlException("table with binlog only support light schema change," + "add column: " + + column); + } + Preconditions.checkState(indexSchemaMap.containsKey(rowBinlogIndexId)); + + LinkedList rowBinlogSchema = indexSchemaMap.get(rowBinlogIndexId); + boolean needHistoricalValue = olapTable.getBinlogConfig().getNeedHistoricalValue(); + if (needHistoricalValue && !column.isKey()) { + newColNameSet.add(Column.generateBeforeColName(column.getName())); + } + addColumnRowBinlog(rowBinlogSchema, column, columnPos, newColNameSet, needHistoricalValue, + colUniqueIdSupplierMap.get(rowBinlogIndexId)); + } + + return lightSchemaChange; } private void processAddColumn(AddColumnOp addColumnOp, Table externalTable, List newSchema) @@ -247,6 +271,16 @@ public boolean processAddColumns(AddColumnsOp addColumnsOp, OlapTable olapTable, newColNameSet.add(column.getName()); } + long rowBinlogIndexId = olapTable.getBaseIndexMeta().getRowBinlogIndexId(); + boolean needHistoricalValue = rowBinlogIndexId > 0 && olapTable.getBinlogConfig().getNeedHistoricalValue(); + if (needHistoricalValue) { + for (Column column : columns) { + if (!column.isKey()) { + newColNameSet.add(Column.generateBeforeColName(column.getName())); + } + } + } + String baseIndexName = olapTable.getName(); checkAssignedTargetIndexName(baseIndexName, targetIndexName); @@ -263,10 +297,107 @@ public boolean processAddColumns(AddColumnsOp addColumnsOp, OlapTable olapTable, if (!result) { lightSchemaChange = false; } + + // add column to binlog schema + if (rowBinlogIndexId > 0) { + if (column.getType().isVariantType()) { + throw new DdlException( + "table with binlog does not support VARIANT column: " + column.getName()); + } + if (!lightSchemaChange) { + throw new DdlException("table with binlog only support light schema change," + "add column: " + + column); + } + Preconditions.checkState(indexSchemaMap.containsKey(rowBinlogIndexId)); + + LinkedList rowBinlogSchema = indexSchemaMap.get(rowBinlogIndexId); + addColumnRowBinlog(rowBinlogSchema, column, null, newColNameSet, needHistoricalValue, + colUniqueIdSupplierMap.get(rowBinlogIndexId)); + } } return lightSchemaChange; } + private void addColumnRowBinlog(List rowBinlogSchema, Column newColumn, ColumnPosition columnPos, + Set newColNameSet, boolean needHistoricalValue, + IntSupplier columnUniqueIdSupplier) throws DdlException { + if (newColumn.isAutoInc() || newColumn.getDataType().isVariantType()) { + throw new DdlException("can't add AutoInc/Variant column " + " on table with binlog, column: " + + newColumn.getDataType()); + } + + if (newColumn.isKey()) { + // key (don't support now) + Column keyBinlogColumn = Column.generateRowBinlogKeyColumn(newColumn); + ColumnPosition keyBinlogColumnPos = + convertToRowBinlogPosition(rowBinlogSchema, columnPos, true, false); + checkAndAddColumn(rowBinlogSchema, keyBinlogColumn, keyBinlogColumnPos, newColNameSet, false, + columnUniqueIdSupplier.getAsInt()); + } else { + // after value + Column afterBinlogColumn = Column.generateAfterValueColumn(newColumn); + ColumnPosition afterBinlogColumnPos = + convertToRowBinlogPosition(rowBinlogSchema, columnPos, false, false); + checkAndAddColumn(rowBinlogSchema, afterBinlogColumn, afterBinlogColumnPos, newColNameSet, false, + columnUniqueIdSupplier.getAsInt()); + + // before value: only exist when table needs historical value. + if (needHistoricalValue) { + Column beforeBinlogColumn = Column.generateBeforeValueColumn(newColumn); + ColumnPosition beforeBinlogColumnPos = + convertToRowBinlogPosition(rowBinlogSchema, columnPos, false, true); + checkAndAddColumn(rowBinlogSchema, beforeBinlogColumn, beforeBinlogColumnPos, newColNameSet, false, + columnUniqueIdSupplier.getAsInt()); + } + } + } + + private ColumnPosition convertToRowBinlogPosition(List rowBinlogSchema, ColumnPosition columnPosition, + boolean isKey, boolean before) { + String lastKeyCol = ""; + String lastValueCol = ""; + String lastBeforeValueCol = ""; + for (Column column : rowBinlogSchema) { + String columnName = column.getName(); + if (column.isKey()) { + lastKeyCol = columnName; + } else { + if (columnName.contains(Column.BINLOG_BEFORE_PREFIX)) { + lastBeforeValueCol = columnName; + } else if (columnName.equals(Column.BINLOG_LSN_COL) || columnName.equals(Column.BINLOG_OPERATION_COL) + || columnName.equals(Column.BINLOG_TIMESTAMP_COL)) { + continue; + } else { + lastValueCol = columnName; + } + } + } + if (Strings.isNullOrEmpty(lastValueCol)) { + lastValueCol = lastKeyCol; + } + if (Strings.isNullOrEmpty(lastBeforeValueCol)) { + lastBeforeValueCol = lastValueCol; + } + if (columnPosition == null) { + // add to last + if (isKey) { + return new ColumnPosition(lastKeyCol); + } else if (!before) { + return new ColumnPosition(lastValueCol); + } else { + return new ColumnPosition(lastBeforeValueCol); + } + } + if (columnPosition == ColumnPosition.FIRST) { + return ColumnPosition.FIRST; + } + String lastCol = columnPosition.getLastCol(); + if (lastCol.equals(lastKeyCol)) { + return new ColumnPosition(before ? lastValueCol : lastKeyCol); + } + return new ColumnPosition(before ? Column.generateBeforeColName(lastCol) : lastCol); + } + private void processAddSequenceMapping(Map> sequenceMapping, OlapTable olapTable, Map properties, List columns) throws DdlException { // not sequence mapping table @@ -660,9 +791,52 @@ private boolean processDropColumn(DropColumnOp dropColumnOp, OlapTable olapTable throw new DdlException("Column does not exists: " + dropColName); } } + + // drop column to binlog schema + long rowBinlogIndexId = olapTable.getBaseIndexMeta().getRowBinlogIndexId(); + if (rowBinlogIndexId > 0) { + if (!lightSchemaChange) { + throw new DdlException("table with binlog only support light schema change," + + "drop column: " + dropColName); + } + Preconditions.checkState(indexSchemaMap.containsKey(rowBinlogIndexId)); + + LinkedList rowBinlogSchema = indexSchemaMap.get(rowBinlogIndexId); + dropColumnRowBinlog(rowBinlogSchema, dropColumnOp); + } return lightSchemaChange; } + private void dropColumnRowBinlog(List rowBinlogSchema, DropColumnOp dropColumnOp) throws DdlException { + String dropColName = dropColumnOp.getColName(); + Iterator rowBinlogIter = rowBinlogSchema.iterator(); + boolean foundKey = false; + boolean foundAfter = false; + boolean foundBefore = false; + while (rowBinlogIter.hasNext()) { + Column column = rowBinlogIter.next(); + if (column.getName().equalsIgnoreCase(dropColName)) { + rowBinlogIter.remove(); + if (column.isKey()) { + foundKey = true; + // key column only exists once + continue; + } else { + // value(after) column + foundAfter = true; + } + } + if (column.getName().equalsIgnoreCase(Column.generateBeforeColName(dropColName))) { + rowBinlogIter.remove(); + foundBefore = true; + continue; + } + } + if (!foundKey && !foundAfter && !foundBefore) { + throw new DdlException("Column does not exists in binlog: " + dropColName); + } + } + private void processDropSequenceMapping(Map> sequenceMapping, String colName) throws DdlException { colName = colName.toLowerCase(); @@ -758,6 +932,12 @@ private boolean processModifyColumn(ModifyColumnOp modifyColumnOp, OlapTable ola + "please use light_schema_change = true."); } + long rowBinlogIndexId = olapTable.getBaseIndexMeta().getRowBinlogIndexId(); + if (rowBinlogIndexId > 0) { + throw new DdlException("table with binlog don't support modify column," + + "modify column: " + modColumn); + } + if (KeysType.AGG_KEYS == olapTable.getKeysType()) { if (modColumn.isKey() && null != modColumn.getAggregationType()) { throw new DdlException("Can not assign aggregation method on key column: " + modColumn.getName()); @@ -2139,7 +2319,7 @@ public void process(String rawSql, List alterOps, Database db, //for multi add columns clauses //index id -> index col_unique_id supplier Map colUniqueIdSupplierMap = new HashMap<>(); - for (Map.Entry> entry : olapTable.getIndexIdToSchema(true).entrySet()) { + for (Map.Entry> entry : olapTable.getIndexIdToSchemaWithRowBinlog(true).entrySet()) { indexSchemaMap.put(entry.getKey(), new LinkedList<>(entry.getValue())); IntSupplier colUniqueIdSupplier = null; @@ -2466,6 +2646,10 @@ public int getAsInt() { lightIndexChange, buildIndexChange, indexSchemaMap); } + if (olapTable.needRowBinlog() && !(lightSchemaChange || lightIndexChange)) { + throw new DdlException("only support light schema change operator when use table with binlog"); + } + if (lightSchemaChange) { long jobId = Env.getCurrentEnv().getNextId(); //for schema change add/drop value column optimize, direct modify table meta. @@ -3286,7 +3470,7 @@ public void modifyTableLightSchemaChange(String rawSql, Database db, OlapTable o } //update base index schema - Map> oldIndexSchemaMap = olapTable.getCopiedIndexIdToSchema(true); + Map> oldIndexSchemaMap = olapTable.getCopiedIndexIdToSchemaWithRowBinlog(true); try { updateBaseIndexSchema(olapTable, indexSchemaMap, indexes); } catch (Exception e) { @@ -3487,6 +3671,10 @@ public void updateBaseIndexSchema(OlapTable olapTable, Map indexIds = new ArrayList(); indexIds.add(baseIndexId); indexIds.addAll(olapTable.getIndexIdListExceptBaseIndex()); + long rowBinlogIndexId = olapTable.getBaseIndexMeta().getRowBinlogIndexId(); + if (rowBinlogIndexId > 0 && indexSchemaMap.containsKey(rowBinlogIndexId)) { + indexIds.add(rowBinlogIndexId); + } for (int i = 0; i < indexIds.size(); i++) { List indexSchema = indexSchemaMap.get(indexIds.get(i)); MaterializedIndexMeta currentIndexMeta = olapTable.getIndexMetaByIndexId(indexIds.get(i)); @@ -3708,33 +3896,17 @@ public boolean updateBinlogConfig(Database db, OlapTable olapTable, List binlogConfigMap = PropertyAnalyzer.analyzeBinlogConfig(Maps.newHashMap(properties)); + if (binlogConfigMap != null) { + Pair mergePropertiesStatus = + newBinlogConfig.mergeFromProperties(binlogConfigMap, false); + if (!mergePropertiesStatus.first) { + throw new AnalysisException(mergePropertiesStatus.second); + } } + } catch (AnalysisException e) { + throw new DdlException(e.getMessage()); } } @@ -3753,8 +3925,8 @@ public boolean updateBinlogConfig(Database db, OlapTable olapTable, List enabled: " + olapTbl.getName()); + } if (!Config.ignore_backup_tmp_partitions && olapTbl.existTempPartitions()) { ErrorReport.reportDdlException(ErrorCode.ERR_COMMON_ERROR, "Do not support backup table " + olapTbl.getName() + " with temp partitions"); diff --git a/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java b/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java index 29ae36d53765fc..9a5fd311ce6b89 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java @@ -715,6 +715,12 @@ private void checkAndPrepareMeta() { if (localTbl != null) { OlapTable localOlapTbl = (OlapTable) localTbl; + if (localOlapTbl.needRowBinlog()) { + status = new Status(ErrCode.COMMON_ERROR, + "Do not support restore into local table with binlog enabled: " + + localOlapTbl.getName()); + return; + } OlapTable remoteOlapTbl = (OlapTable) remoteTbl; if (localOlapTbl.isColocateTable() || (reserveColocate && remoteOlapTbl.isColocateTable())) { @@ -1428,6 +1434,10 @@ protected void createReplicas(Database db, OlapTable localTbl, Partition restore Env.getCurrentInvertedIndex().addTablet(restoreTablet.getId(), tabletMeta); for (Replica restoreReplica : restoreTablet.getReplicas()) { Env.getCurrentInvertedIndex().addReplica(restoreTablet.getId(), restoreReplica); + MaterializedIndexMeta rowBinlogIndexMeta = null; + if (localTbl.needRowBinlog() && restoredIdx.getId() == localTbl.getBaseIndexId()) { + rowBinlogIndexMeta = localTbl.getRowBinlogMeta(); + } CreateReplicaTask task = new CreateReplicaTask(restoreReplica.getBackendIdWithoutException(), dbId, localTbl.getId(), restorePart.getId(), restoredIdx.getId(), restoreTablet.getId(), restoreReplica.getId(), indexMeta.getShortKeyColumnCount(), @@ -1459,7 +1469,8 @@ protected void createReplicas(Database db, OlapTable localTbl, Partition restore localTbl.storagePageSize(), localTbl.getTDEAlgorithm(), localTbl.storageDictPageSize(), localTbl.getColumnSeqMapping(), - localTbl.getVerticalCompactionNumColumnsPerGroup()); + localTbl.getVerticalCompactionNumColumnsPerGroup(), + rowBinlogIndexMeta); task.setInvertedIndexFileStorageFormat(localTbl.getInvertedIndexFileStorageFormat()); task.setInRestoreMode(true); if (baseTabletRef != null) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/binlog/BinlogConfigCache.java b/fe/fe-core/src/main/java/org/apache/doris/binlog/BinlogConfigCache.java index 2cf7f065770ee6..cddb7f14e8ddbe 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/binlog/BinlogConfigCache.java +++ b/fe/fe-core/src/main/java/org/apache/doris/binlog/BinlogConfigCache.java @@ -75,7 +75,8 @@ public boolean isEnableDB(long dbId) { if (dBinlogConfig == null) { return false; } - return dBinlogConfig.isEnable(); + // Only enable db binlog when format is STATEMENT_AND_SNAPSHOT (CCR snapshot mode). + return dBinlogConfig.isEnableForCCR(); } public long getDBTtlSeconds(long dbId) { @@ -168,7 +169,8 @@ public boolean isEnableTable(long dbId, long tableId) { if (tableBinlogConfig == null) { return false; } - return tableBinlogConfig.isEnable(); + // Only enable table binlog when format is STATEMENT_AND_SNAPSHOT (CCR snapshot mode). + return tableBinlogConfig.isEnableForCCR(); } public long getTableTtlSeconds(long dbId, long tableId) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/binlog/BinlogManager.java b/fe/fe-core/src/main/java/org/apache/doris/binlog/BinlogManager.java index 0d17efdb6de861..a6b0540146520b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/binlog/BinlogManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/binlog/BinlogManager.java @@ -825,7 +825,7 @@ public long read(DataInputStream dis, long checksum) throws IOException { continue; } currentDbId = dbId; - currentDbBinlogEnable = db.getBinlogConfig().isEnable(); + currentDbBinlogEnable = db.getBinlogConfig().isEnableForCCR(); tableDummies = Lists.newArrayList(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/binlog/BinlogUtils.java b/fe/fe-core/src/main/java/org/apache/doris/binlog/BinlogUtils.java index 0347b94c5303bf..3c5d093e728fe2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/binlog/BinlogUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/binlog/BinlogUtils.java @@ -29,6 +29,12 @@ import java.util.stream.Collectors; public class BinlogUtils { + private static final String ROW_BINLOG_NAME = "binlog"; + + public static String wrapBinlogName(String originTableName) { + return ROW_BINLOG_NAME + "(" + originTableName + ")"; + } + public static Pair> getBinlog( TreeSet binlogs, long prevCommitSeq, long numAcquired) { TStatus status = new TStatus(TStatusCode.OK); diff --git a/fe/fe-core/src/main/java/org/apache/doris/binlog/DBBinlog.java b/fe/fe-core/src/main/java/org/apache/doris/binlog/DBBinlog.java index 8ab9ff7b920c02..3da434b011df80 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/binlog/DBBinlog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/binlog/DBBinlog.java @@ -335,7 +335,7 @@ public BinlogTombstone gc() { if (dbBinlogConfig == null) { LOG.error("db not found. dbId: {}", dbId); return null; - } else if (!dbBinlogConfig.isEnable()) { + } else if (!dbBinlogConfig.isEnableForCCR()) { return dbBinlogDisableGc(); } else { return dbBinlogEnableGc(dbBinlogConfig); diff --git a/fe/fe-core/src/main/java/org/apache/doris/binlog/TableBinlog.java b/fe/fe-core/src/main/java/org/apache/doris/binlog/TableBinlog.java index 8eb27de8092559..a61b52fa513ea3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/binlog/TableBinlog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/binlog/TableBinlog.java @@ -259,7 +259,7 @@ public BinlogTombstone gc() { Boolean isCleanFullBinlog = false; if (tableBinlogConfig == null) { return null; - } else if (!tableBinlogConfig.isEnable()) { + } else if (!tableBinlogConfig.isEnableForCCR()) { isCleanFullBinlog = true; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/BinlogConfig.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/BinlogConfig.java index 00ecaea3fa97b8..0f874e78954a2b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/BinlogConfig.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/BinlogConfig.java @@ -17,11 +17,15 @@ package org.apache.doris.catalog; +import org.apache.doris.common.Pair; import org.apache.doris.common.util.PropertyAnalyzer; import org.apache.doris.persist.gson.GsonUtils; import org.apache.doris.thrift.TBinlogConfig; +import org.apache.doris.thrift.TBinlogFormat; import com.google.gson.annotations.SerializedName; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.util.HashMap; import java.util.Map; @@ -39,50 +43,114 @@ public class BinlogConfig { @SerializedName("maxHistoryNums") private long maxHistoryNums; + @SerializedName("binlogFormat") + private BinlogFormat binlogFormat; + + public enum BinlogFormat { + // record all meta update operator, and generate snapshot for write data, only used for ccr + STATEMENT_AND_SNAPSHOT(0), + // generate row binlog when write, used for table binlog transform + ROW(1), + // generate row binlog when need, calculate binlog by compaction and read snapshot, + // used for table binlog transform + DELTA(2); + + private final int value; + + BinlogFormat(int value) { + this.value = value; + } + + public int value() { + return this.value; + } + } + + @SerializedName("needHistoricalValue") + private boolean needHistoricalValue; + public static final long NO_TTL = -1L; public static final long TTL_SECONDS = 86400L; // 1 day public static final long MAX_BYTES = 0x7fffffffffffffffL; public static final long MAX_HISTORY_NUMS = 0x7fffffffffffffffL; - public BinlogConfig(boolean enable, long ttlSeconds, long maxBytes, long maxHistoryNums) { + private static final Logger LOG = LogManager.getLogger(BinlogConfig.class); + + public BinlogConfig(boolean enable, long ttlSeconds, long maxBytes, long maxHistoryNums, + BinlogFormat binlogFormat, boolean needHistoricalValue) { this.enable = enable; this.ttlSeconds = ttlSeconds; this.maxBytes = maxBytes; this.maxHistoryNums = maxHistoryNums; + this.binlogFormat = binlogFormat; + this.needHistoricalValue = needHistoricalValue; } public BinlogConfig(BinlogConfig config) { - this(config.enable, config.ttlSeconds, config.maxBytes, config.maxHistoryNums); + this(config.enable, config.ttlSeconds, config.maxBytes, config.maxHistoryNums, + config.getBinlogFormat(), config.needHistoricalValue); } public BinlogConfig() { - this(false, TTL_SECONDS, MAX_BYTES, MAX_HISTORY_NUMS); + this(false, TTL_SECONDS, MAX_BYTES, MAX_HISTORY_NUMS, BinlogFormat.STATEMENT_AND_SNAPSHOT, false); + } + + public Pair mergeFromProperties(Map properties) { + return mergeFromProperties(properties, true); } - public void mergeFromProperties(Map properties) { + public Pair mergeFromProperties(Map properties, boolean force) { if (properties == null) { - return; + return Pair.of(true, null); } if (properties.containsKey(PropertyAnalyzer.PROPERTIES_BINLOG_ENABLE)) { - enable = Boolean.parseBoolean(properties.get( - PropertyAnalyzer.PROPERTIES_BINLOG_ENABLE)); + boolean tmpEnable = Boolean.parseBoolean(properties.get(PropertyAnalyzer.PROPERTIES_BINLOG_ENABLE)); + if (!force && this.enable != tmpEnable && binlogFormat == BinlogFormat.ROW) { + LOG.warn("can't disable binlog when format is [Row]"); + return Pair.of(false, "can't disable binlog when format is [Row]"); + } + enable = tmpEnable; } if (properties.containsKey(PropertyAnalyzer.PROPERTIES_BINLOG_TTL_SECONDS)) { - ttlSeconds = Long.parseLong(properties.get( - PropertyAnalyzer.PROPERTIES_BINLOG_TTL_SECONDS)); - + ttlSeconds = Long.parseLong(properties.get(PropertyAnalyzer.PROPERTIES_BINLOG_TTL_SECONDS)); } if (properties.containsKey(PropertyAnalyzer.PROPERTIES_BINLOG_MAX_BYTES)) { - maxBytes = Long.parseLong(properties.get( - PropertyAnalyzer.PROPERTIES_BINLOG_MAX_BYTES)); + maxBytes = Long.parseLong(properties.get(PropertyAnalyzer.PROPERTIES_BINLOG_MAX_BYTES)); } if (properties.containsKey(PropertyAnalyzer.PROPERTIES_BINLOG_MAX_HISTORY_NUMS)) { - maxHistoryNums = Long.parseLong(properties.get( - PropertyAnalyzer.PROPERTIES_BINLOG_MAX_HISTORY_NUMS)); + maxHistoryNums = Long.parseLong(properties.get(PropertyAnalyzer.PROPERTIES_BINLOG_MAX_HISTORY_NUMS)); + } + + // before binlogFormat change + if (properties.containsKey(PropertyAnalyzer.PROPERTIES_BINLOG_NEED_HISTORICAL_VALUE)) { + boolean tmpNeedHistoricalValue = Boolean.parseBoolean( + properties.get(PropertyAnalyzer.PROPERTIES_BINLOG_NEED_HISTORICAL_VALUE)); + if (!force && this.needHistoricalValue != tmpNeedHistoricalValue) { + LOG.warn("not support change {} from {} to {}", + PropertyAnalyzer.PROPERTIES_BINLOG_NEED_HISTORICAL_VALUE, + this.needHistoricalValue, tmpNeedHistoricalValue); + return Pair.of(false, "not support change " + + PropertyAnalyzer.PROPERTIES_BINLOG_NEED_HISTORICAL_VALUE + + " from " + this.needHistoricalValue + " to " + tmpNeedHistoricalValue); + } + needHistoricalValue = tmpNeedHistoricalValue; + } + if (properties.containsKey(PropertyAnalyzer.PROPERTIES_BINLOG_FORMAT)) { + BinlogFormat tmpBinlogFormat = + BinlogFormat.valueOf(properties.get(PropertyAnalyzer.PROPERTIES_BINLOG_FORMAT)); + if (!force && tmpBinlogFormat != binlogFormat) { + LOG.warn("not support change {} from {} to {}", + PropertyAnalyzer.PROPERTIES_BINLOG_FORMAT, + binlogFormat, tmpBinlogFormat); + return Pair.of(false, "not support change binlog format " + + "from " + binlogFormat + " to " + tmpBinlogFormat); + } + binlogFormat = tmpBinlogFormat; } + return Pair.of(true, null); } - public boolean isEnable() { + public boolean getEnable() { return enable; } @@ -114,12 +182,40 @@ public void setMaxHistoryNums(long maxHistoryNums) { this.maxHistoryNums = maxHistoryNums; } + public BinlogFormat getBinlogFormat() { + return binlogFormat; + } + + public void setBinlogFormat(BinlogFormat binlogFormat) { + this.binlogFormat = binlogFormat; + } + + public boolean getNeedHistoricalValue() { + return needHistoricalValue; + } + + public void setNeedHistoricalValue(boolean needHistoricalValue) { + this.needHistoricalValue = needHistoricalValue; + } + + public boolean isEnableForCCR() { + return enable && binlogFormat != BinlogFormat.ROW; + } + + public boolean isEnableForStreaming() { + return enable && binlogFormat == BinlogFormat.ROW; + } + public TBinlogConfig toThrift() { TBinlogConfig tBinlogConfig = new TBinlogConfig(); tBinlogConfig.setEnable(enable); tBinlogConfig.setTtlSeconds(ttlSeconds); tBinlogConfig.setMaxBytes(maxBytes); tBinlogConfig.setMaxHistoryNums(maxHistoryNums); + if (binlogFormat != null) { + tBinlogConfig.setBinlogFormat(TBinlogFormat.valueOf(binlogFormat.name())); + } + tBinlogConfig.setNeedHistoricalValue(needHistoricalValue); return tBinlogConfig; } @@ -129,29 +225,23 @@ public Map toProperties() { properties.put(PropertyAnalyzer.PROPERTIES_BINLOG_TTL_SECONDS, String.valueOf(ttlSeconds)); properties.put(PropertyAnalyzer.PROPERTIES_BINLOG_MAX_BYTES, String.valueOf(maxBytes)); properties.put(PropertyAnalyzer.PROPERTIES_BINLOG_MAX_HISTORY_NUMS, String.valueOf(maxHistoryNums)); + properties.put(PropertyAnalyzer.PROPERTIES_BINLOG_FORMAT, String.valueOf(binlogFormat)); + properties.put(PropertyAnalyzer.PROPERTIES_BINLOG_NEED_HISTORICAL_VALUE, String.valueOf(needHistoricalValue)); return properties; } @Override public boolean equals(Object obj) { - if (obj == null) { - return false; - } if (!(obj instanceof BinlogConfig)) { return false; } - BinlogConfig other = (BinlogConfig) obj; - if (this.enable != other.enable) { - return false; - } - if (this.ttlSeconds != other.ttlSeconds) { - return false; - } - if (this.maxBytes != other.maxBytes) { - return false; - } - return this.maxHistoryNums == other.maxHistoryNums; + return enable == other.enable + && ttlSeconds == other.ttlSeconds + && maxBytes == other.maxBytes + && maxHistoryNums == other.maxHistoryNums + && binlogFormat == other.binlogFormat + && needHistoricalValue == other.needHistoricalValue; } @Override @@ -160,14 +250,21 @@ public String toString() { } public void appendToShowCreateTable(StringBuilder sb) { - sb.append(",\n\"").append(PropertyAnalyzer.PROPERTIES_BINLOG_ENABLE).append("\" = \"").append(enable) - .append("\""); - sb.append(",\n\"").append(PropertyAnalyzer.PROPERTIES_BINLOG_TTL_SECONDS).append("\" = \"").append(ttlSeconds) - .append("\""); - sb.append(",\n\"").append(PropertyAnalyzer.PROPERTIES_BINLOG_MAX_BYTES).append("\" = \"").append(maxBytes) - .append("\""); + sb.append(",\n\"").append(PropertyAnalyzer.PROPERTIES_BINLOG_ENABLE).append("\" = \"") + .append(enable).append("\""); + sb.append(",\n\"").append(PropertyAnalyzer.PROPERTIES_BINLOG_TTL_SECONDS).append("\" = \"") + .append(ttlSeconds).append("\""); + sb.append(",\n\"").append(PropertyAnalyzer.PROPERTIES_BINLOG_MAX_BYTES).append("\" = \"") + .append(maxBytes).append("\""); sb.append(",\n\"").append(PropertyAnalyzer.PROPERTIES_BINLOG_MAX_HISTORY_NUMS).append("\" = \"") .append(maxHistoryNums).append("\""); + + sb.append(",\n\"").append(PropertyAnalyzer.PROPERTIES_BINLOG_FORMAT).append("\" = \"") + .append(binlogFormat).append("\""); + if (binlogFormat == BinlogFormat.ROW) { + sb.append(",\n\"").append(PropertyAnalyzer.PROPERTIES_BINLOG_NEED_HISTORICAL_VALUE) + .append("\" = \"").append(needHistoricalValue).append("\""); + } } public static BinlogConfig fromProperties(Map properties) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/CloudTabletStatMgr.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/CloudTabletStatMgr.java index c74011d9d7232d..6765a001492301 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/CloudTabletStatMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/CloudTabletStatMgr.java @@ -422,7 +422,7 @@ private void updateStatInfo(List dbIds) { tableStats = new OlapTable.Statistics(db.getName(), table.getName(), tableDataSize, tableTotalReplicaDataSize, 0L, tableReplicaCount, tableRowCount, 0L, 0L, - tableTotalLocalIndexSize, tableTotalLocalSegmentSize, 0L, 0L); + tableTotalLocalIndexSize, tableTotalLocalSegmentSize, 0L, 0L, 0L, 0L); olapTable.setStatistics(tableStats); LOG.debug("finished to set row num for table: {} in database: {}", table.getName(), db.getFullName()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java index cd6690219df89e..9afec0ab1bc4df 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java @@ -922,7 +922,7 @@ public boolean updateDbProperties(Map properties) throws DdlExce BinlogConfig oldBinlogConfig = getBinlogConfig(); BinlogConfig newBinlogConfig = BinlogConfig.fromProperties(properties); - if (newBinlogConfig.isEnable() && !oldBinlogConfig.isEnable()) { + if (newBinlogConfig.isEnableForCCR() && !oldBinlogConfig.isEnableForCCR()) { // check all tables binlog enable is true for (Table table : idToTable.values()) { if (!table.isManagedTable()) { @@ -932,7 +932,7 @@ public boolean updateDbProperties(Map properties) throws DdlExce OlapTable olapTable = (OlapTable) table; olapTable.readLock(); try { - if (!olapTable.getBinlogConfig().isEnable()) { + if (!olapTable.getBinlogConfig().isEnableForCCR()) { String errMsg = String .format("binlog is not enable in table[%s] in db [%s]", table.getDisplayName(), getFullName()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java index f7ea7ba25b9f94..741d42400bbc2f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java @@ -5824,7 +5824,7 @@ public void renameRollup(Database db, OlapTable table, RenameRollupOp renameRoll throw new DdlException("Same rollup name"); } - Map indexNameToIdMap = table.getIndexNameToId(); + Map indexNameToIdMap = table.getMutableIndexNameToId(); if (indexNameToIdMap.get(rollupName) == null) { throw new DdlException("Rollup index[" + rollupName + "] does not exists"); } @@ -5858,7 +5858,7 @@ public void replayRenameRollup(TableInfo tableInfo) throws MetaNotFoundException olapTable.writeLock(); try { String rollupName = olapTable.getIndexNameById(indexId); - Map indexNameToIdMap = olapTable.getIndexNameToId(); + Map indexNameToIdMap = olapTable.getMutableIndexNameToId(); indexNameToIdMap.remove(rollupName); indexNameToIdMap.put(newRollupName, indexId); @@ -6278,9 +6278,7 @@ public void modifyTableProperties(Database db, OlapTable table, Map getIndexNameToId() { + return ImmutableMap.copyOf(Maps.filterValues(indexNameToId, indexId -> + indexIdToMeta.containsKey(indexId) && !indexIdToMeta.get(indexId).isRowBinlogIndex())); + } + + public Map getMutableIndexNameToId() { return indexNameToId; } @@ -675,7 +682,7 @@ private Long getMvIdWithUseMvHint(UseMvHint useMvHint, List names, List< + this.name); return orderedMvs.get(0); } else { - for (Map.Entry entry : indexNameToId.entrySet()) { + for (Map.Entry entry : getIndexNameToId().entrySet()) { String mvName = entry.getKey(); names.add(mvName); if (useMvHint.getUseMvTableColumnMap().containsKey(names)) { @@ -700,7 +707,7 @@ private Long getMvIdWithNoUseMvHint(UseMvHint noUseMvHint, List names, L return getBaseIndex().getId(); } else { Set forbiddenIndexIds = Sets.newHashSet(); - for (Map.Entry entry : indexNameToId.entrySet()) { + for (Map.Entry entry : getIndexNameToId().entrySet()) { String mvName = entry.getKey(); names.add(mvName); if (noUseMvHint.getNoUseMvTableColumnMap().containsKey(names)) { @@ -833,24 +840,29 @@ public Status resetIdsForRestore(Env env, Database db, ReplicaAllocation restore id = env.getNextId(); // copy an origin index id to name map - Map origIdxIdToName = Maps.newHashMap(); + Map origIdxIdToNameWithRowBinlog = Maps.newHashMap(); for (Map.Entry entry : indexNameToId.entrySet()) { - origIdxIdToName.put(entry.getValue(), entry.getKey()); + origIdxIdToNameWithRowBinlog.put(entry.getValue(), entry.getKey()); } // reset all 'indexIdToXXX' map - Map origIdxIdToMeta = indexIdToMeta; + Map origIdxIdToMetaWithRowBinlog = indexIdToMeta; + Map origIdxIdToName = Maps.newHashMap(); indexIdToMeta = Maps.newHashMap(); - for (Map.Entry entry : origIdxIdToName.entrySet()) { + for (Map.Entry entry : origIdxIdToNameWithRowBinlog.entrySet()) { long newIdxId = env.getNextId(); if (entry.getValue().equals(name)) { // base index baseIndexId = newIdxId; } - MaterializedIndexMeta indexMeta = origIdxIdToMeta.get(entry.getKey()); + MaterializedIndexMeta indexMeta = origIdxIdToMetaWithRowBinlog.get(entry.getKey()); indexMeta.resetIndexIdForRestore(newIdxId, srcDbName, db.getName()); indexIdToMeta.put(newIdxId, indexMeta); indexNameToId.put(entry.getValue(), newIdxId); + + if (!indexMeta.isRowBinlogIndex()) { + origIdxIdToName.put(entry.getKey(), entry.getValue()); + } } // generate a partition name to id map @@ -1046,19 +1058,23 @@ && getTableProperty().getDynamicPartitionProperty().getBuckets() } public int getIndexNumber() { - return indexIdToMeta.size(); + return getIndexIdToMeta().size(); } public Map getIndexIdToMeta() { + return ImmutableMap.copyOf(Maps.filterValues(indexIdToMeta, meta -> !meta.isRowBinlogIndex())); + } + + public Map getMutableIndexIdToMeta() { return indexIdToMeta; } public Map getCopyOfIndexIdToMeta() { - return new HashMap<>(indexIdToMeta); + return new HashMap<>(getIndexIdToMeta()); } public Map getCopiedIndexIdToMeta() { - return new HashMap<>(indexIdToMeta); + return new HashMap<>(getIndexIdToMeta()); } public MaterializedIndexMeta getIndexMetaByIndexId(long indexId) { @@ -1067,7 +1083,7 @@ public MaterializedIndexMeta getIndexMetaByIndexId(long indexId) { public List getIndexIdListExceptBaseIndex() { List result = Lists.newArrayList(); - for (Long indexId : indexIdToMeta.keySet()) { + for (Long indexId : getIndexIdToMeta().keySet()) { if (indexId != baseIndexId) { result.add(indexId); } @@ -1077,7 +1093,7 @@ public List getIndexIdListExceptBaseIndex() { public List getIndexIdList() { List result = Lists.newArrayList(); - for (Long indexId : indexIdToMeta.keySet()) { + for (Long indexId : getIndexIdToMeta().keySet()) { result.add(indexId); } return result; @@ -1090,6 +1106,14 @@ public Map> getIndexIdToSchema() { // schema public Map> getIndexIdToSchema(boolean full) { + Map> result = Maps.newHashMap(); + for (Map.Entry entry : getIndexIdToMeta().entrySet()) { + result.put(entry.getKey(), entry.getValue().getSchema(full)); + } + return result; + } + + public Map> getIndexIdToSchemaWithRowBinlog(boolean full) { Map> result = Maps.newHashMap(); for (Map.Entry entry : indexIdToMeta.entrySet()) { result.put(entry.getKey(), entry.getValue().getSchema(full)); @@ -1099,6 +1123,14 @@ public Map> getIndexIdToSchema(boolean full) { // get schemas with a copied column list public Map> getCopiedIndexIdToSchema(boolean full) { + Map> result = Maps.newHashMap(); + for (Map.Entry entry : getIndexIdToMeta().entrySet()) { + result.put(entry.getKey(), new ArrayList<>(entry.getValue().getSchema(full))); + } + return result; + } + + public Map> getCopiedIndexIdToSchemaWithRowBinlog(boolean full) { Map> result = Maps.newHashMap(); for (Map.Entry entry : indexIdToMeta.entrySet()) { result.put(entry.getKey(), new ArrayList<>(entry.getValue().getSchema(full))); @@ -1629,7 +1661,7 @@ public void setSequenceInfo(Type type, Column refColumn) { // add sequence column at last fullSchema.add(sequenceCol); nameToColumn.put(Column.SEQUENCE_COL, sequenceCol); - for (MaterializedIndexMeta indexMeta : indexIdToMeta.values()) { + for (MaterializedIndexMeta indexMeta : getIndexIdToMeta().values()) { List schema = indexMeta.getSchema(); if (indexMeta.getIndexId() != baseIndexId) { sequenceCol = buildSequenceCol(type, refColumn); @@ -1788,7 +1820,7 @@ public Set> getColumnIndexPairs(Set columns) { // Check the schema of all indexes for each given column name, // If the column name exists in the index, add the pair to return list. for (String column : columns) { - for (MaterializedIndexMeta meta : indexIdToMeta.values()) { + for (MaterializedIndexMeta meta : getIndexIdToMeta().values()) { Column col = meta.getColumnByName(column); if (col == null || StatisticsUtil.isUnsupportedType(col.getType())) { continue; @@ -2294,6 +2326,102 @@ public List getBaseSchema(boolean full) { return getSchemaByIndexId(baseIndexId, full); } + public boolean needRowBinlog() { + return getBinlogConfig().isEnableForStreaming(); + } + + public void createNewRowBinlogMeta(IdGeneratorBuffer idGeneratorBuffer) { + writeLock(); + try { + List schema = generateTableRowBinlogSchema(); + long indexId = idGeneratorBuffer.getNextId(); + MaterializedIndexMeta rowBinlogMeta = new MaterializedIndexMeta(indexId, schema, + this.getBaseSchemaVersion(), Util.generateSchemaHash(), + this.getBaseIndexMeta().getShortKeyColumnCount(), TStorageType.COLUMN, + KeysType.DUP_KEYS, null, null, getQualifiedDbName(), null); + rowBinlogMeta.initSchemaColumnUniqueId(); + rowBinlogMeta.setRowBinlogIndexId(indexId); + this.setRowBinlogMeta(rowBinlogMeta, BinlogUtils.wrapBinlogName(this.name)); + } finally { + writeUnlock(); + } + } + + public void setRowBinlogMeta(MaterializedIndexMeta rowBinlogMeta, String indexName) { + long rowBinlogIndexId = getBaseIndexMeta().getRowBinlogIndexId(); + if (rowBinlogIndexId > 0) { + indexIdToMeta.remove(rowBinlogIndexId); + indexNameToId.remove(indexName); + getBaseIndexMeta().resetRowBinlogIndexId(); + } + if (rowBinlogMeta == null) { + return; + } + rowBinlogIndexId = rowBinlogMeta.getIndexId(); + rowBinlogMeta.setRowBinlogIndexId(rowBinlogIndexId); + getBaseIndexMeta().setRowBinlogIndexId(rowBinlogIndexId); + indexIdToMeta.put(rowBinlogIndexId, rowBinlogMeta); + indexNameToId.put(indexName, rowBinlogIndexId); + } + + public MaterializedIndexMeta getRowBinlogMeta() { + Preconditions.checkState(needRowBinlog()); + long rowBinlogIndexId = getBaseIndexMeta().getRowBinlogIndexId(); + if (rowBinlogIndexId <= 0) { + return null; + } + return indexIdToMeta.get(rowBinlogIndexId); + } + + public MaterializedIndexMeta getBaseIndexMeta() { + return indexIdToMeta.get(baseIndexId); + } + + public List generateTableRowBinlogSchema() { + List tableRowBinlogSchema = new ArrayList<>(); + boolean needHistoricalValue = getBinlogConfig().getNeedHistoricalValue(); + List beforeColumns = new ArrayList<>(); + + for (Column column : getBaseSchema(false)) { + Preconditions.checkState(!column.getType().isVariantType(), + "binlog does not support VARIANT column: " + column.getName()); + Preconditions.checkState(!column.isAutoInc(), + "binlog does not support AUTO_INCREMENT column: " + column.getName()); + if (column.isKey()) { + tableRowBinlogSchema.add(Column.generateRowBinlogKeyColumn(column)); + } else { + tableRowBinlogSchema.add(Column.generateAfterValueColumn(column)); + if (needHistoricalValue) { + beforeColumns.add(Column.generateBeforeValueColumn(column)); + } + } + } + + if (needHistoricalValue) { + Preconditions.checkState(keysType == KeysType.PRIMARY_KEYS + || (keysType == KeysType.UNIQUE_KEYS && getEnableUniqueKeyMergeOnWrite()), + "only mow table support record historical value"); + tableRowBinlogSchema.addAll(beforeColumns); + } + + tableRowBinlogSchema.add(new ColumnDef(Column.BINLOG_LSN_COL, ScalarType.createType(PrimitiveType.LARGEINT), + false, AggregateType.NONE, false, -1, ColumnDef.DefaultValue.NOT_SET, + "doris binlog lsn column", false).toColumn()); + tableRowBinlogSchema.add(new ColumnDef(Column.BINLOG_OPERATION_COL, + ScalarType.createType(PrimitiveType.BIGINT), false, AggregateType.NONE, true, -1, + ColumnDef.DefaultValue.NOT_SET, "doris binlog operation column", false).toColumn()); + tableRowBinlogSchema.add(new ColumnDef(Column.BINLOG_TIMESTAMP_COL, + ScalarType.createType(PrimitiveType.BIGINT), false, AggregateType.NONE, true, -1, + ColumnDef.DefaultValue.NOT_SET, "doris binlog timestamp column", false).toColumn()); + + for (Column column : tableRowBinlogSchema) { + if (!column.isKey()) { + column.setAggregationTypeImplicit(true); + } + } + return tableRowBinlogSchema; + } + @Override public boolean equals(Object o) { if (this == o) { @@ -3727,6 +3855,11 @@ public static class Statistics { @Getter private long remoteSegmentSize; // single replica + @Getter + private long localBinlogSize; // single replica binlog size + @Getter + private long totalReplicaBinlogSize; + public Statistics() { this.dbName = null; this.tableName = null; @@ -3745,6 +3878,8 @@ public Statistics() { this.localSegmentSize = 0L; this.remoteInvertedIndexSize = 0L; this.remoteSegmentSize = 0L; + this.localBinlogSize = 0L; + this.totalReplicaBinlogSize = 0L; } public Statistics(String dbName, String tableName, @@ -3752,7 +3887,8 @@ public Statistics(String dbName, String tableName, long remoteDataSize, long replicaCount, long rowCount, long rowsetCount, long segmentCount, long localInvertedIndexSize, long localSegmentSize, - long remoteInvertedIndexSize, long remoteSegmentSize) { + long remoteInvertedIndexSize, long remoteSegmentSize, + long localBinlogSize, long totalReplicaBinlogSize) { this.dbName = dbName; this.tableName = tableName; @@ -3772,6 +3908,9 @@ public Statistics(String dbName, String tableName, this.localSegmentSize = localSegmentSize; this.remoteInvertedIndexSize = remoteInvertedIndexSize; this.remoteSegmentSize = remoteSegmentSize; + + this.localBinlogSize = localBinlogSize; + this.totalReplicaBinlogSize = totalReplicaBinlogSize; } } @@ -3791,6 +3930,18 @@ public long getRemoteDataSize() { return statistics.getRemoteDataSize(); } + public long getBinlogSize() { + return getBinlogSize(false); + } + + public long getBinlogSize(boolean singleReplica) { + if (singleReplica) { + return statistics.getLocalBinlogSize(); + } + + return statistics.getTotalReplicaBinlogSize(); + } + public long getReplicaCount() { return statistics.getReplicaCount(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTableWrapper.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTableWrapper.java new file mode 100644 index 00000000000000..c80b0f45a3f01e --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTableWrapper.java @@ -0,0 +1,140 @@ +// 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. + +package org.apache.doris.catalog; + +import java.util.Collection; +import java.util.List; +import java.util.Set; +import java.util.concurrent.TimeUnit; + +/** + * A lightweight wrapper base class for {@link OlapTable}. + * + *

It delegates table locks and partition-related operations to the wrapped table to keep the + * metadata view consistent. + */ +public class OlapTableWrapper extends OlapTable { + + protected final OlapTable originTable; + + protected OlapTableWrapper(OlapTable originTable, String wrapperName, List baseSchema, KeysType keysType) { + super(originTable.getId(), wrapperName, baseSchema, + keysType, originTable.getPartitionInfo(), originTable.getDefaultDistributionInfo()); + this.originTable = originTable; + } + + protected OlapTableWrapper(OlapTable originTable) { + super(originTable.getId(), originTable.getName(), originTable.getBaseSchema(), + originTable.getKeysType(), originTable.getPartitionInfo(), originTable.getDefaultDistributionInfo()); + this.originTable = originTable; + } + + public OlapTable getOriginTable() { + return originTable; + } + + @Override + public long getBaseIndexId() { + return originTable.getBaseIndexId(); + } + + @Override + public MaterializedIndexMeta getIndexMetaByIndexId(long indexId) { + return originTable.getIndexMetaByIndexId(indexId); + } + + @Override + public String getIndexNameById(long indexId) { + return originTable.getIndexNameById(indexId); + } + + @Override + public int getIndexSchemaVersion(long indexId) { + return originTable.getIndexSchemaVersion(indexId); + } + + @Override + public List getSchemaByIndexId(Long indexId) { + return originTable.getSchemaByIndexId(indexId); + } + + @Override + public List getSchemaByIndexId(Long indexId, boolean full) { + return originTable.getSchemaByIndexId(indexId, full); + } + + @Override + public void readLock() { + originTable.readLock(); + } + + @Override + public boolean tryReadLock(long timeout, TimeUnit unit) { + return originTable.tryReadLock(timeout, unit); + } + + @Override + public void readUnlock() { + originTable.readUnlock(); + } + + @Override + public PartitionInfo getPartitionInfo() { + return originTable.getPartitionInfo(); + } + + @Override + public Partition getPartition(String partitionName, boolean isTempPartition) { + return originTable.getPartition(partitionName, isTempPartition); + } + + @Override + public Partition getPartition(String partitionName) { + return originTable.getPartition(partitionName); + } + + @Override + public Partition getPartition(long partitionId) { + return originTable.getPartition(partitionId); + } + + @Override + public Set getPartitionNames() { + return originTable.getPartitionNames(); + } + + @Override + public List getPartitionIds() { + return originTable.getPartitionIds(); + } + + @Override + public Collection getPartitions() { + return originTable.getPartitions(); + } + + @Override + public List selectNonEmptyPartitionIds(Collection partitionIds) { + return originTable.selectNonEmptyPartitionIds(partitionIds); + } + + @Override + public Set getDistributionColumnNames() { + return originTable.getDistributionColumnNames(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Partition.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Partition.java index d6a3545088cff6..4edc4d5888ec26 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Partition.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Partition.java @@ -279,6 +279,14 @@ public long getRemoteDataSize() { return remoteDataSize; } + public long getBinlogDataSize() { + long binlogDataSize = 0; + for (MaterializedIndex mIndex : getMaterializedIndices(IndexExtState.VISIBLE)) { + binlogDataSize += mIndex.getBinlogSize(); + } + return binlogDataSize; + } + public long getReplicaCount() { long replicaCount = 0; for (MaterializedIndex mIndex : getMaterializedIndices(IndexExtState.VISIBLE)) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Replica.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Replica.java index ad83287925c0c4..61135a2fc7d751 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Replica.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Replica.java @@ -91,6 +91,8 @@ public static class ReplicaContext { private int schemaHash = -1; @SerializedName(value = "ds", alternate = {"dataSize"}) private volatile long dataSize = 0; + @SerializedName(value = "bs", alternate = {"binlogSize"}) + private volatile long binlogSize = 0; @SerializedName(value = "rc", alternate = {"rowCount"}) private volatile long rowCount = 0; @SerializedName(value = "st", alternate = {"state"}) @@ -105,6 +107,10 @@ public static class ReplicaContext { @SerializedName(value = "lss", alternate = {"localSegmentSize"}) private long localSegmentSize = 0L; + // Number of binlog files retained on this replica. + @SerializedName(value = "bfn", alternate = {"binloFileNum"}) + private volatile long binlogFileNum = -1; + public Replica() { } @@ -187,6 +193,22 @@ public void setDataSize(long dataSize) { this.dataSize = dataSize; } + public long getBinlogSize() { + return binlogSize; + } + + public void setBinlogSize(long binlogSize) { + this.binlogSize = binlogSize; + } + + public long getBinlogFileNum() { + return binlogFileNum; + } + + public void setBinlogFileNum(long binlogFileNum) { + this.binlogFileNum = binlogFileNum; + } + public long getRemoteDataSize() { return 0; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/RowBinlogTableWrapper.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/RowBinlogTableWrapper.java new file mode 100644 index 00000000000000..bf7f6f63c6c83c --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/RowBinlogTableWrapper.java @@ -0,0 +1,40 @@ +// 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. + +package org.apache.doris.catalog; + +import com.google.common.base.Preconditions; + +/** + * A lightweight wrapper base for read binlog of table + */ +public class RowBinlogTableWrapper extends OlapTableWrapper { + + private final MaterializedIndexMeta rowBinlogMeta; + + public RowBinlogTableWrapper(OlapTable originTable) { + super(originTable, originTable.getName(), originTable.generateTableRowBinlogSchema(), KeysType.DUP_KEYS); + this.rowBinlogMeta = originTable.getRowBinlogMeta(); + Preconditions.checkNotNull(rowBinlogMeta, "row binlog meta is null, table=%s", originTable.getName()); + this.setBaseIndexId(rowBinlogMeta.getIndexId()); + } + + @Override + public long getBaseIndexId() { + return rowBinlogMeta.getIndexId(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/TableProperty.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/TableProperty.java index 2221a8f8fb7340..394ecc7a1b87df 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/TableProperty.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/TableProperty.java @@ -592,8 +592,15 @@ public TableProperty buildBinlogConfig() { binlogConfig.setMaxHistoryNums( Long.parseLong(properties.get(PropertyAnalyzer.PROPERTIES_BINLOG_MAX_HISTORY_NUMS))); } + if (properties.containsKey(PropertyAnalyzer.PROPERTIES_BINLOG_FORMAT)) { + binlogConfig.setBinlogFormat(BinlogConfig.BinlogFormat.valueOf( + properties.get(PropertyAnalyzer.PROPERTIES_BINLOG_FORMAT))); + } + if (properties.containsKey(PropertyAnalyzer.PROPERTIES_BINLOG_NEED_HISTORICAL_VALUE)) { + binlogConfig.setNeedHistoricalValue(Boolean.parseBoolean( + properties.get(PropertyAnalyzer.PROPERTIES_BINLOG_NEED_HISTORICAL_VALUE))); + } this.binlogConfig = binlogConfig; - return this; } @@ -606,13 +613,17 @@ public BinlogConfig getBinlogConfig() { public void setBinlogConfig(BinlogConfig newBinlogConfig) { Map binlogProperties = Maps.newHashMap(); - binlogProperties.put(PropertyAnalyzer.PROPERTIES_BINLOG_ENABLE, String.valueOf(newBinlogConfig.isEnable())); + binlogProperties.put(PropertyAnalyzer.PROPERTIES_BINLOG_ENABLE, String.valueOf(newBinlogConfig.getEnable())); binlogProperties.put(PropertyAnalyzer.PROPERTIES_BINLOG_TTL_SECONDS, String.valueOf(newBinlogConfig.getTtlSeconds())); binlogProperties.put(PropertyAnalyzer.PROPERTIES_BINLOG_MAX_BYTES, String.valueOf(newBinlogConfig.getMaxBytes())); binlogProperties.put(PropertyAnalyzer.PROPERTIES_BINLOG_MAX_HISTORY_NUMS, String.valueOf(newBinlogConfig.getMaxHistoryNums())); + binlogProperties.put(PropertyAnalyzer.PROPERTIES_BINLOG_FORMAT, + String.valueOf(newBinlogConfig.getBinlogFormat())); + binlogProperties.put(PropertyAnalyzer.PROPERTIES_BINLOG_NEED_HISTORICAL_VALUE, + String.valueOf(newBinlogConfig.getNeedHistoricalValue())); modifyTableProperties(binlogProperties); this.binlogConfig = newBinlogConfig; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Tablet.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Tablet.java index 462464e91823b5..ba8fa5c0fe7241 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Tablet.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Tablet.java @@ -343,6 +343,16 @@ public long getRemoteDataSize() { return 0; } + public long getBinlogDataSize() { + long binlogDataSize = 0; + for (Replica replica : getReplicas()) { + if (replica.getState() == ReplicaState.NORMAL) { + binlogDataSize += replica.getBinlogSize(); + } + } + return binlogDataSize; + } + public abstract long getRowCount(boolean singleReplica); // Get the least row count among all valid replicas. diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/TabletStatMgr.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/TabletStatMgr.java index a493aecc4ac5a3..bcf74528da6fd9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/TabletStatMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/TabletStatMgr.java @@ -159,6 +159,9 @@ protected void runAfterCatalogReady() { long tableRowCount = 0L; + Long tableBinlogSize = 0L; + Long tableTotalBinlogSize = 0L; + if (!table.readLockIfExist()) { continue; } @@ -197,6 +200,8 @@ protected void runAfterCatalogReady() { long tabletRowCount = Long.MAX_VALUE; + Long tabletBinlogSize = 0L; + boolean tabletReported = false; for (Replica replica : tablet.getReplicas()) { LOG.debug("Table {} replica {} current version {}, report version {}", @@ -236,6 +241,11 @@ protected void runAfterCatalogReady() { tableTotalLocalSegmentSize += replica.getLocalSegmentSize(); tableTotalRemoteIndexSize += replica.getRemoteInvertedIndexSize(); tableTotalRemoteSegmentSize += replica.getRemoteSegmentSize(); + + if (replica.getBinlogSize() > tabletBinlogSize) { + tabletBinlogSize = replica.getBinlogSize(); + } + tableTotalBinlogSize += replica.getBinlogSize(); } tableDataSize += tabletDataSize; @@ -256,6 +266,8 @@ protected void runAfterCatalogReady() { indexRowCount += tabletRowCount; // Only when all tablets of this index are reported, we set indexReported to true. indexReported = indexReported && tabletReported; + + tableBinlogSize += tabletBinlogSize; } // end for tablets index.setRowCountReported(indexReported); index.setRowCount(indexRowCount); @@ -282,7 +294,8 @@ protected void runAfterCatalogReady() { tableDataSize, tableTotalReplicaDataSize, tableRemoteDataSize, tableReplicaCount, tableRowCount, 0L, 0L, tableTotalLocalIndexSize, tableTotalLocalSegmentSize, - tableTotalRemoteIndexSize, tableTotalRemoteSegmentSize)); + tableTotalRemoteIndexSize, tableTotalRemoteSegmentSize, + tableBinlogSize, tableTotalBinlogSize)); if (LOG.isDebugEnabled()) { LOG.debug("finished to set row num for table: {} in database: {}", @@ -384,6 +397,11 @@ private void updateTabletStat(Long beId, TTabletStatResult result) { // Older version BE doesn't set visible version. Set it to max for compatibility. replica.setLastReportVersion(stat.isSetVisibleVersion() ? stat.getVisibleVersion() : Long.MAX_VALUE); + + if (stat.isSetBinlogSize()) { + replica.setBinlogSize(stat.getBinlogSize()); + replica.setBinlogFileNum(stat.getBinlogFileNum()); + } } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/proc/ReplicasProcNode.java b/fe/fe-core/src/main/java/org/apache/doris/common/proc/ReplicasProcNode.java index 0ef67d74383841..aaa9da4603771f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/proc/ReplicasProcNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/proc/ReplicasProcNode.java @@ -52,7 +52,8 @@ public class ReplicasProcNode implements ProcNodeInterface { .add("IsUserDrop") .add("VisibleVersionCount").add("VersionCount").add("PathHash").add("Path") .add("MetaUrl").add("CompactionStatus").add("CooldownReplicaId") - .add("CooldownMetaId").add("QueryHits").add("WindowAccessCount").add("LastAccessTime"); + .add("CooldownMetaId").add("QueryHits").add("BinlogSize").add("BinlogFileNum") + .add("WindowAccessCount").add("LastAccessTime"); if (Config.isCloudMode()) { builder.add("PrimaryBackendId"); @@ -148,6 +149,8 @@ public ProcResult fetchResult() throws AnalysisException { String.valueOf(tablet.getCooldownReplicaId()), cooldownMetaId, String.valueOf(queryHits), + String.valueOf(replica.getBinlogSize()), + String.valueOf(replica.getBinlogFileNum()), String.valueOf(accessCount), String.valueOf(lastAccessTime) ); diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/proc/TabletsProcDir.java b/fe/fe-core/src/main/java/org/apache/doris/common/proc/TabletsProcDir.java index 34eff696fd0e1d..d0fd836f6ee810 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/proc/TabletsProcDir.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/proc/TabletsProcDir.java @@ -65,12 +65,11 @@ public class TabletsProcDir implements ProcDirInterface { .add("VisibleVersionCount").add("VersionCount").add("QueryHits").add("WindowAccessCount") .add("LastAccessTime").add("PathHash").add("Path") .add("MetaUrl").add("CompactionStatus") - .add("CooldownReplicaId").add("CooldownMetaId"); - + .add("CooldownReplicaId").add("CooldownMetaId") + .add("BinlogSize").add("BinlogFileNum"); if (Config.isCloudMode()) { builder.add("PrimaryBackendId"); } - TITLE_NAMES = builder.build(); } @@ -151,6 +150,8 @@ public List> fetchComparableResult(long version, long backendId tabletInfo.add(FeConstants.null_string); // compaction status tabletInfo.add(-1); // cooldown replica id tabletInfo.add(""); // cooldown meta id + tabletInfo.add(-1L); // binlog data size + tabletInfo.add(-1L); // binlog file num if (Config.isCloudMode()) { tabletInfo.add(-1L); // primary backend id } @@ -211,6 +212,8 @@ public List> fetchComparableResult(long version, long backendId } else { tabletInfo.add(replica.getCooldownMetaId().toString()); } + tabletInfo.add(replica.getBinlogSize()); + tabletInfo.add(replica.getBinlogFileNum()); if (Config.isCloudMode()) { tabletInfo.add(((CloudReplica) replica).getPrimaryBackendId()); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/util/BufferSizeUtil.java b/fe/fe-core/src/main/java/org/apache/doris/common/util/BufferSizeUtil.java index 9ed7807e8cc68c..d8194861f7425c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/util/BufferSizeUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/util/BufferSizeUtil.java @@ -48,6 +48,8 @@ public static long getBufferSizeForCreateTable(CreateTableInfo createTableInfo, bufferSize = bufferSize + (replicaNum + 1) * indexNum * bucketNum; } } + // Reserve one extra id for binlog index id. + bufferSize += 1; return bufferSize; } @@ -60,6 +62,8 @@ public static long getBufferSizeForTruncateTable(OlapTable table, Collection analyzeBinlogConfig(Map proper } } + // check PROPERTIES_BINLOG_FORMAT = "binlog.format"; + if (properties.containsKey(PROPERTIES_BINLOG_FORMAT)) { + String format = properties.get(PROPERTIES_BINLOG_FORMAT); + try { + BinlogConfig.BinlogFormat.valueOf(format); + binlogConfigMap.put(PROPERTIES_BINLOG_FORMAT, format); + properties.remove(PROPERTIES_BINLOG_FORMAT); + } catch (Exception e) { + throw new AnalysisException("Invalid binlog format value: " + format); + } + } + + // check PROPERTIES_BINLOG_NEED_HISTORICAL_VALUE = "binlog.need_historical_value" + if (properties.containsKey(PROPERTIES_BINLOG_NEED_HISTORICAL_VALUE)) { + String needHistoricalValue = properties.get(PROPERTIES_BINLOG_NEED_HISTORICAL_VALUE); + if (!StringUtils.equalsAnyIgnoreCase(needHistoricalValue, "true", "false")) { + throw new AnalysisException("Invalid binlog need_historical_value value: " + needHistoricalValue); + } + binlogConfigMap.put(PROPERTIES_BINLOG_NEED_HISTORICAL_VALUE, + String.valueOf(Boolean.parseBoolean(needHistoricalValue))); + properties.remove(PROPERTIES_BINLOG_NEED_HISTORICAL_VALUE); + } + + return binlogConfigMap; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java index 900ebdaa3b8905..02a4d3b575aa86 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java @@ -2150,6 +2150,10 @@ protected Partition createPartitionWithIndices(long dbId, OlapTable tbl, long pa long backendId = replica.getBackendIdWithoutException(); long replicaId = replica.getId(); countDownLatch.addMark(backendId, tabletId); + MaterializedIndexMeta rowBinlogIndexMeta = null; + if (tbl.needRowBinlog() && indexId == tbl.getBaseIndexId()) { + rowBinlogIndexMeta = tbl.getRowBinlogMeta(); + } CreateReplicaTask task = new CreateReplicaTask(backendId, dbId, tbl.getId(), partitionId, indexId, tabletId, replicaId, shortKeyColumnCount, schemaHash, version, keysType, storageType, realStorageMedium, schema, bfColumns, tbl.getBfFpp(), countDownLatch, @@ -2169,7 +2173,8 @@ protected Partition createPartitionWithIndices(long dbId, OlapTable tbl, long pa tbl.storagePageSize(), tbl.getTDEAlgorithm(), tbl.storageDictPageSize(), tbl.getColumnSeqMapping(), - tbl.getVerticalCompactionNumColumnsPerGroup()); + tbl.getVerticalCompactionNumColumnsPerGroup(), + rowBinlogIndexMeta); task.setStorageFormat(tbl.getStorageFormat()); task.setInvertedIndexFileStorageFormat(tbl.getInvertedIndexFileStorageFormat()); @@ -2305,10 +2310,10 @@ private boolean createOlapTable(Database db, CreateTableInfo createTableInfo) th } BinlogConfig createTableBinlogConfig = new BinlogConfig(dbBinlogConfig); createTableBinlogConfig.mergeFromProperties(createTableInfo.getProperties()); - if (dbBinlogConfig.isEnable() && !createTableBinlogConfig.isEnable() && !createTableInfo.isTemp()) { + if (dbBinlogConfig.getEnable() && !createTableBinlogConfig.isEnableForCCR() && !createTableInfo.isTemp()) { throw new DdlException("Cannot create table with binlog disabled when database binlog enable"); } - if (createTableInfo.isTemp() && createTableBinlogConfig.isEnable()) { + if (createTableInfo.isTemp() && createTableBinlogConfig.isEnableForCCR()) { throw new DdlException("Cannot create temporary table with binlog enable"); } createTableInfo.getProperties().putAll(createTableBinlogConfig.toProperties()); @@ -2835,6 +2840,24 @@ private boolean createOlapTable(Database db, CreateTableInfo createTableInfo) th if (binlogConfigMap != null) { BinlogConfig binlogConfig = new BinlogConfig(); binlogConfig.mergeFromProperties(binlogConfigMap); + if (binlogConfig.isEnableForStreaming()) { + if (!(keysType == KeysType.DUP_KEYS + || (keysType == KeysType.UNIQUE_KEYS && enableUniqueKeyMergeOnWrite))) { + throw new AnalysisException("Only duplicate and mow table model support binlog, " + + "if you want to use mor or aggregate table model, " + + "please use binlog with snapshot"); + } + if (keysType == KeysType.DUP_KEYS && binlogConfig.getNeedHistoricalValue()) { + throw new AnalysisException("Duplicate table model don't support record historical value"); + } + for (Column column : baseSchema) { + if (column.isAutoInc()) { + throw new AnalysisException("auto-inc column can't be created on table with binlog"); + } else if (column.getDataType().isVariantType()) { + throw new AnalysisException("variant column can't be created on table with binlog"); + } + } + } olapTable.setBinlogConfig(binlogConfig); } } catch (AnalysisException e) { @@ -2940,6 +2963,10 @@ private boolean createOlapTable(Database db, CreateTableInfo createTableInfo) th olapTable.setIndexMeta(baseIndexId, tableName, baseSchema, schemaVersion, schemaHash, shortKeyColumnCount, baseIndexStorageType, keysType, olapTable.getIndexes()); + if (olapTable.getBinlogConfig().isEnableForStreaming()) { + olapTable.createNewRowBinlogMeta(idGeneratorBuffer); + } + for (AlterOp alterOp : createTableInfo.getAddRollupOps()) { if (olapTable.isDuplicateWithoutKey()) { throw new DdlException("Duplicate table without keys do not support add rollup!"); diff --git a/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java b/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java index cf7df808eda14d..289c949b608ca3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java @@ -1060,6 +1060,10 @@ private static void deleteFromMeta(ListMultimap tabletDeleteFromMeta ? olapTable.getCopiedIndexes() : null; List rowStoreColumns = olapTable.getTableProperty().getCopiedRowStoreColumns(); + MaterializedIndexMeta rowBinlogIndexMeta = null; + if (olapTable.needRowBinlog() && indexId == olapTable.getBaseIndexId()) { + rowBinlogIndexMeta = olapTable.getRowBinlogMeta(); + } CreateReplicaTask createReplicaTask = new CreateReplicaTask(backendId, dbId, tableId, partitionId, indexId, tabletId, replica.getId(), indexMeta.getShortKeyColumnCount(), @@ -1090,7 +1094,8 @@ private static void deleteFromMeta(ListMultimap tabletDeleteFromMeta olapTable.storagePageSize(), olapTable.getTDEAlgorithm(), olapTable.storageDictPageSize(), olapTable.getColumnSeqMapping(), - olapTable.getVerticalCompactionNumColumnsPerGroup()); + olapTable.getVerticalCompactionNumColumnsPerGroup(), + rowBinlogIndexMeta); createReplicaTask.setIsRecoverTask(true); createReplicaTask.setInvertedIndexFileStorageFormat(olapTable .getInvertedIndexFileStorageFormat()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowDataCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowDataCommand.java index c6337c0bd14b1b..8f6b66840aead2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowDataCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ShowDataCommand.java @@ -73,6 +73,7 @@ public class ShowDataCommand extends ShowCommand { .addColumn(new Column("DbName", ScalarType.createVarchar(20))) .addColumn(new Column("Size", ScalarType.createVarchar(30))) .addColumn(new Column("RemoteSize", ScalarType.createVarchar(30))) + .addColumn(new Column("BinlogSize", ScalarType.createVarchar(30))) .addColumn(new Column("RecycleSize", ScalarType.createVarchar(30))) .addColumn(new Column("RecycleRemoteSize", ScalarType.createVarchar(30))) .build(); @@ -83,6 +84,7 @@ public class ShowDataCommand extends ShowCommand { .addColumn(new Column("Size", ScalarType.createVarchar(30))) .addColumn(new Column("ReplicaCount", ScalarType.createVarchar(20))) .addColumn(new Column("RemoteSize", ScalarType.createVarchar(30))) + .addColumn(new Column("BinlogSize", ScalarType.createVarchar(30))) .build(); private static final ShowResultSetMetaData SHOW_WAREHOUSE_DATA_META_DATA = @@ -90,6 +92,7 @@ public class ShowDataCommand extends ShowCommand { .addColumn(new Column("DBName", ScalarType.createVarchar(20))) .addColumn(new Column("DataSize", ScalarType.createVarchar(20))) .addColumn(new Column("RecycleSize", ScalarType.createVarchar(20))) + .addColumn(new Column("BinlogSize", ScalarType.createVarchar(20))) .build(); private static final ShowResultSetMetaData SHOW_INDEX_DATA_META_DATA = @@ -100,15 +103,16 @@ public class ShowDataCommand extends ShowCommand { .addColumn(new Column("ReplicaCount", ScalarType.createVarchar(20))) .addColumn(new Column("RowCount", ScalarType.createVarchar(20))) .addColumn(new Column("RemoteSize", ScalarType.createVarchar(30))) + .addColumn(new Column("BinlogSize", ScalarType.createVarchar(30))) .build(); private static final ImmutableList SHOW_TABLE_DATA_META_DATA_ORIGIN = new ImmutableList.Builder().add("TableName").add("Size").add("ReplicaCount") - .add("RemoteSize").build(); + .add("RemoteSize").add("BinlogSize").build(); private static final ImmutableList SHOW_INDEX_DATA_META_DATA_ORIGIN = new ImmutableList.Builder().add("TableName").add("IndexName").add("Size").add("ReplicaCount") - .add("RowCount").add("RemoteSize").build(); + .add("RowCount").add("RemoteSize").add("BinlogSize").build(); private static final ShowResultSetMetaData SHOW_DETAILED_TABLE_DATA_META_DATA = ShowResultSetMetaData.builder() @@ -120,6 +124,7 @@ public class ShowDataCommand extends ShowCommand { .addColumn(new Column("RemoteTotalSize", ScalarType.createVarchar(30))) .addColumn(new Column("RemoteDataSize", ScalarType.createVarchar(30))) .addColumn(new Column("RemoteIndexSize", ScalarType.createVarchar(30))) + .addColumn(new Column("LocalBinlogSize", ScalarType.createVarchar(30))) .build(); private static final ShowResultSetMetaData SHOW_DETAILED_INDEX_DATA_META_DATA = @@ -134,6 +139,7 @@ public class ShowDataCommand extends ShowCommand { .addColumn(new Column("RemoteTotalSize", ScalarType.createVarchar(30))) .addColumn(new Column("RemoteDataSize", ScalarType.createVarchar(30))) .addColumn(new Column("RemoteIndexSize", ScalarType.createVarchar(30))) + .addColumn(new Column("LocalBinlogSize", ScalarType.createVarchar(30))) .build(); private static final String WAREHOUSE = "entire_warehouse"; @@ -153,6 +159,7 @@ public class ShowDataCommand extends ShowCommand { private long totalSize = 0; private long totalReplicaCount = 0; private long totalRemoteSize = 0; + private long totalBinlogSize = 0; private long totalLocalInvertedSize = 0; private long totalLocalSegmentSize = 0; private long totalRemoteInvertedSize = 0; @@ -260,10 +267,12 @@ public int compare(Table t1, Table t2) { long tableSize = 0; long replicaCount = 0; long remoteSize = 0; + long binlogSize = 0; tableSize = olapTable.getDataSize(); replicaCount = olapTable.getReplicaCount(); remoteSize = olapTable.getRemoteDataSize(); + binlogSize = olapTable.getBinlogSize(); boolean useDisplayName = false; if (!isAdmin && olapTable.isTemporary()) { @@ -271,14 +280,14 @@ public int compare(Table t1, Table t2) { } String tableName = useDisplayName ? olapTable.getDisplayName() : olapTable.getName(); if (!detailed) { - totalRowsObject.add(Arrays.asList(tableName, tableSize, replicaCount, remoteSize)); + totalRowsObject.add(Arrays.asList(tableName, tableSize, replicaCount, remoteSize, binlogSize)); } else { long localIndexSize = olapTable.getLocalIndexFileSize(); long localSegmentSize = olapTable.getLocalSegmentSize(); long remoteIndexSize = olapTable.getRemoteIndexFileSize(); long remoteSegmentSize = olapTable.getRemoteSegmentSize(); totalRowsObject.add(Arrays.asList(table.getName(), tableSize, replicaCount, remoteSize, - localIndexSize, localSegmentSize, remoteIndexSize, remoteSegmentSize)); + localIndexSize, localSegmentSize, remoteIndexSize, remoteSegmentSize, binlogSize)); totalLocalInvertedSize += localIndexSize; totalLocalSegmentSize += localSegmentSize; totalRemoteInvertedSize += remoteIndexSize; @@ -288,6 +297,7 @@ public int compare(Table t1, Table t2) { totalSize += tableSize; totalReplicaCount += replicaCount; totalRemoteSize += remoteSize; + totalBinlogSize += binlogSize; } // end for tables } @@ -308,6 +318,7 @@ private void collectTableStats(OlapTable table) { long localSegmentSize = 0; long remoteIndexSize = 0; long remoteSegmentSize = 0; + long localBinlogSize = 0; for (Partition partition : table.getAllPartitions()) { MaterializedIndex mIndex = partition.getIndex(indexId); indexSize += mIndex.getDataSize(false, false); @@ -318,15 +329,17 @@ private void collectTableStats(OlapTable table) { localSegmentSize += mIndex.getLocalSegmentSize(); remoteIndexSize += mIndex.getRemoteIndexSize(); remoteSegmentSize += mIndex.getRemoteSegmentSize(); + localBinlogSize += mIndex.getBinlogSize(); } String indexName = table.getIndexNameById(indexId); if (!detailed) { totalRowsObject.add(Arrays.asList(tableName, indexName, indexSize, indexReplicaCount, - indexRowCount, indexRemoteSize)); + indexRowCount, indexRemoteSize, localBinlogSize)); } else { totalRowsObject.add(Arrays.asList(tableName, indexName, indexSize, indexReplicaCount, indexRowCount, - indexRemoteSize, localIndexSize, localSegmentSize, remoteIndexSize, remoteSegmentSize)); + indexRemoteSize, localIndexSize, localSegmentSize, remoteIndexSize, remoteSegmentSize, + localBinlogSize)); } totalSize += indexSize; @@ -336,6 +349,7 @@ private void collectTableStats(OlapTable table) { totalLocalSegmentSize += localSegmentSize; totalRemoteInvertedSize += remoteIndexSize; totalRemoteSegmentSize += remoteSegmentSize; + totalBinlogSize += localBinlogSize; } // end for indices } @@ -356,13 +370,15 @@ private void buildDbStatsOutput(long quota, long replicaQuota) { if (!detailed) { totalRows.add(Arrays.asList(String.valueOf(row.get(0)), DebugUtil.printByteWithUnit((long) row.get(1)), String.valueOf(row.get(2)), - DebugUtil.printByteWithUnit((long) row.get(3)))); + DebugUtil.printByteWithUnit((long) row.get(3)), + DebugUtil.printByteWithUnit((long) row.get(4)))); } else { totalRows.add(Arrays.asList(String.valueOf(row.get(0)), String.valueOf(row.get(2)), DebugUtil.printByteWithUnit((long) row.get(1)), DebugUtil.printByteWithUnit((long) row.get(5)), DebugUtil.printByteWithUnit((long) row.get(4)), DebugUtil.printByteWithUnit((long) row.get(3)), DebugUtil.printByteWithUnit((long) row.get(7)), - DebugUtil.printByteWithUnit((long) row.get(6)))); + DebugUtil.printByteWithUnit((long) row.get(6)), + DebugUtil.printByteWithUnit((long) row.get(8)))); } } @@ -371,11 +387,12 @@ private void buildDbStatsOutput(long quota, long replicaQuota) { if (!detailed) { totalRows.add(Arrays.asList("Total", DebugUtil.printByteWithUnit(totalSize), - String.valueOf(totalReplicaCount), DebugUtil.printByteWithUnit(totalRemoteSize))); + String.valueOf(totalReplicaCount), DebugUtil.printByteWithUnit(totalRemoteSize), + DebugUtil.printByteWithUnit(totalBinlogSize))); totalRows.add(Arrays.asList("Quota", DebugUtil.printByteWithUnit(quota), - String.valueOf(replicaQuota), "")); + String.valueOf(replicaQuota), "", "")); totalRows.add(Arrays.asList("Left", DebugUtil.printByteWithUnit(left), - String.valueOf(replicaCountLeft), "")); + String.valueOf(replicaCountLeft), "", "")); } else { totalRows.add(Arrays.asList("Total", String.valueOf(totalReplicaCount), DebugUtil.printByteWithUnit(totalSize), @@ -383,7 +400,8 @@ private void buildDbStatsOutput(long quota, long replicaQuota) { DebugUtil.printByteWithUnit(totalLocalInvertedSize), DebugUtil.printByteWithUnit(totalRemoteSize), DebugUtil.printByteWithUnit(totalRemoteSegmentSize), - DebugUtil.printByteWithUnit(totalRemoteInvertedSize))); + DebugUtil.printByteWithUnit(totalRemoteInvertedSize), + DebugUtil.printByteWithUnit(totalBinlogSize))); totalRows.add(Arrays.asList("Quota", String.valueOf(replicaQuota), DebugUtil.printByteWithUnit(quota), "", "", "", "", "")); totalRows.add(Arrays.asList("Left", String.valueOf(replicaCountLeft), @@ -398,34 +416,39 @@ private void buildTableStatsOutput() { if (!detailed) { totalRows.add(Arrays.asList(indexName, String.valueOf(row.get(1)), DebugUtil.printByteWithUnit((long) row.get(2)), String.valueOf(row.get(3)), - String.valueOf(row.get(4)), DebugUtil.printByteWithUnit((long) row.get(5)))); + String.valueOf(row.get(4)), DebugUtil.printByteWithUnit((long) row.get(5)), + DebugUtil.printByteWithUnit((long) row.get(6)))); } else { totalRows.add(Arrays.asList(indexName, String.valueOf(row.get(1)), String.valueOf(row.get(3)), String.valueOf(row.get(4)), DebugUtil.printByteWithUnit((long) row.get(2)), DebugUtil.printByteWithUnit((long) row.get(7)), DebugUtil.printByteWithUnit((long) row.get(6)), DebugUtil.printByteWithUnit((long) row.get(5)), DebugUtil.printByteWithUnit((long) row.get(9)), - DebugUtil.printByteWithUnit((long) row.get(8)))); + DebugUtil.printByteWithUnit((long) row.get(8)), + DebugUtil.printByteWithUnit((long) row.get(10)))); } } // Total if (!detailed) { totalRows.add(Arrays.asList("", "Total", DebugUtil.printByteWithUnit(totalSize), - String.valueOf(totalReplicaCount), "", DebugUtil.printByteWithUnit(totalRemoteSize))); + String.valueOf(totalReplicaCount), "", DebugUtil.printByteWithUnit(totalRemoteSize), + DebugUtil.printByteWithUnit(totalBinlogSize))); } else { totalRows.add(Arrays.asList("", "Total", String.valueOf(totalReplicaCount), "", DebugUtil.printByteWithUnit(totalSize), DebugUtil.printByteWithUnit(totalLocalSegmentSize), DebugUtil.printByteWithUnit(totalLocalInvertedSize), DebugUtil.printByteWithUnit(totalRemoteSize), DebugUtil.printByteWithUnit(totalRemoteSegmentSize), - DebugUtil.printByteWithUnit(totalRemoteInvertedSize))); + DebugUtil.printByteWithUnit(totalRemoteInvertedSize), + DebugUtil.printByteWithUnit(totalBinlogSize))); } } // |TableName|Size|ReplicaCount|RemoteSize| // |TableName|ReplicaCount|LocalTotalSize|LocalDataSize|LocalIndexSize| // |RemoteTotalSize|RemoteDataSize|RemoteIndexSize| + // |LocalBinlogSize| private void getSingleDbStats(Database db) { db.readLock(); long quota = 0; @@ -445,6 +468,7 @@ private void getSingleDbStats(Database db) { // |TableName|IndexName|Size|ReplicaCount|RowCount|RemoteSize| // |TableName|IndexName|ReplicaCount||RowCount|LocalTotalSize |LocalDataSize |LocalIndexSize| // |RemoteTotalSize|RemoteDataSize|RemoteIndexSize| + // |BinlogSize| private void getSingleTableStats(OlapTable table) { table.readLock(); try { @@ -562,7 +586,7 @@ public String toString() { return toSql(); } - // |DBName|DataSize|RecycleSize| + // |DBName|DataSize|RecycleSize|BinlogSize| private boolean getDbStatsByProperties() { if (properties == null) { return false; @@ -577,6 +601,7 @@ private boolean getDbStatsByProperties() { Map dbToDataSize = Env.getCurrentInternalCatalog().getUsedDataQuota(); Map> dbToRecycleSize = Env.getCurrentRecycleBin().getDbToRecycleSize(); Long total = 0L; + Long totalBinlogSize = 0L; Long totalRecycleSize = 0L; if (dbList == null) { for (Map.Entry pair : dbToDataSize.entrySet()) { @@ -584,11 +609,18 @@ private boolean getDbStatsByProperties() { if (db == null) { continue; } + Long binlogSize = 0L; + for (Table table : db.getTables()) { + if (table instanceof OlapTable) { + binlogSize += ((OlapTable) table).getBinlogSize(); + } + } Long recycleSize = dbToRecycleSize.getOrDefault(db.getId(), Pair.of(0L, 0L)).first; List result = Arrays.asList(db.getName(), - String.valueOf(pair.getValue()), String.valueOf(recycleSize)); + String.valueOf(pair.getValue()), String.valueOf(recycleSize), String.valueOf(binlogSize)); totalRows.add(result); total += pair.getValue(); + totalBinlogSize += binlogSize; totalRecycleSize += recycleSize; dbToRecycleSize.remove(db.getId()); } @@ -596,7 +628,7 @@ private boolean getDbStatsByProperties() { // Append left database in recycle bin for (Map.Entry> entry : dbToRecycleSize.entrySet()) { List result = Arrays.asList("NULL:" + entry.getKey(), - "0", String.valueOf(entry.getValue().first)); + "0", String.valueOf(entry.getValue().first), "0"); totalRows.add(result); totalRecycleSize += entry.getValue().first; } @@ -609,23 +641,31 @@ private boolean getDbStatsByProperties() { if (!dbList.contains(db.getName())) { continue; } + Long binlogSize = 0L; + for (Table table : db.getTables()) { + if (table instanceof OlapTable) { + binlogSize += ((OlapTable) table).getBinlogSize(); + } + } Long recycleSize = dbToRecycleSize.getOrDefault(db.getId(), Pair.of(0L, 0L)).first; Long dataSize = dbToDataSize.getOrDefault(databaseName, 0L); - List result = - Arrays.asList(db.getName(), String.valueOf(dataSize), String.valueOf(recycleSize)); + List result = Arrays.asList(db.getName(), String.valueOf(dataSize), + String.valueOf(recycleSize), String.valueOf(binlogSize)); totalRows.add(result); total += dataSize; + totalBinlogSize += binlogSize; totalRecycleSize += recycleSize; } } - List result = Arrays.asList("total", String.valueOf(total), String.valueOf(totalRecycleSize)); + List result = Arrays.asList("total", String.valueOf(total), String.valueOf(totalRecycleSize), + String.valueOf(totalBinlogSize)); totalRows.add(result); return true; } return false; } - // |DbId|DbName|Size|RemoteSize|RecycleSize|RecycleRemoteSize| + // |DbId|DbName|Size|RemoteSize|BinlogSize|RecycleSize|RecycleRemoteSize| private void getAllDbStats() throws AnalysisException { // check auth if (!Env.getCurrentEnv().getAccessManager().checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN)) { @@ -638,6 +678,7 @@ private void getAllDbStats() throws AnalysisException { } long totalSize = 0; long totalRemoteSize = 0; + long totalBinlogSize = 0; long totalRecycleSize = 0; long totalRecycleRemoteSize = 0; Map> dbToRecycleSize = Env.getCurrentRecycleBin().getDbToRecycleSize(); @@ -653,10 +694,19 @@ private void getAllDbStats() throws AnalysisException { dbInfo.add(String.valueOf(db.getId())); dbInfo.add(dbName); Pair usedSize = ((Database) db).getUsedDataSize(); + long usedBinlogSize = 0; + for (Table table : ((Database) db).getTables()) { + if (!table.isManagedTable()) { + continue; + } + usedBinlogSize += ((OlapTable) table).getBinlogSize(); + } dbInfo.add(String.valueOf(usedSize.first)); dbInfo.add(String.valueOf(usedSize.second)); + dbInfo.add(String.valueOf(usedBinlogSize)); totalSize += usedSize.first; totalRemoteSize += usedSize.second; + totalBinlogSize += usedBinlogSize; } finally { db.readUnlock(); } @@ -677,6 +727,7 @@ private void getAllDbStats() throws AnalysisException { dbInfo.add("NULL"); dbInfo.add("0"); dbInfo.add("0"); + dbInfo.add("0"); dbInfo.add(String.valueOf(entry.getValue().first)); dbInfo.add(String.valueOf(entry.getValue().second)); totalRecycleSize += entry.getValue().first; @@ -690,6 +741,7 @@ private void getAllDbStats() throws AnalysisException { dbInfo.add("NULL"); dbInfo.add(String.valueOf(totalSize)); dbInfo.add(String.valueOf(totalRemoteSize)); + dbInfo.add(String.valueOf(totalBinlogSize)); dbInfo.add(String.valueOf(totalRecycleSize)); dbInfo.add(String.valueOf(totalRecycleRemoteSize)); totalRows.add(dbInfo); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AddColumnOp.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AddColumnOp.java index 7843508e43ff04..09c686779a862c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AddColumnOp.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AddColumnOp.java @@ -103,6 +103,11 @@ public boolean needChangeMTMVState() { return false; } + @Override + public boolean allowOpRowBinlog() { + return true; + } + @Override public String toSql() { StringBuilder sb = new StringBuilder(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AddColumnsOp.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AddColumnsOp.java index 242a0883a2e56a..418185ed7557f2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AddColumnsOp.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AddColumnsOp.java @@ -107,6 +107,12 @@ public boolean needChangeMTMVState() { return false; } + @Override + public boolean allowOpRowBinlog() { + // Add columns is considered safe for row binlog tables. + return true; + } + @Override public String toSql() { StringBuilder sb = new StringBuilder(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AddPartitionLikeOp.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AddPartitionLikeOp.java index 63184ad2951f55..4bab7b19fa0b8d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AddPartitionLikeOp.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AddPartitionLikeOp.java @@ -48,6 +48,12 @@ public boolean needChangeMTMVState() { return false; } + @Override + public boolean allowOpRowBinlog() { + // Adding partition like is allowed for row binlog tables. + return true; + } + public String getPartitionName() { return partitionName; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AddPartitionOp.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AddPartitionOp.java index 1bf6002428d273..b02b65bd7e802c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AddPartitionOp.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AddPartitionOp.java @@ -118,6 +118,12 @@ public void validate(ConnectContext ctx) throws UserException { partitionDefinition.validate(properties); } + @Override + public boolean allowOpRowBinlog() { + // Adding partition is allowed for row binlog tables. + return true; + } + public boolean isTempPartition() { return isTempPartition; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AddRollupOp.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AddRollupOp.java index d3abfea5e95cbe..615bc03d5a6cf3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AddRollupOp.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AddRollupOp.java @@ -126,6 +126,12 @@ public Map getProperties() { return this.properties; } + @Override + public boolean allowOpRowBinlog() { + // Adding rollup index is allowed for row binlog tables. + return true; + } + @Override public String toString() { return toSql(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AlterOp.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AlterOp.java index 6916215c8c3cef..b77d8386744d96 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AlterOp.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/AlterOp.java @@ -42,6 +42,13 @@ public AlterOpType getOpType() { public abstract boolean needChangeMTMVState(); + // Whether this alter operation is allowed on tables that enable row binlog. + // Default is false, and only operations that are explicitly marked as safe + // for row binlog should override this to return true. + public boolean allowOpRowBinlog() { + return false; + } + public Map getProperties() { throw new NotImplementedException("AlterOp.getProperties() is not implemented"); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/DropColumnOp.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/DropColumnOp.java index 58ec29a8f8aa1f..be5e91c8a0b7d4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/DropColumnOp.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/DropColumnOp.java @@ -161,6 +161,13 @@ public boolean needChangeMTMVState() { return true; } + @Override + public boolean allowOpRowBinlog() { + // Drop column is considered safe for row binlog tables after whitelist + // check in schema change job. + return true; + } + @Override public String toSql() { StringBuilder sb = new StringBuilder(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/DropPartitionOp.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/DropPartitionOp.java index fe433852640878..2ba211f0dab939 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/DropPartitionOp.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/DropPartitionOp.java @@ -86,6 +86,12 @@ public boolean needChangeMTMVState() { return false; } + @Override + public boolean allowOpRowBinlog() { + // Dropping partition is allowed for row binlog tables. + return true; + } + @Override public String toSql() { StringBuilder sb = new StringBuilder(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/DropRollupOp.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/DropRollupOp.java index 0e45e6e4c91851..1e9cffe2a70c7e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/DropRollupOp.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/DropRollupOp.java @@ -61,6 +61,12 @@ public boolean needChangeMTMVState() { return false; } + @Override + public boolean allowOpRowBinlog() { + // Dropping rollup index is allowed for row binlog tables. + return true; + } + @Override public String toSql() { StringBuilder stringBuilder = new StringBuilder(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ModifyColumnCommentOp.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ModifyColumnCommentOp.java index b1496710153e13..0b1c6ac49558e2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ModifyColumnCommentOp.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ModifyColumnCommentOp.java @@ -70,6 +70,12 @@ public boolean needChangeMTMVState() { return false; } + @Override + public boolean allowOpRowBinlog() { + // Modifying column comment does not change schema, allow on row binlog tables. + return true; + } + @Override public String toSql() { StringBuilder sb = new StringBuilder(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ModifyDistributionOp.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ModifyDistributionOp.java index 2167b657d02d51..945fbc07e66664 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ModifyDistributionOp.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ModifyDistributionOp.java @@ -47,6 +47,12 @@ public boolean needChangeMTMVState() { return false; } + @Override + public boolean allowOpRowBinlog() { + // Changing distribution is allowed for row binlog tables. + return true; + } + @Override public String toSql() { StringBuilder sb = new StringBuilder(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ModifyPartitionOp.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ModifyPartitionOp.java index 793cbed7552333..13247182616da7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ModifyPartitionOp.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ModifyPartitionOp.java @@ -144,6 +144,12 @@ public boolean needChangeMTMVState() { return false; } + @Override + public boolean allowOpRowBinlog() { + // Modify partition does not change schema, allow on row binlog tables. + return true; + } + @Override public String toSql() { StringBuilder sb = new StringBuilder(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ModifyTableCommentOp.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ModifyTableCommentOp.java index 2112eaa3f8b74c..d06aad546513de 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ModifyTableCommentOp.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ModifyTableCommentOp.java @@ -54,6 +54,12 @@ public boolean needChangeMTMVState() { return false; } + @Override + public boolean allowOpRowBinlog() { + // Modifying table comment does not change schema, allow on row binlog tables. + return true; + } + @Override public String toSql() { StringBuilder sb = new StringBuilder(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ModifyTablePropertiesOp.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ModifyTablePropertiesOp.java index 73144e974ec0a4..c8c0d560b99c8b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ModifyTablePropertiesOp.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ModifyTablePropertiesOp.java @@ -69,6 +69,22 @@ public boolean isBeingSynced() { return isBeingSynced; } + @Override + public boolean allowOpRowBinlog() { + // Only allow table property changes that are not related to bloom filter + // when row binlog is enabled. + if (properties == null || properties.isEmpty()) { + return true; + } + // BF related properties are forbidden on row binlog tables. + if (properties.containsKey(PropertyAnalyzer.PROPERTIES_BF_COLUMNS) + || properties.containsKey(PropertyAnalyzer.PROPERTIES_BF_FPP)) { + return false; + } + // Other properties are allowed. + return true; + } + @Override public void validate(ConnectContext ctx) throws UserException { if (properties == null || properties.isEmpty()) { @@ -145,11 +161,10 @@ public void validate(ConnectContext ctx) throws UserException { } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_IS_BEING_SYNCED)) { setIsBeingSynced(Boolean.parseBoolean(properties.getOrDefault( PropertyAnalyzer.PROPERTIES_IS_BEING_SYNCED, "false"))); - } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_BINLOG_ENABLE) - || properties.containsKey(PropertyAnalyzer.PROPERTIES_BINLOG_TTL_SECONDS) - || properties.containsKey(PropertyAnalyzer.PROPERTIES_BINLOG_MAX_BYTES) - || properties.containsKey(PropertyAnalyzer.PROPERTIES_BINLOG_MAX_HISTORY_NUMS)) { - // do nothing, will be alter in SchemaChangeHandler.updateBinlogConfig + } else if (TableProperty.isSamePrefixProperties(properties, PropertyAnalyzer.PROPERTIES_BINLOG_PREFIX)) { + // validate binlog.* properties + PropertyAnalyzer.analyzeBinlogConfig(new java.util.HashMap<>(properties)); + // will be altered in SchemaChangeHandler.updateBinlogConfig } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_COMPACTION_POLICY)) { String compactionPolicy = properties.getOrDefault(PropertyAnalyzer.PROPERTIES_COMPACTION_POLICY, ""); if (compactionPolicy != null diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/RenamePartitionOp.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/RenamePartitionOp.java index 581bf8254f662c..ea9a1a9237e701 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/RenamePartitionOp.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/RenamePartitionOp.java @@ -76,6 +76,12 @@ public boolean needChangeMTMVState() { return false; } + @Override + public boolean allowOpRowBinlog() { + // Renaming partition does not change schema, allow on row binlog tables. + return true; + } + @Override public String toSql() { return "RENAME PARTITION " + partitionName + " " + newPartitionName; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/RenameRollupOp.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/RenameRollupOp.java index 311ed82c8bf177..7362f081c5711b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/RenameRollupOp.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/RenameRollupOp.java @@ -76,6 +76,12 @@ public boolean needChangeMTMVState() { return false; } + @Override + public boolean allowOpRowBinlog() { + // Renaming rollup does not change schema, allow on row binlog tables. + return true; + } + @Override public String toSql() { return "RENAME ROLLUP " + rollupName + " " + newRollupName; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/RenameTableOp.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/RenameTableOp.java index 94f6c403fd9052..4788c03a31976f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/RenameTableOp.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/RenameTableOp.java @@ -66,6 +66,12 @@ public boolean needChangeMTMVState() { return true; } + @Override + public boolean allowOpRowBinlog() { + // Renaming table does not change schema, allow on row binlog tables. + return true; + } + @Override public String toSql() { return "RENAME " + newTableName; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ReplacePartitionOp.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ReplacePartitionOp.java index e2514da42f3e9c..4212bba8dc94d5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ReplacePartitionOp.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ReplacePartitionOp.java @@ -136,6 +136,12 @@ public boolean allowOpMTMV() { return false; } + @Override + public boolean allowOpRowBinlog() { + // Replacing partition does not change schema, allow on row binlog tables. + return true; + } + @Override public boolean needChangeMTMVState() { return false; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ReplaceTableOp.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ReplaceTableOp.java index d02a9c67e1b8a8..2f2c2da8b9773d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ReplaceTableOp.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/ReplaceTableOp.java @@ -85,6 +85,11 @@ public boolean needChangeMTMVState() { return true; } + @Override + public boolean allowOpRowBinlog() { + return true; + } + public boolean isForce() { return isForce; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/task/CreateReplicaTask.java b/fe/fe-core/src/main/java/org/apache/doris/task/CreateReplicaTask.java index 1d10d5c043ab80..ae2bcd1f328ef6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/task/CreateReplicaTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/task/CreateReplicaTask.java @@ -25,6 +25,7 @@ import org.apache.doris.catalog.Index; import org.apache.doris.catalog.IndexToThriftConvertor; import org.apache.doris.catalog.KeysType; +import org.apache.doris.catalog.MaterializedIndexMeta; import org.apache.doris.common.MarkedCountDownLatch; import org.apache.doris.common.Status; import org.apache.doris.common.util.ColumnsUtil; @@ -132,6 +133,8 @@ public class CreateReplicaTask extends AgentTask { private boolean storeRowColumn; private BinlogConfig binlogConfig; + // update binlog schema only when create base index + private MaterializedIndexMeta rowBinlogMeta; private List clusterKeyUids; private Map objectPool; @@ -170,7 +173,8 @@ public CreateReplicaTask(long backendId, long dbId, long tableId, long partition boolean variantEnableFlattenNested, long storagePageSize, TEncryptionAlgorithm tdeAlgorithm, long storageDictPageSize, Map> columnSeqMapping, - int verticalCompactionNumColumnsPerGroup) { + int verticalCompactionNumColumnsPerGroup, + MaterializedIndexMeta rowBinlogMeta) { super(null, backendId, TTaskType.CREATE, dbId, tableId, partitionId, indexId, tabletId); this.replicaId = replicaId; @@ -223,6 +227,7 @@ public CreateReplicaTask(long backendId, long dbId, long tableId, long partition this.storageDictPageSize = storageDictPageSize; this.tdeAlgorithm = tdeAlgorithm; this.columnSeqMapping = columnSeqMapping; + this.rowBinlogMeta = rowBinlogMeta; } public void setIsRecoverTask(boolean isRecoverTask) { @@ -456,6 +461,32 @@ public TCreateTabletReq toThrift() { createTabletReq.setBinlogConfig(binlogConfig.toThrift()); } + if (binlogConfig != null && binlogConfig.isEnableForStreaming() && rowBinlogMeta != null) { + TTabletSchema tRowBinlogSchema = new TTabletSchema(); + tRowBinlogSchema.setShortKeyColumnCount(rowBinlogMeta.getShortKeyColumnCount()); + tRowBinlogSchema.setSchemaHash(rowBinlogMeta.getSchemaHash()); + tRowBinlogSchema.setKeysType(rowBinlogMeta.getKeysType().toThrift()); + tRowBinlogSchema.setStorageType(TStorageType.COLUMN); + + List tRowBinlogColumns = null; + List rowBinlogColumns = rowBinlogMeta.getSchema(true); + Object tRowBinlogCols = objectPool.get(rowBinlogColumns); + if (tRowBinlogCols != null) { + tRowBinlogColumns = (List) tRowBinlogCols; + } else { + tRowBinlogColumns = new ArrayList<>(); + for (int i = 0; i < rowBinlogColumns.size(); i++) { + Column column = rowBinlogColumns.get(i); + TColumn tColumn = ColumnToThrift.toThrift(column); + tColumn.setVisible(column.isVisible()); + tRowBinlogColumns.add(tColumn); + } + objectPool.put(rowBinlogColumns, tRowBinlogColumns); + } + tRowBinlogSchema.setColumns(tRowBinlogColumns); + createTabletReq.setRowBinlogSchema(tRowBinlogSchema); + } + return createTabletReq; } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/alter/SchemaChangeHandlerTest.java b/fe/fe-core/src/test/java/org/apache/doris/alter/SchemaChangeHandlerTest.java index c1b11a32496a0d..9327bdc17ecb96 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/alter/SchemaChangeHandlerTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/alter/SchemaChangeHandlerTest.java @@ -46,7 +46,9 @@ import org.junit.jupiter.api.Test; import org.mockito.Mockito; +import java.util.List; import java.util.Map; +import java.util.stream.Collectors; public class SchemaChangeHandlerTest extends TestWithFeService { private static final Logger LOG = LogManager.getLogger(SchemaChangeHandlerTest.class); @@ -163,6 +165,238 @@ private void expectException(String alterStmt, String expectedErrorMsg) { } } + @Test + public void testWithRowBinlogSchemaChangeNoHistoricalValue() throws Exception { + String tableName = "binlog_no_hist"; + String create = "CREATE TABLE IF NOT EXISTS test." + tableName + " (\n" + + "k1 INT NOT NULL,\n" + + "v1 INT\n" + + ")\n" + + "UNIQUE KEY(k1)\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 1\n" + + "PROPERTIES('replication_num'='1','light_schema_change'='true'," + + "'enable_unique_key_merge_on_write'='true'," + + "'binlog.enable'='true','binlog.format'='ROW','binlog.need_historical_value'='false');"; + createTable(create); + + Database db = Env.getCurrentInternalCatalog().getDbOrMetaException("test"); + OlapTable tbl = (OlapTable) db.getTableOrMetaException(tableName, Table.TableType.OLAP); + + List cols = tbl.getRowBinlogMeta().getSchema(true).stream().map(Column::getName) + .collect(Collectors.toList()); + Assert.assertFalse(cols.contains(Column.generateBeforeColName("v1"))); + + // single add column + alterTable("ALTER TABLE test." + tableName + " ADD COLUMN v2 INT AFTER v1", connectContext); + jobSize++; + waitAlterJobDone(Env.getCurrentEnv().getSchemaChangeHandler().getAlterJobsV2()); + + cols = tbl.getRowBinlogMeta().getSchema(true).stream().map(Column::getName).collect(Collectors.toList()); + Assert.assertEquals(2, cols.indexOf("v2")); + Assert.assertEquals(3, cols.indexOf(Column.BINLOG_LSN_COL)); + Assert.assertFalse(cols.contains(Column.generateBeforeColName("v2"))); + + // multiple add column clauses in one ALTER + alterTable("ALTER TABLE test." + tableName + + " ADD COLUMN v3 INT AFTER v2, ADD COLUMN v4 INT AFTER v3", connectContext); + jobSize++; + waitAlterJobDone(Env.getCurrentEnv().getSchemaChangeHandler().getAlterJobsV2()); + + cols = tbl.getRowBinlogMeta().getSchema(true).stream().map(Column::getName).collect(Collectors.toList()); + Assert.assertEquals(3, cols.indexOf("v3")); + Assert.assertEquals(4, cols.indexOf("v4")); + Assert.assertEquals(5, cols.indexOf(Column.BINLOG_LSN_COL)); + Assert.assertFalse(cols.contains(Column.generateBeforeColName("v3"))); + Assert.assertFalse(cols.contains(Column.generateBeforeColName("v4"))); + + // AddColumnsOp: ADD COLUMN (colDef1, colDef2) + alterTable("ALTER TABLE test." + tableName + " ADD COLUMN (v5 INT, v6 INT)", connectContext); + jobSize++; + waitAlterJobDone(Env.getCurrentEnv().getSchemaChangeHandler().getAlterJobsV2()); + + cols = tbl.getRowBinlogMeta().getSchema(true).stream().map(Column::getName).collect(Collectors.toList()); + Assert.assertEquals(5, cols.indexOf("v5")); + Assert.assertEquals(6, cols.indexOf("v6")); + Assert.assertEquals(7, cols.indexOf(Column.BINLOG_LSN_COL)); + Assert.assertFalse(cols.contains(Column.generateBeforeColName("v5"))); + Assert.assertFalse(cols.contains(Column.generateBeforeColName("v6"))); + + // drop column + alterTable("ALTER TABLE test." + tableName + " DROP COLUMN v6", connectContext); + jobSize++; + waitAlterJobDone(Env.getCurrentEnv().getSchemaChangeHandler().getAlterJobsV2()); + + cols = tbl.getRowBinlogMeta().getSchema(true).stream().map(Column::getName).collect(Collectors.toList()); + Assert.assertFalse(cols.contains("v6")); + Assert.assertEquals(6, cols.indexOf(Column.BINLOG_LSN_COL)); + } + + @Test + public void testWithRowBinlogSchemaChangeWithHistoricalValue() throws Exception { + String tableName = "binlog_hist"; + String create = "CREATE TABLE IF NOT EXISTS test." + tableName + " (\n" + + "k1 INT NOT NULL,\n" + + "v1 INT\n" + + ")\n" + + "UNIQUE KEY(k1)\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 1\n" + + "PROPERTIES('replication_num'='1','light_schema_change'='true'," + + "'enable_unique_key_merge_on_write'='true'," + + "'binlog.enable'='true','binlog.format'='ROW','binlog.need_historical_value'='true');"; + createTable(create); + + Database db = Env.getCurrentInternalCatalog().getDbOrMetaException("test"); + OlapTable tbl = (OlapTable) db.getTableOrMetaException(tableName, Table.TableType.OLAP); + + List cols = tbl.getRowBinlogMeta().getSchema(true).stream().map(Column::getName) + .collect(Collectors.toList()); + Assert.assertTrue(cols.contains(Column.generateBeforeColName("v1"))); + + // single add column + alterTable("ALTER TABLE test." + tableName + " ADD COLUMN v2 INT AFTER v1", connectContext); + jobSize++; + waitAlterJobDone(Env.getCurrentEnv().getSchemaChangeHandler().getAlterJobsV2()); + + cols = tbl.getRowBinlogMeta().getSchema(true).stream().map(Column::getName).collect(Collectors.toList()); + Assert.assertEquals(2, cols.indexOf("v2")); + Assert.assertTrue(cols.contains(Column.generateBeforeColName("v2"))); + Assert.assertEquals(cols.indexOf(Column.generateBeforeColName("v1")) + 1, + cols.indexOf(Column.generateBeforeColName("v2"))); + + // multiple add column clauses in one ALTER + alterTable("ALTER TABLE test." + tableName + + " ADD COLUMN v3 INT AFTER v2, ADD COLUMN v4 INT AFTER v3", connectContext); + jobSize++; + waitAlterJobDone(Env.getCurrentEnv().getSchemaChangeHandler().getAlterJobsV2()); + + cols = tbl.getRowBinlogMeta().getSchema(true).stream().map(Column::getName).collect(Collectors.toList()); + Assert.assertEquals(3, cols.indexOf("v3")); + Assert.assertEquals(4, cols.indexOf("v4")); + Assert.assertTrue(cols.contains(Column.generateBeforeColName("v3"))); + Assert.assertTrue(cols.contains(Column.generateBeforeColName("v4"))); + Assert.assertEquals(cols.indexOf(Column.generateBeforeColName("v2")) + 1, + cols.indexOf(Column.generateBeforeColName("v3"))); + Assert.assertEquals(cols.indexOf(Column.generateBeforeColName("v3")) + 1, + cols.indexOf(Column.generateBeforeColName("v4"))); + + // AddColumnsOp: ADD COLUMN (colDef1, colDef2) + alterTable("ALTER TABLE test." + tableName + " ADD COLUMN (v5 INT, v6 INT)", connectContext); + jobSize++; + waitAlterJobDone(Env.getCurrentEnv().getSchemaChangeHandler().getAlterJobsV2()); + + cols = tbl.getRowBinlogMeta().getSchema(true).stream().map(Column::getName).collect(Collectors.toList()); + Assert.assertEquals(5, cols.indexOf("v5")); + Assert.assertEquals(6, cols.indexOf("v6")); + Assert.assertTrue(cols.contains(Column.generateBeforeColName("v5"))); + Assert.assertTrue(cols.contains(Column.generateBeforeColName("v6"))); + Assert.assertEquals(cols.indexOf(Column.generateBeforeColName("v4")) + 1, + cols.indexOf(Column.generateBeforeColName("v5"))); + Assert.assertEquals(cols.indexOf(Column.generateBeforeColName("v5")) + 1, + cols.indexOf(Column.generateBeforeColName("v6"))); + + // drop column + alterTable("ALTER TABLE test." + tableName + " DROP COLUMN v6", connectContext); + jobSize++; + waitAlterJobDone(Env.getCurrentEnv().getSchemaChangeHandler().getAlterJobsV2()); + cols = tbl.getRowBinlogMeta().getSchema(true).stream().map(Column::getName).collect(Collectors.toList()); + Assert.assertFalse(cols.contains("v6")); + Assert.assertFalse(cols.contains(Column.generateBeforeColName("v6"))); + } + + @Test + public void testWithRowBinlogOpNotSupported() throws Exception { + // 1) MODIFY COLUMN not supported + String tableName = "binlog_mod"; + String create = "CREATE TABLE IF NOT EXISTS test." + tableName + " (\n" + + "k1 INT NOT NULL,\n" + + "v1 INT\n" + + ")\n" + + "UNIQUE KEY(k1)\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 1\n" + + "PROPERTIES('replication_num'='1','light_schema_change'='true'," + + "'enable_unique_key_merge_on_write'='true'," + + "'binlog.enable'='true','binlog.format'='ROW','binlog.need_historical_value'='false');"; + createTable(create); + expectException("ALTER TABLE test." + tableName + " MODIFY COLUMN v1 BIGINT", "Table With Row Binlog"); + + // 2) VARIANT not supported + String createVariant = "CREATE TABLE test.binlog_variant (k1 INT NOT NULL, v1 VARIANT) " + + "UNIQUE KEY(k1) DISTRIBUTED BY HASH(k1) BUCKETS 1 " + + "PROPERTIES('replication_num'='1','light_schema_change'='true'," + + "'enable_unique_key_merge_on_write'='true'," + + "'binlog.enable'='true','binlog.format'='ROW');"; + try { + createTable(createVariant); + Assertions.fail("Expected exception for VARIANT column"); + } catch (Exception e) { + Assert.assertTrue(e.getMessage().toLowerCase().contains("variant")); + } + + String tableName2 = "binlog_add_variant"; + String create2 = "CREATE TABLE test." + tableName2 + " (k1 INT NOT NULL, v1 INT) " + + "UNIQUE KEY(k1) DISTRIBUTED BY HASH(k1) BUCKETS 1 " + + "PROPERTIES('replication_num'='1','light_schema_change'='true'," + + "'enable_unique_key_merge_on_write'='true'," + + "'binlog.enable'='true','binlog.format'='ROW');"; + createTable(create2); + expectException("ALTER TABLE test." + tableName2 + " ADD COLUMN v2 VARIANT", "VARIANT"); + + // 3) AUTO_INCREMENT not supported + String createAutoinc = "CREATE TABLE test.binlog_autoinc (k1 BIGINT NOT NULL AUTO_INCREMENT, v1 INT) " + + "UNIQUE KEY(k1) DISTRIBUTED BY HASH(k1) BUCKETS 1 " + + "PROPERTIES('replication_num'='1','light_schema_change'='true'," + + "'enable_unique_key_merge_on_write'='true'," + + "'binlog.enable'='true','binlog.format'='ROW');"; + try { + createTable(createAutoinc); + Assertions.fail("Expected exception for AUTO_INCREMENT column"); + } catch (Exception e) { + Assert.assertTrue(e.getMessage().toLowerCase().contains("auto")); + } + } + + @Test + public void testWithRowBinlogPartitionOps() throws Exception { + String tableName = "row_binlog_part"; + String create = "CREATE TABLE IF NOT EXISTS test." + tableName + " (\n" + + "k1 INT NOT NULL\n" + + ")\n" + + "DUPLICATE KEY(k1)\n" + + "PARTITION BY RANGE(k1) (\n" + + "PARTITION p1 VALUES LESS THAN (\"10\"),\n" + + "PARTITION p2 VALUES LESS THAN (\"20\")\n" + + ")\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 1\n" + + "PROPERTIES('replication_num'='1','binlog.enable'='true','binlog.format'='ROW');"; + createTable(create); + + alterTable("ALTER TABLE test." + tableName + " ADD PARTITION p3 VALUES LESS THAN (\"30\")", + connectContext); + alterTable("ALTER TABLE test." + tableName + + " ADD TEMPORARY PARTITION tp1 VALUES LESS THAN (\"10\")", + connectContext); + alterTable("ALTER TABLE test." + tableName + + " REPLACE PARTITION (p1) WITH TEMPORARY PARTITION (tp1)", + connectContext); + alterTable("ALTER TABLE test." + tableName + " DROP PARTITION p2", connectContext); + } + + @Test + public void testReplaceTableWithRowBinlog() throws Exception { + String target = "CREATE TABLE test.row_binlog_replace_target (k1 INT) " + + "DUPLICATE KEY(k1) DISTRIBUTED BY HASH(k1) BUCKETS 1 " + + "PROPERTIES('replication_num'='1','binlog.enable'='true','binlog.format'='ROW');"; + String source = "CREATE TABLE test.row_binlog_replace_source (k1 INT) " + + "DUPLICATE KEY(k1) DISTRIBUTED BY HASH(k1) BUCKETS 1 " + + "PROPERTIES('replication_num'='1');"; + createTable(target); + createTable(source); + String alterStmt = "ALTER TABLE test.row_binlog_replace_target REPLACE WITH TABLE " + + "row_binlog_replace_source PROPERTIES('swap' = 'true')"; + alterTable(alterStmt, connectContext); + waitAlterJobDone(Env.getCurrentEnv().getSchemaChangeHandler().getAlterJobsV2()); + } + // In this test we should cover this following cases: // Positive Test Case // 3.1 add sub-column diff --git a/fe/fe-core/src/test/java/org/apache/doris/binlog/BinlogManagerTest.java b/fe/fe-core/src/test/java/org/apache/doris/binlog/BinlogManagerTest.java index 5d4a26b2a6d744..59ffc4c628106d 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/binlog/BinlogManagerTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/binlog/BinlogManagerTest.java @@ -23,6 +23,7 @@ import org.apache.doris.catalog.EnvFactory; import org.apache.doris.common.Config; import org.apache.doris.common.Pair; +import org.apache.doris.common.util.PropertyAnalyzer; import org.apache.doris.datasource.InternalCatalog; import org.apache.doris.persist.BinlogGcInfo; import org.apache.doris.thrift.TBinlog; @@ -94,7 +95,8 @@ public void setUp() { Mockito.withSettings().defaultAnswer(Mockito.CALLS_REAL_METHODS), (mock, context) -> { Mockito.doAnswer(inv -> ttl).when(mock).getTtlSeconds(); - Mockito.doAnswer(inv -> enableDbBinlog).when(mock).isEnable(); + Mockito.doAnswer(inv -> enableDbBinlog).when(mock).getEnable(); + Mockito.doAnswer(inv -> enableDbBinlog).when(mock).isEnableForCCR(); Mockito.doReturn(BinlogTestUtils.MAX_BYTES).when(mock).getMaxBytes(); Mockito.doReturn(BinlogTestUtils.MAX_HISTORY_NUMS).when(mock).getMaxHistoryNums(); }); @@ -153,6 +155,41 @@ public void tearDown() { } } + @Test + public void testBinlogConfigEquals() { + BinlogConfig c1 = new BinlogConfig(true, 10L, 20L, 30L, BinlogConfig.BinlogFormat.ROW, true); + BinlogConfig c2 = new BinlogConfig(true, 10L, 20L, 30L, BinlogConfig.BinlogFormat.ROW, true); + BinlogConfig c3 = new BinlogConfig(true, 10L, 20L, 30L, BinlogConfig.BinlogFormat.ROW, false); + + Assert.assertEquals(c1, c2); + Assert.assertNotEquals(c1, c3); + Assert.assertNotEquals(c1, "not_binlog"); + } + + @Test + public void testBinlogConfigAppendToShowCreateTable() { + BinlogConfig rowCfg = new BinlogConfig(true, 11L, 22L, 33L, BinlogConfig.BinlogFormat.ROW, true); + StringBuilder sb = new StringBuilder(); + rowCfg.appendToShowCreateTable(sb); + String out = sb.toString(); + Assert.assertTrue(out.contains("\"" + PropertyAnalyzer.PROPERTIES_BINLOG_ENABLE + "\" = \"true\"")); + Assert.assertTrue(out.contains("\"" + PropertyAnalyzer.PROPERTIES_BINLOG_TTL_SECONDS + "\" = \"11\"")); + Assert.assertTrue(out.contains("\"" + PropertyAnalyzer.PROPERTIES_BINLOG_MAX_BYTES + "\" = \"22\"")); + Assert.assertTrue(out.contains("\"" + PropertyAnalyzer.PROPERTIES_BINLOG_MAX_HISTORY_NUMS + + "\" = \"33\"")); + Assert.assertTrue(out.contains("\"" + PropertyAnalyzer.PROPERTIES_BINLOG_FORMAT + "\" = \"ROW\"")); + Assert.assertTrue(out.contains(PropertyAnalyzer.PROPERTIES_BINLOG_NEED_HISTORICAL_VALUE)); + + BinlogConfig stmtCfg = new BinlogConfig(true, 11L, 22L, 33L, + BinlogConfig.BinlogFormat.STATEMENT_AND_SNAPSHOT, true); + sb = new StringBuilder(); + stmtCfg.appendToShowCreateTable(sb); + out = sb.toString(); + Assert.assertTrue(out.contains("\"" + PropertyAnalyzer.PROPERTIES_BINLOG_FORMAT + + "\" = \"STATEMENT_AND_SNAPSHOT\"")); + Assert.assertFalse(out.contains(PropertyAnalyzer.PROPERTIES_BINLOG_NEED_HISTORICAL_VALUE)); + } + @Test public void testGetBinlog() throws NoSuchMethodException, InvocationTargetException, IllegalAccessException { diff --git a/fe/fe-core/src/test/java/org/apache/doris/binlog/BinlogTestUtils.java b/fe/fe-core/src/test/java/org/apache/doris/binlog/BinlogTestUtils.java index af5eabfa3d6192..2eff9f02ecfc0c 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/binlog/BinlogTestUtils.java +++ b/fe/fe-core/src/test/java/org/apache/doris/binlog/BinlogTestUtils.java @@ -32,12 +32,18 @@ public class BinlogTestUtils { public static final long MAX_BYTES = 0x7fffffffffffffffL; public static final long MAX_HISTORY_NUMS = 0x7fffffffffffffffL; - public static BinlogConfig newTestBinlogConfig(boolean enableBinlog, long expiredTime) { - return new BinlogConfig(enableBinlog, expiredTime, MAX_BYTES, MAX_HISTORY_NUMS); + public static BinlogConfig newTestCCRBinlogConfig(boolean enableBinlog, long expiredTime) { + return new BinlogConfig(enableBinlog, expiredTime, MAX_BYTES, MAX_HISTORY_NUMS, + BinlogConfig.BinlogFormat.STATEMENT_AND_SNAPSHOT, false); + } + + public static BinlogConfig newTestRowBinlogConfig(boolean enableBinlog, boolean needHistoricalValue) { + return new BinlogConfig(enableBinlog, BinlogConfig.NO_TTL, MAX_BYTES, MAX_HISTORY_NUMS, + BinlogConfig.BinlogFormat.ROW, needHistoricalValue); } public static BinlogConfigCache newMockBinlogConfigCache(long dbId, long tableId, long expiredTime) { - BinlogConfig binlogConfig = newTestBinlogConfig(true, expiredTime); + BinlogConfig binlogConfig = newTestCCRBinlogConfig(true, expiredTime); return new MockBinlogConfigCache( Collections.singletonMap(String.format("%d_%d", dbId, tableId), binlogConfig)); } @@ -45,7 +51,7 @@ public static BinlogConfigCache newMockBinlogConfigCache(long dbId, long tableId public static MockBinlogConfigCache newMockBinlogConfigCache(Map ttlMap) { Map configMap = Maps.newHashMap(); for (Map.Entry entry : ttlMap.entrySet()) { - configMap.put(entry.getKey(), newTestBinlogConfig(true, entry.getValue())); + configMap.put(entry.getKey(), newTestCCRBinlogConfig(true, entry.getValue())); } return new MockBinlogConfigCache(configMap); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/binlog/MockBinlogConfigCache.java b/fe/fe-core/src/test/java/org/apache/doris/binlog/MockBinlogConfigCache.java index d2720bf61d5b9f..2bab9de6063710 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/binlog/MockBinlogConfigCache.java +++ b/fe/fe-core/src/test/java/org/apache/doris/binlog/MockBinlogConfigCache.java @@ -30,12 +30,12 @@ public MockBinlogConfigCache(Map mockedConfigs) { } public void addDbBinlogConfig(long dbId, boolean enableBinlog, long expiredTime) { - BinlogConfig config = BinlogTestUtils.newTestBinlogConfig(enableBinlog, expiredTime); + BinlogConfig config = BinlogTestUtils.newTestCCRBinlogConfig(enableBinlog, expiredTime); mockedConfigs.put(String.valueOf(dbId), config); } public void addTableBinlogConfig(long dbId, long tableId, boolean enableBinlog, long expiredTime) { - BinlogConfig config = BinlogTestUtils.newTestBinlogConfig(enableBinlog, expiredTime); + BinlogConfig config = BinlogTestUtils.newTestCCRBinlogConfig(enableBinlog, expiredTime); mockedConfigs.put(String.format("%d_%d", dbId, tableId), config); } @@ -58,7 +58,7 @@ public boolean isEnableTable(long dbId, long tableId) { public boolean isEnableDB(long dbId) { BinlogConfig config = mockedConfigs.get(String.valueOf(dbId)); if (config != null) { - return config.isEnable(); + return config.getEnable(); } return false; } diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateTableTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateTableTest.java index c81231bf3d0873..3bf05aaa5c5e04 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateTableTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateTableTest.java @@ -242,6 +242,48 @@ public void testNormal() throws DdlException, ConfigException { + "distributed by hash(k2) buckets 1\n" + "properties('replication_num' = '1'); ")); } + @Test + public void testCreateTempTable() throws Exception { + String create = "CREATE TEMPORARY TABLE test.temp_normal (k1 INT) " + + "DUPLICATE KEY(k1) DISTRIBUTED BY HASH(k1) BUCKETS 1 " + + "PROPERTIES('replication_num'='1');"; + createTable(create); + } + + @Test + public void testCreateTempTableWithRowBinlog() throws Exception { + String create = "CREATE TEMPORARY TABLE test.temp_row_binlog (k1 INT) " + + "DUPLICATE KEY(k1) DISTRIBUTED BY HASH(k1) BUCKETS 1 " + + "PROPERTIES('replication_num'='1','binlog.enable'='true','binlog.format'='ROW');"; + createTable(create); + } + + @Test + public void testCreateRowBinlogTable() throws Exception { + String create = "CREATE TABLE test.row_binlog_normal (k1 INT) " + + "DUPLICATE KEY(k1) DISTRIBUTED BY HASH(k1) BUCKETS 1 " + + "PROPERTIES('replication_num'='1','binlog.enable'='true','binlog.format'='ROW');"; + createTable(create); + } + + @Test + public void testCreateRowBinlogUniqueKeyTable() throws Exception { + String create = "CREATE TABLE test.row_binlog_unique (k1 INT, v1 INT) " + + "UNIQUE KEY(k1) DISTRIBUTED BY HASH(k1) BUCKETS 1 " + + "PROPERTIES('replication_num'='1','enable_unique_key_merge_on_write'='true'," + + "'binlog.enable'='true','binlog.format'='ROW');"; + createTable(create); + } + + @Test + public void testCreateRowBinlogAggregateKeyNotSupported() throws Exception { + String create = "CREATE TABLE test.row_binlog_agg (k1 INT, v1 INT SUM) " + + "AGGREGATE KEY(k1) DISTRIBUTED BY HASH(k1) BUCKETS 1 " + + "PROPERTIES('replication_num'='1','binlog.enable'='true','binlog.format'='ROW');"; + Exception exception = Assert.assertThrows(Exception.class, () -> createTable(create)); + Assert.assertTrue(exception.getMessage(), exception.getMessage().contains("binlog")); + } + @Test public void testAbnormal() throws DdlException, ConfigException { ExceptionChecker.expectThrowsWithMsg(DdlException.class, diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/OlapTableRowBinlogSchemaTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/OlapTableRowBinlogSchemaTest.java new file mode 100755 index 00000000000000..f6130782d73b19 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/OlapTableRowBinlogSchemaTest.java @@ -0,0 +1,92 @@ +// 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. + +package org.apache.doris.catalog; + +import org.apache.doris.binlog.BinlogTestUtils; +import org.apache.doris.thrift.TStorageType; + +import com.google.common.collect.Lists; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.List; +import java.util.stream.Collectors; + +public class OlapTableRowBinlogSchemaTest { + + private static OlapTable newTestTable(BinlogConfig binlogConfig) { + long baseIndexId = 1L; + Column key = new Column("k1", PrimitiveType.INT); + key.setIsKey(true); + Column value = new Column("v1", PrimitiveType.INT); + value.setIsKey(false); + List baseSchema = Lists.newArrayList(key, value); + + // Construct a minimal olap table for row binlog schema generation. + OlapTable table = new OlapTable(1L, "tbl", baseSchema, KeysType.PRIMARY_KEYS, null, null); + table.setBaseIndexId(baseIndexId); + MaterializedIndexMeta baseIndexMeta = new MaterializedIndexMeta(baseIndexId, baseSchema, 1, 1, (short) 1, + TStorageType.COLUMN, KeysType.PRIMARY_KEYS, null); + table.addIndexIdToMetaForUnitTest(baseIndexId, baseIndexMeta); + table.addIndexNameToIdForUnitTest("base", baseIndexId); + table.setBinlogConfig(binlogConfig); + + if (binlogConfig.isEnableForStreaming()) { + // Mock row binlog meta by using generated schema to make getRowBinlogMeta() work in pure unit test. + long rowBinlogIndexId = 2L; + List rowBinlogSchema = table.generateTableRowBinlogSchema(); + MaterializedIndexMeta rowBinlogMeta = new MaterializedIndexMeta(rowBinlogIndexId, rowBinlogSchema, 1, 1, + (short) 1, TStorageType.COLUMN, KeysType.DUP_KEYS, null, null, null, null); + rowBinlogMeta.initSchemaColumnUniqueId(); + table.setRowBinlogMeta(rowBinlogMeta, "row_binlog"); + } + return table; + } + + @Test + public void testRowBinlogSchemaOnEnable() { + OlapTable tableWithoutBefore = newTestTable(BinlogTestUtils.newTestRowBinlogConfig(true, false)); + Assertions.assertTrue(tableWithoutBefore.needRowBinlog()); + List tableWithoutBeforeColumns = + tableWithoutBefore.getRowBinlogMeta().getSchema(true).stream().map(Column::getName) + .collect(Collectors.toList()); + Assertions.assertFalse(tableWithoutBeforeColumns.contains(Column.generateBeforeColName("v1"))); + Assertions.assertEquals(tableWithoutBeforeColumns.indexOf(Column.BINLOG_LSN_COL), 2); + Assertions.assertEquals(tableWithoutBeforeColumns.indexOf(Column.BINLOG_OPERATION_COL), 3); + Assertions.assertEquals(tableWithoutBeforeColumns.indexOf(Column.BINLOG_TIMESTAMP_COL), 4); + Assertions.assertEquals(tableWithoutBeforeColumns.size(), 5); + + OlapTable tableWithBefore = newTestTable(BinlogTestUtils.newTestRowBinlogConfig(true, true)); + Assertions.assertTrue(tableWithBefore.needRowBinlog()); + List tableWithBeforeColumns = + tableWithBefore.getRowBinlogMeta().getSchema(true).stream().map(Column::getName) + .collect(Collectors.toList()); + Assertions.assertTrue(tableWithBeforeColumns.contains(Column.generateBeforeColName("v1"))); + Assertions.assertEquals(tableWithBeforeColumns.indexOf(Column.BINLOG_LSN_COL), 3); + Assertions.assertEquals(tableWithBeforeColumns.indexOf(Column.BINLOG_OPERATION_COL), 4); + Assertions.assertEquals(tableWithBeforeColumns.indexOf(Column.BINLOG_TIMESTAMP_COL), 5); + Assertions.assertEquals(tableWithBeforeColumns.size(), 6); + } + + @Test + public void testRowBinlogSchemaOnDisable() { + OlapTable table = newTestTable(BinlogTestUtils.newTestRowBinlogConfig(false, false)); + Assertions.assertFalse(table.needRowBinlog()); + Assertions.assertTrue(table.getBaseIndexMeta().getRowBinlogIndexId() <= 0); + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/OlapTableWrapperTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/OlapTableWrapperTest.java new file mode 100644 index 00000000000000..3d084f9e270057 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/OlapTableWrapperTest.java @@ -0,0 +1,120 @@ +// 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. + +package org.apache.doris.catalog; + +import org.apache.doris.binlog.BinlogTestUtils; +import org.apache.doris.thrift.TStorageType; + +import com.google.common.collect.Lists; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.List; +import java.util.concurrent.TimeUnit; + +public class OlapTableWrapperTest { + + private static OlapTable newTestTable(BinlogConfig binlogConfig) { + long baseIndexId = 1L; + Column key = new Column("k1", PrimitiveType.INT); + key.setIsKey(true); + Column value = new Column("v1", PrimitiveType.INT); + value.setIsKey(false); + List baseSchema = Lists.newArrayList(key, value); + + // Use non-null partition/distribution info so wrapper delegation can be asserted meaningfully. + OlapTable table = new OlapTable(1L, "tbl", baseSchema, KeysType.PRIMARY_KEYS, + new SinglePartitionInfo(), new RandomDistributionInfo(32)); + table.setBaseIndexId(baseIndexId); + MaterializedIndexMeta baseIndexMeta = new MaterializedIndexMeta(baseIndexId, baseSchema, 1, 1, (short) 1, + TStorageType.COLUMN, KeysType.PRIMARY_KEYS, null); + table.addIndexIdToMetaForUnitTest(baseIndexId, baseIndexMeta); + table.addIndexNameToIdForUnitTest("base", baseIndexId); + table.setBinlogConfig(binlogConfig); + + if (binlogConfig.isEnableForStreaming()) { + long rowBinlogIndexId = 2L; + List rowBinlogSchema = table.generateTableRowBinlogSchema(); + MaterializedIndexMeta rowBinlogMeta = new MaterializedIndexMeta(rowBinlogIndexId, rowBinlogSchema, 1, 1, + (short) 1, TStorageType.COLUMN, KeysType.DUP_KEYS, null, null, null, null); + rowBinlogMeta.initSchemaColumnUniqueId(); + table.setRowBinlogMeta(rowBinlogMeta, "row_binlog"); + } + return table; + } + + @Test + public void testOlapTableWrapper() { + OlapTable table = newTestTable(BinlogTestUtils.newTestRowBinlogConfig(false, false)); + OlapTableWrapper wrapper = new OlapTableWrapper(table); + + Assertions.assertEquals(table, wrapper.getOriginTable()); + + // base index id & schema delegation + Assertions.assertEquals(table.getBaseIndexId(), wrapper.getBaseIndexId()); + Assertions.assertEquals(table.getIndexNameById(table.getBaseIndexId()), wrapper.getIndexNameById(table.getBaseIndexId())); + Assertions.assertEquals(table.getIndexMetaByIndexId(table.getBaseIndexId()), + wrapper.getIndexMetaByIndexId(table.getBaseIndexId())); + Assertions.assertEquals(table.getSchemaByIndexId(table.getBaseIndexId()), wrapper.getSchemaByIndexId(table.getBaseIndexId())); + Assertions.assertEquals(table.getIndexSchemaVersion(table.getBaseIndexId()), + wrapper.getIndexSchemaVersion(table.getBaseIndexId())); + + // lock delegation should not throw + wrapper.readLock(); + try { + Assertions.assertNotNull(table.getPartitionInfo()); + Assertions.assertEquals(table.getPartitionInfo(), wrapper.getPartitionInfo()); + } finally { + wrapper.readUnlock(); + } + + // tryReadLock delegation should not throw + boolean locked = wrapper.tryReadLock(1, TimeUnit.SECONDS); + if (locked) { + wrapper.readUnlock(); + } + } + + @Test + public void testRowBinlogTableWrapper() { + OlapTable table = newTestTable(BinlogTestUtils.newTestRowBinlogConfig(true, true)); + Assertions.assertTrue(table.needRowBinlog()); + MaterializedIndexMeta rowBinlogMeta = table.getRowBinlogMeta(); + Assertions.assertNotNull(rowBinlogMeta); + + RowBinlogTableWrapper wrapper = new RowBinlogTableWrapper(table); + + // test getOriginTable + Assertions.assertEquals(table, wrapper.getOriginTable()); + + // test wrapped baseIndexId + Assertions.assertEquals(rowBinlogMeta.getIndexId(), wrapper.getBaseIndexId()); + + // test index name & id delegations + Assertions.assertEquals(table.getIndexNameById(table.getBaseIndexId()), wrapper.getIndexNameById(table.getBaseIndexId())); + Assertions.assertEquals(table.getIndexMetaByIndexId(table.getBaseIndexId()), wrapper.getIndexMetaByIndexId(table.getBaseIndexId())); + + // row binlog index meta should also be reachable through delegation + Assertions.assertEquals(table.getIndexMetaByIndexId(rowBinlogMeta.getIndexId()), + wrapper.getIndexMetaByIndexId(rowBinlogMeta.getIndexId())); + + // test schema delegation + Assertions.assertEquals(table.getSchemaByIndexId(table.getBaseIndexId()), wrapper.getSchemaByIndexId(table.getBaseIndexId())); + Assertions.assertEquals(table.getIndexSchemaVersion(table.getBaseIndexId()), wrapper.getIndexSchemaVersion(table.getBaseIndexId())); + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/common/proc/ProcServiceTest.java b/fe/fe-core/src/test/java/org/apache/doris/common/proc/ProcServiceTest.java index 2feb909cd75ee8..c8f2780f378e6f 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/common/proc/ProcServiceTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/common/proc/ProcServiceTest.java @@ -24,6 +24,8 @@ import org.junit.Before; import org.junit.Test; +import java.util.List; + public class ProcServiceTest { private class EmptyProcNode implements ProcNodeInterface { @Override @@ -173,4 +175,15 @@ public void testOpenFail() { Assert.assertEquals(5, errCount); } + @Test + public void testWithRowBinlogColumnsInShowProc() { + List tabletsTitles = TabletsProcDir.TITLE_NAMES; + Assert.assertTrue(tabletsTitles.contains("BinlogSize")); + Assert.assertTrue(tabletsTitles.contains("BinlogFileNum")); + + List replicasTitles = ReplicasProcNode.TITLE_NAMES; + Assert.assertTrue(replicasTitles.contains("BinlogSize")); + Assert.assertTrue(replicasTitles.contains("BinlogFileNum")); + } + } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/ShowDataCommandTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/ShowDataCommandTest.java index 43a77d04453e00..b1f7f1aea2bd92 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/ShowDataCommandTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/ShowDataCommandTest.java @@ -18,6 +18,7 @@ package org.apache.doris.nereids.trees.plans.commands; import org.apache.doris.backup.CatalogMocker; +import org.apache.doris.catalog.CatalogRecycleBin; import org.apache.doris.catalog.Database; import org.apache.doris.catalog.Env; import org.apache.doris.catalog.KeysType; @@ -28,6 +29,7 @@ import org.apache.doris.catalog.TableIf; import org.apache.doris.catalog.info.TableNameInfo; import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.Pair; import org.apache.doris.datasource.InternalCatalog; import org.apache.doris.mysql.privilege.AccessControllerManager; import org.apache.doris.mysql.privilege.PrivPredicate; @@ -36,6 +38,7 @@ import org.apache.doris.nereids.types.IntegerType; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.QueryState; +import org.apache.doris.qe.ShowResultSet; import com.google.common.collect.ImmutableList; import org.junit.jupiter.api.AfterEach; @@ -59,7 +62,7 @@ public class ShowDataCommandTest { KeysType.AGG_KEYS, new SinglePartitionInfo(), new RandomDistributionInfo(32)); - + private Env env = Mockito.mock(Env.class); private InternalCatalog catalog = Mockito.mock(InternalCatalog.class); private AccessControllerManager accessControllerManager = Mockito.mock(AccessControllerManager.class); @@ -112,29 +115,98 @@ public void testValidateNormal() throws Exception { Map properties = new HashMap<>(); ShowDataCommand command = new ShowDataCommand(tableNameInfo, keys, properties, false); Assertions.assertDoesNotThrow(() -> command.validate(connectContext)); + + // Ensure show data result includes binlog columns in metadata. + Assertions.assertTrue(command.getMetaData().getColumns().stream() + .anyMatch(c -> c.getName().equalsIgnoreCase("BinlogSize")), + "SHOW DATA should contain BinlogSize column"); } @Test - void testValidateNoPrivilege() throws Exception { + public void testValidateShowAllDataNormal() throws Exception { + Mockito.when(connectContext.getDatabase()).thenReturn(CatalogMocker.TEST_DB_NAME); + Mockito.when(connectContext.isSkipAuth()).thenReturn(true); + Mockito.when(catalog.getDbOrAnalysisException(Mockito.anyString())).thenReturn(CatalogMocker.mockDb()); + + SlotReference tableName = new SlotReference("TableName", IntegerType.INSTANCE); + List keys = ImmutableList.of(new OrderKey(tableName, true, false)); + ShowDataCommand command = new ShowDataCommand(null, keys, new HashMap<>(), false); + + Assertions.assertDoesNotThrow(() -> command.validate(connectContext)); + Assertions.assertTrue(command.getMetaData().getColumns().stream() + .anyMatch(c -> c.getName().equalsIgnoreCase("BinlogSize")), + "SHOW DATA should contain BinlogSize column"); + } + + @Test + public void testValidateShowAllDataGetAllDbStats() throws Exception { + CatalogRecycleBin recycleBin = new CatalogRecycleBin(); + mockedEnv.when(Env::getCurrentRecycleBin).thenReturn(recycleBin); + + Mockito.when(accessControllerManager.checkGlobalPriv(connectContext, PrivPredicate.ADMIN)).thenReturn(true); + Mockito.when(catalog.getDbNames()).thenReturn(ImmutableList.of("db1", "db2")); + + Database db1 = Mockito.mock(Database.class); + Database db2 = Mockito.mock(Database.class); + Mockito.when(catalog.getDbNullable("db1")).thenReturn(db1); + Mockito.when(catalog.getDbNullable("db2")).thenReturn(db2); + + OlapTable t1 = Mockito.mock(OlapTable.class); + OlapTable t2 = Mockito.mock(OlapTable.class); + + Mockito.when(db1.getId()).thenReturn(101L); + Mockito.when(db1.getUsedDataSize()).thenReturn(Pair.of(10L, 1L)); + Mockito.when(db1.getTables()).thenReturn(ImmutableList.of(t1)); + Mockito.doNothing().when(db1).readLock(); + Mockito.doNothing().when(db1).readUnlock(); + + Mockito.when(db2.getId()).thenReturn(102L); + Mockito.when(db2.getUsedDataSize()).thenReturn(Pair.of(20L, 2L)); + Mockito.when(db2.getTables()).thenReturn(ImmutableList.of(t2)); + Mockito.doNothing().when(db2).readLock(); + Mockito.doNothing().when(db2).readUnlock(); + + Mockito.when(t1.isManagedTable()).thenReturn(true); + Mockito.when(t2.isManagedTable()).thenReturn(true); + Mockito.when(t1.getBinlogSize()).thenReturn(5L); + Mockito.when(t2.getBinlogSize()).thenReturn(7L); + + SlotReference tableName = new SlotReference("TableName", IntegerType.INSTANCE); + List keys = ImmutableList.of(new OrderKey(tableName, true, false)); + ShowDataCommand command = new ShowDataCommand(null, keys, new HashMap<>(), false); + + ShowResultSet rs = command.doRun(connectContext, null); + List> rows = rs.getResultRows(); + + Assertions.assertEquals(3, rows.size()); + Assertions.assertEquals(ImmutableList.of("101", "db1", "10", "1", "5", "0", "0"), rows.get(0)); + Assertions.assertEquals(ImmutableList.of("102", "db2", "20", "2", "7", "0", "0"), rows.get(1)); + Assertions.assertEquals(ImmutableList.of("Total", "NULL", "30", "3", "12", "0", "0"), rows.get(2)); + } + + @Test + public void testValidateNoPrivilege() throws Exception { Mockito.doReturn(database).when(catalog).getDbOrAnalysisException(Mockito.anyString()); Mockito.doReturn(olapTable).when(database).getTableOrMetaException( - Mockito.anyString(), Mockito.any(TableIf.TableType.class)); + Mockito.eq(tableNameInfo.getTbl()), Mockito.eq(TableIf.TableType.OLAP)); SlotReference tableName = new SlotReference("TableName", IntegerType.INSTANCE); - List keys = ImmutableList.of( - new OrderKey(tableName, true, false) - ); + List keys = ImmutableList.of(new OrderKey(tableName, true, false)); - // test not exist table + // table not exist TableNameInfo tableNameInfoNotExist = new TableNameInfo(CatalogMocker.TEST_DB_NAME, "tbl_not_exist"); - - Map properties = new HashMap<>(); - ShowDataCommand command = new ShowDataCommand(tableNameInfoNotExist, keys, properties, false); + Mockito.doThrow(new AnalysisException("not exist")).when(database) + .getTableOrMetaException(Mockito.eq("tbl_not_exist"), Mockito.eq(TableIf.TableType.OLAP)); + ShowDataCommand command = new ShowDataCommand(tableNameInfoNotExist, keys, new HashMap<>(), false); Assertions.assertThrows(AnalysisException.class, () -> command.validate(connectContext)); - // test no priv - ShowDataCommand command2 = new ShowDataCommand(tableNameInfo, keys, properties, false); + // no privilege + Mockito.when(accessControllerManager.checkTblPriv( + Mockito.nullable(ConnectContext.class), + Mockito.any(TableNameInfo.class), + Mockito.any(PrivPredicate.class))).thenReturn(false); + ShowDataCommand command2 = new ShowDataCommand(tableNameInfo, keys, new HashMap<>(), false); Assertions.assertThrows(AnalysisException.class, () -> command2.validate(connectContext)); } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/TruncateTableCommandTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/TruncateTableCommandTest.java index bdcaec5930209f..e229c4c5f3c1b8 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/TruncateTableCommandTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/TruncateTableCommandTest.java @@ -261,6 +261,23 @@ public void testTruncateTableFailed() throws Exception { } } + @Test + public void testTruncateRowBinlogTable() throws Exception { + String createTableStr = "create table internal.testcommand.tbl_row_binlog(d1 date, k1 int)" + + "duplicate key(d1, k1) " + + "PARTITION BY RANGE(d1)" + + "(PARTITION p20210901 VALUES [('2021-09-01'), ('2021-09-02')))" + + "distributed by hash(k1) buckets 1 " + + "properties('replication_num' = '1', 'binlog.enable'='true', 'binlog.format'='ROW');"; + createTable(createTableStr); + + String truncateStr = "truncate table internal.testcommand.tbl_row_binlog;"; + NereidsParser nereidsParser = new NereidsParser(); + LogicalPlan plan = nereidsParser.parseSingle(truncateStr); + Assertions.assertTrue(plan instanceof TruncateTableCommand); + Env.getCurrentEnv().truncateTable((TruncateTableCommand) plan); + } + private List> checkShowTabletResultNum(String tbl, String partition, int expected) throws Exception { String showStr = "show tablets from " + tbl + " partition(" + partition + ")"; diff --git a/fe/fe-core/src/test/java/org/apache/doris/task/AgentTaskTest.java b/fe/fe-core/src/test/java/org/apache/doris/task/AgentTaskTest.java index a1cef6161bdec7..4a90d7d3f1fd43 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/task/AgentTaskTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/task/AgentTaskTest.java @@ -18,9 +18,12 @@ package org.apache.doris.task; import org.apache.doris.analysis.PartitionValue; +import org.apache.doris.binlog.BinlogTestUtils; import org.apache.doris.catalog.AggregateType; +import org.apache.doris.catalog.BinlogConfig; import org.apache.doris.catalog.Column; import org.apache.doris.catalog.KeysType; +import org.apache.doris.catalog.MaterializedIndexMeta; import org.apache.doris.catalog.PartitionKey; import org.apache.doris.catalog.PrimitiveType; import org.apache.doris.catalog.ScalarType; @@ -85,6 +88,7 @@ public class AgentTaskTest { private Range range1; private Range range2; + private Map objectPool; private AgentTask createReplicaTask; private AgentTask dropTask; @@ -110,13 +114,13 @@ public void setUp() throws AnalysisException { range2 = Range.closedOpen(pk2, pk3); // create tasks - Map objectPool = new HashMap(); + objectPool = new HashMap(); // create createReplicaTask = new CreateReplicaTask(backendId1, dbId, tableId, partitionId, indexId1, tabletId1, replicaId1, shortKeyNum, schemaHash1, version, KeysType.AGG_KEYS, storageType, TStorageMedium.SSD, columns, null, 0, latch, null, false, TTabletType.TABLET_TYPE_DISK, null, TCompressionType.LZ4F, false, "", false, false, false, "", 0, 0, 0, 0, 0, false, null, null, objectPool, rowStorePageSize, false, - storagePageSize, TEncryptionAlgorithm.PLAINTEXT, storageDictPageSize, new HashMap<>(), 5); + storagePageSize, TEncryptionAlgorithm.PLAINTEXT, storageDictPageSize, new HashMap<>(), 5, null); // drop dropTask = new DropReplicaTask(backendId1, tabletId1, replicaId1, schemaHash1, false); @@ -168,6 +172,27 @@ public void toThriftTest() throws Exception { Assert.assertEquals(createReplicaTask.getSignature(), request.getSignature()); Assert.assertNotNull(request.getCreateTabletReq()); + // create with row binlog schema + BinlogConfig binlogConfig = BinlogTestUtils.newTestRowBinlogConfig(true, false); + List rowBinlogColumns = new LinkedList<>(); + rowBinlogColumns.add(new Column("k1", ScalarType.createType(PrimitiveType.INT), true, null, "1", "")); + rowBinlogColumns.add(new Column("v1", ScalarType.createType(PrimitiveType.INT), false, + AggregateType.NONE, "1", "")); + MaterializedIndexMeta rowBinlogMeta = new MaterializedIndexMeta(9999L, rowBinlogColumns, 1, 1, + (short) 1, TStorageType.COLUMN, KeysType.DUP_KEYS, null); + rowBinlogMeta.initSchemaColumnUniqueId(); + + AgentTask createWithRowBinlog = new CreateReplicaTask(backendId1, dbId, tableId, partitionId, + indexId1, tabletId1, replicaId1, shortKeyNum, schemaHash1, version, KeysType.AGG_KEYS, storageType, + TStorageMedium.SSD, columns, null, 0, latch, null, false, TTabletType.TABLET_TYPE_DISK, null, + TCompressionType.LZ4F, false, "", false, false, false, "", 0, 0, 0, 0, 0, false, + binlogConfig, null, objectPool, rowStorePageSize, false, storagePageSize, + TEncryptionAlgorithm.PLAINTEXT, storageDictPageSize, new HashMap<>(), 5, rowBinlogMeta); + TAgentTaskRequest requestWithRowBinlog = + (TAgentTaskRequest) toAgentTaskRequest.invoke(agentBatchTask, createWithRowBinlog); + Assert.assertNotNull(requestWithRowBinlog.getCreateTabletReq()); + Assert.assertNotNull(requestWithRowBinlog.getCreateTabletReq().getRowBinlogSchema()); + // drop TAgentTaskRequest request2 = (TAgentTaskRequest) toAgentTaskRequest.invoke(agentBatchTask, dropTask); Assert.assertEquals(TTaskType.DROP, request2.getTaskType()); diff --git a/gensrc/proto/olap_file.proto b/gensrc/proto/olap_file.proto index 3a72f77cd75e96..5df4af94f918ad 100644 --- a/gensrc/proto/olap_file.proto +++ b/gensrc/proto/olap_file.proto @@ -139,6 +139,7 @@ message RowsetMetaPB { // rows count for each segment repeated int64 num_segment_rows = 56; + optional bool is_row_binlog = 57 [default = false]; // If true, `segments_key_bounds` contains a single aggregated // [rowset_min_key, rowset_max_key] entry instead of per-segment bounds. @@ -250,6 +251,7 @@ message RowsetMetaCloudPB { // rows count for each segment repeated int64 num_segment_rows = 56; + optional bool is_row_binlog = 57 [default = false]; // If true, `segments_key_bounds` contains a single aggregated // [rowset_min_key, rowset_max_key] entry instead of per-segment bounds. @@ -594,11 +596,18 @@ message S3StorageParamPB { optional string root_path = 8; } +enum BinlogFormatPB { + STATEMENT_AND_SNAPSHOT = 0; + ROW = 1; +} + message BinlogConfigPB { optional bool enable = 1; optional int64 ttl_seconds = 2; optional int64 max_bytes = 3; optional int64 max_history_nums = 4; + optional BinlogFormatPB binlog_format = 5; + optional bool need_historical_value = 6; } // ATTN: When adding or deleting fields, please update `message TabletMetaCloudPB` @@ -638,6 +647,9 @@ message TabletMetaPB { optional int64 storage_policy_id = 25; optional PUniqueId cooldown_meta_id = 26; optional BinlogConfigPB binlog_config = 27; + optional TabletSchemaPB row_binlog_schema = 42; + repeated RowsetMetaPB row_binlog_rs_metas = 43; + optional int32 row_binlog_schema_hash = 44; optional string compaction_policy = 28 [default = "size_based"]; optional int64 time_series_compaction_goal_size_mbytes = 29 [default = 1024]; optional int64 time_series_compaction_file_count_threshold = 30 [default = 1000]; @@ -694,6 +706,9 @@ message TabletMetaCloudPB { optional int64 storage_policy_id = 28; optional PUniqueId cooldown_meta_id = 29; optional BinlogConfigPB binlog_config = 30; + optional TabletSchemaCloudPB row_binlog_schema = 42; + repeated RowsetMetaCloudPB row_binlog_rs_metas = 43; + optional int32 row_binlog_schema_hash = 44; optional string compaction_policy = 31 [default = "size_based"]; optional int64 time_series_compaction_goal_size_mbytes = 32 [default = 1024]; optional int64 time_series_compaction_file_count_threshold = 33 [default = 1000]; @@ -721,6 +736,7 @@ message DeleteBitmapPB { repeated int64 versions = 3; // Serialized roaring bitmaps indexed with {rowset_id, segment_id, version} repeated bytes segment_delete_bitmaps = 4; + repeated bool is_binlog_delvec = 5; } message BinlogMetaEntryPB { diff --git a/gensrc/thrift/AgentService.thrift b/gensrc/thrift/AgentService.thrift index c24753f3ec94ab..de1354f4de3e66 100644 --- a/gensrc/thrift/AgentService.thrift +++ b/gensrc/thrift/AgentService.thrift @@ -186,11 +186,18 @@ enum TInvertedIndexStorageFormat { V2 = 2 // Segment id per idx: Indexes are organized based on segment identifiers, grouping indexes by their associated segment. } +enum TBinlogFormat { + STATEMENT_AND_SNAPSHOT = 0, + ROW = 1 +} + struct TBinlogConfig { 1: optional bool enable; 2: optional i64 ttl_seconds; 3: optional i64 max_bytes; 4: optional i64 max_history_nums; + 5: optional TBinlogFormat binlog_format; + 6: optional bool need_historical_value; } struct TCreateTabletReq { @@ -230,6 +237,7 @@ struct TCreateTabletReq { 29: optional Types.TInvertedIndexFileStorageFormat inverted_index_file_storage_format = Types.TInvertedIndexFileStorageFormat.V2 30: optional TEncryptionAlgorithm tde_algorithm 31: optional i32 vertical_compaction_num_columns_per_group = 5 + 32: optional TTabletSchema row_binlog_schema // For cloud 1000: optional bool is_in_memory = false diff --git a/gensrc/thrift/BackendService.thrift b/gensrc/thrift/BackendService.thrift index 44c1e7cd94c619..cc2cffac59e22e 100644 --- a/gensrc/thrift/BackendService.thrift +++ b/gensrc/thrift/BackendService.thrift @@ -40,6 +40,8 @@ struct TTabletStat { 9: optional i64 local_segment_size = 0 // .dat 10: optional i64 remote_index_size = 0 // .idx 11: optional i64 remote_segment_size = 0 // .dat + 12: optional i64 binlog_size = 0 // __row_binlog/xxx.dat + 13: optional i64 binlog_file_num = 0 } struct TTabletStatResult { diff --git a/gensrc/thrift/MasterService.thrift b/gensrc/thrift/MasterService.thrift index 2caf3c7696512d..a468ab9d6a88a4 100644 --- a/gensrc/thrift/MasterService.thrift +++ b/gensrc/thrift/MasterService.thrift @@ -51,7 +51,9 @@ struct TTabletInfo { 23: optional i64 local_segment_size = 0 // .dat 24: optional i64 remote_index_size = 0 // .idx 25: optional i64 remote_segment_size = 0 // .dat - + 26: optional i64 binlog_size = 0 // __row_binlog/xxx.dat + 27: optional i64 binlog_file_num = 0 + // For cloud 1000: optional bool is_persistent } diff --git a/regression-test/data/insert_p0/test_group_commit_interval_ms_property.out b/regression-test/data/insert_p0/test_group_commit_interval_ms_property.out index 4b2a17ea07a0d5..92e0a26d832398 100644 --- a/regression-test/data/insert_p0/test_group_commit_interval_ms_property.out +++ b/regression-test/data/insert_p0/test_group_commit_interval_ms_property.out @@ -1,13 +1,12 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !1 -- -test_group_commit_interval_ms_property_tbl_legacy CREATE TABLE `test_group_commit_interval_ms_property_tbl_legacy` (\n `k` BIGINT NULL,\n `v` BIGINT NULL\n) ENGINE=OLAP\nUNIQUE KEY(`k`)\nCOMMENT 'OLAP'\nDISTRIBUTED BY HASH(`v`) BUCKETS 8\nPROPERTIES (\n"replication_allocation" = "tag.location.default: 1",\n"min_load_replica_num" = "-1",\n"is_being_synced" = "false",\n"storage_format" = "V2",\n"enable_unique_key_merge_on_write" = "false",\n"light_schema_change" = "true",\n"disable_auto_compaction" = "false",\n"binlog.enable" = "false",\n"binlog.ttl_seconds" = "86400",\n"binlog.max_bytes" = "9223372036854775807",\n"binlog.max_history_nums" = "9223372036854775807",\n"enable_single_replica_compaction" = "false",\n"group_commit_interval_ms" = "10000"\n); +test_group_commit_interval_ms_property_tbl_legacy CREATE TABLE `test_group_commit_interval_ms_property_tbl_legacy` (\n `k` BIGINT NULL,\n `v` BIGINT NULL\n) ENGINE=OLAP\nUNIQUE KEY(`k`)\nCOMMENT 'OLAP'\nDISTRIBUTED BY HASH(`v`) BUCKETS 8\nPROPERTIES (\n"replication_allocation" = "tag.location.default: 1",\n"min_load_replica_num" = "-1",\n"is_being_synced" = "false",\n"storage_format" = "V2",\n"enable_unique_key_merge_on_write" = "false",\n"light_schema_change" = "true",\n"disable_auto_compaction" = "false",\n"binlog.enable" = "false",\n"binlog.ttl_seconds" = "86400",\n"binlog.max_bytes" = "9223372036854775807",\n"binlog.max_history_nums" = "9223372036854775807",\n"binlog.format" = "STATEMENT_AND_SNAPSHOT",\n"enable_single_replica_compaction" = "false",\n"group_commit_interval_ms" = "10000"\n); -- !2 -- -test_group_commit_interval_ms_property_tbl_legacy CREATE TABLE `test_group_commit_interval_ms_property_tbl_legacy` (\n `k` BIGINT NULL,\n `v` BIGINT NULL\n) ENGINE=OLAP\nUNIQUE KEY(`k`)\nCOMMENT 'OLAP'\nDISTRIBUTED BY HASH(`v`) BUCKETS 8\nPROPERTIES (\n"replication_allocation" = "tag.location.default: 1",\n"min_load_replica_num" = "-1",\n"is_being_synced" = "false",\n"storage_format" = "V2",\n"enable_unique_key_merge_on_write" = "false",\n"light_schema_change" = "true",\n"disable_auto_compaction" = "false",\n"binlog.enable" = "false",\n"binlog.ttl_seconds" = "86400",\n"binlog.max_bytes" = "9223372036854775807",\n"binlog.max_history_nums" = "9223372036854775807",\n"enable_single_replica_compaction" = "false",\n"group_commit_interval_ms" = "1000"\n); +test_group_commit_interval_ms_property_tbl_legacy CREATE TABLE `test_group_commit_interval_ms_property_tbl_legacy` (\n `k` BIGINT NULL,\n `v` BIGINT NULL\n) ENGINE=OLAP\nUNIQUE KEY(`k`)\nCOMMENT 'OLAP'\nDISTRIBUTED BY HASH(`v`) BUCKETS 8\nPROPERTIES (\n"replication_allocation" = "tag.location.default: 1",\n"min_load_replica_num" = "-1",\n"is_being_synced" = "false",\n"storage_format" = "V2",\n"enable_unique_key_merge_on_write" = "false",\n"light_schema_change" = "true",\n"disable_auto_compaction" = "false",\n"binlog.enable" = "false",\n"binlog.ttl_seconds" = "86400",\n"binlog.max_bytes" = "9223372036854775807",\n"binlog.max_history_nums" = "9223372036854775807",\n"binlog.format" = "STATEMENT_AND_SNAPSHOT",\n"enable_single_replica_compaction" = "false",\n"group_commit_interval_ms" = "1000"\n); -- !1 -- -test_group_commit_interval_ms_property_tbl_nereids CREATE TABLE `test_group_commit_interval_ms_property_tbl_nereids` (\n `k` BIGINT NULL,\n `v` BIGINT NULL\n) ENGINE=OLAP\nUNIQUE KEY(`k`)\nCOMMENT 'OLAP'\nDISTRIBUTED BY HASH(`v`) BUCKETS 8\nPROPERTIES (\n"replication_allocation" = "tag.location.default: 1",\n"min_load_replica_num" = "-1",\n"is_being_synced" = "false",\n"storage_format" = "V2",\n"enable_unique_key_merge_on_write" = "false",\n"light_schema_change" = "true",\n"disable_auto_compaction" = "false",\n"binlog.enable" = "false",\n"binlog.ttl_seconds" = "86400",\n"binlog.max_bytes" = "9223372036854775807",\n"binlog.max_history_nums" = "9223372036854775807",\n"enable_single_replica_compaction" = "false",\n"group_commit_interval_ms" = "10000"\n); +test_group_commit_interval_ms_property_tbl_nereids CREATE TABLE `test_group_commit_interval_ms_property_tbl_nereids` (\n `k` BIGINT NULL,\n `v` BIGINT NULL\n) ENGINE=OLAP\nUNIQUE KEY(`k`)\nCOMMENT 'OLAP'\nDISTRIBUTED BY HASH(`v`) BUCKETS 8\nPROPERTIES (\n"replication_allocation" = "tag.location.default: 1",\n"min_load_replica_num" = "-1",\n"is_being_synced" = "false",\n"storage_format" = "V2",\n"enable_unique_key_merge_on_write" = "false",\n"light_schema_change" = "true",\n"disable_auto_compaction" = "false",\n"binlog.enable" = "false",\n"binlog.ttl_seconds" = "86400",\n"binlog.max_bytes" = "9223372036854775807",\n"binlog.max_history_nums" = "9223372036854775807",\n"binlog.format" = "STATEMENT_AND_SNAPSHOT",\n"enable_single_replica_compaction" = "false",\n"group_commit_interval_ms" = "10000"\n); -- !2 -- -test_group_commit_interval_ms_property_tbl_nereids CREATE TABLE `test_group_commit_interval_ms_property_tbl_nereids` (\n `k` BIGINT NULL,\n `v` BIGINT NULL\n) ENGINE=OLAP\nUNIQUE KEY(`k`)\nCOMMENT 'OLAP'\nDISTRIBUTED BY HASH(`v`) BUCKETS 8\nPROPERTIES (\n"replication_allocation" = "tag.location.default: 1",\n"min_load_replica_num" = "-1",\n"is_being_synced" = "false",\n"storage_format" = "V2",\n"enable_unique_key_merge_on_write" = "false",\n"light_schema_change" = "true",\n"disable_auto_compaction" = "false",\n"binlog.enable" = "false",\n"binlog.ttl_seconds" = "86400",\n"binlog.max_bytes" = "9223372036854775807",\n"binlog.max_history_nums" = "9223372036854775807",\n"enable_single_replica_compaction" = "false",\n"group_commit_interval_ms" = "1000"\n); - +test_group_commit_interval_ms_property_tbl_nereids CREATE TABLE `test_group_commit_interval_ms_property_tbl_nereids` (\n `k` BIGINT NULL,\n `v` BIGINT NULL\n) ENGINE=OLAP\nUNIQUE KEY(`k`)\nCOMMENT 'OLAP'\nDISTRIBUTED BY HASH(`v`) BUCKETS 8\nPROPERTIES (\n"replication_allocation" = "tag.location.default: 1",\n"min_load_replica_num" = "-1",\n"is_being_synced" = "false",\n"storage_format" = "V2",\n"enable_unique_key_merge_on_write" = "false",\n"light_schema_change" = "true",\n"disable_auto_compaction" = "false",\n"binlog.enable" = "false",\n"binlog.ttl_seconds" = "86400",\n"binlog.max_bytes" = "9223372036854775807",\n"binlog.max_history_nums" = "9223372036854775807",\n"binlog.format" = "STATEMENT_AND_SNAPSHOT",\n"enable_single_replica_compaction" = "false",\n"group_commit_interval_ms" = "1000"\n); diff --git a/regression-test/data/query_p0/ddl/alter/test_nereids_alter_database_set_quota.out b/regression-test/data/query_p0/ddl/alter/test_nereids_alter_database_set_quota.out index 3d8e9f59790848..97a8da2fdd1140 100644 --- a/regression-test/data/query_p0/ddl/alter/test_nereids_alter_database_set_quota.out +++ b/regression-test/data/query_p0/ddl/alter/test_nereids_alter_database_set_quota.out @@ -1,31 +1,30 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !show_data_sql -- -Total 0.000 0 0.000 -Quota 8388608.000 TB 1073741824 -Left 8388608.000 TB 1073741824 +Total 0.000 0 0.000 0.000 +Quota 8388608.000 TB 1073741824 +Left 8388608.000 TB 1073741824 -- !show_data_sql_100m -- -Total 0.000 0 0.000 -Quota 100.000 MB 1073741824 -Left 100.000 MB 1073741824 +Total 0.000 0 0.000 0.000 +Quota 100.000 MB 1073741824 +Left 100.000 MB 1073741824 -- !show_data_sql_1024g -- -Total 0.000 0 0.000 -Quota 1024.000 GB 1073741824 -Left 1024.000 GB 1073741824 +Total 0.000 0 0.000 0.000 +Quota 1024.000 GB 1073741824 +Left 1024.000 GB 1073741824 -- !show_data_sql_100t -- -Total 0.000 0 0.000 -Quota 100.000 TB 1073741824 -Left 100.000 TB 1073741824 +Total 0.000 0 0.000 0.000 +Quota 100.000 TB 1073741824 +Left 100.000 TB 1073741824 -- !show_data_sql_10t -- -Total 0.000 0 0.000 -Quota 10.000 TB 1073741824 -Left 10.000 TB 1073741824 +Total 0.000 0 0.000 0.000 +Quota 10.000 TB 1073741824 +Left 10.000 TB 1073741824 -- !show_data_sql_replica_num -- -Total 0.000 0 0.000 -Quota 10.000 TB 102400 -Left 10.000 TB 102400 - +Total 0.000 0 0.000 0.000 +Quota 10.000 TB 102400 +Left 10.000 TB 102400 diff --git a/regression-test/data/query_p0/show/test_show_create_table.out b/regression-test/data/query_p0/show/test_show_create_table.out index 0f621de01726b4..569a2edfae7518 100644 --- a/regression-test/data/query_p0/show/test_show_create_table.out +++ b/regression-test/data/query_p0/show/test_show_create_table.out @@ -1,7 +1,6 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -tb_show_create_table CREATE TABLE `tb_show_create_table` (\n `datek1` date NULL COMMENT 'a',\n `datetimek1` datetime NULL COMMENT 'b',\n `datetimek2` datetime(3) NULL COMMENT 'c',\n `datetimek3` datetime(6) NULL COMMENT 'd',\n `datev1` date MAX NOT NULL COMMENT 'e',\n `datetimev1` datetime MAX NOT NULL COMMENT 'f',\n `datetimev2` datetime(3) MAX NOT NULL COMMENT 'g',\n `datetimev3` datetime(6) MAX NOT NULL COMMENT 'h'\n) ENGINE=OLAP\nAGGREGATE KEY(`datek1`, `datetimek1`, `datetimek2`, `datetimek3`)\nCOMMENT 'OLAP'\nDISTRIBUTED BY HASH(`datek1`) BUCKETS 5\nPROPERTIES (\n"replication_allocation" = "tag.location.default: 1",\n"storage_format" = "V2",\n"light_schema_change" = "true",\n"disable_auto_compaction" = "false",\n"binlog.enable" = "false",\n"binlog.ttl_seconds" = "86400",\n"binlog.max_bytes" = "9223372036854775807",\n"binlog.max_history_nums" = "9223372036854775807",\n"enable_single_replica_compaction" = "false"\n); +tb_show_create_table CREATE TABLE `tb_show_create_table` (\n `datek1` date NULL COMMENT 'a',\n `datetimek1` datetime NULL COMMENT 'b',\n `datetimek2` datetime(3) NULL COMMENT 'c',\n `datetimek3` datetime(6) NULL COMMENT 'd',\n `datev1` date MAX NOT NULL COMMENT 'e',\n `datetimev1` datetime MAX NOT NULL COMMENT 'f',\n `datetimev2` datetime(3) MAX NOT NULL COMMENT 'g',\n `datetimev3` datetime(6) MAX NOT NULL COMMENT 'h'\n) ENGINE=OLAP\nAGGREGATE KEY(`datek1`, `datetimek1`, `datetimek2`, `datetimek3`)\nCOMMENT 'OLAP'\nDISTRIBUTED BY HASH(`datek1`) BUCKETS 5\nPROPERTIES (\n"replication_allocation" = "tag.location.default: 1",\n"storage_format" = "V2",\n"light_schema_change" = "true",\n"disable_auto_compaction" = "false",\n"binlog.enable" = "false",\n"binlog.ttl_seconds" = "86400",\n"binlog.max_bytes" = "9223372036854775807",\n"binlog.max_history_nums" = "9223372036854775807",\n"binlog.format" = "STATEMENT_AND_SNAPSHOT",\n"enable_single_replica_compaction" = "false"\n); -- !select -- -tb_show_create_table CREATE TABLE `tb_show_create_table` (\n `datek1` date NULL COMMENT 'a',\n `datetimek1` datetime NULL COMMENT 'b',\n `datetimek2` datetime(3) NULL COMMENT 'c',\n `datetimek3` datetime(6) NULL COMMENT 'd',\n `datev1` date NOT NULL COMMENT 'e',\n `datetimev1` datetime NOT NULL COMMENT 'f',\n `datetimev2` datetime(3) NOT NULL COMMENT 'g',\n `datetimev3` datetime(6) NOT NULL COMMENT 'h'\n) ENGINE=OLAP\nDUPLICATE KEY(`datek1`, `datetimek1`, `datetimek2`, `datetimek3`)\nCOMMENT 'OLAP'\nDISTRIBUTED BY RANDOM BUCKETS 5\nPROPERTIES (\n"replication_allocation" = "tag.location.default: 1",\n"storage_format" = "V2",\n"light_schema_change" = "true",\n"disable_auto_compaction" = "false",\n"binlog.enable" = "false",\n"binlog.ttl_seconds" = "86400",\n"binlog.max_bytes" = "9223372036854775807",\n"binlog.max_history_nums" = "9223372036854775807",\n"enable_single_replica_compaction" = "false"\n); - +tb_show_create_table CREATE TABLE `tb_show_create_table` (\n `datek1` date NULL COMMENT 'a',\n `datetimek1` datetime NULL COMMENT 'b',\n `datetimek2` datetime(3) NULL COMMENT 'c',\n `datetimek3` datetime(6) NULL COMMENT 'd',\n `datev1` date NOT NULL COMMENT 'e',\n `datetimev1` datetime NOT NULL COMMENT 'f',\n `datetimev2` datetime(3) NOT NULL COMMENT 'g',\n `datetimev3` datetime(6) NOT NULL COMMENT 'h'\n) ENGINE=OLAP\nDUPLICATE KEY(`datek1`, `datetimek1`, `datetimek2`, `datetimek3`)\nCOMMENT 'OLAP'\nDISTRIBUTED BY RANDOM BUCKETS 5\nPROPERTIES (\n"replication_allocation" = "tag.location.default: 1",\n"storage_format" = "V2",\n"light_schema_change" = "true",\n"disable_auto_compaction" = "false",\n"binlog.enable" = "false",\n"binlog.ttl_seconds" = "86400",\n"binlog.max_bytes" = "9223372036854775807",\n"binlog.max_history_nums" = "9223372036854775807",\n"binlog.format" = "STATEMENT_AND_SNAPSHOT",\n"enable_single_replica_compaction" = "false"\n); diff --git a/regression-test/data/query_p0/system/test_table_properties.out b/regression-test/data/query_p0/system/test_table_properties.out index 9f30b6cfde8d48..7561214f2e6cd1 100644 --- a/regression-test/data/query_p0/system/test_table_properties.out +++ b/regression-test/data/query_p0/system/test_table_properties.out @@ -1,12 +1,14 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select_check_1 -- -117 +123 -- !select_check_2 -- internal test_table_properties_db duplicate_table _auto_bucket false internal test_table_properties_db duplicate_table binlog.enable false +internal test_table_properties_db duplicate_table binlog.format STATEMENT_AND_SNAPSHOT internal test_table_properties_db duplicate_table binlog.max_bytes 9223372036854775807 internal test_table_properties_db duplicate_table binlog.max_history_nums 9223372036854775807 +internal test_table_properties_db duplicate_table binlog.need_historical_value false internal test_table_properties_db duplicate_table binlog.ttl_seconds 86400 internal test_table_properties_db duplicate_table compaction_policy size_based internal test_table_properties_db duplicate_table compression ZSTD @@ -43,8 +45,10 @@ internal test_table_properties_db duplicate_table time_series_compaction_time_th internal test_table_properties_db duplicate_table vertical_compaction_num_columns_per_group 5 internal test_table_properties_db listtable _auto_bucket false internal test_table_properties_db listtable binlog.enable false +internal test_table_properties_db listtable binlog.format STATEMENT_AND_SNAPSHOT internal test_table_properties_db listtable binlog.max_bytes 9223372036854775807 internal test_table_properties_db listtable binlog.max_history_nums 9223372036854775807 +internal test_table_properties_db listtable binlog.need_historical_value false internal test_table_properties_db listtable binlog.ttl_seconds 86400 internal test_table_properties_db listtable compaction_policy size_based internal test_table_properties_db listtable compression ZSTD @@ -81,8 +85,10 @@ internal test_table_properties_db listtable time_series_compaction_time_threshol internal test_table_properties_db listtable vertical_compaction_num_columns_per_group 5 internal test_table_properties_db unique_table _auto_bucket false internal test_table_properties_db unique_table binlog.enable false +internal test_table_properties_db unique_table binlog.format STATEMENT_AND_SNAPSHOT internal test_table_properties_db unique_table binlog.max_bytes 9223372036854775807 internal test_table_properties_db unique_table binlog.max_history_nums 9223372036854775807 +internal test_table_properties_db unique_table binlog.need_historical_value false internal test_table_properties_db unique_table binlog.ttl_seconds 86400 internal test_table_properties_db unique_table compaction_policy size_based internal test_table_properties_db unique_table compression ZSTD @@ -121,8 +127,10 @@ internal test_table_properties_db unique_table vertical_compaction_num_columns_p -- !select_check_3 -- internal test_table_properties_db duplicate_table _auto_bucket false internal test_table_properties_db duplicate_table binlog.enable false +internal test_table_properties_db duplicate_table binlog.format STATEMENT_AND_SNAPSHOT internal test_table_properties_db duplicate_table binlog.max_bytes 9223372036854775807 internal test_table_properties_db duplicate_table binlog.max_history_nums 9223372036854775807 +internal test_table_properties_db duplicate_table binlog.need_historical_value false internal test_table_properties_db duplicate_table binlog.ttl_seconds 86400 internal test_table_properties_db duplicate_table compaction_policy size_based internal test_table_properties_db duplicate_table compression ZSTD @@ -159,8 +167,10 @@ internal test_table_properties_db duplicate_table time_series_compaction_time_th internal test_table_properties_db duplicate_table vertical_compaction_num_columns_per_group 5 internal test_table_properties_db unique_table _auto_bucket false internal test_table_properties_db unique_table binlog.enable false +internal test_table_properties_db unique_table binlog.format STATEMENT_AND_SNAPSHOT internal test_table_properties_db unique_table binlog.max_bytes 9223372036854775807 internal test_table_properties_db unique_table binlog.max_history_nums 9223372036854775807 +internal test_table_properties_db unique_table binlog.need_historical_value false internal test_table_properties_db unique_table binlog.ttl_seconds 86400 internal test_table_properties_db unique_table compaction_policy size_based internal test_table_properties_db unique_table compression ZSTD @@ -201,8 +211,10 @@ internal test_table_properties_db unique_table vertical_compaction_num_columns_p -- !select_check_5 -- internal test_table_properties_db duplicate_table _auto_bucket false internal test_table_properties_db duplicate_table binlog.enable false +internal test_table_properties_db duplicate_table binlog.format STATEMENT_AND_SNAPSHOT internal test_table_properties_db duplicate_table binlog.max_bytes 9223372036854775807 internal test_table_properties_db duplicate_table binlog.max_history_nums 9223372036854775807 +internal test_table_properties_db duplicate_table binlog.need_historical_value false internal test_table_properties_db duplicate_table binlog.ttl_seconds 86400 internal test_table_properties_db duplicate_table compaction_policy size_based internal test_table_properties_db duplicate_table compression ZSTD @@ -239,4 +251,3 @@ internal test_table_properties_db duplicate_table time_series_compaction_time_th internal test_table_properties_db duplicate_table vertical_compaction_num_columns_per_group 5 -- !select_check_6 -- - diff --git a/regression-test/data/show_p0/test_show_create_table_and_views_nereids.out b/regression-test/data/show_p0/test_show_create_table_and_views_nereids.out index 7929b470514276..d2d16111c29e4a 100644 --- a/regression-test/data/show_p0/test_show_create_table_and_views_nereids.out +++ b/regression-test/data/show_p0/test_show_create_table_and_views_nereids.out @@ -1,6 +1,6 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !show -- -show_create_table_and_views_nereids_table CREATE TABLE `show_create_table_and_views_nereids_table` (\n `user_id` largeint NOT NULL,\n `good_id` largeint NOT NULL,\n `cost` bigint SUM NULL DEFAULT "0",\n INDEX index_user_id (`user_id`) USING INVERTED COMMENT "test index comment",\n INDEX index_good_id (`good_id`) USING INVERTED COMMENT "test index\\" comment"\n) ENGINE=OLAP\nAGGREGATE KEY(`user_id`, `good_id`)\nPARTITION BY RANGE(`good_id`)\n(PARTITION p1 VALUES [("-170141183460469231731687303715884105728"), ("100")),\nPARTITION p2 VALUES [("100"), ("200")),\nPARTITION p3 VALUES [("200"), ("300")),\nPARTITION p4 VALUES [("300"), ("400")),\nPARTITION p5 VALUES [("400"), ("500")),\nPARTITION p6 VALUES [("500"), ("600")),\nPARTITION p7 VALUES [("600"), (MAXVALUE)))\nDISTRIBUTED BY HASH(`user_id`) BUCKETS 2\nPROPERTIES (\n"replication_allocation" = "tag.location.default: 1",\n"min_load_replica_num" = "-1",\n"is_being_synced" = "false",\n"storage_medium" = "hdd",\n"storage_format" = "V2",\n"inverted_index_storage_format" = "V3",\n"light_schema_change" = "true",\n"disable_auto_compaction" = "false",\n"binlog.enable" = "false",\n"binlog.ttl_seconds" = "86400",\n"binlog.max_bytes" = "9223372036854775807",\n"binlog.max_history_nums" = "9223372036854775807",\n"enable_single_replica_compaction" = "false",\n"group_commit_interval_ms" = "10000",\n"group_commit_data_bytes" = "134217728"\n); +show_create_table_and_views_nereids_table CREATE TABLE `show_create_table_and_views_nereids_table` (\n `user_id` largeint NOT NULL,\n `good_id` largeint NOT NULL,\n `cost` bigint SUM NULL DEFAULT "0",\n INDEX index_user_id (`user_id`) USING INVERTED COMMENT "test index comment",\n INDEX index_good_id (`good_id`) USING INVERTED COMMENT "test index\\" comment"\n) ENGINE=OLAP\nAGGREGATE KEY(`user_id`, `good_id`)\nPARTITION BY RANGE(`good_id`)\n(PARTITION p1 VALUES [("-170141183460469231731687303715884105728"), ("100")),\nPARTITION p2 VALUES [("100"), ("200")),\nPARTITION p3 VALUES [("200"), ("300")),\nPARTITION p4 VALUES [("300"), ("400")),\nPARTITION p5 VALUES [("400"), ("500")),\nPARTITION p6 VALUES [("500"), ("600")),\nPARTITION p7 VALUES [("600"), (MAXVALUE)))\nDISTRIBUTED BY HASH(`user_id`) BUCKETS 2\nPROPERTIES (\n"replication_allocation" = "tag.location.default: 1",\n"min_load_replica_num" = "-1",\n"is_being_synced" = "false",\n"storage_medium" = "hdd",\n"storage_format" = "V2",\n"inverted_index_storage_format" = "V3",\n"light_schema_change" = "true",\n"disable_auto_compaction" = "false",\n"binlog.enable" = "false",\n"binlog.ttl_seconds" = "86400",\n"binlog.max_bytes" = "9223372036854775807",\n"binlog.max_history_nums" = "9223372036854775807",\n"binlog.format" = "STATEMENT_AND_SNAPSHOT",\n"enable_single_replica_compaction" = "false",\n"group_commit_interval_ms" = "10000",\n"group_commit_data_bytes" = "134217728"\n); -- !select -- 1 1 30 @@ -36,11 +36,10 @@ show_create_table_and_views_nereids_view CREATE VIEW `show_create_table_and_view 300 1 -- !show -- -show_create_table_and_views_nereids_table CREATE TABLE `show_create_table_and_views_nereids_table` (\n `user_id` largeint NOT NULL,\n `good_id` largeint NOT NULL,\n `cost` bigint SUM NULL DEFAULT "0",\n INDEX index_user_id (`user_id`) USING INVERTED COMMENT "test index comment",\n INDEX index_good_id (`good_id`) USING INVERTED COMMENT "test index\\" comment"\n) ENGINE=OLAP\nAGGREGATE KEY(`user_id`, `good_id`)\nPARTITION BY RANGE(`good_id`)\n(PARTITION p1 VALUES [("-170141183460469231731687303715884105728"), ("100")),\nPARTITION p2 VALUES [("100"), ("200")),\nPARTITION p3 VALUES [("200"), ("300")),\nPARTITION p4 VALUES [("300"), ("400")),\nPARTITION p5 VALUES [("400"), ("500")),\nPARTITION p6 VALUES [("500"), ("600")),\nPARTITION p7 VALUES [("600"), (MAXVALUE)))\nDISTRIBUTED BY HASH(`user_id`) BUCKETS 2\nPROPERTIES (\n"replication_allocation" = "tag.location.default: 1",\n"min_load_replica_num" = "-1",\n"is_being_synced" = "false",\n"storage_medium" = "hdd",\n"storage_format" = "V2",\n"inverted_index_storage_format" = "V3",\n"light_schema_change" = "true",\n"disable_auto_compaction" = "false",\n"binlog.enable" = "false",\n"binlog.ttl_seconds" = "86400",\n"binlog.max_bytes" = "9223372036854775807",\n"binlog.max_history_nums" = "9223372036854775807",\n"enable_single_replica_compaction" = "false",\n"group_commit_interval_ms" = "10000",\n"group_commit_data_bytes" = "134217728"\n); +show_create_table_and_views_nereids_table CREATE TABLE `show_create_table_and_views_nereids_table` (\n `user_id` largeint NOT NULL,\n `good_id` largeint NOT NULL,\n `cost` bigint SUM NULL DEFAULT "0",\n INDEX index_user_id (`user_id`) USING INVERTED COMMENT "test index comment",\n INDEX index_good_id (`good_id`) USING INVERTED COMMENT "test index\\" comment"\n) ENGINE=OLAP\nAGGREGATE KEY(`user_id`, `good_id`)\nPARTITION BY RANGE(`good_id`)\n(PARTITION p1 VALUES [("-170141183460469231731687303715884105728"), ("100")),\nPARTITION p2 VALUES [("100"), ("200")),\nPARTITION p3 VALUES [("200"), ("300")),\nPARTITION p4 VALUES [("300"), ("400")),\nPARTITION p5 VALUES [("400"), ("500")),\nPARTITION p6 VALUES [("500"), ("600")),\nPARTITION p7 VALUES [("600"), (MAXVALUE)))\nDISTRIBUTED BY HASH(`user_id`) BUCKETS 2\nPROPERTIES (\n"replication_allocation" = "tag.location.default: 1",\n"min_load_replica_num" = "-1",\n"is_being_synced" = "false",\n"storage_medium" = "hdd",\n"storage_format" = "V2",\n"inverted_index_storage_format" = "V3",\n"light_schema_change" = "true",\n"disable_auto_compaction" = "false",\n"binlog.enable" = "false",\n"binlog.ttl_seconds" = "86400",\n"binlog.max_bytes" = "9223372036854775807",\n"binlog.max_history_nums" = "9223372036854775807",\n"binlog.format" = "STATEMENT_AND_SNAPSHOT",\n"enable_single_replica_compaction" = "false",\n"group_commit_interval_ms" = "10000",\n"group_commit_data_bytes" = "134217728"\n); -- !show -- -show_create_table_and_views_nereids_like CREATE TABLE `show_create_table_and_views_nereids_like` (\n `user_id` largeint NOT NULL,\n `good_id` largeint NOT NULL,\n `cost` bigint SUM NULL DEFAULT "0",\n INDEX index_user_id (`user_id`) USING INVERTED COMMENT "test index comment",\n INDEX index_good_id (`good_id`) USING INVERTED COMMENT "test index\\" comment"\n) ENGINE=OLAP\nAGGREGATE KEY(`user_id`, `good_id`)\nPARTITION BY RANGE(`good_id`)\n(PARTITION p1 VALUES [("-170141183460469231731687303715884105728"), ("100")),\nPARTITION p2 VALUES [("100"), ("200")),\nPARTITION p3 VALUES [("200"), ("300")),\nPARTITION p4 VALUES [("300"), ("400")),\nPARTITION p5 VALUES [("400"), ("500")),\nPARTITION p6 VALUES [("500"), ("600")),\nPARTITION p7 VALUES [("600"), (MAXVALUE)))\nDISTRIBUTED BY HASH(`user_id`) BUCKETS 2\nPROPERTIES (\n"replication_allocation" = "tag.location.default: 1",\n"min_load_replica_num" = "-1",\n"is_being_synced" = "false",\n"storage_medium" = "hdd",\n"storage_format" = "V2",\n"inverted_index_storage_format" = "V3",\n"light_schema_change" = "true",\n"disable_auto_compaction" = "false",\n"binlog.enable" = "false",\n"binlog.ttl_seconds" = "86400",\n"binlog.max_bytes" = "9223372036854775807",\n"binlog.max_history_nums" = "9223372036854775807",\n"enable_single_replica_compaction" = "false",\n"group_commit_interval_ms" = "10000",\n"group_commit_data_bytes" = "134217728"\n); +show_create_table_and_views_nereids_like CREATE TABLE `show_create_table_and_views_nereids_like` (\n `user_id` largeint NOT NULL,\n `good_id` largeint NOT NULL,\n `cost` bigint SUM NULL DEFAULT "0",\n INDEX index_user_id (`user_id`) USING INVERTED COMMENT "test index comment",\n INDEX index_good_id (`good_id`) USING INVERTED COMMENT "test index\\" comment"\n) ENGINE=OLAP\nAGGREGATE KEY(`user_id`, `good_id`)\nPARTITION BY RANGE(`good_id`)\n(PARTITION p1 VALUES [("-170141183460469231731687303715884105728"), ("100")),\nPARTITION p2 VALUES [("100"), ("200")),\nPARTITION p3 VALUES [("200"), ("300")),\nPARTITION p4 VALUES [("300"), ("400")),\nPARTITION p5 VALUES [("400"), ("500")),\nPARTITION p6 VALUES [("500"), ("600")),\nPARTITION p7 VALUES [("600"), (MAXVALUE)))\nDISTRIBUTED BY HASH(`user_id`) BUCKETS 2\nPROPERTIES (\n"replication_allocation" = "tag.location.default: 1",\n"min_load_replica_num" = "-1",\n"is_being_synced" = "false",\n"storage_medium" = "hdd",\n"storage_format" = "V2",\n"inverted_index_storage_format" = "V3",\n"light_schema_change" = "true",\n"disable_auto_compaction" = "false",\n"binlog.enable" = "false",\n"binlog.ttl_seconds" = "86400",\n"binlog.max_bytes" = "9223372036854775807",\n"binlog.max_history_nums" = "9223372036854775807",\n"binlog.format" = "STATEMENT_AND_SNAPSHOT",\n"enable_single_replica_compaction" = "false",\n"group_commit_interval_ms" = "10000",\n"group_commit_data_bytes" = "134217728"\n); -- !show -- -show_create_table_and_views_nereids_like_with_rollup CREATE TABLE `show_create_table_and_views_nereids_like_with_rollup` (\n `user_id` largeint NOT NULL,\n `good_id` largeint NOT NULL,\n `cost` bigint SUM NULL DEFAULT "0",\n INDEX index_user_id (`user_id`) USING INVERTED COMMENT "test index comment",\n INDEX index_good_id (`good_id`) USING INVERTED COMMENT "test index\\" comment"\n) ENGINE=OLAP\nAGGREGATE KEY(`user_id`, `good_id`)\nPARTITION BY RANGE(`good_id`)\n(PARTITION p1 VALUES [("-170141183460469231731687303715884105728"), ("100")),\nPARTITION p2 VALUES [("100"), ("200")),\nPARTITION p3 VALUES [("200"), ("300")),\nPARTITION p4 VALUES [("300"), ("400")),\nPARTITION p5 VALUES [("400"), ("500")),\nPARTITION p6 VALUES [("500"), ("600")),\nPARTITION p7 VALUES [("600"), (MAXVALUE)))\nDISTRIBUTED BY HASH(`user_id`) BUCKETS 2\nPROPERTIES (\n"replication_allocation" = "tag.location.default: 1",\n"min_load_replica_num" = "-1",\n"is_being_synced" = "false",\n"storage_medium" = "hdd",\n"storage_format" = "V2",\n"inverted_index_storage_format" = "V3",\n"light_schema_change" = "true",\n"disable_auto_compaction" = "false",\n"binlog.enable" = "false",\n"binlog.ttl_seconds" = "86400",\n"binlog.max_bytes" = "9223372036854775807",\n"binlog.max_history_nums" = "9223372036854775807",\n"enable_single_replica_compaction" = "false",\n"group_commit_interval_ms" = "10000",\n"group_commit_data_bytes" = "134217728"\n); - +show_create_table_and_views_nereids_like_with_rollup CREATE TABLE `show_create_table_and_views_nereids_like_with_rollup` (\n `user_id` largeint NOT NULL,\n `good_id` largeint NOT NULL,\n `cost` bigint SUM NULL DEFAULT "0",\n INDEX index_user_id (`user_id`) USING INVERTED COMMENT "test index comment",\n INDEX index_good_id (`good_id`) USING INVERTED COMMENT "test index\\" comment"\n) ENGINE=OLAP\nAGGREGATE KEY(`user_id`, `good_id`)\nPARTITION BY RANGE(`good_id`)\n(PARTITION p1 VALUES [("-170141183460469231731687303715884105728"), ("100")),\nPARTITION p2 VALUES [("100"), ("200")),\nPARTITION p3 VALUES [("200"), ("300")),\nPARTITION p4 VALUES [("300"), ("400")),\nPARTITION p5 VALUES [("400"), ("500")),\nPARTITION p6 VALUES [("500"), ("600")),\nPARTITION p7 VALUES [("600"), (MAXVALUE)))\nDISTRIBUTED BY HASH(`user_id`) BUCKETS 2\nPROPERTIES (\n"replication_allocation" = "tag.location.default: 1",\n"min_load_replica_num" = "-1",\n"is_being_synced" = "false",\n"storage_medium" = "hdd",\n"storage_format" = "V2",\n"inverted_index_storage_format" = "V3",\n"light_schema_change" = "true",\n"disable_auto_compaction" = "false",\n"binlog.enable" = "false",\n"binlog.ttl_seconds" = "86400",\n"binlog.max_bytes" = "9223372036854775807",\n"binlog.max_history_nums" = "9223372036854775807",\n"binlog.format" = "STATEMENT_AND_SNAPSHOT",\n"enable_single_replica_compaction" = "false",\n"group_commit_interval_ms" = "10000",\n"group_commit_data_bytes" = "134217728"\n); diff --git a/regression-test/suites/show_p0/test_show_data.groovy b/regression-test/suites/show_p0/test_show_data.groovy index 20adc299d55d83..f0a7a4ec5e64ea 100644 --- a/regression-test/suites/show_p0/test_show_data.groovy +++ b/regression-test/suites/show_p0/test_show_data.groovy @@ -27,5 +27,5 @@ suite("test_show_data") { } assertTrue(result2.size() > 0); - assertTrue(result2[0].size() == 6); -} \ No newline at end of file + assertTrue(result2[0].size() == 7); +} From b08f1251e2488d436c428ef191a45a5e85a21426 Mon Sep 17 00:00:00 2001 From: UserWhite <925945594@qq.com> Date: Fri, 17 Apr 2026 14:30:23 +0800 Subject: [PATCH 2/5] fix conflict --- be/src/storage/rowset_builder.cpp | 3 +- .../apache/doris/alter/AlterOperations.java | 2 +- .../apache/doris/catalog/BinlogConfig.java | 5 +- .../doris/alter/SchemaChangeHandlerTest.java | 2 +- .../doris/binlog/BinlogManagerTest.java | 6 +- .../apache/doris/catalog/CreateTableTest.java | 63 ++++++--------- .../doris/common/proc/ProcServiceTest.java | 79 +++++++++++++++++-- .../plans/commands/ShowDataCommandTest.java | 30 ++++--- .../commands/TruncateTableCommandTest.java | 30 +++---- 9 files changed, 133 insertions(+), 87 deletions(-) diff --git a/be/src/storage/rowset_builder.cpp b/be/src/storage/rowset_builder.cpp index 4d258a0b8d75b3..aa075f386c48c3 100644 --- a/be/src/storage/rowset_builder.cpp +++ b/be/src/storage/rowset_builder.cpp @@ -478,7 +478,7 @@ Status BaseRowsetBuilder::_build_current_tablet_schema( table_schema_param->sequence_map_col_uid(), _max_version_in_flush_phase)); return Status::OK(); } - + GroupRowsetBuilder::GroupRowsetBuilder(StorageEngine& engine, const WriteRequest& req, const WriteRequest& row_binlog_req, RuntimeProfile* profile) : BaseRowsetBuilder( @@ -566,5 +566,4 @@ Status BaseRowsetBuilder::attach_pending_rs_guard_to_txn(const RowsetId& rowset_ return Status::OK(); } -#include "common/compile_check_end.h" } // namespace doris diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/AlterOperations.java b/fe/fe-core/src/main/java/org/apache/doris/alter/AlterOperations.java index c07aa3b1b794ed..3de8b2c6659961 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/alter/AlterOperations.java +++ b/fe/fe-core/src/main/java/org/apache/doris/alter/AlterOperations.java @@ -60,7 +60,7 @@ public void checkMTMVAllow(List alterOps) throws DdlException { public void checkRowBinlogAllow(List alterOps) throws DdlException { for (AlterOp alterOp : alterOps) { if (!alterOp.allowOpRowBinlog()) { - throw new DdlException("Not allowed to perform current operation on Table With Row Binlog"); + throw new DdlException("Not allowed to perform current operation on Table With binlog"); } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/BinlogConfig.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/BinlogConfig.java index 0f874e78954a2b..80c1e8903ba131 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/BinlogConfig.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/BinlogConfig.java @@ -50,10 +50,7 @@ public enum BinlogFormat { // record all meta update operator, and generate snapshot for write data, only used for ccr STATEMENT_AND_SNAPSHOT(0), // generate row binlog when write, used for table binlog transform - ROW(1), - // generate row binlog when need, calculate binlog by compaction and read snapshot, - // used for table binlog transform - DELTA(2); + ROW(1); private final int value; diff --git a/fe/fe-core/src/test/java/org/apache/doris/alter/SchemaChangeHandlerTest.java b/fe/fe-core/src/test/java/org/apache/doris/alter/SchemaChangeHandlerTest.java index 9327bdc17ecb96..9aaf482df5018f 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/alter/SchemaChangeHandlerTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/alter/SchemaChangeHandlerTest.java @@ -317,7 +317,7 @@ public void testWithRowBinlogOpNotSupported() throws Exception { + "'enable_unique_key_merge_on_write'='true'," + "'binlog.enable'='true','binlog.format'='ROW','binlog.need_historical_value'='false');"; createTable(create); - expectException("ALTER TABLE test." + tableName + " MODIFY COLUMN v1 BIGINT", "Table With Row Binlog"); + expectException("ALTER TABLE test." + tableName + " MODIFY COLUMN v1 BIGINT", "Table With binlog"); // 2) VARIANT not supported String createVariant = "CREATE TABLE test.binlog_variant (k1 INT NOT NULL, v1 VARIANT) " diff --git a/fe/fe-core/src/test/java/org/apache/doris/binlog/BinlogManagerTest.java b/fe/fe-core/src/test/java/org/apache/doris/binlog/BinlogManagerTest.java index 59ffc4c628106d..0e0b0d0ce231a4 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/binlog/BinlogManagerTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/binlog/BinlogManagerTest.java @@ -157,6 +157,8 @@ public void tearDown() { @Test public void testBinlogConfigEquals() { + mockedBinlogConfigConstruction.close(); + mockedBinlogConfigConstruction = null; BinlogConfig c1 = new BinlogConfig(true, 10L, 20L, 30L, BinlogConfig.BinlogFormat.ROW, true); BinlogConfig c2 = new BinlogConfig(true, 10L, 20L, 30L, BinlogConfig.BinlogFormat.ROW, true); BinlogConfig c3 = new BinlogConfig(true, 10L, 20L, 30L, BinlogConfig.BinlogFormat.ROW, false); @@ -167,7 +169,9 @@ public void testBinlogConfigEquals() { } @Test - public void testBinlogConfigAppendToShowCreateTable() { + public void testBinlogConfigShowDDL() { + mockedBinlogConfigConstruction.close(); + mockedBinlogConfigConstruction = null; BinlogConfig rowCfg = new BinlogConfig(true, 11L, 22L, 33L, BinlogConfig.BinlogFormat.ROW, true); StringBuilder sb = new StringBuilder(); rowCfg.appendToShowCreateTable(sb); diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateTableTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateTableTest.java index 3bf05aaa5c5e04..bd82180f93fcd6 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateTableTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateTableTest.java @@ -84,6 +84,23 @@ public void testNormal() throws DdlException, ConfigException { () -> createTable("create table test.tbl1\n" + "(k1 int, k2 int)\n" + "duplicate key(k1)\n" + "distributed by hash(k2) buckets 1\n" + "properties('replication_num' = '1'); ")); + ExceptionChecker.expectThrowsNoException(() -> createTable("CREATE TEMPORARY TABLE test.temp_normal (k1 INT) " + + "DUPLICATE KEY(k1) DISTRIBUTED BY HASH(k1) BUCKETS 1 " + + "PROPERTIES('replication_num'='1');")); + + ExceptionChecker.expectThrowsNoException(() -> createTable("CREATE TEMPORARY TABLE test.temp_row_binlog (k1 INT) " + + "DUPLICATE KEY(k1) DISTRIBUTED BY HASH(k1) BUCKETS 1 " + + "PROPERTIES('replication_num'='1','binlog.enable'='true','binlog.format'='ROW');")); + + ExceptionChecker.expectThrowsNoException(() -> createTable("CREATE TABLE test.row_binlog_normal (k1 INT) " + + "DUPLICATE KEY(k1) DISTRIBUTED BY HASH(k1) BUCKETS 1 " + + "PROPERTIES('replication_num'='1','binlog.enable'='true','binlog.format'='ROW');")); + + ExceptionChecker.expectThrowsNoException(() -> createTable("CREATE TABLE test.row_binlog_unique (k1 INT, v1 INT) " + + "UNIQUE KEY(k1) DISTRIBUTED BY HASH(k1) BUCKETS 1 " + + "PROPERTIES('replication_num'='1','enable_unique_key_merge_on_write'='true'," + + "'binlog.enable'='true','binlog.format'='ROW');")); + ExceptionChecker.expectThrowsNoException(() -> createTable("create table test.tbl2\n" + "(k1 int, k2 int)\n" + "duplicate key(k1)\n" + "partition by range(k2)\n" + "(partition p1 values less than(\"10\"))\n" + "distributed by hash(k2) buckets 1\n" + "properties('replication_num' = '1'); ")); @@ -243,49 +260,13 @@ public void testNormal() throws DdlException, ConfigException { } @Test - public void testCreateTempTable() throws Exception { - String create = "CREATE TEMPORARY TABLE test.temp_normal (k1 INT) " - + "DUPLICATE KEY(k1) DISTRIBUTED BY HASH(k1) BUCKETS 1 " - + "PROPERTIES('replication_num'='1');"; - createTable(create); - } - - @Test - public void testCreateTempTableWithRowBinlog() throws Exception { - String create = "CREATE TEMPORARY TABLE test.temp_row_binlog (k1 INT) " - + "DUPLICATE KEY(k1) DISTRIBUTED BY HASH(k1) BUCKETS 1 " - + "PROPERTIES('replication_num'='1','binlog.enable'='true','binlog.format'='ROW');"; - createTable(create); - } - - @Test - public void testCreateRowBinlogTable() throws Exception { - String create = "CREATE TABLE test.row_binlog_normal (k1 INT) " - + "DUPLICATE KEY(k1) DISTRIBUTED BY HASH(k1) BUCKETS 1 " - + "PROPERTIES('replication_num'='1','binlog.enable'='true','binlog.format'='ROW');"; - createTable(create); - } - - @Test - public void testCreateRowBinlogUniqueKeyTable() throws Exception { - String create = "CREATE TABLE test.row_binlog_unique (k1 INT, v1 INT) " - + "UNIQUE KEY(k1) DISTRIBUTED BY HASH(k1) BUCKETS 1 " - + "PROPERTIES('replication_num'='1','enable_unique_key_merge_on_write'='true'," - + "'binlog.enable'='true','binlog.format'='ROW');"; - createTable(create); - } - - @Test - public void testCreateRowBinlogAggregateKeyNotSupported() throws Exception { - String create = "CREATE TABLE test.row_binlog_agg (k1 INT, v1 INT SUM) " - + "AGGREGATE KEY(k1) DISTRIBUTED BY HASH(k1) BUCKETS 1 " - + "PROPERTIES('replication_num'='1','binlog.enable'='true','binlog.format'='ROW');"; - Exception exception = Assert.assertThrows(Exception.class, () -> createTable(create)); + public void testAbnormal() throws DdlException, ConfigException { + Exception exception = Assert.assertThrows(Exception.class, + () -> createTable("CREATE TABLE test.row_binlog_agg (k1 INT, v1 INT SUM) " + + "AGGREGATE KEY(k1) DISTRIBUTED BY HASH(k1) BUCKETS 1 " + + "PROPERTIES('replication_num'='1','binlog.enable'='true','binlog.format'='ROW');")); Assert.assertTrue(exception.getMessage(), exception.getMessage().contains("binlog")); - } - @Test - public void testAbnormal() throws DdlException, ConfigException { ExceptionChecker.expectThrowsWithMsg(DdlException.class, "Unknown properties: {aa=bb}", () -> createTable("create table test.atbl1\n" + "(k1 int, k2 float)\n" + "duplicate key(k1)\n" diff --git a/fe/fe-core/src/test/java/org/apache/doris/common/proc/ProcServiceTest.java b/fe/fe-core/src/test/java/org/apache/doris/common/proc/ProcServiceTest.java index c8f2780f378e6f..15493325314187 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/common/proc/ProcServiceTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/common/proc/ProcServiceTest.java @@ -17,13 +17,29 @@ package org.apache.doris.common.proc; +import org.apache.doris.catalog.Database; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.MaterializedIndex; +import org.apache.doris.catalog.OlapTable; +import org.apache.doris.catalog.Partition; +import org.apache.doris.catalog.Replica; +import org.apache.doris.catalog.Tablet; +import org.apache.doris.catalog.TabletInvertedIndex; +import org.apache.doris.catalog.TabletMeta; import org.apache.doris.common.AnalysisException; +import org.apache.doris.datasource.InternalCatalog; +import org.apache.doris.system.SystemInfoService; +import org.apache.doris.thrift.TStorageMedium; +import com.google.common.collect.ImmutableMap; import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import org.mockito.MockedStatic; +import org.mockito.Mockito; +import java.util.Collections; import java.util.List; public class ProcServiceTest { @@ -176,14 +192,63 @@ public void testOpenFail() { } @Test - public void testWithRowBinlogColumnsInShowProc() { - List tabletsTitles = TabletsProcDir.TITLE_NAMES; - Assert.assertTrue(tabletsTitles.contains("BinlogSize")); - Assert.assertTrue(tabletsTitles.contains("BinlogFileNum")); - + public void testTabletProc() throws AnalysisException { List replicasTitles = ReplicasProcNode.TITLE_NAMES; - Assert.assertTrue(replicasTitles.contains("BinlogSize")); - Assert.assertTrue(replicasTitles.contains("BinlogFileNum")); + int replicaIdIdx = replicasTitles.indexOf("ReplicaId"); + int backendIdIdx = replicasTitles.indexOf("BackendId"); + int versionIdx = replicasTitles.indexOf("Version"); + int lastSuccessVersionIdx = replicasTitles.indexOf("LstSuccessVersion"); + int replicasBinlogSizeIdx = replicasTitles.indexOf("BinlogSize"); + int replicasBinlogFileNumIdx = replicasTitles.indexOf("BinlogFileNum"); + + long tabletId = 10001L; + long backendId = 10002L; + + SystemInfoService systemInfoService = Mockito.mock(SystemInfoService.class); + TabletInvertedIndex tabletInvertedIndex = Mockito.mock(TabletInvertedIndex.class); + InternalCatalog internalCatalog = Mockito.mock(InternalCatalog.class); + Database database = Mockito.mock(Database.class); + OlapTable table = Mockito.mock(OlapTable.class); + Partition partition = Mockito.mock(Partition.class); + MaterializedIndex index = Mockito.mock(MaterializedIndex.class); + Tablet tablet = Mockito.mock(Tablet.class); + Replica replica = Mockito.mock(Replica.class); + + TabletMeta tabletMeta = new TabletMeta(20001L, 20002L, 20003L, 20004L, 12345, TStorageMedium.HDD); + + Mockito.when(systemInfoService.getAllBackendsByAllCluster()).thenReturn(ImmutableMap.of()); + Mockito.when(tabletInvertedIndex.getTabletMeta(tabletId)).thenReturn(tabletMeta); + Mockito.when(internalCatalog.getDbNullable(tabletMeta.getDbId())).thenReturn(database); + Mockito.when(database.getTableNullable(tabletMeta.getTableId())).thenReturn(table); + Mockito.when(table.getPartition(tabletMeta.getPartitionId())).thenReturn(partition); + Mockito.when(partition.getIndex(tabletMeta.getIndexId())).thenReturn(index); + Mockito.when(index.getTablet(tabletId)).thenReturn(tablet); + + Mockito.when(replica.getId()).thenReturn(6006L); + Mockito.when(replica.getBackendIdWithoutException()).thenReturn(backendId); + Mockito.when(replica.getVersion()).thenReturn(101L); + Mockito.when(replica.getLastSuccessVersion()).thenReturn(100L); + Mockito.when(replica.getBinlogSize()).thenReturn(8192L); + Mockito.when(replica.getBinlogFileNum()).thenReturn(5L); + + try (MockedStatic mockedEnvStatic = Mockito.mockStatic(Env.class)) { + mockedEnvStatic.when(Env::getCurrentSystemInfo).thenReturn(systemInfoService); + mockedEnvStatic.when(Env::getCurrentInvertedIndex).thenReturn(tabletInvertedIndex); + mockedEnvStatic.when(Env::getCurrentInternalCatalog).thenReturn(internalCatalog); + + ProcResult result = new ReplicasProcNode(tabletId, Collections.singletonList(replica)).fetchResult(); + List> rows = result.getRows(); + Assert.assertEquals(1, rows.size()); + + List row = rows.get(0); + Assert.assertEquals(replicasTitles.size(), row.size()); + Assert.assertEquals("6006", row.get(replicaIdIdx)); + Assert.assertEquals("10002", row.get(backendIdIdx)); + Assert.assertEquals("101", row.get(versionIdx)); + Assert.assertEquals("100", row.get(lastSuccessVersionIdx)); + Assert.assertEquals("8192", row.get(replicasBinlogSizeIdx)); + Assert.assertEquals("5", row.get(replicasBinlogFileNumIdx)); + } } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/ShowDataCommandTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/ShowDataCommandTest.java index b1f7f1aea2bd92..00eec87d99ad38 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/ShowDataCommandTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/ShowDataCommandTest.java @@ -27,6 +27,7 @@ import org.apache.doris.catalog.RandomDistributionInfo; import org.apache.doris.catalog.SinglePartitionInfo; import org.apache.doris.catalog.TableIf; +import org.apache.doris.catalog.TabletInvertedIndex; import org.apache.doris.catalog.info.TableNameInfo; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.Pair; @@ -126,7 +127,12 @@ public void testValidateNormal() throws Exception { public void testValidateShowAllDataNormal() throws Exception { Mockito.when(connectContext.getDatabase()).thenReturn(CatalogMocker.TEST_DB_NAME); Mockito.when(connectContext.isSkipAuth()).thenReturn(true); - Mockito.when(catalog.getDbOrAnalysisException(Mockito.anyString())).thenReturn(CatalogMocker.mockDb()); + mockedEnv.when(Env::getCurrentInvertedIndex).thenReturn(Mockito.mock(TabletInvertedIndex.class)); + Database mockDb = CatalogMocker.mockDb(); + Mockito.when(catalog.getDbOrAnalysisException(Mockito.anyString())).thenReturn(mockDb); + Mockito.when(accessControllerManager.checkTblPriv( + Mockito.nullable(ConnectContext.class), Mockito.anyString(), Mockito.anyString(), + Mockito.anyString(), Mockito.any(PrivPredicate.class))).thenReturn(true); SlotReference tableName = new SlotReference("TableName", IntegerType.INSTANCE); List keys = ImmutableList.of(new OrderKey(tableName, true, false)); @@ -188,25 +194,23 @@ public void testValidateShowAllDataGetAllDbStats() throws Exception { public void testValidateNoPrivilege() throws Exception { Mockito.doReturn(database).when(catalog).getDbOrAnalysisException(Mockito.anyString()); Mockito.doReturn(olapTable).when(database).getTableOrMetaException( - Mockito.eq(tableNameInfo.getTbl()), Mockito.eq(TableIf.TableType.OLAP)); + Mockito.anyString(), Mockito.any(TableIf.TableType.class)); SlotReference tableName = new SlotReference("TableName", IntegerType.INSTANCE); - List keys = ImmutableList.of(new OrderKey(tableName, true, false)); + List keys = ImmutableList.of( + new OrderKey(tableName, true, false) + ); - // table not exist + // test not exist table TableNameInfo tableNameInfoNotExist = new TableNameInfo(CatalogMocker.TEST_DB_NAME, "tbl_not_exist"); - Mockito.doThrow(new AnalysisException("not exist")).when(database) - .getTableOrMetaException(Mockito.eq("tbl_not_exist"), Mockito.eq(TableIf.TableType.OLAP)); - ShowDataCommand command = new ShowDataCommand(tableNameInfoNotExist, keys, new HashMap<>(), false); + + Map properties = new HashMap<>(); + ShowDataCommand command = new ShowDataCommand(tableNameInfoNotExist, keys, properties, false); Assertions.assertThrows(AnalysisException.class, () -> command.validate(connectContext)); - // no privilege - Mockito.when(accessControllerManager.checkTblPriv( - Mockito.nullable(ConnectContext.class), - Mockito.any(TableNameInfo.class), - Mockito.any(PrivPredicate.class))).thenReturn(false); - ShowDataCommand command2 = new ShowDataCommand(tableNameInfo, keys, new HashMap<>(), false); + // test no priv + ShowDataCommand command2 = new ShowDataCommand(tableNameInfo, keys, properties, false); Assertions.assertThrows(AnalysisException.class, () -> command2.validate(connectContext)); } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/TruncateTableCommandTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/TruncateTableCommandTest.java index e229c4c5f3c1b8..3bc1e8a43dfa9e 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/TruncateTableCommandTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/TruncateTableCommandTest.java @@ -210,6 +210,19 @@ public void testTruncateTable() throws Exception { Assertions.assertTrue(plan instanceof TruncateTableCommand); Env.getCurrentEnv().truncateTable((TruncateTableCommand) plan); checkShowTabletResultNum("internal.testcommand.tblcommand", "p20210904", 5); + + String createTableStr = "create table internal.testcommand.tbl_row_binlog(d1 date, k1 int)" + + "duplicate key(d1, k1) " + + "PARTITION BY RANGE(d1)" + + "(PARTITION p20210901 VALUES [('2021-09-01'), ('2021-09-02')))" + + "distributed by hash(k1) buckets 1 " + + "properties('replication_num' = '1', 'binlog.enable'='true', 'binlog.format'='ROW');"; + createTable(createTableStr); + + truncateStr = "truncate table internal.testcommand.tbl_row_binlog;"; + plan = nereidsParser.parseSingle(truncateStr); + Assertions.assertTrue(plan instanceof TruncateTableCommand); + Env.getCurrentEnv().truncateTable((TruncateTableCommand) plan); } @Test @@ -261,23 +274,6 @@ public void testTruncateTableFailed() throws Exception { } } - @Test - public void testTruncateRowBinlogTable() throws Exception { - String createTableStr = "create table internal.testcommand.tbl_row_binlog(d1 date, k1 int)" - + "duplicate key(d1, k1) " - + "PARTITION BY RANGE(d1)" - + "(PARTITION p20210901 VALUES [('2021-09-01'), ('2021-09-02')))" - + "distributed by hash(k1) buckets 1 " - + "properties('replication_num' = '1', 'binlog.enable'='true', 'binlog.format'='ROW');"; - createTable(createTableStr); - - String truncateStr = "truncate table internal.testcommand.tbl_row_binlog;"; - NereidsParser nereidsParser = new NereidsParser(); - LogicalPlan plan = nereidsParser.parseSingle(truncateStr); - Assertions.assertTrue(plan instanceof TruncateTableCommand); - Env.getCurrentEnv().truncateTable((TruncateTableCommand) plan); - } - private List> checkShowTabletResultNum(String tbl, String partition, int expected) throws Exception { String showStr = "show tablets from " + tbl + " partition(" + partition + ")"; From 9d63c2cc795a04b020722b34ad4fa6664b63cc08 Mon Sep 17 00:00:00 2001 From: UserWhite <925945594@qq.com> Date: Fri, 17 Apr 2026 17:44:13 +0800 Subject: [PATCH 3/5] add guard for cloud mode --- .../main/java/org/apache/doris/datasource/InternalCatalog.java | 3 +++ .../nereids/trees/plans/commands/ShowDataCommandTest.java | 2 +- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java index 02a4d3b575aa86..5ed2a2edde3b81 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java @@ -2847,6 +2847,9 @@ private boolean createOlapTable(Database db, CreateTableInfo createTableInfo) th + "if you want to use mor or aggregate table model, " + "please use binlog with snapshot"); } + if (Config.isCloudMode()) { + throw new AnalysisException("Binlog is not supported in the cloud mode yet"); + } if (keysType == KeysType.DUP_KEYS && binlogConfig.getNeedHistoricalValue()) { throw new AnalysisException("Duplicate table model don't support record historical value"); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/ShowDataCommandTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/ShowDataCommandTest.java index 00eec87d99ad38..d6410fdafe5e9a 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/ShowDataCommandTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/ShowDataCommandTest.java @@ -63,7 +63,7 @@ public class ShowDataCommandTest { KeysType.AGG_KEYS, new SinglePartitionInfo(), new RandomDistributionInfo(32)); - + private Env env = Mockito.mock(Env.class); private InternalCatalog catalog = Mockito.mock(InternalCatalog.class); private AccessControllerManager accessControllerManager = Mockito.mock(AccessControllerManager.class); From bdd977e77944c8e177b7dedc85e172fb62f09dee Mon Sep 17 00:00:00 2001 From: UserWhite <925945594@qq.com> Date: Tue, 28 Apr 2026 20:56:06 +0800 Subject: [PATCH 4/5] fix id --- gensrc/proto/olap_file.proto | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/gensrc/proto/olap_file.proto b/gensrc/proto/olap_file.proto index 5df4af94f918ad..593a89d0aa992e 100644 --- a/gensrc/proto/olap_file.proto +++ b/gensrc/proto/olap_file.proto @@ -139,13 +139,14 @@ message RowsetMetaPB { // rows count for each segment repeated int64 num_segment_rows = 56; - optional bool is_row_binlog = 57 [default = false]; // If true, `segments_key_bounds` contains a single aggregated // [rowset_min_key, rowset_max_key] entry instead of per-segment bounds. // Only applies to non-MOW rowsets to reduce meta size on cloud FDB. optional bool segments_key_bounds_aggregated = 57; + optional bool is_row_binlog = 58 [default = false]; + // For cloud // for data recycling optional int64 txn_expiration = 1000; @@ -251,13 +252,14 @@ message RowsetMetaCloudPB { // rows count for each segment repeated int64 num_segment_rows = 56; - optional bool is_row_binlog = 57 [default = false]; // If true, `segments_key_bounds` contains a single aggregated // [rowset_min_key, rowset_max_key] entry instead of per-segment bounds. // Only applies to non-MOW rowsets to reduce meta size on cloud FDB. optional bool segments_key_bounds_aggregated = 57; + optional bool is_row_binlog = 58 [default = false]; + // cloud // the field is a vector, rename it repeated int64 segments_file_size = 100; From c286a46d3be717017a712e1ea3536e440ff199f5 Mon Sep 17 00:00:00 2001 From: UserWhite <925945594@qq.com> Date: Thu, 30 Apr 2026 14:20:43 +0800 Subject: [PATCH 5/5] fix typo --- fe/fe-core/src/main/java/org/apache/doris/catalog/Replica.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Replica.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Replica.java index 61135a2fc7d751..da08e535438a75 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Replica.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Replica.java @@ -108,7 +108,7 @@ public static class ReplicaContext { private long localSegmentSize = 0L; // Number of binlog files retained on this replica. - @SerializedName(value = "bfn", alternate = {"binloFileNum"}) + @SerializedName(value = "bfn", alternate = {"binlogFileNum"}) private volatile long binlogFileNum = -1; public Replica() {