From 9575a2b52b8456170718db53e3fd8330c042646e Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Thu, 1 Feb 2024 15:53:32 +0800 Subject: [PATCH 01/37] make RegionDataReadInfo easy to add fields Signed-off-by: CalvinNeo --- .../KVStore/Decode/PartitionStreams.cpp | 2 +- .../KVStore/Decode/RegionBlockReader.cpp | 2 +- .../Storages/KVStore/Decode/RegionDataRead.h | 35 +++++++++++++++++-- .../Storages/KVStore/MultiRaft/RegionData.cpp | 8 ++--- .../tests/gtest_region_block_reader.cpp | 2 +- 5 files changed, 40 insertions(+), 9 deletions(-) diff --git a/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp b/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp index d076b0e0a54..1169eb1ed8b 100644 --- a/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp +++ b/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp @@ -356,7 +356,7 @@ static inline void reportUpstreamLatency(const RegionDataReadInfoList & data_lis { return; } - auto ts = std::get<2>(data_list_read.front()); + auto ts = data_list_read.front().commit_ts; auto [physical_ms, logical] = parseTS(ts); std::ignore = logical; UInt64 curr_ms = std::chrono::time_point_cast(std::chrono::system_clock::now()) diff --git a/dbms/src/Storages/KVStore/Decode/RegionBlockReader.cpp b/dbms/src/Storages/KVStore/Decode/RegionBlockReader.cpp index d6b00b09cf4..c3462c84b82 100644 --- a/dbms/src/Storages/KVStore/Decode/RegionBlockReader.cpp +++ b/dbms/src/Storages/KVStore/Decode/RegionBlockReader.cpp @@ -98,7 +98,7 @@ bool RegionBlockReader::read(Block & block, const RegionDataReadInfoList & data_ exc.addMessage("TiKV value contains: "); for (const auto & data : data_list) { - exc.addMessage(fmt::format("{}, ", std::get<3>(data)->toDebugString())); + exc.addMessage(fmt::format("{}, ", data.value->toDebugString())); } exc.rethrow(); return false; diff --git a/dbms/src/Storages/KVStore/Decode/RegionDataRead.h b/dbms/src/Storages/KVStore/Decode/RegionDataRead.h index 6b77782b985..a1a996b666a 100644 --- a/dbms/src/Storages/KVStore/Decode/RegionDataRead.h +++ b/dbms/src/Storages/KVStore/Decode/RegionDataRead.h @@ -22,8 +22,39 @@ namespace DB { -// << PK, write_type, commit_ts, value >> -using RegionDataReadInfo = std::tuple>; +struct RegionDataReadInfo +{ + RegionDataReadInfo( + RawTiDBPK && pk_, + UInt8 && write_type_, + Timestamp && commit_ts_, + std::shared_ptr && value_) + : pk(std::move(pk_)) + , write_type(write_type_) + , commit_ts(std::move(commit_ts_)) + , value(std::move(value_)) + {} + RegionDataReadInfo( + const RawTiDBPK & pk_, + const UInt8 & write_type_, + const Timestamp & commit_ts_, + const std::shared_ptr & value_) + : pk(pk_) + , write_type(write_type_) + , commit_ts(commit_ts_) + , value(value_) + {} + RegionDataReadInfo(const RegionDataReadInfo &) = default; + RegionDataReadInfo(RegionDataReadInfo &&) = default; + RegionDataReadInfo & operator=(const RegionDataReadInfo &) = default; + RegionDataReadInfo & operator=(RegionDataReadInfo &&) = default; + +public: + RawTiDBPK pk; + UInt8 write_type; + Timestamp commit_ts; + std::shared_ptr value; +}; using RegionDataReadInfoList = std::vector; diff --git a/dbms/src/Storages/KVStore/MultiRaft/RegionData.cpp b/dbms/src/Storages/KVStore/MultiRaft/RegionData.cpp index 031b4fcf824..d82a4c8a1e9 100644 --- a/dbms/src/Storages/KVStore/MultiRaft/RegionData.cpp +++ b/dbms/src/Storages/KVStore/MultiRaft/RegionData.cpp @@ -156,17 +156,17 @@ std::optional RegionData::readDataByWriteIt( } if (!need_value) - return std::make_tuple(pk, decoded_val.write_type, ts, nullptr); + return RegionDataReadInfo{pk, decoded_val.write_type, ts, nullptr}; if (decoded_val.write_type != RecordKVFormat::CFModifyFlag::PutFlag) - return std::make_tuple(pk, decoded_val.write_type, ts, nullptr); + return RegionDataReadInfo{pk, decoded_val.write_type, ts, nullptr}; std::string orphan_key_debug_msg; if (!decoded_val.short_value) { const auto & map = default_cf.getData(); if (auto data_it = map.find({pk, decoded_val.prewrite_ts}); data_it != map.end()) - return std::make_tuple(pk, decoded_val.write_type, ts, RegionDefaultCFDataTrait::getTiKVValue(data_it)); + return RegionDataReadInfo{pk, decoded_val.write_type, ts, RegionDefaultCFDataTrait::getTiKVValue(data_it)}; else { if (!hard_error) @@ -198,7 +198,7 @@ std::optional RegionData::readDataByWriteIt( } } - return std::make_tuple(pk, decoded_val.write_type, ts, decoded_val.short_value); + return RegionDataReadInfo{pk, decoded_val.write_type, ts, decoded_val.short_value}; } DecodedLockCFValuePtr RegionData::getLockInfo(const RegionLockReadQuery & query) const diff --git a/dbms/src/Storages/KVStore/tests/gtest_region_block_reader.cpp b/dbms/src/Storages/KVStore/tests/gtest_region_block_reader.cpp index 65e0e3e7917..d76fbe6c078 100644 --- a/dbms/src/Storages/KVStore/tests/gtest_region_block_reader.cpp +++ b/dbms/src/Storages/KVStore/tests/gtest_region_block_reader.cpp @@ -157,7 +157,7 @@ class RegionBlockReaderTest : public ::testing::Test { if (decoding_schema->is_common_handle) { - ASSERT_FIELD_EQ((*column_element.column)[row], Field(*std::get<0>(data_list_read[row]))) + ASSERT_FIELD_EQ((*column_element.column)[row], Field(*data_list_read[row].pk)) << gen_error_log(); } else From c2c43448110ab5a4cec92e8988a5e116c1baca73 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Thu, 1 Feb 2024 17:54:25 +0800 Subject: [PATCH 02/37] rename writeBlockByRegion to writeCommittedByRegion Signed-off-by: CalvinNeo --- dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp | 2 +- dbms/src/Storages/KVStore/Decode/RegionTable.cpp | 2 +- dbms/src/Storages/KVStore/Decode/RegionTable.h | 4 ++-- dbms/src/Storages/KVStore/MultiRaft/RaftCommands.cpp | 9 +++++---- 4 files changed, 9 insertions(+), 8 deletions(-) diff --git a/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp b/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp index 1169eb1ed8b..1e6a665366e 100644 --- a/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp +++ b/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp @@ -369,7 +369,7 @@ static inline void reportUpstreamLatency(const RegionDataReadInfoList & data_lis } } -DM::WriteResult RegionTable::writeBlockByRegion( +DM::WriteResult RegionTable::writeCommittedByRegion( Context & context, const RegionPtrWithBlock & region, RegionDataReadInfoList & data_list_to_remove, diff --git a/dbms/src/Storages/KVStore/Decode/RegionTable.cpp b/dbms/src/Storages/KVStore/Decode/RegionTable.cpp index 81b8f1b4770..a13d55cd3d0 100644 --- a/dbms/src/Storages/KVStore/Decode/RegionTable.cpp +++ b/dbms/src/Storages/KVStore/Decode/RegionTable.cpp @@ -286,7 +286,7 @@ RegionDataReadInfoList RegionTable::tryWriteBlockByRegion(const RegionPtrWithBlo try { /// Write region data into corresponding storage. - writeBlockByRegion(*context, region, data_list_to_remove, log); + writeCommittedByRegion(*context, region, data_list_to_remove, log); } catch (const Exception & e) { diff --git a/dbms/src/Storages/KVStore/Decode/RegionTable.h b/dbms/src/Storages/KVStore/Decode/RegionTable.h index 2d1204e80c0..07e7d41a2f9 100644 --- a/dbms/src/Storages/KVStore/Decode/RegionTable.h +++ b/dbms/src/Storages/KVStore/Decode/RegionTable.h @@ -133,7 +133,7 @@ class RegionTable : private boost::noncopyable /// Will trigger schema sync on read error for only once, /// assuming that newer schema can always apply to older data by setting force_decode to true in RegionBlockReader::read. /// Note that table schema must be keep unchanged throughout the process of read then write, we take good care of the lock. - static DM::WriteResult writeBlockByRegion( + static DM::WriteResult writeCommittedByRegion( Context & context, const RegionPtrWithBlock & region, RegionDataReadInfoList & data_list_to_remove, @@ -141,7 +141,7 @@ class RegionTable : private boost::noncopyable bool lock_region = true); /// Check transaction locks in region, and write committed data in it into storage engine if check passed. Otherwise throw an LockException. - /// The write logic is the same as #writeBlockByRegion, with some extra checks about region version and conf_version. + /// The write logic is the same as #writeCommittedByRegion, with some extra checks about region version and conf_version. using ResolveLocksAndWriteRegionRes = std::variant; static ResolveLocksAndWriteRegionRes resolveLocksAndWriteRegion( TMTContext & tmt, diff --git a/dbms/src/Storages/KVStore/MultiRaft/RaftCommands.cpp b/dbms/src/Storages/KVStore/MultiRaft/RaftCommands.cpp index f481e736771..86dd9c2ee64 100644 --- a/dbms/src/Storages/KVStore/MultiRaft/RaftCommands.cpp +++ b/dbms/src/Storages/KVStore/MultiRaft/RaftCommands.cpp @@ -453,12 +453,12 @@ std::pair Region::handleWriteRaftCmd( DM::WriteResult write_result = std::nullopt; { { - // RegionTable::writeBlockByRegion may lead to persistRegion when flush proactively. + // RegionTable::writeCommittedByRegion may lead to persistRegion when flush proactively. // So we can't lock here. // Safety: Mutations to a region come from raft applying and bg flushing of storage layer. // 1. A raft applying process should acquire the region task lock. // 2. While bg/fg flushing, applying raft logs should also be prevented with region task lock. - // So between here and RegionTable::writeBlockByRegion, there will be no new data applied. + // So between here and RegionTable::writeCommittedByRegion, there will be no new data applied. std::unique_lock lock(mutex); handle_write_cmd_func(); } @@ -471,7 +471,7 @@ std::pair Region::handleWriteRaftCmd( try { write_result - = RegionTable::writeBlockByRegion(context, shared_from_this(), data_list_to_remove, log, true); + = RegionTable::writeCommittedByRegion(context, shared_from_this(), data_list_to_remove, log, true); } catch (DB::Exception & e) { @@ -489,7 +489,8 @@ std::pair Region::handleWriteRaftCmd( } LOG_ERROR( log, - "{} catch exception: {}, while applying `RegionTable::writeBlockByRegion` on [term {}, index {}], " + "{} catch exception: {}, while applying `RegionTable::writeCommittedByRegion` on [term {}, index " + "{}], " "entries {}", toString(), e.message(), From 4503747742f109f7e42a13279f9665230fbc348e Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Fri, 2 Feb 2024 12:44:54 +0800 Subject: [PATCH 03/37] template ReadList Signed-off-by: CalvinNeo --- .../KVStore/Decode/PartitionStreams.cpp | 245 +++++++++--------- 1 file changed, 128 insertions(+), 117 deletions(-) diff --git a/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp b/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp index 1e6a665366e..f6584b5eb06 100644 --- a/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp +++ b/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp @@ -53,138 +53,149 @@ extern const int ILLFORMAT_RAFT_ROW; extern const int TABLE_IS_DROPPED; } // namespace ErrorCodes - -static DM::WriteResult writeRegionDataToStorage( - Context & context, +template +static inline bool atomicReadWrite( + const LoggerPtr & log, + const Context & context, + const TMTContext & tmt, + KeyspaceID keyspace_id, + TableID table_id, const RegionPtrWithBlock & region, - RegionDataReadInfoList & data_list_read, - const LoggerPtr & log) -{ - const auto & tmt = context.getTMTContext(); - const auto keyspace_id = region->getKeyspaceID(); - const auto table_id = region->getMappedTableID(); + ReadList & data_list_read, + bool force_decode, + DM::WriteResult & write_result +) { UInt64 region_decode_cost = -1, write_part_cost = -1; + /// Get storage based on table ID. + auto storage = tmt.getStorages().get(keyspace_id, table_id); + if (storage == nullptr) + { + // - force_decode == false and storage not exist, let upper level sync schema and retry. + // - force_decode == true and storage not exist. It could be the RaftLog or Snapshot comes + // after the schema is totally exceed the GC safepoint. And TiFlash know nothing about + // the schema. We can only throw away those committed rows. + return force_decode; + } - DM::WriteResult write_result = std::nullopt; - /// Declare lambda of atomic read then write to call multiple times. - auto atomic_read_write = [&](bool force_decode) { - /// Get storage based on table ID. - auto storage = tmt.getStorages().get(keyspace_id, table_id); - if (storage == nullptr) - { - // - force_decode == false and storage not exist, let upper level sync schema and retry. - // - force_decode == true and storage not exist. It could be the RaftLog or Snapshot comes - // after the schema is totally exceed the GC safepoint. And TiFlash know nothing about - // the schema. We can only throw away those committed rows. - return force_decode; - } + /// Get a structure read lock throughout decode, during which schema must not change. + TableStructureLockHolder lock; + try + { + lock = storage->lockStructureForShare(getThreadNameAndID()); + } + catch (DB::Exception & e) + { + // If the storage is physical dropped (but not removed from `ManagedStorages`) when we want to write raft data into it, consider the write done. + if (e.code() == ErrorCodes::TABLE_IS_DROPPED) + return true; + else + throw; + } - /// Get a structure read lock throughout decode, during which schema must not change. - TableStructureLockHolder lock; - try - { - lock = storage->lockStructureForShare(getThreadNameAndID()); - } - catch (DB::Exception & e) - { - // If the storage is physical dropped (but not removed from `ManagedStorages`) when we want to write raft data into it, consider the write done. - if (e.code() == ErrorCodes::TABLE_IS_DROPPED) - return true; - else - throw; - } + Block block; + bool need_decode = true; - Block block; - bool need_decode = true; + // try to use block cache if exists + if (region.pre_decode_cache) + { + auto schema_version = storage->getTableInfo().schema_version; + std::stringstream ss; + region.pre_decode_cache->toString(ss); + LOG_DEBUG( + log, + "{} got pre-decode cache {}, storage schema version: {}", + region->toString(), + ss.str(), + schema_version); - // try to use block cache if exists - if (region.pre_decode_cache) + if (region.pre_decode_cache->schema_version == schema_version) { - auto schema_version = storage->getTableInfo().schema_version; - std::stringstream ss; - region.pre_decode_cache->toString(ss); - LOG_DEBUG( - log, - "{} got pre-decode cache {}, storage schema version: {}", - region->toString(), - ss.str(), - schema_version); - - if (region.pre_decode_cache->schema_version == schema_version) - { - block = std::move(region.pre_decode_cache->block); - need_decode = false; - } - else - { - LOG_DEBUG(log, "schema version not equal, try to re-decode region cache into block"); - region.pre_decode_cache->block.clear(); - } + block = std::move(region.pre_decode_cache->block); + need_decode = false; + } + else + { + LOG_DEBUG(log, "schema version not equal, try to re-decode region cache into block"); + region.pre_decode_cache->block.clear(); } + } - /// Read region data as block. - Stopwatch watch; + /// Read region data as block. + Stopwatch watch; + Int64 block_decoding_schema_epoch = -1; + BlockUPtr block_ptr = nullptr; + if (need_decode) + { + LOG_TRACE(log, "begin to decode keyspace={} table_id={} region_id={}", keyspace_id, table_id, region->id()); + DecodingStorageSchemaSnapshotConstPtr decoding_schema_snapshot; + std::tie(decoding_schema_snapshot, block_ptr) = storage->getSchemaSnapshotAndBlockForDecoding(lock, true); + block_decoding_schema_epoch = decoding_schema_snapshot->decoding_schema_epoch; + + auto reader = RegionBlockReader(decoding_schema_snapshot); + if (!reader.read(*block_ptr, data_list_read, force_decode)) + return false; + region_decode_cost = watch.elapsedMilliseconds(); + GET_METRIC(tiflash_raft_write_data_to_storage_duration_seconds, type_decode) + .Observe(region_decode_cost / 1000.0); + } - Int64 block_decoding_schema_epoch = -1; - BlockUPtr block_ptr = nullptr; + /// Write block into storage. + // Release the alter lock so that writing does not block DDL operations + TableLockHolder drop_lock; + std::tie(std::ignore, drop_lock) = std::move(lock).release(); + watch.restart(); + // Note: do NOT use typeid_cast, since Storage is multi-inherited and typeid_cast will return nullptr + switch (storage->engineType()) + { + case ::TiDB::StorageEngine::DT: + { + auto dm_storage = std::dynamic_pointer_cast(storage); if (need_decode) { - LOG_TRACE(log, "begin to decode keyspace={} table_id={} region_id={}", keyspace_id, table_id, region->id()); - DecodingStorageSchemaSnapshotConstPtr decoding_schema_snapshot; - std::tie(decoding_schema_snapshot, block_ptr) = storage->getSchemaSnapshotAndBlockForDecoding(lock, true); - block_decoding_schema_epoch = decoding_schema_snapshot->decoding_schema_epoch; - - auto reader = RegionBlockReader(decoding_schema_snapshot); - if (!reader.read(*block_ptr, data_list_read, force_decode)) - return false; - region_decode_cost = watch.elapsedMilliseconds(); - GET_METRIC(tiflash_raft_write_data_to_storage_duration_seconds, type_decode) - .Observe(region_decode_cost / 1000.0); + write_result = dm_storage->write(*block_ptr, context.getSettingsRef()); } - - /// Write block into storage. - // Release the alter lock so that writing does not block DDL operations - TableLockHolder drop_lock; - std::tie(std::ignore, drop_lock) = std::move(lock).release(); - watch.restart(); - // Note: do NOT use typeid_cast, since Storage is multi-inherited and typeid_cast will return nullptr - switch (storage->engineType()) - { - case ::TiDB::StorageEngine::DT: + else { - auto dm_storage = std::dynamic_pointer_cast(storage); - if (need_decode) - { - write_result = dm_storage->write(*block_ptr, context.getSettingsRef()); - } - else - { - write_result = dm_storage->write(block, context.getSettingsRef()); - } - break; - } - default: - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Unknown StorageEngine: {}", - static_cast(storage->engineType())); + write_result = dm_storage->write(block, context.getSettingsRef()); } + break; + } + default: + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Unknown StorageEngine: {}", + static_cast(storage->engineType())); + } - write_part_cost = watch.elapsedMilliseconds(); - GET_METRIC(tiflash_raft_write_data_to_storage_duration_seconds, type_write).Observe(write_part_cost / 1000.0); - if (need_decode) - storage->releaseDecodingBlock(block_decoding_schema_epoch, std::move(block_ptr)); + write_part_cost = watch.elapsedMilliseconds(); + GET_METRIC(tiflash_raft_write_data_to_storage_duration_seconds, type_write).Observe(write_part_cost / 1000.0); + if (need_decode) + storage->releaseDecodingBlock(block_decoding_schema_epoch, std::move(block_ptr)); - LOG_TRACE( - log, - "keyspace={} table_id={} region_id={} cost [region decode {}, write part {}] ms", - keyspace_id, - table_id, - region->id(), - region_decode_cost, - write_part_cost); - return true; - }; + LOG_TRACE( + log, + "keyspace={} table_id={} region_id={} cost [region decode {}, write part {}] ms", + keyspace_id, + table_id, + region->id(), + region_decode_cost, + write_part_cost); + return true; +} + +// ReadList could be RegionDataReadInfoList +template +static DM::WriteResult writeRegionDataToStorage( + Context & context, + const RegionPtrWithBlock & region, + ReadList & data_list_read, + const LoggerPtr & log) +{ + const auto & tmt = context.getTMTContext(); + const auto keyspace_id = region->getKeyspaceID(); + const auto table_id = region->getMappedTableID(); + + DM::WriteResult write_result = std::nullopt; /// In TiFlash, the actions between applying raft log and schema changes are not strictly synchronized. /// There could be a chance that some raft logs come after a table gets tombstoned. Take care of it when @@ -201,7 +212,7 @@ static DM::WriteResult writeRegionDataToStorage( /// Try read then write once. { - if (atomic_read_write(false)) + if (atomicReadWrite(log, context, tmt, keyspace_id, table_id, region, data_list_read, false, write_result)) { return write_result; } @@ -214,7 +225,7 @@ static DM::WriteResult writeRegionDataToStorage( tmt.getSchemaSyncerManager()->syncTableSchema(context, keyspace_id, table_id); auto schema_sync_cost = watch.elapsedMilliseconds(); LOG_INFO(log, "sync schema cost {} ms, keyspace={} table_id={}", schema_sync_cost, keyspace_id, table_id); - if (!atomic_read_write(true)) + if (!atomicReadWrite(log, context, tmt, keyspace_id, table_id, region, data_list_read, true, write_result)) { // Failure won't be tolerated this time. throw Exception( From 815b236b9d5e8011acd2003dc14c962a142d2c32 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Fri, 2 Feb 2024 17:43:57 +0800 Subject: [PATCH 04/37] some basic spill structure Signed-off-by: CalvinNeo --- .../KVStore/Decode/PartitionStreams.cpp | 4 +- .../Spill/RegionUncommittedDataList.h | 54 +++++++++++++++++++ .../Storages/KVStore/MultiRaft/Spill/Spill.h | 38 +++++++++++++ .../KVStore/MultiRaft/Spill/SpillImpl.h | 21 ++++++++ dbms/src/Storages/KVStore/Region.h | 8 ++- 5 files changed, 122 insertions(+), 3 deletions(-) create mode 100644 dbms/src/Storages/KVStore/MultiRaft/Spill/RegionUncommittedDataList.h create mode 100644 dbms/src/Storages/KVStore/MultiRaft/Spill/Spill.h create mode 100644 dbms/src/Storages/KVStore/MultiRaft/Spill/SpillImpl.h diff --git a/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp b/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp index f6584b5eb06..34d5a07011c 100644 --- a/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp +++ b/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp @@ -63,8 +63,8 @@ static inline bool atomicReadWrite( const RegionPtrWithBlock & region, ReadList & data_list_read, bool force_decode, - DM::WriteResult & write_result -) { + DM::WriteResult & write_result) +{ UInt64 region_decode_cost = -1, write_part_cost = -1; /// Get storage based on table ID. auto storage = tmt.getStorages().get(keyspace_id, table_id); diff --git a/dbms/src/Storages/KVStore/MultiRaft/Spill/RegionUncommittedDataList.h b/dbms/src/Storages/KVStore/MultiRaft/Spill/RegionUncommittedDataList.h new file mode 100644 index 00000000000..f62070402d7 --- /dev/null +++ b/dbms/src/Storages/KVStore/MultiRaft/Spill/RegionUncommittedDataList.h @@ -0,0 +1,54 @@ +// Copyright 2024 PingCAP, Inc. +// +// Licensed 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 +#include + +namespace DB +{ +struct RegionUncommittedData +{ + RegionUncommittedData(RawTiDBPK && pk_, UInt8 && write_type_, std::shared_ptr && value_) + : pk(std::move(pk_)) + , write_type(write_type_) + , value(std::move(value_)) + {} + RegionUncommittedData( + const RawTiDBPK & pk_, + const UInt8 & write_type_, + const std::shared_ptr & value_) + : pk(pk_) + , write_type(write_type_) + , value(value_) + {} + RegionUncommittedData(const RegionDataReadInfo &) = default; + RegionUncommittedData(RegionDataReadInfo &&) = default; + RegionUncommittedData & operator=(const RegionDataReadInfo &) = default; + RegionUncommittedData & operator=(RegionDataReadInfo &&) = default; + +public: + RawTiDBPK pk; + UInt8 write_type; + std::shared_ptr value; +}; + +struct RegionUncommittedDataList +{ + std::vector data; + Timestamp start_ts; +} + +} // namespace DB \ No newline at end of file diff --git a/dbms/src/Storages/KVStore/MultiRaft/Spill/Spill.h b/dbms/src/Storages/KVStore/MultiRaft/Spill/Spill.h new file mode 100644 index 00000000000..af193766765 --- /dev/null +++ b/dbms/src/Storages/KVStore/MultiRaft/Spill/Spill.h @@ -0,0 +1,38 @@ +// Copyright 2024 PingCAP, Inc. +// +// Licensed 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 +#include +#include + +namespace DB +{ + +struct SpillFileVersion +{ +}; + +struct SpillingTxn +{ + RegionDefaultCFData default_cf; +}; + +struct SpilledMemtable +{ + // start_ts -> SpillingTxn +}; + +} // namespace DB diff --git a/dbms/src/Storages/KVStore/MultiRaft/Spill/SpillImpl.h b/dbms/src/Storages/KVStore/MultiRaft/Spill/SpillImpl.h new file mode 100644 index 00000000000..9fd03838eb0 --- /dev/null +++ b/dbms/src/Storages/KVStore/MultiRaft/Spill/SpillImpl.h @@ -0,0 +1,21 @@ +// Copyright 2024 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#pragma once + +namespace DB +{ + + +} // namespace DB diff --git a/dbms/src/Storages/KVStore/Region.h b/dbms/src/Storages/KVStore/Region.h index 52d068af6df..491c05807be 100644 --- a/dbms/src/Storages/KVStore/Region.h +++ b/dbms/src/Storages/KVStore/Region.h @@ -20,6 +20,7 @@ #include #include #include +#include #include #include @@ -120,6 +121,7 @@ class Region : public std::enable_shared_from_this public: // Simple Read and Write explicit Region(RegionMeta && meta_, const TiFlashRaftProxyHelper *); + Region() = delete; ~Region(); void insert(const std::string & cf, TiKVKey && key, TiKVValue && value, DupCheck mode = DupCheck::Deny); @@ -253,7 +255,11 @@ class Region : public std::enable_shared_from_this void beforePrehandleSnapshot(uint64_t region_id, std::optional deadline_index); void afterPrehandleSnapshot(int64_t ongoing); - Region() = delete; +public: // Spill + SpilledMemtable spillDefaultCf(const TiKVKey & start_ts, const RegionTaskLock &); + + // Don't require Region task lock + void ingestSpilledFile(const RegionRange & region_range, Timestamp start_ts, SpillFileVersion spill_file_version); private: friend class RegionRaftCommandDelegate; From 690cf941cd5119b2d4053ee1747fb83c6974d41f Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Sat, 3 Feb 2024 00:40:32 +0800 Subject: [PATCH 05/37] try decode without version Signed-off-by: CalvinNeo --- dbms/src/Debug/dbgFuncMockRaftSnapshot.cpp | 3 +- .../tests/gtest_sst_files_stream.cpp | 20 ++--- dbms/src/Storages/IManageableStorage.h | 3 +- .../Decode/DecodingStorageSchemaSnapshot.cpp | 10 ++- .../Decode/DecodingStorageSchemaSnapshot.h | 2 +- .../KVStore/Decode/PartitionStreams.cpp | 5 +- .../KVStore/Decode/RegionBlockReader.cpp | 31 +++++-- .../KVStore/Decode/RegionBlockReader.h | 7 +- .../KVStore/MultiRaft/PrehandleSnapshot.cpp | 2 + .../Storages/KVStore/tests/gtest_spill.cpp | 82 +++++++++++++++++++ dbms/src/Storages/StorageDeltaMerge.cpp | 8 +- dbms/src/Storages/StorageDeltaMerge.h | 3 +- 12 files changed, 145 insertions(+), 31 deletions(-) create mode 100644 dbms/src/Storages/KVStore/tests/gtest_spill.cpp diff --git a/dbms/src/Debug/dbgFuncMockRaftSnapshot.cpp b/dbms/src/Debug/dbgFuncMockRaftSnapshot.cpp index 7d651b3cc25..99368435cd1 100644 --- a/dbms/src/Debug/dbgFuncMockRaftSnapshot.cpp +++ b/dbms/src/Debug/dbgFuncMockRaftSnapshot.cpp @@ -610,7 +610,8 @@ RegionPtrWithBlock::CachePtr GenRegionPreDecodeBlockData(const RegionPtr & regio } DecodingStorageSchemaSnapshotConstPtr decoding_schema_snapshot; - std::tie(decoding_schema_snapshot, std::ignore) = storage->getSchemaSnapshotAndBlockForDecoding(lock, false); + std::tie(decoding_schema_snapshot, std::ignore) + = storage->getSchemaSnapshotAndBlockForDecoding(lock, false, true); res_block = createBlockSortByColumnID(decoding_schema_snapshot); auto reader = RegionBlockReader(decoding_schema_snapshot); return reader.read(res_block, *data_list_read, force_decode); diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_sst_files_stream.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_sst_files_stream.cpp index 142f5b3b814..cab17d4d709 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_sst_files_stream.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_sst_files_stream.cpp @@ -179,7 +179,7 @@ TEST_F(SSTFilesToDTFilesOutputStreamTest, OutputNoDTFile) try { auto table_lock = storage->lockStructureForShare("foo_query_id"); - auto [schema_snapshot, unused] = storage->getSchemaSnapshotAndBlockForDecoding(table_lock, false); + auto [schema_snapshot, unused] = storage->getSchemaSnapshotAndBlockForDecoding(table_lock, false, true); auto mock_stream = makeMockChild(prepareBlocks(100, 100, /*block_size=*/5)); auto prehandle_task = std::make_shared(); @@ -209,7 +209,7 @@ TEST_F(SSTFilesToDTFilesOutputStreamTest, OutputSingleDTFile) try { auto table_lock = storage->lockStructureForShare("foo_query_id"); - auto [schema_snapshot, unused] = storage->getSchemaSnapshotAndBlockForDecoding(table_lock, false); + auto [schema_snapshot, unused] = storage->getSchemaSnapshotAndBlockForDecoding(table_lock, false, true); auto mock_stream = makeMockChild(prepareBlocks(50, 100, /*block_size=*/5)); auto prehandle_task = std::make_shared(); @@ -241,7 +241,7 @@ TEST_F(SSTFilesToDTFilesOutputStreamTest, OutputSingleDTFileWithOneBlock) try { auto table_lock = storage->lockStructureForShare("foo_query_id"); - auto [schema_snapshot, unused] = storage->getSchemaSnapshotAndBlockForDecoding(table_lock, false); + auto [schema_snapshot, unused] = storage->getSchemaSnapshotAndBlockForDecoding(table_lock, false, true); auto mock_stream = makeMockChild(prepareBlocks(50, 100, /*block_size=*/1000)); auto prehandle_task = std::make_shared(); @@ -274,7 +274,7 @@ TEST_F(SSTFilesToDTFilesOutputStreamTest, OutputMultipleDTFile) try { auto table_lock = storage->lockStructureForShare("foo_query_id"); - auto [schema_snapshot, unused] = storage->getSchemaSnapshotAndBlockForDecoding(table_lock, false); + auto [schema_snapshot, unused] = storage->getSchemaSnapshotAndBlockForDecoding(table_lock, false, true); auto mock_stream = makeMockChild(prepareBlocks(50, 100, /*block_size=*/1)); auto prehandle_task = std::make_shared(); @@ -313,7 +313,7 @@ TEST_F(SSTFilesToDTFilesOutputStreamTest, SplitAtBlockBoundary) try { auto table_lock = storage->lockStructureForShare("foo_query_id"); - auto [schema_snapshot, unused] = storage->getSchemaSnapshotAndBlockForDecoding(table_lock, false); + auto [schema_snapshot, unused] = storage->getSchemaSnapshotAndBlockForDecoding(table_lock, false, true); auto mock_stream = makeMockChild(prepareBlocks(50, 100, /*block_size=*/20)); auto prehandle_task = std::make_shared(); @@ -348,7 +348,7 @@ TEST_F(SSTFilesToDTFilesOutputStreamTest, VeryLargeSplitThreshold) try { auto table_lock = storage->lockStructureForShare("foo_query_id"); - auto [schema_snapshot, unused] = storage->getSchemaSnapshotAndBlockForDecoding(table_lock, false); + auto [schema_snapshot, unused] = storage->getSchemaSnapshotAndBlockForDecoding(table_lock, false, true); auto mock_stream = makeMockChild(prepareBlocks(50, 100, /*block_size=*/20)); auto prehandle_task = std::make_shared(); @@ -380,7 +380,7 @@ TEST_F(SSTFilesToDTFilesOutputStreamTest, NonContinuousBlock) try { auto table_lock = storage->lockStructureForShare("foo_query_id"); - auto [schema_snapshot, unused] = storage->getSchemaSnapshotAndBlockForDecoding(table_lock, false); + auto [schema_snapshot, unused] = storage->getSchemaSnapshotAndBlockForDecoding(table_lock, false, true); auto blocks1 = prepareBlocks(50, 100, /*block_size=*/20); auto blocks2 = prepareBlocks(130, 150, /*block_size=*/10); @@ -422,7 +422,7 @@ TEST_F(SSTFilesToDTFilesOutputStreamTest, BrokenChild) try { auto table_lock = storage->lockStructureForShare("foo_query_id"); - auto [schema_snapshot, unused] = storage->getSchemaSnapshotAndBlockForDecoding(table_lock, false); + auto [schema_snapshot, unused] = storage->getSchemaSnapshotAndBlockForDecoding(table_lock, false, true); auto blocks1 = prepareBlocks(50, 100, /*block_size=*/20); auto blocks2 = prepareBlocks(0, 30, /*block_size=*/20); @@ -458,7 +458,7 @@ TEST_F(SSTFilesToDTFilesOutputStreamTest, Cancel) try { auto table_lock = storage->lockStructureForShare("foo_query_id"); - auto [schema_snapshot, unused] = storage->getSchemaSnapshotAndBlockForDecoding(table_lock, false); + auto [schema_snapshot, unused] = storage->getSchemaSnapshotAndBlockForDecoding(table_lock, false, true); auto mock_stream = makeMockChild(prepareBlocks(50, 100, /*block_size=*/1)); auto prehandle_task = std::make_shared(); @@ -507,7 +507,7 @@ TEST_F(SSTFilesToDTFilesOutputStreamTest, UpperLayerCancel) try { auto table_lock = storage->lockStructureForShare("foo_query_id"); - auto [schema_snapshot, unused] = storage->getSchemaSnapshotAndBlockForDecoding(table_lock, false); + auto [schema_snapshot, unused] = storage->getSchemaSnapshotAndBlockForDecoding(table_lock, false, true); auto mock_stream = makeMockChild(prepareBlocks(50, 100, /*block_size=*/1)); auto prehandle_task = std::make_shared(); diff --git a/dbms/src/Storages/IManageableStorage.h b/dbms/src/Storages/IManageableStorage.h index e887e4dddc2..05f3db588f1 100644 --- a/dbms/src/Storages/IManageableStorage.h +++ b/dbms/src/Storages/IManageableStorage.h @@ -180,7 +180,8 @@ class IManageableStorage : public IStorage /// This method must be called under the protection of table structure lock virtual std::pair getSchemaSnapshotAndBlockForDecoding( const TableStructureLockHolder & /* table_structure_lock */, - bool /* need_block */) + bool /* need_block */, + bool /* has_version_block */) { throw Exception( "Method getDecodingSchemaSnapshot is not supported by storage " + getName(), diff --git a/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.cpp b/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.cpp index c9e77206959..584f002613c 100644 --- a/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.cpp +++ b/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.cpp @@ -32,15 +32,23 @@ TMTPKType getTMTPKType(const IDataType & rhs) return TMTPKType::UNSPECIFIED; } -Block createBlockSortByColumnID(DecodingStorageSchemaSnapshotConstPtr schema_snapshot) +Block createBlockSortByColumnID(DecodingStorageSchemaSnapshotConstPtr schema_snapshot, bool has_version_column) { + UNUSED(has_version_column); Block block; for (auto iter = schema_snapshot->sorted_column_id_with_pos.begin(); iter != schema_snapshot->sorted_column_id_with_pos.end(); iter++) { + // col_id == cd.id + // Including some internal columns: + // - (VersionColumnID, _INTERNAL_VERSION, u64) + // - (DelMarkColumnID, _INTERNAL_DELMARK, u8) + // - (TiDBPkColumnID, _tidb_rowid, i64) auto col_id = iter->first; auto & cd = (*(schema_snapshot->column_defines))[iter->second]; + if (!has_version_column && cd.id == VersionColumnID) + continue; block.insert({cd.type->createColumn(), cd.type, cd.name, col_id}); } return block; diff --git a/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.h b/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.h index d52439d91bc..5a68c68c028 100644 --- a/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.h +++ b/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.h @@ -156,7 +156,7 @@ struct DecodingStorageSchemaSnapshot using DecodingStorageSchemaSnapshotPtr = std::shared_ptr; using DecodingStorageSchemaSnapshotConstPtr = std::shared_ptr; -Block createBlockSortByColumnID(DecodingStorageSchemaSnapshotConstPtr schema_snapshot); +Block createBlockSortByColumnID(DecodingStorageSchemaSnapshotConstPtr schema_snapshot, bool has_version_column = true); void clearBlockData(Block & block); diff --git a/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp b/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp index 34d5a07011c..e507d7ad2e1 100644 --- a/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp +++ b/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp @@ -53,6 +53,7 @@ extern const int ILLFORMAT_RAFT_ROW; extern const int TABLE_IS_DROPPED; } // namespace ErrorCodes +// TODO Fix too many arguments template static inline bool atomicReadWrite( const LoggerPtr & log, @@ -128,7 +129,7 @@ static inline bool atomicReadWrite( { LOG_TRACE(log, "begin to decode keyspace={} table_id={} region_id={}", keyspace_id, table_id, region->id()); DecodingStorageSchemaSnapshotConstPtr decoding_schema_snapshot; - std::tie(decoding_schema_snapshot, block_ptr) = storage->getSchemaSnapshotAndBlockForDecoding(lock, true); + std::tie(decoding_schema_snapshot, block_ptr) = storage->getSchemaSnapshotAndBlockForDecoding(lock, true, true); block_decoding_schema_epoch = decoding_schema_snapshot->decoding_schema_epoch; auto reader = RegionBlockReader(decoding_schema_snapshot); @@ -479,7 +480,7 @@ AtomicGetStorageSchema(const RegionPtr & region, TMTContext & tmt) auto table_lock = storage->lockStructureForShare(getThreadNameAndID()); dm_storage = std::dynamic_pointer_cast(storage); // only dt storage engine support `getSchemaSnapshotAndBlockForDecoding`, other engine will throw exception - std::tie(schema_snapshot, std::ignore) = storage->getSchemaSnapshotAndBlockForDecoding(table_lock, false); + std::tie(schema_snapshot, std::ignore) = storage->getSchemaSnapshotAndBlockForDecoding(table_lock, false, true); std::tie(std::ignore, drop_lock) = std::move(table_lock).release(); return true; }; diff --git a/dbms/src/Storages/KVStore/Decode/RegionBlockReader.cpp b/dbms/src/Storages/KVStore/Decode/RegionBlockReader.cpp index c3462c84b82..86de3ff2dcf 100644 --- a/dbms/src/Storages/KVStore/Decode/RegionBlockReader.cpp +++ b/dbms/src/Storages/KVStore/Decode/RegionBlockReader.cpp @@ -38,8 +38,8 @@ RegionBlockReader::RegionBlockReader(DecodingStorageSchemaSnapshotConstPtr schem : schema_snapshot{std::move(schema_snapshot_)} {} - -bool RegionBlockReader::read(Block & block, const RegionDataReadInfoList & data_list, bool force_decode) +template +bool RegionBlockReader::read(Block & block, const ReadList & data_list, bool force_decode) { try { @@ -105,8 +105,23 @@ bool RegionBlockReader::read(Block & block, const RegionDataReadInfoList & data_ } } -template -bool RegionBlockReader::readImpl(Block & block, const RegionDataReadInfoList & data_list, bool force_decode) +template bool RegionBlockReader::read( + Block & block, + const RegionDataReadInfoList & data_list, + bool force_decode); + +template +static size_t getExpectedReservedColCount() +{ + if constexpr (std::is_same_v) + { + return 3; + } + return 2; +} + +template +bool RegionBlockReader::readImpl(Block & block, const ReadList & data_list, bool force_decode) { if (unlikely(block.columns() != schema_snapshot->column_defines->size())) throw Exception("block structure doesn't match schema_snapshot.", ErrorCodes::LOGICAL_ERROR); @@ -150,15 +165,15 @@ bool RegionBlockReader::readImpl(Block & block, const RegionDataReadInfoList & d column_ids_iter++; } // extra handle, del, version must exists - constexpr size_t MustHaveColCnt = 3; // NOLINT(readability-identifier-naming) - if (unlikely(next_column_pos != MustHaveColCnt)) - throw Exception("del, version column must exist before all other visible columns.", ErrorCodes::LOGICAL_ERROR); + if (unlikely(next_column_pos != getExpectedReservedColCount())) + throw Exception(ErrorCodes::LOGICAL_ERROR, "del, version column mismatch, actual_size={}", next_column_pos); + // constexpr bool has_version_col = std::is_same_v(); ColumnUInt8::Container & delmark_data = raw_delmark_col->getData(); ColumnUInt64::Container & version_data = raw_version_col->getData(); delmark_data.reserve(data_list.size()); version_data.reserve(data_list.size()); - bool need_decode_value = block.columns() > MustHaveColCnt; + bool need_decode_value = block.columns() > getExpectedReservedColCount(); if (need_decode_value) { size_t expected_rows = data_list.size(); diff --git a/dbms/src/Storages/KVStore/Decode/RegionBlockReader.h b/dbms/src/Storages/KVStore/Decode/RegionBlockReader.h index 1d074d7f4de..f404e872d6f 100644 --- a/dbms/src/Storages/KVStore/Decode/RegionBlockReader.h +++ b/dbms/src/Storages/KVStore/Decode/RegionBlockReader.h @@ -35,11 +35,12 @@ class RegionBlockReader : private boost::noncopyable /// /// `RegionBlockReader::read` is the common routine used by both 'flush' and 'read' processes of Delta-Tree engine, /// which will use carefully adjusted 'force_decode' with appropriate error handling/retry to get what they want. - bool read(Block & block, const RegionDataReadInfoList & data_list, bool force_decode); + template + bool read(Block & block, const ReadList & data_list, bool force_decode); private: - template - bool readImpl(Block & block, const RegionDataReadInfoList & data_list, bool force_decode); + template + bool readImpl(Block & block, const ReadList & data_list, bool force_decode); private: DecodingStorageSchemaSnapshotConstPtr schema_snapshot; diff --git a/dbms/src/Storages/KVStore/MultiRaft/PrehandleSnapshot.cpp b/dbms/src/Storages/KVStore/MultiRaft/PrehandleSnapshot.cpp index 3d475de30d4..75b2ccbddd3 100644 --- a/dbms/src/Storages/KVStore/MultiRaft/PrehandleSnapshot.cpp +++ b/dbms/src/Storages/KVStore/MultiRaft/PrehandleSnapshot.cpp @@ -403,6 +403,7 @@ struct ParallelPrehandleCtx }; using ParallelPrehandleCtxPtr = std::shared_ptr; +// TODO fix the very long argument list static void runInParallel( LoggerPtr log, RegionPtr new_region, @@ -483,6 +484,7 @@ static void runInParallel( } } +// TODO fix the very long argument list void executeParallelTransform( LoggerPtr log, ReadFromStreamResult & result, diff --git a/dbms/src/Storages/KVStore/tests/gtest_spill.cpp b/dbms/src/Storages/KVStore/tests/gtest_spill.cpp new file mode 100644 index 00000000000..bf2215fb42c --- /dev/null +++ b/dbms/src/Storages/KVStore/tests/gtest_spill.cpp @@ -0,0 +1,82 @@ +// Copyright 2024 PingCAP, Inc. +// +// Licensed 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 + +namespace DB::tests +{ +class KVStoreSpillTest : public KVStoreTestBase +{ +public: + void SetUp() override + { + log = DB::Logger::get("KVStoreSpillTest"); + KVStoreTestBase::SetUp(); + setupStorage(); + } + + void TearDown() override + { + storage->drop(); + KVStoreTestBase::TearDown(); + } + + void setupStorage() + { + auto & ctx = TiFlashTestEnv::getGlobalContext(); + auto columns = DM::tests::DMTestEnv::getDefaultTableColumns(pk_type); + auto table_info = DM::tests::DMTestEnv::getMinimalTableInfo(/* table id */ 100, pk_type); + auto astptr = DM::tests::DMTestEnv::getPrimaryKeyExpr("test_table", pk_type); + + storage = StorageDeltaMerge::create( + "TiFlash", + "default" /* db_name */, + "test_table" /* table_name */, + table_info, + ColumnsDescription{columns}, + astptr, + 0, + ctx); + storage->startup(); + } + +protected: + StorageDeltaMergePtr storage; + DM::tests::DMTestEnv::PkType pk_type = DM::tests::DMTestEnv::PkType::HiddenTiDBRowID; +}; + +TEST_F(KVStoreSpillTest, KVStoreSpill) +try +{ + auto table_lock = storage->lockStructureForShare("foo_query_id"); + { + auto [schema_snapshot, block] = storage->getSchemaSnapshotAndBlockForDecoding(table_lock, true, true); + UNUSED(schema_snapshot); + ASSERT_EQ(block->columns(), 3); + } + { + auto [schema_snapshot, block] = storage->getSchemaSnapshotAndBlockForDecoding(table_lock, true, false); + UNUSED(schema_snapshot); + EXPECT_NO_THROW(block->getPositionByName(MutableSupport::delmark_column_name)); + EXPECT_THROW(block->getPositionByName(MutableSupport::version_column_name), Exception); + ASSERT_EQ(block->columns(), 2); + } +} +CATCH + +} // namespace DB::tests \ No newline at end of file diff --git a/dbms/src/Storages/StorageDeltaMerge.cpp b/dbms/src/Storages/StorageDeltaMerge.cpp index 7815b23a1c0..b1cb73d8afb 100644 --- a/dbms/src/Storages/StorageDeltaMerge.cpp +++ b/dbms/src/Storages/StorageDeltaMerge.cpp @@ -1303,7 +1303,8 @@ DM::DeltaMergeStorePtr StorageDeltaMerge::getStoreIfInited() const std::pair StorageDeltaMerge::getSchemaSnapshotAndBlockForDecoding( const TableStructureLockHolder & table_structure_lock, - bool need_block) + bool need_block, + bool has_version_column) { (void)table_structure_lock; std::lock_guard lock{decode_schema_mutex}; @@ -1321,9 +1322,10 @@ std::pair StorageDeltaMerg if (need_block) { - if (cache_blocks.empty()) + if (cache_blocks.empty() || !has_version_column) { - BlockUPtr block = std::make_unique(createBlockSortByColumnID(decoding_schema_snapshot)); + BlockUPtr block + = std::make_unique(createBlockSortByColumnID(decoding_schema_snapshot, has_version_column)); auto digest = hashSchema(*block); auto schema = global_context.getSharedBlockSchemas()->find(digest); if (schema) diff --git a/dbms/src/Storages/StorageDeltaMerge.h b/dbms/src/Storages/StorageDeltaMerge.h index e78f85db474..d91d8e12f57 100644 --- a/dbms/src/Storages/StorageDeltaMerge.h +++ b/dbms/src/Storages/StorageDeltaMerge.h @@ -201,7 +201,8 @@ class StorageDeltaMerge std::pair getSchemaSnapshotAndBlockForDecoding( const TableStructureLockHolder & table_structure_lock, - bool /* need_block */) override; + bool need_block, + bool has_version_column) override; void releaseDecodingBlock(Int64 block_decoding_schema_epoch, BlockUPtr block) override; From b67cd193972de4c29db145a81110f764599900cd Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Sun, 4 Feb 2024 21:33:49 +0800 Subject: [PATCH 06/37] still need to fix DecodingStorageSchemaSnapshot Signed-off-by: CalvinNeo --- .../KVStore/Decode/PartitionStreams.cpp | 7 +- .../KVStore/Decode/RegionBlockReader.cpp | 109 ++++++++++++++---- .../Spill/RegionUncommittedDataList.h | 42 ++++++- .../Storages/KVStore/tests/gtest_spill.cpp | 52 ++++++--- dbms/src/TiDB/Decode/RowCodec.cpp | 2 +- 5 files changed, 164 insertions(+), 48 deletions(-) diff --git a/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp b/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp index e507d7ad2e1..dce7287e3e3 100644 --- a/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp +++ b/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp @@ -13,6 +13,7 @@ // limitations under the License. #include +#include #include #include #include @@ -125,11 +126,15 @@ static inline bool atomicReadWrite( Stopwatch watch; Int64 block_decoding_schema_epoch = -1; BlockUPtr block_ptr = nullptr; + bool should_handle_version_col = true; + if constexpr (std::is_same_v) { + should_handle_version_col = false; + } if (need_decode) { LOG_TRACE(log, "begin to decode keyspace={} table_id={} region_id={}", keyspace_id, table_id, region->id()); DecodingStorageSchemaSnapshotConstPtr decoding_schema_snapshot; - std::tie(decoding_schema_snapshot, block_ptr) = storage->getSchemaSnapshotAndBlockForDecoding(lock, true, true); + std::tie(decoding_schema_snapshot, block_ptr) = storage->getSchemaSnapshotAndBlockForDecoding(lock, true, should_handle_version_col); block_decoding_schema_epoch = decoding_schema_snapshot->decoding_schema_epoch; auto reader = RegionBlockReader(decoding_schema_snapshot); diff --git a/dbms/src/Storages/KVStore/Decode/RegionBlockReader.cpp b/dbms/src/Storages/KVStore/Decode/RegionBlockReader.cpp index 86de3ff2dcf..a409591810e 100644 --- a/dbms/src/Storages/KVStore/Decode/RegionBlockReader.cpp +++ b/dbms/src/Storages/KVStore/Decode/RegionBlockReader.cpp @@ -20,6 +20,7 @@ #include #include #include +#include #include #include #include @@ -110,22 +111,78 @@ template bool RegionBlockReader::read( const RegionDataReadInfoList & data_list, bool force_decode); -template -static size_t getExpectedReservedColCount() -{ - if constexpr (std::is_same_v) - { +template bool RegionBlockReader::read( + Block & block, + const RegionUncommittedDataList & data_list, + bool force_decode); + +template +struct VersionColResolver { + VersionColResolver() {} + bool needBuild() const { + return raw_version_col == nullptr; + } + void build(ColumnUInt64 * raw_version_col_) { + raw_version_col = raw_version_col_; + } + void preRead(size_t size) { + RUNTIME_CHECK(raw_version_col); + ColumnUInt64::Container & version_data = raw_version_col->getData(); + version_data.reserve(size); + } + void read(const RegionDataReadInfo & info) { + RUNTIME_CHECK(raw_version_col); + ColumnUInt64::Container & version_data = raw_version_col->getData(); + version_data.emplace_back(info.commit_ts); + } + void check(const Block & block, size_t expected) const { + if (unlikely(block.columns() != expected)) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Block structure doesn't match schema_snapshot, block={} def={}", + block.columns(), + expected + ); + } + size_t reservedCount() const { return 3; } - return 2; -} +private: + ColumnUInt64 * raw_version_col = nullptr; +}; + +template<> +struct VersionColResolver { + VersionColResolver() {} + bool needBuild() const { + return raw_version_col == nullptr; + } + void build(ColumnUInt64 * raw_version_col_) { + raw_version_col = raw_version_col_; + } + void preRead(size_t) { + } + void read(const RegionUncommittedData &) { + } + void check(const Block & block, size_t expected) const { + if (unlikely(block.columns() + 1 != expected)) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Block structure doesn't match schema_snapshot, block={} def={}", + block.columns(), + expected + ); + } + size_t reservedCount() const { + return 2; + } +private: + ColumnUInt64 * raw_version_col = nullptr; +}; template bool RegionBlockReader::readImpl(Block & block, const ReadList & data_list, bool force_decode) { - if (unlikely(block.columns() != schema_snapshot->column_defines->size())) - throw Exception("block structure doesn't match schema_snapshot.", ErrorCodes::LOGICAL_ERROR); - + VersionColResolver version_col_resolver; + version_col_resolver.check(block, schema_snapshot->column_defines->size()); const auto & read_column_ids = schema_snapshot->sorted_column_id_with_pos; const auto & pk_column_ids = schema_snapshot->pk_column_ids; const auto & pk_pos_map = schema_snapshot->pk_pos_map; @@ -141,11 +198,11 @@ bool RegionBlockReader::readImpl(Block & block, const ReadList & data_list, bool /// extra handle, del, version column is with column id smaller than other visible column id, /// so they must exists before all other columns, and we can get them before decoding other columns ColumnUInt8 * raw_delmark_col = nullptr; - ColumnUInt64 * raw_version_col = nullptr; const size_t invalid_column_pos = std::numeric_limits::max(); // we cannot figure out extra_handle's column type now, so we just remember it's pos here size_t extra_handle_column_pos = invalid_column_pos; - while (raw_delmark_col == nullptr || raw_version_col == nullptr || extra_handle_column_pos == invalid_column_pos) + + while (raw_delmark_col == nullptr || version_col_resolver.needBuild() || extra_handle_column_pos == invalid_column_pos) { if (column_ids_iter->first == DelMarkColumnID) { @@ -154,8 +211,7 @@ bool RegionBlockReader::readImpl(Block & block, const ReadList & data_list, bool } else if (column_ids_iter->first == VersionColumnID) { - raw_version_col - = static_cast(const_cast(block.getByPosition(next_column_pos).column.get())); + version_col_resolver.build(static_cast(const_cast(block.getByPosition(next_column_pos).column.get()))); } else if (column_ids_iter->first == TiDBPkColumnID) { @@ -164,16 +220,15 @@ bool RegionBlockReader::readImpl(Block & block, const ReadList & data_list, bool next_column_pos++; column_ids_iter++; } - // extra handle, del, version must exists - if (unlikely(next_column_pos != getExpectedReservedColCount())) + // extra handle, del, version + constexpr size_t MustHaveCntInSchema = 3; + if (unlikely(next_column_pos != MustHaveCntInSchema)) throw Exception(ErrorCodes::LOGICAL_ERROR, "del, version column mismatch, actual_size={}", next_column_pos); - // constexpr bool has_version_col = std::is_same_v(); ColumnUInt8::Container & delmark_data = raw_delmark_col->getData(); - ColumnUInt64::Container & version_data = raw_version_col->getData(); delmark_data.reserve(data_list.size()); - version_data.reserve(data_list.size()); - bool need_decode_value = block.columns() > getExpectedReservedColCount(); + version_col_resolver.preRead(data_list.size()); + bool need_decode_value = block.columns() > version_col_resolver.reservedCount(); if (need_decode_value) { size_t expected_rows = data_list.size(); @@ -185,11 +240,14 @@ bool RegionBlockReader::readImpl(Block & block, const ReadList & data_list, bool } size_t index = 0; - for (const auto & [pk, write_type, commit_ts, value_ptr] : data_list) + for (const auto & item : data_list) { + const auto & pk = item.pk; + const auto & write_type = item.write_type; + const auto & value_ptr = item.value; /// set delmark and version column delmark_data.emplace_back(write_type == Region::DelFlag); - version_data.emplace_back(commit_ts); + version_col_resolver.read(item); if (need_decode_value) { @@ -213,6 +271,13 @@ bool RegionBlockReader::readImpl(Block & block, const ReadList & data_list, bool } else { + LOG_INFO(DB::Logger::get(), "!!!!!! Z next_column_pos {}", next_column_pos); + for (auto it = column_ids_iter; it != read_column_ids.end(); it ++) { + LOG_INFO(DB::Logger::get(), "!!!!!! Z col_id {}", it->first); + } + for (const auto & q : block.getNames()) { + LOG_INFO(DB::Logger::get(), "!!!!!! Z gq {}", q); + } // Parse column value from encoded value if (!appendRowToBlock( *value_ptr, diff --git a/dbms/src/Storages/KVStore/MultiRaft/Spill/RegionUncommittedDataList.h b/dbms/src/Storages/KVStore/MultiRaft/Spill/RegionUncommittedDataList.h index f62070402d7..78b5fea9514 100644 --- a/dbms/src/Storages/KVStore/MultiRaft/Spill/RegionUncommittedDataList.h +++ b/dbms/src/Storages/KVStore/MultiRaft/Spill/RegionUncommittedDataList.h @@ -34,10 +34,10 @@ struct RegionUncommittedData , write_type(write_type_) , value(value_) {} - RegionUncommittedData(const RegionDataReadInfo &) = default; - RegionUncommittedData(RegionDataReadInfo &&) = default; - RegionUncommittedData & operator=(const RegionDataReadInfo &) = default; - RegionUncommittedData & operator=(RegionDataReadInfo &&) = default; + RegionUncommittedData(const RegionUncommittedData &) = default; + RegionUncommittedData(RegionUncommittedData &&) = default; + RegionUncommittedData & operator=(const RegionUncommittedData &) = default; + RegionUncommittedData & operator=(RegionUncommittedData &&) = default; public: RawTiDBPK pk; @@ -47,8 +47,38 @@ struct RegionUncommittedData struct RegionUncommittedDataList { - std::vector data; + using Inner = std::vector; + + Inner data; Timestamp start_ts; -} + + Inner::const_iterator cbegin() const { + return data.cbegin(); + } + + Inner::const_iterator cend() const { + return data.cend(); + } + + Inner::iterator begin() { + return data.begin(); + } + + Inner::const_iterator begin() const { + return data.begin(); + } + + Inner::iterator end() { + return data.end(); + } + + Inner::const_iterator end() const { + return data.end(); + } + + size_t size() const { + return data.size(); + } +}; } // namespace DB \ No newline at end of file diff --git a/dbms/src/Storages/KVStore/tests/gtest_spill.cpp b/dbms/src/Storages/KVStore/tests/gtest_spill.cpp index bf2215fb42c..5532fe8883c 100644 --- a/dbms/src/Storages/KVStore/tests/gtest_spill.cpp +++ b/dbms/src/Storages/KVStore/tests/gtest_spill.cpp @@ -13,10 +13,13 @@ // limitations under the License. #include +#include +#include #include #include #include #include +#include namespace DB::tests { @@ -39,44 +42,57 @@ class KVStoreSpillTest : public KVStoreTestBase void setupStorage() { auto & ctx = TiFlashTestEnv::getGlobalContext(); - auto columns = DM::tests::DMTestEnv::getDefaultTableColumns(pk_type); - auto table_info = DM::tests::DMTestEnv::getMinimalTableInfo(/* table id */ 100, pk_type); - auto astptr = DM::tests::DMTestEnv::getPrimaryKeyExpr("test_table", pk_type); - - storage = StorageDeltaMerge::create( - "TiFlash", - "default" /* db_name */, - "test_table" /* table_name */, - table_info, - ColumnsDescription{columns}, - astptr, - 0, - ctx); - storage->startup(); + initStorages(); + KVStore & kvs = getKVS(); + table_id = proxy_instance->bootstrapTable(ctx, kvs, ctx.getTMTContext()); + auto maybe_storage = ctx.getTMTContext().getStorages().get(NullspaceID, table_id); + RUNTIME_CHECK(maybe_storage); + storage = std::dynamic_pointer_cast(maybe_storage); } protected: StorageDeltaMergePtr storage; - DM::tests::DMTestEnv::PkType pk_type = DM::tests::DMTestEnv::PkType::HiddenTiDBRowID; + TableID table_id; }; -TEST_F(KVStoreSpillTest, KVStoreSpill) +TEST_F(KVStoreSpillTest, CreateBlock) try { auto table_lock = storage->lockStructureForShare("foo_query_id"); { auto [schema_snapshot, block] = storage->getSchemaSnapshotAndBlockForDecoding(table_lock, true, true); UNUSED(schema_snapshot); - ASSERT_EQ(block->columns(), 3); + ASSERT_EQ(block->columns(), 4); } { auto [schema_snapshot, block] = storage->getSchemaSnapshotAndBlockForDecoding(table_lock, true, false); UNUSED(schema_snapshot); EXPECT_NO_THROW(block->getPositionByName(MutableSupport::delmark_column_name)); EXPECT_THROW(block->getPositionByName(MutableSupport::version_column_name), Exception); - ASSERT_EQ(block->columns(), 2); + ASSERT_EQ(block->columns(), 3); } } CATCH +TEST_F(KVStoreSpillTest, BlockReader) +try +{ + auto table_lock = storage->lockStructureForShare("foo_query_id"); + auto [decoding_schema_snapshot, block_ptr] = storage->getSchemaSnapshotAndBlockForDecoding(table_lock, true, false); + + DB::RegionUncommittedDataList data_list_read; + + auto str_key = RecordKVFormat::genKey(table_id, 1, 111); + auto [str_val_write, str_val_default] = proxy_instance->generateTiKVKeyValue(111, 999); + auto str_lock_value + = RecordKVFormat::encodeLockCfValue(RecordKVFormat::CFModifyFlag::PutFlag, "PK", 111, 999).toString(); + auto pk = RecordKVFormat::getRawTiDBPK(RecordKVFormat::decodeTiKVKey(str_key)); + auto value = std::make_shared(TiKVValue::copyFrom(str_val_default)); + data_list_read.data.push_back(RegionUncommittedData(std::move(pk), RecordKVFormat::CFModifyFlag::PutFlag, value)); + + auto reader = RegionBlockReader(decoding_schema_snapshot); + ASSERT_TRUE(reader.read(*block_ptr, data_list_read, true)); +} +CATCH + } // namespace DB::tests \ No newline at end of file diff --git a/dbms/src/TiDB/Decode/RowCodec.cpp b/dbms/src/TiDB/Decode/RowCodec.cpp index 3b15785b72d..723b5e60114 100644 --- a/dbms/src/TiDB/Decode/RowCodec.cpp +++ b/dbms/src/TiDB/Decode/RowCodec.cpp @@ -654,7 +654,7 @@ bool appendRowV1ToBlock( continue; } - auto * raw_column = const_cast((block.getByPosition(block_column_pos)).column.get()); + auto * raw_column = const_cast((block.safeGetByPosition(block_column_pos)).column.get()); const auto & column_info = column_infos[column_ids_iter->second]; DatumFlat datum(decoded_field_iter->second, column_info.tp); const Field & unflattened = datum.field(); From f56a23dc7d4c6614f26a2f433eda03663278f614 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Mon, 5 Feb 2024 14:23:44 +0800 Subject: [PATCH 07/37] make it work in a pathetic way Signed-off-by: CalvinNeo --- .../Decode/DecodingStorageSchemaSnapshot.cpp | 109 +++++++++++++++++- .../Decode/DecodingStorageSchemaSnapshot.h | 97 +++------------- .../KVStore/Decode/RegionBlockReader.cpp | 9 +- dbms/src/Storages/StorageDeltaMerge.cpp | 5 +- dbms/src/TiDB/Decode/RowCodec.cpp | 1 + dbms/src/TiDB/tests/RowCodecTestUtils.h | 6 +- 6 files changed, 133 insertions(+), 94 deletions(-) diff --git a/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.cpp b/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.cpp index 584f002613c..15b14241ff4 100644 --- a/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.cpp +++ b/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.cpp @@ -17,6 +17,107 @@ namespace DB { +DecodingStorageSchemaSnapshot::DecodingStorageSchemaSnapshot( + DM::ColumnDefinesPtr column_defines_, + const TiDB::TableInfo & table_info_, + const DM::ColumnDefine & original_handle_, + Int64 decoding_schema_epoch_, + bool has_version_column) + : column_defines{std::move(column_defines_)} + , pk_is_handle{table_info_.pk_is_handle} + , is_common_handle{table_info_.is_common_handle} + , decoding_schema_epoch{decoding_schema_epoch_} +{ + std::unordered_map column_lut; + // col id -> tidb pos, has no internal cols. + for (size_t i = 0; i < table_info_.columns.size(); i++) + { + const auto & ci = table_info_.columns[i]; + column_lut.emplace(ci.id, i); + } + LOG_INFO(DB::Logger::get(), "!!!!!! fs1 define {}", column_defines->size()); + // column_defines has internal cols. + size_t index_in_block = 0; + for (size_t i = 0; i < column_defines->size(); i++) + { + auto & cd = (*column_defines)[i]; + LOG_INFO(DB::Logger::get(), "!!!!!! ZX cd.id {} i {} lut {}", cd.id, i, column_lut.size()); + if(cd.id != VersionColumnID || has_version_column) { + LOG_INFO(DB::Logger::get(), "!!!!!! sorted_column_id_with_pos cd.id {} i {} lut {} index_in_block {}", cd.id, i, column_lut.size(), index_in_block); + sorted_column_id_with_pos.insert({cd.id, index_in_block++}); + } + sorted_column_id_with_pos_total.insert({cd.id, i}); + if(cd.id == VersionColumnID) + { + LOG_INFO(DB::Logger::get(), "!!!!!! VL cd.id {} i {} lut {}", cd.id, i, column_lut.size()); + if (has_version_column) { + column_infos.push_back(ColumnInfo()); + } else { + // Do nothing. + } + } + else if (cd.id != TiDBPkColumnID && cd.id != DelMarkColumnID) + { + LOG_INFO(DB::Logger::get(), "!!!!!! VK cd.id {} i {} lut {}", cd.id, i, column_lut.size()); + const auto & columns = table_info_.columns; + column_infos.push_back(columns[column_lut.at(cd.id)]); + } + else + { + LOG_INFO(DB::Logger::get(), "!!!!!! VM cd.id {} i {} lut {}", cd.id, i, column_lut.size()); + column_infos.push_back(ColumnInfo()); + } + } + + LOG_INFO(DB::Logger::get(), "!!!!!! fs2"); + // create pk related metadata if needed + if (is_common_handle) + { + const auto & primary_index_cols = table_info_.getPrimaryIndexInfo().idx_cols; + for (const auto & primary_index_col : primary_index_cols) + { + auto pk_column_id = table_info_.columns[primary_index_col.offset].id; + pk_column_ids.emplace_back(pk_column_id); + pk_pos_map.emplace(pk_column_id, reinterpret_cast(std::numeric_limits::max())); + } + pk_type = TMTPKType::STRING; + rowkey_column_size = pk_column_ids.size(); + } + else if (table_info_.pk_is_handle) + { + pk_column_ids.emplace_back(original_handle_.id); + pk_pos_map.emplace(original_handle_.id, reinterpret_cast(std::numeric_limits::max())); + pk_type = getTMTPKType(*original_handle_.type); + rowkey_column_size = 1; + } + else + { + pk_type = TMTPKType::INT64; + rowkey_column_size = 1; + } + + // calculate pk column pos in block + if (!pk_pos_map.empty()) + { + auto pk_pos_iter = pk_pos_map.begin(); + size_t column_pos_in_block = 0; + for (auto & column_id_with_pos : sorted_column_id_with_pos) + { + if (pk_pos_iter == pk_pos_map.end()) + break; + if (pk_pos_iter->first == column_id_with_pos.first) + { + pk_pos_iter->second = column_pos_in_block; + pk_pos_iter++; + } + column_pos_in_block++; + } + if (unlikely(pk_pos_iter != pk_pos_map.end())) + throw Exception("Cannot find all pk columns in block", ErrorCodes::LOGICAL_ERROR); + } +} + + TMTPKType getTMTPKType(const IDataType & rhs) { static const DataTypeInt64 & dataTypeInt64 = {}; // NOLINT @@ -34,10 +135,10 @@ TMTPKType getTMTPKType(const IDataType & rhs) Block createBlockSortByColumnID(DecodingStorageSchemaSnapshotConstPtr schema_snapshot, bool has_version_column) { - UNUSED(has_version_column); Block block; - for (auto iter = schema_snapshot->sorted_column_id_with_pos.begin(); - iter != schema_snapshot->sorted_column_id_with_pos.end(); + LOG_INFO(DB::Logger::get(), "!!!!! createBlockSortByColumnID KPI size {}", schema_snapshot->sorted_column_id_with_pos_total.size()); + for (auto iter = schema_snapshot->sorted_column_id_with_pos_total.begin(); + iter != schema_snapshot->sorted_column_id_with_pos_total.end(); iter++) { // col_id == cd.id @@ -46,7 +147,9 @@ Block createBlockSortByColumnID(DecodingStorageSchemaSnapshotConstPtr schema_sna // - (DelMarkColumnID, _INTERNAL_DELMARK, u8) // - (TiDBPkColumnID, _tidb_rowid, i64) auto col_id = iter->first; + LOG_INFO(DB::Logger::get(), "!!!!! createBlockSortByColumnID Z iter->second {}", iter->second); auto & cd = (*(schema_snapshot->column_defines))[iter->second]; + LOG_INFO(DB::Logger::get(), "!!!!! createBlockSortByColumnID cd.id {} cd.name {} iter->second {}", cd.id, cd.name, iter->second); if (!has_version_column && cd.id == VersionColumnID) continue; block.insert({cd.type->createColumn(), cd.type, cd.name, col_id}); diff --git a/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.h b/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.h index 5a68c68c028..dbdefac2bac 100644 --- a/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.h +++ b/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.h @@ -46,14 +46,27 @@ using ColumnInfo = TiDB::ColumnInfo; using ColumnInfos = std::vector; struct DecodingStorageSchemaSnapshot { + DecodingStorageSchemaSnapshot( + DM::ColumnDefinesPtr column_defines_, + const TiDB::TableInfo & table_info_, + const DM::ColumnDefine & original_handle_, + Int64 decoding_schema_epoch_, + bool has_version_column); + + DISALLOW_COPY(DecodingStorageSchemaSnapshot); + + DecodingStorageSchemaSnapshot(DecodingStorageSchemaSnapshot &&) = default; + // There is a one-to-one correspondence between elements in `column_defines` and elements in `column_infos` // Note that some columns(EXTRA_HANDLE_COLUMN, VERSION_COLUMN, TAG_COLUMN) may not be a real column in tidb schema, // so their corresponding elements in `column_infos` are just nullptr and won't be used when decoding. DM::ColumnDefinesPtr column_defines; ColumnInfos column_infos; - // column id -> column pos in column_defines/column_infos + // column id -> column pos in block in column_defines/column_infos SortedColumnIDWithPos sorted_column_id_with_pos; + // column id -> column pos in column_defines/column_infos + SortedColumnIDWithPos sorted_column_id_with_pos_total; // 1. when the table doesn't have a common handle, // 1) if `pk_is_handle` is false, `pk_column_ids` is empty @@ -70,88 +83,6 @@ struct DecodingStorageSchemaSnapshot TMTPKType pk_type = TMTPKType::UNSPECIFIED; // an internal increasing version for `DecodingStorageSchemaSnapshot`, has no relation with the table schema version Int64 decoding_schema_epoch; - - DecodingStorageSchemaSnapshot( - DM::ColumnDefinesPtr column_defines_, - const TiDB::TableInfo & table_info_, - const DM::ColumnDefine & original_handle_, - Int64 decoding_schema_epoch_) - : column_defines{std::move(column_defines_)} - , pk_is_handle{table_info_.pk_is_handle} - , is_common_handle{table_info_.is_common_handle} - , decoding_schema_epoch{decoding_schema_epoch_} - { - std::unordered_map column_lut; - for (size_t i = 0; i < table_info_.columns.size(); i++) - { - const auto & ci = table_info_.columns[i]; - column_lut.emplace(ci.id, i); - } - for (size_t i = 0; i < column_defines->size(); i++) - { - auto & cd = (*column_defines)[i]; - sorted_column_id_with_pos.insert({cd.id, i}); - if (cd.id != TiDBPkColumnID && cd.id != VersionColumnID && cd.id != DelMarkColumnID) - { - const auto & columns = table_info_.columns; - column_infos.push_back(columns[column_lut.at(cd.id)]); - } - else - { - column_infos.push_back(ColumnInfo()); - } - } - - // create pk related metadata if needed - if (is_common_handle) - { - const auto & primary_index_cols = table_info_.getPrimaryIndexInfo().idx_cols; - for (const auto & primary_index_col : primary_index_cols) - { - auto pk_column_id = table_info_.columns[primary_index_col.offset].id; - pk_column_ids.emplace_back(pk_column_id); - pk_pos_map.emplace(pk_column_id, reinterpret_cast(std::numeric_limits::max())); - } - pk_type = TMTPKType::STRING; - rowkey_column_size = pk_column_ids.size(); - } - else if (table_info_.pk_is_handle) - { - pk_column_ids.emplace_back(original_handle_.id); - pk_pos_map.emplace(original_handle_.id, reinterpret_cast(std::numeric_limits::max())); - pk_type = getTMTPKType(*original_handle_.type); - rowkey_column_size = 1; - } - else - { - pk_type = TMTPKType::INT64; - rowkey_column_size = 1; - } - - // calculate pk column pos in block - if (!pk_pos_map.empty()) - { - auto pk_pos_iter = pk_pos_map.begin(); - size_t column_pos_in_block = 0; - for (auto & column_id_with_pos : sorted_column_id_with_pos) - { - if (pk_pos_iter == pk_pos_map.end()) - break; - if (pk_pos_iter->first == column_id_with_pos.first) - { - pk_pos_iter->second = column_pos_in_block; - pk_pos_iter++; - } - column_pos_in_block++; - } - if (unlikely(pk_pos_iter != pk_pos_map.end())) - throw Exception("Cannot find all pk columns in block", ErrorCodes::LOGICAL_ERROR); - } - } - - DISALLOW_COPY(DecodingStorageSchemaSnapshot); - - DecodingStorageSchemaSnapshot(DecodingStorageSchemaSnapshot &&) = default; }; using DecodingStorageSchemaSnapshotPtr = std::shared_ptr; using DecodingStorageSchemaSnapshotConstPtr = std::shared_ptr; diff --git a/dbms/src/Storages/KVStore/Decode/RegionBlockReader.cpp b/dbms/src/Storages/KVStore/Decode/RegionBlockReader.cpp index a409591810e..accc3bf8a89 100644 --- a/dbms/src/Storages/KVStore/Decode/RegionBlockReader.cpp +++ b/dbms/src/Storages/KVStore/Decode/RegionBlockReader.cpp @@ -154,7 +154,7 @@ template<> struct VersionColResolver { VersionColResolver() {} bool needBuild() const { - return raw_version_col == nullptr; + return false; } void build(ColumnUInt64 * raw_version_col_) { raw_version_col = raw_version_col_; @@ -221,8 +221,7 @@ bool RegionBlockReader::readImpl(Block & block, const ReadList & data_list, bool column_ids_iter++; } // extra handle, del, version - constexpr size_t MustHaveCntInSchema = 3; - if (unlikely(next_column_pos != MustHaveCntInSchema)) + if (unlikely(next_column_pos != version_col_resolver.reservedCount())) throw Exception(ErrorCodes::LOGICAL_ERROR, "del, version column mismatch, actual_size={}", next_column_pos); ColumnUInt8::Container & delmark_data = raw_delmark_col->getData(); @@ -273,10 +272,10 @@ bool RegionBlockReader::readImpl(Block & block, const ReadList & data_list, bool { LOG_INFO(DB::Logger::get(), "!!!!!! Z next_column_pos {}", next_column_pos); for (auto it = column_ids_iter; it != read_column_ids.end(); it ++) { - LOG_INFO(DB::Logger::get(), "!!!!!! Z col_id {}", it->first); + LOG_INFO(DB::Logger::get(), "!!!!!! Z col_id {} -> {}", it->first, it->second); } for (const auto & q : block.getNames()) { - LOG_INFO(DB::Logger::get(), "!!!!!! Z gq {}", q); + LOG_INFO(DB::Logger::get(), "!!!!!! Z gq {} next_column_pos {}", q, next_column_pos); } // Parse column value from encoded value if (!appendRowToBlock( diff --git a/dbms/src/Storages/StorageDeltaMerge.cpp b/dbms/src/Storages/StorageDeltaMerge.cpp index b1cb73d8afb..b64380f4ba7 100644 --- a/dbms/src/Storages/StorageDeltaMerge.cpp +++ b/dbms/src/Storages/StorageDeltaMerge.cpp @@ -1306,6 +1306,7 @@ std::pair StorageDeltaMerg bool need_block, bool has_version_column) { + LOG_INFO(DB::Logger::get(), "!!!!! BB"); (void)table_structure_lock; std::lock_guard lock{decode_schema_mutex}; if (!decoding_schema_snapshot || decoding_schema_changed) @@ -1315,11 +1316,13 @@ std::pair StorageDeltaMerg store->getStoreColumns(), tidb_table_info, store->getHandle(), - decoding_schema_epoch++); + decoding_schema_epoch++, + has_version_column); cache_blocks.clear(); decoding_schema_changed = false; } + LOG_INFO(DB::Logger::get(), "!!!!! AA"); if (need_block) { if (cache_blocks.empty() || !has_version_column) diff --git a/dbms/src/TiDB/Decode/RowCodec.cpp b/dbms/src/TiDB/Decode/RowCodec.cpp index 723b5e60114..01179e8bbf4 100644 --- a/dbms/src/TiDB/Decode/RowCodec.cpp +++ b/dbms/src/TiDB/Decode/RowCodec.cpp @@ -654,6 +654,7 @@ bool appendRowV1ToBlock( continue; } + LOG_INFO(DB::Logger::get(), "!!!! block_column_pos {} column_ids_iter->second {}", block_column_pos, column_ids_iter->second); auto * raw_column = const_cast((block.safeGetByPosition(block_column_pos)).column.get()); const auto & column_info = column_infos[column_ids_iter->second]; DatumFlat datum(decoded_field_iter->second, column_info.tp); diff --git a/dbms/src/TiDB/tests/RowCodecTestUtils.h b/dbms/src/TiDB/tests/RowCodecTestUtils.h index 26f0275bea9..0963e3ded56 100644 --- a/dbms/src/TiDB/tests/RowCodecTestUtils.h +++ b/dbms/src/TiDB/tests/RowCodecTestUtils.h @@ -315,7 +315,8 @@ inline DecodingStorageSchemaSnapshotConstPtr getDecodingStorageSchemaSnapshot(co std::make_shared(store_columns), table_info, *iter, - /* decoding_schema_epoch_ */ 1); + /* decoding_schema_epoch_ */ 1, + true); } else { @@ -323,7 +324,8 @@ inline DecodingStorageSchemaSnapshotConstPtr getDecodingStorageSchemaSnapshot(co std::make_shared(store_columns), table_info, store_columns[0], - /* decoding_schema_epoch_ */ 1); + /* decoding_schema_epoch_ */ 1, + true); } } From 0c9e50bfab65fd2cb99ce67c8640fffa38022110 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Mon, 5 Feb 2024 14:27:40 +0800 Subject: [PATCH 08/37] fmt Signed-off-by: CalvinNeo --- .../Decode/DecodingStorageSchemaSnapshot.cpp | 22 ++---- .../KVStore/Decode/PartitionStreams.cpp | 8 +- .../KVStore/Decode/RegionBlockReader.cpp | 79 ++++++++----------- .../Spill/RegionUncommittedDataList.h | 28 ++----- .../Storages/KVStore/tests/gtest_spill.cpp | 4 +- dbms/src/Storages/StorageDeltaMerge.cpp | 2 - dbms/src/TiDB/Decode/RowCodec.cpp | 1 - 7 files changed, 56 insertions(+), 88 deletions(-) diff --git a/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.cpp b/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.cpp index 15b14241ff4..31e27a2bb74 100644 --- a/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.cpp +++ b/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.cpp @@ -35,41 +35,38 @@ DecodingStorageSchemaSnapshot::DecodingStorageSchemaSnapshot( const auto & ci = table_info_.columns[i]; column_lut.emplace(ci.id, i); } - LOG_INFO(DB::Logger::get(), "!!!!!! fs1 define {}", column_defines->size()); // column_defines has internal cols. size_t index_in_block = 0; for (size_t i = 0; i < column_defines->size(); i++) { auto & cd = (*column_defines)[i]; - LOG_INFO(DB::Logger::get(), "!!!!!! ZX cd.id {} i {} lut {}", cd.id, i, column_lut.size()); - if(cd.id != VersionColumnID || has_version_column) { - LOG_INFO(DB::Logger::get(), "!!!!!! sorted_column_id_with_pos cd.id {} i {} lut {} index_in_block {}", cd.id, i, column_lut.size(), index_in_block); + if (cd.id != VersionColumnID || has_version_column) + { sorted_column_id_with_pos.insert({cd.id, index_in_block++}); } sorted_column_id_with_pos_total.insert({cd.id, i}); - if(cd.id == VersionColumnID) + if (cd.id == VersionColumnID) { - LOG_INFO(DB::Logger::get(), "!!!!!! VL cd.id {} i {} lut {}", cd.id, i, column_lut.size()); - if (has_version_column) { + if (has_version_column) + { column_infos.push_back(ColumnInfo()); - } else { + } + else + { // Do nothing. } } else if (cd.id != TiDBPkColumnID && cd.id != DelMarkColumnID) { - LOG_INFO(DB::Logger::get(), "!!!!!! VK cd.id {} i {} lut {}", cd.id, i, column_lut.size()); const auto & columns = table_info_.columns; column_infos.push_back(columns[column_lut.at(cd.id)]); } else { - LOG_INFO(DB::Logger::get(), "!!!!!! VM cd.id {} i {} lut {}", cd.id, i, column_lut.size()); column_infos.push_back(ColumnInfo()); } } - LOG_INFO(DB::Logger::get(), "!!!!!! fs2"); // create pk related metadata if needed if (is_common_handle) { @@ -136,7 +133,6 @@ TMTPKType getTMTPKType(const IDataType & rhs) Block createBlockSortByColumnID(DecodingStorageSchemaSnapshotConstPtr schema_snapshot, bool has_version_column) { Block block; - LOG_INFO(DB::Logger::get(), "!!!!! createBlockSortByColumnID KPI size {}", schema_snapshot->sorted_column_id_with_pos_total.size()); for (auto iter = schema_snapshot->sorted_column_id_with_pos_total.begin(); iter != schema_snapshot->sorted_column_id_with_pos_total.end(); iter++) @@ -147,9 +143,7 @@ Block createBlockSortByColumnID(DecodingStorageSchemaSnapshotConstPtr schema_sna // - (DelMarkColumnID, _INTERNAL_DELMARK, u8) // - (TiDBPkColumnID, _tidb_rowid, i64) auto col_id = iter->first; - LOG_INFO(DB::Logger::get(), "!!!!! createBlockSortByColumnID Z iter->second {}", iter->second); auto & cd = (*(schema_snapshot->column_defines))[iter->second]; - LOG_INFO(DB::Logger::get(), "!!!!! createBlockSortByColumnID cd.id {} cd.name {} iter->second {}", cd.id, cd.name, iter->second); if (!has_version_column && cd.id == VersionColumnID) continue; block.insert({cd.type->createColumn(), cd.type, cd.name, col_id}); diff --git a/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp b/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp index dce7287e3e3..637d8533bf4 100644 --- a/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp +++ b/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp @@ -13,7 +13,6 @@ // limitations under the License. #include -#include #include #include #include @@ -26,6 +25,7 @@ #include #include #include +#include #include #include #include @@ -127,14 +127,16 @@ static inline bool atomicReadWrite( Int64 block_decoding_schema_epoch = -1; BlockUPtr block_ptr = nullptr; bool should_handle_version_col = true; - if constexpr (std::is_same_v) { + if constexpr (std::is_same_v) + { should_handle_version_col = false; } if (need_decode) { LOG_TRACE(log, "begin to decode keyspace={} table_id={} region_id={}", keyspace_id, table_id, region->id()); DecodingStorageSchemaSnapshotConstPtr decoding_schema_snapshot; - std::tie(decoding_schema_snapshot, block_ptr) = storage->getSchemaSnapshotAndBlockForDecoding(lock, true, should_handle_version_col); + std::tie(decoding_schema_snapshot, block_ptr) + = storage->getSchemaSnapshotAndBlockForDecoding(lock, true, should_handle_version_col); block_decoding_schema_epoch = decoding_schema_snapshot->decoding_schema_epoch; auto reader = RegionBlockReader(decoding_schema_snapshot); diff --git a/dbms/src/Storages/KVStore/Decode/RegionBlockReader.cpp b/dbms/src/Storages/KVStore/Decode/RegionBlockReader.cpp index accc3bf8a89..c9ebde3bfd0 100644 --- a/dbms/src/Storages/KVStore/Decode/RegionBlockReader.cpp +++ b/dbms/src/Storages/KVStore/Decode/RegionBlockReader.cpp @@ -116,64 +116,58 @@ template bool RegionBlockReader::read( const RegionUncommittedDataList & data_list, bool force_decode); -template -struct VersionColResolver { +template +struct VersionColResolver +{ VersionColResolver() {} - bool needBuild() const { - return raw_version_col == nullptr; - } - void build(ColumnUInt64 * raw_version_col_) { - raw_version_col = raw_version_col_; - } - void preRead(size_t size) { + bool needBuild() const { return raw_version_col == nullptr; } + void build(ColumnUInt64 * raw_version_col_) { raw_version_col = raw_version_col_; } + void preRead(size_t size) + { RUNTIME_CHECK(raw_version_col); ColumnUInt64::Container & version_data = raw_version_col->getData(); version_data.reserve(size); } - void read(const RegionDataReadInfo & info) { + void read(const RegionDataReadInfo & info) + { RUNTIME_CHECK(raw_version_col); ColumnUInt64::Container & version_data = raw_version_col->getData(); version_data.emplace_back(info.commit_ts); } - void check(const Block & block, size_t expected) const { + void check(const Block & block, size_t expected) const + { if (unlikely(block.columns() != expected)) - throw Exception(ErrorCodes::LOGICAL_ERROR, + throw Exception( + ErrorCodes::LOGICAL_ERROR, "Block structure doesn't match schema_snapshot, block={} def={}", block.columns(), - expected - ); - } - size_t reservedCount() const { - return 3; + expected); } + size_t reservedCount() const { return 3; } + private: ColumnUInt64 * raw_version_col = nullptr; }; -template<> -struct VersionColResolver { +template <> +struct VersionColResolver +{ VersionColResolver() {} - bool needBuild() const { - return false; - } - void build(ColumnUInt64 * raw_version_col_) { - raw_version_col = raw_version_col_; - } - void preRead(size_t) { - } - void read(const RegionUncommittedData &) { - } - void check(const Block & block, size_t expected) const { + bool needBuild() const { return false; } + void build(ColumnUInt64 * raw_version_col_) { raw_version_col = raw_version_col_; } + void preRead(size_t) {} + void read(const RegionUncommittedData &) {} + void check(const Block & block, size_t expected) const + { if (unlikely(block.columns() + 1 != expected)) - throw Exception(ErrorCodes::LOGICAL_ERROR, + throw Exception( + ErrorCodes::LOGICAL_ERROR, "Block structure doesn't match schema_snapshot, block={} def={}", block.columns(), - expected - ); - } - size_t reservedCount() const { - return 2; + expected); } + size_t reservedCount() const { return 2; } + private: ColumnUInt64 * raw_version_col = nullptr; }; @@ -202,7 +196,8 @@ bool RegionBlockReader::readImpl(Block & block, const ReadList & data_list, bool // we cannot figure out extra_handle's column type now, so we just remember it's pos here size_t extra_handle_column_pos = invalid_column_pos; - while (raw_delmark_col == nullptr || version_col_resolver.needBuild() || extra_handle_column_pos == invalid_column_pos) + while (raw_delmark_col == nullptr || version_col_resolver.needBuild() + || extra_handle_column_pos == invalid_column_pos) { if (column_ids_iter->first == DelMarkColumnID) { @@ -211,7 +206,8 @@ bool RegionBlockReader::readImpl(Block & block, const ReadList & data_list, bool } else if (column_ids_iter->first == VersionColumnID) { - version_col_resolver.build(static_cast(const_cast(block.getByPosition(next_column_pos).column.get()))); + version_col_resolver.build( + static_cast(const_cast(block.getByPosition(next_column_pos).column.get()))); } else if (column_ids_iter->first == TiDBPkColumnID) { @@ -270,13 +266,6 @@ bool RegionBlockReader::readImpl(Block & block, const ReadList & data_list, bool } else { - LOG_INFO(DB::Logger::get(), "!!!!!! Z next_column_pos {}", next_column_pos); - for (auto it = column_ids_iter; it != read_column_ids.end(); it ++) { - LOG_INFO(DB::Logger::get(), "!!!!!! Z col_id {} -> {}", it->first, it->second); - } - for (const auto & q : block.getNames()) { - LOG_INFO(DB::Logger::get(), "!!!!!! Z gq {} next_column_pos {}", q, next_column_pos); - } // Parse column value from encoded value if (!appendRowToBlock( *value_ptr, diff --git a/dbms/src/Storages/KVStore/MultiRaft/Spill/RegionUncommittedDataList.h b/dbms/src/Storages/KVStore/MultiRaft/Spill/RegionUncommittedDataList.h index 78b5fea9514..cd1fbc15e5d 100644 --- a/dbms/src/Storages/KVStore/MultiRaft/Spill/RegionUncommittedDataList.h +++ b/dbms/src/Storages/KVStore/MultiRaft/Spill/RegionUncommittedDataList.h @@ -52,33 +52,19 @@ struct RegionUncommittedDataList Inner data; Timestamp start_ts; - Inner::const_iterator cbegin() const { - return data.cbegin(); - } + Inner::const_iterator cbegin() const { return data.cbegin(); } - Inner::const_iterator cend() const { - return data.cend(); - } + Inner::const_iterator cend() const { return data.cend(); } - Inner::iterator begin() { - return data.begin(); - } + Inner::iterator begin() { return data.begin(); } - Inner::const_iterator begin() const { - return data.begin(); - } + Inner::const_iterator begin() const { return data.begin(); } - Inner::iterator end() { - return data.end(); - } + Inner::iterator end() { return data.end(); } - Inner::const_iterator end() const { - return data.end(); - } + Inner::const_iterator end() const { return data.end(); } - size_t size() const { - return data.size(); - } + size_t size() const { return data.size(); } }; } // namespace DB \ No newline at end of file diff --git a/dbms/src/Storages/KVStore/tests/gtest_spill.cpp b/dbms/src/Storages/KVStore/tests/gtest_spill.cpp index 5532fe8883c..068b278b294 100644 --- a/dbms/src/Storages/KVStore/tests/gtest_spill.cpp +++ b/dbms/src/Storages/KVStore/tests/gtest_spill.cpp @@ -13,13 +13,13 @@ // limitations under the License. #include -#include +#include #include +#include #include #include #include #include -#include namespace DB::tests { diff --git a/dbms/src/Storages/StorageDeltaMerge.cpp b/dbms/src/Storages/StorageDeltaMerge.cpp index b64380f4ba7..a9e02cbdf41 100644 --- a/dbms/src/Storages/StorageDeltaMerge.cpp +++ b/dbms/src/Storages/StorageDeltaMerge.cpp @@ -1306,7 +1306,6 @@ std::pair StorageDeltaMerg bool need_block, bool has_version_column) { - LOG_INFO(DB::Logger::get(), "!!!!! BB"); (void)table_structure_lock; std::lock_guard lock{decode_schema_mutex}; if (!decoding_schema_snapshot || decoding_schema_changed) @@ -1322,7 +1321,6 @@ std::pair StorageDeltaMerg decoding_schema_changed = false; } - LOG_INFO(DB::Logger::get(), "!!!!! AA"); if (need_block) { if (cache_blocks.empty() || !has_version_column) diff --git a/dbms/src/TiDB/Decode/RowCodec.cpp b/dbms/src/TiDB/Decode/RowCodec.cpp index 01179e8bbf4..723b5e60114 100644 --- a/dbms/src/TiDB/Decode/RowCodec.cpp +++ b/dbms/src/TiDB/Decode/RowCodec.cpp @@ -654,7 +654,6 @@ bool appendRowV1ToBlock( continue; } - LOG_INFO(DB::Logger::get(), "!!!! block_column_pos {} column_ids_iter->second {}", block_column_pos, column_ids_iter->second); auto * raw_column = const_cast((block.safeGetByPosition(block_column_pos)).column.get()); const auto & column_info = column_infos[column_ids_iter->second]; DatumFlat datum(decoded_field_iter->second, column_info.tp); From be71bd5fe81c4c51235e52fd1db594c916aec7d1 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Mon, 5 Feb 2024 15:01:52 +0800 Subject: [PATCH 09/37] make it kind of readable Signed-off-by: CalvinNeo --- .../Decode/DecodingStorageSchemaSnapshot.cpp | 13 ++++++++----- .../KVStore/Decode/DecodingStorageSchemaSnapshot.h | 14 +++++++++----- .../Storages/KVStore/Decode/RegionBlockReader.cpp | 8 +++++--- dbms/src/Storages/KVStore/tests/gtest_spill.cpp | 14 ++++++++++++++ dbms/src/TiDB/tests/RowCodecTestUtils.h | 6 +++--- 5 files changed, 39 insertions(+), 16 deletions(-) diff --git a/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.cpp b/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.cpp index 31e27a2bb74..461387eb5ad 100644 --- a/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.cpp +++ b/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.cpp @@ -42,9 +42,9 @@ DecodingStorageSchemaSnapshot::DecodingStorageSchemaSnapshot( auto & cd = (*column_defines)[i]; if (cd.id != VersionColumnID || has_version_column) { - sorted_column_id_with_pos.insert({cd.id, index_in_block++}); + col_id_to_block_pos.insert({cd.id, index_in_block++}); } - sorted_column_id_with_pos_total.insert({cd.id, i}); + col_id_to_def_pos.insert({cd.id, i}); if (cd.id == VersionColumnID) { if (has_version_column) @@ -98,7 +98,7 @@ DecodingStorageSchemaSnapshot::DecodingStorageSchemaSnapshot( { auto pk_pos_iter = pk_pos_map.begin(); size_t column_pos_in_block = 0; - for (auto & column_id_with_pos : sorted_column_id_with_pos) + for (auto & column_id_with_pos : col_id_to_block_pos) { if (pk_pos_iter == pk_pos_map.end()) break; @@ -133,8 +133,11 @@ TMTPKType getTMTPKType(const IDataType & rhs) Block createBlockSortByColumnID(DecodingStorageSchemaSnapshotConstPtr schema_snapshot, bool has_version_column) { Block block; - for (auto iter = schema_snapshot->sorted_column_id_with_pos_total.begin(); - iter != schema_snapshot->sorted_column_id_with_pos_total.end(); + // # Safety + // Though `col_id_to_block_pos` lasks some fields in `col_id_to_def_pos`, + // it is always a sub-sequence of `col_id_to_def_pos`. + for (auto iter = schema_snapshot->getColId2DefPosMap().begin(); + iter != schema_snapshot->getColId2DefPosMap().end(); iter++) { // col_id == cd.id diff --git a/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.h b/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.h index dbdefac2bac..23dec2c7110 100644 --- a/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.h +++ b/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.h @@ -57,17 +57,15 @@ struct DecodingStorageSchemaSnapshot DecodingStorageSchemaSnapshot(DecodingStorageSchemaSnapshot &&) = default; + const SortedColumnIDWithPos & getColId2BlockPosMap() const { return col_id_to_block_pos; } + const SortedColumnIDWithPos & getColId2DefPosMap() const { return col_id_to_def_pos; } + // There is a one-to-one correspondence between elements in `column_defines` and elements in `column_infos` // Note that some columns(EXTRA_HANDLE_COLUMN, VERSION_COLUMN, TAG_COLUMN) may not be a real column in tidb schema, // so their corresponding elements in `column_infos` are just nullptr and won't be used when decoding. DM::ColumnDefinesPtr column_defines; ColumnInfos column_infos; - // column id -> column pos in block in column_defines/column_infos - SortedColumnIDWithPos sorted_column_id_with_pos; - // column id -> column pos in column_defines/column_infos - SortedColumnIDWithPos sorted_column_id_with_pos_total; - // 1. when the table doesn't have a common handle, // 1) if `pk_is_handle` is false, `pk_column_ids` is empty // 2) if `pk_is_handle` is true, `pk_column_ids` contain a single element which is the column id of the pk column @@ -83,6 +81,12 @@ struct DecodingStorageSchemaSnapshot TMTPKType pk_type = TMTPKType::UNSPECIFIED; // an internal increasing version for `DecodingStorageSchemaSnapshot`, has no relation with the table schema version Int64 decoding_schema_epoch; +private: + // `col_id_to_def_pos` is originally `sorted_column_id_with_pos`. + // We may omit some cols in block, e.g. version col. So `col_id_to_def_pos` may have more items than `col_id_to_block_pos`. + // Both of the maps are sorted in ColumnID order, which makes the internal cols in first. + SortedColumnIDWithPos col_id_to_block_pos; + SortedColumnIDWithPos col_id_to_def_pos; }; using DecodingStorageSchemaSnapshotPtr = std::shared_ptr; using DecodingStorageSchemaSnapshotConstPtr = std::shared_ptr; diff --git a/dbms/src/Storages/KVStore/Decode/RegionBlockReader.cpp b/dbms/src/Storages/KVStore/Decode/RegionBlockReader.cpp index c9ebde3bfd0..b4528b8fcf2 100644 --- a/dbms/src/Storages/KVStore/Decode/RegionBlockReader.cpp +++ b/dbms/src/Storages/KVStore/Decode/RegionBlockReader.cpp @@ -87,12 +87,14 @@ bool RegionBlockReader::read(Block & block, const ReadList & data_list, bool for }; exc.addMessage(fmt::format( - "pk_type is {}, schema_snapshot->sorted_column_id_with_pos is {}, " + "pk_type is {}, schema_snapshot->col_id_to_block_pos is {}, " + "schema_snapshot->col_id_to_def_pos is {}," "schema_snapshot->column_defines is {}, " "decoding_snapshot_epoch is {}, " "block schema is {} ", magic_enum::enum_name(schema_snapshot->pk_type), - print_map(schema_snapshot->sorted_column_id_with_pos), + print_map(schema_snapshot->getColId2BlockPosMap()), + print_map(schema_snapshot->getColId2DefPosMap()), print_column_defines(schema_snapshot->column_defines), schema_snapshot->decoding_schema_epoch, block.dumpJsonStructure())); @@ -177,7 +179,7 @@ bool RegionBlockReader::readImpl(Block & block, const ReadList & data_list, bool { VersionColResolver version_col_resolver; version_col_resolver.check(block, schema_snapshot->column_defines->size()); - const auto & read_column_ids = schema_snapshot->sorted_column_id_with_pos; + const auto & read_column_ids = schema_snapshot->getColId2BlockPosMap(); const auto & pk_column_ids = schema_snapshot->pk_column_ids; const auto & pk_pos_map = schema_snapshot->pk_pos_map; diff --git a/dbms/src/Storages/KVStore/tests/gtest_spill.cpp b/dbms/src/Storages/KVStore/tests/gtest_spill.cpp index 068b278b294..486d6b7c726 100644 --- a/dbms/src/Storages/KVStore/tests/gtest_spill.cpp +++ b/dbms/src/Storages/KVStore/tests/gtest_spill.cpp @@ -71,6 +71,20 @@ try EXPECT_THROW(block->getPositionByName(MutableSupport::version_column_name), Exception); ASSERT_EQ(block->columns(), 3); } + { + // getColId2BlockPosMap should be a sub-sequence of getColId2DefPosMap. + auto [schema_snapshot, block] = storage->getSchemaSnapshotAndBlockForDecoding(table_lock, true, false); + UNUSED(block); + auto it2 = schema_snapshot->getColId2DefPosMap().begin(); + auto it2_end = schema_snapshot->getColId2DefPosMap().end(); + auto it = schema_snapshot->getColId2BlockPosMap().begin(); + auto it_end = schema_snapshot->getColId2BlockPosMap().end(); + for (; it != it_end; it++) { + while (it2->first != it->first && it2 != it2_end) it2++; + if (it2 == it2_end) break; + } + ASSERT_TRUE(it == it_end && it2 == it2_end); + } } CATCH diff --git a/dbms/src/TiDB/tests/RowCodecTestUtils.h b/dbms/src/TiDB/tests/RowCodecTestUtils.h index 0963e3ded56..fdf88a4d9ea 100644 --- a/dbms/src/TiDB/tests/RowCodecTestUtils.h +++ b/dbms/src/TiDB/tests/RowCodecTestUtils.h @@ -337,15 +337,15 @@ size_t valueStartPos(const TableInfo & table_info) inline Block decodeRowToBlock(const String & row_value, DecodingStorageSchemaSnapshotConstPtr decoding_schema) { - const auto & sorted_column_id_with_pos = decoding_schema->sorted_column_id_with_pos; - auto iter = sorted_column_id_with_pos.begin(); + const auto & col_id_to_block_pos = decoding_schema->getColId2BlockPosMap(); + auto iter = col_id_to_block_pos.begin(); const size_t value_column_num = 3; // skip first three column which is EXTRA_HANDLE_COLUMN, VERSION_COLUMN, TAG_COLUMN for (size_t i = 0; i < value_column_num; i++) iter++; Block block = createBlockSortByColumnID(decoding_schema); - appendRowToBlock(row_value, iter, sorted_column_id_with_pos.end(), block, value_column_num, decoding_schema, true); + appendRowToBlock(row_value, iter, col_id_to_block_pos.end(), block, value_column_num, decoding_schema, true); // remove first three column for (size_t i = 0; i < value_column_num; i++) From 82d7fe16b60cf1fd148fba25f4f97117c813ca78 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Mon, 5 Feb 2024 15:15:34 +0800 Subject: [PATCH 10/37] remote spill Signed-off-by: CalvinNeo --- .../Decode/DecodingStorageSchemaSnapshot.cpp | 3 +-- .../KVStore/Decode/DecodingStorageSchemaSnapshot.h | 1 + dbms/src/Storages/KVStore/MultiRaft/Spill/Spill.h | 14 -------------- .../Storages/KVStore/MultiRaft/Spill/SpillImpl.h | 1 - dbms/src/Storages/KVStore/Region.h | 6 ------ dbms/src/Storages/KVStore/tests/gtest_spill.cpp | 9 ++++++--- 6 files changed, 8 insertions(+), 26 deletions(-) diff --git a/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.cpp b/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.cpp index 461387eb5ad..aae68d820f1 100644 --- a/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.cpp +++ b/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.cpp @@ -136,8 +136,7 @@ Block createBlockSortByColumnID(DecodingStorageSchemaSnapshotConstPtr schema_sna // # Safety // Though `col_id_to_block_pos` lasks some fields in `col_id_to_def_pos`, // it is always a sub-sequence of `col_id_to_def_pos`. - for (auto iter = schema_snapshot->getColId2DefPosMap().begin(); - iter != schema_snapshot->getColId2DefPosMap().end(); + for (auto iter = schema_snapshot->getColId2DefPosMap().begin(); iter != schema_snapshot->getColId2DefPosMap().end(); iter++) { // col_id == cd.id diff --git a/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.h b/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.h index 23dec2c7110..b9603f5e751 100644 --- a/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.h +++ b/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.h @@ -81,6 +81,7 @@ struct DecodingStorageSchemaSnapshot TMTPKType pk_type = TMTPKType::UNSPECIFIED; // an internal increasing version for `DecodingStorageSchemaSnapshot`, has no relation with the table schema version Int64 decoding_schema_epoch; + private: // `col_id_to_def_pos` is originally `sorted_column_id_with_pos`. // We may omit some cols in block, e.g. version col. So `col_id_to_def_pos` may have more items than `col_id_to_block_pos`. diff --git a/dbms/src/Storages/KVStore/MultiRaft/Spill/Spill.h b/dbms/src/Storages/KVStore/MultiRaft/Spill/Spill.h index af193766765..fe9967c8e70 100644 --- a/dbms/src/Storages/KVStore/MultiRaft/Spill/Spill.h +++ b/dbms/src/Storages/KVStore/MultiRaft/Spill/Spill.h @@ -21,18 +21,4 @@ namespace DB { -struct SpillFileVersion -{ -}; - -struct SpillingTxn -{ - RegionDefaultCFData default_cf; -}; - -struct SpilledMemtable -{ - // start_ts -> SpillingTxn -}; - } // namespace DB diff --git a/dbms/src/Storages/KVStore/MultiRaft/Spill/SpillImpl.h b/dbms/src/Storages/KVStore/MultiRaft/Spill/SpillImpl.h index 9fd03838eb0..8fccec877d0 100644 --- a/dbms/src/Storages/KVStore/MultiRaft/Spill/SpillImpl.h +++ b/dbms/src/Storages/KVStore/MultiRaft/Spill/SpillImpl.h @@ -17,5 +17,4 @@ namespace DB { - } // namespace DB diff --git a/dbms/src/Storages/KVStore/Region.h b/dbms/src/Storages/KVStore/Region.h index 491c05807be..9ee4b4f5eb2 100644 --- a/dbms/src/Storages/KVStore/Region.h +++ b/dbms/src/Storages/KVStore/Region.h @@ -255,12 +255,6 @@ class Region : public std::enable_shared_from_this void beforePrehandleSnapshot(uint64_t region_id, std::optional deadline_index); void afterPrehandleSnapshot(int64_t ongoing); -public: // Spill - SpilledMemtable spillDefaultCf(const TiKVKey & start_ts, const RegionTaskLock &); - - // Don't require Region task lock - void ingestSpilledFile(const RegionRange & region_range, Timestamp start_ts, SpillFileVersion spill_file_version); - private: friend class RegionRaftCommandDelegate; friend class RegionMockTest; diff --git a/dbms/src/Storages/KVStore/tests/gtest_spill.cpp b/dbms/src/Storages/KVStore/tests/gtest_spill.cpp index 486d6b7c726..fbdf3eba660 100644 --- a/dbms/src/Storages/KVStore/tests/gtest_spill.cpp +++ b/dbms/src/Storages/KVStore/tests/gtest_spill.cpp @@ -79,9 +79,12 @@ try auto it2_end = schema_snapshot->getColId2DefPosMap().end(); auto it = schema_snapshot->getColId2BlockPosMap().begin(); auto it_end = schema_snapshot->getColId2BlockPosMap().end(); - for (; it != it_end; it++) { - while (it2->first != it->first && it2 != it2_end) it2++; - if (it2 == it2_end) break; + for (; it != it_end; it++) + { + while (it2->first != it->first && it2 != it2_end) + it2++; + if (it2 == it2_end) + break; } ASSERT_TRUE(it == it_end && it2 == it2_end); } From d8da170ab6ae20fab6df691f5cceda69ffed1ccf Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Mon, 5 Feb 2024 16:30:15 +0800 Subject: [PATCH 11/37] tidy Signed-off-by: CalvinNeo --- .../Storages/KVStore/Decode/RegionBlockReader.cpp | 12 ++++++------ dbms/src/Storages/KVStore/tests/gtest_spill.cpp | 4 ++++ 2 files changed, 10 insertions(+), 6 deletions(-) diff --git a/dbms/src/Storages/KVStore/Decode/RegionBlockReader.cpp b/dbms/src/Storages/KVStore/Decode/RegionBlockReader.cpp index b4528b8fcf2..803dea156d0 100644 --- a/dbms/src/Storages/KVStore/Decode/RegionBlockReader.cpp +++ b/dbms/src/Storages/KVStore/Decode/RegionBlockReader.cpp @@ -121,7 +121,7 @@ template bool RegionBlockReader::read( template struct VersionColResolver { - VersionColResolver() {} + VersionColResolver() = default; bool needBuild() const { return raw_version_col == nullptr; } void build(ColumnUInt64 * raw_version_col_) { raw_version_col = raw_version_col_; } void preRead(size_t size) @@ -154,11 +154,11 @@ struct VersionColResolver template <> struct VersionColResolver { - VersionColResolver() {} - bool needBuild() const { return false; } + VersionColResolver() = default; + bool needBuild() const { return false; } // NOLINT conform to main template void build(ColumnUInt64 * raw_version_col_) { raw_version_col = raw_version_col_; } - void preRead(size_t) {} - void read(const RegionUncommittedData &) {} + void preRead(size_t) {} // NOLINT conform to main template + void read(const RegionUncommittedData &) {} // NOLINT conform to main template void check(const Block & block, size_t expected) const { if (unlikely(block.columns() + 1 != expected)) @@ -168,7 +168,7 @@ struct VersionColResolver block.columns(), expected); } - size_t reservedCount() const { return 2; } + size_t reservedCount() const { return 2; } // NOLINT conform to main template private: ColumnUInt64 * raw_version_col = nullptr; diff --git a/dbms/src/Storages/KVStore/tests/gtest_spill.cpp b/dbms/src/Storages/KVStore/tests/gtest_spill.cpp index fbdf3eba660..a9aef5d6445 100644 --- a/dbms/src/Storages/KVStore/tests/gtest_spill.cpp +++ b/dbms/src/Storages/KVStore/tests/gtest_spill.cpp @@ -109,6 +109,10 @@ try auto reader = RegionBlockReader(decoding_schema_snapshot); ASSERT_TRUE(reader.read(*block_ptr, data_list_read, true)); + auto block_pos = decoding_schema_snapshot->getColId2BlockPosMap().find(1)->second; + const auto & col_data = block_ptr->safeGetByPosition(block_pos); + ASSERT_EQ(col_data.name, "a"); + ASSERT_EQ(col_data.column->getName(), "Int64"); } CATCH From f59533841c9d65cb2aaef954bed8862f9a83b33e Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Mon, 5 Feb 2024 16:53:05 +0800 Subject: [PATCH 12/37] tidy2 Signed-off-by: CalvinNeo --- dbms/src/Storages/KVStore/Decode/RegionBlockReader.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/KVStore/Decode/RegionBlockReader.cpp b/dbms/src/Storages/KVStore/Decode/RegionBlockReader.cpp index 803dea156d0..d4d5c2515d0 100644 --- a/dbms/src/Storages/KVStore/Decode/RegionBlockReader.cpp +++ b/dbms/src/Storages/KVStore/Decode/RegionBlockReader.cpp @@ -159,7 +159,7 @@ struct VersionColResolver void build(ColumnUInt64 * raw_version_col_) { raw_version_col = raw_version_col_; } void preRead(size_t) {} // NOLINT conform to main template void read(const RegionUncommittedData &) {} // NOLINT conform to main template - void check(const Block & block, size_t expected) const + void check(const Block & block, size_t expected) const // NOLINT conform to main template { if (unlikely(block.columns() + 1 != expected)) throw Exception( From b6d54c584527df8f1b337b3ed90bccbc1bf1f275 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Mon, 5 Feb 2024 17:54:56 +0800 Subject: [PATCH 13/37] introduce AtomicReadWriteCtx to disable decoding uncommitted data Signed-off-by: CalvinNeo --- .../KVStore/Decode/PartitionStreams.cpp | 160 ++++++++++++------ .../KVStore/Decode/PartitionStreams.h | 9 + .../Storages/KVStore/tests/gtest_spill.cpp | 48 ++++-- 3 files changed, 154 insertions(+), 63 deletions(-) diff --git a/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp b/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp index 637d8533bf4..8189949e999 100644 --- a/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp +++ b/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp @@ -54,22 +54,78 @@ extern const int ILLFORMAT_RAFT_ROW; extern const int TABLE_IS_DROPPED; } // namespace ErrorCodes +struct AtomicReadWriteCtx +{ + AtomicReadWriteCtx(const Context & context_, const TMTContext & tmt_, KeyspaceID keyspace_id_, TableID table_id_) + : context(context_) + , tmt(tmt_) + , keyspace_id(keyspace_id_) + , table_id(table_id_) + {} + const Context & context; + const TMTContext & tmt; + KeyspaceID keyspace_id; + TableID table_id; + DM::WriteResult write_result = std::nullopt; + UInt64 region_decode_cost = -1; + UInt64 write_part_cost = -1; +}; + +static void writeCommittedBlockDataIntoStorage( + AtomicReadWriteCtx & rw_ctx, + TableStructureLockHolder & lock, + ManageableStoragePtr & storage, + bool need_decode, + Int64 block_decoding_schema_epoch, + BlockUPtr & block_ptr, + Block & block) +{ + /// Write block into storage. + // Release the alter lock so that writing does not block DDL operations + TableLockHolder drop_lock; + std::tie(std::ignore, drop_lock) = std::move(lock).release(); + Stopwatch watch; + // Note: do NOT use typeid_cast, since Storage is multi-inherited and typeid_cast will return nullptr + switch (storage->engineType()) + { + case ::TiDB::StorageEngine::DT: + { + auto dm_storage = std::dynamic_pointer_cast(storage); + if (need_decode) + { + rw_ctx.write_result = dm_storage->write(*block_ptr, rw_ctx.context.getSettingsRef()); + } + else + { + rw_ctx.write_result = dm_storage->write(block, rw_ctx.context.getSettingsRef()); + } + break; + } + default: + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Unknown StorageEngine: {}", + static_cast(storage->engineType())); + } + + rw_ctx.write_part_cost = watch.elapsedMilliseconds(); + GET_METRIC(tiflash_raft_write_data_to_storage_duration_seconds, type_write) + .Observe(rw_ctx.write_part_cost / 1000.0); + if (need_decode) + storage->releaseDecodingBlock(block_decoding_schema_epoch, std::move(block_ptr)); +} + // TODO Fix too many arguments template static inline bool atomicReadWrite( const LoggerPtr & log, - const Context & context, - const TMTContext & tmt, - KeyspaceID keyspace_id, - TableID table_id, + AtomicReadWriteCtx & rw_ctx, const RegionPtrWithBlock & region, ReadList & data_list_read, - bool force_decode, - DM::WriteResult & write_result) + bool force_decode) { - UInt64 region_decode_cost = -1, write_part_cost = -1; /// Get storage based on table ID. - auto storage = tmt.getStorages().get(keyspace_id, table_id); + auto storage = rw_ctx.tmt.getStorages().get(rw_ctx.keyspace_id, rw_ctx.table_id); if (storage == nullptr) { // - force_decode == false and storage not exist, let upper level sync schema and retry. @@ -94,6 +150,7 @@ static inline bool atomicReadWrite( throw; } + // TODO get rid of pre_decode logic with rw_ctx. Block block; bool need_decode = true; @@ -133,7 +190,12 @@ static inline bool atomicReadWrite( } if (need_decode) { - LOG_TRACE(log, "begin to decode keyspace={} table_id={} region_id={}", keyspace_id, table_id, region->id()); + LOG_TRACE( + log, + "begin to decode keyspace={} table_id={} region_id={}", + rw_ctx.keyspace_id, + rw_ctx.table_id, + region->id()); DecodingStorageSchemaSnapshotConstPtr decoding_schema_snapshot; std::tie(decoding_schema_snapshot, block_ptr) = storage->getSchemaSnapshotAndBlockForDecoding(lock, true, should_handle_version_col); @@ -142,58 +204,52 @@ static inline bool atomicReadWrite( auto reader = RegionBlockReader(decoding_schema_snapshot); if (!reader.read(*block_ptr, data_list_read, force_decode)) return false; - region_decode_cost = watch.elapsedMilliseconds(); + rw_ctx.region_decode_cost = watch.elapsedMilliseconds(); GET_METRIC(tiflash_raft_write_data_to_storage_duration_seconds, type_decode) - .Observe(region_decode_cost / 1000.0); + .Observe(rw_ctx.region_decode_cost / 1000.0); } - - /// Write block into storage. - // Release the alter lock so that writing does not block DDL operations - TableLockHolder drop_lock; - std::tie(std::ignore, drop_lock) = std::move(lock).release(); - watch.restart(); - // Note: do NOT use typeid_cast, since Storage is multi-inherited and typeid_cast will return nullptr - switch (storage->engineType()) - { - case ::TiDB::StorageEngine::DT: + if constexpr (std::is_same_v) { - auto dm_storage = std::dynamic_pointer_cast(storage); - if (need_decode) - { - write_result = dm_storage->write(*block_ptr, context.getSettingsRef()); - } - else - { - write_result = dm_storage->write(block, context.getSettingsRef()); - } - break; + writeCommittedBlockDataIntoStorage( + rw_ctx, + lock, + storage, + need_decode, + block_decoding_schema_epoch, + block_ptr, + block); } - default: - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Unknown StorageEngine: {}", - static_cast(storage->engineType())); + else + { + // TODO Implement spill logic. + RUNTIME_CHECK(false); } - - write_part_cost = watch.elapsedMilliseconds(); - GET_METRIC(tiflash_raft_write_data_to_storage_duration_seconds, type_write).Observe(write_part_cost / 1000.0); - if (need_decode) - storage->releaseDecodingBlock(block_decoding_schema_epoch, std::move(block_ptr)); - LOG_TRACE( log, "keyspace={} table_id={} region_id={} cost [region decode {}, write part {}] ms", - keyspace_id, - table_id, + rw_ctx.keyspace_id, + rw_ctx.table_id, region->id(), - region_decode_cost, - write_part_cost); + rw_ctx.region_decode_cost, + rw_ctx.write_part_cost); return true; } +template DM::WriteResult writeRegionDataToStorage( + Context & context, + const RegionPtrWithBlock & region, + RegionUncommittedDataList & data_list_read, + const LoggerPtr & log); +template DM::WriteResult writeRegionDataToStorage( + Context & context, + const RegionPtrWithBlock & region, + RegionDataReadInfoList & data_list_read, + const LoggerPtr & log); + +// TODO rename it after we support spill. // ReadList could be RegionDataReadInfoList template -static DM::WriteResult writeRegionDataToStorage( +DM::WriteResult writeRegionDataToStorage( Context & context, const RegionPtrWithBlock & region, ReadList & data_list_read, @@ -203,7 +259,7 @@ static DM::WriteResult writeRegionDataToStorage( const auto keyspace_id = region->getKeyspaceID(); const auto table_id = region->getMappedTableID(); - DM::WriteResult write_result = std::nullopt; + AtomicReadWriteCtx rw_ctx(context, tmt, keyspace_id, table_id); /// In TiFlash, the actions between applying raft log and schema changes are not strictly synchronized. /// There could be a chance that some raft logs come after a table gets tombstoned. Take care of it when @@ -220,9 +276,9 @@ static DM::WriteResult writeRegionDataToStorage( /// Try read then write once. { - if (atomicReadWrite(log, context, tmt, keyspace_id, table_id, region, data_list_read, false, write_result)) + if (atomicReadWrite(log, rw_ctx, region, data_list_read, false)) { - return write_result; + return std::move(rw_ctx.write_result); } } @@ -233,7 +289,7 @@ static DM::WriteResult writeRegionDataToStorage( tmt.getSchemaSyncerManager()->syncTableSchema(context, keyspace_id, table_id); auto schema_sync_cost = watch.elapsedMilliseconds(); LOG_INFO(log, "sync schema cost {} ms, keyspace={} table_id={}", schema_sync_cost, keyspace_id, table_id); - if (!atomicReadWrite(log, context, tmt, keyspace_id, table_id, region, data_list_read, true, write_result)) + if (!atomicReadWrite(log, rw_ctx, region, data_list_read, true)) { // Failure won't be tolerated this time. throw Exception( @@ -243,7 +299,7 @@ static DM::WriteResult writeRegionDataToStorage( keyspace_id, table_id); } - return write_result; + return std::move(rw_ctx.write_result); } } diff --git a/dbms/src/Storages/KVStore/Decode/PartitionStreams.h b/dbms/src/Storages/KVStore/Decode/PartitionStreams.h index 934ab282198..ee3c28a0a2b 100644 --- a/dbms/src/Storages/KVStore/Decode/PartitionStreams.h +++ b/dbms/src/Storages/KVStore/Decode/PartitionStreams.h @@ -16,6 +16,7 @@ #include #include +#include #include #include #include @@ -27,6 +28,7 @@ class Region; using RegionPtr = std::shared_ptr; class StorageDeltaMerge; class TMTContext; +struct RegionPtrWithBlock; std::optional ReadRegionCommitCache(const RegionPtr & region, bool lock_region); void RemoveRegionCommitCache( @@ -44,4 +46,11 @@ Block GenRegionBlockDataWithSchema( bool force_decode, TMTContext & tmt); +template +DM::WriteResult writeRegionDataToStorage( + Context & context, + const RegionPtrWithBlock & region, + ReadList & data_list_read, + const LoggerPtr & log); + } // namespace DB diff --git a/dbms/src/Storages/KVStore/tests/gtest_spill.cpp b/dbms/src/Storages/KVStore/tests/gtest_spill.cpp index a9aef5d6445..c527ca4eb71 100644 --- a/dbms/src/Storages/KVStore/tests/gtest_spill.cpp +++ b/dbms/src/Storages/KVStore/tests/gtest_spill.cpp @@ -91,12 +91,8 @@ try } CATCH -TEST_F(KVStoreSpillTest, BlockReader) -try +static DB::RegionUncommittedDataList buildUncommitReadList(TableID table_id, MockRaftStoreProxy * proxy_instance) { - auto table_lock = storage->lockStructureForShare("foo_query_id"); - auto [decoding_schema_snapshot, block_ptr] = storage->getSchemaSnapshotAndBlockForDecoding(table_lock, true, false); - DB::RegionUncommittedDataList data_list_read; auto str_key = RecordKVFormat::genKey(table_id, 1, 111); @@ -107,12 +103,42 @@ try auto value = std::make_shared(TiKVValue::copyFrom(str_val_default)); data_list_read.data.push_back(RegionUncommittedData(std::move(pk), RecordKVFormat::CFModifyFlag::PutFlag, value)); - auto reader = RegionBlockReader(decoding_schema_snapshot); - ASSERT_TRUE(reader.read(*block_ptr, data_list_read, true)); - auto block_pos = decoding_schema_snapshot->getColId2BlockPosMap().find(1)->second; - const auto & col_data = block_ptr->safeGetByPosition(block_pos); - ASSERT_EQ(col_data.name, "a"); - ASSERT_EQ(col_data.column->getName(), "Int64"); + return data_list_read; +} + +TEST_F(KVStoreSpillTest, BlockReader) +try +{ + auto & ctx = TiFlashTestEnv::getGlobalContext(); + { + // Test if a block with correct cols can be constructed. + auto table_lock = storage->lockStructureForShare("foo_query_id"); + auto [decoding_schema_snapshot, block_ptr] + = storage->getSchemaSnapshotAndBlockForDecoding(table_lock, true, false); + + DB::RegionUncommittedDataList data_list_read = buildUncommitReadList(table_id, proxy_instance.get()); + + auto reader = RegionBlockReader(decoding_schema_snapshot); + ASSERT_TRUE(reader.read(*block_ptr, data_list_read, true)); + auto block_pos = decoding_schema_snapshot->getColId2BlockPosMap().find(1)->second; + const auto & col_data = block_ptr->safeGetByPosition(block_pos); + ASSERT_EQ(col_data.name, "a"); + ASSERT_EQ(col_data.column->getName(), "Int64"); + } + { + auto table_lock = storage->lockStructureForShare("foo_query_id"); + auto [decoding_schema_snapshot, block_ptr] + = storage->getSchemaSnapshotAndBlockForDecoding(table_lock, true, false); + + DB::RegionUncommittedDataList data_list_read = buildUncommitReadList(table_id, proxy_instance.get()); + + KVStore & kvs = getKVS(); + proxy_instance->bootstrapWithRegion(kvs, ctx.getTMTContext(), 1, std::nullopt); + auto region = kvs.getRegion(1); + RegionPtrWithBlock region_with_cache = RegionPtrWithBlock(region, nullptr); + // TODO spill logic + EXPECT_THROW(writeRegionDataToStorage(ctx, region_with_cache, data_list_read, log), Exception); + } } CATCH From b78482d6cbc474b1c90a63714580257351157c8b Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Mon, 5 Feb 2024 18:13:22 +0800 Subject: [PATCH 14/37] remove useless codes Signed-off-by: CalvinNeo --- .../Decode/DecodingStorageSchemaSnapshot.cpp | 14 ++------------ 1 file changed, 2 insertions(+), 12 deletions(-) diff --git a/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.cpp b/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.cpp index aae68d820f1..bc3152f95e5 100644 --- a/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.cpp +++ b/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.cpp @@ -45,18 +45,8 @@ DecodingStorageSchemaSnapshot::DecodingStorageSchemaSnapshot( col_id_to_block_pos.insert({cd.id, index_in_block++}); } col_id_to_def_pos.insert({cd.id, i}); - if (cd.id == VersionColumnID) - { - if (has_version_column) - { - column_infos.push_back(ColumnInfo()); - } - else - { - // Do nothing. - } - } - else if (cd.id != TiDBPkColumnID && cd.id != DelMarkColumnID) + + if (cd.id != TiDBPkColumnID && cd.id != VersionColumnID && cd.id != DelMarkColumnID) { const auto & columns = table_info_.columns; column_infos.push_back(columns[column_lut.at(cd.id)]); From 1c06f7068633e3d91a62d861ae37cab99fc44588 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Mon, 5 Feb 2024 19:37:38 +0800 Subject: [PATCH 15/37] fix decode Signed-off-by: CalvinNeo --- .../KVStore/Decode/PartitionStreams.cpp | 107 ++++++++++-------- .../Storages/KVStore/tests/gtest_spill.cpp | 6 +- 2 files changed, 59 insertions(+), 54 deletions(-) diff --git a/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp b/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp index 8189949e999..7fd2f36a469 100644 --- a/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp +++ b/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp @@ -56,12 +56,47 @@ extern const int TABLE_IS_DROPPED; struct AtomicReadWriteCtx { - AtomicReadWriteCtx(const Context & context_, const TMTContext & tmt_, KeyspaceID keyspace_id_, TableID table_id_) - : context(context_) + AtomicReadWriteCtx( + const LoggerPtr & log_, + const Context & context_, + const TMTContext & tmt_, + KeyspaceID keyspace_id_, + TableID table_id_) + : log(log_) + , context(context_) , tmt(tmt_) , keyspace_id(keyspace_id_) , table_id(table_id_) {} + + std::optional tryUseDecodeCache(const RegionPtrWithBlock & region, ManageableStoragePtr & storage) + { + if unlikely (region.pre_decode_cache) + { + auto schema_version = storage->getTableInfo().schema_version; + std::stringstream ss; + region.pre_decode_cache->toString(ss); + LOG_DEBUG( + log, + "{} got pre-decode cache {}, storage schema version: {}", + region->toString(), + ss.str(), + schema_version); + + if (region.pre_decode_cache->schema_version == schema_version) + { + return std::move(region.pre_decode_cache->block); + } + else + { + LOG_DEBUG(log, "schema version not equal, try to re-decode region cache into block"); + region.pre_decode_cache->block.clear(); + } + } + return std::nullopt; + } + + const LoggerPtr & log; const Context & context; const TMTContext & tmt; KeyspaceID keyspace_id; @@ -75,9 +110,6 @@ static void writeCommittedBlockDataIntoStorage( AtomicReadWriteCtx & rw_ctx, TableStructureLockHolder & lock, ManageableStoragePtr & storage, - bool need_decode, - Int64 block_decoding_schema_epoch, - BlockUPtr & block_ptr, Block & block) { /// Write block into storage. @@ -91,14 +123,7 @@ static void writeCommittedBlockDataIntoStorage( case ::TiDB::StorageEngine::DT: { auto dm_storage = std::dynamic_pointer_cast(storage); - if (need_decode) - { - rw_ctx.write_result = dm_storage->write(*block_ptr, rw_ctx.context.getSettingsRef()); - } - else - { - rw_ctx.write_result = dm_storage->write(block, rw_ctx.context.getSettingsRef()); - } + rw_ctx.write_result = dm_storage->write(block, rw_ctx.context.getSettingsRef()); break; } default: @@ -111,11 +136,8 @@ static void writeCommittedBlockDataIntoStorage( rw_ctx.write_part_cost = watch.elapsedMilliseconds(); GET_METRIC(tiflash_raft_write_data_to_storage_duration_seconds, type_write) .Observe(rw_ctx.write_part_cost / 1000.0); - if (need_decode) - storage->releaseDecodingBlock(block_decoding_schema_epoch, std::move(block_ptr)); } -// TODO Fix too many arguments template static inline bool atomicReadWrite( const LoggerPtr & log, @@ -132,6 +154,7 @@ static inline bool atomicReadWrite( // - force_decode == true and storage not exist. It could be the RaftLog or Snapshot comes // after the schema is totally exceed the GC safepoint. And TiFlash know nothing about // the schema. We can only throw away those committed rows. + // In both cases, no exception will be thrown. return force_decode; } @@ -150,33 +173,15 @@ static inline bool atomicReadWrite( throw; } - // TODO get rid of pre_decode logic with rw_ctx. + // TODO get rid of pre_decode logic. Block block; bool need_decode = true; - - // try to use block cache if exists - if (region.pre_decode_cache) + // Try to use block cache if exists + auto maybe_block = rw_ctx.tryUseDecodeCache(region, storage); + if unlikely (maybe_block.has_value()) { - auto schema_version = storage->getTableInfo().schema_version; - std::stringstream ss; - region.pre_decode_cache->toString(ss); - LOG_DEBUG( - log, - "{} got pre-decode cache {}, storage schema version: {}", - region->toString(), - ss.str(), - schema_version); - - if (region.pre_decode_cache->schema_version == schema_version) - { - block = std::move(region.pre_decode_cache->block); - need_decode = false; - } - else - { - LOG_DEBUG(log, "schema version not equal, try to re-decode region cache into block"); - region.pre_decode_cache->block.clear(); - } + block = std::move(maybe_block.value()); + need_decode = false; } /// Read region data as block. @@ -188,7 +193,7 @@ static inline bool atomicReadWrite( { should_handle_version_col = false; } - if (need_decode) + if likely (need_decode) { LOG_TRACE( log, @@ -210,14 +215,16 @@ static inline bool atomicReadWrite( } if constexpr (std::is_same_v) { - writeCommittedBlockDataIntoStorage( - rw_ctx, - lock, - storage, - need_decode, - block_decoding_schema_epoch, - block_ptr, - block); + if likely (need_decode) + { + RUNTIME_CHECK(block_ptr != nullptr); + writeCommittedBlockDataIntoStorage(rw_ctx, lock, storage, *block_ptr); + storage->releaseDecodingBlock(block_decoding_schema_epoch, std::move(block_ptr)); + } + else + { + writeCommittedBlockDataIntoStorage(rw_ctx, lock, storage, block); + } } else { @@ -259,7 +266,7 @@ DM::WriteResult writeRegionDataToStorage( const auto keyspace_id = region->getKeyspaceID(); const auto table_id = region->getMappedTableID(); - AtomicReadWriteCtx rw_ctx(context, tmt, keyspace_id, table_id); + AtomicReadWriteCtx rw_ctx(log, context, tmt, keyspace_id, table_id); /// In TiFlash, the actions between applying raft log and schema changes are not strictly synchronized. /// There could be a chance that some raft logs come after a table gets tombstoned. Take care of it when diff --git a/dbms/src/Storages/KVStore/tests/gtest_spill.cpp b/dbms/src/Storages/KVStore/tests/gtest_spill.cpp index c527ca4eb71..a29144982d1 100644 --- a/dbms/src/Storages/KVStore/tests/gtest_spill.cpp +++ b/dbms/src/Storages/KVStore/tests/gtest_spill.cpp @@ -79,12 +79,10 @@ try auto it2_end = schema_snapshot->getColId2DefPosMap().end(); auto it = schema_snapshot->getColId2BlockPosMap().begin(); auto it_end = schema_snapshot->getColId2BlockPosMap().end(); - for (; it != it_end; it++) + for (; it != it_end && it2 != it2_end; it++) { - while (it2->first != it->first && it2 != it2_end) + if (it2->first == it->first) it2++; - if (it2 == it2_end) - break; } ASSERT_TRUE(it == it_end && it2 == it2_end); } From 99feff87f33a5b3e571a87bfa9e999a73df1246f Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Mon, 5 Feb 2024 21:51:26 +0800 Subject: [PATCH 16/37] fix fap test Signed-off-by: CalvinNeo --- dbms/src/Storages/KVStore/tests/gtest_kvstore_fast_add_peer.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/src/Storages/KVStore/tests/gtest_kvstore_fast_add_peer.cpp b/dbms/src/Storages/KVStore/tests/gtest_kvstore_fast_add_peer.cpp index 75fb4131554..85662e1af0f 100644 --- a/dbms/src/Storages/KVStore/tests/gtest_kvstore_fast_add_peer.cpp +++ b/dbms/src/Storages/KVStore/tests/gtest_kvstore_fast_add_peer.cpp @@ -63,6 +63,7 @@ class RegionKVStoreTestFAP : public KVStoreTestBase public: void SetUp() override { + test_path = TiFlashTestEnv::getTemporaryPath("/region_kvs_fap_test"); auto & global_context = TiFlashTestEnv::getGlobalContext(); // clean data and create path pool instance path_pool = TiFlashTestEnv::createCleanPathPool(test_path); From f87956ad463098b987d2f198c7b66e8ba81404b0 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Tue, 6 Feb 2024 11:13:36 +0800 Subject: [PATCH 17/37] log Signed-off-by: CalvinNeo --- dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp b/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp index 7fd2f36a469..d3cf3a6c9c8 100644 --- a/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp +++ b/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp @@ -106,7 +106,7 @@ struct AtomicReadWriteCtx UInt64 write_part_cost = -1; }; -static void writeCommittedBlockDataIntoStorage( +static void inline writeCommittedBlockDataIntoStorage( AtomicReadWriteCtx & rw_ctx, TableStructureLockHolder & lock, ManageableStoragePtr & storage, @@ -140,7 +140,6 @@ static void writeCommittedBlockDataIntoStorage( template static inline bool atomicReadWrite( - const LoggerPtr & log, AtomicReadWriteCtx & rw_ctx, const RegionPtrWithBlock & region, ReadList & data_list_read, @@ -196,7 +195,7 @@ static inline bool atomicReadWrite( if likely (need_decode) { LOG_TRACE( - log, + rw_ctx.log, "begin to decode keyspace={} table_id={} region_id={}", rw_ctx.keyspace_id, rw_ctx.table_id, @@ -232,7 +231,7 @@ static inline bool atomicReadWrite( RUNTIME_CHECK(false); } LOG_TRACE( - log, + rw_ctx.log, "keyspace={} table_id={} region_id={} cost [region decode {}, write part {}] ms", rw_ctx.keyspace_id, rw_ctx.table_id, From 2b4116c8b1bfe59e129d104102b800da694f893e Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Tue, 6 Feb 2024 11:27:54 +0800 Subject: [PATCH 18/37] remove spill impl Signed-off-by: CalvinNeo --- .../KVStore/MultiRaft/Spill/SpillImpl.h | 20 ------------------- 1 file changed, 20 deletions(-) delete mode 100644 dbms/src/Storages/KVStore/MultiRaft/Spill/SpillImpl.h diff --git a/dbms/src/Storages/KVStore/MultiRaft/Spill/SpillImpl.h b/dbms/src/Storages/KVStore/MultiRaft/Spill/SpillImpl.h deleted file mode 100644 index 8fccec877d0..00000000000 --- a/dbms/src/Storages/KVStore/MultiRaft/Spill/SpillImpl.h +++ /dev/null @@ -1,20 +0,0 @@ -// Copyright 2024 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -#pragma once - -namespace DB -{ - -} // namespace DB From f733fc4010d408141cf24ce282739abe8366767a Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Tue, 6 Feb 2024 11:45:38 +0800 Subject: [PATCH 19/37] fmt Signed-off-by: CalvinNeo --- dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp b/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp index d3cf3a6c9c8..04491b06358 100644 --- a/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp +++ b/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp @@ -282,7 +282,7 @@ DM::WriteResult writeRegionDataToStorage( /// Try read then write once. { - if (atomicReadWrite(log, rw_ctx, region, data_list_read, false)) + if (atomicReadWrite(rw_ctx, region, data_list_read, false)) { return std::move(rw_ctx.write_result); } @@ -295,7 +295,7 @@ DM::WriteResult writeRegionDataToStorage( tmt.getSchemaSyncerManager()->syncTableSchema(context, keyspace_id, table_id); auto schema_sync_cost = watch.elapsedMilliseconds(); LOG_INFO(log, "sync schema cost {} ms, keyspace={} table_id={}", schema_sync_cost, keyspace_id, table_id); - if (!atomicReadWrite(log, rw_ctx, region, data_list_read, true)) + if (!atomicReadWrite(rw_ctx, region, data_list_read, true)) { // Failure won't be tolerated this time. throw Exception( From db58a19f8669eea512aefc2bc2d1e64f96069ed1 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Tue, 6 Feb 2024 16:00:50 +0800 Subject: [PATCH 20/37] address Signed-off-by: CalvinNeo --- dbms/src/Debug/dbgFuncMockRaftSnapshot.cpp | 5 ++--- .../KVStore/Decode/DecodingStorageSchemaSnapshot.cpp | 4 +++- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/dbms/src/Debug/dbgFuncMockRaftSnapshot.cpp b/dbms/src/Debug/dbgFuncMockRaftSnapshot.cpp index 99368435cd1..9315dc1c318 100644 --- a/dbms/src/Debug/dbgFuncMockRaftSnapshot.cpp +++ b/dbms/src/Debug/dbgFuncMockRaftSnapshot.cpp @@ -609,9 +609,8 @@ RegionPtrWithBlock::CachePtr GenRegionPreDecodeBlockData(const RegionPtr & regio throw; } - DecodingStorageSchemaSnapshotConstPtr decoding_schema_snapshot; - std::tie(decoding_schema_snapshot, std::ignore) - = storage->getSchemaSnapshotAndBlockForDecoding(lock, false, true); + DecodingStorageSchemaSnapshotConstPtr decoding_schema_snapshot + = storage->getSchemaSnapshotAndBlockForDecoding(lock, false, true).first; res_block = createBlockSortByColumnID(decoding_schema_snapshot); auto reader = RegionBlockReader(decoding_schema_snapshot); return reader.read(res_block, *data_list_read, force_decode); diff --git a/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.cpp b/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.cpp index bc3152f95e5..d508420ca1b 100644 --- a/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.cpp +++ b/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.cpp @@ -27,8 +27,10 @@ DecodingStorageSchemaSnapshot::DecodingStorageSchemaSnapshot( , pk_is_handle{table_info_.pk_is_handle} , is_common_handle{table_info_.is_common_handle} , decoding_schema_epoch{decoding_schema_epoch_} + , col_id_to_block_pos(column_defines.size()) + , col_id_to_def_pos(column_defines.size()) { - std::unordered_map column_lut; + std::unordered_map column_lut(table_info_.columns.size()); // col id -> tidb pos, has no internal cols. for (size_t i = 0; i < table_info_.columns.size(); i++) { From 5b5fc1879f40bca638be550971bf67b152cf3b81 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Tue, 6 Feb 2024 16:24:34 +0800 Subject: [PATCH 21/37] address Signed-off-by: CalvinNeo --- .../Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.cpp b/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.cpp index d508420ca1b..6e5b340f7cd 100644 --- a/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.cpp +++ b/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.cpp @@ -27,10 +27,9 @@ DecodingStorageSchemaSnapshot::DecodingStorageSchemaSnapshot( , pk_is_handle{table_info_.pk_is_handle} , is_common_handle{table_info_.is_common_handle} , decoding_schema_epoch{decoding_schema_epoch_} - , col_id_to_block_pos(column_defines.size()) - , col_id_to_def_pos(column_defines.size()) { std::unordered_map column_lut(table_info_.columns.size()); + col_id_to_block_pos. // col id -> tidb pos, has no internal cols. for (size_t i = 0; i < table_info_.columns.size(); i++) { From ab48246d084f4c438b9679ca4bad737639f6cc6e Mon Sep 17 00:00:00 2001 From: Calvin Neo Date: Tue, 6 Feb 2024 16:31:04 +0800 Subject: [PATCH 22/37] Update dbms/src/Storages/KVStore/Decode/RegionDataRead.h Co-authored-by: JaySon --- dbms/src/Storages/KVStore/Decode/RegionDataRead.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/KVStore/Decode/RegionDataRead.h b/dbms/src/Storages/KVStore/Decode/RegionDataRead.h index a1a996b666a..22862d869f1 100644 --- a/dbms/src/Storages/KVStore/Decode/RegionDataRead.h +++ b/dbms/src/Storages/KVStore/Decode/RegionDataRead.h @@ -26,7 +26,7 @@ struct RegionDataReadInfo { RegionDataReadInfo( RawTiDBPK && pk_, - UInt8 && write_type_, + UInt8 write_type_, Timestamp && commit_ts_, std::shared_ptr && value_) : pk(std::move(pk_)) From 57a5b6ff1f94a3318939c86857f7239b47a73123 Mon Sep 17 00:00:00 2001 From: Calvin Neo Date: Tue, 6 Feb 2024 16:31:13 +0800 Subject: [PATCH 23/37] Update dbms/src/Storages/KVStore/Decode/RegionDataRead.h Co-authored-by: JaySon --- dbms/src/Storages/KVStore/Decode/RegionDataRead.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/KVStore/Decode/RegionDataRead.h b/dbms/src/Storages/KVStore/Decode/RegionDataRead.h index 22862d869f1..e5ae380a94f 100644 --- a/dbms/src/Storages/KVStore/Decode/RegionDataRead.h +++ b/dbms/src/Storages/KVStore/Decode/RegionDataRead.h @@ -36,7 +36,7 @@ struct RegionDataReadInfo {} RegionDataReadInfo( const RawTiDBPK & pk_, - const UInt8 & write_type_, + UInt8 write_type_, const Timestamp & commit_ts_, const std::shared_ptr & value_) : pk(pk_) From 24f1d4d55ca22251dbc330c59942547a1974cff0 Mon Sep 17 00:00:00 2001 From: Calvin Neo Date: Tue, 6 Feb 2024 16:58:50 +0800 Subject: [PATCH 24/37] Update dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp Co-authored-by: Lloyd-Pottiger <60744015+Lloyd-Pottiger@users.noreply.github.com> --- dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp b/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp index 04491b06358..faeed50bd29 100644 --- a/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp +++ b/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp @@ -99,8 +99,8 @@ struct AtomicReadWriteCtx const LoggerPtr & log; const Context & context; const TMTContext & tmt; - KeyspaceID keyspace_id; - TableID table_id; + const KeyspaceID keyspace_id; + const TableID table_id; DM::WriteResult write_result = std::nullopt; UInt64 region_decode_cost = -1; UInt64 write_part_cost = -1; From 8c2625d983d0f5e3d7408ad18a76f5e22a417027 Mon Sep 17 00:00:00 2001 From: Calvin Neo Date: Tue, 6 Feb 2024 17:24:24 +0800 Subject: [PATCH 25/37] Update dbms/src/Storages/KVStore/MultiRaft/Spill/RegionUncommittedDataList.h Co-authored-by: JaySon --- .../KVStore/MultiRaft/Spill/RegionUncommittedDataList.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/KVStore/MultiRaft/Spill/RegionUncommittedDataList.h b/dbms/src/Storages/KVStore/MultiRaft/Spill/RegionUncommittedDataList.h index cd1fbc15e5d..5c7e9154ad1 100644 --- a/dbms/src/Storages/KVStore/MultiRaft/Spill/RegionUncommittedDataList.h +++ b/dbms/src/Storages/KVStore/MultiRaft/Spill/RegionUncommittedDataList.h @@ -21,7 +21,7 @@ namespace DB { struct RegionUncommittedData { - RegionUncommittedData(RawTiDBPK && pk_, UInt8 && write_type_, std::shared_ptr && value_) + RegionUncommittedData(RawTiDBPK && pk_, UInt8 write_type_, std::shared_ptr && value_) : pk(std::move(pk_)) , write_type(write_type_) , value(std::move(value_)) From b1a89f49eb5d6901b70036e7b07f2b122d299b7a Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Tue, 6 Feb 2024 17:25:53 +0800 Subject: [PATCH 26/37] replace has_version_column with with_version_column Signed-off-by: CalvinNeo --- .../KVStore/Decode/DecodingStorageSchemaSnapshot.cpp | 9 ++++----- .../KVStore/Decode/DecodingStorageSchemaSnapshot.h | 4 ++-- dbms/src/Storages/StorageDeltaMerge.cpp | 8 ++++---- dbms/src/Storages/StorageDeltaMerge.h | 2 +- 4 files changed, 11 insertions(+), 12 deletions(-) diff --git a/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.cpp b/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.cpp index 6e5b340f7cd..4f8e8d339cd 100644 --- a/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.cpp +++ b/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.cpp @@ -22,14 +22,13 @@ DecodingStorageSchemaSnapshot::DecodingStorageSchemaSnapshot( const TiDB::TableInfo & table_info_, const DM::ColumnDefine & original_handle_, Int64 decoding_schema_epoch_, - bool has_version_column) + bool with_version_column) : column_defines{std::move(column_defines_)} , pk_is_handle{table_info_.pk_is_handle} , is_common_handle{table_info_.is_common_handle} , decoding_schema_epoch{decoding_schema_epoch_} { std::unordered_map column_lut(table_info_.columns.size()); - col_id_to_block_pos. // col id -> tidb pos, has no internal cols. for (size_t i = 0; i < table_info_.columns.size(); i++) { @@ -41,7 +40,7 @@ DecodingStorageSchemaSnapshot::DecodingStorageSchemaSnapshot( for (size_t i = 0; i < column_defines->size(); i++) { auto & cd = (*column_defines)[i]; - if (cd.id != VersionColumnID || has_version_column) + if (cd.id != VersionColumnID || with_version_column) { col_id_to_block_pos.insert({cd.id, index_in_block++}); } @@ -121,7 +120,7 @@ TMTPKType getTMTPKType(const IDataType & rhs) return TMTPKType::UNSPECIFIED; } -Block createBlockSortByColumnID(DecodingStorageSchemaSnapshotConstPtr schema_snapshot, bool has_version_column) +Block createBlockSortByColumnID(DecodingStorageSchemaSnapshotConstPtr schema_snapshot, bool with_version_column) { Block block; // # Safety @@ -137,7 +136,7 @@ Block createBlockSortByColumnID(DecodingStorageSchemaSnapshotConstPtr schema_sna // - (TiDBPkColumnID, _tidb_rowid, i64) auto col_id = iter->first; auto & cd = (*(schema_snapshot->column_defines))[iter->second]; - if (!has_version_column && cd.id == VersionColumnID) + if (!with_version_column && cd.id == VersionColumnID) continue; block.insert({cd.type->createColumn(), cd.type, cd.name, col_id}); } diff --git a/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.h b/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.h index b9603f5e751..1ad996c6257 100644 --- a/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.h +++ b/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.h @@ -51,7 +51,7 @@ struct DecodingStorageSchemaSnapshot const TiDB::TableInfo & table_info_, const DM::ColumnDefine & original_handle_, Int64 decoding_schema_epoch_, - bool has_version_column); + bool with_version_column); DISALLOW_COPY(DecodingStorageSchemaSnapshot); @@ -92,7 +92,7 @@ struct DecodingStorageSchemaSnapshot using DecodingStorageSchemaSnapshotPtr = std::shared_ptr; using DecodingStorageSchemaSnapshotConstPtr = std::shared_ptr; -Block createBlockSortByColumnID(DecodingStorageSchemaSnapshotConstPtr schema_snapshot, bool has_version_column = true); +Block createBlockSortByColumnID(DecodingStorageSchemaSnapshotConstPtr schema_snapshot, bool with_version_column = true); void clearBlockData(Block & block); diff --git a/dbms/src/Storages/StorageDeltaMerge.cpp b/dbms/src/Storages/StorageDeltaMerge.cpp index a9e02cbdf41..9f27d70b936 100644 --- a/dbms/src/Storages/StorageDeltaMerge.cpp +++ b/dbms/src/Storages/StorageDeltaMerge.cpp @@ -1304,7 +1304,7 @@ DM::DeltaMergeStorePtr StorageDeltaMerge::getStoreIfInited() const std::pair StorageDeltaMerge::getSchemaSnapshotAndBlockForDecoding( const TableStructureLockHolder & table_structure_lock, bool need_block, - bool has_version_column) + bool with_version_column) { (void)table_structure_lock; std::lock_guard lock{decode_schema_mutex}; @@ -1316,17 +1316,17 @@ std::pair StorageDeltaMerg tidb_table_info, store->getHandle(), decoding_schema_epoch++, - has_version_column); + with_version_column); cache_blocks.clear(); decoding_schema_changed = false; } if (need_block) { - if (cache_blocks.empty() || !has_version_column) + if (cache_blocks.empty() || !with_version_column) { BlockUPtr block - = std::make_unique(createBlockSortByColumnID(decoding_schema_snapshot, has_version_column)); + = std::make_unique(createBlockSortByColumnID(decoding_schema_snapshot, with_version_column)); auto digest = hashSchema(*block); auto schema = global_context.getSharedBlockSchemas()->find(digest); if (schema) diff --git a/dbms/src/Storages/StorageDeltaMerge.h b/dbms/src/Storages/StorageDeltaMerge.h index 4a75658add9..90fc5496dcd 100644 --- a/dbms/src/Storages/StorageDeltaMerge.h +++ b/dbms/src/Storages/StorageDeltaMerge.h @@ -200,7 +200,7 @@ class StorageDeltaMerge std::pair getSchemaSnapshotAndBlockForDecoding( const TableStructureLockHolder & table_structure_lock, bool need_block, - bool has_version_column) override; + bool with_version_column) override; void releaseDecodingBlock(Int64 block_decoding_schema_epoch, BlockUPtr block) override; From 8a01e3ddf42d3f5549d6963e6472cc9e29ffa6ec Mon Sep 17 00:00:00 2001 From: Calvin Neo Date: Tue, 6 Feb 2024 17:29:50 +0800 Subject: [PATCH 27/37] Update dbms/src/Storages/KVStore/MultiRaft/Spill/RegionUncommittedDataList.h Co-authored-by: JaySon --- .../KVStore/MultiRaft/Spill/RegionUncommittedDataList.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/KVStore/MultiRaft/Spill/RegionUncommittedDataList.h b/dbms/src/Storages/KVStore/MultiRaft/Spill/RegionUncommittedDataList.h index 5c7e9154ad1..8f60c7cc8d8 100644 --- a/dbms/src/Storages/KVStore/MultiRaft/Spill/RegionUncommittedDataList.h +++ b/dbms/src/Storages/KVStore/MultiRaft/Spill/RegionUncommittedDataList.h @@ -28,7 +28,7 @@ struct RegionUncommittedData {} RegionUncommittedData( const RawTiDBPK & pk_, - const UInt8 & write_type_, + const UInt8 write_type_, const std::shared_ptr & value_) : pk(pk_) , write_type(write_type_) From ffeab592d710f1ca385e3e354365ae716f822861 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Tue, 6 Feb 2024 17:50:53 +0800 Subject: [PATCH 28/37] fix Signed-off-by: CalvinNeo --- .../KVStore/Decode/DecodingStorageSchemaSnapshot.cpp | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.cpp b/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.cpp index 4f8e8d339cd..dcfe175d011 100644 --- a/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.cpp +++ b/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.cpp @@ -126,16 +126,14 @@ Block createBlockSortByColumnID(DecodingStorageSchemaSnapshotConstPtr schema_sna // # Safety // Though `col_id_to_block_pos` lasks some fields in `col_id_to_def_pos`, // it is always a sub-sequence of `col_id_to_def_pos`. - for (auto iter = schema_snapshot->getColId2DefPosMap().begin(); iter != schema_snapshot->getColId2DefPosMap().end(); - iter++) + for (const auto & [col_id, def_pos] : schema_snapshot->getColId2DefPosMap()) { // col_id == cd.id // Including some internal columns: // - (VersionColumnID, _INTERNAL_VERSION, u64) // - (DelMarkColumnID, _INTERNAL_DELMARK, u8) // - (TiDBPkColumnID, _tidb_rowid, i64) - auto col_id = iter->first; - auto & cd = (*(schema_snapshot->column_defines))[iter->second]; + auto & cd = (*(schema_snapshot->column_defines))[def_pos]; if (!with_version_column && cd.id == VersionColumnID) continue; block.insert({cd.type->createColumn(), cd.type, cd.name, col_id}); From f9f04c421ebac9fdfdeccf4ae2f124ac67b47e0d Mon Sep 17 00:00:00 2001 From: Calvin Neo Date: Tue, 6 Feb 2024 17:52:29 +0800 Subject: [PATCH 29/37] Update dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.cpp Co-authored-by: JaySon --- .../Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.cpp b/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.cpp index dcfe175d011..0cc8ce7f463 100644 --- a/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.cpp +++ b/dbms/src/Storages/KVStore/Decode/DecodingStorageSchemaSnapshot.cpp @@ -124,7 +124,7 @@ Block createBlockSortByColumnID(DecodingStorageSchemaSnapshotConstPtr schema_sna { Block block; // # Safety - // Though `col_id_to_block_pos` lasks some fields in `col_id_to_def_pos`, + // Though `col_id_to_block_pos` lacks some fields in `col_id_to_def_pos`, // it is always a sub-sequence of `col_id_to_def_pos`. for (const auto & [col_id, def_pos] : schema_snapshot->getColId2DefPosMap()) { From 902239a553fdf00fe7f3d6a7de62c27544bbd413 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Wed, 7 Feb 2024 12:11:02 +0800 Subject: [PATCH 30/37] address Signed-off-by: CalvinNeo --- .../Spill/RegionUncommittedDataList.h | 11 +++++---- .../Storages/KVStore/MultiRaft/Spill/Spill.h | 24 ------------------- dbms/src/Storages/KVStore/Region.h | 1 - .../Storages/KVStore/tests/gtest_spill.cpp | 3 ++- 4 files changed, 9 insertions(+), 30 deletions(-) delete mode 100644 dbms/src/Storages/KVStore/MultiRaft/Spill/Spill.h diff --git a/dbms/src/Storages/KVStore/MultiRaft/Spill/RegionUncommittedDataList.h b/dbms/src/Storages/KVStore/MultiRaft/Spill/RegionUncommittedDataList.h index 8f60c7cc8d8..a5ff515d2fe 100644 --- a/dbms/src/Storages/KVStore/MultiRaft/Spill/RegionUncommittedDataList.h +++ b/dbms/src/Storages/KVStore/MultiRaft/Spill/RegionUncommittedDataList.h @@ -39,7 +39,6 @@ struct RegionUncommittedData RegionUncommittedData & operator=(const RegionUncommittedData &) = default; RegionUncommittedData & operator=(RegionUncommittedData &&) = default; -public: RawTiDBPK pk; UInt8 write_type; std::shared_ptr value; @@ -49,9 +48,6 @@ struct RegionUncommittedDataList { using Inner = std::vector; - Inner data; - Timestamp start_ts; - Inner::const_iterator cbegin() const { return data.cbegin(); } Inner::const_iterator cend() const { return data.cend(); } @@ -65,6 +61,13 @@ struct RegionUncommittedDataList Inner::const_iterator end() const { return data.end(); } size_t size() const { return data.size(); } + + Inner & getInner() { return data; } + const Inner & getInner() const { return data; } + +private: + Inner data; + // Timestamp start_ts; }; } // namespace DB \ No newline at end of file diff --git a/dbms/src/Storages/KVStore/MultiRaft/Spill/Spill.h b/dbms/src/Storages/KVStore/MultiRaft/Spill/Spill.h deleted file mode 100644 index fe9967c8e70..00000000000 --- a/dbms/src/Storages/KVStore/MultiRaft/Spill/Spill.h +++ /dev/null @@ -1,24 +0,0 @@ -// Copyright 2024 PingCAP, Inc. -// -// Licensed 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 -#include -#include - -namespace DB -{ - -} // namespace DB diff --git a/dbms/src/Storages/KVStore/Region.h b/dbms/src/Storages/KVStore/Region.h index 9ee4b4f5eb2..0c483c7589d 100644 --- a/dbms/src/Storages/KVStore/Region.h +++ b/dbms/src/Storages/KVStore/Region.h @@ -20,7 +20,6 @@ #include #include #include -#include #include #include diff --git a/dbms/src/Storages/KVStore/tests/gtest_spill.cpp b/dbms/src/Storages/KVStore/tests/gtest_spill.cpp index a29144982d1..425b82447f0 100644 --- a/dbms/src/Storages/KVStore/tests/gtest_spill.cpp +++ b/dbms/src/Storages/KVStore/tests/gtest_spill.cpp @@ -99,7 +99,8 @@ static DB::RegionUncommittedDataList buildUncommitReadList(TableID table_id, Moc = RecordKVFormat::encodeLockCfValue(RecordKVFormat::CFModifyFlag::PutFlag, "PK", 111, 999).toString(); auto pk = RecordKVFormat::getRawTiDBPK(RecordKVFormat::decodeTiKVKey(str_key)); auto value = std::make_shared(TiKVValue::copyFrom(str_val_default)); - data_list_read.data.push_back(RegionUncommittedData(std::move(pk), RecordKVFormat::CFModifyFlag::PutFlag, value)); + data_list_read.getInner().push_back( + RegionUncommittedData(std::move(pk), RecordKVFormat::CFModifyFlag::PutFlag, value)); return data_list_read; } From 0f2b954b39f04b4d54ae5000c9da44a25a9b4cc1 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Mon, 19 Feb 2024 10:57:37 +0800 Subject: [PATCH 31/37] f Signed-off-by: CalvinNeo --- .../KVStore/Decode/PartitionStreams.cpp | 22 ++++++------------- 1 file changed, 7 insertions(+), 15 deletions(-) diff --git a/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp b/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp index faeed50bd29..91e6d61b111 100644 --- a/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp +++ b/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp @@ -117,22 +117,14 @@ static void inline writeCommittedBlockDataIntoStorage( TableLockHolder drop_lock; std::tie(std::ignore, drop_lock) = std::move(lock).release(); Stopwatch watch; - // Note: do NOT use typeid_cast, since Storage is multi-inherited and typeid_cast will return nullptr - switch (storage->engineType()) - { - case ::TiDB::StorageEngine::DT: - { - auto dm_storage = std::dynamic_pointer_cast(storage); - rw_ctx.write_result = dm_storage->write(block, rw_ctx.context.getSettingsRef()); - break; - } - default: - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Unknown StorageEngine: {}", - static_cast(storage->engineType())); - } + RUNTIME_CHECK_MSG( + storage->engineType() == ::TiDB::StorageEngine::DT, + "Unknown StorageEngine: {}", + static_cast(storage->engineType())); + // Note: do NOT use typeid_cast, since Storage is multi-inherited and typeid_cast will return nullptr + auto dm_storage = std::dynamic_pointer_cast(storage); + rw_ctx.write_result = dm_storage->write(block, rw_ctx.context.getSettingsRef()); rw_ctx.write_part_cost = watch.elapsedMilliseconds(); GET_METRIC(tiflash_raft_write_data_to_storage_duration_seconds, type_write) .Observe(rw_ctx.write_part_cost / 1000.0); From 589593ad79ab32c0b4bfcf7ce3ef5538d5455c04 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Mon, 19 Feb 2024 13:21:52 +0800 Subject: [PATCH 32/37] change back Signed-off-by: CalvinNeo --- dbms/src/TiDB/Decode/RowCodec.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/TiDB/Decode/RowCodec.cpp b/dbms/src/TiDB/Decode/RowCodec.cpp index e3cf4fe01c3..3e7f1fa4c20 100644 --- a/dbms/src/TiDB/Decode/RowCodec.cpp +++ b/dbms/src/TiDB/Decode/RowCodec.cpp @@ -654,7 +654,7 @@ bool appendRowV1ToBlock( continue; } - auto * raw_column = const_cast((block.safeGetByPosition(block_column_pos)).column.get()); + auto * raw_column = const_cast((block.getByPosition(block_column_pos)).column.get()); const auto & column_info = column_infos[column_ids_iter->second]; DatumFlat datum(decoded_field_iter->second, column_info.tp); const Field & unflattened = datum.field(); From 07e67054aed1c26284ebf43d26ceb5abce4f0d78 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Mon, 19 Feb 2024 14:11:00 +0800 Subject: [PATCH 33/37] add spill Signed-off-by: CalvinNeo --- dbms/src/Storages/KVStore/CMakeLists.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/dbms/src/Storages/KVStore/CMakeLists.txt b/dbms/src/Storages/KVStore/CMakeLists.txt index a78cbc85dcf..6a7bcedd5b6 100644 --- a/dbms/src/Storages/KVStore/CMakeLists.txt +++ b/dbms/src/Storages/KVStore/CMakeLists.txt @@ -16,12 +16,14 @@ add_headers_and_sources(kvstore .) add_headers_and_sources(kvstore ./FFI) add_headers_and_sources(kvstore ./MultiRaft) add_headers_and_sources(kvstore ./MultiRaft/Disagg) +add_headers_and_sources(kvstore ./MultiRaft/Spill) add_headers_and_sources(kvstore ./Utils) add_headers_and_sources(kvstore ./TiKVHelpers) add_headers_and_sources(kvstore ./Decode) add_headers_and_sources(kvstore ./Read) add_subdirectory (./MultiRaft/Disagg) +add_subdirectory (./MultiRaft/Spill) add_library(kvstore ${kvstore_headers} ${kvstore_sources}) target_include_directories(kvstore PRIVATE ${TiFlash_SOURCE_DIR}/contrib/GmSSL/include) From ba40684ffe26b75715cfcf463ca7933f631004e5 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Mon, 19 Feb 2024 14:35:13 +0800 Subject: [PATCH 34/37] add spill 2 Signed-off-by: CalvinNeo --- dbms/src/Storages/KVStore/CMakeLists.txt | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/src/Storages/KVStore/CMakeLists.txt b/dbms/src/Storages/KVStore/CMakeLists.txt index 6a7bcedd5b6..f522c25ea16 100644 --- a/dbms/src/Storages/KVStore/CMakeLists.txt +++ b/dbms/src/Storages/KVStore/CMakeLists.txt @@ -23,7 +23,6 @@ add_headers_and_sources(kvstore ./Decode) add_headers_and_sources(kvstore ./Read) add_subdirectory (./MultiRaft/Disagg) -add_subdirectory (./MultiRaft/Spill) add_library(kvstore ${kvstore_headers} ${kvstore_sources}) target_include_directories(kvstore PRIVATE ${TiFlash_SOURCE_DIR}/contrib/GmSSL/include) From 24be10c0fb126385607ad86aad07f379023aea91 Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Mon, 19 Feb 2024 15:00:05 +0800 Subject: [PATCH 35/37] resolve some todb Signed-off-by: CalvinNeo --- .../KVStore/Decode/PartitionStreams.cpp | 23 ++++++++++--------- dbms/src/Storages/KVStore/FFI/ProxyFFI.cpp | 2 +- dbms/src/Storages/KVStore/KVStore.h | 8 ++++++- .../KVStore/MultiRaft/Persistence.cpp | 2 +- .../Storages/KVStore/tests/gtest_spill.cpp | 2 +- 5 files changed, 22 insertions(+), 15 deletions(-) diff --git a/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp b/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp index 91e6d61b111..2443bb7f2ab 100644 --- a/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp +++ b/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp @@ -219,7 +219,7 @@ static inline bool atomicReadWrite( } else { - // TODO Implement spill logic. + // TODO(Spill) Implement spill logic. RUNTIME_CHECK(false); } LOG_TRACE( @@ -244,7 +244,7 @@ template DM::WriteResult writeRegionDataToStorage( RegionDataReadInfoList & data_list_read, const LoggerPtr & log); -// TODO rename it after we support spill. +// TODO(Spill) rename it after we support spill. // ReadList could be RegionDataReadInfoList template DM::WriteResult writeRegionDataToStorage( @@ -449,24 +449,25 @@ DM::WriteResult RegionTable::writeCommittedByRegion( const LoggerPtr & log, bool lock_region) { - std::optional data_list_read = std::nullopt; + std::optional maybe_data_list_read = std::nullopt; if (region.pre_decode_cache) { // if schema version changed, use the kv data to rebuild block cache - data_list_read = std::move(region.pre_decode_cache->data_list_read); + maybe_data_list_read = std::move(region.pre_decode_cache->data_list_read); } else { - data_list_read = ReadRegionCommitCache(region, lock_region); + maybe_data_list_read = ReadRegionCommitCache(region, lock_region); } - if (!data_list_read) + if (!maybe_data_list_read.has_value()) return std::nullopt; - - reportUpstreamLatency(*data_list_read); - auto write_result = writeRegionDataToStorage(context, region, *data_list_read, log); + + RegionDataReadInfoList & data_list_read = maybe_data_list_read.value(); + reportUpstreamLatency(data_list_read); + auto write_result = writeRegionDataToStorage(context, region, data_list_read, log); auto prev_region_size = region->dataSize(); - RemoveRegionCommitCache(region, *data_list_read, lock_region); + RemoveRegionCommitCache(region, data_list_read, lock_region); auto new_region_size = region->dataSize(); if likely (new_region_size <= prev_region_size) { @@ -476,7 +477,7 @@ DM::WriteResult RegionTable::writeCommittedByRegion( GET_METRIC(tiflash_raft_raft_frequent_events_count, type_write_commit).Increment(1); } /// Save removed data to outer. - data_list_to_remove = std::move(*data_list_read); + data_list_to_remove = std::move(data_list_read); return write_result; } diff --git a/dbms/src/Storages/KVStore/FFI/ProxyFFI.cpp b/dbms/src/Storages/KVStore/FFI/ProxyFFI.cpp index 80c0a078f11..05c60dc3ab9 100644 --- a/dbms/src/Storages/KVStore/FFI/ProxyFFI.cpp +++ b/dbms/src/Storages/KVStore/FFI/ProxyFFI.cpp @@ -676,7 +676,7 @@ RawCppPtr PreHandleSnapshot( #endif // Pre-decode and save as DTFiles - // TODO Forward deadline_index when TiKV supports. + // TODO(raftstore-v2) Forward deadline_index when TiKV supports. auto prehandle_result = kvstore->preHandleSnapshotToFiles(new_region, snaps, index, term, std::nullopt, tmt); auto * res = new PreHandledSnapshotWithFiles{new_region, std::move(prehandle_result)}; return GenRawCppPtr(res, RawCppPtrTypeImpl::PreHandledSnapshotWithFiles); diff --git a/dbms/src/Storages/KVStore/KVStore.h b/dbms/src/Storages/KVStore/KVStore.h index fadcce7f459..0c37f89e6f3 100644 --- a/dbms/src/Storages/KVStore/KVStore.h +++ b/dbms/src/Storages/KVStore/KVStore.h @@ -121,7 +121,13 @@ struct ProxyConfigSummary size_t snap_handle_pool_size = 0; }; -/// TODO: brief design document. +/// KVStore manages raft replication and transactions. +/// - Holds all regions in this TiFlash store. +/// - Manages region -> table mapping. +/// - Manages persistence of all regions. +/// - Implements learner read. +/// - Wraps FFI interfaces. +/// - Use `Decoder` to transform row format into col format. class KVStore final : private boost::noncopyable { public: diff --git a/dbms/src/Storages/KVStore/MultiRaft/Persistence.cpp b/dbms/src/Storages/KVStore/MultiRaft/Persistence.cpp index 130c4190efd..b34d863d88e 100644 --- a/dbms/src/Storages/KVStore/MultiRaft/Persistence.cpp +++ b/dbms/src/Storages/KVStore/MultiRaft/Persistence.cpp @@ -84,7 +84,7 @@ bool KVStore::needFlushRegionData(UInt64 region_id, TMTContext & tmt) { auto region_task_lock = region_manager.genRegionTaskLock(region_id); const RegionPtr curr_region_ptr = getRegion(region_id); - // TODO Should handle when curr_region_ptr is null. + // If region not found, throws. return canFlushRegionDataImpl(curr_region_ptr, false, false, tmt, region_task_lock, 0, 0, 0, 0); } diff --git a/dbms/src/Storages/KVStore/tests/gtest_spill.cpp b/dbms/src/Storages/KVStore/tests/gtest_spill.cpp index 425b82447f0..7f7523bd936 100644 --- a/dbms/src/Storages/KVStore/tests/gtest_spill.cpp +++ b/dbms/src/Storages/KVStore/tests/gtest_spill.cpp @@ -135,7 +135,7 @@ try proxy_instance->bootstrapWithRegion(kvs, ctx.getTMTContext(), 1, std::nullopt); auto region = kvs.getRegion(1); RegionPtrWithBlock region_with_cache = RegionPtrWithBlock(region, nullptr); - // TODO spill logic + // TODO(Spill) spill logic EXPECT_THROW(writeRegionDataToStorage(ctx, region_with_cache, data_list_read, log), Exception); } } From c0aff919b5f22e8fb015bc6a6b835c71aefc510c Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Mon, 19 Feb 2024 15:53:36 +0800 Subject: [PATCH 36/37] fix Signed-off-by: CalvinNeo --- dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp b/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp index 2443bb7f2ab..e7dfd44d7da 100644 --- a/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp +++ b/dbms/src/Storages/KVStore/Decode/PartitionStreams.cpp @@ -462,7 +462,7 @@ DM::WriteResult RegionTable::writeCommittedByRegion( if (!maybe_data_list_read.has_value()) return std::nullopt; - + RegionDataReadInfoList & data_list_read = maybe_data_list_read.value(); reportUpstreamLatency(data_list_read); auto write_result = writeRegionDataToStorage(context, region, data_list_read, log); From 125c98bb9c459bb1540d28d9e79fd3db8083a01f Mon Sep 17 00:00:00 2001 From: CalvinNeo Date: Mon, 19 Feb 2024 16:31:31 +0800 Subject: [PATCH 37/37] fix Signed-off-by: CalvinNeo --- dbms/src/Storages/KVStore/KVStore.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/src/Storages/KVStore/KVStore.cpp b/dbms/src/Storages/KVStore/KVStore.cpp index f90722415ce..2bf4b1cfcd4 100644 --- a/dbms/src/Storages/KVStore/KVStore.cpp +++ b/dbms/src/Storages/KVStore/KVStore.cpp @@ -147,6 +147,7 @@ RegionPtr KVStore::getRegion(RegionID region_id) const return it->second; return nullptr; } + // TODO: may get regions not in segment? RegionMap KVStore::getRegionsByRangeOverlap(const RegionRange & range) const {