Skip to content

Commit

Permalink
TiFlash accept lossless data type change for mydatetime(release-6.5) (#…
Browse files Browse the repository at this point in the history
…7813)

close #7809
  • Loading branch information
JaySon-Huang authored Jul 18, 2023
1 parent 72da328 commit bae3aef
Show file tree
Hide file tree
Showing 8 changed files with 170 additions and 22 deletions.
4 changes: 2 additions & 2 deletions dbms/src/DataTypes/DataTypeMyDateTime.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@ DataTypeMyDateTime::DataTypeMyDateTime(int fraction_)
{
fraction = fraction_;
if (fraction < 0 || fraction > 6)
throw Exception("fraction must >= 0 and <= 6", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "fraction must >= 0 and <= 6, fraction={}", fraction_);
}

void DataTypeMyDateTime::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr) const
Expand Down Expand Up @@ -132,7 +132,7 @@ static DataTypePtr create(const ASTPtr & arguments)
"MyDateTime data type can optionally have only one argument - fractional",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);

const ASTLiteral * arg = typeid_cast<const ASTLiteral *>(arguments->children[0].get());
const auto * arg = typeid_cast<const ASTLiteral *>(arguments->children[0].get());
if (!arg || arg->value.getType() != Field::Types::UInt64)
throw Exception("Parameter for MyDateTime data type must be uint literal", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);

