mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 09:32:06 +00:00
Merge branch 'master' into PF202201261620
This commit is contained in:
commit
1c0af06011
@ -78,15 +78,21 @@ When working with the `MaterializedMySQL` database engine, [ReplacingMergeTree](
|
||||
| DATE, NEWDATE | [Date](../../sql-reference/data-types/date.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) |
|
||||
| TIME | [Int64](../../sql-reference/data-types/int-uint.md) |
|
||||
| ENUM | [Enum](../../sql-reference/data-types/enum.md) |
|
||||
| STRING | [String](../../sql-reference/data-types/string.md) |
|
||||
| VARCHAR, VAR_STRING | [String](../../sql-reference/data-types/string.md) |
|
||||
| BLOB | [String](../../sql-reference/data-types/string.md) |
|
||||
| GEOMETRY | [String](../../sql-reference/data-types/string.md) |
|
||||
| BINARY | [FixedString](../../sql-reference/data-types/fixedstring.md) |
|
||||
| BIT | [UInt64](../../sql-reference/data-types/int-uint.md) |
|
||||
| SET | [UInt64](../../sql-reference/data-types/int-uint.md) |
|
||||
|
||||
[Nullable](../../sql-reference/data-types/nullable.md) is supported.
|
||||
|
||||
The data of TIME type in MySQL is converted to microseconds in ClickHouse.
|
||||
|
||||
Other types are not supported. If MySQL table contains a column of such type, ClickHouse throws exception "Unhandled data type" and stops replication.
|
||||
|
||||
## Specifics and Recommendations {#specifics-and-recommendations}
|
||||
|
@ -16,7 +16,15 @@ using MYSQL_ROW = char**;
|
||||
struct st_mysql_field;
|
||||
using MYSQL_FIELD = st_mysql_field;
|
||||
|
||||
enum struct enum_field_types;
|
||||
enum struct enum_field_types { MYSQL_TYPE_DECIMAL, MYSQL_TYPE_TINY,
|
||||
MYSQL_TYPE_SHORT, MYSQL_TYPE_LONG,
|
||||
MYSQL_TYPE_FLOAT, MYSQL_TYPE_DOUBLE,
|
||||
MYSQL_TYPE_NULL, MYSQL_TYPE_TIMESTAMP,
|
||||
MYSQL_TYPE_LONGLONG, MYSQL_TYPE_INT24,
|
||||
MYSQL_TYPE_DATE, MYSQL_TYPE_TIME,
|
||||
MYSQL_TYPE_DATETIME, MYSQL_TYPE_YEAR,
|
||||
MYSQL_TYPE_NEWDATE, MYSQL_TYPE_VARCHAR,
|
||||
MYSQL_TYPE_BIT };
|
||||
|
||||
#endif
|
||||
|
||||
|
@ -204,6 +204,7 @@ namespace MySQLReplication
|
||||
case MYSQL_TYPE_DATE:
|
||||
case MYSQL_TYPE_DATETIME:
|
||||
case MYSQL_TYPE_NEWDATE:
|
||||
case MYSQL_TYPE_YEAR:
|
||||
{
|
||||
/// No data here.
|
||||
column_meta.emplace_back(0);
|
||||
@ -214,7 +215,9 @@ namespace MySQLReplication
|
||||
case MYSQL_TYPE_DOUBLE:
|
||||
case MYSQL_TYPE_TIMESTAMP2:
|
||||
case MYSQL_TYPE_DATETIME2:
|
||||
case MYSQL_TYPE_TIME2:
|
||||
case MYSQL_TYPE_BLOB:
|
||||
case MYSQL_TYPE_GEOMETRY:
|
||||
{
|
||||
column_meta.emplace_back(UInt16(meta[pos]));
|
||||
pos += 1;
|
||||
@ -432,6 +435,98 @@ namespace MySQLReplication
|
||||
row.push_back(Field(date_day_number.toUnderType()));
|
||||
break;
|
||||
}
|
||||
case MYSQL_TYPE_YEAR: {
|
||||
Int16 val = 0;
|
||||
payload.readStrict(reinterpret_cast<char *>(&val), 1);
|
||||
row.push_back(Field{UInt16{static_cast<UInt16>(val + 1900)}});
|
||||
break;
|
||||
}
|
||||
case MYSQL_TYPE_TIME2:
|
||||
{
|
||||
UInt64 uintpart = 0UL;
|
||||
Int32 frac = 0U;
|
||||
Int64 ltime;
|
||||
Int64 intpart;
|
||||
switch (meta)
|
||||
{
|
||||
case 0:
|
||||
{
|
||||
readBigEndianStrict(payload, reinterpret_cast<char *>(&uintpart), 3);
|
||||
intpart = uintpart - 0x800000L;
|
||||
ltime = intpart << 24;
|
||||
break;
|
||||
}
|
||||
case 1:
|
||||
case 2:
|
||||
{
|
||||
readBigEndianStrict(payload, reinterpret_cast<char *>(&uintpart), 3);
|
||||
intpart = uintpart - 0x800000L;
|
||||
readBigEndianStrict(payload, reinterpret_cast<char *>(&frac), 1);
|
||||
if (intpart < 0 && frac > 0)
|
||||
{
|
||||
intpart ++;
|
||||
frac -= 0x100;
|
||||
}
|
||||
frac = frac * 10000;
|
||||
ltime = intpart << 24;
|
||||
break;
|
||||
}
|
||||
case 3:
|
||||
case 4:
|
||||
{
|
||||
readBigEndianStrict(payload, reinterpret_cast<char *>(&uintpart), 3);
|
||||
intpart = uintpart - 0x800000L;
|
||||
readBigEndianStrict(payload, reinterpret_cast<char *>(&frac), 2);
|
||||
if (intpart < 0 && frac > 0)
|
||||
{
|
||||
intpart ++;
|
||||
frac -= 0x10000;
|
||||
}
|
||||
frac = frac * 100;
|
||||
ltime = intpart << 24;
|
||||
break;
|
||||
}
|
||||
case 5:
|
||||
case 6:
|
||||
{
|
||||
readBigEndianStrict(payload, reinterpret_cast<char *>(&uintpart), 6);
|
||||
intpart = uintpart - 0x800000000000L;
|
||||
ltime = intpart;
|
||||
frac = std::abs(intpart % (1L << 24));
|
||||
break;
|
||||
}
|
||||
default:
|
||||
{
|
||||
readBigEndianStrict(payload, reinterpret_cast<char *>(&uintpart), 3);
|
||||
intpart = uintpart - 0x800000L;
|
||||
ltime = intpart << 24;
|
||||
break;
|
||||
}
|
||||
}
|
||||
Int64 hh, mm, ss;
|
||||
bool negative = false;
|
||||
if (intpart == 0)
|
||||
{
|
||||
hh = 0;
|
||||
mm = 0;
|
||||
ss = 0;
|
||||
}
|
||||
else
|
||||
{
|
||||
if (ltime < 0) negative= true;
|
||||
UInt64 ultime = std::abs(ltime);
|
||||
intpart = ultime >> 24;
|
||||
hh = (intpart >> 12) % (1 << 10);
|
||||
mm = (intpart >> 6) % (1 << 6);
|
||||
ss = intpart % (1 << 6);
|
||||
}
|
||||
|
||||
Int64 time_micro = 0;
|
||||
time_micro = (hh * 3600 + mm * 60 + ss) * 1000000 + std::abs(frac);
|
||||
if (negative) time_micro = - time_micro;
|
||||
row.push_back(Field{Int64{time_micro}});
|
||||
break;
|
||||
}
|
||||
case MYSQL_TYPE_DATETIME2:
|
||||
{
|
||||
Int64 val = 0;
|
||||
@ -585,6 +680,14 @@ namespace MySQLReplication
|
||||
}
|
||||
break;
|
||||
}
|
||||
case MYSQL_TYPE_SET:
|
||||
{
|
||||
UInt32 size = (meta & 0xff);
|
||||
Bitmap bitmap1;
|
||||
readBitmap(payload, bitmap1, size);
|
||||
row.push_back(Field{UInt64{bitmap1.to_ulong()}});
|
||||
break;
|
||||
}
|
||||
case MYSQL_TYPE_BIT:
|
||||
{
|
||||
UInt32 bits = ((meta >> 8) * 8) + (meta & 0xff);
|
||||
@ -631,6 +734,7 @@ namespace MySQLReplication
|
||||
row.push_back(Field{String{val}});
|
||||
break;
|
||||
}
|
||||
case MYSQL_TYPE_GEOMETRY:
|
||||
case MYSQL_TYPE_BLOB:
|
||||
{
|
||||
UInt32 size = 0;
|
||||
|
@ -92,5 +92,7 @@ void registerDataTypeString(DataTypeFactory & factory)
|
||||
factory.registerAlias("BINARY LARGE OBJECT", "String", DataTypeFactory::CaseInsensitive);
|
||||
factory.registerAlias("BINARY VARYING", "String", DataTypeFactory::CaseInsensitive);
|
||||
factory.registerAlias("VARBINARY", "String", DataTypeFactory::CaseInsensitive);
|
||||
factory.registerAlias("GEOMETRY", "String", DataTypeFactory::CaseInsensitive); //mysql
|
||||
|
||||
}
|
||||
}
|
||||
|
@ -86,7 +86,10 @@ void registerDataTypeNumbers(DataTypeFactory & factory)
|
||||
factory.registerAlias("INT UNSIGNED", "UInt32", DataTypeFactory::CaseInsensitive);
|
||||
factory.registerAlias("INTEGER UNSIGNED", "UInt32", DataTypeFactory::CaseInsensitive);
|
||||
factory.registerAlias("BIGINT UNSIGNED", "UInt64", DataTypeFactory::CaseInsensitive);
|
||||
factory.registerAlias("BIT", "UInt64", DataTypeFactory::CaseInsensitive);
|
||||
factory.registerAlias("BIT", "UInt64", DataTypeFactory::CaseInsensitive); /// MySQL
|
||||
factory.registerAlias("SET", "UInt64", DataTypeFactory::CaseInsensitive); /// MySQL
|
||||
factory.registerAlias("YEAR", "UInt16", DataTypeFactory::CaseInsensitive);
|
||||
factory.registerAlias("TIME", "Int64", DataTypeFactory::CaseInsensitive);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -17,6 +17,7 @@
|
||||
#include <Databases/MySQL/MaterializeMetadata.h>
|
||||
#include <Processors/Sources/MySQLSource.h>
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/executeQuery.h>
|
||||
#include <Storages/StorageMergeTree.h>
|
||||
@ -315,6 +316,47 @@ getTableOutput(const String & database_name, const String & table_name, ContextM
|
||||
return std::move(res.pipeline);
|
||||
}
|
||||
|
||||
static inline String reWriteMysqlQueryColumn(mysqlxx::Pool::Entry & connection, const String & database_name, const String & table_name, const Settings & global_settings)
|
||||
{
|
||||
Block tables_columns_sample_block
|
||||
{
|
||||
{ std::make_shared<DataTypeString>(), "column_name" },
|
||||
{ std::make_shared<DataTypeString>(), "column_type" }
|
||||
};
|
||||
|
||||
const String & query = "SELECT COLUMN_NAME AS column_name, COLUMN_TYPE AS column_type FROM INFORMATION_SCHEMA.COLUMNS"
|
||||
" WHERE TABLE_SCHEMA = '" + backQuoteIfNeed(database_name) +
|
||||
"' AND TABLE_NAME = '" + backQuoteIfNeed(table_name) + "' ORDER BY ORDINAL_POSITION";
|
||||
|
||||
StreamSettings mysql_input_stream_settings(global_settings, false, true);
|
||||
auto mysql_source = std::make_unique<MySQLSource>(connection, query, tables_columns_sample_block, mysql_input_stream_settings);
|
||||
|
||||
Block block;
|
||||
WriteBufferFromOwnString query_columns;
|
||||
QueryPipeline pipeline(std::move(mysql_source));
|
||||
PullingPipelineExecutor executor(pipeline);
|
||||
while (executor.pull(block))
|
||||
{
|
||||
const auto & column_name_col = *block.getByPosition(0).column;
|
||||
const auto & column_type_col = *block.getByPosition(1).column;
|
||||
size_t rows = block.rows();
|
||||
for (size_t i = 0; i < rows; ++i)
|
||||
{
|
||||
String column_name = column_name_col[i].safeGet<String>();
|
||||
String column_type = column_type_col[i].safeGet<String>();
|
||||
//we can do something special conversion to guarantee select results is the same as the binlog parse results
|
||||
if (column_type.starts_with("set"))
|
||||
{
|
||||
query_columns << (backQuote(column_name) + " + 0");
|
||||
} else
|
||||
query_columns << backQuote(column_name);
|
||||
query_columns << ",";
|
||||
}
|
||||
}
|
||||
String query_columns_str = query_columns.str();
|
||||
return query_columns_str.substr(0, query_columns_str.length() - 1);
|
||||
}
|
||||
|
||||
static inline void dumpDataForTables(
|
||||
mysqlxx::Pool::Entry & connection, const std::unordered_map<String, String> & need_dumping_tables,
|
||||
const String & query_prefix, const String & database_name, const String & mysql_database_name,
|
||||
@ -334,9 +376,10 @@ static inline void dumpDataForTables(
|
||||
|
||||
auto pipeline = getTableOutput(database_name, table_name, query_context);
|
||||
StreamSettings mysql_input_stream_settings(context->getSettingsRef());
|
||||
auto input = std::make_unique<MySQLSource>(
|
||||
connection, "SELECT * FROM " + backQuoteIfNeed(mysql_database_name) + "." + backQuoteIfNeed(table_name),
|
||||
pipeline.getHeader(), mysql_input_stream_settings);
|
||||
String mysql_select_all_query = "SELECT " + reWriteMysqlQueryColumn(connection, mysql_database_name, table_name, context->getSettings()) + " FROM "
|
||||
+ backQuoteIfNeed(mysql_database_name) + "." + backQuoteIfNeed(table_name);
|
||||
LOG_INFO(&Poco::Logger::get("MaterializedMySQLSyncThread(" + database_name + ")"), "mysql_select_all_query is {}", mysql_select_all_query);
|
||||
auto input = std::make_unique<MySQLSource>(connection, mysql_select_all_query, pipeline.getHeader(), mysql_input_stream_settings);
|
||||
auto counting = std::make_shared<CountingTransform>(pipeline.getHeader());
|
||||
Pipe pipe(std::move(input));
|
||||
pipe.addTransform(counting);
|
||||
|
@ -108,6 +108,9 @@ static NamesAndTypesList getColumnsList(const ASTExpressionList * columns_defini
|
||||
data_type_function->name = type_name_upper + " UNSIGNED";
|
||||
}
|
||||
|
||||
if (type_name_upper == "SET")
|
||||
data_type_function->arguments.reset();
|
||||
|
||||
/// Transforms MySQL ENUM's list of strings to ClickHouse string-integer pairs
|
||||
/// For example ENUM('a', 'b', 'c') -> ENUM('a'=1, 'b'=2, 'c'=3)
|
||||
/// Elements on a position further than 32767 are assigned negative values, starting with -32768.
|
||||
|
@ -40,7 +40,8 @@ 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"},
|
||||
{"TIMESTAMP", "DateTime"}, {"BOOLEAN", "Bool"}, {"BIT", "UInt64"}
|
||||
{"TIMESTAMP", "DateTime"}, {"BOOLEAN", "Bool"}, {"BIT", "UInt64"}, {"SET", "UInt64"},
|
||||
{"YEAR", "UInt16"}, {"TIME", "Int64"}, {"GEOMETRY", "String"}
|
||||
};
|
||||
|
||||
for (const auto & [test_type, mapped_type] : test_types)
|
||||
|
@ -19,6 +19,7 @@
|
||||
#include <base/range.h>
|
||||
#include <base/logger_useful.h>
|
||||
#include <Processors/Sources/MySQLSource.h>
|
||||
#include <boost/algorithm/string.hpp>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -145,8 +146,7 @@ namespace
|
||||
break;
|
||||
case ValueType::vtUInt64:
|
||||
{
|
||||
//we don't have enum enum_field_types definition in mysqlxx/Types.h, so we use literal values directly here.
|
||||
if (static_cast<int>(mysql_type) == 16)
|
||||
if (mysql_type == enum_field_types::MYSQL_TYPE_BIT)
|
||||
{
|
||||
size_t n = value.size();
|
||||
UInt64 val = 0UL;
|
||||
@ -175,9 +175,32 @@ namespace
|
||||
read_bytes_size += 4;
|
||||
break;
|
||||
case ValueType::vtInt64:
|
||||
assert_cast<ColumnInt64 &>(column).insertValue(value.getInt());
|
||||
read_bytes_size += 8;
|
||||
{
|
||||
if (mysql_type == enum_field_types::MYSQL_TYPE_TIME)
|
||||
{
|
||||
String time_str(value.data(), value.size());
|
||||
bool negative = time_str.starts_with("-");
|
||||
if (negative) time_str = time_str.substr(1);
|
||||
std::vector<String> hhmmss;
|
||||
boost::split(hhmmss, time_str, [](char c) { return c == ':'; });
|
||||
Int64 v = 0;
|
||||
if (hhmmss.size() == 3)
|
||||
{
|
||||
v = (std::stoi(hhmmss[0]) * 3600 + std::stoi(hhmmss[1]) * 60 + std::stold(hhmmss[2])) * 1000000;
|
||||
}
|
||||
else
|
||||
throw Exception("Unsupported value format", ErrorCodes::NOT_IMPLEMENTED);
|
||||
if (negative) v = -v;
|
||||
assert_cast<ColumnInt64 &>(column).insertValue(v);
|
||||
read_bytes_size += value.size();
|
||||
}
|
||||
else
|
||||
{
|
||||
assert_cast<ColumnInt64 &>(column).insertValue(value.getInt());
|
||||
read_bytes_size += 8;
|
||||
}
|
||||
break;
|
||||
}
|
||||
case ValueType::vtFloat32:
|
||||
assert_cast<ColumnFloat32 &>(column).insertValue(value.getDouble());
|
||||
read_bytes_size += 4;
|
||||
|
@ -1141,14 +1141,14 @@ def materialized_database_support_all_kinds_of_mysql_datatype(clickhouse_node, m
|
||||
`v19` datetime(6) DEFAULT CURRENT_TIMESTAMP(6),
|
||||
`v20` TIMESTAMP DEFAULT CURRENT_TIMESTAMP,
|
||||
`v21` TIMESTAMP(6) DEFAULT CURRENT_TIMESTAMP(6),
|
||||
/* todo support */
|
||||
# `v22` YEAR,
|
||||
# `v23` TIME,
|
||||
# `v24` TIME(3),
|
||||
# `v25` GEOMETRY,
|
||||
`v22` YEAR,
|
||||
`v23` TIME,
|
||||
`v24` TIME(6),
|
||||
`v25` GEOMETRY,
|
||||
`v26` bit(4),
|
||||
/* todo support */
|
||||
# `v27` JSON DEFAULT NULL,
|
||||
# `v28` set('a', 'c', 'f', 'd', 'e', 'b'),
|
||||
`v28` set('a', 'c', 'f', 'd', 'e', 'b'),
|
||||
`v29` mediumint(4) unsigned NOT NULL DEFAULT '0',
|
||||
`v30` varbinary(255) DEFAULT NULL COMMENT 'varbinary support',
|
||||
`v31` binary(200) DEFAULT NULL,
|
||||
@ -1158,8 +1158,9 @@ def materialized_database_support_all_kinds_of_mysql_datatype(clickhouse_node, m
|
||||
""")
|
||||
|
||||
mysql_node.query("""
|
||||
INSERT INTO test_database_datatype.t1 (v2, v3, v4, v5, v6, v7, v8, v9, v10, v11, v12, v13, v14, v15, v16, v17, v18, v19, v20, v21, v26, v29, v30, v31, v32) values
|
||||
(1, 11, 9223372036854775807, -1, 1, 11, 18446744073709551615, -1.1, 1.1, -1.111, 1.111, 1.1111, '2021-10-06', 'text', 'varchar', 'BLOB', '2021-10-06 18:32:57', '2021-10-06 18:32:57.482786', '2021-10-06 18:32:57', '2021-10-06 18:32:57.482786', b'1010', 11, 'varbinary', 'binary', 'RED');
|
||||
INSERT INTO test_database_datatype.t1 (v2, v3, v4, v5, v6, v7, v8, v9, v10, v11, v12, v13, v14, v15, v16, v17, v18, v19, v20, v21, v22, v23, v24, v25, v26, v28, v29, v30, v31, v32) values
|
||||
(1, 11, 9223372036854775807, -1, 1, 11, 18446744073709551615, -1.1, 1.1, -1.111, 1.111, 1.1111, '2021-10-06', 'text', 'varchar', 'BLOB', '2021-10-06 18:32:57',
|
||||
'2021-10-06 18:32:57.482786', '2021-10-06 18:32:57', '2021-10-06 18:32:57.482786', '2021', '838:59:59', '838:59:59.000000', ST_GeometryFromText('point(0.0 0.0)'), b'1010', 'a', 11, 'varbinary', 'binary', 'RED');
|
||||
""")
|
||||
clickhouse_node.query(
|
||||
"CREATE DATABASE test_database_datatype ENGINE = MaterializeMySQL('{}:3306', 'test_database_datatype', 'root', 'clickhouse')".format(
|
||||
@ -1167,14 +1168,18 @@ def materialized_database_support_all_kinds_of_mysql_datatype(clickhouse_node, m
|
||||
|
||||
check_query(clickhouse_node, "SELECT name FROM system.tables WHERE database = 'test_database_datatype'", "t1\n")
|
||||
# full synchronization check
|
||||
check_query(clickhouse_node, "SELECT v1, v2, v3, v4, v5, v6, v7, v8, v9, v10, v11, v12, v13, v14, v15, v16, v17, v18, v19, v20, v21, v26, v29, v30, v32 FROM test_database_datatype.t1 FORMAT TSV",
|
||||
"1\t1\t11\t9223372036854775807\t-1\t1\t11\t18446744073709551615\t-1.1\t1.1\t-1.111\t1.111\t1.1111\t2021-10-06\ttext\tvarchar\tBLOB\t2021-10-06 18:32:57\t2021-10-06 18:32:57.482786\t2021-10-06 18:32:57\t2021-10-06 18:32:57.482786\t10\t11\tvarbinary\tRED\n")
|
||||
check_query(clickhouse_node, "SELECT v1, v2, v3, v4, v5, v6, v7, v8, v9, v10, v11, v12, v13, v14, v15, v16, v17, v18, v19, v20, v21, v22, v23, v24, hex(v25), v26, v28, v29, v30, v32 FROM test_database_datatype.t1 FORMAT TSV",
|
||||
"1\t1\t11\t9223372036854775807\t-1\t1\t11\t18446744073709551615\t-1.1\t1.1\t-1.111\t1.111\t1.1111\t2021-10-06\ttext\tvarchar\tBLOB\t2021-10-06 18:32:57\t2021-10-06 18:32:57.482786\t2021-10-06 18:32:57" +
|
||||
"\t2021-10-06 18:32:57.482786\t2021\t3020399000000\t3020399000000\t00000000010100000000000000000000000000000000000000\t10\t1\t11\tvarbinary\tRED\n")
|
||||
|
||||
mysql_node.query("""
|
||||
INSERT INTO test_database_datatype.t1 (v2, v3, v4, v5, v6, v7, v8, v9, v10, v11, v12, v13, v14, v15, v16, v17, v18, v19, v20, v21, v26, v29, v30, v31, v32) values
|
||||
(2, 22, 9223372036854775807, -2, 2, 22, 18446744073709551615, -2.2, 2.2, -2.22, 2.222, 2.2222, '2021-10-07', 'text', 'varchar', 'BLOB', '2021-10-07 18:32:57', '2021-10-07 18:32:57.482786', '2021-10-07 18:32:57', '2021-10-07 18:32:57.482786', b'1011', 22, 'varbinary', 'binary', 'GREEN' );
|
||||
INSERT INTO test_database_datatype.t1 (v2, v3, v4, v5, v6, v7, v8, v9, v10, v11, v12, v13, v14, v15, v16, v17, v18, v19, v20, v21, v22, v23, v24, v25, v26, v28, v29, v30, v31, v32) values
|
||||
(2, 22, 9223372036854775807, -2, 2, 22, 18446744073709551615, -2.2, 2.2, -2.22, 2.222, 2.2222, '2021-10-07', 'text', 'varchar', 'BLOB', '2021-10-07 18:32:57',
|
||||
'2021-10-07 18:32:57.482786', '2021-10-07 18:32:57', '2021-10-07 18:32:57.482786', '2021', '-838:59:59', '-12:59:58.000001', ST_GeometryFromText('point(120.153576 30.287459)'), b'1011', 'a,c', 22, 'varbinary', 'binary', 'GREEN' );
|
||||
""")
|
||||
# increment synchronization check
|
||||
check_query(clickhouse_node, "SELECT v1, v2, v3, v4, v5, v6, v7, v8, v9, v10, v11, v12, v13, v14, v15, v16, v17, v18, v19, v20, v21, v26, v29, v30, v32 FROM test_database_datatype.t1 ORDER BY v1 FORMAT TSV",
|
||||
"1\t1\t11\t9223372036854775807\t-1\t1\t11\t18446744073709551615\t-1.1\t1.1\t-1.111\t1.111\t1.1111\t2021-10-06\ttext\tvarchar\tBLOB\t2021-10-06 18:32:57\t2021-10-06 18:32:57.482786\t2021-10-06 18:32:57\t2021-10-06 18:32:57.482786\t10\t11\tvarbinary\tRED\n" +
|
||||
"2\t2\t22\t9223372036854775807\t-2\t2\t22\t18446744073709551615\t-2.2\t2.2\t-2.22\t2.222\t2.2222\t2021-10-07\ttext\tvarchar\tBLOB\t2021-10-07 18:32:57\t2021-10-07 18:32:57.482786\t2021-10-07 18:32:57\t2021-10-07 18:32:57.482786\t11\t22\tvarbinary\tGREEN\n")
|
||||
check_query(clickhouse_node, "SELECT v1, v2, v3, v4, v5, v6, v7, v8, v9, v10, v11, v12, v13, v14, v15, v16, v17, v18, v19, v20, v21, v22, v23, v24, hex(v25), v26, v28, v29, v30, v32 FROM test_database_datatype.t1 FORMAT TSV",
|
||||
"1\t1\t11\t9223372036854775807\t-1\t1\t11\t18446744073709551615\t-1.1\t1.1\t-1.111\t1.111\t1.1111\t2021-10-06\ttext\tvarchar\tBLOB\t2021-10-06 18:32:57\t2021-10-06 18:32:57.482786\t2021-10-06 18:32:57\t2021-10-06 18:32:57.482786" +
|
||||
"\t2021\t3020399000000\t3020399000000\t00000000010100000000000000000000000000000000000000\t10\t1\t11\tvarbinary\tRED\n" +
|
||||
"2\t2\t22\t9223372036854775807\t-2\t2\t22\t18446744073709551615\t-2.2\t2.2\t-2.22\t2.222\t2.2222\t2021-10-07\ttext\tvarchar\tBLOB\t2021-10-07 18:32:57\t2021-10-07 18:32:57.482786\t2021-10-07 18:32:57\t2021-10-07 18:32:57.482786" +
|
||||
"\t2021\t-3020399000000\t-46798000001\t000000000101000000D55C6E30D4095E40DCF0BBE996493E40\t11\t3\t22\tvarbinary\tGREEN\n")
|
||||
|
Loading…
Reference in New Issue
Block a user