Skip to content

Commit 8e3d547

Browse files
authored
ddl: Fix compatibility with null expression index (#10162)
close #9891 ddl: Fix TiFlash panic when meets expression index with format `((NULL))` Using `ColumnNothing` in IStorage may bring unexpected behavior when we try to write or read the column. So TiFlash create `ColumnInt8` instead of `ColumnNothing` when `TiDB::ColumnInfo.Tp == Null`. However, we don't expected the column is used when reading or writing to TiFlash. * `((null))` is not a normal expression index, this could be created by the user in accident (e.g. executing wrong SQL statement to create the expression index), TiDB should not write or read from it * Even for normal expression index, TiDB does not use expression when accessing to TiFlash Signed-off-by: JaySon-Huang <[email protected]>
1 parent 5c59a51 commit 8e3d547

File tree

5 files changed

+173
-6
lines changed

5 files changed

+173
-6
lines changed

dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_merge_store.cpp

Lines changed: 79 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -18,6 +18,7 @@
1818
#include <Common/SyncPoint/SyncPoint.h>
1919
#include <Core/Defines.h>
2020
#include <DataTypes/DataTypeMyDateTime.h>
21+
#include <DataTypes/DataTypesNumber.h>
2122
#include <Debug/TiFlashTestEnv.h>
2223
#include <Interpreters/Context.h>
2324
#include <Storages/DeltaMerge/DeltaMergeDefines.h>
@@ -2473,6 +2474,84 @@ try
24732474
}
24742475
CATCH
24752476

