mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
to #31469_MaterializedMysql_mysqlDate2CkDate32
This commit is contained in:
parent
df1a031851
commit
67b9f81104
@ -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) |
|
||||
|
@ -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) |
|
||||
|
||||
|
@ -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()));
|
||||
|
@ -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")
|
||||
|
@ -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));
|
||||
|
@ -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"}
|
||||
};
|
||||
|
||||
|
@ -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);
|
||||
|
@ -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")
|
||||
|
||||
|
||||
|
@ -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")
|
@ -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"),
|
||||
|
Loading…
Reference in New Issue
Block a user