Expand Down
31 changes: 20 additions & 11 deletions dbms/src/DataTypes/isSupportedDataTypeCast.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeEnum.h>
#include <DataTypes/DataTypeFixedString.h>
#include <DataTypes/DataTypeMyDateTime.h>
#include <DataTypes/DataTypeNothing.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeString.h>
Expand All @@ -41,7 +42,7 @@ bool isSupportedDataTypeCast(const DataTypePtr & from, const DataTypePtr & to)
{
bool has_nullable = false;
DataTypePtr from_not_null;
if (const DataTypeNullable * type_nullable = typeid_cast<const DataTypeNullable *>(from.get()))
if (const auto * type_nullable = typeid_cast<const DataTypeNullable *>(from.get()))
{
has_nullable = true;
from_not_null = type_nullable->getNestedType();
Expand All @@ -52,7 +53,7 @@ bool isSupportedDataTypeCast(const DataTypePtr & from, const DataTypePtr & to)
}

DataTypePtr to_not_null;
if (const DataTypeNullable * type_nullable = typeid_cast<const DataTypeNullable *>(to.get()))
if (const auto * type_nullable = typeid_cast<const DataTypeNullable *>(to.get()))
{
has_nullable = true;
to_not_null = type_nullable->getNestedType();
Expand Down Expand Up @@ -96,15 +97,23 @@ bool isSupportedDataTypeCast(const DataTypePtr & from, const DataTypePtr & to)
if (from->isStringOrFixedString() && to->isStringOrFixedString())
{
size_t from_sz = std::numeric_limits<size_t>::max();
if (const DataTypeFixedString * type_fixed_str = typeid_cast<const DataTypeFixedString *>(from.get()))
if (const auto * type_fixed_str = typeid_cast<const DataTypeFixedString *>(from.get()))
from_sz = type_fixed_str->getN();
size_t to_sz = std::numeric_limits<size_t>::max();
if (const DataTypeFixedString * type_fixed_str = typeid_cast<const DataTypeFixedString *>(to.get()))
if (const auto * type_fixed_str = typeid_cast<const DataTypeFixedString *>(to.get()))
to_sz = type_fixed_str->getN();
return from_sz <= to_sz;
}

/// For Date and DateTime, not supported
if (from->getTypeId() == TypeIndex::MyDateTime && to->getTypeId() == TypeIndex::MyDateTime)
{
const auto * const from_mydatetime = checkAndGetDataType<DataTypeMyDateTime>(from.get());
const auto * const to_mydatetime = checkAndGetDataType<DataTypeMyDateTime>(to.get());
// Enlarging the `fsp` of `mydatetime`/`timestamp`/`time` is a lossless change, TiFlash should detect and change the data type in place.
// Narrowing down the `fsp` is a lossy change, TiDB will add a temporary column and reorganize the column data as other lossy type change.
return (from_mydatetime->getFraction() < to_mydatetime->getFraction());
}
/// For other cases of Date and DateTime, not supported
if (from->isDateOrDateTime() || to->isDateOrDateTime())
{
return false;
Expand All @@ -123,21 +132,21 @@ bool isSupportedDataTypeCast(const DataTypePtr & from, const DataTypePtr & to)
if (from->isEnum() && to->isEnum())
{
/// support cast Enum to Enum if the from type is a subset of the target type
const auto from_enum8 = checkAndGetDataType<DataTypeEnum8>(from.get());
const auto to_enum8 = checkAndGetDataType<DataTypeEnum8>(to.get());
const auto * const from_enum8 = checkAndGetDataType<DataTypeEnum8>(from.get());
const auto * const to_enum8 = checkAndGetDataType<DataTypeEnum8>(to.get());
if (from_enum8 && to_enum8)
{
for (auto & value : from_enum8->getValues())
for (const auto & value : from_enum8->getValues())
{
if (!to_enum8->hasElement(value.first) || to_enum8->getValue(value.first) != value.second)
return false;
}
}
const auto from_enum16 = checkAndGetDataType<DataTypeEnum16>(from.get());
const auto to_enum16 = checkAndGetDataType<DataTypeEnum16>(to.get());
const auto * const from_enum16 = checkAndGetDataType<DataTypeEnum16>(from.get());
const auto * const to_enum16 = checkAndGetDataType<DataTypeEnum16>(to.get());
if (from_enum16 && to_enum16)
{
for (auto & value : from_enum16->getValues())
for (const auto & value : from_enum16->getValues())
{
if (!to_enum16->hasElement(value.first) || to_enum16->getValue(value.first) != value.second)
return false;
Expand Down
7 changes: 7 additions & 0 deletions dbms/src/DataTypes/tests/gtest_data_type_get_common_type.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -235,9 +235,16 @@ try
// float64 -> float32 is lossy
ASSERT_FALSE(isSupportedDataTypeCast(typeFromString("Float64"), typeFromString("Float32")));

// Enlarging the `fsp` of `mydatetime`/`timestamp`/`time` is ok
ASSERT_TRUE(isSupportedDataTypeCast(typeFromString("MyDateTime(3)"), typeFromString("MyDateTime(6)")));
// Narrowing down the `fsp` is lossy
ASSERT_FALSE(isSupportedDataTypeCast(typeFromString("MyDateTime(3)"), typeFromString("MyDateTime(0)")));

// not support datetime <-> date
ASSERT_FALSE(isSupportedDataTypeCast(typeFromString("DateTime"), typeFromString("Date")));
ASSERT_FALSE(isSupportedDataTypeCast(typeFromString("Date"), typeFromString("DateTime")));
ASSERT_FALSE(isSupportedDataTypeCast(typeFromString("MyDate"), typeFromString("MyDateTime(6)")));
ASSERT_FALSE(isSupportedDataTypeCast(typeFromString("MyDateTime(3)"), typeFromString("MyDate")));

// strings
ASSERT_TRUE(isSupportedDataTypeCast(typeFromString("FixedString(16)"), typeFromString("FixedString(100)")));
Expand Down
13 changes: 13 additions & 0 deletions dbms/src/Storages/DeltaMerge/convertColumnTypeHelpers.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
#include <Functions/FunctionHelpers.h>
#include <Storages/DeltaMerge/DeltaMergeHelpers.h>
#include <Storages/DeltaMerge/convertColumnTypeHelpers.h>
#include <common/types.h>

namespace DB
{
Expand Down Expand Up @@ -315,6 +316,18 @@ bool castNonNullNumericColumn(const DataTypePtr & disk_type_not_null_,
rows_limit);
return true;
}
else if (checkDataType<DataTypeMyDateTime>(disk_type_not_null) && checkDataType<DataTypeMyDateTime>(read_type_not_null))
{
static_assert(std::is_same_v<DataTypeMyDateTime::FieldType, UInt64>, "Ensure the MyDateTime/MyTime is stored as UInt64");
insertRangeFromWithNumericTypeCast<UInt64, UInt64>(
disk_col_not_null,
null_map,
read_define,
memory_col_not_null,
rows_offset,
rows_limit);
return true;
}
// else is not support
return false;
}
Expand Down
50 changes: 42 additions & 8 deletions dbms/src/Storages/DeltaMerge/tests/gtest_convert_column.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -21,12 +21,9 @@
#include <Storages/DeltaMerge/tests/DMTestEnv.h>
#include <Storages/Transaction/TiDB.h>

namespace DB
{
namespace DM
{
namespace tests
namespace DB::DM::tests
{

TEST(ConvertColumnTypeTest, CastNumeric)
{
{
Expand Down Expand Up @@ -179,6 +176,45 @@ TEST(ConvertColumnTypeTest, CastNumeric)
}
}

TEST(ConvertColumnTypeTest, CastMyDateTime)
try
{
{
const Strings to_types = {"MyDateTime(1)", "MyDateTime(3)", "MyDateTime(6)"};

DataTypePtr disk_data_type = typeFromString("MyDateTime(0)");
MutableColumnPtr disk_col = disk_data_type->createColumn();
disk_col->insert(Field(static_cast<UInt64>(MyDateTime(2023, 7, 17, 15, 39, 29, 20).toPackedUInt())));
disk_col->insert(Field(static_cast<UInt64>(MyDateTime(2020, 2, 29, 15, 39, 29, 20).toPackedUInt())));

for (const String & to_type : to_types)
{
ColumnDefine read_define(0, "c", typeFromString(to_type));
auto memory_column = convertColumnByColumnDefineIfNeed(disk_data_type, disk_col->getPtr(), read_define);

UInt64 val1 = memory_column->getUInt(0);
MyDateTime myval(val1);
EXPECT_EQ(myval.year, 2023);
EXPECT_EQ(myval.month, 7);
EXPECT_EQ(myval.day, 17);
EXPECT_EQ(myval.hour, 15);
EXPECT_EQ(myval.minute, 39);
EXPECT_EQ(myval.second, 29);
EXPECT_EQ(myval.micro_second, 20);

UInt64 val2 = memory_column->getUInt(1);
MyDateTime myval2(val2);
EXPECT_EQ(myval2.year, 2020);
EXPECT_EQ(myval2.month, 2);
EXPECT_EQ(myval2.day, 29);
EXPECT_EQ(myval2.hour, 15);
EXPECT_EQ(myval2.minute, 39);
EXPECT_EQ(myval2.second, 29);
}
}
}
CATCH

TEST(ConvertColumnTypeTest, CastNullableToNotNull)
{
const Strings to_types = {"Int8", "Int16", "Int32", "Int64"};
Expand Down Expand Up @@ -440,6 +476,4 @@ try
}
CATCH

} // namespace tests
} // namespace DM
} // namespace DB
} // namespace DB::DM::tests
2 changes: 1 addition & 1 deletion dbms/src/Storages/StorageDeltaMerge.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -1190,7 +1190,7 @@ try
// check that lossy changes is forbidden
// check that changing the UNSIGNED attribute is forbidden
throw Exception(
fmt::format("Storage engine {} doesn't support lossy data type modification. Try to modify column {}({}) from {} to {}",
fmt::format("Storage engine {} doesn't support lossy data type modification. Try to modify column {} (column_id={}) from {} to {}",
getName(),
command.column_name,
command.column_id,
Expand Down
5 changes: 5 additions & 0 deletions dbms/src/TiDB/Schema/SchemaBuilder.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -139,6 +139,11 @@ bool typeDiffers(const TiDB::ColumnInfo & a, const TiDB::ColumnInfo & b)
{
return a.flen != b.flen || a.decimal != b.decimal;
}
else if (a.tp == TiDB::TypeDatetime || a.tp == TiDB::TypeDate || a.tp == TiDB::TypeTimestamp)
{
// detect fsp changed in MyDateTime/MyTime
return a.flen != b.flen || a.decimal != b.decimal;
}
return false;
}

Expand Down
80 changes: 80 additions & 0 deletions tests/fullstack-test2/ddl/alter_column_datetime.test
Original file line number Diff line number Diff line change
@@ -0,0 +1,80 @@
# 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.

# datetime,timestamp,time enlarge fsp
mysql> drop table if exists test.a1;
mysql> create table test.a1(id int(11) NOT NULL AUTO_INCREMENT, `atime` datetime(0) DEFAULT '2023-01-01 00:00:00', `btime` timestamp(0) DEFAULT '2023-01-01 00:00:00', `ctime` time(0) DEFAULT '00:00:00', PRIMARY KEY (id));
mysql> insert into test.a1 values (1,'2023-01-01 11:11:11','2023-01-01 11:11:11','11:11:11'),(2,'2023-02-01 02:22:22','2023-02-01 02:22:22','02:22:22'),(3,'2023-03-01 03:33:33','2023-03-01 03:33:33','03:33:33');
mysql> alter table test.a1 set tiflash replica 1;

func> wait_table test a1

mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.a1 order by id;
+----+---------------------+---------------------+----------+
| id | atime | btime | ctime |
+----+---------------------+---------------------+----------+
| 1 | 2023-01-01 11:11:11 | 2023-01-01 11:11:11 | 11:11:11 |
| 2 | 2023-02-01 02:22:22 | 2023-02-01 02:22:22 | 02:22:22 |
| 3 | 2023-03-01 03:33:33 | 2023-03-01 03:33:33 | 03:33:33 |
+----+---------------------+---------------------+----------+

mysql> alter table test.a1 modify column atime datetime(3);
mysql> alter table test.a1 modify column btime timestamp(3);
mysql> alter table test.a1 modify column ctime time(3);
mysql> insert into test.a1 values (4,'2023-04-01 04:44:44.444','2023-04-01 04:44:44.444','04:44:44.444');
mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.a1 order by id;
+----+-------------------------+-------------------------+--------------+
| id | atime | btime | ctime |
+----+-------------------------+-------------------------+--------------+
| 1 | 2023-01-01 11:11:11.000 | 2023-01-01 11:11:11.000 | 11:11:11.000 |
| 2 | 2023-02-01 02:22:22.000 | 2023-02-01 02:22:22.000 | 02:22:22.000 |
| 3 | 2023-03-01 03:33:33.000 | 2023-03-01 03:33:33.000 | 03:33:33.000 |
| 4 | 2023-04-01 04:44:44.444 | 2023-04-01 04:44:44.444 | 04:44:44.444 |
+----+-------------------------+-------------------------+--------------+

mysql> drop table if exists test.a1;

# datetime,timestamp,time narrow down fsp
mysql> drop table if exists test.a1;
mysql> create table test.a1(id int(11) NOT NULL AUTO_INCREMENT, `atime` datetime(3) DEFAULT '2023-01-01 00:00:00.000', `btime` timestamp(3) DEFAULT '2023-01-01 00:00:00.000', `ctime` time(3) DEFAULT '00:00:00.000', PRIMARY KEY (id));
mysql> insert into test.a1 values (1,'2023-01-01 11:11:11.111','2023-01-01 11:11:11.111','11:11:11.111'),(2,'2023-02-01 02:22:22.222','2023-02-01 02:22:22.222','02:22:22.222'),(3,'2023-03-01 03:33:33.333','2023-03-01 03:33:33.333','03:33:33.333');
mysql> alter table test.a1 set tiflash replica 1;

func> wait_table test a1

mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.a1 order by id;
+----+-------------------------+-------------------------+--------------+
| id | atime | btime | ctime |
+----+-------------------------+-------------------------+--------------+
| 1 | 2023-01-01 11:11:11.111 | 2023-01-01 11:11:11.111 | 11:11:11.111 |
| 2 | 2023-02-01 02:22:22.222 | 2023-02-01 02:22:22.222 | 02:22:22.222 |
| 3 | 2023-03-01 03:33:33.333 | 2023-03-01 03:33:33.333 | 03:33:33.333 |
+----+-------------------------+-------------------------+--------------+

mysql> alter table test.a1 modify column atime datetime(0);
mysql> alter table test.a1 modify column btime timestamp(0);
mysql> alter table test.a1 modify column ctime time(0);
mysql> insert into test.a1 values (4,'2023-04-01 04:44:44','2023-04-01 04:44:44','04:44:44');

mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.a1 order by id;
+----+---------------------+---------------------+----------+
| id | atime | btime | ctime |
+----+---------------------+---------------------+----------+
| 1 | 2023-01-01 11:11:11 | 2023-01-01 11:11:11 | 11:11:11 |
| 2 | 2023-02-01 02:22:22 | 2023-02-01 02:22:22 | 02:22:22 |
| 3 | 2023-03-01 03:33:33 | 2023-03-01 03:33:33 | 03:33:33 |
| 4 | 2023-04-01 04:44:44 | 2023-04-01 04:44:44 | 04:44:44 |
+----+---------------------+---------------------+----------+

mysql> drop table if exists test.a1;

0 comments on commit bae3aef

Please sign in to comment.