2477+
TEST_P(DeltaMergeStoreRWTest, DDLAddColumnInvalidExpressionIndex)
2478+
try
2479+
{
2480+
// const String col_name_to_add = "_v$_idx_name_0";
2481+
// const DataTypePtr col_type_to_add = DataTypeFactory::instance().getOrSet("Int8");
2482+
2483+
// write some rows before DDL
2484+
size_t num_rows_write = 1;
2485+
{
2486+
Block block = DMTestEnv::prepareSimpleWriteBlock(0, num_rows_write, false);
2487+
store->write(*db_context, db_context->getSettingsRef(), block);
2488+
}
2489+
2490+
// DDL add column for invalid expression index
2491+
// actual ddl is like: ADD COLUMN `_v$_idx_name_0` Nullable(Int8)
2492+
{
2493+
TiDB::TableInfo new_table_info;
2494+
static const String json_table_info = R"(
2495+
{"cols":[{"id":1,"name":{"L":"id","O":"id"},"offset":0,"state":5,"type":{"Charset":"binary","Collate":"binary","Decimal":0,"Flag":0,"Flen":11,"Tp":3}},{"id":2,"name":{"L":"_v$_idx_name_0","O":"_v$_idx_name_0"},"offset":1,"state":5,"type":{"Charset":"binary","Collate":"binary","Decimal":0,"Flag":136,"Flen":0,"Tp":6}}],"id":1145,"index_info":[],"is_common_handle":false,"keyspace_id":4294967295,"name":{"L":"t","O":"t"},"pk_is_handle":false,"schema_version":-1,"state":5,"tiflash_replica":{"Available":false,"Count":1},"update_timestamp":456163970651521027}
2496+
)";
2497+
new_table_info.deserialize(json_table_info);
2498+
store->applySchemaChanges(new_table_info);
2499+
}
2500+
2501+
// try read
2502+
{
2503+
ColumnDefines cols_to_read;
2504+
bool has_v_index = false;
2505+
bool has_id = false;
2506+
for (const auto & col : store->getTableColumns())
2507+
{
2508+
if (col.name == DMTestEnv::pk_name)
2509+
{
2510+
cols_to_read.emplace_back(col);
2511+
}
2512+
else if (col.name == "id")
2513+
{
2514+
cols_to_read.emplace_back(col);
2515+
has_id = true;
2516+
}
2517+
else if (col.name == "_v$_idx_name_0")
2518+
{
2519+
// we check the type of this column, but not read from it
2520+
ASSERT_EQ(col.type->getName(), "Nullable(Int8)") << store->getHeader()->dumpJsonStructure();
2521+
has_v_index = true;
2522+
}
2523+
}
2524+
ASSERT_TRUE(has_id) << store->getHeader()->dumpJsonStructure();
2525+
ASSERT_TRUE(has_v_index) << store->getHeader()->dumpJsonStructure();
2526+
2527+
auto in = store->read(
2528+
*db_context,
2529+
db_context->getSettingsRef(),
2530+
cols_to_read,
2531+
{RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())},
2532+
/* num_streams= */ 1,
2533+
/* start_ts= */ std::numeric_limits<UInt64>::max(),
2534+
EMPTY_FILTER,
2535+
std::vector<RuntimeFilterPtr>{},
2536+
0,
2537+
TRACING_NAME,
2538+
/* keep_order= */ false,
2539+
/* is_fast_scan= */ false,
2540+
/* expected_block_size= */ 1024)[0];
2541+
ASSERT_UNORDERED_INPUTSTREAM_COLS_UR(
2542+
in,
2543+
Strings({DMTestEnv::pk_name, "id"}),
2544+
createColumns({
2545+
createColumn<Int64>(createNumbers<Int64>(0, num_rows_write)),
2546+
// all "id" are NULL
2547+
createNullableColumn<Int32>(
2548+
std::vector<Int64>(num_rows_write, 0),
2549+
std::vector<Int32>(num_rows_write, 1)),
2550+
}));
2551+
}
2552+
}
2553+
CATCH
2554+
24762555
TEST_P(DeltaMergeStoreRWTest, DDLRenameColumn)
24772556
try
24782557
{

dbms/src/TiDB/Decode/TypeMapping.cpp

Lines changed: 16 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -204,7 +204,22 @@ DataTypePtr TypeMapping::getDataType(const ColumnInfo & column_info)
204204
// This does not support the "duration" type.
205205
DataTypePtr getDataTypeByColumnInfo(const ColumnInfo & column_info)
206206
{
207-
DataTypePtr base = TypeMapping::instance().getDataType(column_info);
207+
DataTypePtr base;
208+
if (likely(column_info.tp != TiDB::TP::TypeNull))
209+
{
210+
base = TypeMapping::instance().getDataType(column_info);
211+
}
212+
else
213+
{
214+
// Storing a column with `ColumnNothing` is not allowed in `StorageFactory::get/checkAllTypesAreAllowedInTable`
215+
// Using `ColumnNothing` in IStorage may bring unexpected behavior when we
216+
// try to write or read the column. So we change it to `ColumnInt8`.
217+
LOG_WARNING(
218+
Logger::get(),
219+
"Column type is TiDB::TP::TypeNull, change it to Int8 for compatibility, column_id={}",
220+
column_info.id);
221+
base = DataTypeFactory::instance().getOrSet("Int8");
222+
}
208223

209224
if (!column_info.hasNotNullFlag())
210225
{

dbms/src/TiDB/Schema/tests/gtest_table_info.cpp

Lines changed: 19 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -27,6 +27,7 @@
2727
#include <TiDB/Decode/TypeMapping.h>
2828
#include <TiDB/Schema/SchemaSyncer.h>
2929
#include <TiDB/Schema/TiDB.h>
30+
#include <TiDB/Schema/TiDBTypes.h>
3031
#include <gtest/gtest.h>
3132

3233
#include <boost/algorithm/string/replace.hpp>
@@ -126,6 +127,16 @@ try
126127
getDataTypeByColumnInfo(ci);
127128
}
128129
}},
130+
// tiflash#9891, expression index like `KEY idx_name ((null))` generate column in this format
131+
ParseCase{
132+
R"json({"cols":[{"id":1,"name":{"L":"id","O":"id"},"offset":0,"state":5,"type":{"Charset":"binary","Collate":"binary","Decimal":0,"Flag":0,"Flen":11,"Tp":3}},{"id":2,"name":{"L":"_v$_idx_name_0","O":"_v$_idx_name_0"},"offset":1,"state":5,"type":{"Charset":"binary","Collate":"binary","Decimal":0,"Flag":136,"Flen":0,"Tp":6}}],"id":242807,"index_info":[],"is_common_handle":false,"keyspace_id":4294967295,"name":{"L":"t","O":"t"},"pk_is_handle":false,"schema_version":-1,"state":5,"tiflash_replica":{"Available":false,"Count":1},"update_timestamp":456163970651521027})json",
133+
[](const TableInfo & table_info) {
134+
ASSERT_EQ(table_info.name, "t");
135+
auto col_1 = table_info.getColumnInfo(2);
136+
ASSERT_EQ(col_1.tp, TiDB::TP::TypeNull);
137+
ASSERT_TRUE(col_1.hasMultipleKeyFlag());
138+
ASSERT_TRUE(col_1.hasBinaryFlag());
139+
}},
129140
};
130141

