Skip to content

Commit

Permalink
This is an automated cherry-pick of pingcap#5166
Browse files Browse the repository at this point in the history
Signed-off-by: ti-chi-bot <ti-community-prow-bot@tidb.io>
  • Loading branch information
hongyunyan authored and ti-chi-bot committed Jun 22, 2022
1 parent 25545c4 commit f1415ef
Show file tree
Hide file tree
Showing 16 changed files with 480 additions and 77 deletions.
14 changes: 11 additions & 3 deletions dbms/src/Core/Block.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -238,10 +238,18 @@ void Block::checkNumberOfRows() const
if (rows == -1)
rows = size;
else if (rows != size)
throw Exception("Sizes of columns doesn't match: "
+ data.front().name + ": " + toString(rows)
+ ", " + elem.name + ": " + toString(size),
{
auto first_col = data.front();
throw Exception(fmt::format(
"Sizes of columns doesn't match: {}(id={}): {}, {}(id={}): {}",
first_col.name,
first_col.column_id,
rows,
elem.name,
elem.column_id,
size),
ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
}
}
}

Expand Down
5 changes: 2 additions & 3 deletions dbms/src/Debug/MockTiDB.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -221,7 +221,6 @@ TiDB::TableInfoPtr MockTiDB::parseColumns(
{
String & name = string_tokens[index];
index_info.idx_cols[index].name = name;
index_info.idx_cols[index].offset = pk_column_pos_map[name];
index_info.idx_cols[index].length = -1;
}
}
Expand Down Expand Up @@ -302,7 +301,7 @@ int MockTiDB::newTables(
tables_by_id.emplace(table->table_info.id, table);
tables_by_name.emplace(qualified_name, table);

AffectedOption opt;
AffectedOption opt{};
opt.schema_id = table->database_id;
opt.table_id = table->id();
opt.old_schema_id = table->database_id;
Expand Down Expand Up @@ -571,7 +570,7 @@ void MockTiDB::renameTables(const std::vector<std::tuple<std::string, std::strin
tables_by_name.erase(qualified_name);
tables_by_name.emplace(new_qualified_name, new_table);

AffectedOption opt;
AffectedOption opt{};
opt.schema_id = table->database_id;
opt.table_id = new_table->id();
opt.old_schema_id = table->database_id;
Expand Down
42 changes: 25 additions & 17 deletions dbms/src/Debug/dbgFuncMockRaftCommand.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,7 @@ void MockRaftCommand::dbgFuncRegionBatchSplit(Context & context, const ASTs & ar
auto & tmt = context.getTMTContext();
auto & kvstore = tmt.getKVStore();

RegionID region_id = (RegionID)safeGet<UInt64>(typeid_cast<const ASTLiteral &>(*args[0]).value);
auto region_id = static_cast<RegionID>(safeGet<UInt64>(typeid_cast<const ASTLiteral &>(*args[0]).value));
const String & database_name = typeid_cast<const ASTIdentifier &>(*args[1]).name;
const String & table_name = typeid_cast<const ASTIdentifier &>(*args[2]).name;
auto table = MockTiDB::instance().getTableByName(database_name, table_name);
Expand All @@ -49,7 +49,7 @@ void MockRaftCommand::dbgFuncRegionBatchSplit(Context & context, const ASTs & ar
if (4 + handle_column_size * 4 != args.size())
throw Exception("Args not matched, should be: region-id1, database-name, table-name, start1, end1, start2, end2, region-id2",
ErrorCodes::BAD_ARGUMENTS);
RegionID region_id2 = (RegionID)safeGet<UInt64>(typeid_cast<const ASTLiteral &>(*args[args.size() - 1]).value);
auto region_id2 = static_cast<RegionID>(safeGet<UInt64>(typeid_cast<const ASTLiteral &>(*args[args.size() - 1]).value));

auto table_id = table->id();
TiKVKey start_key1, start_key2, end_key1, end_key2;
Expand All @@ -59,9 +59,17 @@ void MockRaftCommand::dbgFuncRegionBatchSplit(Context & context, const ASTs & ar
std::vector<Field> start_keys2;
std::vector<Field> end_keys1;
std::vector<Field> end_keys2;

std::unordered_map<String, size_t> column_name_columns_index_map;
for (size_t i = 0; i < table_info.columns.size(); i++)
{
column_name_columns_index_map.emplace(table_info.columns[i].name, i);
}

for (size_t i = 0; i < handle_column_size; i++)
{
auto & column_info = table_info.columns[table_info.getPrimaryIndexInfo().idx_cols[i].offset];
auto idx = column_name_columns_index_map[table_info.getPrimaryIndexInfo().idx_cols[i].name];
auto & column_info = table_info.columns[idx];

auto start_field1 = RegionBench::convertField(column_info, typeid_cast<const ASTLiteral &>(*args[3 + i]).value);
TiDB::DatumBumpy start_datum1 = TiDB::DatumBumpy(start_field1, column_info.tp);
Expand All @@ -88,10 +96,10 @@ void MockRaftCommand::dbgFuncRegionBatchSplit(Context & context, const ASTs & ar
}
else
{
HandleID start1 = (HandleID)safeGet<UInt64>(typeid_cast<const ASTLiteral &>(*args[3]).value);
HandleID end1 = (HandleID)safeGet<UInt64>(typeid_cast<const ASTLiteral &>(*args[4]).value);
HandleID start2 = (HandleID)safeGet<UInt64>(typeid_cast<const ASTLiteral &>(*args[5]).value);
HandleID end2 = (HandleID)safeGet<UInt64>(typeid_cast<const ASTLiteral &>(*args[6]).value);
auto start1 = static_cast<HandleID>(safeGet<UInt64>(typeid_cast<const ASTLiteral &>(*args[3]).value));
auto end1 = static_cast<HandleID>(safeGet<UInt64>(typeid_cast<const ASTLiteral &>(*args[4]).value));
auto start2 = static_cast<HandleID>(safeGet<UInt64>(typeid_cast<const ASTLiteral &>(*args[5]).value));
auto end2 = static_cast<HandleID>(safeGet<UInt64>(typeid_cast<const ASTLiteral &>(*args[6]).value));
start_key1 = RecordKVFormat::genKey(table_id, start1);
start_key2 = RecordKVFormat::genKey(table_id, start2);
end_key1 = RecordKVFormat::genKey(table_id, end1);
Expand All @@ -110,15 +118,15 @@ void MockRaftCommand::dbgFuncRegionBatchSplit(Context & context, const ASTs & ar
request.set_cmd_type(raft_cmdpb::AdminCmdType::BatchSplit);
raft_cmdpb::BatchSplitResponse * splits = response.mutable_splits();
{
auto region = splits->add_regions();
auto * region = splits->add_regions();
region->set_id(region_id);
region->set_start_key(start_key1);
region->set_end_key(end_key1);
region->add_peers();
*region->mutable_region_epoch() = new_epoch;
}
{
auto region = splits->add_regions();
auto * region = splits->add_regions();
region->set_id(region_id2);
region->set_start_key(start_key2);
region->set_end_key(end_key2);
Expand All @@ -144,8 +152,8 @@ void MockRaftCommand::dbgFuncPrepareMerge(Context & context, const ASTs & args,
throw Exception("Args not matched, should be: source-id1, target-id2", ErrorCodes::BAD_ARGUMENTS);
}

RegionID region_id = (RegionID)safeGet<UInt64>(typeid_cast<const ASTLiteral &>(*args[0]).value);
RegionID target_id = (RegionID)safeGet<UInt64>(typeid_cast<const ASTLiteral &>(*args[1]).value);
auto region_id = static_cast<RegionID>(safeGet<UInt64>(typeid_cast<const ASTLiteral &>(*args[0]).value));
auto target_id = static_cast<RegionID>(safeGet<UInt64>(typeid_cast<const ASTLiteral &>(*args[1]).value));

auto & tmt = context.getTMTContext();
auto & kvstore = tmt.getKVStore();
Expand All @@ -157,7 +165,7 @@ void MockRaftCommand::dbgFuncPrepareMerge(Context & context, const ASTs & args,
{
request.set_cmd_type(raft_cmdpb::AdminCmdType::PrepareMerge);

auto prepare_merge = request.mutable_prepare_merge();
auto * prepare_merge = request.mutable_prepare_merge();
{
auto min_index = region->appliedIndex();
prepare_merge->set_min_index(min_index);
Expand All @@ -184,8 +192,8 @@ void MockRaftCommand::dbgFuncCommitMerge(Context & context, const ASTs & args, D
throw Exception("Args not matched, should be: source-id1, current-id2", ErrorCodes::BAD_ARGUMENTS);
}

RegionID source_id = (RegionID)safeGet<UInt64>(typeid_cast<const ASTLiteral &>(*args[0]).value);
RegionID current_id = (RegionID)safeGet<UInt64>(typeid_cast<const ASTLiteral &>(*args[1]).value);
auto source_id = static_cast<RegionID>(safeGet<UInt64>(typeid_cast<const ASTLiteral &>(*args[0]).value));
auto current_id = static_cast<RegionID>(safeGet<UInt64>(typeid_cast<const ASTLiteral &>(*args[1]).value));

auto & tmt = context.getTMTContext();
auto & kvstore = tmt.getKVStore();
Expand All @@ -196,7 +204,7 @@ void MockRaftCommand::dbgFuncCommitMerge(Context & context, const ASTs & args, D

{
request.set_cmd_type(raft_cmdpb::AdminCmdType::CommitMerge);
auto commit_merge = request.mutable_commit_merge();
auto * commit_merge = request.mutable_commit_merge();
{
commit_merge->set_commit(source_region->appliedIndex());
*commit_merge->mutable_source() = source_region->getMetaRegion();
Expand All @@ -220,7 +228,7 @@ void MockRaftCommand::dbgFuncRollbackMerge(Context & context, const ASTs & args,
throw Exception("Args not matched, should be: region-id", ErrorCodes::BAD_ARGUMENTS);
}

RegionID region_id = (RegionID)safeGet<UInt64>(typeid_cast<const ASTLiteral &>(*args[0]).value);
auto region_id = static_cast<RegionID>(safeGet<UInt64>(typeid_cast<const ASTLiteral &>(*args[0]).value));

auto & tmt = context.getTMTContext();
auto & kvstore = tmt.getKVStore();
Expand All @@ -231,7 +239,7 @@ void MockRaftCommand::dbgFuncRollbackMerge(Context & context, const ASTs & args,
{
request.set_cmd_type(raft_cmdpb::AdminCmdType::RollbackMerge);

auto rollback_merge = request.mutable_rollback_merge();
auto * rollback_merge = request.mutable_rollback_merge();
{
auto merge_state = region->getMergeState();
rollback_merge->set_commit(merge_state.commit());
Expand Down
24 changes: 19 additions & 5 deletions dbms/src/Debug/dbgFuncMockRaftSnapshot.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -68,6 +68,12 @@ RegionPtr GenDbgRegionSnapshotWithData(Context & context, const ASTs & args)
size_t handle_column_size = is_common_handle ? table_info.getPrimaryIndexInfo().idx_cols.size() : 1;
RegionPtr region;

std::unordered_map<String, size_t> column_name_columns_index_map;
for (size_t i = 0; i < table_info.columns.size(); i++)
{
column_name_columns_index_map.emplace(table_info.columns[i].name, i);
}

if (!is_common_handle)
{
HandleID start = static_cast<HandleID>(safeGet<UInt64>(typeid_cast<const ASTLiteral &>(*args[3]).value));
Expand All @@ -81,7 +87,8 @@ RegionPtr GenDbgRegionSnapshotWithData(Context & context, const ASTs & args)
std::vector<Field> end_keys;
for (size_t i = 0; i < handle_column_size; i++)
{
auto & column_info = table_info.columns[table_info.getPrimaryIndexInfo().idx_cols[i].offset];
auto idx = column_name_columns_index_map[table_info.getPrimaryIndexInfo().idx_cols[i].name];
auto & column_info = table_info.columns[idx];
auto start_field = RegionBench::convertField(column_info, typeid_cast<const ASTLiteral &>(*args[3 + i]).value);
TiDB::DatumBumpy start_datum = TiDB::DatumBumpy(start_field, column_info.tp);
start_keys.emplace_back(start_datum.field());
Expand Down Expand Up @@ -122,9 +129,9 @@ RegionPtr GenDbgRegionSnapshotWithData(Context & context, const ASTs & args)
std::vector<Field> keys; // handle key
for (size_t i = 0; i < table_info.getPrimaryIndexInfo().idx_cols.size(); i++)
{
auto & idx_col = table_info.getPrimaryIndexInfo().idx_cols[i];
auto & column_info = table_info.columns[idx_col.offset];
auto start_field = RegionBench::convertField(column_info, fields[idx_col.offset]);
auto idx = column_name_columns_index_map[table_info.getPrimaryIndexInfo().idx_cols[i].name];
auto & column_info = table_info.columns[idx];
auto start_field = RegionBench::convertField(column_info, fields[idx]);
TiDB::DatumBumpy start_datum = TiDB::DatumBumpy(start_field, column_info.tp);
keys.emplace_back(start_datum.field());
}
Expand Down Expand Up @@ -198,9 +205,16 @@ void MockRaftCommand::dbgFuncRegionSnapshot(Context & context, const ASTs & args
// Get start key and end key form multiple column if it is clustered_index.
std::vector<Field> start_keys;
std::vector<Field> end_keys;

std::unordered_map<String, size_t> column_name_columns_index_map;
for (size_t i = 0; i < table_info.columns.size(); i++)
{
column_name_columns_index_map.emplace(table_info.columns[i].name, i);
}
for (size_t i = 0; i < handle_column_size; i++)
{
const auto & column_info = table_info.columns[table_info.getPrimaryIndexInfo().idx_cols[i].offset];
auto idx = column_name_columns_index_map[table_info.getPrimaryIndexInfo().idx_cols[i].name];
const auto & column_info = table_info.columns[idx];
auto start_field = RegionBench::convertField(column_info, typeid_cast<const ASTLiteral &>(*args[1 + i]).value);
TiDB::DatumBumpy start_datum = TiDB::DatumBumpy(start_field, column_info.tp);
start_keys.emplace_back(start_datum.field());
Expand Down
8 changes: 7 additions & 1 deletion dbms/src/Debug/dbgFuncRegion.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -61,9 +61,15 @@ void dbgFuncPutRegion(Context & context, const ASTs & args, DBGInvoker::Printer
{
std::vector<Field> start_keys;
std::vector<Field> end_keys;
std::unordered_map<String, size_t> column_name_columns_index_map;
for (size_t i = 0; i < table_info.columns.size(); i++)
{
column_name_columns_index_map.emplace(table_info.columns[i].name, i);
}
for (size_t i = 0; i < handle_column_size; i++)
{
const auto & column_info = table_info.columns[table_info.getPrimaryIndexInfo().idx_cols[i].offset];
auto idx = column_name_columns_index_map[table_info.getPrimaryIndexInfo().idx_cols[i].name];
const auto & column_info = table_info.columns[idx];
auto start_field = RegionBench::convertField(column_info, typeid_cast<const ASTLiteral &>(*args[1 + i]).value);
TiDB::DatumBumpy start_datum = TiDB::DatumBumpy(start_field, column_info.tp);
start_keys.emplace_back(start_datum.field());
Expand Down
15 changes: 11 additions & 4 deletions dbms/src/Debug/dbgTools.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -310,7 +310,7 @@ void insert( //
// Parse the fields in the inserted row
std::vector<Field> fields;
{
for (ASTs::const_iterator it = values_begin; it != values_end; ++it)
for (auto it = values_begin; it != values_end; ++it)
{
auto field = typeid_cast<const ASTLiteral *>((*it).get())->value;
fields.emplace_back(field);
Expand All @@ -330,11 +330,18 @@ void insert( //
if (table_info.is_common_handle)
{
std::vector<Field> keys;

std::unordered_map<String, size_t> column_name_columns_index_map;
for (size_t i = 0; i < table_info.columns.size(); i++)
{
column_name_columns_index_map.emplace(table_info.columns[i].name, i);
}

for (size_t i = 0; i < table_info.getPrimaryIndexInfo().idx_cols.size(); i++)
{
const auto & idx_col = table_info.getPrimaryIndexInfo().idx_cols[i];
const auto & column_info = table_info.columns[idx_col.offset];
auto start_field = RegionBench::convertField(column_info, fields[idx_col.offset]);
const auto & col_idx = column_name_columns_index_map[table_info.getPrimaryIndexInfo().idx_cols[i].name];
const auto & column_info = table_info.columns[col_idx];
auto start_field = RegionBench::convertField(column_info, fields[col_idx]);
TiDB::DatumBumpy start_datum = TiDB::DatumBumpy(start_field, column_info.tp);
keys.emplace_back(start_datum.field());
}
Expand Down
14 changes: 10 additions & 4 deletions dbms/src/Storages/Transaction/DecodingStorageSchemaSnapshot.h
Original file line number Diff line number Diff line change
Expand Up @@ -75,18 +75,20 @@ struct DecodingStorageSchemaSnapshot
, schema_version{table_info_.schema_version}
{
std::unordered_map<ColumnID, size_t> column_lut;
std::unordered_map<String, ColumnID> column_name_id_map;
for (size_t i = 0; i < table_info_.columns.size(); i++)
{
const auto & ci = table_info_.columns[i];
column_lut.emplace(ci.id, i);
column_name_id_map.emplace(ci.name, ci.id);
}
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)
{
auto & columns = table_info_.columns;
const auto & columns = table_info_.columns;
column_infos.push_back(columns[column_lut.at(cd.id)]);
}
else
Expand All @@ -98,10 +100,14 @@ struct DecodingStorageSchemaSnapshot
// create pk related metadata if needed
if (is_common_handle)
{
const auto & primary_index_info = table_info_.getPrimaryIndexInfo();
for (size_t i = 0; i < primary_index_info.idx_cols.size(); i++)
/// we will not update the IndexInfo except Rename DDL.
/// When the add column / drop column action happenes, the offset of each column may change
/// Thus, we should not use offset to get the column we want,
/// but use to compare the column name to get the column id.
const auto & primary_index_cols = table_info_.getPrimaryIndexInfo().idx_cols;
for (const auto & col : primary_index_cols)
{
auto pk_column_id = table_info_.columns[primary_index_info.idx_cols[i].offset].id;
auto pk_column_id = column_name_id_map[col.name];
pk_column_ids.emplace_back(pk_column_id);
pk_pos_map.emplace(pk_column_id, reinterpret_cast<size_t>(std::numeric_limits<size_t>::max()));
}
Expand Down
2 changes: 2 additions & 0 deletions dbms/src/Storages/Transaction/RegionBlockReader.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -226,6 +226,8 @@ bool RegionBlockReader::readImpl(Block & block, const RegionDataReadInfoList & d
}
index++;
}
block.checkNumberOfRows();

return true;
}

Expand Down
5 changes: 5 additions & 0 deletions dbms/src/Storages/Transaction/TiDB.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -612,6 +612,11 @@ catch (const Poco::Exception & e)
///////////////////////

IndexColumnInfo::IndexColumnInfo(Poco::JSON::Object::Ptr json)
<<<<<<< HEAD
=======
: length(0)
, offset(0)
>>>>>>> 18325f9eb4 (DDL: Use Column Name Instead of Offset to Find the common handle cluster index (#5166))
{
deserialize(json);
}
Expand Down
21 changes: 18 additions & 3 deletions dbms/src/Storages/Transaction/TiDB.h
Original file line number Diff line number Diff line change
Expand Up @@ -179,7 +179,6 @@ struct ColumnInfo

ColumnID id = -1;
String name;
Int32 offset = -1;
Poco::Dynamic::Var origin_default_value;
Poco::Dynamic::Var default_value;
Poco::Dynamic::Var default_bit_value;
Expand Down Expand Up @@ -212,6 +211,12 @@ struct ColumnInfo
static Int64 getTimeValue(const String &);
static Int64 getYearValue(const String &);
static UInt64 getBitValue(const String &);

private:
/// please be very careful when you have to use offset,
/// because we never update offset when DDL action changes.
/// Thus, our offset will not exactly correspond the order of columns.
Int32 offset = -1;
};

enum PartitionType
Expand Down Expand Up @@ -298,8 +303,13 @@ struct IndexColumnInfo
void deserialize(Poco::JSON::Object::Ptr json);

String name;
Int32 offset;
Int32 length;

private:
/// please be very careful when you have to use offset,
/// because we never update offset when DDL action changes.
/// Thus, our offset will not exactly correspond the order of columns.
Int32 offset;
};
struct IndexInfo
{
Expand Down Expand Up @@ -385,7 +395,12 @@ struct TableInfo

bool isLogicalPartitionTable() const { return is_partition_table && belonging_table_id == DB::InvalidTableID && partition.enable; }

/// should not be called if is_common_handle = false
/// should not be called if is_common_handle = false.
/// when use IndexInfo, please avoid to use the offset info
/// the offset value may be wrong in some cases,
/// due to we will not update IndexInfo except RENAME DDL action,
/// but DDL like add column / drop column may change the offset of columns
/// Thus, please be very careful when you must have to use offset information !!!!!
const IndexInfo & getPrimaryIndexInfo() const { return index_infos[0]; }

IndexInfo & getPrimaryIndexInfo() { return index_infos[0]; }
Expand Down
Loading

0 comments on commit f1415ef

Please sign in to comment.