mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 16:42:05 +00:00
MaterializedMySQL: Introduce charset conversion
This commit is contained in:
parent
c68de09e0a
commit
6ae5207819
@ -43,11 +43,12 @@ void LimitedReadPacket::readPayloadWithUnpacked(ReadBuffer & in)
|
||||
IMySQLReadPacket::readPayloadWithUnpacked(limited);
|
||||
}
|
||||
|
||||
uint64_t readLengthEncodedNumber(ReadBuffer & buffer)
|
||||
uint64_t readLengthEncodedNumber(ReadBuffer & buffer, UInt16 & bytes_read)
|
||||
{
|
||||
char c{};
|
||||
uint64_t buf = 0;
|
||||
buffer.readStrict(c);
|
||||
bytes_read = 1;
|
||||
auto cc = static_cast<uint8_t>(c);
|
||||
switch (cc)
|
||||
{
|
||||
@ -56,12 +57,15 @@ uint64_t readLengthEncodedNumber(ReadBuffer & buffer)
|
||||
break;
|
||||
case 0xfc:
|
||||
buffer.readStrict(reinterpret_cast<char *>(&buf), 2);
|
||||
bytes_read += 2;
|
||||
break;
|
||||
case 0xfd:
|
||||
buffer.readStrict(reinterpret_cast<char *>(&buf), 3);
|
||||
bytes_read += 3;
|
||||
break;
|
||||
case 0xfe:
|
||||
buffer.readStrict(reinterpret_cast<char *>(&buf), 8);
|
||||
bytes_read += 8;
|
||||
break;
|
||||
default:
|
||||
return cc;
|
||||
@ -69,6 +73,12 @@ uint64_t readLengthEncodedNumber(ReadBuffer & buffer)
|
||||
return buf;
|
||||
}
|
||||
|
||||
uint64_t readLengthEncodedNumber(ReadBuffer & buffer)
|
||||
{
|
||||
UInt16 bytes_read = 0;
|
||||
return readLengthEncodedNumber(buffer, bytes_read);
|
||||
}
|
||||
|
||||
void readLengthEncodedString(String & s, ReadBuffer & buffer)
|
||||
{
|
||||
uint64_t len = readLengthEncodedNumber(buffer);
|
||||
|
@ -34,6 +34,7 @@ public:
|
||||
};
|
||||
|
||||
uint64_t readLengthEncodedNumber(ReadBuffer & buffer);
|
||||
uint64_t readLengthEncodedNumber(ReadBuffer & buffer, UInt16 & bytes_read);
|
||||
void readLengthEncodedString(String & s, ReadBuffer & buffer);
|
||||
|
||||
}
|
||||
|
301
src/Core/MySQL/MySQLCharset.cpp
Normal file
301
src/Core/MySQL/MySQLCharset.cpp
Normal file
@ -0,0 +1,301 @@
|
||||
#include "MySQLCharset.h"
|
||||
#include "config.h"
|
||||
#include <iostream>
|
||||
#include <Common/Exception.h>
|
||||
|
||||
#if USE_ICU
|
||||
#include <unicode/ucnv.h>
|
||||
#define CHUNK_SIZE 1024
|
||||
static const char * TARGET_CHARSET = "utf8";
|
||||
#endif
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int UNKNOWN_EXCEPTION;
|
||||
}
|
||||
|
||||
const std::unordered_map<Int32, String> MySQLCharset::charsets
|
||||
= {
|
||||
{1, "big5"},
|
||||
{2, "latin2"},
|
||||
{3, "dec8"},
|
||||
{4, "cp850"},
|
||||
{5, "latin1"},
|
||||
{6, "hp8"},
|
||||
{7, "koi8r"},
|
||||
{8, "latin1"},
|
||||
{9, "latin2"},
|
||||
{10, "swe7"},
|
||||
{11, "ascii"},
|
||||
{12, "ujis"},
|
||||
{13, "sjis"},
|
||||
{14, "cp1251"},
|
||||
{15, "latin1"},
|
||||
{16, "hebrew"},
|
||||
{18, "tis620"},
|
||||
{19, "euckr"},
|
||||
{20, "latin7"},
|
||||
{21, "latin2"},
|
||||
{22, "koi8u"},
|
||||
{23, "cp1251"},
|
||||
{24, "gb2312"},
|
||||
{25, "greek"},
|
||||
{26, "cp1250"},
|
||||
{27, "latin2"},
|
||||
{28, "gbk"},
|
||||
{29, "cp1257"},
|
||||
{30, "latin5"},
|
||||
{31, "latin1"},
|
||||
{32, "armscii8"},
|
||||
{34, "cp1250"},
|
||||
{35, "ucs2"},
|
||||
{36, "cp866"},
|
||||
{37, "keybcs2"},
|
||||
{38, "macce"},
|
||||
{39, "macroman"},
|
||||
{40, "cp852"},
|
||||
{41, "latin7"},
|
||||
{42, "latin7"},
|
||||
{43, "macce"},
|
||||
{44, "cp1250"},
|
||||
{47, "latin1"},
|
||||
{48, "latin1"},
|
||||
{49, "latin1"},
|
||||
{50, "cp1251"},
|
||||
{51, "cp1251"},
|
||||
{52, "cp1251"},
|
||||
{53, "macroman"},
|
||||
{54, "utf16"},
|
||||
{55, "utf16"},
|
||||
{56, "utf16le"},
|
||||
{57, "cp1256"},
|
||||
{58, "cp1257"},
|
||||
{59, "cp1257"},
|
||||
{60, "utf32"},
|
||||
{61, "utf32"},
|
||||
{62, "utf16le"},
|
||||
{64, "armscii8"},
|
||||
{65, "ascii"},
|
||||
{66, "cp1250"},
|
||||
{67, "cp1256"},
|
||||
{68, "cp866"},
|
||||
{69, "dec8"},
|
||||
{70, "greek"},
|
||||
{71, "hebrew"},
|
||||
{72, "hp8"},
|
||||
{73, "keybcs2"},
|
||||
{74, "koi8r"},
|
||||
{75, "koi8u"},
|
||||
{77, "latin2"},
|
||||
{78, "latin5"},
|
||||
{79, "latin7"},
|
||||
{80, "cp850"},
|
||||
{81, "cp852"},
|
||||
{82, "swe7"},
|
||||
{84, "big5"},
|
||||
{85, "euckr"},
|
||||
{86, "gb2312"},
|
||||
{87, "gbk"},
|
||||
{88, "sjis"},
|
||||
{89, "tis620"},
|
||||
{90, "ucs2"},
|
||||
{91, "ujis"},
|
||||
{92, "geostd8"},
|
||||
{93, "geostd8"},
|
||||
{94, "latin1"},
|
||||
{95, "cp932"},
|
||||
{96, "cp932"},
|
||||
{97, "eucjpms"},
|
||||
{98, "eucjpms"},
|
||||
{99, "cp1250"},
|
||||
{101, "utf16"},
|
||||
{102, "utf16"},
|
||||
{103, "utf16"},
|
||||
{104, "utf16"},
|
||||
{105, "utf16"},
|
||||
{106, "utf16"},
|
||||
{107, "utf16"},
|
||||
{108, "utf16"},
|
||||
{109, "utf16"},
|
||||
{110, "utf16"},
|
||||
{111, "utf16"},
|
||||
{112, "utf16"},
|
||||
{113, "utf16"},
|
||||
{114, "utf16"},
|
||||
{115, "utf16"},
|
||||
{116, "utf16"},
|
||||
{117, "utf16"},
|
||||
{118, "utf16"},
|
||||
{119, "utf16"},
|
||||
{120, "utf16"},
|
||||
{121, "utf16"},
|
||||
{122, "utf16"},
|
||||
{123, "utf16"},
|
||||
{124, "utf16"},
|
||||
{128, "ucs2"},
|
||||
{129, "ucs2"},
|
||||
{130, "ucs2"},
|
||||
{131, "ucs2"},
|
||||
{132, "ucs2"},
|
||||
{133, "ucs2"},
|
||||
{134, "ucs2"},
|
||||
{135, "ucs2"},
|
||||
{136, "ucs2"},
|
||||
{137, "ucs2"},
|
||||
{138, "ucs2"},
|
||||
{139, "ucs2"},
|
||||
{140, "ucs2"},
|
||||
{141, "ucs2"},
|
||||
{142, "ucs2"},
|
||||
{143, "ucs2"},
|
||||
{144, "ucs2"},
|
||||
{145, "ucs2"},
|
||||
{146, "ucs2"},
|
||||
{147, "ucs2"},
|
||||
{148, "ucs2"},
|
||||
{149, "ucs2"},
|
||||
{150, "ucs2"},
|
||||
{151, "ucs2"},
|
||||
{159, "ucs2"},
|
||||
{160, "utf32"},
|
||||
{161, "utf32"},
|
||||
{162, "utf32"},
|
||||
{163, "utf32"},
|
||||
{164, "utf32"},
|
||||
{165, "utf32"},
|
||||
{166, "utf32"},
|
||||
{167, "utf32"},
|
||||
{168, "utf32"},
|
||||
{169, "utf32"},
|
||||
{170, "utf32"},
|
||||
{171, "utf32"},
|
||||
{172, "utf32"},
|
||||
{173, "utf32"},
|
||||
{174, "utf32"},
|
||||
{175, "utf32"},
|
||||
{176, "utf32"},
|
||||
{177, "utf32"},
|
||||
{178, "utf32"},
|
||||
{179, "utf32"},
|
||||
{180, "utf32"},
|
||||
{181, "utf32"},
|
||||
{182, "utf32"},
|
||||
{183, "utf32"},
|
||||
{248, "gb18030"},
|
||||
{249, "gb18030"},
|
||||
{250, "gb18030"}
|
||||
};
|
||||
|
||||
MySQLCharset::~MySQLCharset()
|
||||
{
|
||||
#if USE_ICU
|
||||
std::lock_guard lock(mutex);
|
||||
for (auto & conv : conv_cache)
|
||||
{
|
||||
ucnv_close(conv.second);
|
||||
}
|
||||
conv_cache.clear();
|
||||
#endif
|
||||
}
|
||||
|
||||
bool MySQLCharset::needConvert(UInt32 id)
|
||||
{
|
||||
return charsets.contains(id);
|
||||
}
|
||||
|
||||
String MySQLCharset::getCharsetFromId(UInt32 id)
|
||||
{
|
||||
return charsets.at(id);
|
||||
}
|
||||
|
||||
UConverter * MySQLCharset::getCachedConverter(const String & charset [[maybe_unused]])
|
||||
{
|
||||
UConverter * conv = nullptr;
|
||||
#if USE_ICU
|
||||
UErrorCode error = U_ZERO_ERROR;
|
||||
/// Get conv from cache
|
||||
auto result = conv_cache.find(charset);
|
||||
if (result != conv_cache.end())
|
||||
{
|
||||
conv = result->second;
|
||||
//reset to init state
|
||||
ucnv_reset(conv);
|
||||
}
|
||||
else
|
||||
{
|
||||
conv = ucnv_open(charset.c_str(), &error);
|
||||
if (error != U_ZERO_ERROR)
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::UNKNOWN_EXCEPTION, "MySQLCharset::getCachedConveter: ucnv_open failed, error={}", std::to_string(error));
|
||||
}
|
||||
conv_cache[charset.c_str()] = conv;
|
||||
}
|
||||
#endif
|
||||
return conv;
|
||||
}
|
||||
|
||||
Int32 MySQLCharset::convertFromId(UInt32 id [[maybe_unused]], String & to, const String & from)
|
||||
{
|
||||
#if USE_ICU
|
||||
std::lock_guard lock(mutex);
|
||||
UErrorCode error = U_ZERO_ERROR;
|
||||
String source_charset = getCharsetFromId(id);
|
||||
to.clear();
|
||||
if (source_charset.empty())
|
||||
{
|
||||
return U_ILLEGAL_ARGUMENT_ERROR;
|
||||
}
|
||||
|
||||
UChar pivot_buf[CHUNK_SIZE]; // stream mode must use this buf
|
||||
char target_buf[CHUNK_SIZE];
|
||||
UChar * pivot;
|
||||
UChar * pivot2;
|
||||
UConverter * in_conv;
|
||||
UConverter * out_conv;
|
||||
char * cur_target;
|
||||
const char * source_end;
|
||||
const char * target_end;
|
||||
|
||||
size_t source_len = from.size();
|
||||
const char * source = from.data();
|
||||
source_end = source + source_len;
|
||||
|
||||
out_conv = getCachedConverter(TARGET_CHARSET);
|
||||
in_conv = getCachedConverter(source_charset);
|
||||
pivot = pivot_buf;
|
||||
pivot2 = pivot_buf;
|
||||
|
||||
target_end = target_buf + CHUNK_SIZE;
|
||||
do
|
||||
{
|
||||
error = U_ZERO_ERROR;
|
||||
cur_target = target_buf;
|
||||
ucnv_convertEx(
|
||||
out_conv,
|
||||
in_conv,
|
||||
&cur_target,
|
||||
target_end,
|
||||
&source,
|
||||
source_end,
|
||||
pivot_buf,
|
||||
&pivot,
|
||||
&pivot2,
|
||||
pivot_buf + CHUNK_SIZE,
|
||||
false,
|
||||
true,
|
||||
&error);
|
||||
to.append(target_buf, cur_target - target_buf);
|
||||
} while (error == U_BUFFER_OVERFLOW_ERROR);
|
||||
|
||||
return error;
|
||||
#else
|
||||
to = from;
|
||||
return 0;
|
||||
#endif
|
||||
}
|
||||
|
||||
}
|
26
src/Core/MySQL/MySQLCharset.h
Normal file
26
src/Core/MySQL/MySQLCharset.h
Normal file
@ -0,0 +1,26 @@
|
||||
#pragma once
|
||||
#include <unordered_map>
|
||||
#include <base/types.h>
|
||||
#include <boost/noncopyable.hpp>
|
||||
#include <mutex>
|
||||
|
||||
struct UConverter;
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class MySQLCharset final : boost::noncopyable
|
||||
{
|
||||
public:
|
||||
~MySQLCharset();
|
||||
String getCharsetFromId(UInt32 id);
|
||||
Int32 convertFromId(UInt32 id, String & to, const String & from);
|
||||
bool needConvert(UInt32 id);
|
||||
private:
|
||||
std::mutex mutex;
|
||||
std::unordered_map<String, UConverter *> conv_cache;
|
||||
UConverter * getCachedConverter(const String & charset);
|
||||
static const std::unordered_map<Int32, String> charsets;
|
||||
};
|
||||
|
||||
using MySQLCharsetPtr = std::shared_ptr<MySQLCharset>;
|
||||
}
|
@ -176,9 +176,9 @@ namespace MySQLReplication
|
||||
size_t null_bitmap_size = (column_count + 7) / 8;
|
||||
readBitmap(payload, null_bitmap, null_bitmap_size);
|
||||
|
||||
/// Ignore MySQL 8.0 optional metadata fields.
|
||||
/// Parse MySQL 8.0 optional metadata fields.
|
||||
/// https://mysqlhighavailability.com/more-metadata-is-written-into-binary-log/
|
||||
payload.ignoreAll();
|
||||
parseOptionalMetaField(payload);
|
||||
}
|
||||
|
||||
/// Types that do not used in the binlog event:
|
||||
@ -252,6 +252,118 @@ namespace MySQLReplication
|
||||
}
|
||||
}
|
||||
|
||||
void TableMapEvent::parseOptionalMetaField(ReadBuffer & payload)
|
||||
{
|
||||
char type = 0;
|
||||
while (payload.read(type))
|
||||
{
|
||||
UInt64 len = readLengthEncodedNumber(payload);
|
||||
if (len == 0)
|
||||
{
|
||||
payload.ignoreAll();
|
||||
return;
|
||||
}
|
||||
switch (type)
|
||||
{
|
||||
/// It may be useful, parse later
|
||||
case SIGNEDNESS:
|
||||
payload.ignore(len);
|
||||
break;
|
||||
case DEFAULT_CHARSET:
|
||||
{
|
||||
UInt32 total_read = 0;
|
||||
UInt16 once_read = 0;
|
||||
default_charset = static_cast<UInt32>(readLengthEncodedNumber(payload, once_read));
|
||||
total_read += once_read;
|
||||
while (total_read < len)
|
||||
{
|
||||
UInt32 col_index = static_cast<UInt32>(readLengthEncodedNumber(payload, once_read));
|
||||
total_read += once_read;
|
||||
UInt32 col_charset = static_cast<UInt32>(readLengthEncodedNumber(payload, once_read));
|
||||
total_read += once_read;
|
||||
default_charset_pairs.emplace(col_index, col_charset);
|
||||
}
|
||||
break;
|
||||
}
|
||||
case COLUMN_CHARSET:
|
||||
{
|
||||
UInt32 total_read = 0;
|
||||
UInt16 once_read = 0;
|
||||
while (total_read < len)
|
||||
{
|
||||
UInt32 collation_id = static_cast<UInt32>(readLengthEncodedNumber(payload, once_read));
|
||||
column_charset.emplace_back(collation_id);
|
||||
total_read += once_read;
|
||||
}
|
||||
break;
|
||||
}
|
||||
case COLUMN_NAME:
|
||||
payload.ignore(len);
|
||||
break;
|
||||
case SET_STR_VALUE:
|
||||
case GEOMETRY_TYPE:
|
||||
case SIMPLE_PRIMARY_KEY:
|
||||
case PRIMARY_KEY_WITH_PREFIX:
|
||||
case ENUM_AND_SET_DEFAULT_CHARSET:
|
||||
case COLUMN_VISIBILITY:
|
||||
default:
|
||||
payload.ignore(len);
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
UInt32 TableMapEvent::getColumnCharsetId(UInt32 column_index)
|
||||
{
|
||||
if (!column_charset.empty())
|
||||
{
|
||||
UInt32 str_index = 0xFFFFFFFF;
|
||||
/// Calc the index in the column_charset
|
||||
for (UInt32 i = 0; i <= column_index; ++i)
|
||||
{
|
||||
switch (column_type[i])
|
||||
{
|
||||
case MYSQL_TYPE_STRING:
|
||||
case MYSQL_TYPE_VAR_STRING:
|
||||
case MYSQL_TYPE_VARCHAR:
|
||||
case MYSQL_TYPE_BLOB:
|
||||
++str_index;
|
||||
break;
|
||||
default:
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
if (str_index != 0xFFFFFFFF && str_index < column_charset.size())
|
||||
{
|
||||
return column_charset[str_index];
|
||||
}
|
||||
}
|
||||
else if (!default_charset_pairs.empty())
|
||||
{
|
||||
UInt32 str_index = 0xFFFFFFFF;
|
||||
for (UInt32 i = 0; i <= column_index; ++i)
|
||||
{
|
||||
switch (column_type[i])
|
||||
{
|
||||
case MYSQL_TYPE_STRING:
|
||||
case MYSQL_TYPE_VAR_STRING:
|
||||
case MYSQL_TYPE_VARCHAR:
|
||||
case MYSQL_TYPE_BLOB:
|
||||
++str_index;
|
||||
break;
|
||||
default:
|
||||
break;
|
||||
}
|
||||
}
|
||||
if (default_charset_pairs.contains(str_index))
|
||||
{
|
||||
return default_charset_pairs[str_index];
|
||||
}
|
||||
}
|
||||
return default_charset;
|
||||
}
|
||||
|
||||
void TableMapEvent::dump(WriteBuffer & out) const
|
||||
{
|
||||
header.dump(out);
|
||||
@ -308,6 +420,22 @@ namespace MySQLReplication
|
||||
}
|
||||
}
|
||||
|
||||
static inline String convertCharsetIfNeeded(
|
||||
const std::shared_ptr<TableMapEvent> & table_map,
|
||||
UInt32 i,
|
||||
const String & val)
|
||||
{
|
||||
const auto collation_id = table_map->getColumnCharsetId(i);
|
||||
if (table_map->charset_ptr->needConvert(collation_id))
|
||||
{
|
||||
String target;
|
||||
auto err = table_map->charset_ptr->convertFromId(collation_id, target, val);
|
||||
if (err == 0)
|
||||
return target;
|
||||
}
|
||||
return val;
|
||||
}
|
||||
|
||||
/// Types that do not used in the binlog event:
|
||||
/// MYSQL_TYPE_SET
|
||||
/// MYSQL_TYPE_TINY_BLOB
|
||||
@ -716,7 +844,7 @@ namespace MySQLReplication
|
||||
String val;
|
||||
val.resize(size);
|
||||
payload.readStrict(reinterpret_cast<char *>(val.data()), size);
|
||||
row.push_back(Field{String{val}});
|
||||
row.emplace_back(Field{convertCharsetIfNeeded(table_map, i, val)});
|
||||
break;
|
||||
}
|
||||
case MYSQL_TYPE_STRING:
|
||||
@ -734,7 +862,7 @@ namespace MySQLReplication
|
||||
String val;
|
||||
val.resize(size);
|
||||
payload.readStrict(reinterpret_cast<char *>(val.data()), size);
|
||||
row.push_back(Field{String{val}});
|
||||
row.emplace_back(Field{convertCharsetIfNeeded(table_map, i, val)});
|
||||
break;
|
||||
}
|
||||
case MYSQL_TYPE_GEOMETRY:
|
||||
@ -766,7 +894,10 @@ namespace MySQLReplication
|
||||
String val;
|
||||
val.resize(size);
|
||||
payload.readStrict(reinterpret_cast<char *>(val.data()), size);
|
||||
row.push_back(Field{String{val}});
|
||||
row.emplace_back(Field{
|
||||
field_type == MYSQL_TYPE_BLOB
|
||||
? convertCharsetIfNeeded(table_map, i, val)
|
||||
: val});
|
||||
break;
|
||||
}
|
||||
default:
|
||||
@ -966,7 +1097,7 @@ namespace MySQLReplication
|
||||
map_event_header.parse(event_payload);
|
||||
if (doReplicate(map_event_header.schema, map_event_header.table))
|
||||
{
|
||||
event = std::make_shared<TableMapEvent>(std::move(event_header), map_event_header);
|
||||
event = std::make_shared<TableMapEvent>(std::move(event_header), map_event_header, flavor_charset);
|
||||
event->parseEvent(event_payload);
|
||||
auto table_map = std::static_pointer_cast<TableMapEvent>(event);
|
||||
table_maps[table_map->table_id] = table_map;
|
||||
|
@ -2,6 +2,7 @@
|
||||
#include <Core/Field.h>
|
||||
#include <Core/MySQL/PacketsReplication.h>
|
||||
#include <Core/MySQL/MySQLGtid.h>
|
||||
#include <Core/MySQL/MySQLCharset.h>
|
||||
#include <base/types.h>
|
||||
#include <IO/ReadBuffer.h>
|
||||
#include <IO/WriteBuffer.h>
|
||||
@ -436,9 +437,24 @@ namespace MySQLReplication
|
||||
UInt32 column_count;
|
||||
std::vector<UInt8> column_type;
|
||||
std::vector<UInt16> column_meta;
|
||||
/// Character set of string columns
|
||||
std::vector<UInt32> column_charset;
|
||||
/// Character set of string columns,
|
||||
/// optimized to minimize space when many
|
||||
/// columns have the same charset
|
||||
UInt32 default_charset = 255; /// utf8mb4_0900_ai_ci
|
||||
std::unordered_map<UInt32, UInt32> default_charset_pairs;
|
||||
/// Points to flavor_charset object
|
||||
MySQLCharsetPtr charset_ptr;
|
||||
Bitmap null_bitmap;
|
||||
|
||||
TableMapEvent(EventHeader && header_, const TableMapEventHeader & map_event_header) : EventBase(std::move(header_)), column_count(0)
|
||||
TableMapEvent(
|
||||
EventHeader && header_,
|
||||
const TableMapEventHeader & map_event_header,
|
||||
const MySQLCharsetPtr & charset_ptr_)
|
||||
: EventBase(std::move(header_))
|
||||
, column_count(0)
|
||||
, charset_ptr(charset_ptr_)
|
||||
{
|
||||
table_id = map_event_header.table_id;
|
||||
flags = map_event_header.flags;
|
||||
@ -448,10 +464,52 @@ namespace MySQLReplication
|
||||
table = map_event_header.table;
|
||||
}
|
||||
void dump(WriteBuffer & out) const override;
|
||||
UInt32 getColumnCharsetId(UInt32 column_index);
|
||||
/// https://mysqlhighavailability.com/more-metadata-is-written-into-binary-log/
|
||||
/// https://github.com/mysql/mysql-server/blob/8.0/libbinlogevents/include/rows_event.h#L50
|
||||
/// DEFAULT_CHARSET and COLUMN_CHARSET don't appear together, and
|
||||
/// ENUM_AND_SET_DEFAULT_CHARSET and ENUM_AND_SET_COLUMN_CHARSET don't appear together.
|
||||
enum OptionalMetaType : char
|
||||
{
|
||||
/// UNSIGNED flag of numeric columns
|
||||
SIGNEDNESS = 1,
|
||||
/// Character set of string columns, optimized to
|
||||
/// minimize space when many columns have the
|
||||
/// same charset
|
||||
DEFAULT_CHARSET,
|
||||
/// Character set of string columns, optimized to
|
||||
/// minimize space when columns have many
|
||||
/// different charsets
|
||||
COLUMN_CHARSET,
|
||||
COLUMN_NAME,
|
||||
/// String value of SET columns
|
||||
SET_STR_VALUE,
|
||||
/// String value of ENUM columns
|
||||
ENUM_STR_VALUE,
|
||||
/// Real type of geometry columns
|
||||
GEOMETRY_TYPE,
|
||||
/// Primary key without prefix
|
||||
SIMPLE_PRIMARY_KEY,
|
||||
/// Primary key with prefix
|
||||
PRIMARY_KEY_WITH_PREFIX,
|
||||
/// Character set of enum and set
|
||||
/// columns, optimized to minimize
|
||||
/// space when many columns have the
|
||||
/// same charset
|
||||
ENUM_AND_SET_DEFAULT_CHARSET,
|
||||
/// Character set of enum and set
|
||||
/// columns, optimized to minimize
|
||||
/// space when many columns have the
|
||||
/// same charset
|
||||
ENUM_AND_SET_COLUMN_CHARSET,
|
||||
/// Flag to indicate column visibility attribute
|
||||
COLUMN_VISIBILITY
|
||||
};
|
||||
|
||||
protected:
|
||||
void parseImpl(ReadBuffer & payload) override;
|
||||
void parseMeta(String meta);
|
||||
void parseOptionalMetaField(ReadBuffer & payload);
|
||||
};
|
||||
|
||||
enum RowsEventFlags
|
||||
@ -598,6 +656,7 @@ namespace MySQLReplication
|
||||
std::unordered_set<String> replicate_tables;
|
||||
std::map<UInt64, std::shared_ptr<TableMapEvent> > table_maps;
|
||||
size_t checksum_signature_length = 4;
|
||||
MySQLCharsetPtr flavor_charset = std::make_shared<MySQLCharset>();
|
||||
|
||||
bool doReplicate(UInt64 table_id);
|
||||
bool doReplicate(const String & db, const String & table_name);
|
||||
|
351
src/Core/tests/gtest_charset_conv.cpp
Normal file
351
src/Core/tests/gtest_charset_conv.cpp
Normal file
@ -0,0 +1,351 @@
|
||||
#include <Core/MySQL/MySQLCharset.h>
|
||||
#include <gtest/gtest.h>
|
||||
#include <cstdio>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct CheckResult
|
||||
{
|
||||
Int32 id;
|
||||
String name;
|
||||
bool need_convert;
|
||||
};
|
||||
|
||||
TEST(CharsetTest, CharsetTest)
|
||||
{
|
||||
MySQLCharset charset;
|
||||
UInt32 big5_id = 1;
|
||||
UInt32 gbk_id = 28;
|
||||
UInt32 gb2312_id = 24;
|
||||
UInt32 utf8mb4_ai_ci_id = 255;
|
||||
EXPECT_TRUE(charset.needConvert(big5_id));
|
||||
EXPECT_TRUE(charset.needConvert(gbk_id));
|
||||
EXPECT_TRUE(charset.needConvert(gb2312_id));
|
||||
EXPECT_FALSE(charset.needConvert(utf8mb4_ai_ci_id));
|
||||
EXPECT_FALSE(charset.needConvert(0));
|
||||
EXPECT_FALSE(charset.needConvert(1000));
|
||||
|
||||
EXPECT_EQ(charset.getCharsetFromId(big5_id), String("big5"));
|
||||
EXPECT_EQ(charset.getCharsetFromId(gbk_id), String("gbk"));
|
||||
EXPECT_EQ(charset.getCharsetFromId(gb2312_id), String("gb2312"));
|
||||
}
|
||||
|
||||
TEST(CharsetTest, ConvTest)
|
||||
{
|
||||
MySQLCharset charset;
|
||||
UInt32 big5_id = 1;
|
||||
UInt32 gbk_id = 28;
|
||||
UInt32 gb2312_id = 24;
|
||||
Int32 error = 0;
|
||||
String source("\xc4\xe3\xba\xc3"); // gbk "你好"
|
||||
String target;
|
||||
String expect("\xe4\xbd\xa0\xe5\xa5\xbd");
|
||||
|
||||
error = charset.convertFromId(gbk_id, target, source);
|
||||
EXPECT_EQ(error, 0);
|
||||
EXPECT_TRUE(target == expect);
|
||||
|
||||
error = charset.convertFromId(gb2312_id, target, source);
|
||||
EXPECT_EQ(error, 0);
|
||||
EXPECT_TRUE(target == expect);
|
||||
|
||||
source.assign("\xa7\x41\xa6\x6e"); // big5 "你好"
|
||||
error = charset.convertFromId(big5_id, target, source);
|
||||
EXPECT_EQ(error, 0);
|
||||
EXPECT_TRUE(target == expect);
|
||||
}
|
||||
|
||||
TEST(CharsetTest, FullCharsetCheck)
|
||||
{
|
||||
CheckResult result[] =
|
||||
{
|
||||
{1, "big5", true}, // "big5_chinese_ci",
|
||||
{2, "latin2", true}, // "latin2_czech_cs",
|
||||
{3, "dec8", true}, // "dec8_swedish_ci",
|
||||
{4, "cp850", true}, // "cp850_general_ci",
|
||||
{5, "latin1", true}, // "latin1_german1_ci",
|
||||
{6, "hp8", true}, // "hp8_english_ci",
|
||||
{7, "koi8r", true}, // "koi8r_general_ci",
|
||||
{8, "latin1", true}, // "latin1_swedish_ci",
|
||||
{9, "latin2", true}, // "latin2_general_ci",
|
||||
{10, "swe7", true}, // "swe7_swedish_ci",
|
||||
{11, "ascii", true}, // "ascii_general_ci",
|
||||
{12, "ujis", true}, // "ujis_japanese_ci",
|
||||
{13, "sjis", true}, // "sjis_japanese_ci",
|
||||
{14, "cp1251", true}, // "cp1251_bulgarian_ci",
|
||||
{15, "latin1", true}, // "latin1_danish_ci",
|
||||
{16, "hebrew", true}, // "hebrew_general_ci",
|
||||
{18, "tis620", true}, // "tis620_thai_ci",
|
||||
{19, "euckr", true}, // "euckr_korean_ci",
|
||||
{20, "latin7", true}, // "latin7_estonian_cs",
|
||||
{21, "latin2", true}, // "latin2_hungarian_ci",
|
||||
{22, "koi8u", true}, // "koi8u_general_ci",
|
||||
{23, "cp1251", true}, // "cp1251_ukrainian_ci",
|
||||
{24, "gb2312", true}, // "gb2312_chinese_ci",
|
||||
{25, "greek", true}, // "greek_general_ci",
|
||||
{26, "cp1250", true}, // "cp1250_general_ci",
|
||||
{27, "latin2", true}, // "latin2_croatian_ci",
|
||||
{28, "gbk", true}, // "gbk_chinese_ci",
|
||||
{29, "cp1257", true}, // "cp1257_lithuanian_ci",
|
||||
{30, "latin5", true}, // "latin5_turkish_ci",
|
||||
{31, "latin1", true}, // "latin1_german2_ci",
|
||||
{32, "armscii8", true}, // "armscii8_general_ci",
|
||||
{33, "utf8", false}, // "utf8_general_ci",
|
||||
{34, "cp1250", true}, // "cp1250_czech_cs",
|
||||
{35, "ucs2", true}, // "ucs2_general_ci",
|
||||
{36, "cp866", true}, // "cp866_general_ci",
|
||||
{37, "keybcs2", true}, // "keybcs2_general_ci",
|
||||
{38, "macce", true}, // "macce_general_ci",
|
||||
{39, "macroman", true}, // "macroman_general_ci",
|
||||
{40, "cp852", true}, // "cp852_general_ci",
|
||||
{41, "latin7", true}, // "latin7_general_ci",
|
||||
{42, "latin7", true}, // "latin7_general_cs",
|
||||
{43, "macce", true}, // "macce_bin",
|
||||
{44, "cp1250", true}, // "cp1250_croatian_ci",
|
||||
{45, "utf8mb4", false}, // "utf8mb4_general_ci",
|
||||
{46, "utf8mb4", false}, // "utf8mb4_bin",
|
||||
{47, "latin1", true}, // "latin1_bin",
|
||||
{48, "latin1", true}, // "latin1_general_ci",
|
||||
{49, "latin1", true}, // "latin1_general_cs",
|
||||
{50, "cp1251", true}, // "cp1251_bin",
|
||||
{51, "cp1251", true}, // "cp1251_general_ci",
|
||||
{52, "cp1251", true}, // "cp1251_general_cs",
|
||||
{53, "macroman", true}, // "macroman_bin",
|
||||
{54, "utf16", true}, // "utf16_general_ci",
|
||||
{55, "utf16", true}, // "utf16_bin",
|
||||
{56, "utf16le", true}, // "utf16le_general_ci",
|
||||
{57, "cp1256", true}, // "cp1256_general_ci",
|
||||
{58, "cp1257", true}, // "cp1257_bin",
|
||||
{59, "cp1257", true}, // "cp1257_general_ci",
|
||||
{60, "utf32", true}, // "utf32_general_ci",
|
||||
{61, "utf32", true}, // "utf32_bin",
|
||||
{62, "utf16le", true}, // "utf16le_bin",
|
||||
{64, "armscii8", true}, // "armscii8_bin",
|
||||
{65, "ascii", true}, // "ascii_bin",
|
||||
{66, "cp1250", true}, // "cp1250_bin",
|
||||
{67, "cp1256", true}, // "cp1256_bin",
|
||||
{68, "cp866", true}, // "cp866_bin",
|
||||
{69, "dec8", true}, // "dec8_bin",
|
||||
{70, "greek", true}, // "greek_bin",
|
||||
{71, "hebrew", true}, // "hebrew_bin",
|
||||
{72, "hp8", true}, // "hp8_bin",
|
||||
{73, "keybcs2", true}, // "keybcs2_bin",
|
||||
{74, "koi8r", true}, // "koi8r_bin",
|
||||
{75, "koi8u", true}, // "koi8u_bin",
|
||||
{77, "latin2", true}, // "latin2_bin",
|
||||
{78, "latin5", true}, // "latin5_bin",
|
||||
{79, "latin7", true}, // "latin7_bin",
|
||||
{80, "cp850", true}, // "cp850_bin",
|
||||
{81, "cp852", true}, // "cp852_bin",
|
||||
{82, "swe7", true}, // "swe7_bin",
|
||||
{83, "utf8", false}, // "utf8_bin",
|
||||
{84, "big5", true}, // "big5_bin",
|
||||
{85, "euckr", true}, // "euckr_bin",
|
||||
{86, "gb2312", true}, // "gb2312_bin",
|
||||
{87, "gbk", true}, // "gbk_bin",
|
||||
{88, "sjis", true}, // "sjis_bin",
|
||||
{89, "tis620", true}, // "tis620_bin",
|
||||
{90, "ucs2", true}, // "ucs2_bin",
|
||||
{91, "ujis", true}, // "ujis_bin",
|
||||
{92, "geostd8", true}, // "geostd8_general_ci",
|
||||
{93, "geostd8", true}, // "geostd8_bin",
|
||||
{94, "latin1", true}, // "latin1_spanish_ci",
|
||||
{95, "cp932", true}, // "cp932_japanese_ci",
|
||||
{96, "cp932", true}, // "cp932_bin",
|
||||
{97, "eucjpms", true}, // "eucjpms_japanese_ci",
|
||||
{98, "eucjpms", true}, // "eucjpms_bin",
|
||||
{99, "cp1250", true}, // "cp1250_polish_ci",
|
||||
{101, "utf16", true}, // "utf16_unicode_ci",
|
||||
{102, "utf16", true}, // "utf16_icelandic_ci",
|
||||
{103, "utf16", true}, // "utf16_latvian_ci",
|
||||
{104, "utf16", true}, // "utf16_romanian_ci",
|
||||
{105, "utf16", true}, // "utf16_slovenian_ci",
|
||||
{106, "utf16", true}, // "utf16_polish_ci",
|
||||
{107, "utf16", true}, // "utf16_estonian_ci",
|
||||
{108, "utf16", true}, // "utf16_spanish_ci",
|
||||
{109, "utf16", true}, // "utf16_swedish_ci",
|
||||
{110, "utf16", true}, // "utf16_turkish_ci",
|
||||
{111, "utf16", true}, // "utf16_czech_ci",
|
||||
{112, "utf16", true}, // "utf16_danish_ci",
|
||||
{113, "utf16", true}, // "utf16_lithuanian_ci",
|
||||
{114, "utf16", true}, // "utf16_slovak_ci",
|
||||
{115, "utf16", true}, // "utf16_spanish2_ci",
|
||||
{116, "utf16", true}, // "utf16_roman_ci",
|
||||
{117, "utf16", true}, // "utf16_persian_ci",
|
||||
{118, "utf16", true}, // "utf16_esperanto_ci",
|
||||
{119, "utf16", true}, // "utf16_hungarian_ci",
|
||||
{120, "utf16", true}, // "utf16_sinhala_ci",
|
||||
{121, "utf16", true}, // "utf16_german2_ci",
|
||||
{122, "utf16", true}, // "utf16_croatian_ci",
|
||||
{123, "utf16", true}, // "utf16_unicode_520_ci",
|
||||
{124, "utf16", true}, // "utf16_vietnamese_ci",
|
||||
{128, "ucs2", true}, // "ucs2_unicode_ci",
|
||||
{129, "ucs2", true}, // "ucs2_icelandic_ci",
|
||||
{130, "ucs2", true}, // "ucs2_latvian_ci",
|
||||
{131, "ucs2", true}, // "ucs2_romanian_ci",
|
||||
{132, "ucs2", true}, // "ucs2_slovenian_ci",
|
||||
{133, "ucs2", true}, // "ucs2_polish_ci",
|
||||
{134, "ucs2", true}, // "ucs2_estonian_ci",
|
||||
{135, "ucs2", true}, // "ucs2_spanish_ci",
|
||||
{136, "ucs2", true}, // "ucs2_swedish_ci",
|
||||
{137, "ucs2", true}, // "ucs2_turkish_ci",
|
||||
{138, "ucs2", true}, // "ucs2_czech_ci",
|
||||
{139, "ucs2", true}, // "ucs2_danish_ci",
|
||||
{140, "ucs2", true}, // "ucs2_lithuanian_ci",
|
||||
{141, "ucs2", true}, // "ucs2_slovak_ci",
|
||||
{142, "ucs2", true}, // "ucs2_spanish2_ci",
|
||||
{143, "ucs2", true}, // "ucs2_roman_ci",
|
||||
{144, "ucs2", true}, // "ucs2_persian_ci",
|
||||
{145, "ucs2", true}, // "ucs2_esperanto_ci",
|
||||
{146, "ucs2", true}, // "ucs2_hungarian_ci",
|
||||
{147, "ucs2", true}, // "ucs2_sinhala_ci",
|
||||
{148, "ucs2", true}, // "ucs2_german2_ci",
|
||||
{149, "ucs2", true}, // "ucs2_croatian_ci",
|
||||
{150, "ucs2", true}, // "ucs2_unicode_520_ci",
|
||||
{151, "ucs2", true}, // "ucs2_vietnamese_ci",
|
||||
{159, "ucs2", true}, // "ucs2_general_mysql500_ci",
|
||||
{160, "utf32", true}, // "utf32_unicode_ci",
|
||||
{161, "utf32", true}, // "utf32_icelandic_ci",
|
||||
{162, "utf32", true}, // "utf32_latvian_ci",
|
||||
{163, "utf32", true}, // "utf32_romanian_ci",
|
||||
{164, "utf32", true}, // "utf32_slovenian_ci",
|
||||
{165, "utf32", true}, // "utf32_polish_ci",
|
||||
{166, "utf32", true}, // "utf32_estonian_ci",
|
||||
{167, "utf32", true}, // "utf32_spanish_ci",
|
||||
{168, "utf32", true}, // "utf32_swedish_ci",
|
||||
{169, "utf32", true}, // "utf32_turkish_ci",
|
||||
{170, "utf32", true}, // "utf32_czech_ci",
|
||||
{171, "utf32", true}, // "utf32_danish_ci",
|
||||
{172, "utf32", true}, // "utf32_lithuanian_ci",
|
||||
{173, "utf32", true}, // "utf32_slovak_ci",
|
||||
{174, "utf32", true}, // "utf32_spanish2_ci",
|
||||
{175, "utf32", true}, // "utf32_roman_ci",
|
||||
{176, "utf32", true}, // "utf32_persian_ci",
|
||||
{177, "utf32", true}, // "utf32_esperanto_ci",
|
||||
{178, "utf32", true}, // "utf32_hungarian_ci",
|
||||
{179, "utf32", true}, // "utf32_sinhala_ci",
|
||||
{180, "utf32", true}, // "utf32_german2_ci",
|
||||
{181, "utf32", true}, // "utf32_croatian_ci",
|
||||
{182, "utf32", true}, // "utf32_unicode_520_ci",
|
||||
{183, "utf32", true}, // "utf32_vietnamese_ci",
|
||||
{192, "utf8", false}, // "utf8_unicode_ci",
|
||||
{193, "utf8", false}, // "utf8_icelandic_ci",
|
||||
{194, "utf8", false}, // "utf8_latvian_ci",
|
||||
{195, "utf8", false}, // "utf8_romanian_ci",
|
||||
{196, "utf8", false}, // "utf8_slovenian_ci",
|
||||
{197, "utf8", false}, // "utf8_polish_ci",
|
||||
{198, "utf8", false}, // "utf8_estonian_ci",
|
||||
{199, "utf8", false}, // "utf8_spanish_ci",
|
||||
{200, "utf8", false}, // "utf8_swedish_ci",
|
||||
{201, "utf8", false}, // "utf8_turkish_ci",
|
||||
{202, "utf8", false}, // "utf8_czech_ci",
|
||||
{203, "utf8", false}, // "utf8_danish_ci",
|
||||
{204, "utf8", false}, // "utf8_lithuanian_ci",
|
||||
{205, "utf8", false}, // "utf8_slovak_ci",
|
||||
{206, "utf8", false}, // "utf8_spanish2_ci",
|
||||
{207, "utf8", false}, // "utf8_roman_ci",
|
||||
{208, "utf8", false}, // "utf8_persian_ci",
|
||||
{209, "utf8", false}, // "utf8_esperanto_ci",
|
||||
{210, "utf8", false}, // "utf8_hungarian_ci",
|
||||
{211, "utf8", false}, // "utf8_sinhala_ci",
|
||||
{212, "utf8", false}, // "utf8_german2_ci",
|
||||
{213, "utf8", false}, // "utf8_croatian_ci",
|
||||
{214, "utf8", false}, // "utf8_unicode_520_ci",
|
||||
{215, "utf8", false}, // "utf8_vietnamese_ci",
|
||||
{223, "utf8", false}, // "utf8_general_mysql500_ci",
|
||||
{224, "utf8mb4", false}, // "utf8mb4_unicode_ci",
|
||||
{225, "utf8mb4", false}, // "utf8mb4_icelandic_ci",
|
||||
{226, "utf8mb4", false}, // "utf8mb4_latvian_ci",
|
||||
{227, "utf8mb4", false}, // "utf8mb4_romanian_ci",
|
||||
{228, "utf8mb4", false}, // "utf8mb4_slovenian_ci",
|
||||
{229, "utf8mb4", false}, // "utf8mb4_polish_ci",
|
||||
{230, "utf8mb4", false}, // "utf8mb4_estonian_ci",
|
||||
{231, "utf8mb4", false}, // "utf8mb4_spanish_ci",
|
||||
{232, "utf8mb4", false}, // "utf8mb4_swedish_ci",
|
||||
{233, "utf8mb4", false}, // "utf8mb4_turkish_ci",
|
||||
{234, "utf8mb4", false}, // "utf8mb4_czech_ci",
|
||||
{235, "utf8mb4", false}, // "utf8mb4_danish_ci",
|
||||
{236, "utf8mb4", false}, // "utf8mb4_lithuanian_ci",
|
||||
{237, "utf8mb4", false}, // "utf8mb4_slovak_ci",
|
||||
{238, "utf8mb4", false}, // "utf8mb4_spanish2_ci",
|
||||
{239, "utf8mb4", false}, // "utf8mb4_roman_ci",
|
||||
{240, "utf8mb4", false}, // "utf8mb4_persian_ci",
|
||||
{241, "utf8mb4", false}, // "utf8mb4_esperanto_ci",
|
||||
{242, "utf8mb4", false}, // "utf8mb4_hungarian_ci",
|
||||
{243, "utf8mb4", false}, // "utf8mb4_sinhala_ci",
|
||||
{244, "utf8mb4", false}, // "utf8mb4_german2_ci",
|
||||
{245, "utf8mb4", false}, // "utf8mb4_croatian_ci",
|
||||
{246, "utf8mb4", false}, // "utf8mb4_unicode_520_ci",
|
||||
{247, "utf8mb4", false}, // "utf8mb4_vietnamese_ci",
|
||||
{248, "gb18030", true}, // "gb18030_chinese_ci",
|
||||
{249, "gb18030", true}, // "gb18030_bin",
|
||||
{250, "gb18030", true}, // "gb18030_unicode_520_ci",
|
||||
{255, "utf8mb4", false}, // "utf8mb4_0900_ai_ci",
|
||||
{256, "utf8mb4", false}, // "utf8mb4_de_pb_0900_ai_ci",
|
||||
{257, "utf8mb4", false}, // "utf8mb4_is_0900_ai_ci",
|
||||
{258, "utf8mb4", false}, // "utf8mb4_lv_0900_ai_ci",
|
||||
{259, "utf8mb4", false}, // "utf8mb4_ro_0900_ai_ci",
|
||||
{260, "utf8mb4", false}, // "utf8mb4_sl_0900_ai_ci",
|
||||
{261, "utf8mb4", false}, // "utf8mb4_pl_0900_ai_ci",
|
||||
{262, "utf8mb4", false}, // "utf8mb4_et_0900_ai_ci",
|
||||
{263, "utf8mb4", false}, // "utf8mb4_es_0900_ai_ci",
|
||||
{264, "utf8mb4", false}, // "utf8mb4_is_0900_ai_ci",
|
||||
{265, "utf8mb4", false}, // "utf8mb4_tr_0900_ai_ci",
|
||||
{266, "utf8mb4", false}, // "utf8mb4_cs_0900_ai_ci",
|
||||
{267, "utf8mb4", false}, // "utf8mb4_da_0900_ai_ci",
|
||||
{268, "utf8mb4", false}, // "utf8mb4_lt_0900_ai_ci",
|
||||
{269, "utf8mb4", false}, // "utf8mb4_sk_0900_ai_ci",
|
||||
{270, "utf8mb4", false}, // "utf8mb4_es_trad_0900_ai_ci",
|
||||
{271, "utf8mb4", false}, // "utf8mb4_la_0900_ai_ci",
|
||||
{272, "utf8mb4", false}, // "utf8mb4_fa_0900_ai_ci",
|
||||
{273, "utf8mb4", false}, // "utf8mb4_eo_0900_ai_ci",
|
||||
{274, "utf8mb4", false}, // "utf8mb4_hu_0900_ai_ci",
|
||||
{275, "utf8mb4", false}, // "utf8mb4_hr_0900_ai_ci",
|
||||
{276, "utf8mb4", false}, // "utf8mb4_si_0900_ai_ci",
|
||||
{277, "utf8mb4", false}, // "utf8mb4_vi_0900_ai_ci",
|
||||
{278, "utf8mb4", false}, // "utf8mb4_0900_as_cs",
|
||||
{279, "utf8mb4", false}, // "utf8mb4_de_pb_0900_as_cs",
|
||||
{280, "utf8mb4", false}, // "utf8mb4_is_0900_as_cs",
|
||||
{281, "utf8mb4", false}, // "utf8mb4_lv_0900_as_cs",
|
||||
{282, "utf8mb4", false}, // "utf8mb4_ro_0900_as_cs",
|
||||
{283, "utf8mb4", false}, // "utf8mb4_sl_0900_as_cs",
|
||||
{284, "utf8mb4", false}, // "utf8mb4_pl_0900_as_cs",
|
||||
{285, "utf8mb4", false}, // "utf8mb4_et_0900_as_cs",
|
||||
{286, "utf8mb4", false}, // "utf8mb4_es_0900_as_cs",
|
||||
{287, "utf8mb4", false}, // "utf8mb4_sv_0900_as_cs",
|
||||
{288, "utf8mb4", false}, // "utf8mb4_tr_0900_as_cs",
|
||||
{289, "utf8mb4", false}, // "utf8mb4_cs_0900_as_cs",
|
||||
{290, "utf8mb4", false}, // "utf8mb4_da_0900_as_cs"
|
||||
{291, "utf8mb4", false}, // "utf8mb4_lt_0900_as_cs"
|
||||
{292, "utf8mb4", false}, // "utf8mb4_sk_0900_as_cs"
|
||||
{293, "utf8mb4", false}, // "utf8mb4_es_trad_0900_as_cs"
|
||||
{294, "utf8mb4", false}, // "utf8mb4_la_0900_as_cs"
|
||||
{295, "utf8mb4", false}, // "utf8mb4_fa_0900_as_cs"
|
||||
{296, "utf8mb4", false}, // "utf8mb4_eo_0900_as_cs"
|
||||
{297, "utf8mb4", false}, // "utf8mb4_hu_0900_as_cs"
|
||||
{298, "utf8mb4", false}, // "utf8mb4_hr_0900_as_cs"
|
||||
{299, "utf8mb4", false}, // "utf8mb4_si_0900_as_cs"
|
||||
{300, "utf8mb4", false}, // "utf8mb4_vi_0900_as_cs"
|
||||
{303, "utf8mb4", false}, // "utf8mb4_ja_0900_as_cs_ks"
|
||||
{304, "utf8mb4", false}, // "utf8mb4_la_0900_as_cs"
|
||||
{305, "utf8mb4", false}, // "utf8mb4_0900_as_ci"
|
||||
{306, "utf8mb4", false}, // "utf8mb4_ru_0900_ai_ci"
|
||||
{307, "utf8mb4", false}, // "utf8mb4_ru_0900_as_cs"
|
||||
{308, "utf8mb4", false}, // "utf8mb4_zh_0900_as_cs"
|
||||
{309, "utf8mb4", false} // "utf8mb4_0900_bin"
|
||||
};
|
||||
|
||||
MySQLCharset charset;
|
||||
|
||||
for (auto & item : result)
|
||||
{
|
||||
EXPECT_TRUE(charset.needConvert(item.id) == item.need_convert);
|
||||
if (charset.needConvert(item.id))
|
||||
{
|
||||
EXPECT_TRUE(charset.getCharsetFromId(item.id) == item.name);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
@ -980,6 +980,89 @@ def query_event_with_empty_transaction(clickhouse_node, mysql_node, service_name
|
||||
mysql_node.query("DROP DATABASE test_database_event")
|
||||
|
||||
|
||||
def text_blob_with_charset_test(clickhouse_node, mysql_node, service_name):
|
||||
db = "text_blob_with_charset_test"
|
||||
mysql_node.query(f"DROP DATABASE IF EXISTS {db}")
|
||||
clickhouse_node.query(f"DROP DATABASE IF EXISTS {db}")
|
||||
mysql_node.query(f"CREATE DATABASE {db} DEFAULT CHARACTER SET 'utf8'")
|
||||
|
||||
mysql_node.query(
|
||||
f"CREATE TABLE {db}.test_table_1 (a INT NOT NULL PRIMARY KEY, b text CHARACTER SET gbk, c tinytext CHARSET big5, d longtext, e varchar(256), f char(4)) ENGINE = InnoDB DEFAULT CHARSET=gbk"
|
||||
)
|
||||
mysql_node.query(
|
||||
f"CREATE TABLE {db}.test_table_2 (a INT NOT NULL PRIMARY KEY, b blob, c longblob) ENGINE = InnoDB DEFAULT CHARSET=gbk"
|
||||
)
|
||||
mysql_node.query(
|
||||
f"CREATE TABLE {db}.test_table_3 (a INT NOT NULL PRIMARY KEY, b text CHARACTER SET gbk, c tinytext CHARSET gbk, d tinytext CHARSET big5, e varchar(256), f char(4)) ENGINE = InnoDB"
|
||||
)
|
||||
|
||||
mysql_node.query(
|
||||
f"INSERT INTO {db}.test_table_1 VALUES (1, '你好', '世界', '哈罗', '您Hi您', '您Hi您')"
|
||||
)
|
||||
mysql_node.query(
|
||||
f"INSERT INTO {db}.test_table_2 VALUES (1, '你好', 0xFAAA00000000000DDCC)"
|
||||
)
|
||||
mysql_node.query(
|
||||
f"INSERT INTO {db}.test_table_3 VALUES (1, '你好', '世界', 'hello', '您Hi您', '您Hi您')"
|
||||
)
|
||||
|
||||
clickhouse_node.query(
|
||||
f"CREATE DATABASE {db} ENGINE = MaterializedMySQL('{service_name}:3306', '{db}', 'root', 'clickhouse')"
|
||||
)
|
||||
assert db in clickhouse_node.query("SHOW DATABASES")
|
||||
|
||||
# from full replication
|
||||
check_query(
|
||||
clickhouse_node,
|
||||
f"SHOW TABLES FROM {db} FORMAT TSV",
|
||||
"test_table_1\ntest_table_2\ntest_table_3\n",
|
||||
)
|
||||
check_query(
|
||||
clickhouse_node,
|
||||
f"SELECT b, c, d, e, f FROM {db}.test_table_1 WHERE a = 1 FORMAT TSV",
|
||||
"你好\t世界\t哈罗\t您Hi您\t您Hi您\n",
|
||||
)
|
||||
check_query(
|
||||
clickhouse_node,
|
||||
f"SELECT hex(b), hex(c) FROM {db}.test_table_2 WHERE a = 1 FORMAT TSV",
|
||||
"E4BDA0E5A5BD\t0FAAA00000000000DDCC\n",
|
||||
)
|
||||
check_query(
|
||||
clickhouse_node,
|
||||
f"SELECT b, c, d, e, f FROM {db}.test_table_3 WHERE a = 1 FORMAT TSV",
|
||||
"你好\t世界\thello\t您Hi您\t您Hi您\n",
|
||||
)
|
||||
|
||||
# from increment replication
|
||||
mysql_node.query(
|
||||
f"INSERT INTO {db}.test_table_1 VALUES (2, '你好', '世界', '哈罗', '您Hi您', '您Hi您')"
|
||||
)
|
||||
mysql_node.query(
|
||||
f"INSERT INTO {db}.test_table_2 VALUES (2, '你好', 0xFAAA00000000000DDCC)"
|
||||
)
|
||||
mysql_node.query(
|
||||
f"INSERT INTO {db}.test_table_3 VALUES (2, '你好', '世界', 'hello', '您Hi您', '您Hi您')"
|
||||
)
|
||||
|
||||
check_query(
|
||||
clickhouse_node,
|
||||
f"SELECT b, c, d, e, f FROM {db}.test_table_1 WHERE a = 2 FORMAT TSV",
|
||||
"你好\t世界\t哈罗\t您Hi您\t您Hi您\n",
|
||||
)
|
||||
check_query(
|
||||
clickhouse_node,
|
||||
f"SELECT hex(b), hex(c) FROM {db}.test_table_2 WHERE a = 2 FORMAT TSV",
|
||||
"E4BDA0E5A5BD\t0FAAA00000000000DDCC\n",
|
||||
)
|
||||
check_query(
|
||||
clickhouse_node,
|
||||
f"SELECT b, c, d, e, f FROM {db}.test_table_3 WHERE a = 2 FORMAT TSV",
|
||||
"你好\t世界\thello\t您Hi您\t您Hi您\n",
|
||||
)
|
||||
clickhouse_node.query(f"DROP DATABASE {db}")
|
||||
mysql_node.query(f"DROP DATABASE {db}")
|
||||
|
||||
|
||||
def select_without_columns(clickhouse_node, mysql_node, service_name):
|
||||
mysql_node.query("DROP DATABASE IF EXISTS db")
|
||||
clickhouse_node.query("DROP DATABASE IF EXISTS db")
|
||||
|
@ -262,6 +262,12 @@ def test_materialized_database_ddl_with_empty_transaction_8_0(
|
||||
)
|
||||
|
||||
|
||||
def test_text_blob_charset(started_cluster, started_mysql_8_0, clickhouse_node):
|
||||
materialized_with_ddl.text_blob_with_charset_test(
|
||||
clickhouse_node, started_mysql_8_0, "mysql80"
|
||||
)
|
||||
|
||||
|
||||
def test_select_without_columns_5_7(
|
||||
started_cluster, started_mysql_5_7, clickhouse_node
|
||||
):
|
||||
|
@ -11,7 +11,9 @@
|
||||
#include <IO/WriteBufferFromFileDescriptor.h>
|
||||
#include <IO/WriteBufferFromOStream.h>
|
||||
#include <Core/MySQL/MySQLReplication.h>
|
||||
#include <Core/MySQL/MySQLCharset.h>
|
||||
|
||||
static DB::MySQLCharsetPtr charset = std::make_shared<DB::MySQLCharset>();
|
||||
static DB::MySQLReplication::BinlogEventPtr parseSingleEventBody(
|
||||
DB::MySQLReplication::EventHeader & header, DB::ReadBuffer & payload,
|
||||
std::shared_ptr<DB::MySQLReplication::TableMapEvent> & last_table_map_event, bool exist_checksum)
|
||||
@ -64,7 +66,7 @@ static DB::MySQLReplication::BinlogEventPtr parseSingleEventBody(
|
||||
{
|
||||
DB::MySQLReplication::TableMapEventHeader map_event_header;
|
||||
map_event_header.parse(*event_payload);
|
||||
event = std::make_shared<DB::MySQLReplication::TableMapEvent>(std::move(header), map_event_header);
|
||||
event = std::make_shared<DB::MySQLReplication::TableMapEvent>(std::move(header), map_event_header, charset);
|
||||
event->parseEvent(*event_payload);
|
||||
last_table_map_event = std::static_pointer_cast<DB::MySQLReplication::TableMapEvent>(event);
|
||||
break;
|
||||
|
Loading…
Reference in New Issue
Block a user