131142
for (const auto & c : cases)
@@ -368,6 +379,14 @@ try
368379
R"json({"id":546,"name":{"O":"tcfc7825f","L":"tcfc7825f"},"charset":"utf8mb4","collate":"utf8mb4_general_ci","cols":[{"id":1,"name":{"O":"col_86","L":"col_86"},"offset":0,"default_is_expr":false,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":252,"Flag":128,"Flen":65535,"Decimal":0,"Charset":"binary","Collate":"binary","ElemsIsBinaryLit":null,"Array":false},"state":5,"comment":"","hidden":false,"change_state_info":null,"version":2},{"id":2,"name":{"O":"col_87","L":"col_87"},"offset":1,"default":"1994-05-0600:00:00","default_is_expr":false,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":12,"Flag":129,"Flen":19,"Decimal":0,"Charset":"binary","Collate":"binary","ElemsIsBinaryLit":null,"Array":false},"state":5,"comment":"","hidden":false,"change_state_info":null,"version":2},{"id":3,"name":{"O":"col_88","L":"col_88"},"offset":2,"default_is_expr":false,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":16,"Flag":32,"Flen":42,"Decimal":0,"Charset":"binary","Collate":"binary","ElemsIsBinaryLit":null,"Array":false},"state":5,"comment":"","hidden":false,"change_state_info":null,"version":2},{"id":4,"name":{"O":"col_89","L":"col_89"},"offset":3,"default":"\u0000\u0000\u0000\u0000\u0000\u0000\u0000\u0000\u0000\u0000\u0000\u0000\u0000\u0000\u0000\u0000\u0000\u0000\u0000\u0000\u0000","default_is_expr":false,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":254,"Flag":129,"Flen":21,"Decimal":0,"Charset":"binary","Collate":"binary","ElemsIsBinaryLit":null,"Array":false},"state":5,"comment":"","hidden":false,"change_state_info":null,"version":2},{"id":5,"name":{"O":"col_90","L":"col_90"},"offset":4,"default_is_expr":false,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":1,"Flag":4129,"Flen":3,"Decimal":0,"Charset":"binary","Collate":"binary","ElemsIsBinaryLit":null,"Array":false},"state":5,"comment":"","hidden":false,"change_state_info":null,"version":2},{"id":6,"name":{"O":"col_91","L":"col_91"},"offset":5,"default":"\u0007\u0007","default_bit":"Bwc=","default_is_expr":false,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":16,"Flag":32,"Flen":12,"Decimal":0,"Charset":"binary","Collate":"binary","ElemsIsBinaryLit":null,"Array":false},"state":5,"comment":"","hidden":false,"change_state_info":null,"version":2},{"id":7,"name":{"O":"col_92","L":"col_92"},"offset":6,"default":"kY~6to6H4ut*QAPrj@\u0026","default_is_expr":false,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":15,"Flag":129,"Flen":343,"Decimal":0,"Charset":"binary","Collate":"binary","ElemsIsBinaryLit":null,"Array":false},"state":5,"comment":"","hidden":false,"change_state_info":null,"version":2},{"id":8,"name":{"O":"col_93","L":"col_93"},"offset":7,"default_is_expr":false,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":245,"Flag":128,"Flen":4294967295,"Decimal":0,"Charset":"binary","Collate":"binary","ElemsIsBinaryLit":null,"Array":false},"state":5,"comment":"","hidden":false,"change_state_info":null,"version":2}],"index_info":null,"constraint_info":null,"fk_info":null,"state":5,"pk_is_handle":false,"is_common_handle":false,"common_handle_version":0,"comment":"","auto_inc_id":0,"auto_id_cache":0,"auto_rand_id":0,"max_col_id":8,"max_idx_id":0,"max_fk_id":0,"max_cst_id":0,"update_timestamp":452653255976550448,"ShardRowIDBits":0,"max_shard_row_id_bits":0,"auto_random_bits":0,"auto_random_range_bits":0,"pre_split_regions":0,"compression":"","view":null,"sequence":null,"Lock":null,"version":5,"tiflash_replica":{"Count":1,"LocationLabels":[],"Available":false,"AvailablePartitionIDs":null},"is_columnar":false,"temp_table_type":0,"cache_table_status":0,"policy_ref_info":null,"stats_options":null,"exchange_partition_info":null,"ttl_info":null,"revision":1})json", //
369380
replace_string_name(R"stmt(CREATE TABLE `db_2`.`t_546`(`col_86` Nullable({StringName}), `col_87` MyDateTime(0), `col_88` Nullable(UInt64), `col_89` {StringName}, `col_90` UInt8, `col_91` Nullable(UInt64), `col_92` {StringName}, `col_93` Nullable({StringName}), `_tidb_rowid` Int64) Engine = DeltaMerge((`_tidb_rowid`), '{"cols":[{"id":1,"name":{"L":"col_86","O":"col_86"},"offset":0,"state":5,"type":{"Charset":"binary","Collate":"binary","Decimal":0,"Flag":128,"Flen":65535,"Tp":252}},{"default":"1994-05-0600:00:00","id":2,"name":{"L":"col_87","O":"col_87"},"offset":1,"state":5,"type":{"Charset":"binary","Collate":"binary","Decimal":0,"Flag":129,"Flen":19,"Tp":12}},{"id":3,"name":{"L":"col_88","O":"col_88"},"offset":2,"state":5,"type":{"Charset":"binary","Collate":"binary","Decimal":0,"Flag":32,"Flen":42,"Tp":16}},{"default":"\\u0000\\u0000\\u0000\\u0000\\u0000\\u0000\\u0000\\u0000\\u0000\\u0000\\u0000\\u0000\\u0000\\u0000\\u0000\\u0000\\u0000\\u0000\\u0000\\u0000\\u0000","id":4,"name":{"L":"col_89","O":"col_89"},"offset":3,"state":5,"type":{"Charset":"binary","Collate":"binary","Decimal":0,"Flag":129,"Flen":21,"Tp":254}},{"id":5,"name":{"L":"col_90","O":"col_90"},"offset":4,"state":5,"type":{"Charset":"binary","Collate":"binary","Decimal":0,"Flag":4129,"Flen":3,"Tp":1}},{"default":"\\u0007\\u0007","default_bit":"Bwc=","id":6,"name":{"L":"col_91","O":"col_91"},"offset":5,"state":5,"type":{"Charset":"binary","Collate":"binary","Decimal":0,"Flag":32,"Flen":12,"Tp":16}},{"default":"kY~6to6H4ut*QAPrj@&","id":7,"name":{"L":"col_92","O":"col_92"},"offset":6,"state":5,"type":{"Charset":"binary","Collate":"binary","Decimal":0,"Flag":129,"Flen":343,"Tp":15}},{"id":8,"name":{"L":"col_93","O":"col_93"},"offset":7,"state":5,"type":{"Charset":"binary","Collate":"binary","Decimal":0,"Flag":128,"Flen":-1,"Tp":245}}],"id":546,"index_info":[],"is_common_handle":false,"keyspace_id":4294967295,"name":{"L":"tcfc7825f","O":"tcfc7825f"},"pk_is_handle":false,"state":5,"tiflash_replica":{"Available":false,"Count":1},"update_timestamp":452653255976550448}', 0))stmt"), //
370381
},
382+
StmtCase{
383+
1145, //
384+
0,
385+
R"json({"id":1939,"db_name":{"O":"customer","L":"customer"},"charset":"utf8mb4","collate":"utf8mb4_bin","state":5})json", //
386+
R"json({"cols":[{"id":1,"name":{"L":"id","O":"id"},"offset":0,"state":5,"type":{"Charset":"binary","Collate":"binary","Decimal":0,"Flag":0,"Flen":11,"Tp":3}},{"id":2,"name":{"L":"_v$_idx_name_0","O":"_v$_idx_name_0"},"offset":1,"state":5,"type":{"Charset":"binary","Collate":"binary","Decimal":0,"Flag":136,"Flen":0,"Tp":6}}],"id":1145,"index_info":[],"is_common_handle":false,"keyspace_id":4294967295,"name":{"L":"t","O":"t"},"pk_is_handle":false,"schema_version":-1,"state":5,"tiflash_replica":{"Available":false,"Count":1},"update_timestamp":456163970651521027})json", //
387+
// Note that the `v$_idx_name_0` column is created as Nullable(Int8) rather than Nullable(Nothing) for compatibility with null expression index (tiflash#9891)
388+
R"stmt(CREATE TABLE `db_1939`.`t_1145`(`id` Nullable(Int32), `_v$_idx_name_0` Nullable(Int8), `_tidb_rowid` Int64) Engine = DeltaMerge((`_tidb_rowid`), '{"cols":[{"id":1,"name":{"L":"id","O":"id"},"offset":0,"state":5,"type":{"Charset":"binary","Collate":"binary","Decimal":0,"Flag":0,"Flen":11,"Tp":3}},{"id":2,"name":{"L":"_v$_idx_name_0","O":"_v$_idx_name_0"},"offset":1,"state":5,"type":{"Charset":"binary","Collate":"binary","Decimal":0,"Flag":136,"Flen":0,"Tp":6}}],"id":1145,"index_info":[],"is_common_handle":false,"keyspace_id":4294967295,"name":{"L":"t","O":"t"},"pk_is_handle":false,"state":5,"tiflash_replica":{"Available":false,"Count":1},"update_timestamp":456163970651521027}', 0))stmt", //
389+
},
371390
};
372391
// clang-format on
373392

