From ad926db1ef61a693d9aaf4b26ea3e075590c4fa9 Mon Sep 17 00:00:00 2001 From: JaySon Date: Fri, 16 Sep 2022 15:21:00 +0800 Subject: [PATCH] This is an automated cherry-pick of #5899 Signed-off-by: ti-chi-bot --- dbms/src/Storages/Transaction/RowCodec.cpp | 314 ++++++++++++++ .../Transaction/tests/RowCodecTestUtils.h | 361 ++++++++++++++++ .../tests/gtest_region_block_reader.cpp | 400 ++++++++++++++++++ dbms/src/TestUtils/TiFlashTestBasic.cpp | 16 + dbms/src/TestUtils/TiFlashTestBasic.h | 9 + 5 files changed, 1100 insertions(+) create mode 100644 dbms/src/Storages/Transaction/tests/RowCodecTestUtils.h create mode 100644 dbms/src/Storages/Transaction/tests/gtest_region_block_reader.cpp diff --git a/dbms/src/Storages/Transaction/RowCodec.cpp b/dbms/src/Storages/Transaction/RowCodec.cpp index 55a18d9a3c9..a49d38bb99f 100644 --- a/dbms/src/Storages/Transaction/RowCodec.cpp +++ b/dbms/src/Storages/Transaction/RowCodec.cpp @@ -416,10 +416,22 @@ struct RowEncoderV2 /// Cache encoded individual columns. for (size_t i_col = 0, i_val = 0; i_col < table_info.columns.size(); i_col++) { + if (i_val == fields.size()) + break; + const auto & column_info = table_info.columns[i_col]; const auto & field = fields[i_val]; +<<<<<<< HEAD if (table_info.pk_is_handle && column_info.hasPriKeyFlag()) +======= + if ((table_info.pk_is_handle || table_info.is_common_handle) && column_info.hasPriKeyFlag()) + { + // for common handle/pk is handle table, + // the field with primary key flag is usually encoded to key instead of value +>>>>>>> aae88b120d (tests: Fix RegionBlockReaderTest helper functions (#5899)) continue; + } + if (column_info.id > std::numeric_limits::ColumnIDType>::max()) is_big = true; if (!field.isNull()) @@ -504,4 +516,306 @@ void encodeRowV2(const TiDB::TableInfo & table_info, const std::vector & RowEncoderV2(table_info, fields).encode(ss); } +<<<<<<< HEAD +======= +bool appendRowToBlock( + const TiKVValue::Base & raw_value, + SortedColumnIDWithPosConstIter column_ids_iter, + SortedColumnIDWithPosConstIter column_ids_iter_end, + Block & block, + size_t block_column_pos, + const ColumnInfos & column_infos, + ColumnID pk_handle_id, + bool force_decode) +{ + switch (static_cast(raw_value[0])) + { + case static_cast(RowCodecVer::ROW_V2): + return appendRowV2ToBlock(raw_value, column_ids_iter, column_ids_iter_end, block, block_column_pos, column_infos, pk_handle_id, force_decode); + default: + return appendRowV1ToBlock(raw_value, column_ids_iter, column_ids_iter_end, block, block_column_pos, column_infos, pk_handle_id, force_decode); + } +} + +bool appendRowV2ToBlock( + const TiKVValue::Base & raw_value, + SortedColumnIDWithPosConstIter column_ids_iter, + SortedColumnIDWithPosConstIter column_ids_iter_end, + Block & block, + size_t block_column_pos, + const ColumnInfos & column_infos, + ColumnID pk_handle_id, + bool force_decode) +{ + auto row_flag = readLittleEndian(&raw_value[1]); + bool is_big = row_flag & RowV2::BigRowMask; + return is_big ? appendRowV2ToBlockImpl(raw_value, column_ids_iter, column_ids_iter_end, block, block_column_pos, column_infos, pk_handle_id, force_decode) + : appendRowV2ToBlockImpl(raw_value, column_ids_iter, column_ids_iter_end, block, block_column_pos, column_infos, pk_handle_id, force_decode); +} + +inline bool addDefaultValueToColumnIfPossible(const ColumnInfo & column_info, Block & block, size_t block_column_pos, bool force_decode) +{ + // We consider a missing column could be safely filled with NULL, unless it has not default value and is NOT NULL. + // This could saves lots of unnecessary schema syncs for old data with a schema that has newly added columns. + // for clustered index, if the pk column does not exists, it can still be decoded from the key + if (column_info.hasPriKeyFlag()) + return true; + + if (column_info.hasNoDefaultValueFlag() && column_info.hasNotNullFlag()) + { + if (!force_decode) + return false; + } + // not null or has no default value, tidb will fill with specific value. + auto * raw_column = const_cast((block.getByPosition(block_column_pos)).column.get()); + raw_column->insert(column_info.defaultValueToField()); + return true; +} + +template +bool appendRowV2ToBlockImpl( + const TiKVValue::Base & raw_value, + SortedColumnIDWithPosConstIter column_ids_iter, + SortedColumnIDWithPosConstIter column_ids_iter_end, + Block & block, + size_t block_column_pos, + const ColumnInfos & column_infos, + ColumnID pk_handle_id, + bool force_decode) +{ + size_t cursor = 2; // Skip the initial codec ver and row flag. + size_t num_not_null_columns = decodeUInt(cursor, raw_value); + size_t num_null_columns = decodeUInt(cursor, raw_value); + std::vector not_null_column_ids; + std::vector null_column_ids; + std::vector value_offsets; + decodeUInts::ColumnIDType>(cursor, raw_value, num_not_null_columns, not_null_column_ids); + decodeUInts::ColumnIDType>(cursor, raw_value, num_null_columns, null_column_ids); + decodeUInts::ValueOffsetType>(cursor, raw_value, num_not_null_columns, value_offsets); + size_t values_start_pos = cursor; + size_t idx_not_null = 0; + size_t idx_null = 0; + // Merge ordered not null/null columns to keep order. + while (idx_not_null < not_null_column_ids.size() || idx_null < null_column_ids.size()) + { + if (column_ids_iter == column_ids_iter_end) + { + // extra column + return force_decode; + } + + bool is_null; + if (idx_not_null < not_null_column_ids.size() && idx_null < null_column_ids.size()) + is_null = not_null_column_ids[idx_not_null] > null_column_ids[idx_null]; + else + is_null = idx_null < null_column_ids.size(); + + auto next_datum_column_id = is_null ? null_column_ids[idx_null] : not_null_column_ids[idx_not_null]; + const auto next_column_id = column_ids_iter->first; + if (next_column_id > next_datum_column_id) + { + // The next column id to read is bigger than the column id of next datum in encoded row. + // It means this is the datum of extra column. May happen when reading after dropping + // a column. + if (!force_decode) + return false; + // Ignore the extra column and continue to parse other datum + if (is_null) + idx_null++; + else + idx_not_null++; + } + else if (next_column_id < next_datum_column_id) + { + // The next column id to read is less than the column id of next datum in encoded row. + // It means this is the datum of missing column. May happen when reading after adding + // a column. + // Fill with default value and continue to read data for next column id. + const auto & column_info = column_infos[column_ids_iter->second]; + if (!addDefaultValueToColumnIfPossible(column_info, block, block_column_pos, force_decode)) + return false; + column_ids_iter++; + block_column_pos++; + } + else + { + // If pk_handle_id is a valid column id, then it means the table's pk_is_handle is true + // we can just ignore the pk value encoded in value part + if (unlikely(next_column_id == pk_handle_id)) + { + column_ids_iter++; + block_column_pos++; + if (is_null) + { + idx_null++; + } + else + { + idx_not_null++; + } + continue; + } + + // Parse the datum. + auto * raw_column = const_cast((block.getByPosition(block_column_pos)).column.get()); + const auto & column_info = column_infos[column_ids_iter->second]; + if (is_null) + { + if (!raw_column->isColumnNullable()) + { + if (!force_decode) + { + return false; + } + else + { + throw Exception("Detected invalid null when decoding data of column " + column_info.name + " with column type " + raw_column->getName(), + ErrorCodes::LOGICAL_ERROR); + } + } + // ColumnNullable::insertDefault just insert a null value + raw_column->insertDefault(); + idx_null++; + } + else + { + size_t start = idx_not_null ? value_offsets[idx_not_null - 1] : 0; + size_t length = value_offsets[idx_not_null] - start; + if (!raw_column->decodeTiDBRowV2Datum(values_start_pos + start, raw_value, length, force_decode)) + return false; + idx_not_null++; + } + column_ids_iter++; + block_column_pos++; + } + } + while (column_ids_iter != column_ids_iter_end) + { + if (column_ids_iter->first != pk_handle_id) + { + const auto & column_info = column_infos[column_ids_iter->second]; + if (!addDefaultValueToColumnIfPossible(column_info, block, block_column_pos, force_decode)) + return false; + } + column_ids_iter++; + block_column_pos++; + } + return true; +} + +using TiDB::DatumFlat; +bool appendRowV1ToBlock( + const TiKVValue::Base & raw_value, + SortedColumnIDWithPosConstIter column_ids_iter, + SortedColumnIDWithPosConstIter column_ids_iter_end, + Block & block, + size_t block_column_pos, + const ColumnInfos & column_infos, + ColumnID pk_handle_id, + bool force_decode) +{ + size_t cursor = 0; + std::map decoded_fields; + while (cursor < raw_value.size()) + { + Field f = DecodeDatum(cursor, raw_value); + if (f.isNull()) + break; + ColumnID col_id = f.get(); + decoded_fields.emplace(col_id, DecodeDatum(cursor, raw_value)); + } + if (cursor != raw_value.size()) + throw Exception(std::string(__PRETTY_FUNCTION__) + ": cursor is not end, remaining: " + raw_value.substr(cursor), + ErrorCodes::LOGICAL_ERROR); + + auto decoded_field_iter = decoded_fields.begin(); + while (decoded_field_iter != decoded_fields.end()) + { + if (column_ids_iter == column_ids_iter_end) + { + // extra column + return force_decode; + } + + auto next_field_column_id = decoded_field_iter->first; + if (column_ids_iter->first > next_field_column_id) + { + // extra column + if (!force_decode) + return false; + decoded_field_iter++; + } + else if (column_ids_iter->first < next_field_column_id) + { + const auto & column_info = column_infos[column_ids_iter->second]; + if (!addDefaultValueToColumnIfPossible(column_info, block, block_column_pos, force_decode)) + return false; + column_ids_iter++; + block_column_pos++; + } + else + { + // if pk_handle_id is a valid column id, then it means the table's pk_is_handle is true + // we can just ignore the pk value encoded in value part + if (unlikely(column_ids_iter->first == pk_handle_id)) + { + decoded_field_iter++; + column_ids_iter++; + block_column_pos++; + continue; + } + + 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(); + if (datum.overflow(column_info)) + { + // Overflow detected, fatal if force_decode is true, + // as schema being newer and narrow shouldn't happen. + // Otherwise return false to outer, outer should sync schema and try again. + if (force_decode) + { + throw Exception("Detected overflow when decoding data " + std::to_string(unflattened.get()) + " of column " + + column_info.name + " with column " + raw_column->getName(), + ErrorCodes::LOGICAL_ERROR); + } + + return false; + } + if (datum.invalidNull(column_info)) + { + // Null value with non-null type detected, fatal if force_decode is true, + // as schema being newer and with invalid null shouldn't happen. + // Otherwise return false to outer, outer should sync schema and try again. + if (force_decode) + { + throw Exception("Detected invalid null when decoding data " + std::to_string(unflattened.get()) + + " of column " + column_info.name + " with type " + raw_column->getName(), + ErrorCodes::LOGICAL_ERROR); + } + + return false; + } + raw_column->insert(unflattened); + decoded_field_iter++; + column_ids_iter++; + block_column_pos++; + } + } + while (column_ids_iter != column_ids_iter_end) + { + if (column_ids_iter->first != pk_handle_id) + { + const auto & column_info = column_infos[column_ids_iter->second]; + if (!addDefaultValueToColumnIfPossible(column_info, block, block_column_pos, force_decode)) + return false; + } + column_ids_iter++; + block_column_pos++; + } + return true; +} + +>>>>>>> aae88b120d (tests: Fix RegionBlockReaderTest helper functions (#5899)) } // namespace DB diff --git a/dbms/src/Storages/Transaction/tests/RowCodecTestUtils.h b/dbms/src/Storages/Transaction/tests/RowCodecTestUtils.h new file mode 100644 index 00000000000..0cc5bfe6bff --- /dev/null +++ b/dbms/src/Storages/Transaction/tests/RowCodecTestUtils.h @@ -0,0 +1,361 @@ +// Copyright 2022 PingCAP, Ltd. +// +// 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 +#include +#include + +namespace DB::tests +{ +using DM::ColumnDefine; +using DM::ColumnDefines; +using TiDB::ColumnInfo; +using TiDB::TableInfo; +using ColumnIDs = std::vector; + +template +struct ColumnTP +{ +}; +template <> +struct ColumnTP +{ + static const auto tp = TiDB::TypeTiny; +}; +template <> +struct ColumnTP +{ + static const auto tp = TiDB::TypeShort; +}; +template <> +struct ColumnTP +{ + static const auto tp = TiDB::TypeLong; +}; +template <> +struct ColumnTP +{ + static const auto tp = TiDB::TypeLongLong; +}; +template <> +struct ColumnTP +{ + static const auto tp = TiDB::TypeTiny; +}; +template <> +struct ColumnTP +{ + static const auto tp = TiDB::TypeShort; +}; +template <> +struct ColumnTP +{ + static const auto tp = TiDB::TypeLong; +}; +template <> +struct ColumnTP +{ + static const auto tp = TiDB::TypeLongLong; +}; +template <> +struct ColumnTP +{ + static const auto tp = TiDB::TypeFloat; +}; +template <> +struct ColumnTP +{ + static const auto tp = TiDB::TypeDouble; +}; +template <> +struct ColumnTP +{ + static const auto tp = TiDB::TypeString; +}; +template <> +struct ColumnTP> +{ + static const auto tp = TiDB::TypeNewDecimal; +}; +template <> +struct ColumnTP> +{ + static const auto tp = TiDB::TypeNewDecimal; +}; +template <> +struct ColumnTP> +{ + static const auto tp = TiDB::TypeNewDecimal; +}; +template <> +struct ColumnTP> +{ + static const auto tp = TiDB::TypeNewDecimal; +}; + +inline String getTestColumnName(ColumnID id) +{ + return "column" + std::to_string(id); +} + +template +ColumnInfo getColumnInfo(ColumnID id) +{ + ColumnInfo column_info; + column_info.id = id; + column_info.tp = ColumnTP::tp; + column_info.name = getTestColumnName(id); + if constexpr (std::is_unsigned_v) + column_info.setUnsignedFlag(); + if constexpr (!nullable) + column_info.setNotNullFlag(); + return column_info; +} + +template +struct ColumnIDValue +{ + static constexpr bool value_is_null = is_null; + using ValueType = std::decay_t; + ColumnIDValue(ColumnID id_, const T & value_) + : id(id_) + , value(value_) + {} + ColumnIDValue(ColumnID id_, T && value_) + : id(id_) + , value(std::move(value_)) + {} + ColumnID id; + ValueType value; +}; + +template +struct ColumnIDValue +{ + static constexpr bool value_is_null = true; + using ValueType = std::decay_t; + explicit ColumnIDValue(ColumnID id_) + : id(id_) + {} + ColumnID id; +}; + +template +using ColumnIDValueNull = ColumnIDValue; + +using OrderedColumnInfoFields = std::map>; + +template +constexpr bool IsDecimalFieldType = false; +template <> +inline constexpr bool IsDecimalFieldType> = true; +template <> +inline constexpr bool IsDecimalFieldType> = true; +template <> +inline constexpr bool IsDecimalFieldType> = true; +template <> +inline constexpr bool IsDecimalFieldType> = true; + +template +void getTableInfoFieldsInternal(OrderedColumnInfoFields & column_info_fields, Type && column_id_value) +{ + using DecayType = std::decay_t; + using ValueType = typename DecayType::ValueType; + using NearestType = typename NearestFieldType::Type; + if constexpr (DecayType::value_is_null) + { + ColumnInfo column_info = getColumnInfo(column_id_value.id); + // create non zero flen and decimal to avoid error when creating decimal type + if constexpr (IsDecimalFieldType) + { + column_info.flen = 1; + column_info.decimal = 1; + } + column_info_fields.emplace(column_id_value.id, std::make_tuple(column_info, Field())); + } + else + { + if constexpr (IsDecimalFieldType) + { + ColumnInfo column_info = getColumnInfo(column_id_value.id); + auto field = static_cast(std::move(column_id_value.value)); + column_info.flen = field.getPrec(); + column_info.decimal = field.getScale(); + column_info_fields.emplace(column_id_value.id, std::make_tuple(column_info, field)); + } + else + { + column_info_fields.emplace(column_id_value.id, + std::make_tuple(getColumnInfo(column_id_value.id), static_cast(std::move(column_id_value.value)))); + } + } +} + +template +void getTableInfoFieldsInternal(OrderedColumnInfoFields & column_info_fields, Type && first, Rest &&... rest) +{ + getTableInfoFieldsInternal(column_info_fields, first); + getTableInfoFieldsInternal(column_info_fields, std::forward(rest)...); +} + +template +std::pair> getTableInfoAndFields(ColumnIDs pk_col_ids, bool is_common_handle, Types &&... column_value_ids) +{ + OrderedColumnInfoFields column_info_fields; + getTableInfoFieldsInternal(column_info_fields, std::forward(column_value_ids)...); + TableInfo table_info; + std::vector fields; + bool pk_is_handle = pk_col_ids.size() == 1 && pk_col_ids[0] != ::DB::TiDBPkColumnID; + + for (auto & column_info_field : column_info_fields) + { + auto & column = std::get<0>(column_info_field.second); + auto & field = std::get<1>(column_info_field.second); + if (std::find(pk_col_ids.begin(), pk_col_ids.end(), column.id) != pk_col_ids.end()) + { + column.setPriKeyFlag(); + if (column.tp != TiDB::TypeLong && column.tp != TiDB::TypeTiny && column.tp != TiDB::TypeLongLong && column.tp != TiDB::TypeShort && column.tp != TiDB::TypeInt24) + { + pk_is_handle = false; + } + } + table_info.columns.emplace_back(std::move(column)); + fields.emplace_back(std::move(field)); + } + + table_info.pk_is_handle = pk_is_handle; + table_info.is_common_handle = is_common_handle; + if (is_common_handle) + { + table_info.is_common_handle = true; + // TiFlash maintains the column name of primary key + // for common handle table + TiDB::IndexInfo pk_index_info; + pk_index_info.is_primary = true; + pk_index_info.idx_name = "PRIMARY"; + pk_index_info.is_unique = true; + for (auto pk_col_id : pk_col_ids) + { + TiDB::IndexColumnInfo index_column_info; + for (auto & column : table_info.columns) + { + if (column.id == pk_col_id) + { + index_column_info.name = column.name; + break; + } + } + pk_index_info.idx_cols.emplace_back(index_column_info); + } + table_info.index_infos.emplace_back(pk_index_info); + } + + return std::make_pair(std::move(table_info), std::move(fields)); +} + +inline DecodingStorageSchemaSnapshotConstPtr getDecodingStorageSchemaSnapshot(const TableInfo & table_info) +{ + ColumnDefines store_columns; + if (table_info.is_common_handle) + { + DM::ColumnDefine extra_handle_column{EXTRA_HANDLE_COLUMN_ID, EXTRA_HANDLE_COLUMN_NAME, EXTRA_HANDLE_COLUMN_STRING_TYPE}; + store_columns.emplace_back(extra_handle_column); + } + else + { + DM::ColumnDefine extra_handle_column{EXTRA_HANDLE_COLUMN_ID, EXTRA_HANDLE_COLUMN_NAME, EXTRA_HANDLE_COLUMN_INT_TYPE}; + store_columns.emplace_back(extra_handle_column); + } + store_columns.emplace_back(VERSION_COLUMN_ID, VERSION_COLUMN_NAME, VERSION_COLUMN_TYPE); + store_columns.emplace_back(TAG_COLUMN_ID, TAG_COLUMN_NAME, TAG_COLUMN_TYPE); + ColumnID handle_id = EXTRA_HANDLE_COLUMN_ID; + for (const auto & column_info : table_info.columns) + { + if (table_info.pk_is_handle) + { + if (column_info.hasPriKeyFlag()) + handle_id = column_info.id; + } + store_columns.emplace_back(column_info.id, column_info.name, DB::getDataTypeByColumnInfo(column_info)); + } + + if (handle_id != EXTRA_HANDLE_COLUMN_ID) + { + auto iter = std::find_if(store_columns.begin(), store_columns.end(), [&](const ColumnDefine & cd) { return cd.id == handle_id; }); + return std::make_shared(std::make_shared(store_columns), table_info, *iter, /* decoding_schema_version_ */ 1); + } + else + { + return std::make_shared(std::make_shared(store_columns), table_info, store_columns[0], /* decoding_schema_version_ */ 1); + } +} + +template +size_t valueStartPos(const TableInfo & table_info) +{ + return 1 + 1 + 2 + 2 + (is_big ? 8 : 3) * table_info.columns.size(); +} + +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 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); + if (decoding_schema->pk_is_handle) + appendRowToBlock(row_value, iter, sorted_column_id_with_pos.end(), block, value_column_num, decoding_schema->column_infos, decoding_schema->pk_column_ids[0], true); + else + appendRowToBlock(row_value, iter, sorted_column_id_with_pos.end(), block, value_column_num, decoding_schema->column_infos, InvalidColumnID, true); + + // remove first three column + for (size_t i = 0; i < value_column_num; i++) + block.erase(0); + return block; +} + +template +std::tuple getValueLengthByRowV2(const T & v) +{ + using NearestType = typename NearestFieldType::Type; + auto [table_info, fields] = getTableInfoAndFields({EXTRA_HANDLE_COLUMN_ID}, false, ColumnIDValue(1, v)); + auto decoding_schema = getDecodingStorageSchemaSnapshot(table_info); + WriteBufferFromOwnString ss; + encodeRowV2(table_info, fields, ss); + auto encoded = ss.str(); + Block block = decodeRowToBlock(encoded, decoding_schema); + return std::make_tuple(static_cast(std::move((*block.getByPosition(0).column)[0].template safeGet())), + encoded.size() - valueStartPos(table_info)); +} + +template +T getValueByRowV1(const T & v) +{ + using NearestType = typename NearestFieldType::Type; + auto [table_info, fields] = getTableInfoAndFields({EXTRA_HANDLE_COLUMN_ID}, false, ColumnIDValue(1, v)); + auto decoding_schema = getDecodingStorageSchemaSnapshot(table_info); + WriteBufferFromOwnString ss; + encodeRowV1(table_info, fields, ss); + auto encoded = ss.str(); + Block block = decodeRowToBlock(encoded, decoding_schema); + return static_cast(std::move((*block.getByPosition(0).column)[0].template safeGet())); +} + +} // namespace DB::tests diff --git a/dbms/src/Storages/Transaction/tests/gtest_region_block_reader.cpp b/dbms/src/Storages/Transaction/tests/gtest_region_block_reader.cpp new file mode 100644 index 00000000000..dd58f166dac --- /dev/null +++ b/dbms/src/Storages/Transaction/tests/gtest_region_block_reader.cpp @@ -0,0 +1,400 @@ +// Copyright 2022 PingCAP, Ltd. +// +// 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 +#include +#include +#include +#include + +using TableInfo = TiDB::TableInfo; + +namespace DB::tests +{ +using ColumnIDs = std::vector; +class RegionBlockReaderTest : public ::testing::Test +{ +public: + RegionBlockReaderTest() + : logger(Logger::get("RegionBlockReaderTest")) + {} + +protected: + Int64 handle_value = 100; + UInt8 del_mark_value = 0; + UInt64 version_value = 100; + size_t rows = 3; + + RegionDataReadInfoList data_list_read; + std::unordered_map fields_map; + + LoggerPtr logger; + + enum RowEncodeVersion + { + RowV1, + RowV2 + }; + +protected: + void SetUp() override + { + data_list_read.clear(); + fields_map.clear(); + } + + void TearDown() override {} + + void encodeColumns(const TableInfo & table_info, const std::vector & fields, RowEncodeVersion row_version) + { + // for later check + std::unordered_map column_name_columns_index_map; + for (size_t i = 0; i < table_info.columns.size(); i++) + { + fields_map.emplace(table_info.columns[i].id, fields[i]); + column_name_columns_index_map.emplace(table_info.columns[i].name, i); + } + + std::vector value_encode_fields; + std::vector key_encode_fields; + for (size_t i = 0; i < table_info.columns.size(); i++) + { + if (table_info.is_common_handle || table_info.pk_is_handle) + { + if (!table_info.columns[i].hasPriKeyFlag()) + value_encode_fields.emplace_back(fields[i]); + else + key_encode_fields.emplace_back(fields[i]); + } + else + { + value_encode_fields.emplace_back(fields[i]); + } + } + + // create the RawTiDBPK section of encoded key + WriteBufferFromOwnString pk_buf; + if (table_info.is_common_handle) + { + auto & primary_index_info = table_info.getPrimaryIndexInfo(); + for (size_t i = 0; i < primary_index_info.idx_cols.size(); i++) + { + auto idx = column_name_columns_index_map[primary_index_info.idx_cols[i].name]; + EncodeDatum(key_encode_fields[i], table_info.columns[idx].getCodecFlag(), pk_buf); + } + } + else + { + DB::EncodeInt64(handle_value, pk_buf); + } + RawTiDBPK pk{std::make_shared(pk_buf.releaseStr())}; + + // create encoded value + WriteBufferFromOwnString value_buf; + if (row_version == RowEncodeVersion::RowV1) + { + encodeRowV1(table_info, value_encode_fields, value_buf); + } + else if (row_version == RowEncodeVersion::RowV2) + { + encodeRowV2(table_info, value_encode_fields, value_buf); + } + else + { + throw Exception("Unknown row format " + std::to_string(row_version), ErrorCodes::LOGICAL_ERROR); + } + auto row_value = std::make_shared(value_buf.releaseStr()); + for (size_t i = 0; i < rows; i++) + data_list_read.emplace_back(pk, del_mark_value, version_value, row_value); + } + + void checkBlock(DecodingStorageSchemaSnapshotConstPtr decoding_schema, const Block & block) const + { + ASSERT_EQ(block.columns(), decoding_schema->column_defines->size()); + for (size_t row = 0; row < rows; row++) + { + for (size_t pos = 0; pos < block.columns(); pos++) + { + const auto & column_element = block.getByPosition(pos); + auto gen_error_log = [&]() { + return fmt::format( + " when checking column\n id={}, name={}, nrow={}\n decoded block is:\n{}\n", + column_element.column_id, + column_element.name, + row, + getColumnsContent(block.getColumnsWithTypeAndName())); + }; + if (row == 0) + { + ASSERT_EQ(column_element.column->size(), rows); + } + if (column_element.name == EXTRA_HANDLE_COLUMN_NAME) + { + if (decoding_schema->is_common_handle) + { + ASSERT_FIELD_EQ((*column_element.column)[row], Field(*std::get<0>(data_list_read[row]))) << gen_error_log(); + } + else + { + ASSERT_FIELD_EQ((*column_element.column)[row], Field(handle_value)) << gen_error_log(); + } + } + else if (column_element.name == VERSION_COLUMN_NAME) + { + ASSERT_FIELD_EQ((*column_element.column)[row], Field(version_value)) << gen_error_log(); + } + else if (column_element.name == TAG_COLUMN_NAME) + { + ASSERT_FIELD_EQ((*column_element.column)[row], Field(NearestFieldType::Type(del_mark_value))) << gen_error_log(); + } + else + { + ASSERT_FIELD_EQ((*column_element.column)[row], fields_map.at(column_element.column_id)) << gen_error_log(); + } + } + } + } + + bool decodeAndCheckColumns(DecodingStorageSchemaSnapshotConstPtr decoding_schema, bool force_decode) const + { + RegionBlockReader reader{decoding_schema}; + Block block = createBlockSortByColumnID(decoding_schema); + if (!reader.read(block, data_list_read, force_decode)) + return false; + + checkBlock(decoding_schema, block); + return true; + } + + std::pair> getNormalTableInfoFields(const ColumnIDs & pk_col_ids, bool is_common_handle) const + { + return getTableInfoAndFields( + pk_col_ids, + is_common_handle, + ColumnIDValue(2, handle_value), + ColumnIDValue(3, std::numeric_limits::max()), + ColumnIDValue(4, std::numeric_limits::min()), + ColumnIDValue(9, String("aaa")), + ColumnIDValue(10, DecimalField(ToDecimal(12345678910ULL, 4), 4)), + ColumnIDValueNull(11)); + } + + TableInfo getTableInfoWithMoreColumns(const ColumnIDs & handle_ids, bool is_common_handle) + { + TableInfo table_info; + std::tie(table_info, std::ignore) = getTableInfoAndFields( + handle_ids, + is_common_handle, + ColumnIDValue(1, String("")), + ColumnIDValue(2, handle_value), + ColumnIDValue(3, std::numeric_limits::max()), + ColumnIDValue(4, std::numeric_limits::min()), + ColumnIDValue(8, String("")), + ColumnIDValue(9, String("aaa")), + ColumnIDValue(10, DecimalField(ToDecimal(12345678910ULL, 4), 4)), + ColumnIDValueNull(11), + ColumnIDValue(13, String(""))); + + // add default value for missing column + std::vector missing_column_ids{1, 8, 13}; + String missing_column_default_value = String("default"); + for (auto & column : table_info.columns) + { + if (std::find(missing_column_ids.begin(), missing_column_ids.end(), column.id) != missing_column_ids.end()) + { + column.origin_default_value = missing_column_default_value; + fields_map.emplace(column.id, Field(missing_column_default_value)); + } + } + return table_info; + } + + TableInfo getTableInfoWithLessColumns(const ColumnIDs & handle_ids, bool is_common_handle) const + { + TableInfo table_info; + std::tie(table_info, std::ignore) = getTableInfoAndFields( + handle_ids, + is_common_handle, + ColumnIDValue(2, handle_value), + ColumnIDValue(4, std::numeric_limits::min()), + ColumnIDValue(9, String("aaa")), + ColumnIDValue(10, DecimalField(ToDecimal(12345678910ULL, 4), 4))); + return table_info; + } + + TableInfo getTableInfoWithMoreNarrowIntType(const ColumnIDs & handle_ids, bool is_common_handle) const + { + TableInfo table_info; + std::tie(table_info, std::ignore) = getTableInfoAndFields( + handle_ids, + is_common_handle, + ColumnIDValue(2, handle_value), + ColumnIDValue(3, std::numeric_limits::max()), + ColumnIDValue(4, std::numeric_limits::min()), + ColumnIDValue(9, String("aaa")), + ColumnIDValue(10, DecimalField(ToDecimal(12345678910ULL, 4), 4)), + ColumnIDValueNull(11)); + return table_info; + } + + TableInfo getTableInfoFieldsForInvalidNULLTest(const ColumnIDs & handle_ids, bool is_common_handle) const + { + TableInfo table_info; + std::tie(table_info, std::ignore) = getTableInfoAndFields( + handle_ids, + is_common_handle, + ColumnIDValue(2, handle_value), + ColumnIDValue(3, std::numeric_limits::max()), + ColumnIDValue(4, std::numeric_limits::min()), + ColumnIDValue(9, String("aaa")), + ColumnIDValue(10, DecimalField(ToDecimal(12345678910ULL, 4), 4)), + ColumnIDValue(11, std::numeric_limits::min())); + return table_info; + } +}; + +TEST_F(RegionBlockReaderTest, PKIsNotHandle) +{ + auto [table_info, fields] = getNormalTableInfoFields({EXTRA_HANDLE_COLUMN_ID}, false); + ASSERT_EQ(table_info.is_common_handle, false); + ASSERT_EQ(table_info.pk_is_handle, false); + ASSERT_FALSE(table_info.getColumnInfo(2).hasPriKeyFlag()); + + encodeColumns(table_info, fields, RowEncodeVersion::RowV2); + auto decoding_schema = getDecodingStorageSchemaSnapshot(table_info); + ASSERT_TRUE(decodeAndCheckColumns(decoding_schema, true)); +} + +TEST_F(RegionBlockReaderTest, PKIsHandle) +{ + auto [table_info, fields] = getNormalTableInfoFields({2}, false); + ASSERT_EQ(table_info.is_common_handle, false); + ASSERT_EQ(table_info.pk_is_handle, true); + ASSERT_TRUE(table_info.getColumnInfo(2).hasPriKeyFlag()); + + encodeColumns(table_info, fields, RowEncodeVersion::RowV2); + auto decoding_schema = getDecodingStorageSchemaSnapshot(table_info); + ASSERT_TRUE(decodeAndCheckColumns(decoding_schema, true)); +} + +TEST_F(RegionBlockReaderTest, CommonHandle) +{ + auto [table_info, fields] = getNormalTableInfoFields({2, 3, 4}, true); + ASSERT_EQ(table_info.is_common_handle, true); + ASSERT_EQ(table_info.pk_is_handle, false); + ASSERT_TRUE(table_info.getColumnInfo(2).hasPriKeyFlag()); + ASSERT_TRUE(table_info.getColumnInfo(3).hasPriKeyFlag()); + ASSERT_TRUE(table_info.getColumnInfo(4).hasPriKeyFlag()); + + encodeColumns(table_info, fields, RowEncodeVersion::RowV2); + auto decoding_schema = getDecodingStorageSchemaSnapshot(table_info); + ASSERT_TRUE(decodeAndCheckColumns(decoding_schema, true)); +} + +TEST_F(RegionBlockReaderTest, MissingColumnRowV2) +{ + auto [table_info, fields] = getNormalTableInfoFields({EXTRA_HANDLE_COLUMN_ID}, false); + encodeColumns(table_info, fields, RowEncodeVersion::RowV2); + auto new_table_info = getTableInfoWithMoreColumns({EXTRA_HANDLE_COLUMN_ID}, false); + auto new_decoding_schema = getDecodingStorageSchemaSnapshot(new_table_info); + ASSERT_TRUE(decodeAndCheckColumns(new_decoding_schema, false)); +} + +TEST_F(RegionBlockReaderTest, MissingColumnRowV1) +{ + auto [table_info, fields] = getNormalTableInfoFields({EXTRA_HANDLE_COLUMN_ID}, false); + encodeColumns(table_info, fields, RowEncodeVersion::RowV1); + auto new_table_info = getTableInfoWithMoreColumns({EXTRA_HANDLE_COLUMN_ID}, false); + auto new_decoding_schema = getDecodingStorageSchemaSnapshot(new_table_info); + ASSERT_TRUE(decodeAndCheckColumns(new_decoding_schema, false)); +} + +TEST_F(RegionBlockReaderTest, ExtraColumnRowV2) +{ + auto [table_info, fields] = getNormalTableInfoFields({EXTRA_HANDLE_COLUMN_ID}, false); + encodeColumns(table_info, fields, RowEncodeVersion::RowV2); + auto new_table_info = getTableInfoWithLessColumns({EXTRA_HANDLE_COLUMN_ID}, false); + auto new_decoding_schema = getDecodingStorageSchemaSnapshot(new_table_info); + ASSERT_FALSE(decodeAndCheckColumns(new_decoding_schema, false)); + ASSERT_TRUE(decodeAndCheckColumns(new_decoding_schema, true)); +} + +TEST_F(RegionBlockReaderTest, ExtraColumnRowV1) +{ + auto [table_info, fields] = getNormalTableInfoFields({EXTRA_HANDLE_COLUMN_ID}, false); + encodeColumns(table_info, fields, RowEncodeVersion::RowV1); + auto new_table_info = getTableInfoWithLessColumns({EXTRA_HANDLE_COLUMN_ID}, false); + auto new_decoding_schema = getDecodingStorageSchemaSnapshot(new_table_info); + ASSERT_FALSE(decodeAndCheckColumns(new_decoding_schema, false)); + ASSERT_TRUE(decodeAndCheckColumns(new_decoding_schema, true)); +} + +TEST_F(RegionBlockReaderTest, OverflowColumnRowV2) +{ + auto [table_info, fields] = getNormalTableInfoFields({EXTRA_HANDLE_COLUMN_ID}, false); + encodeColumns(table_info, fields, RowEncodeVersion::RowV2); + auto new_table_info = getTableInfoWithMoreNarrowIntType({EXTRA_HANDLE_COLUMN_ID}, false); + auto new_decoding_schema = getDecodingStorageSchemaSnapshot(new_table_info); + ASSERT_FALSE(decodeAndCheckColumns(new_decoding_schema, false)); + ASSERT_ANY_THROW(decodeAndCheckColumns(new_decoding_schema, true)); + + auto decoding_schema = getDecodingStorageSchemaSnapshot(table_info); + ASSERT_TRUE(decodeAndCheckColumns(decoding_schema, true)); +} + +TEST_F(RegionBlockReaderTest, OverflowColumnRowV1) +{ + auto [table_info, fields] = getNormalTableInfoFields({EXTRA_HANDLE_COLUMN_ID}, false); + encodeColumns(table_info, fields, RowEncodeVersion::RowV1); + auto new_table_info = getTableInfoWithMoreNarrowIntType({EXTRA_HANDLE_COLUMN_ID}, false); + auto new_decoding_schema = getDecodingStorageSchemaSnapshot(new_table_info); + ASSERT_FALSE(decodeAndCheckColumns(new_decoding_schema, false)); + ASSERT_ANY_THROW(decodeAndCheckColumns(new_decoding_schema, true)); + + auto decoding_schema = getDecodingStorageSchemaSnapshot(table_info); + ASSERT_TRUE(decodeAndCheckColumns(decoding_schema, true)); +} + +TEST_F(RegionBlockReaderTest, InvalidNULLRowV2) +try +{ + auto [table_info, fields] = getNormalTableInfoFields({EXTRA_HANDLE_COLUMN_ID}, false); + ASSERT_FALSE(table_info.getColumnInfo(11).hasNotNullFlag()); // col 11 is nullable + + encodeColumns(table_info, fields, RowEncodeVersion::RowV2); + + auto new_table_info = getTableInfoFieldsForInvalidNULLTest({EXTRA_HANDLE_COLUMN_ID}, false); + ASSERT_TRUE(new_table_info.getColumnInfo(11).hasNotNullFlag()); // col 11 is not null + + auto new_decoding_schema = getDecodingStorageSchemaSnapshot(new_table_info); + ASSERT_FALSE(decodeAndCheckColumns(new_decoding_schema, false)); + ASSERT_ANY_THROW(decodeAndCheckColumns(new_decoding_schema, true)); +} +CATCH + +TEST_F(RegionBlockReaderTest, InvalidNULLRowV1) +{ + auto [table_info, fields] = getNormalTableInfoFields({EXTRA_HANDLE_COLUMN_ID}, false); + encodeColumns(table_info, fields, RowEncodeVersion::RowV1); + auto new_table_info = getTableInfoFieldsForInvalidNULLTest({EXTRA_HANDLE_COLUMN_ID}, false); + auto new_decoding_schema = getDecodingStorageSchemaSnapshot(new_table_info); + ASSERT_FALSE(decodeAndCheckColumns(new_decoding_schema, false)); + ASSERT_ANY_THROW(decodeAndCheckColumns(new_decoding_schema, true)); +} + +} // namespace DB::tests diff --git a/dbms/src/TestUtils/TiFlashTestBasic.cpp b/dbms/src/TestUtils/TiFlashTestBasic.cpp index 3879c05b08c..adc615f0685 100644 --- a/dbms/src/TestUtils/TiFlashTestBasic.cpp +++ b/dbms/src/TestUtils/TiFlashTestBasic.cpp @@ -2,6 +2,8 @@ #include #include #include +#include +#include namespace DB::tests { @@ -68,4 +70,18 @@ ::testing::AssertionResult DataTypeCompare(const char * lhs_expr, const char * r return ::testing::internal::EqFailure(lhs_expr, rhs_expr, lhs->getName(), rhs->getName(), false); } +<<<<<<< HEAD +======= +::testing::AssertionResult fieldCompare( + const char * lhs_expr, + const char * rhs_expr, + const Field & lhs, + const Field & rhs) +{ + if (lhs == rhs) + return ::testing::AssertionSuccess(); + return ::testing::internal::EqFailure(lhs_expr, rhs_expr, lhs.toString(), rhs.toString(), false); +} + +>>>>>>> aae88b120d (tests: Fix RegionBlockReaderTest helper functions (#5899)) } // namespace DB::tests diff --git a/dbms/src/TestUtils/TiFlashTestBasic.h b/dbms/src/TestUtils/TiFlashTestBasic.h index 1f2e16075dd..596f94d9fac 100644 --- a/dbms/src/TestUtils/TiFlashTestBasic.h +++ b/dbms/src/TestUtils/TiFlashTestBasic.h @@ -55,6 +55,15 @@ ::testing::AssertionResult DataTypeCompare( // #define ASSERT_DATATYPE_EQ(val1, val2) ASSERT_PRED_FORMAT2(::DB::tests::DataTypeCompare, val1, val2) #define EXPECT_DATATYPE_EQ(val1, val2) EXPECT_PRED_FORMAT2(::DB::tests::DataTypeCompare, val1, val2) +::testing::AssertionResult fieldCompare( + const char * lhs_expr, + const char * rhs_expr, + const Field & lhs, + const Field & rhs); + +#define ASSERT_FIELD_EQ(val1, val2) ASSERT_PRED_FORMAT2(::DB::tests::fieldCompare, val1, val2) +#define EXPECT_FIELD_EQ(val1, val2) EXPECT_PRED_FORMAT2(::DB::tests::fieldCompare, val1, val2) + // A simple helper for getting DataType from type name inline DataTypePtr typeFromString(const String & str) {