to #31469_MaterializedMysql_mysqlDate2CkDate32

This commit is contained in:
zzsmdfj 2022-03-08 18:17:22 +08:00
parent df1a031851
commit 67b9f81104
10 changed files with 48 additions and 10 deletions

View File

@ -76,7 +76,7 @@ When working with the `MaterializedMySQL` database engine, [ReplacingMergeTree](
| FLOAT | [Float32](../../sql-reference/data-types/float.md) |
| DOUBLE | [Float64](../../sql-reference/data-types/float.md) |
| DECIMAL, NEWDECIMAL | [Decimal](../../sql-reference/data-types/decimal.md) |
| DATE, NEWDATE | [Date](../../sql-reference/data-types/date.md) |
| DATE, NEWDATE | [Date32](../../sql-reference/data-types/date32.md) |
| DATETIME, TIMESTAMP | [DateTime](../../sql-reference/data-types/datetime.md) |
| DATETIME2, TIMESTAMP2 | [DateTime64](../../sql-reference/data-types/datetime64.md) |
| YEAR | [UInt16](../../sql-reference/data-types/int-uint.md) |

View File

@ -43,7 +43,7 @@ ENGINE = MySQL('host:port', ['database' | database], 'user', 'password')
| BIGINT | [Int64](../../sql-reference/data-types/int-uint.md) |
| FLOAT | [Float32](../../sql-reference/data-types/float.md) |
| DOUBLE | [Float64](../../sql-reference/data-types/float.md) |
| DATE | [Date](../../sql-reference/data-types/date.md) |
| DATE | [Date32](../../sql-reference/data-types/date32.md) |
| DATETIME, TIMESTAMP | [DateTime](../../sql-reference/data-types/datetime.md) |
| BINARY | [FixedString](../../sql-reference/data-types/fixedstring.md) |

View File

@ -431,7 +431,7 @@ namespace MySQLReplication
UInt32 i24 = 0;
payload.readStrict(reinterpret_cast<char *>(&i24), 3);
const DayNum date_day_number(DateLUT::instance().makeDayNum(
const ExtendedDayNum date_day_number(DateLUT::instance().makeDayNum(
static_cast<int>((i24 >> 9) & 0x7fff), static_cast<int>((i24 >> 5) & 0xf), static_cast<int>(i24 & 0x1f)).toUnderType());
row.push_back(Field(date_day_number.toUnderType()));

View File

@ -6,7 +6,7 @@
#include <Core/SettingsEnums.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/IAST.h>
#include "DataTypeDate.h"
#include "DataTypeDate32.h"
#include "DataTypeDateTime.h"
#include "DataTypeDateTime64.h"
#include "DataTypeEnum.h"
@ -73,7 +73,7 @@ DataTypePtr convertMySQLDataType(MultiEnum<MySQLDataTypesSupport> type_support,
else if (type_name == "double")
res = std::make_shared<DataTypeFloat64>();
else if (type_name == "date")
res = std::make_shared<DataTypeDate>();
res = std::make_shared<DataTypeDate32>();
else if (type_name == "binary")
res = std::make_shared<DataTypeFixedString>(length);
else if (type_name == "datetime" || type_name == "timestamp")

View File

@ -130,6 +130,9 @@ static NamesAndTypesList getColumnsList(const ASTExpressionList * columns_defini
child = new_child;
}
}
if (type_name_upper == "DATE")
data_type_function->name = "Date32";
}
if (is_nullable)
data_type = makeASTFunction("Nullable", data_type);
@ -335,7 +338,7 @@ static ASTPtr getPartitionPolicy(const NamesAndTypesList & primary_keys)
if (which.isNullable())
throw Exception("LOGICAL ERROR: MySQL primary key must be not null, it is a bug.", ErrorCodes::LOGICAL_ERROR);
if (which.isDate() || which.isDateTime() || which.isDateTime64())
if (which.isDate() || which.isDate32() || which.isDateTime() || which.isDateTime64())
{
/// In any case, date or datetime is always the best partitioning key
return makeASTFunction("toYYYYMM", std::make_shared<ASTIdentifier>(primary_key.name));

View File

@ -39,7 +39,7 @@ TEST(MySQLCreateRewritten, ColumnsDataType)
{
{"TINYINT", "Int8"}, {"SMALLINT", "Int16"}, {"MEDIUMINT", "Int32"}, {"INT", "Int32"},
{"INTEGER", "Int32"}, {"BIGINT", "Int64"}, {"FLOAT", "Float32"}, {"DOUBLE", "Float64"},
{"VARCHAR(10)", "String"}, {"CHAR(10)", "String"}, {"Date", "Date"}, {"DateTime", "DateTime"},
{"VARCHAR(10)", "String"}, {"CHAR(10)", "String"}, {"Date", "Date32"}, {"DateTime", "DateTime"},
{"TIMESTAMP", "DateTime"}, {"BOOLEAN", "Bool"}, {"BIT", "UInt64"}, {"SET", "UInt64"},
{"YEAR", "UInt16"}, {"TIME", "Int64"}, {"GEOMETRY", "String"}
};
@ -104,7 +104,7 @@ TEST(MySQLCreateRewritten, PartitionPolicy)
{"MEDIUMINT", "Int32", " PARTITION BY intDiv(key, 4294967)"}, {"INT", "Int32", " PARTITION BY intDiv(key, 4294967)"},
{"INTEGER", "Int32", " PARTITION BY intDiv(key, 4294967)"}, {"BIGINT", "Int64", " PARTITION BY intDiv(key, 18446744073709551)"},
{"FLOAT", "Float32", ""}, {"DOUBLE", "Float64", ""}, {"VARCHAR(10)", "String", ""}, {"CHAR(10)", "String", ""},
{"Date", "Date", " PARTITION BY toYYYYMM(key)"}, {"DateTime", "DateTime", " PARTITION BY toYYYYMM(key)"},
{"Date", "Date32", " PARTITION BY toYYYYMM(key)"}, {"DateTime", "DateTime", " PARTITION BY toYYYYMM(key)"},
{"TIMESTAMP", "DateTime", " PARTITION BY toYYYYMM(key)"}, {"BOOLEAN", "Bool", " PARTITION BY key"}
};
@ -135,7 +135,7 @@ TEST(MySQLCreateRewritten, OrderbyPolicy)
{"MEDIUMINT", "Int32", " PARTITION BY intDiv(key, 4294967)"}, {"INT", "Int32", " PARTITION BY intDiv(key, 4294967)"},
{"INTEGER", "Int32", " PARTITION BY intDiv(key, 4294967)"}, {"BIGINT", "Int64", " PARTITION BY intDiv(key, 18446744073709551)"},
{"FLOAT", "Float32", ""}, {"DOUBLE", "Float64", ""}, {"VARCHAR(10)", "String", ""}, {"CHAR(10)", "String", ""},
{"Date", "Date", " PARTITION BY toYYYYMM(key)"}, {"DateTime", "DateTime", " PARTITION BY toYYYYMM(key)"},
{"Date", "Date32", " PARTITION BY toYYYYMM(key)"}, {"DateTime", "DateTime", " PARTITION BY toYYYYMM(key)"},
{"TIMESTAMP", "DateTime", " PARTITION BY toYYYYMM(key)"}, {"BOOLEAN", "Bool", " PARTITION BY key"}
};

View File

@ -225,6 +225,10 @@ namespace
assert_cast<ColumnUInt16 &>(column).insertValue(UInt16(value.getDate().getDayNum()));
read_bytes_size += 2;
break;
case ValueType::vtDate32:
assert_cast<ColumnInt32 &>(column).insertValue(Int32(value.getDate().getExtenedDayNum()));
read_bytes_size += 4;
break;
case ValueType::vtDateTime:
{
ReadBufferFromString in(value);

View File

@ -1218,4 +1218,28 @@ def materialized_database_settings_materialized_mysql_tables_list(clickhouse_nod
check_query(clickhouse_node, "SELECT COUNT() FROM test_database.c FORMAT TSV", "2\n")
clickhouse_node.query("DROP DATABASE test_database")
mysql_node.query("DROP DATABASE test_database")
mysql_node.query("DROP DATABASE test_database")
def materialized_database_mysql_date_type_to_date32(clickhouse_node, mysql_node, service_name):
mysql_node.query("DROP DATABASE IF EXISTS test_database")
clickhouse_node.query("DROP DATABASE IF EXISTS test_database")
mysql_node.query("CREATE DATABASE test_database")
mysql_node.query("CREATE TABLE test_database.a (a INT(11) NOT NULL PRIMARY KEY, b date DEFAULT NULL)")
# can't support date that less than 1925 year for now
mysql_node.query("INSERT INTO test_database.a VALUES(1, '1900-04-16')")
# test date that is older than 1925
mysql_node.query("INSERT INTO test_database.a VALUES(2, '1925-03-16')")
mysql_node.query("INSERT INTO test_database.a VALUES(3, '1971-02-16')")
mysql_node.query("INSERT INTO test_database.a VALUES(4, '2101-05-16')")
clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializedMySQL('{}:3306', 'test_database', 'root', 'clickhouse')".format(service_name))
check_query(clickhouse_node, "SELECT b from test_database.a order by a FORMAT TSV", "1970-01-01\n1925-03-16\n1971-02-16\n2101-05-16\n")
mysql_node.query("INSERT INTO test_database.a VALUES(5, '1925-04-16')")
mysql_node.query("INSERT INTO test_database.a VALUES(6, '2022-02-16')")
mysql_node.query("INSERT INTO test_database.a VALUES(7, '2283-11-11')")
check_query(clickhouse_node, "SELECT b from test_database.a order by a FORMAT TSV", "1970-01-01\n1925-03-16\n1971-02-16\n2101-05-16\n1925-04-16\n2022-02-16\n" +
"2283-11-11\n")

View File

@ -261,3 +261,7 @@ def test_materialized_database_support_all_kinds_of_mysql_datatype(started_clust
def test_materialized_database_settings_materialized_mysql_tables_list(started_cluster, started_mysql_8_0, started_mysql_5_7, clickhouse_node):
materialize_with_ddl.materialized_database_settings_materialized_mysql_tables_list(clickhouse_node, started_mysql_8_0, "mysql80")
materialize_with_ddl.materialized_database_settings_materialized_mysql_tables_list(clickhouse_node, started_mysql_5_7, "mysql57")
def test_materialized_database_mysql_date_type_to_date32(started_cluster, started_mysql_8_0, started_mysql_5_7, clickhouse_node):
materialize_with_ddl.materialized_database_mysql_date_type_to_date32(clickhouse_node, started_mysql_8_0, "mysql80")
materialize_with_ddl.materialized_database_mysql_date_type_to_date32(clickhouse_node, started_mysql_5_7, "mysql57")

View File

@ -235,6 +235,7 @@ int8_values = [0, 1, -1, 127, -128]
uint8_values = [0, 1, 255]
# string_values = ["'ClickHouse'", 'NULL']
string_values = ["'ClickHouse'"]
date_values=["'1925-01-01'", "'2283-11-11'"]
decimal_values = [0, 0.123, 0.4, 5.67, 8.91011, 123456789.123, -0.123, -0.4, -5.67, -8.91011, -123456789.123]
@ -274,6 +275,8 @@ timestamp_values_no_subsecond = ["'2015-05-18 07:40:01'", "'2019-09-16 19:20:11'
pytest.param("common_types", "VARCHAR(10)", "Nullable(String)", string_values, "", id="common_types_20"),
pytest.param("common_types", "DATE", "Nullable(Date32)", date_values, "", id="common_types_21"),
pytest.param("decimal_default", "decimal NOT NULL", "Decimal(10, 0)", decimal_values,
"decimal,datetime64", id="decimal_1"),