dbms/src/TiDB/tests/gtest_type_mapping.cpp

Lines changed: 12 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -18,9 +18,7 @@
1818
#include <TiDB/Decode/TypeMapping.h>
1919
#include <TiDB/Schema/TiDB.h>
2020

21-
namespace DB
22-
{
23-
namespace tests
21+
namespace DB::tests
2422
{
2523

2624
TEST(TypeMappingTest, DataTypeToColumnInfo)
@@ -93,9 +91,18 @@ try
9391
ASSERT_EQ(data_type->getName(), DataTypeString::getDefaultName());
9492
}
9593

94+
{
95+
auto str_type = typeFromString("Int8");
96+
column_info = reverseGetColumnInfo(NameAndTypePair{name, str_type}, 1, default_field, true);
97+
column_info.tp = TiDB::TP::TypeNull; // test for TypeNull
98+
ASSERT_EQ(column_info.tp, TiDB::TP::TypeNull);
99+
auto data_type = getDataTypeByColumnInfo(column_info);
100+
// Use Int8 to compatible "storing" TypeNull in IStorage
101+
ASSERT_EQ(data_type->getName(), "Int8");
102+
}
103+
96104
// TODO: test decimal, datetime, enum
97105
}
98106
CATCH
99107

100-
} // namespace tests
101-
} // namespace DB
108+
} // namespace DB::tests
Lines changed: 47 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,47 @@
1+
# Copyright 2025 PingCAP, Inc.
2+
#
3+
# Licensed under the Apache License, Version 2.0 (the "License");
4+
# you may not use this file except in compliance with the License.
5+
# You may obtain a copy of the License at
6+
#
7+
# http://www.apache.org/licenses/LICENSE-2.0
8+
#
9+
# Unless required by applicable law or agreed to in writing, software
10+
# distributed under the License is distributed on an "AS IS" BASIS,
11+
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12+
# See the License for the specific language governing permissions and
13+
# limitations under the License.
14+
15+
mysql> drop table if exists test.t;
16+
17+
# test for null expression express tiflash#9891
18+
mysql> CREATE TABLE test.t (id int, KEY idx_name ((null)));
19+
mysql> alter table test.t set tiflash replica 1;
20+
mysql> insert test.t values(0),(1);
21+
22+
func> wait_table test t
23+
24+
mysql> insert into test.t values (2), (3);
25+
mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t order by id;
26+
+----+
27+
| id |
28+
+----+
29+
| 0 |
30+
| 1 |
31+
| 2 |
32+
| 3 |
33+
+----+
34+
35+
mysql> CREATE INDEX idx_n ON test.t ((null));
36+
mysql> alter table test.t add column c1 int;
37+
mysql> set session tidb_isolation_read_engines='tiflash'; select id,c1 from test.t order by id;
38+
+------+------+
39+
| id | c1 |
40+
+------+------+
41+
| 0 | NULL |
42+
| 1 | NULL |
43+
| 2 | NULL |
44+
| 3 | NULL |
45+
+------+------+
46+
47+
mysql> drop table if exists test.t;

0 commit comments

Comments
 (0)