diff --git a/dbms/src/Storages/Transaction/RowCodec.cpp b/dbms/src/Storages/Transaction/RowCodec.cpp index 8e5cfa34f2e..47eb0864c78 100644 --- a/dbms/src/Storages/Transaction/RowCodec.cpp +++ b/dbms/src/Storages/Transaction/RowCodec.cpp @@ -180,10 +180,18 @@ 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]; 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 continue; + } + if (column_info.id > std::numeric_limits::ColumnIDType>::max()) is_big = true; if (!field.isNull()) @@ -199,9 +207,6 @@ struct RowEncoderV2 null_column_ids.emplace(column_info.id); } i_val++; - - if (i_val == fields.size()) - break; } is_big = is_big || value_length > std::numeric_limits::ValueOffsetType>::max(); @@ -300,7 +305,7 @@ bool appendRowV2ToBlock( ColumnID pk_handle_id, bool force_decode) { - UInt8 row_flag = readLittleEndian(&raw_value[1]); + 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); @@ -346,9 +351,10 @@ bool appendRowV2ToBlockImpl( 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 id_not_null = 0, id_null = 0; + size_t idx_not_null = 0; + size_t idx_null = 0; // Merge ordered not null/null columns to keep order. - while (id_not_null < not_null_column_ids.size() || id_null < null_column_ids.size()) + while (idx_not_null < not_null_column_ids.size() || idx_null < null_column_ids.size()) { if (column_ids_iter == column_ids_iter_end) { @@ -357,24 +363,32 @@ bool appendRowV2ToBlockImpl( } bool is_null; - if (id_not_null < not_null_column_ids.size() && id_null < null_column_ids.size()) - is_null = not_null_column_ids[id_not_null] > null_column_ids[id_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 = id_null < null_column_ids.size(); + is_null = idx_null < null_column_ids.size(); - auto next_datum_column_id = is_null ? null_column_ids[id_null] : not_null_column_ids[id_not_null]; - if (column_ids_iter->first > next_datum_column_id) + 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) { - // extra column + // 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) - id_null++; + idx_null++; else - id_not_null++; + idx_not_null++; } - else if (column_ids_iter->first < next_datum_column_id) + 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; @@ -383,23 +397,24 @@ bool appendRowV2ToBlockImpl( } else { - // if pk_handle_id is a valid column id, then it means the table's pk_is_handle is true + // 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)) + if (unlikely(next_column_id == pk_handle_id)) { column_ids_iter++; block_column_pos++; if (is_null) { - id_null++; + idx_null++; } else { - id_not_null++; + 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) @@ -418,15 +433,15 @@ bool appendRowV2ToBlockImpl( } // ColumnNullable::insertDefault just insert a null value raw_column->insertDefault(); - id_null++; + idx_null++; } else { - size_t start = id_not_null ? value_offsets[id_not_null - 1] : 0; - size_t length = value_offsets[id_not_null] - start; + 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; - id_not_null++; + idx_not_null++; } column_ids_iter++; block_column_pos++; diff --git a/dbms/src/Storages/Transaction/tests/RowCodecTestUtils.h b/dbms/src/Storages/Transaction/tests/RowCodecTestUtils.h index 5d1450ab566..ebd7ec15bd4 100644 --- a/dbms/src/Storages/Transaction/tests/RowCodecTestUtils.h +++ b/dbms/src/Storages/Transaction/tests/RowCodecTestUtils.h @@ -1,7 +1,9 @@ #pragma once #include #include +#include #include +#include namespace DB::tests { @@ -132,7 +134,7 @@ struct ColumnIDValue { static constexpr bool value_is_null = true; using ValueType = std::decay_t; - ColumnIDValue(ColumnID id_) + explicit ColumnIDValue(ColumnID id_) : id(id_) {} ColumnID id; @@ -197,46 +199,55 @@ void getTableInfoFieldsInternal(OrderedColumnInfoFields & column_info_fields, Ty } template -std::pair> getTableInfoAndFields(ColumnIDs handle_ids, bool is_common_handle, Types &&... column_value_ids) +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(handle_ids.begin(), handle_ids.end(), column.id) != handle_ids.end()) + 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)); } - if (!is_common_handle) - { - if (handle_ids[0] != EXTRA_HANDLE_COLUMN_ID) - table_info.pk_is_handle = true; - } - else + + 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; - TiDB::IndexInfo index_info; - for (auto handle_id : handle_ids) + // 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 == handle_id) + if (column.id == pk_col_id) { index_column_info.name = column.name; break; } } - index_info.idx_cols.emplace_back(index_column_info); + pk_index_info.idx_cols.emplace_back(index_column_info); } - table_info.index_infos.emplace_back(index_info); + table_info.index_infos.emplace_back(pk_index_info); } return std::make_pair(std::move(table_info), std::move(fields)); @@ -258,7 +269,7 @@ inline DecodingStorageSchemaSnapshotConstPtr getDecodingStorageSchemaSnapshot(co 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 (auto & column_info : table_info.columns) + for (const auto & column_info : table_info.columns) { if (table_info.pk_is_handle) { @@ -287,7 +298,7 @@ size_t valueStartPos(const TableInfo & table_info) inline Block decodeRowToBlock(const String & row_value, DecodingStorageSchemaSnapshotConstPtr decoding_schema) { - auto & sorted_column_id_with_pos = decoding_schema->sorted_column_id_with_pos; + 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 @@ -333,4 +344,4 @@ T getValueByRowV1(const T & v) return static_cast(std::move((*block.getByPosition(0).column)[0].template safeGet())); } -} // namespace DB::tests \ No newline at end of file +} // 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 index 6314142d500..83a589b029a 100644 --- a/dbms/src/Storages/Transaction/tests/gtest_region_block_reader.cpp +++ b/dbms/src/Storages/Transaction/tests/gtest_region_block_reader.cpp @@ -1,16 +1,39 @@ +// 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 "RowCodecTestUtils.h" +#include +#include +#include +#include using TableInfo = TiDB::TableInfo; namespace DB::tests { using ColumnIDs = std::vector; -class RegionBlockReaderTestFixture : public ::testing::Test +class RegionBlockReaderTest : public ::testing::Test { +public: + RegionBlockReaderTest() + : logger(&Poco::Logger::get("RegionBlockReaderTest")) + {} + protected: Int64 handle_value = 100; UInt8 del_mark_value = 0; @@ -20,6 +43,8 @@ class RegionBlockReaderTestFixture : public ::testing::Test RegionDataReadInfoList data_list_read; std::unordered_map fields_map; + Poco::Logger * logger; + enum RowEncodeVersion { RowV1, @@ -35,7 +60,7 @@ class RegionBlockReaderTestFixture : public ::testing::Test void TearDown() override {} - void encodeColumns(TableInfo & table_info, std::vector & fields, RowEncodeVersion row_version) + void encodeColumns(const TableInfo & table_info, const std::vector & fields, RowEncodeVersion row_version) { // for later check std::unordered_map column_name_columns_index_map; @@ -45,17 +70,24 @@ class RegionBlockReaderTestFixture : public ::testing::Test column_name_columns_index_map.emplace(table_info.columns[i].name, i); } - std::vector value_fields; - std::vector pk_fields; + std::vector value_encode_fields; + std::vector key_encode_fields; for (size_t i = 0; i < table_info.columns.size(); i++) { - if (!table_info.columns[i].hasPriKeyFlag()) - value_fields.emplace_back(fields[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 - pk_fields.emplace_back(fields[i]); + { + value_encode_fields.emplace_back(fields[i]); + } } - // create PK + // create the RawTiDBPK section of encoded key WriteBufferFromOwnString pk_buf; if (table_info.is_common_handle) { @@ -63,7 +95,7 @@ class RegionBlockReaderTestFixture : public ::testing::Test 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(pk_fields[i], table_info.columns[idx].getCodecFlag(), pk_buf); + EncodeDatum(key_encode_fields[i], table_info.columns[idx].getCodecFlag(), pk_buf); } } else @@ -71,21 +103,22 @@ class RegionBlockReaderTestFixture : public ::testing::Test DB::EncodeInt64(handle_value, pk_buf); } RawTiDBPK pk{std::make_shared(pk_buf.releaseStr())}; - // create value + + // create encoded value WriteBufferFromOwnString value_buf; if (row_version == RowEncodeVersion::RowV1) { - encodeRowV1(table_info, value_fields, value_buf); + encodeRowV1(table_info, value_encode_fields, value_buf); } else if (row_version == RowEncodeVersion::RowV2) { - encodeRowV2(table_info, value_fields, value_buf); + 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(std::move(value_buf.str())); + 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); } @@ -98,6 +131,14 @@ class RegionBlockReaderTestFixture : public ::testing::Test 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); @@ -106,24 +147,24 @@ class RegionBlockReaderTestFixture : public ::testing::Test { if (decoding_schema->is_common_handle) { - ASSERT_EQ((*column_element.column)[row], Field(*std::get<0>(data_list_read[row]))); + ASSERT_FIELD_EQ((*column_element.column)[row], Field(*std::get<0>(data_list_read[row]))) << gen_error_log(); } else { - ASSERT_EQ((*column_element.column)[row], Field(handle_value)); + ASSERT_FIELD_EQ((*column_element.column)[row], Field(handle_value)) << gen_error_log(); } } else if (column_element.name == VERSION_COLUMN_NAME) { - ASSERT_EQ((*column_element.column)[row], Field(version_value)); + ASSERT_FIELD_EQ((*column_element.column)[row], Field(version_value)) << gen_error_log(); } else if (column_element.name == TAG_COLUMN_NAME) { - ASSERT_EQ((*column_element.column)[row], Field(NearestFieldType::Type(del_mark_value))); + ASSERT_FIELD_EQ((*column_element.column)[row], Field(NearestFieldType::Type(del_mark_value))) << gen_error_log(); } else { - ASSERT_EQ((*column_element.column)[row], fields_map.at(column_element.column_id)); + ASSERT_FIELD_EQ((*column_element.column)[row], fields_map.at(column_element.column_id)) << gen_error_log(); } } } @@ -140,10 +181,10 @@ class RegionBlockReaderTestFixture : public ::testing::Test return true; } - std::pair> getNormalTableInfoFields(const ColumnIDs & handle_ids, bool is_common_handle) const + std::pair> getNormalTableInfoFields(const ColumnIDs & pk_col_ids, bool is_common_handle) const { return getTableInfoAndFields( - handle_ids, + pk_col_ids, is_common_handle, ColumnIDValue(2, handle_value), ColumnIDValue(3, std::numeric_limits::max()), @@ -227,31 +268,45 @@ class RegionBlockReaderTestFixture : public ::testing::Test } }; -TEST_F(RegionBlockReaderTestFixture, PKIsNotHandle) +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(RegionBlockReaderTestFixture, PKIsHandle) +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(RegionBlockReaderTestFixture, CommonHandle) +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(RegionBlockReaderTestFixture, MissingColumnRowV2) +TEST_F(RegionBlockReaderTest, MissingColumnRowV2) { auto [table_info, fields] = getNormalTableInfoFields({EXTRA_HANDLE_COLUMN_ID}, false); encodeColumns(table_info, fields, RowEncodeVersion::RowV2); @@ -260,7 +315,7 @@ TEST_F(RegionBlockReaderTestFixture, MissingColumnRowV2) ASSERT_TRUE(decodeAndCheckColumns(new_decoding_schema, false)); } -TEST_F(RegionBlockReaderTestFixture, MissingColumnRowV1) +TEST_F(RegionBlockReaderTest, MissingColumnRowV1) { auto [table_info, fields] = getNormalTableInfoFields({EXTRA_HANDLE_COLUMN_ID}, false); encodeColumns(table_info, fields, RowEncodeVersion::RowV1); @@ -269,7 +324,7 @@ TEST_F(RegionBlockReaderTestFixture, MissingColumnRowV1) ASSERT_TRUE(decodeAndCheckColumns(new_decoding_schema, false)); } -TEST_F(RegionBlockReaderTestFixture, ExtraColumnRowV2) +TEST_F(RegionBlockReaderTest, ExtraColumnRowV2) { auto [table_info, fields] = getNormalTableInfoFields({EXTRA_HANDLE_COLUMN_ID}, false); encodeColumns(table_info, fields, RowEncodeVersion::RowV2); @@ -279,7 +334,7 @@ TEST_F(RegionBlockReaderTestFixture, ExtraColumnRowV2) ASSERT_TRUE(decodeAndCheckColumns(new_decoding_schema, true)); } -TEST_F(RegionBlockReaderTestFixture, ExtraColumnRowV1) +TEST_F(RegionBlockReaderTest, ExtraColumnRowV1) { auto [table_info, fields] = getNormalTableInfoFields({EXTRA_HANDLE_COLUMN_ID}, false); encodeColumns(table_info, fields, RowEncodeVersion::RowV1); @@ -289,7 +344,7 @@ TEST_F(RegionBlockReaderTestFixture, ExtraColumnRowV1) ASSERT_TRUE(decodeAndCheckColumns(new_decoding_schema, true)); } -TEST_F(RegionBlockReaderTestFixture, OverflowColumnRowV2) +TEST_F(RegionBlockReaderTest, OverflowColumnRowV2) { auto [table_info, fields] = getNormalTableInfoFields({EXTRA_HANDLE_COLUMN_ID}, false); encodeColumns(table_info, fields, RowEncodeVersion::RowV2); @@ -302,7 +357,7 @@ TEST_F(RegionBlockReaderTestFixture, OverflowColumnRowV2) ASSERT_TRUE(decodeAndCheckColumns(decoding_schema, true)); } -TEST_F(RegionBlockReaderTestFixture, OverflowColumnRowV1) +TEST_F(RegionBlockReaderTest, OverflowColumnRowV1) { auto [table_info, fields] = getNormalTableInfoFields({EXTRA_HANDLE_COLUMN_ID}, false); encodeColumns(table_info, fields, RowEncodeVersion::RowV1); @@ -315,17 +370,24 @@ TEST_F(RegionBlockReaderTestFixture, OverflowColumnRowV1) ASSERT_TRUE(decodeAndCheckColumns(decoding_schema, true)); } -TEST_F(RegionBlockReaderTestFixture, InvalidNULLRowV2) +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(RegionBlockReaderTestFixture, InvalidNULLRowV1) +TEST_F(RegionBlockReaderTest, InvalidNULLRowV1) { auto [table_info, fields] = getNormalTableInfoFields({EXTRA_HANDLE_COLUMN_ID}, false); encodeColumns(table_info, fields, RowEncodeVersion::RowV1); diff --git a/dbms/src/TestUtils/FunctionTestUtils.cpp b/dbms/src/TestUtils/FunctionTestUtils.cpp index cb11ee0e22f..9ba15f1e2b5 100644 --- a/dbms/src/TestUtils/FunctionTestUtils.cpp +++ b/dbms/src/TestUtils/FunctionTestUtils.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -154,5 +155,58 @@ ColumnWithTypeAndName createDateTimeColumnConst(size_t size, const MyDateTime & return {std::move(col), data_type_ptr, "datetime"}; } +String getColumnsContent(const ColumnsWithTypeAndName & cols) +{ + if (cols.empty()) + return ""; + return getColumnsContent(cols, 0, cols[0].column->size()); +} + +String getColumnsContent(const ColumnsWithTypeAndName & cols, size_t begin, size_t end) +{ + const size_t col_num = cols.size(); + if (col_num <= 0) + return ""; + + const size_t col_size = cols[0].column->size(); + assert(begin <= end); + assert(col_size >= end); + assert(col_size > begin); + + bool is_same = true; + + for (size_t i = 1; i < col_num; ++i) + { + if (cols[i].column->size() != col_size) + is_same = false; + } + + assert(is_same); /// Ensure the sizes of columns in cols are the same + + std::vector> col_content; + FmtBuffer fmt_buf; + for (size_t i = 0; i < col_num; ++i) + { + /// Push the column name + fmt_buf.append(fmt::format("{}: (", cols[i].name)); + for (size_t j = begin; j < end; ++j) + col_content.push_back(std::make_pair(j, (*cols[i].column)[j].toString())); + + /// Add content + fmt_buf.joinStr( + col_content.begin(), + col_content.end(), + [](const auto & content, FmtBuffer & fmt_buf) { + fmt_buf.append(fmt::format("{}: {}", content.first, content.second)); + }, + ", "); + + fmt_buf.append(")\n"); + col_content.clear(); + } + + return fmt_buf.toString(); +} + } // namespace tests } // namespace DB diff --git a/dbms/src/TestUtils/FunctionTestUtils.h b/dbms/src/TestUtils/FunctionTestUtils.h index d0401800413..9217b651bcb 100644 --- a/dbms/src/TestUtils/FunctionTestUtils.h +++ b/dbms/src/TestUtils/FunctionTestUtils.h @@ -370,6 +370,11 @@ ColumnWithTypeAndName createConstColumn( return createConstColumn(data_type_args, size, InferredFieldType(std::nullopt), name); } +String getColumnsContent(const ColumnsWithTypeAndName & cols); + +/// We can designate the range of columns printed with begin and end. range: [begin, end) +String getColumnsContent(const ColumnsWithTypeAndName & cols, size_t begin, size_t end); + ::testing::AssertionResult dataTypeEqual( const DataTypePtr & expected, const DataTypePtr & actual); diff --git a/dbms/src/TestUtils/TiFlashTestBasic.cpp b/dbms/src/TestUtils/TiFlashTestBasic.cpp index 28d63f8350b..d52a3d19519 100644 --- a/dbms/src/TestUtils/TiFlashTestBasic.cpp +++ b/dbms/src/TestUtils/TiFlashTestBasic.cpp @@ -13,4 +13,16 @@ ::testing::AssertionResult DataTypeCompare( else return ::testing::internal::EqFailure(lhs_expr, rhs_expr, lhs->getName(), rhs->getName(), false); } + +::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); +} + } // namespace DB::tests diff --git a/dbms/src/TestUtils/TiFlashTestBasic.h b/dbms/src/TestUtils/TiFlashTestBasic.h index 39380ea6682..de397e6921e 100644 --- a/dbms/src/TestUtils/TiFlashTestBasic.h +++ b/dbms/src/TestUtils/TiFlashTestBasic.h @@ -65,6 +65,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) {