diff --git a/docs/en/operations/settings/mysql-binlog-client.md b/docs/en/operations/settings/mysql-binlog-client.md new file mode 100644 index 000000000000..1e1a2449e1cc --- /dev/null +++ b/docs/en/operations/settings/mysql-binlog-client.md @@ -0,0 +1,176 @@ +# The MySQL Binlog Client + +The MySQL Binlog Client provides a mechanism in ClickHouse to share the binlog from a MySQL instance among multiple [MaterializedMySQL](../../engines/database-engines/materialized-mysql.md) databases. This avoids consuming unnecessary bandwidth and CPU when replicating more than one schema/database. + +The implementation is resilient against crashes and disk issues. The executed GTID sets of the binlog itself and the consuming databases have persisted only after the data they describe has been safely persisted as well. The implementation also tolerates re-doing aborted operations (at-least-once delivery). + +# Settings + +## use_binlog_client + +Forces to reuse existing MySQL binlog connection or creates new one if does not exist. The connection is defined by `user:pass@host:port`. + +Default value: 0 + +**Example** + +```sql +-- create MaterializedMySQL databases that read the events from the binlog client +CREATE DATABASE db1 ENGINE = MaterializedMySQL('host:port', 'db1', 'user', 'password') SETTINGS use_binlog_client=1 +CREATE DATABASE db2 ENGINE = MaterializedMySQL('host:port', 'db2', 'user', 'password') SETTINGS use_binlog_client=1 +CREATE DATABASE db3 ENGINE = MaterializedMySQL('host:port', 'db3', 'user2', 'password2') SETTINGS use_binlog_client=1 +``` + +Databases `db1` and `db2` will use the same binlog connection, since they use the same `user:pass@host:port`. Database `db3` will use separate binlog connection. + +## max_bytes_in_binlog_queue + +Defines the limit of bytes in the events binlog queue. If bytes in the queue increases this limit, it will stop reading new events from MySQL until the space for new events will be freed. This introduces the memory limits. Very high value could consume all available memory. Very low value could make the databases to wait for new events. + +Default value: 67108864 + +**Example** + +```sql +CREATE DATABASE db1 ENGINE = MaterializedMySQL('host:port', 'db1', 'user', 'password') SETTINGS use_binlog_client=1, max_bytes_in_binlog_queue=33554432 +CREATE DATABASE db2 ENGINE = MaterializedMySQL('host:port', 'db2', 'user', 'password') SETTINGS use_binlog_client=1 +``` + +If database `db1` is unable to consume binlog events fast enough and the size of the events queue exceeds `33554432` bytes, reading of new events from MySQL is postponed until `db1` +consumes the events and releases some space. + +NOTE: This will impact to `db2`, and it will be waiting for new events too, since they share the same connection. + +## max_milliseconds_to_wait_in_binlog_queue + +Defines the max milliseconds to wait when `max_bytes_in_binlog_queue` exceeded. After that it will detach the database from current binlog connection and will retry establish new one to prevent other databases to wait for this database. + +Default value: 10000 + +**Example** + +```sql +CREATE DATABASE db1 ENGINE = MaterializedMySQL('host:port', 'db1', 'user', 'password') SETTINGS use_binlog_client=1, max_bytes_in_binlog_queue=33554432, max_milliseconds_to_wait_in_binlog_queue=1000 +CREATE DATABASE db2 ENGINE = MaterializedMySQL('host:port', 'db2', 'user', 'password') SETTINGS use_binlog_client=1 +``` + +If the event queue of database `db1` is full, the binlog connection will be waiting in `1000`ms and if the database is not able to consume the events, it will be detached from the connection to create another one. + +NOTE: If the database `db1` has been detached from the shared connection and created new one, after the binlog connections for `db1` and `db2` have the same positions they will be merged to one. And `db1` and `db2` will use the same connection again. + +## max_bytes_in_binlog_dispatcher_buffer + +Defines the max bytes in the binlog dispatcher's buffer before it is flushed to attached binlog. The events from MySQL binlog connection are buffered before sending to attached databases. It increases the events throughput from the binlog to databases. + +Default value: 1048576 + +## max_flush_milliseconds_in_binlog_dispatcher + +Defines the max milliseconds in the binlog dispatcher's buffer to wait before it is flushed to attached binlog. If there are no events received from MySQL binlog connection for a while, after some time buffered events should be sent to the attached databases. + +Default value: 1000 + +# Design + +## The Binlog Events Dispatcher + +Currently each MaterializedMySQL database opens its own connection to MySQL to subscribe to binlog events. There is a need to have only one connection and _dispatch_ the binlog events to all databases that replicate from the same MySQL instance. + +## Each MaterializedMySQL Database Has Its Own Event Queue + +To prevent slowing down other instances there should be an _event queue_ per MaterializedMySQL database to handle the events independently of the speed of other instances. The dispatcher reads an event from the binlog, and sends it to every MaterializedMySQL database that needs it. Each database handles its events in separate threads. + +## Catching up + +If several databases have the same binlog position, they can use the same dispatcher. If a newly created database (or one that has been detached for some time) requests events that have been already processed, we need to create another communication _channel_ to the binlog. We do this by creating another temporary dispatcher for such databases. When the new dispatcher _catches up with_ the old one, the new/temporary dispatcher is not needed anymore and all databases getting events from this dispatcher can be moved to the old one. + +## Memory Limit + +There is a _memory limit_ to control event queue memory consumption per MySQL Client. If a database is not able to handle events fast enough, and the event queue is getting full, we have the following options: + +1. The dispatcher is blocked until the slowest database frees up space for new events. All other databases are waiting for the slowest one. (Preferred) +2. The dispatcher is _never_ blocked, but suspends incremental sync for the slow database and continues dispatching events to remained databases. + +## Performance + +A lot of CPU can be saved by not processing every event in every database. The binlog contains events for all databases, it is wasteful to distribute row events to a database that it will not process it, especially if there are a lot of databases. This requires some sort of per-database binlog filtering and buffering. + +Currently all events are sent to all MaterializedMySQL databases but parsing the event which consumes CPU is up to the database. + +# Detailed Design + +1. If a client (e.g. database) wants to read a stream of the events from MySQL binlog, it creates a connection to remote binlog by host/user/password and _executed GTID set_ params. +2. If another client wants to read the events from the binlog but for different _executed GTID set_, it is **not** possible to reuse existing connection to MySQL, then need to create another connection to the same remote binlog. (_This is how it is implemented today_). +3. When these 2 connections get the same binlog positions, they read the same events. It is logical to drop duplicate connection and move all its users out. And now one connection dispatches binlog events to several clients. Obviously only connections to the same binlog should be merged. + +## Classes + +1. One connection can send (or dispatch) events to several clients and might be called `BinlogEventsDispatcher`. +2. Several dispatchers grouped by _user:password@host:port_ in `BinlogClient`. Since they point to the same binlog. +3. The clients should communicate only with public API from `BinlogClient`. The result of using `BinlogClient` is an object that implements `IBinlog` to read events from. This implementation of `IBinlog` must be compatible with old implementation `MySQLFlavor` -> when replacing old implementation by new one, the behavior must not be changed. + +## SQL + +```sql +-- create MaterializedMySQL databases that read the events from the binlog client +CREATE DATABASE db1_client1 ENGINE = MaterializedMySQL('host:port', 'db', 'user', 'password') SETTINGS use_binlog_client=1, max_bytes_in_binlog_queue=1024; +CREATE DATABASE db2_client1 ENGINE = MaterializedMySQL('host:port', 'db', 'user', 'password') SETTINGS use_binlog_client=1; +CREATE DATABASE db3_client1 ENGINE = MaterializedMySQL('host:port', 'db2', 'user', 'password') SETTINGS use_binlog_client=1; +CREATE DATABASE db4_client2 ENGINE = MaterializedMySQL('host2:port', 'db', 'user', 'password') SETTINGS use_binlog_client=1; +CREATE DATABASE db5_client3 ENGINE = MaterializedMySQL('host:port', 'db', 'user1', 'password') SETTINGS use_binlog_client=1; +CREATE DATABASE db6_old ENGINE = MaterializedMySQL('host:port', 'db', 'user1', 'password') SETTINGS use_binlog_client=0; +``` + +Databases `db1_client1`, `db2_client1` and `db3_client1` share one instance of `BinlogClient` since they have the same params. `BinlogClient` will create 3 connections to MySQL server thus 3 instances of `BinlogEventsDispatcher`, but if these connections would have the same binlog position, they should be merged to one connection. Means all clients will be moved to one dispatcher and others will be closed. Databases `db4_client2` and `db5_client3` would use 2 different independent `BinlogClient` instances. Database `db6_old` will use old implementation. NOTE: By default `use_binlog_client` is disabled. Setting `max_bytes_in_binlog_queue` defines the max allowed bytes in the binlog queue. By default, it is `1073741824` bytes. If number of bytes exceeds this limit, the dispatching will be stopped until the space will be freed for new events. + +## Binlog Table Structure + +To see the status of the all `BinlogClient` instances there is `system.mysql_binlogs` system table. It shows the list of all created and _alive_ `IBinlog` instances with information about its `BinlogEventsDispatcher` and `BinlogClient`. + +Example: + +``` +SELECT * FROM system.mysql_binlogs FORMAT Vertical +Row 1: +────── +binlog_client_name: root@127.0.0.1:3306 +name: test_Clickhouse1 +mysql_binlog_name: binlog.001154 +mysql_binlog_pos: 7142294 +mysql_binlog_timestamp: 1660082447 +mysql_binlog_executed_gtid_set: a9d88f83-c14e-11ec-bb36-244bfedf7766:1-30523304 +dispatcher_name: Applier +dispatcher_mysql_binlog_name: binlog.001154 +dispatcher_mysql_binlog_pos: 7142294 +dispatcher_mysql_binlog_timestamp: 1660082447 +dispatcher_mysql_binlog_executed_gtid_set: a9d88f83-c14e-11ec-bb36-244bfedf7766:1-30523304 +size: 0 +bytes: 0 +max_bytes: 0 +``` + +### Tests + +Unit tests: + +``` +$ ./unit_tests_dbms --gtest_filter=MySQLBinlog.* +``` + +Integration tests: + +``` +$ pytest -s -vv test_materialized_mysql_database/test.py::test_binlog_client +``` + +Dumps events from the file + +``` +$ ./utils/check-mysql-binlog/check-mysql-binlog --binlog binlog.001392 +``` + +Dumps events from the server + +``` +$ ./utils/check-mysql-binlog/check-mysql-binlog --host 127.0.0.1 --port 3306 --user root --password pass --gtid a9d88f83-c14e-11ec-bb36-244bfedf7766:1-30462856 +``` diff --git a/src/Common/randomNumber.h b/src/Common/randomNumber.h new file mode 100644 index 000000000000..b795f32baca3 --- /dev/null +++ b/src/Common/randomNumber.h @@ -0,0 +1,12 @@ +#pragma once + +#include +#include + +inline UInt32 randomNumber() +{ + pcg64_fast rng{randomSeed()}; + std::uniform_int_distribution dist6( + std::numeric_limits::min(), std::numeric_limits::max()); + return static_cast(dist6(rng)); +} diff --git a/src/Core/MySQL/MySQLGtid.cpp b/src/Core/MySQL/MySQLGtid.cpp index 2b46c3d14ad7..7916f8829793 100644 --- a/src/Core/MySQL/MySQLGtid.cpp +++ b/src/Core/MySQL/MySQLGtid.cpp @@ -188,4 +188,46 @@ String GTIDSets::toPayload() const return buffer.str(); } +bool GTIDSet::contains(const GTIDSet & gtid_set) const +{ + //we contain the other set if each of its intervals are contained in any of our intervals. + //use the fact that intervals are sorted to make this linear instead of quadratic. + if (uuid != gtid_set.uuid) { return false; } + + auto mine = intervals.begin(), other = gtid_set.intervals.begin(); + auto my_end = intervals.end(), other_end = gtid_set.intervals.end(); + while (mine != my_end && other != other_end) + { + bool mine_contains_other = mine->start <= other->start && mine->end >= other->end; + if (mine_contains_other) + { + ++other; + } + else + { + ++mine; + } + } + + return other == other_end; //if we've iterated through all intervals in the argument, all its intervals are contained in this +} + +bool GTIDSets::contains(const GTIDSet & gtid_set) const +{ + for (const auto & my_gtid_set : sets) + { + if (my_gtid_set.contains(gtid_set)) { return true; } + } + return false; +} + +bool GTIDSets::contains(const GTIDSets & gtid_sets) const +{ + for (const auto & gtid_set : gtid_sets.sets) + { + if (!this->contains(gtid_set)) { return false; } + } + return true; +} + } diff --git a/src/Core/MySQL/MySQLGtid.h b/src/Core/MySQL/MySQLGtid.h index 45eeaf02fa2e..b7cff39cca6d 100644 --- a/src/Core/MySQL/MySQLGtid.h +++ b/src/Core/MySQL/MySQLGtid.h @@ -28,6 +28,8 @@ class GTIDSet void tryMerge(size_t i); static void tryShrink(GTIDSet & set, unsigned int i, Interval & current); + + bool contains(const GTIDSet & gtid_set) const; }; class GTIDSets @@ -40,6 +42,31 @@ class GTIDSets String toString() const; String toPayload() const; + bool contains(const GTIDSet & gtid_set) const; + bool contains(const GTIDSets & gtid_sets) const; }; +inline bool operator==(const GTID & left, const GTID & right) +{ + return left.uuid == right.uuid + && left.seq_no == right.seq_no; +} + +inline bool operator==(const GTIDSet::Interval & left, const GTIDSet::Interval & right) +{ + return left.start == right.start + && left.end == right.end; +} + +inline bool operator==(const GTIDSet & left, const GTIDSet & right) +{ + return left.uuid == right.uuid + && left.intervals == right.intervals; +} + +inline bool operator==(const GTIDSets & left, const GTIDSets & right) +{ + return left.sets == right.sets; +} + } diff --git a/src/Core/MySQL/MySQLReplication.cpp b/src/Core/MySQL/MySQLReplication.cpp index dcf42134b0b2..403f98360c1b 100644 --- a/src/Core/MySQL/MySQLReplication.cpp +++ b/src/Core/MySQL/MySQLReplication.cpp @@ -967,6 +967,59 @@ namespace MySQLReplication out << "[DryRun Event]" << '\n'; } + void UnparsedRowsEvent::dump(WriteBuffer & out) const + { + std::lock_guard lock(mutex); + header.dump(out); + out << "[UnparsedRowsEvent Event]" << '\n'; + out << "Unparsed Data Size: " << unparsed_data.size() << '\n'; + } + + void UnparsedRowsEvent::parseImpl(ReadBuffer & payload_) + { + char c = 0; + if (payload_.position() < payload_.buffer().end()) + unparsed_data.reserve(payload_.buffer().end() - payload_.position()); + /// Prevent reading after the end + /// payload.available() might have incorrect value + while (payload_.position() <= payload_.buffer().end() && payload_.read(c)) + unparsed_data.push_back(c); + if (!payload_.eof()) + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Cannot read all data. Available {} bytes but not eof", payload_.available()); + } + + std::shared_ptr UnparsedRowsEvent::parse() + { + std::lock_guard lock(mutex); + if (!unparsed_data.empty()) + { + RowsEventHeader rows_header(header.type); + rows_header.table_id = table_id; + rows_header.flags = flags; + switch (header.type) + { + case WRITE_ROWS_EVENT_V1: + case WRITE_ROWS_EVENT_V2: + parsed_event = std::make_shared(table_map, EventHeader(header), rows_header); + break; + case DELETE_ROWS_EVENT_V1: + case DELETE_ROWS_EVENT_V2: + parsed_event = std::make_shared(table_map, EventHeader(header), rows_header); + break; + case UPDATE_ROWS_EVENT_V1: + case UPDATE_ROWS_EVENT_V2: + parsed_event = std::make_shared(table_map, EventHeader(header), rows_header); + break; + default: + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown event type: {}", magic_enum::enum_name(header.type)); + } + ReadBufferFromMemory payload(unparsed_data.data(), unparsed_data.size()); + parsed_event->parseEvent(payload); + unparsed_data.clear(); + } + return parsed_event; + } + /// Update binlog name/position/gtid based on the event type. void Position::update(BinlogEventPtr event) { @@ -998,7 +1051,8 @@ namespace MySQLReplication case ROTATE_EVENT: { auto rotate = std::static_pointer_cast(event); binlog_name = rotate->next_binlog; - binlog_pos = event->header.log_pos; + /// If binlog name has changed, need to use position from next binlog + binlog_pos = rotate->position; break; } case GTID_EVENT: { @@ -1012,13 +1066,18 @@ namespace MySQLReplication default: throw ReplicationError(ErrorCodes::LOGICAL_ERROR, "Position update with unsupported event"); } + if (event->header.timestamp > 0) + { + timestamp = event->header.timestamp; + } } - void Position::update(UInt64 binlog_pos_, const String & binlog_name_, const String & gtid_sets_) + void Position::update(UInt64 binlog_pos_, const String & binlog_name_, const String & gtid_sets_, UInt32 binlog_time_) { binlog_pos = binlog_pos_; binlog_name = binlog_name_; gtid_sets.parse(gtid_sets_); + timestamp = binlog_time_; } void Position::dump(WriteBuffer & out) const diff --git a/src/Core/MySQL/MySQLReplication.h b/src/Core/MySQL/MySQLReplication.h index 1584dbd42ac9..6ba507245b39 100644 --- a/src/Core/MySQL/MySQLReplication.h +++ b/src/Core/MySQL/MySQLReplication.h @@ -181,6 +181,7 @@ namespace MySQLReplication MYSQL_WRITE_ROWS_EVENT = 2, MYSQL_UPDATE_ROWS_EVENT = 3, MYSQL_DELETE_ROWS_EVENT = 4, + MYSQL_UNPARSED_ROWS_EVENT = 100, }; class ReplicationError : public DB::Exception @@ -274,6 +275,8 @@ namespace MySQLReplication String status; String schema; String query; + String query_database_name; + String query_table_name; QueryType typ = QUERY_EVENT_DDL; bool transaction_complete = true; @@ -446,7 +449,6 @@ namespace MySQLReplication void parseImpl(ReadBuffer & payload) override; void parseRow(ReadBuffer & payload, Bitmap & bitmap); - private: std::shared_ptr table_map; }; @@ -497,17 +499,38 @@ namespace MySQLReplication void parseImpl(ReadBuffer & payload) override; }; + class UnparsedRowsEvent : public RowsEvent + { + public: + UnparsedRowsEvent(const std::shared_ptr & table_map_, EventHeader && header_, const RowsEventHeader & rows_header) + : RowsEvent(table_map_, std::move(header_), rows_header) + { + } + + void dump(WriteBuffer & out) const override; + MySQLEventType type() const override { return MYSQL_UNPARSED_ROWS_EVENT; } + std::shared_ptr parse(); + + protected: + void parseImpl(ReadBuffer & payload) override; + std::vector unparsed_data; + std::shared_ptr parsed_event; + mutable std::mutex mutex; + }; + class Position { public: UInt64 binlog_pos; String binlog_name; GTIDSets gtid_sets; + UInt32 timestamp; - Position() : binlog_pos(0) { } + Position() : binlog_pos(0), timestamp(0) { } void update(BinlogEventPtr event); - void update(UInt64 binlog_pos_, const String & binlog_name_, const String & gtid_sets_); + void update(UInt64 binlog_pos_, const String & binlog_name_, const String & gtid_sets_, UInt32 binlog_time_); void dump(WriteBuffer & out) const; + void resetPendingGTID() { pending_gtid.reset(); } private: std::optional pending_gtid; diff --git a/src/Core/MySQL/tests/gtest_MySQLGtid.cpp b/src/Core/MySQL/tests/gtest_MySQLGtid.cpp new file mode 100644 index 000000000000..e31a87aaa396 --- /dev/null +++ b/src/Core/MySQL/tests/gtest_MySQLGtid.cpp @@ -0,0 +1,40 @@ +#include +#include + +using namespace DB; + + +GTEST_TEST(GTIDSetsContains, Tests) +{ + GTIDSets gtid_set, + contained1, contained2, contained3, contained4, contained5, + not_contained1, not_contained2, not_contained3, not_contained4, not_contained5, not_contained6; + + gtid_set.parse("2174B383-5441-11E8-B90A-C80AA9429562:1-3:11:47-49, 24DA167-0C0C-11E8-8442-00059A3C7B00:1-19:47-49:60"); + contained1.parse("2174B383-5441-11E8-B90A-C80AA9429562:1-3:11:47-49, 24DA167-0C0C-11E8-8442-00059A3C7B00:1-19:47-49:60"); + contained2.parse("2174B383-5441-11E8-B90A-C80AA9429562:2-3:11:47-49"); + contained3.parse("2174B383-5441-11E8-B90A-C80AA9429562:11"); + contained4.parse("24DA167-0C0C-11E8-8442-00059A3C7B00:2-16:47-49:60"); + contained5.parse("24DA167-0C0C-11E8-8442-00059A3C7B00:60"); + + not_contained1.parse("2174B383-5441-11E8-B90A-C80AA9429562:1-3:11:47-50, 24DA167-0C0C-11E8-8442-00059A3C7B00:1-19:47-49:60"); + not_contained2.parse("2174B383-5441-11E8-B90A-C80AA9429562:0-3:11:47-49"); + not_contained3.parse("2174B383-5441-11E8-B90A-C80AA9429562:99"); + not_contained4.parse("24DA167-0C0C-11E8-8442-00059A3C7B00:2-16:46-49:60"); + not_contained5.parse("24DA167-0C0C-11E8-8442-00059A3C7B00:99"); + not_contained6.parse("2174B383-5441-11E8-B90A-C80AA9429562:1-3:11:47-49, 24DA167-0C0C-11E8-8442-00059A3C7B00:1-19:47-49:60, 00000000-0000-0000-0000-000000000000"); + + + ASSERT_TRUE(gtid_set.contains(contained1)); + ASSERT_TRUE(gtid_set.contains(contained2)); + ASSERT_TRUE(gtid_set.contains(contained3)); + ASSERT_TRUE(gtid_set.contains(contained4)); + ASSERT_TRUE(gtid_set.contains(contained5)); + + ASSERT_FALSE(gtid_set.contains(not_contained1)); + ASSERT_FALSE(gtid_set.contains(not_contained2)); + ASSERT_FALSE(gtid_set.contains(not_contained3)); + ASSERT_FALSE(gtid_set.contains(not_contained4)); + ASSERT_FALSE(gtid_set.contains(not_contained5)); + ASSERT_FALSE(gtid_set.contains(not_contained6)); +} diff --git a/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp b/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp index cbb080a0baa3..99dd337189c6 100644 --- a/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp @@ -10,6 +10,7 @@ # include # include # include +# include # include # include # include @@ -39,10 +40,11 @@ DatabaseMaterializedMySQL::DatabaseMaterializedMySQL( const String & mysql_database_name_, mysqlxx::Pool && pool_, MySQLClient && client_, + const MySQLReplication::BinlogClientPtr & binlog_client_, std::unique_ptr settings_) : DatabaseAtomic(database_name_, metadata_path_, uuid, "DatabaseMaterializedMySQL(" + database_name_ + ")", context_) , settings(std::move(settings_)) - , materialize_thread(context_, database_name_, mysql_database_name_, std::move(pool_), std::move(client_), settings.get()) + , materialize_thread(context_, database_name_, mysql_database_name_, std::move(pool_), std::move(client_), binlog_client_, settings.get()) { } @@ -197,6 +199,7 @@ void registerDatabaseMaterializedMySQL(DatabaseFactory & factory) if (!engine->arguments) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Engine `{}` must have arguments", engine_name); + MySQLReplication::BinlogClientPtr binlog_client; StorageMySQL::Configuration configuration; ASTs & arguments = engine->arguments->children; auto mysql_settings = std::make_unique(); @@ -241,6 +244,12 @@ void registerDatabaseMaterializedMySQL(DatabaseFactory & factory) if (engine_define->settings) materialize_mode_settings->loadFromQuery(*engine_define); + if (materialize_mode_settings->use_binlog_client) + binlog_client = DB::MySQLReplication::BinlogClientFactory::instance().getClient( + configuration.host, configuration.port, configuration.username, configuration.password, + materialize_mode_settings->max_bytes_in_binlog_dispatcher_buffer, + materialize_mode_settings->max_flush_milliseconds_in_binlog_dispatcher); + if (args.uuid == UUIDHelpers::Nil) { auto print_create_ast = args.create_query.clone(); @@ -261,6 +270,7 @@ void registerDatabaseMaterializedMySQL(DatabaseFactory & factory) configuration.database, std::move(mysql_pool), std::move(client), + binlog_client, std::move(materialize_mode_settings)); }; factory.registerDatabase("MaterializeMySQL", create_fn); diff --git a/src/Databases/MySQL/DatabaseMaterializedMySQL.h b/src/Databases/MySQL/DatabaseMaterializedMySQL.h index 895498723fdc..4d7871d49d67 100644 --- a/src/Databases/MySQL/DatabaseMaterializedMySQL.h +++ b/src/Databases/MySQL/DatabaseMaterializedMySQL.h @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -31,6 +32,7 @@ class DatabaseMaterializedMySQL : public DatabaseAtomic const String & mysql_database_name_, mysqlxx::Pool && pool_, MySQLClient && client_, + const MySQLReplication::BinlogClientPtr & binlog_client_, std::unique_ptr settings_); void rethrowExceptionIfNeeded() const; diff --git a/src/Databases/MySQL/MaterializedMySQLSettings.h b/src/Databases/MySQL/MaterializedMySQLSettings.h index 82342f8c76d5..557d48be85bc 100644 --- a/src/Databases/MySQL/MaterializedMySQLSettings.h +++ b/src/Databases/MySQL/MaterializedMySQLSettings.h @@ -17,6 +17,11 @@ class ASTStorage; M(Int64, max_wait_time_when_mysql_unavailable, 1000, "Retry interval when MySQL is not available (milliseconds). Negative value disable retry.", 0) \ M(Bool, allows_query_when_mysql_lost, false, "Allow query materialized table when mysql is lost.", 0) \ M(String, materialized_mysql_tables_list, "", "a comma-separated list of mysql database tables, which will be replicated by MaterializedMySQL database engine. Default value: empty list — means whole tables will be replicated.", 0) \ + M(Bool, use_binlog_client, false, "Use MySQL Binlog Client.", 0) \ + M(UInt64, max_bytes_in_binlog_queue, 64 * 1024 * 1024, "Max bytes in binlog's queue created from MySQL Binlog Client.", 0) \ + M(UInt64, max_milliseconds_to_wait_in_binlog_queue, 10000, "Max milliseconds to wait when max bytes exceeded in a binlog queue.", 0) \ + M(UInt64, max_bytes_in_binlog_dispatcher_buffer, DBMS_DEFAULT_BUFFER_SIZE, "Max bytes in the binlog dispatcher's buffer before it is flushed to attached binlogs.", 0) \ + M(UInt64, max_flush_milliseconds_in_binlog_dispatcher, 1000, "Max milliseconds in the binlog dispatcher's buffer to wait before it is flushed to attached binlogs.", 0) \ DECLARE_SETTINGS_TRAITS(MaterializedMySQLSettingsTraits, LIST_OF_MATERIALIZE_MODE_SETTINGS) diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp index 14cd89e1ff65..5834fb96dc6a 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp @@ -26,14 +26,13 @@ #include #include #include -#include +#include #include #include #include #include #include #include -#include namespace DB { @@ -48,8 +47,43 @@ namespace ErrorCodes extern const int UNKNOWN_DATABASE; extern const int UNKNOWN_EXCEPTION; extern const int CANNOT_READ_ALL_DATA; + extern const int ATTEMPT_TO_READ_AFTER_EOF; + extern const int NETWORK_ERROR; + extern const int UNKNOWN_TABLE; + extern const int CANNOT_GET_CREATE_TABLE_QUERY; + extern const int THERE_IS_NO_QUERY; + extern const int QUERY_WAS_CANCELLED; + extern const int TABLE_ALREADY_EXISTS; + extern const int DATABASE_ALREADY_EXISTS; + extern const int DATABASE_NOT_EMPTY; + extern const int TABLE_IS_DROPPED; + extern const int TABLE_SIZE_EXCEEDS_MAX_DROP_SIZE_LIMIT; + extern const int CANNOT_CREATE_CHARSET_CONVERTER; + extern const int UNKNOWN_FUNCTION; + extern const int UNKNOWN_IDENTIFIER; + extern const int UNKNOWN_TYPE; + extern const int TIMEOUT_EXCEEDED; + extern const int MEMORY_LIMIT_EXCEEDED; + extern const int MYSQL_SYNTAX_ERROR; } +// USE MySQL ERROR CODE: +// https://dev.mysql.com/doc/mysql-errors/5.7/en/server-error-reference.html +constexpr int ER_ACCESS_DENIED_ERROR = 1045; /// NOLINT +constexpr int ER_DBACCESS_DENIED_ERROR = 1044; /// NOLINT +constexpr int ER_BAD_DB_ERROR = 1049; /// NOLINT +constexpr int ER_MASTER_HAS_PURGED_REQUIRED_GTIDS = 1789; /// NOLINT +constexpr int ER_MASTER_FATAL_ERROR_READING_BINLOG = 1236; /// NOLINT + +// https://dev.mysql.com/doc/mysql-errors/8.0/en/client-error-reference.html +constexpr int CR_CONN_HOST_ERROR = 2003; /// NOLINT +constexpr int CR_SERVER_GONE_ERROR = 2006; /// NOLINT +constexpr int CR_SERVER_LOST = 2013; /// NOLINT +constexpr int ER_SERVER_SHUTDOWN = 1053; /// NOLINT +constexpr int ER_LOCK_DEADLOCK = 1213; /// NOLINT +constexpr int ER_LOCK_WAIT_TIMEOUT = 1205; /// NOLINT +constexpr int ER_OPTION_PREVENTS_STATEMENT = 1290; /// NOLINT + static constexpr auto MYSQL_BACKGROUND_THREAD_NAME = "MySQLDBSync"; static ContextMutablePtr createQueryContext(ContextPtr context) @@ -157,12 +191,68 @@ static void checkMySQLVariables(const mysqlxx::Pool::Entry & connection, const S } } +static bool shouldReconnectOnException(const std::exception_ptr & e) +{ + try + { + std::rethrow_exception(e); + } + catch (const mysqlxx::ConnectionFailed &) {} /// NOLINT + catch (const mysqlxx::ConnectionLost &) {} /// NOLINT + catch (const Poco::Net::ConnectionResetException &) {} /// NOLINT + catch (const Poco::Net::ConnectionRefusedException &) {} /// NOLINT + catch (const DB::NetException &) {} /// NOLINT + catch (const Poco::Net::NetException & e) + { + if (e.code() != POCO_ENETDOWN && + e.code() != POCO_ENETUNREACH && + e.code() != POCO_ENETRESET && + e.code() != POCO_ESYSNOTREADY) + return false; + } + catch (const mysqlxx::BadQuery & e) + { + // Lost connection to MySQL server during query + if (e.code() != CR_SERVER_LOST && + e.code() != ER_SERVER_SHUTDOWN && + e.code() != CR_SERVER_GONE_ERROR && + e.code() != CR_CONN_HOST_ERROR && + e.code() != ER_LOCK_DEADLOCK && + e.code() != ER_LOCK_WAIT_TIMEOUT && + e.code() != ER_OPTION_PREVENTS_STATEMENT) + return false; + } + catch (const mysqlxx::Exception & e) + { + // ER_SERVER_SHUTDOWN is thrown in different types under different conditions. + // E.g. checkError() in Common/mysqlxx/Exception.cpp will throw mysqlxx::Exception. + if (e.code() != CR_SERVER_LOST && e.code() != ER_SERVER_SHUTDOWN && e.code() != CR_SERVER_GONE_ERROR && e.code() != CR_CONN_HOST_ERROR) + return false; + } + catch (const Poco::Exception & e) + { + if (e.code() != ErrorCodes::NETWORK_ERROR && + e.code() != ErrorCodes::MEMORY_LIMIT_EXCEEDED && + e.code() != ErrorCodes::UNKNOWN_TABLE && // Since we have ignored the DDL exception when the tables without primary key, insert into those tables will get UNKNOWN_TABLE. + e.code() != ErrorCodes::CANNOT_READ_ALL_DATA && + e.code() != ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF && + e.code() != ErrorCodes::TIMEOUT_EXCEEDED) + return false; + } + catch (...) + { + return false; + } + return true; +} + MaterializedMySQLSyncThread::MaterializedMySQLSyncThread( ContextPtr context_, const String & database_name_, const String & mysql_database_name_, mysqlxx::Pool && pool_, MySQLClient && client_, + const MySQLReplication::BinlogClientPtr & binlog_client_, MaterializedMySQLSettings * settings_) : WithContext(context_->getGlobalContext()) , log(&Poco::Logger::get("MaterializedMySQLSyncThread")) @@ -170,6 +260,7 @@ MaterializedMySQLSyncThread::MaterializedMySQLSyncThread( , mysql_database_name(mysql_database_name_) , pool(std::move(pool_)) /// NOLINT , client(std::move(client_)) + , binlog_client(binlog_client_) , settings(settings_) { query_prefix = "EXTERNAL DDL FROM MySQL(" + backQuoteIfNeed(database_name) + ", " + backQuoteIfNeed(mysql_database_name) + ") "; @@ -216,14 +307,23 @@ void MaterializedMySQLSyncThread::synchronization() UInt64 elapsed_ms = watch.elapsedMilliseconds(); if (elapsed_ms < max_flush_time) { - BinlogEventPtr binlog_event = client.readOneBinlogEvent(max_flush_time - elapsed_ms); - if (binlog_event) + const auto timeout_ms = max_flush_time - elapsed_ms; + BinlogEventPtr binlog_event; + if (binlog) + binlog->tryReadEvent(binlog_event, timeout_ms); + else + binlog_event = client.readOneBinlogEvent(timeout_ms); + if (binlog_event && !ignoreEvent(binlog_event)) onEvent(buffers, binlog_event, metadata); } } catch (const Exception & e) { - if (e.code() != ErrorCodes::CANNOT_READ_ALL_DATA || settings->max_wait_time_when_mysql_unavailable < 0) + if (settings->max_wait_time_when_mysql_unavailable < 0) + throw; + bool binlog_was_purged = e.code() == ER_MASTER_FATAL_ERROR_READING_BINLOG || + e.code() == ER_MASTER_HAS_PURGED_REQUIRED_GTIDS; + if (!binlog_was_purged && !shouldReconnectOnException(std::current_exception())) throw; flushBuffersData(buffers, metadata); @@ -246,6 +346,7 @@ void MaterializedMySQLSyncThread::synchronization() catch (...) { client.disconnect(); + binlog = nullptr; tryLogCurrentException(log); setSynchronizationThreadException(std::current_exception()); } @@ -259,6 +360,7 @@ void MaterializedMySQLSyncThread::stopSynchronization() if (background_thread_pool->joinable()) background_thread_pool->join(); client.disconnect(); + binlog = nullptr; } } @@ -428,14 +530,6 @@ static inline void dumpDataForTables( } } -static inline UInt32 randomNumber() -{ - pcg64_fast rng{randomSeed()}; - std::uniform_int_distribution dist6( - std::numeric_limits::min(), std::numeric_limits::max()); - return static_cast(dist6(rng)); -} - bool MaterializedMySQLSyncThread::prepareSynchronized(MaterializeMetadata & metadata) { bool opened_transaction = false; @@ -463,7 +557,7 @@ bool MaterializedMySQLSyncThread::prepareSynchronized(MaterializeMetadata & meta if (!need_dumping_tables.empty()) { Position position; - position.update(metadata.binlog_position, metadata.binlog_file, metadata.executed_gtid_set); + position.update(metadata.binlog_position, metadata.binlog_file, metadata.executed_gtid_set, 0); metadata.transaction(position, [&]() { @@ -487,8 +581,20 @@ bool MaterializedMySQLSyncThread::prepareSynchronized(MaterializeMetadata & meta if (opened_transaction) connection->query("COMMIT").execute(); - client.connect(); - client.startBinlogDumpGTID(randomNumber(), mysql_database_name, materialized_tables_list, metadata.executed_gtid_set, metadata.binlog_checksum); + if (binlog_client) + { + binlog_client->setBinlogChecksum(metadata.binlog_checksum); + binlog = binlog_client->createBinlog(metadata.executed_gtid_set, + database_name, + {mysql_database_name}, + settings->max_bytes_in_binlog_queue, + settings->max_milliseconds_to_wait_in_binlog_queue); + } + else + { + client.connect(); + client.startBinlogDumpGTID(randomNumber(), mysql_database_name, materialized_tables_list, metadata.executed_gtid_set, metadata.binlog_checksum); + } setSynchronizationThreadException(nullptr); return true; @@ -500,20 +606,11 @@ bool MaterializedMySQLSyncThread::prepareSynchronized(MaterializeMetadata & meta if (opened_transaction) connection->query("ROLLBACK").execute(); - try - { + if (settings->max_wait_time_when_mysql_unavailable < 0) + throw; + + if (!shouldReconnectOnException(std::current_exception())) throw; - } - catch (const mysqlxx::ConnectionFailed & ex) - { - LOG_TRACE(log, "Connection to MySQL failed {}", ex.displayText()); - } - catch (const mysqlxx::BadQuery & e) - { - // Lost connection to MySQL server during query - if (e.code() != CR_SERVER_LOST || settings->max_wait_time_when_mysql_unavailable < 0) - throw; - } setSynchronizationThreadException(std::current_exception()); /// Avoid busy loop when MySQL is not available. @@ -524,17 +621,55 @@ bool MaterializedMySQLSyncThread::prepareSynchronized(MaterializeMetadata & meta return false; } +bool MaterializedMySQLSyncThread::isTableIgnored(const String & table_name) const +{ + return !materialized_tables_list.empty() && !materialized_tables_list.contains(table_name); +} + +bool MaterializedMySQLSyncThread::ignoreEvent(const BinlogEventPtr & event) const +{ + switch (event->type()) + { + case MYSQL_WRITE_ROWS_EVENT: + case MYSQL_DELETE_ROWS_EVENT: + case MYSQL_UPDATE_ROWS_EVENT: + case MYSQL_UNPARSED_ROWS_EVENT: + { + auto table_name = static_cast(*event).table; + if (!table_name.empty() && isTableIgnored(table_name)) + { + switch (event->header.type) + { + case WRITE_ROWS_EVENT_V1: + case WRITE_ROWS_EVENT_V2: + case DELETE_ROWS_EVENT_V1: + case DELETE_ROWS_EVENT_V2: + case UPDATE_ROWS_EVENT_V1: + case UPDATE_ROWS_EVENT_V2: + break; + default: + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown event type: {}", magic_enum::enum_name(event->header.type)); + } + return true; + } + } break; + default: + break; + } + return false; +} + void MaterializedMySQLSyncThread::flushBuffersData(Buffers & buffers, MaterializeMetadata & metadata) { if (buffers.data.empty()) return; - metadata.transaction(client.getPosition(), [&]() { buffers.commit(getContext()); }); + metadata.transaction(getPosition(), [&]() { buffers.commit(getContext()); }); const auto & position_message = [&]() { WriteBufferFromOwnString buf; - client.getPosition().dump(buf); + getPosition().dump(buf); return buf.str(); }; LOG_INFO(log, "MySQL executed position: \n {}", position_message()); @@ -783,10 +918,33 @@ void MaterializedMySQLSyncThread::onEvent(Buffers & buffers, const BinlogEventPt else if (receive_event->type() == MYSQL_QUERY_EVENT) { QueryEvent & query_event = static_cast(*receive_event); + /// Skip events for different databases if any + if (!query_event.query_database_name.empty() && query_event.query_database_name != mysql_database_name) + { + LOG_WARNING( + log, + "Skipped QueryEvent, current mysql database name: {}, ddl schema: {}, query: {}", + mysql_database_name, + query_event.query_database_name, + query_event.query); + return; + } + if (!query_event.query_table_name.empty() && isTableIgnored(query_event.query_table_name)) + { + LOG_WARNING(log, "Due to the table filter rules, query_event on {} is ignored.", database_name); + return; + } + Position position_before_ddl; - position_before_ddl.update(metadata.binlog_position, metadata.binlog_file, metadata.executed_gtid_set); + position_before_ddl.update(metadata.binlog_position, metadata.binlog_file, metadata.executed_gtid_set, query_event.header.timestamp); metadata.transaction(position_before_ddl, [&]() { buffers.commit(getContext()); }); - metadata.transaction(client.getPosition(),[&](){ executeDDLAtomic(query_event); }); + metadata.transaction(getPosition(),[&]() { executeDDLAtomic(query_event); }); + } + else if (receive_event->type() == MYSQL_UNPARSED_ROWS_EVENT) + { + UnparsedRowsEvent & unparsed_event = static_cast(*receive_event); + auto nested_event = unparsed_event.parse(); + onEvent(buffers, nested_event, metadata); } else { @@ -796,7 +954,10 @@ void MaterializedMySQLSyncThread::onEvent(Buffers & buffers, const BinlogEventPt /// Some behaviors(such as changing the value of "binlog_checksum") rotate the binlog file. /// To ensure that the synchronization continues, we need to handle these events metadata.fetchMasterVariablesValue(pool.get(/* wait_timeout= */ UINT64_MAX)); - client.setBinlogChecksum(metadata.binlog_checksum); + if (binlog_client) + binlog_client->setBinlogChecksum(metadata.binlog_checksum); + else + client.setBinlogChecksum(metadata.binlog_checksum); } else if (receive_event->header.type != HEARTBEAT_EVENT) { @@ -827,7 +988,7 @@ void MaterializedMySQLSyncThread::executeDDLAtomic(const QueryEvent & query_even auto table_id = tryParseTableIDFromDDL(query, query_event.schema); if (!table_id.table_name.empty()) { - if (table_id.database_name != mysql_database_name || !materialized_tables_list.contains(table_id.table_name)) + if (table_id.database_name != mysql_database_name || isTableIgnored(table_id.table_name)) { LOG_DEBUG(log, "Skip MySQL DDL for {}.{}:\n{}", table_id.database_name, table_id.table_name, query); return; @@ -845,8 +1006,28 @@ void MaterializedMySQLSyncThread::executeDDLAtomic(const QueryEvent & query_even tryLogCurrentException(log); /// If some DDL query was not successfully parsed and executed - /// Then replication may fail on next binlog events anyway - if (exception.code() != ErrorCodes::SYNTAX_ERROR) + /// Then replication may fail on next binlog events anyway. + /// We can skip the error binlog evetns and continue to execute the right ones. + /// eg. The user creates a table without primary key and finds it is wrong, then + /// drops it and creates a new right one. We guarantee the right one can be executed. + + if (exception.code() != ErrorCodes::SYNTAX_ERROR && + exception.code() != ErrorCodes::MYSQL_SYNTAX_ERROR && + exception.code() != ErrorCodes::NOT_IMPLEMENTED && + exception.code() != ErrorCodes::UNKNOWN_TABLE && + exception.code() != ErrorCodes::CANNOT_GET_CREATE_TABLE_QUERY && + exception.code() != ErrorCodes::THERE_IS_NO_QUERY && + exception.code() != ErrorCodes::QUERY_WAS_CANCELLED && + exception.code() != ErrorCodes::TABLE_ALREADY_EXISTS && + exception.code() != ErrorCodes::UNKNOWN_DATABASE && + exception.code() != ErrorCodes::DATABASE_ALREADY_EXISTS && + exception.code() != ErrorCodes::DATABASE_NOT_EMPTY && + exception.code() != ErrorCodes::TABLE_IS_DROPPED && + exception.code() != ErrorCodes::TABLE_SIZE_EXCEEDS_MAX_DROP_SIZE_LIMIT && + exception.code() != ErrorCodes::CANNOT_CREATE_CHARSET_CONVERTER && + exception.code() != ErrorCodes::UNKNOWN_FUNCTION && + exception.code() != ErrorCodes::UNKNOWN_IDENTIFIER && + exception.code() != ErrorCodes::UNKNOWN_TYPE) throw; } } diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.h b/src/Databases/MySQL/MaterializedMySQLSyncThread.h index 4abea5e72dfb..004a4d67d324 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.h +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.h @@ -11,6 +11,7 @@ # include # include # include +# include # include # include # include @@ -45,6 +46,7 @@ class MaterializedMySQLSyncThread : WithContext const String & mysql_database_name_, mysqlxx::Pool && pool_, MySQLClient && client_, + const MySQLReplication::BinlogClientPtr & binlog_client_, MaterializedMySQLSettings * settings_); void stopSynchronization(); @@ -61,19 +63,12 @@ class MaterializedMySQLSyncThread : WithContext mutable mysqlxx::Pool pool; mutable MySQLClient client; + BinlogClientPtr binlog_client; + BinlogPtr binlog; MaterializedMySQLSettings * settings; String query_prefix; NameSet materialized_tables_list; - // USE MySQL ERROR CODE: - // https://dev.mysql.com/doc/mysql-errors/5.7/en/server-error-reference.html - const int ER_ACCESS_DENIED_ERROR = 1045; /// NOLINT - const int ER_DBACCESS_DENIED_ERROR = 1044; /// NOLINT - const int ER_BAD_DB_ERROR = 1049; /// NOLINT - - // https://dev.mysql.com/doc/mysql-errors/8.0/en/client-error-reference.html - const int CR_SERVER_LOST = 2013; /// NOLINT - struct Buffers { String database; @@ -99,12 +94,16 @@ class MaterializedMySQLSyncThread : WithContext BufferAndSortingColumnsPtr getTableDataBuffer(const String & table, ContextPtr context); }; + Position getPosition() const { return binlog ? binlog->getPosition() : client.getPosition(); } void synchronization(); bool isCancelled() { return sync_quit.load(std::memory_order_relaxed); } bool prepareSynchronized(MaterializeMetadata & metadata); + bool isTableIgnored(const String & table_name) const; + bool ignoreEvent(const BinlogEventPtr & event) const; + void flushBuffersData(Buffers & buffers, MaterializeMetadata & metadata); void onEvent(Buffers & buffers, const MySQLReplication::BinlogEventPtr & event, MaterializeMetadata & metadata); diff --git a/src/Databases/MySQL/MySQLBinlog.cpp b/src/Databases/MySQL/MySQLBinlog.cpp new file mode 100644 index 000000000000..3e3aca220bbc --- /dev/null +++ b/src/Databases/MySQL/MySQLBinlog.cpp @@ -0,0 +1,500 @@ +#include "MySQLBinlog.h" +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include + +namespace DB +{ +using namespace Replication; +using namespace Authentication; +using namespace ConnectionPhase; + +namespace ErrorCodes +{ + extern const int UNKNOWN_EXCEPTION; + extern const int UNKNOWN_PACKET_FROM_SERVER; + extern const int ATTEMPT_TO_READ_AFTER_EOF; + extern const int CANNOT_READ_ALL_DATA; + extern const int LOGICAL_ERROR; + extern const int NETWORK_ERROR; +} + +namespace MySQLReplication +{ + +class WriteCommand : public IMySQLWritePacket +{ +public: + const char command; + const String query; + + WriteCommand(char command_, const String & query_) : command(command_), query(query_) { } + + size_t getPayloadSize() const override { return 1 + query.size(); } + + void writePayloadImpl(WriteBuffer & buffer) const override + { + buffer.write(command); + if (!query.empty()) + buffer.write(query.data(), query.size()); + } +}; + +IBinlog::Checksum IBinlog::checksumFromString(const String & checksum) +{ + auto str = Poco::toUpper(checksum); + if (str == "CRC32") + return IBinlog::CRC32; + if (str != "NONE") + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown checksum: {}", checksum); + return IBinlog::NONE; +} + +void BinlogParser::setChecksum(Checksum checksum) +{ + switch (checksum) + { + case Checksum::CRC32: + checksum_signature_length = 4; + break; + case Checksum::NONE: + checksum_signature_length = 0; + break; + } +} + +void BinlogParser::parseEvent(EventHeader & event_header, ReadBuffer & event_payload) +{ + switch (event_header.type) + { + case FORMAT_DESCRIPTION_EVENT: + { + event = std::make_shared(EventHeader(event_header)); + event->parseEvent(event_payload); + break; + } + case ROTATE_EVENT: + { + event = std::make_shared(EventHeader(event_header)); + event->parseEvent(event_payload); + break; + } + case QUERY_EVENT: + { + event = std::make_shared(EventHeader(event_header)); + event->parseEvent(event_payload); + + auto query = std::static_pointer_cast(event); + switch (query->typ) + { + case QUERY_EVENT_MULTI_TXN_FLAG: + case QUERY_EVENT_XA: + case QUERY_SAVEPOINT: + { + event = std::make_shared(EventHeader(query->header)); + break; + } + default: + { + String quoted_query = query->query; + tryQuoteUnrecognizedTokens(quoted_query); + tryConvertStringLiterals(quoted_query); + auto table_id = tryParseTableIDFromDDL(query->query, query->schema); + query->query_database_name = table_id.database_name; + query->query_table_name = table_id.table_name; + break; + } + } + break; + } + case XID_EVENT: + { + event = std::make_shared(EventHeader(event_header)); + event->parseEvent(event_payload); + break; + } + case TABLE_MAP_EVENT: + { + TableMapEventHeader map_event_header; + map_event_header.parse(event_payload); + event = std::make_shared(EventHeader(event_header), map_event_header, flavor_charset); + try + { + event->parseEvent(event_payload); + auto table_map = std::static_pointer_cast(event); + table_maps[table_map->table_id] = table_map; + } + catch (const Poco::Exception & exc) + { + /// Ignore parsing issues + if (exc.code() != ErrorCodes::UNKNOWN_EXCEPTION) + throw; + event = std::make_shared(std::move(event_header)); + event->parseEvent(event_payload); + } + break; + } + case WRITE_ROWS_EVENT_V1: + case WRITE_ROWS_EVENT_V2: + case DELETE_ROWS_EVENT_V1: + case DELETE_ROWS_EVENT_V2: + case UPDATE_ROWS_EVENT_V1: + case UPDATE_ROWS_EVENT_V2: + { + RowsEventHeader rows_header(event_header.type); + rows_header.parse(event_payload); + if (table_maps.contains(rows_header.table_id)) + event = std::make_shared(table_maps.at(rows_header.table_id), EventHeader(event_header), rows_header); + else + event = std::make_shared(std::move(event_header)); + event->parseEvent(event_payload); + if (rows_header.flags & ROWS_END_OF_STATEMENT) + table_maps.clear(); + break; + } + case GTID_EVENT: + { + event = std::make_shared(EventHeader(event_header)); + event->parseEvent(event_payload); + break; + } + default: + { + event = std::make_shared(EventHeader(event_header)); + event->parseEvent(event_payload); + break; + } + } + updatePosition(event, position); +} + +void BinlogParser::updatePosition(const BinlogEventPtr & event, Position & position) +{ + const UInt64 binlog_pos_prev = position.binlog_pos; + position.binlog_pos = event->header.log_pos; + if (event->header.timestamp > 0) + position.timestamp = event->header.timestamp; + + switch (event->header.type) + { + case QUERY_EVENT: + if (event->type() == MYSQL_UNHANDLED_EVENT) + break; + [[fallthrough]]; + case GTID_EVENT: + case XID_EVENT: + case ROTATE_EVENT: + position.update(event); + break; + default: + break; + } + + if (event->header.type != ROTATE_EVENT) + { + /// UInt32 overflow when Pos > End_log_pos + /// https://dev.mysql.com/doc/refman/8.0/en/show-binlog-events.html + /// binlog_pos - The position at which the next event begins, which is equal to Pos plus the size of the event + const UInt64 binlog_pos_correct = binlog_pos_prev + event->header.event_size; + if (position.binlog_pos < binlog_pos_prev && binlog_pos_correct > std::numeric_limits::max()) + position.binlog_pos = binlog_pos_correct; + } +} + +bool BinlogParser::isNew(const Position & older, const Position & newer) +{ + if (older.gtid_sets.contains(newer.gtid_sets)) + return false; + /// Check if all sets in newer position have the same UUID from older sets + std::set older_set; + for (const auto & set : older.gtid_sets.sets) + older_set.insert(set.uuid); + for (const auto & set : newer.gtid_sets.sets) + { + if (!older_set.contains(set.uuid)) + return false; + } + return true; +} + +void BinlogFromSocket::connect(const String & host, UInt16 port, const String & user, const String & password) +{ + if (connected) + disconnect(); + + const Poco::Timespan connection_timeout(10'000'000'000); + const Poco::Timespan receive_timeout(5'000'000'000); + const Poco::Timespan send_timeout(5'000'000'000); + + socket = std::make_unique(); + address = DNSResolver::instance().resolveAddress(host, port); + socket->connect(*address, connection_timeout); + socket->setReceiveTimeout(receive_timeout); + socket->setSendTimeout(send_timeout); + socket->setNoDelay(true); + connected = true; + + in = std::make_unique(*socket); + out = std::make_unique(*socket); + packet_endpoint = std::make_shared(*in, *out, sequence_id); + + handshake(user, password); +} + +void BinlogFromSocket::disconnect() +{ + in = nullptr; + out = nullptr; + if (socket) + socket->close(); + socket = nullptr; + connected = false; + sequence_id = 0; + + GTIDSets sets; + position.gtid_sets = sets; + position.resetPendingGTID(); +} + +/// https://dev.mysql.com/doc/internals/en/connection-phase-packets.html +void BinlogFromSocket::handshake(const String & user, const String & password) +{ + const String mysql_native_password = "mysql_native_password"; + Handshake handshake; + packet_endpoint->receivePacket(handshake); + if (handshake.auth_plugin_name != mysql_native_password) + { + throw Exception( + ErrorCodes::UNKNOWN_PACKET_FROM_SERVER, + "Only support {} auth plugin name, but got {}", + mysql_native_password, + handshake.auth_plugin_name); + } + + Native41 native41(password, handshake.auth_plugin_data); + String auth_plugin_data = native41.getAuthPluginData(); + + const UInt8 charset_utf8 = 33; + HandshakeResponse handshake_response( + client_capabilities, MAX_PACKET_LENGTH, charset_utf8, user, "", auth_plugin_data, mysql_native_password); + packet_endpoint->sendPacket(handshake_response, true); + + ResponsePacket packet_response(client_capabilities, true); + packet_endpoint->receivePacket(packet_response); + packet_endpoint->resetSequenceId(); + + if (packet_response.getType() == PACKET_ERR) + throw Exception::createDeprecated(packet_response.err.error_message, ErrorCodes::UNKNOWN_PACKET_FROM_SERVER); + else if (packet_response.getType() == PACKET_AUTH_SWITCH) + throw Exception(ErrorCodes::UNKNOWN_PACKET_FROM_SERVER, "Access denied for user {}", user); +} + +void BinlogFromSocket::writeCommand(char command, const String & query) +{ + WriteCommand write_command(command, query); + packet_endpoint->sendPacket(write_command, true); + + ResponsePacket packet_response(client_capabilities); + packet_endpoint->receivePacket(packet_response); + switch (packet_response.getType()) + { + case PACKET_ERR: + throw Exception::createDeprecated(packet_response.err.error_message, ErrorCodes::UNKNOWN_PACKET_FROM_SERVER); + case PACKET_OK: + break; + default: + break; + } + packet_endpoint->resetSequenceId(); +} + +void BinlogFromSocket::registerSlaveOnMaster(UInt32 slave_id) +{ + RegisterSlave register_slave(slave_id); + packet_endpoint->sendPacket(register_slave, true); + + ResponsePacket packet_response(client_capabilities); + packet_endpoint->receivePacket(packet_response); + packet_endpoint->resetSequenceId(); + if (packet_response.getType() == PACKET_ERR) + throw Exception::createDeprecated(packet_response.err.error_message, ErrorCodes::UNKNOWN_PACKET_FROM_SERVER); +} + +void BinlogFromSocket::start(UInt32 slave_id, const String & executed_gtid_set) +{ + if (!connected) + return; + + /// Maybe CRC32 or NONE. mysqlbinlog.cc use NONE, see its below comments: + /// Make a notice to the server that this client is checksum-aware. + /// It does not need the first fake Rotate necessary checksummed. + writeCommand(Command::COM_QUERY, "SET @master_binlog_checksum = 'CRC32'"); + + /// Set heartbeat 1s + const UInt64 period_ns = 1'000'000'000; + writeCommand(Command::COM_QUERY, "SET @master_heartbeat_period = " + std::to_string(period_ns)); + + /// Register slave. + registerSlaveOnMaster(slave_id); + + position.gtid_sets = {}; + position.gtid_sets.parse(executed_gtid_set); + + BinlogDumpGTID binlog_dump(slave_id, position.gtid_sets.toPayload()); + packet_endpoint->sendPacket(binlog_dump, true); +} + +class ReadPacketFromSocket : public IMySQLReadPacket +{ +public: + using ReadPayloadFunc = std::function; + explicit ReadPacketFromSocket(ReadPayloadFunc fn) : read_payload_func(std::move(fn)) { } + void readPayloadImpl(ReadBuffer & payload) override; + ReadPayloadFunc read_payload_func; +}; + +void ReadPacketFromSocket::readPayloadImpl(ReadBuffer & payload) +{ + if (payload.eof()) + throw Exception(ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF, "Attempt to read after EOF."); + + UInt8 header = static_cast(*payload.position()); + switch (header) // NOLINT(bugprone-switch-missing-default-case) + { + case PACKET_EOF: + throw ReplicationError(ErrorCodes::CANNOT_READ_ALL_DATA, "Master maybe lost"); + case PACKET_ERR: + { + ERRPacket err; + err.readPayloadWithUnpacked(payload); + throw ReplicationError::createDeprecated(err.error_message, ErrorCodes::UNKNOWN_EXCEPTION); + } + default: + break; + } + /// Skip the generic response packets header flag + payload.ignore(1); + read_payload_func(payload); +} + +bool BinlogFromSocket::tryReadEvent(BinlogEventPtr & to, UInt64 ms) +{ + ReadPacketFromSocket packet([this](ReadBuffer & payload) + { + MySQLBinlogEventReadBuffer event_payload(payload, checksum_signature_length); + + EventHeader event_header; + event_header.parse(event_payload); + + parseEvent(event_header, event_payload); + }); + + if (packet_endpoint && packet_endpoint->tryReceivePacket(packet, ms)) + { + to = event; + return static_cast(to); + } + + return false; +} + +void BinlogFromFile::open(const String & filename) +{ + in = std::make_unique(filename); + assertString("\xfe\x62\x69\x6e", *in); /// magic number +} + +bool BinlogFromFile::tryReadEvent(BinlogEventPtr & to, UInt64 /*ms*/) +{ + if (in && !in->eof()) + { + EventHeader event_header; + event_header.parse(*in); + + LimitReadBuffer limit_read_buffer(*in, event_header.event_size - EVENT_HEADER_LENGTH, /* throw_exception */ false, /* exact_limit */ {}); + MySQLBinlogEventReadBuffer event_payload(limit_read_buffer, checksum_signature_length); + parseEvent(event_header, event_payload); + to = event; + return static_cast(to); + } + + return false; +} + +BinlogFromFileFactory::BinlogFromFileFactory(const String & filename_) + : filename(filename_) +{ +} + +BinlogPtr BinlogFromFileFactory::createBinlog(const String & executed_gtid_set) +{ + auto ret = std::make_shared(); + ret->open(filename); + if (!executed_gtid_set.empty()) + { + /// NOTE: Used for testing only! + GTIDSets sets; + sets.parse(executed_gtid_set); + if (sets.sets.size() != 1 || sets.sets[0].intervals.size() != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Too many intervals: {}", executed_gtid_set); + BinlogEventPtr event; + while (ret->tryReadEvent(event, 0)) + { + const auto & s = ret->getPosition().gtid_sets.sets; + if (!s.empty() && !s[0].intervals.empty() && s[0].intervals[0].end == sets.sets[0].intervals[0].end) + break; + } + + auto pos = ret->getPosition(); + pos.gtid_sets.sets.front().intervals.front().start = sets.sets.front().intervals.front().start; + ret->setPosition(pos); + } + return ret; +} + +BinlogFromSocketFactory::BinlogFromSocketFactory(const String & host_, UInt16 port_, const String & user_, const String & password_) + : host(host_) + , port(port_) + , user(user_) + , password(password_) +{ +} + +BinlogPtr BinlogFromSocketFactory::createBinlog(const String & executed_gtid_set) +{ + auto ret = std::make_shared(); + ret->connect(host, port, user, password); + ret->start(randomNumber(), executed_gtid_set); + auto pos = ret->getPosition(); + if (pos.gtid_sets.sets.empty() || pos.gtid_sets.sets.front().intervals.front().start != 1) + throw Exception(ErrorCodes::NETWORK_ERROR, "Could not create: Wrong executed_gtid_set: {} -> {}", executed_gtid_set, pos.gtid_sets.toString()); + return ret; +} + +/// Should be in MySQLReplication namespace +bool operator==(const Position & left, const Position & right) +{ + return left.binlog_name == right.binlog_name && + left.binlog_pos == right.binlog_pos && + left.gtid_sets == right.gtid_sets; +} + +} +} diff --git a/src/Databases/MySQL/MySQLBinlog.h b/src/Databases/MySQL/MySQLBinlog.h new file mode 100644 index 000000000000..0b8f7543590f --- /dev/null +++ b/src/Databases/MySQL/MySQLBinlog.h @@ -0,0 +1,120 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ +using namespace MySQLProtocol; +using namespace Generic; + +namespace MySQLReplication +{ + +class IBinlog +{ +public: + virtual ~IBinlog() = default; + virtual bool tryReadEvent(BinlogEventPtr & to, UInt64 ms) = 0; + virtual Position getPosition() const = 0; + enum Checksum : UInt8 + { + NONE = 0, + CRC32 = 1 + }; + virtual void setChecksum(Checksum /*checksum*/) { } + static Checksum checksumFromString(const String & checksum); +}; + +using BinlogPtr = std::shared_ptr; + +class BinlogParser : public IBinlog +{ +public: + Position getPosition() const override { return position; } + void setPosition(const Position & position_) { position = position_; } + void setChecksum(Checksum checksum) override; + static void updatePosition(const BinlogEventPtr & event, Position & position); + /// Checks if \a older is older position than \a newer + static bool isNew(const Position & older, const Position & newer); + +protected: + Position position; + BinlogEventPtr event; + std::map> table_maps; + size_t checksum_signature_length = 4; + MySQLCharsetPtr flavor_charset = std::make_shared(); + void parseEvent(EventHeader & event_header, ReadBuffer & event_payload); +}; + +class BinlogFromSocket : public BinlogParser +{ +public: + void connect(const String & host, UInt16 port, const String & user, const String & password); + void start(UInt32 slave_id, const String & executed_gtid_set); + bool tryReadEvent(BinlogEventPtr & to, UInt64 ms) override; + +private: + void disconnect(); + bool connected = false; + uint8_t sequence_id = 0; + const uint32_t client_capabilities = CLIENT_PROTOCOL_41 | CLIENT_PLUGIN_AUTH | CLIENT_SECURE_CONNECTION; + + std::unique_ptr in; + std::unique_ptr out; + std::unique_ptr socket; + std::optional address; + std::shared_ptr packet_endpoint; + + void handshake(const String & user, const String & password); + void registerSlaveOnMaster(UInt32 slave_id); + void writeCommand(char command, const String & query); +}; + +class BinlogFromFile : public BinlogParser +{ +public: + void open(const String & filename); + bool tryReadEvent(BinlogEventPtr & to, UInt64 ms) override; + +private: + std::unique_ptr in; +}; + +class IBinlogFactory +{ +public: + virtual ~IBinlogFactory() = default; + virtual BinlogPtr createBinlog(const String & executed_gtid_set) = 0; +}; + +using BinlogFactoryPtr = std::shared_ptr; + +class BinlogFromFileFactory : public IBinlogFactory +{ +public: + BinlogFromFileFactory(const String & filename_); + BinlogPtr createBinlog(const String & executed_gtid_set) override; + +private: + const String filename; +}; + +class BinlogFromSocketFactory : public IBinlogFactory +{ +public: + BinlogFromSocketFactory(const String & host_, UInt16 port_, const String & user_, const String & password_); + BinlogPtr createBinlog(const String & executed_gtid_set) override; + +private: + const String host; + const UInt16 port; + const String user; + const String password; +}; + +bool operator==(const Position & left, const Position & right); + +} +} diff --git a/src/Databases/MySQL/MySQLBinlogClient.cpp b/src/Databases/MySQL/MySQLBinlogClient.cpp new file mode 100644 index 000000000000..e7d707f76ce4 --- /dev/null +++ b/src/Databases/MySQL/MySQLBinlogClient.cpp @@ -0,0 +1,104 @@ +#include "MySQLBinlogClient.h" +#include + +namespace DB::ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +namespace DB::MySQLReplication +{ + +BinlogClient::BinlogClient(const BinlogFactoryPtr & factory_, + const String & name, + UInt64 max_bytes_in_buffer_, + UInt64 max_flush_ms_) + : factory(factory_) + , binlog_client_name(name) + , max_bytes_in_buffer(max_bytes_in_buffer_) + , max_flush_ms(max_flush_ms_) + , logger(&Poco::Logger::get("BinlogClient(" + name + ")")) +{ +} + +BinlogPtr BinlogClient::createBinlog(const String & executed_gtid_set, + const String & name, + const NameSet & mysql_database_names, + size_t max_bytes, + UInt64 max_waiting_ms) +{ + std::lock_guard lock(mutex); + BinlogPtr ret; + for (auto it = dispatchers.begin(); it != dispatchers.end();) + { + auto & dispatcher = *it; + if (!ret) + { + const auto metadata = dispatcher->getDispatcherMetadata(); + LOG_DEBUG(logger, "({} -> {}): Trying dispatcher: {}, size: {} -> {}:{}.{}", + name, executed_gtid_set, metadata.name, metadata.binlogs.size(), + metadata.position.binlog_name, metadata.position.gtid_sets.toString(), metadata.position.binlog_pos); + ret = dispatcher->attach(executed_gtid_set, name, mysql_database_names, max_bytes, max_waiting_ms); + if (ret) + LOG_DEBUG(logger, "({} -> {}): Reused dispatcher: {}, size: {} -> {}:{}.{}", + name, executed_gtid_set, metadata.name, metadata.binlogs.size(), + metadata.position.binlog_name, metadata.position.gtid_sets.toString(), metadata.position.binlog_pos); + } + + if (dispatcher->cleanupBinlogsAndStop()) + { + const auto metadata = dispatcher->getDispatcherMetadata(); + LOG_DEBUG(logger, "({} -> {}): Deleting dispatcher: {}, size: {}, total dispatchers: {}", + name, executed_gtid_set, metadata.name, metadata.binlogs.size(), dispatchers.size()); + it = dispatchers.erase(it); + continue; + } + ++it; + } + + if (!ret) + { + String dispatcher_name = name + ":" + std::to_string(dispatchers_count++); + LOG_DEBUG(logger, "({} -> {}): Creating dispatcher: {}, total dispatchers: {}", + name, executed_gtid_set, dispatcher_name, dispatchers.size()); + auto dispatcher = std::make_shared(dispatcher_name, max_bytes_in_buffer, max_flush_ms); + if (!binlog_checksum.empty()) + dispatcher->setBinlogChecksum(binlog_checksum); + for (const auto & it : dispatchers) + dispatcher->syncTo(it); + ret = dispatcher->start(factory->createBinlog(executed_gtid_set), name, mysql_database_names, max_bytes, max_waiting_ms); + if (!ret) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not create binlog: {}", executed_gtid_set); + dispatchers.push_back(dispatcher); + } + + return ret; +} + +BinlogClient::Metadata BinlogClient::getMetadata() const +{ + std::lock_guard lock(mutex); + Metadata ret; + ret.binlog_client_name = binlog_client_name; + for (const auto & dispatcher : dispatchers) + { + auto metadata = dispatcher->getDispatcherMetadata(); + if (!metadata.binlogs.empty()) + ret.dispatchers.push_back(metadata); + } + return ret; +} + +void BinlogClient::setBinlogChecksum(const String & checksum) +{ + std::lock_guard lock(mutex); + if (binlog_checksum != checksum) + { + LOG_DEBUG(logger, "Setting binlog_checksum: {} -> {}, total dispatchers: {}", binlog_checksum, checksum, dispatchers.size()); + binlog_checksum = checksum; + for (const auto & dispatcher : dispatchers) + dispatcher->setBinlogChecksum(checksum); + } +} + +} diff --git a/src/Databases/MySQL/MySQLBinlogClient.h b/src/Databases/MySQL/MySQLBinlogClient.h new file mode 100644 index 000000000000..b76934d08cf6 --- /dev/null +++ b/src/Databases/MySQL/MySQLBinlogClient.h @@ -0,0 +1,57 @@ +#pragma once + +#include +#include + +namespace DB::MySQLReplication +{ + +/** It is supposed to reduce the number of connections to remote MySQL binlog by reusing one connection between several consumers. + * Such reusing of the connection makes the time of reading from the remote binlog independent to number of the consumers. + * It tracks a list of BinlogEventsDispatcher instances for consumers with different binlog position. + * The dispatchers with the same binlog position will be merged to one. + */ +class BinlogClient +{ +public: + BinlogClient(const BinlogFactoryPtr & factory, + const String & name = {}, + UInt64 max_bytes_in_buffer_ = DBMS_DEFAULT_BUFFER_SIZE, + UInt64 max_flush_ms_ = 1000); + BinlogClient(const BinlogClient & other) = delete; + ~BinlogClient() = default; + BinlogClient & operator=(const BinlogClient & other) = delete; + + /// Creates a binlog to receive events + BinlogPtr createBinlog(const String & executed_gtid_set = {}, + const String & name = {}, + const NameSet & mysql_database_names = {}, + size_t max_bytes = 0, + UInt64 max_waiting_ms = 0); + + /// The binlog checksum is related to entire connection + void setBinlogChecksum(const String & checksum); + + struct Metadata + { + String binlog_client_name; + std::vector dispatchers; + }; + /// Returns only not empty dispatchers + Metadata getMetadata() const; + +private: + BinlogFactoryPtr factory; + const String binlog_client_name; + UInt64 max_bytes_in_buffer = 0; + UInt64 max_flush_ms = 0; + std::vector dispatchers; + String binlog_checksum; + mutable std::mutex mutex; + Poco::Logger * logger = nullptr; + int dispatchers_count = 0; +}; + +using BinlogClientPtr = std::shared_ptr; + +} diff --git a/src/Databases/MySQL/MySQLBinlogClientFactory.cpp b/src/Databases/MySQL/MySQLBinlogClientFactory.cpp new file mode 100644 index 000000000000..03a777ff352f --- /dev/null +++ b/src/Databases/MySQL/MySQLBinlogClientFactory.cpp @@ -0,0 +1,46 @@ +#include + +namespace DB::MySQLReplication +{ + +BinlogClientFactory & BinlogClientFactory::instance() +{ + static BinlogClientFactory ret; + return ret; +} + +BinlogClientPtr BinlogClientFactory::getClient(const String & host, UInt16 port, const String & user, const String & password, UInt64 max_bytes_in_buffer, UInt64 max_flush_ms) +{ + std::lock_guard lock(mutex); + String binlog_client_name; + WriteBufferFromString stream(binlog_client_name); + stream << user << "@" << host << ":" << port; + stream.finalize(); + String binlog_client_key = binlog_client_name + ":" + password; + auto it = clients.find(binlog_client_key); + BinlogClientPtr ret = it != clients.end() ? it->second.lock() : nullptr; + if (ret) + return ret; + auto factory = std::make_shared(host, port, user, password); + auto client = std::make_shared(factory, binlog_client_name, max_bytes_in_buffer, max_flush_ms); + clients[binlog_client_key] = client; + return client; +} + +std::vector BinlogClientFactory::getMetadata() const +{ + std::lock_guard lock(mutex); + std::vector ret; + for (const auto & it : clients) + { + if (auto c = it.second.lock()) + { + auto metadata = c->getMetadata(); + if (!metadata.dispatchers.empty()) + ret.push_back(metadata); + } + } + return ret; +} + +} diff --git a/src/Databases/MySQL/MySQLBinlogClientFactory.h b/src/Databases/MySQL/MySQLBinlogClientFactory.h new file mode 100644 index 000000000000..544b88e32019 --- /dev/null +++ b/src/Databases/MySQL/MySQLBinlogClientFactory.h @@ -0,0 +1,38 @@ +#pragma once + +#include +#include +#include + +namespace DB::MySQLReplication +{ + +/** Global instance to create or reuse MySQL Binlog Clients. + * If a binlog client already exists for specific params, + * it will be returned and reused to read binlog events from MySQL. + * Otherwise new instance will be created. + */ +class BinlogClientFactory final : boost::noncopyable +{ +public: + static BinlogClientFactory & instance(); + + BinlogClientPtr getClient(const String & host, + UInt16 port, + const String & user, + const String & password, + UInt64 max_bytes_in_buffer, + UInt64 max_flush_ms); + + /// Returns info of all registered clients + std::vector getMetadata() const; + +private: + BinlogClientFactory() = default; + + // Keeps track of already destroyed clients + std::unordered_map> clients; + mutable std::mutex mutex; +}; + +} diff --git a/src/Databases/MySQL/MySQLBinlogEventsDispatcher.cpp b/src/Databases/MySQL/MySQLBinlogEventsDispatcher.cpp new file mode 100644 index 000000000000..4af307f9c0f9 --- /dev/null +++ b/src/Databases/MySQL/MySQLBinlogEventsDispatcher.cpp @@ -0,0 +1,626 @@ +#include "MySQLBinlogEventsDispatcher.h" +#include +#include + +namespace DB::ErrorCodes +{ + extern const int UNKNOWN_EXCEPTION; + extern const int TIMEOUT_EXCEEDED; +} + +namespace DB::MySQLReplication +{ + +class BinlogFromDispatcher : public IBinlog +{ +public: + BinlogFromDispatcher(const String & name_, const NameSet & mysql_database_names_, size_t max_bytes_, UInt64 max_waiting_ms_) + : name(name_) + , mysql_database_names(mysql_database_names_) + , max_bytes(max_bytes_) + , max_waiting_ms(max_waiting_ms_) + , logger(&Poco::Logger::get("BinlogFromDispatcher(" + name + ")")) + { + } + + ~BinlogFromDispatcher() override + { + stop(); + } + + void stop() + { + { + std::lock_guard lock(mutex); + if (is_cancelled) + return; + is_cancelled = true; + } + cv.notify_all(); + } + + std::string getName() const + { + return name; + } + + bool tryReadEvent(BinlogEventPtr & to, UInt64 ms) override; + Position getPosition() const override; + void setPosition(const Position & initial, const Position & wait); + void setException(const std::exception_ptr & exception_); + void push(const BinlogEventsDispatcher::Buffer & buffer); + BinlogEventsDispatcher::BinlogMetadata getBinlogMetadata() const; + +private: + const String name; + const NameSet mysql_database_names; + const size_t max_bytes = 0; + const UInt64 max_waiting_ms = 0; + + Position position; + GTIDSets gtid_sets_wait; + + BinlogEventsDispatcher::Buffer buffer; + mutable std::mutex mutex; + + std::condition_variable cv; + bool is_cancelled = false; + Poco::Logger * logger = nullptr; + std::exception_ptr exception; +}; + +static String getBinlogNames(const std::vector> & binlogs) +{ + std::vector names; + for (const auto & it : binlogs) + { + if (auto binlog = it.lock()) + names.push_back(binlog->getName()); + } + return boost::algorithm::join(names, ", "); +} + +BinlogEventsDispatcher::BinlogEventsDispatcher(const String & logger_name_, size_t max_bytes_in_buffer_, UInt64 max_flush_ms_) + : logger_name(logger_name_) + , max_bytes_in_buffer(max_bytes_in_buffer_) + , max_flush_ms(max_flush_ms_) + , logger(&Poco::Logger::get("BinlogEventsDispatcher(" + logger_name + ")")) + , dispatching_thread(std::make_unique([this]() { dispatchEvents(); })) +{ +} + +BinlogEventsDispatcher::~BinlogEventsDispatcher() +{ + { + std::lock_guard lock(mutex); + is_cancelled = true; + auto exc = std::make_exception_ptr(Exception(ErrorCodes::UNKNOWN_EXCEPTION, "Dispatcher {} has been already destroyed", logger_name)); + try + { + cleanupLocked([&](const auto & binlog) + { + /// Notify the binlogs that the dispatcher is already destroyed + /// and it needs to recreate new binlogs if needed + binlog->setException(exc); + }); + } + catch (const std::exception & exc) + { + LOG_ERROR(logger, "Unexpected exception: {}", exc.what()); + } + } + cv.notify_all(); + if (dispatching_thread) + dispatching_thread->join(); +} + +static void flushTimers(Stopwatch & watch, UInt64 & total_time, UInt64 & size, float & size_per_sec, UInt64 & bytes, float & bytes_per_sec, float threshold_flush, float threshold_reset) +{ + total_time += watch.elapsedMicroseconds(); + const float elapsed_seconds = total_time * 1e-6f; + if (elapsed_seconds >= threshold_flush) + { + size_per_sec = size / elapsed_seconds; + bytes_per_sec = bytes / elapsed_seconds; + } + if (elapsed_seconds >= threshold_reset) + { + size = 0; + bytes = 0; + total_time = 0; + } +} + +void BinlogEventsDispatcher::flushBufferLocked() +{ + Stopwatch watch; + if (buffer.bytes) + cleanupLocked([&](const auto & b) { b->push(buffer); }); + events_flush += buffer.events.size(); + bytes_flush += buffer.bytes; + flushTimers(watch, events_flush_total_time, events_flush, events_flush_per_sec, bytes_flush, bytes_flush_per_sec, 0.1f, 1.0); + buffer = {}; +} + +static bool isDispatcherEventIgnored(const BinlogEventPtr & event) +{ + switch (event->header.type) + { + /// Sending to all databases: + case GTID_EVENT: /// Catch up requested executed gtid set, used only in BinlogFromDispatcher + case ROTATE_EVENT: /// Change binlog_checksum + case XID_EVENT: /// Commit transaction + /// Sending to all attached binlogs without filtering on dispatcher thread + /// to keep the connection as up-to-date as possible, + /// but these events should be filtered on databases' threads + /// and sent only to requested databases: + case QUERY_EVENT: /// Apply DDL + case WRITE_ROWS_EVENT_V1: /// Apply DML + case WRITE_ROWS_EVENT_V2: + case DELETE_ROWS_EVENT_V1: + case DELETE_ROWS_EVENT_V2: + case UPDATE_ROWS_EVENT_V1: + case UPDATE_ROWS_EVENT_V2: + return false; + default: + break; + } + return true; +} + +void BinlogEventsDispatcher::dispatchEvents() +{ + LOG_TRACE(logger, "{}: started", __FUNCTION__); + BinlogEventPtr event; + BinlogPtr binlog_; + Stopwatch watch; + UInt64 events_read = 0; + UInt64 bytes_read = 0; + UInt64 events_read_total_time = 0; + Stopwatch watch_events_read; + + while (!is_cancelled) + { + try + { + { + std::unique_lock lock(mutex); + cv.wait(lock, [&] { return is_cancelled || (binlog_read_from && !binlogs.empty()); }); + if (is_cancelled) + break; + + for (auto it = sync_to.begin(); it != sync_to.end() && !binlogs.empty();) + { + if (auto d = it->lock()) + { + /// If we can catch up the position of a dispatcher we synced to, + /// need to move all binlogs out + if (trySyncLocked(d)) + { + /// Don't keep connection longer than needed + stopLocked(); + break; + } + ++it; + } + else + { + it = sync_to.erase(it); + } + } + + if (binlog_read_from) + binlog_read_from->setChecksum(binlog_checksum); + binlog_ = binlog_read_from; + if (watch.elapsedMilliseconds() >= max_flush_ms || buffer.bytes >= max_bytes_in_buffer) + { + flushBufferLocked(); + watch.restart(); + } + } + + watch_events_read.restart(); + if (!is_cancelled && binlog_ && binlog_->tryReadEvent(event, max_flush_ms) && event) + { + ++events_read; + bytes_read += event->header.event_size; + { + std::lock_guard lock(mutex); + flushTimers(watch_events_read, events_read_total_time, events_read, events_read_per_sec, bytes_read, bytes_read_per_sec, 1.0, 5.0); + BinlogParser::updatePosition(event, position); + /// Ignore meaningless events + if (isDispatcherEventIgnored(event)) + continue; + buffer.events.push_back(event); + buffer.bytes += event->header.event_size; + buffer.position = position; + /// Deliver ROTATE event ASAP if there binlog_checksum should be changed + if (event->header.type == ROTATE_EVENT) + flushBufferLocked(); + } + } + } + catch (const std::exception & exc) + { + std::lock_guard lock(mutex); + LOG_ERROR(logger, "Exception: {}", exc.what()); + stopLocked(); + /// All attached binlogs should be recreated + cleanupLocked([&](const auto & b) { b->setException(std::current_exception()); }); + binlogs.clear(); + buffer = {}; + position = {}; + } + } + LOG_TRACE(logger, "{}: finished", __FUNCTION__); +} + +bool BinlogEventsDispatcher::cleanupLocked(const std::function & binlog)> & fn) +{ + for (auto it = binlogs.begin(); it != binlogs.end();) + { + if (auto binlog = it->lock()) + { + if (fn) + fn(binlog); + ++it; + } + else + { + it = binlogs.erase(it); + } + } + + return binlogs.empty(); +} + +bool BinlogEventsDispatcher::cleanupBinlogsAndStop() +{ + std::lock_guard lock(mutex); + const bool is_empty = cleanupLocked(); + if (is_empty && binlog_read_from) + stopLocked(); + return is_empty; +} + +void BinlogEventsDispatcher::stopLocked() +{ + if (!binlog_read_from) + { + LOG_DEBUG(logger, "Could not stop. Already stopped"); + return; + } + + cleanupLocked(); + binlog_read_from = nullptr; + LOG_DEBUG(logger, "Stopped: {}:{}.{}: ({})", position.binlog_name, position.gtid_sets.toString(), position.binlog_pos, getBinlogNames(binlogs)); +} + +BinlogPtr BinlogEventsDispatcher::createBinlogLocked(const String & name_, + const NameSet & mysql_database_names, + size_t max_bytes, + UInt64 max_waiting_ms, + const Position & pos_initial, + const Position & pos_wait) +{ + static int client_cnt = 0; + const String client_id = !name_.empty() ? name_ : "binlog_" + std::to_string(++client_cnt); + auto binlog = std::make_shared(client_id, mysql_database_names, max_bytes, max_waiting_ms); + binlogs.push_back(binlog); + binlog->setPosition(pos_initial, pos_wait); + LOG_DEBUG(logger, "Created binlog: {} -> {}", name_, binlog->getPosition().gtid_sets.toString()); + return binlog; +} + +BinlogPtr BinlogEventsDispatcher::start(const BinlogPtr & binlog_read_from_, + const String & name_, + const NameSet & mysql_database_names, + size_t max_bytes, + UInt64 max_waiting_ms) +{ + BinlogPtr ret; + { + std::lock_guard lock(mutex); + if (is_started) + return {}; + binlog_read_from = binlog_read_from_; + /// It is used for catching up + /// binlog_read_from should return position with requested executed GTID set: 1-N + position = binlog_read_from->getPosition(); + ret = createBinlogLocked(name_, mysql_database_names, max_bytes, max_waiting_ms, position); + is_started = true; + } + cv.notify_all(); + return ret; +} + +BinlogPtr BinlogEventsDispatcher::attach(const String & executed_gtid_set, + const String & name_, + const NameSet & mysql_database_names, + size_t max_bytes, + UInt64 max_waiting_ms) +{ + BinlogPtr ret; + { + std::lock_guard lock(mutex); + /// Check if binlog_read_from can be reused: + /// Attach to only active dispatchers + /// and if executed_gtid_set is higher value than current + if (!binlog_read_from || !is_started || cleanupLocked() || executed_gtid_set.empty()) + return {}; + Position pos_wait; + pos_wait.gtid_sets.parse(executed_gtid_set); + if (!BinlogParser::isNew(position, pos_wait)) + return {}; + ret = createBinlogLocked(name_, mysql_database_names, max_bytes, max_waiting_ms, position, pos_wait); + } + cv.notify_all(); + return ret; +} + +void BinlogEventsDispatcher::syncToLocked(const BinlogEventsDispatcherPtr & to) +{ + if (to && this != to.get()) + { + std::vector names; + for (const auto & it : sync_to) + { + if (auto dispatcher = it.lock()) + names.push_back(dispatcher->logger_name); + } + LOG_DEBUG(logger, "Syncing -> ({}) + ({})", boost::algorithm::join(names, ", "), to->logger_name); + sync_to.emplace_back(to); + } +} + +void BinlogEventsDispatcher::syncTo(const BinlogEventsDispatcherPtr & to) +{ + std::lock_guard lock(mutex); + syncToLocked(to); +} + +Position BinlogEventsDispatcher::getPosition() const +{ + std::lock_guard lock(mutex); + return position; +} + +bool BinlogEventsDispatcher::trySyncLocked(BinlogEventsDispatcherPtr & to) +{ + { + std::lock_guard lock(to->mutex); + /// Don't catch up if positions do not have GTIDs yet + const auto & cur_sets = position.gtid_sets.sets; + const auto & sets = to->position.gtid_sets.sets; + /// Sync to only started dispatchers + if (!to->binlog_read_from || (cur_sets.empty() && sets.empty()) || to->position != position) + return false; + + flushBufferLocked(); + to->flushBufferLocked(); + LOG_DEBUG(logger, "Synced up: {} -> {}: {}:{}.{}: ({}) + ({})", logger_name, to->logger_name, + position.binlog_name, position.gtid_sets.toString(), position.binlog_pos, getBinlogNames(to->binlogs), getBinlogNames(binlogs)); + std::move(binlogs.begin(), binlogs.end(), std::back_inserter(to->binlogs)); + } + + /// Notify that new binlogs arrived + to->cv.notify_all(); + return true; +} + +void BinlogEventsDispatcher::setBinlogChecksum(const String & checksum) +{ + std::lock_guard lock(mutex); + LOG_DEBUG(logger, "Setting binlog_checksum: {}", checksum); + binlog_checksum = IBinlog::checksumFromString(checksum); +} + +void BinlogFromDispatcher::push(const BinlogEventsDispatcher::Buffer & buffer_) +{ + std::unique_lock lock(mutex); + cv.wait_for(lock, std::chrono::milliseconds(max_waiting_ms), + [&] + { + bool ret = is_cancelled || exception || max_bytes == 0 || buffer.bytes < max_bytes; + if (!ret) + LOG_TRACE(logger, "Waiting: bytes: {} >= {}", buffer.bytes, max_bytes); + return ret; + }); + + if (is_cancelled || exception) + return; + + if (max_bytes != 0 && buffer.bytes >= max_bytes) + { + lock.unlock(); + setException(std::make_exception_ptr( + Exception(ErrorCodes::TIMEOUT_EXCEEDED, + "Timeout exceeded: Waiting: bytes: {} >= {}", buffer.bytes, max_bytes))); + return; + } + + auto it = buffer_.events.begin(); + size_t bytes = buffer_.bytes; + if (!gtid_sets_wait.sets.empty()) + { + if (!buffer_.position.gtid_sets.contains(gtid_sets_wait)) + { + LOG_TRACE(logger, "(wait_until: {} / {}) Skipped bytes: {}", + gtid_sets_wait.toString(), buffer_.position.gtid_sets.toString(), buffer_.bytes); + return; + } + + std::vector seqs; + for (auto & s : gtid_sets_wait.sets) + { + GTID g; + g.uuid = s.uuid; + for (auto & in : s.intervals) + { + g.seq_no = in.end; + seqs.push_back(g); + } + } + for (; it != buffer_.events.end(); ++it) + { + const auto & event = *it; + auto find_if_func = [&](auto & a) + { + return std::static_pointer_cast(event)->gtid == a; + }; + if (event->header.type != GTID_EVENT || std::find_if(seqs.begin(), seqs.end(), find_if_func) == seqs.end()) + { + LOG_TRACE(logger, "(wait_until: {} / {}) Skipped {}", + gtid_sets_wait.toString(), buffer_.position.gtid_sets.toString(), magic_enum::enum_name(event->header.type)); + bytes -= event->header.event_size; + continue; + } + LOG_DEBUG(logger, "(wait_until: {} / {}) Starting {}: gtid seq_no: {}", + gtid_sets_wait.toString(), buffer_.position.gtid_sets.toString(), magic_enum::enum_name(event->header.type), + std::static_pointer_cast(event)->gtid.seq_no); + break; + } + gtid_sets_wait = {}; + } + + if (it != buffer_.events.end()) + { + std::copy(it, buffer_.events.end(), std::back_inserter(buffer.events)); + buffer.bytes += bytes; + buffer.position = buffer_.position; + } + lock.unlock(); + /// Notify that added some event + cv.notify_all(); +} + +static void rethrowIfNeeded(const std::exception_ptr & exception, size_t events_size) +{ + try + { + std::rethrow_exception(exception); + } + catch (const Exception & e) + { + /// If timeout exceeded, it is safe to read all events before rethrowning + if (e.code() == ErrorCodes::TIMEOUT_EXCEEDED && events_size > 0) + return; + throw; + } +} + +static bool isBinlogEventIgnored(const NameSet & mysql_database_names, const BinlogEventPtr & event) +{ + bool ret = false; + switch (event->header.type) + { + case WRITE_ROWS_EVENT_V1: + case WRITE_ROWS_EVENT_V2: + case DELETE_ROWS_EVENT_V1: + case DELETE_ROWS_EVENT_V2: + case UPDATE_ROWS_EVENT_V1: + case UPDATE_ROWS_EVENT_V2: + ret = !mysql_database_names.empty() && !mysql_database_names.contains(std::static_pointer_cast(event)->schema); + break; + case QUERY_EVENT: + if (event->type() != MYSQL_UNHANDLED_EVENT) + { + auto query_event = std::static_pointer_cast(event); + ret = !mysql_database_names.empty() && + !query_event->query_database_name.empty() && + !mysql_database_names.contains(query_event->query_database_name); + } + break; + default: + break; + } + return ret; +} + +bool BinlogFromDispatcher::tryReadEvent(BinlogEventPtr & to, UInt64 ms) +{ + auto wake_up_func = [&] + { + if (exception) + rethrowIfNeeded(exception, buffer.events.size()); + return is_cancelled || !buffer.events.empty(); + }; + to = nullptr; + std::unique_lock lock(mutex); + if (!cv.wait_for(lock, std::chrono::milliseconds(ms), wake_up_func) || is_cancelled || buffer.events.empty()) + return false; + to = buffer.events.front(); + buffer.events.pop_front(); + BinlogParser::updatePosition(to, position); + buffer.bytes -= to->header.event_size; + if (isBinlogEventIgnored(mysql_database_names, to)) + to = std::make_shared(EventHeader(to->header)); + lock.unlock(); + /// Notify that removed some event + cv.notify_all(); + return true; +} + +Position BinlogFromDispatcher::getPosition() const +{ + std::lock_guard lock(mutex); + return position; +} + +void BinlogFromDispatcher::setPosition(const Position & initial, const Position & wait) +{ + std::lock_guard lock(mutex); + if (wait.gtid_sets.sets.empty()) + { + position = initial; + } + else + { + position = wait; + gtid_sets_wait = wait.gtid_sets; + } +} + +void BinlogFromDispatcher::setException(const std::exception_ptr & exception_) +{ + { + std::lock_guard lock(mutex); + exception = exception_; + } + cv.notify_all(); +} + +BinlogEventsDispatcher::BinlogMetadata BinlogFromDispatcher::getBinlogMetadata() const +{ + std::lock_guard lock(mutex); + BinlogEventsDispatcher::BinlogMetadata ret; + ret.name = name; + ret.position_write = buffer.position; + ret.position_read = position; + ret.size = buffer.events.size(); + ret.bytes = buffer.bytes; + ret.max_bytes = max_bytes; + ret.max_waiting_ms = max_waiting_ms; + return ret; +} + +BinlogEventsDispatcher::DispatcherMetadata BinlogEventsDispatcher::getDispatcherMetadata() const +{ + std::lock_guard lock(mutex); + DispatcherMetadata ret; + ret.name = logger_name; + ret.position = position; + ret.events_read_per_sec = events_read_per_sec; + ret.bytes_read_per_sec = bytes_read_per_sec; + ret.events_flush_per_sec = events_flush_per_sec; + ret.bytes_flush_per_sec = bytes_flush_per_sec; + + for (const auto & it : binlogs) + { + if (auto binlog = it.lock()) + ret.binlogs.push_back(binlog->getBinlogMetadata()); + } + return ret; +} + +} diff --git a/src/Databases/MySQL/MySQLBinlogEventsDispatcher.h b/src/Databases/MySQL/MySQLBinlogEventsDispatcher.h new file mode 100644 index 000000000000..433796970150 --- /dev/null +++ b/src/Databases/MySQL/MySQLBinlogEventsDispatcher.h @@ -0,0 +1,136 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB::MySQLReplication +{ + +class BinlogEventsDispatcher; +using BinlogEventsDispatcherPtr = std::shared_ptr; +class BinlogFromDispatcher; + +/** Reads the binlog events from one source and dispatches them over consumers. + * If it can catch up the position of the another dispatcher, it will move all consumers to this dispatcher. + */ +class BinlogEventsDispatcher final : boost::noncopyable +{ +public: + BinlogEventsDispatcher(const String & logger_name_ = "BinlogDispatcher", size_t max_bytes_in_buffer_ = 1_MiB, UInt64 max_flush_ms_ = 1000); + ~BinlogEventsDispatcher(); + + /// Moves all IBinlog objects to \a to if it has the same position + /// Supports syncing to multiple dispatchers + void syncTo(const BinlogEventsDispatcherPtr & to); + + /** Creates a binlog and starts the dispatching + * binlog_read_from Source binlog to read events from + * name Identifies the binlog, could be not unique + * mysql_database_names Returns events only from these databases + * max_bytes Defines a limit in bytes for this binlog + * Note: Dispatching will be stopped for all binlogs if bytes in queue increases this limit + * max_waiting_ms Max wait time when max_bytes exceeded + */ + BinlogPtr start(const BinlogPtr & binlog_read_from_, + const String & name = {}, + const NameSet & mysql_database_names = {}, + size_t max_bytes = 0, + UInt64 max_waiting_ms = 0); + + /** Creates a binlog if the dispatcher is started + * executed_gtid_set Can be higher value than current, + * otherwise not possible to attach + * name Identifies the binlog, could be not unique + * mysql_database_names Returns events only from these databases + * max_bytes Defines a limit in bytes for this binlog + * max_waiting_ms Max wait time when max_bytes exceeded + */ + BinlogPtr attach(const String & executed_gtid_set, + const String & name = {}, + const NameSet & mysql_database_names = {}, + size_t max_bytes = 0, + UInt64 max_waiting_ms = 0); + + /// Cleans the destroyed binlogs up and returns true if empty + bool cleanupBinlogsAndStop(); + + /// Changes binlog_checksum for binlog_read_from + void setBinlogChecksum(const String & checksum); + + Position getPosition() const; + + struct BinlogMetadata + { + String name; + /// Position that was written to + Position position_write; + /// Position that was read from + Position position_read; + size_t size = 0; + size_t bytes = 0; + size_t max_bytes = 0; + UInt64 max_waiting_ms = 0; + }; + struct DispatcherMetadata + { + String name; + Position position; + float events_read_per_sec = 0; + float bytes_read_per_sec = 0; + float events_flush_per_sec = 0; + float bytes_flush_per_sec = 0; + std::vector binlogs; + }; + DispatcherMetadata getDispatcherMetadata() const; + + struct Buffer + { + std::deque events; + size_t bytes = 0; + Position position; + }; + +private: + bool cleanupLocked(const std::function & binlog)> & fn = {}); + bool startLocked(const String & executed_gtid_set); + void stopLocked(); + BinlogPtr createBinlogLocked(const String & name = {}, + const NameSet & mysql_database_names = {}, + size_t max_bytes = 0, + UInt64 max_waiting_ms = 0, + const Position & pos_initial = {}, + const Position & pos_wait = {}); + void syncToLocked(const BinlogEventsDispatcherPtr & to); + bool trySyncLocked(BinlogEventsDispatcherPtr & to); + void flushBufferLocked(); + void dispatchEvents(); + + const String logger_name; + const size_t max_bytes_in_buffer = 0; + const UInt64 max_flush_ms = 0; + Poco::Logger * logger = nullptr; + + BinlogPtr binlog_read_from; + + Position position; + std::vector> sync_to; + std::vector> binlogs; + std::atomic_bool is_cancelled{false}; + mutable std::mutex mutex; + std::condition_variable cv; + std::unique_ptr dispatching_thread; + IBinlog::Checksum binlog_checksum = IBinlog::CRC32; + bool is_started = false; + Buffer buffer; + float events_read_per_sec = 0; + float bytes_read_per_sec = 0; + UInt64 events_flush = 0; + UInt64 events_flush_total_time = 0; + float events_flush_per_sec = 0; + UInt64 bytes_flush = 0; + float bytes_flush_per_sec = 0; +}; + +} diff --git a/src/Databases/MySQL/tests/data/binlog.000016 b/src/Databases/MySQL/tests/data/binlog.000016 new file mode 100644 index 000000000000..e27a2bac9ff7 Binary files /dev/null and b/src/Databases/MySQL/tests/data/binlog.000016 differ diff --git a/src/Databases/MySQL/tests/data/binlog.001390 b/src/Databases/MySQL/tests/data/binlog.001390 new file mode 100644 index 000000000000..1fbaae08a986 Binary files /dev/null and b/src/Databases/MySQL/tests/data/binlog.001390 differ diff --git a/src/Databases/MySQL/tests/gtest_mysql_binlog.cpp b/src/Databases/MySQL/tests/gtest_mysql_binlog.cpp new file mode 100644 index 000000000000..df8433f7ccef --- /dev/null +++ b/src/Databases/MySQL/tests/gtest_mysql_binlog.cpp @@ -0,0 +1,1754 @@ +#include +#include +#include +#include +#include +#include + +#include +#include + +using namespace DB; +using namespace DB::MySQLReplication; + +#define TRY_LOOP_IMPL(expr, timeout) \ + const unsigned long _test_step = (timeout) < 350 ? (timeout) / 7 + 1 : 50; \ + for (int _i = 0; _i < (timeout) && !(expr); _i += _test_step) \ + std::this_thread::sleep_for(std::chrono::milliseconds(_test_step)); \ + +#define TRY_ASSERT_EQ(expr, expected, timeout) \ +do { \ + TRY_LOOP_IMPL(((expr) == (expected)), timeout) \ + ASSERT_EQ((expr), expected); \ +} while (false) + +#define TRY_ASSERT_TRUE(expr, timeout) \ + TRY_ASSERT_EQ((expr), true, timeout) + +static std::string getTestDataRoot() +{ + static auto root = []() -> std::string + { + std::filesystem::path testdata_path("src/Databases/MySQL/tests/data"); + auto basedir = std::filesystem::current_path(); + while (basedir != basedir.parent_path()) + { + if (std::filesystem::exists(basedir / testdata_path)) + { + testdata_path = basedir / testdata_path; + break; + } + basedir = basedir.parent_path(); + } + auto path = basedir / testdata_path; + return std::filesystem::exists(path) ? path.string() : ""; + }(); + return root; +} + +static String getTestDataPath(const String & testdata_file) +{ + return (std::filesystem::path(getTestDataRoot()) / testdata_file).string(); +} + +class MySQLBinlog : public ::testing::Test +{ +protected: + void SetUp() override + { + if (getTestDataRoot().empty()) + GTEST_SKIP() << "Skipping all tests since no test data files found"; + } + + UInt64 timeout = 25000; +}; + +TEST_F(MySQLBinlog, positionEndLogPosOverflow) +{ + Position position; + EventHeader header; + header.event_size = 8161; + header.log_pos = 4294958114; + BinlogParser::updatePosition(std::make_shared(EventHeader(header)), position); + ASSERT_EQ(position.binlog_pos, header.log_pos); + ASSERT_TRUE(position.binlog_name.empty()); + ASSERT_TRUE(position.gtid_sets.toString().empty()); + ASSERT_EQ(position.timestamp, 0); + + header.log_pos = 4294966149; + BinlogParser::updatePosition(std::make_shared(EventHeader(header)), position); + ASSERT_EQ(position.binlog_pos, header.log_pos); + UInt64 prev = position.binlog_pos; + + header.log_pos = 7014; + BinlogParser::updatePosition(std::make_shared(EventHeader(header)), position); + ASSERT_EQ(position.binlog_pos, prev + header.event_size); + prev = position.binlog_pos; + + header.event_size = 8107; + header.log_pos = 15121; + BinlogParser::updatePosition(std::make_shared(EventHeader(header)), position); + ASSERT_EQ(position.binlog_pos, prev + header.event_size); + prev = position.binlog_pos; + + header.event_size = 8131; + header.log_pos = 23252; + BinlogParser::updatePosition(std::make_shared(EventHeader(header)), position); + ASSERT_EQ(position.binlog_pos, prev + header.event_size); + + position.binlog_pos = 4724501662; + prev = position.binlog_pos; + + header.event_size = 8125; + header.log_pos = 429542491; + BinlogParser::updatePosition(std::make_shared(EventHeader(header)), position); + ASSERT_EQ(position.binlog_pos, prev + header.event_size); + + position.binlog_pos = 5474055640; + prev = position.binlog_pos; + + header.event_size = 31; + header.log_pos = 1179088375; + BinlogParser::updatePosition(std::make_shared(EventHeader(header)), position); + ASSERT_EQ(position.binlog_pos, prev + header.event_size); + + position = {}; + header.log_pos = 4294965445; + BinlogParser::updatePosition(std::make_shared(EventHeader(header)), position); + ASSERT_EQ(position.binlog_pos, header.log_pos); + prev = position.binlog_pos; + + header.event_size = 7927; + header.log_pos = 6076; + BinlogParser::updatePosition(std::make_shared(EventHeader(header)), position); + ASSERT_EQ(position.binlog_pos, prev + header.event_size); +} + +TEST_F(MySQLBinlog, positionEquals) +{ + Position p1; + Position p2; + ASSERT_EQ(p1, p2); + p1.binlog_pos = 1; + ASSERT_NE(p1, p2); + p2.binlog_pos = 1; + ASSERT_EQ(p1, p2); + p1.gtid_sets.parse("a9d88f83-c14e-11ec-bb36-244bfedf7766:87828"); + ASSERT_NE(p1, p2); + p2.gtid_sets.parse("a9d88f83-c14e-11ec-bb36-244bfedf7766:87828"); + ASSERT_EQ(p1, p2); + p1.binlog_name = "name"; + ASSERT_NE(p1, p2); + p2.binlog_name = "name"; + ASSERT_EQ(p1, p2); +} + +TEST_F(MySQLBinlog, positionMultimaster) +{ + Position p1; + Position p2; + p1.gtid_sets.parse("f189aee3-3cd2-11ed-a407-fa163ea7d4ed:1-3602,ff9de833-3cd2-11ed-87b7-fa163e99d975:1-172"); + p2.gtid_sets.parse("ff9de833-3cd2-11ed-87b7-fa163e99d975:1-172"); + ASSERT_TRUE(p1.gtid_sets.contains(p2.gtid_sets)); + ASSERT_FALSE(p2.gtid_sets.contains(p1.gtid_sets)); + ASSERT_FALSE(BinlogParser::isNew(p1, p2)); + + p2.gtid_sets = {}; + p2.gtid_sets.parse("ff9de833-3cd2-11ed-87b7-fa163e99d975:1-10"); + ASSERT_FALSE(BinlogParser::isNew(p1, p2)); + + p2.gtid_sets = {}; + p2.gtid_sets.parse("ff9de833-3cd2-11ed-87b7-fa163e99d975:172"); + ASSERT_FALSE(BinlogParser::isNew(p1, p2)); + + p2.gtid_sets = {}; + p2.gtid_sets.parse("ff9de833-3cd2-11ed-87b7-fa163e99d975:171-172"); + ASSERT_FALSE(BinlogParser::isNew(p1, p2)); + + p2.gtid_sets = {}; + p2.gtid_sets.parse("ff9de833-3cd2-11ed-87b7-fa163e99d975:171-173"); + ASSERT_TRUE(BinlogParser::isNew(p1, p2)); + + p2.gtid_sets = {}; + p2.gtid_sets.parse("ff9de833-3cd2-11ed-87b7-fa163e99d975:173"); + ASSERT_TRUE(BinlogParser::isNew(p1, p2)); + + p2.gtid_sets = {}; + p2.gtid_sets.parse("xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx:173"); + ASSERT_FALSE(BinlogParser::isNew(p1, p2)); + + p2.gtid_sets = {}; + p2.gtid_sets.parse("f189aee3-3cd2-11ed-a407-fa163ea7d4ed:1-3602,ff9de833-3cd2-11ed-87b7-fa163e99d975:1-172"); + ASSERT_FALSE(BinlogParser::isNew(p1, p2)); + + p2.gtid_sets = {}; + p2.gtid_sets.parse("f189aee3-3cd2-11ed-a407-fa163ea7d4ed:1-3602,ff9de833-3cd2-11ed-87b7-fa163e99d975:1-173"); + ASSERT_TRUE(BinlogParser::isNew(p1, p2)); +} + +static void testFile1(IBinlog & binlog, UInt64 timeout, bool filtered = false) +{ + BinlogEventPtr event; + int count = 0; + + if (!filtered) + { + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, FORMAT_DESCRIPTION_EVENT); + ASSERT_EQ(event->header.timestamp, 1651442421); + ASSERT_EQ(event->header.event_size, 122); + ASSERT_EQ(event->header.log_pos, 126); + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, PREVIOUS_GTIDS_EVENT); + ASSERT_EQ(event->header.timestamp, 1651442421); + ASSERT_EQ(event->header.event_size, 71); + ASSERT_EQ(event->header.log_pos, 197); + } + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, GTID_EVENT); + ASSERT_EQ(event->header.timestamp, 1651475081); + ASSERT_EQ(event->header.event_size, 79); + ASSERT_EQ(event->header.log_pos, 276); + + auto gtid_event = std::static_pointer_cast(event); + ASSERT_TRUE(gtid_event); + ASSERT_EQ(gtid_event->commit_flag, 0); + GTIDSets gtid_expected; + gtid_expected.parse("a9d88f83-c14e-11ec-bb36-244bfedf7766:87828"); + GTIDSets gtid_actual; + gtid_actual.update(gtid_event->gtid); + ASSERT_EQ(gtid_actual.toString(), gtid_expected.toString()); + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, QUERY_EVENT); + ASSERT_EQ(event->header.timestamp, 1651475081); + ASSERT_EQ(event->header.event_size, 73); + ASSERT_EQ(event->header.log_pos, 349); + + if (!filtered) + { + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, TABLE_MAP_EVENT); + ASSERT_EQ(event->header.timestamp, 1651475081); + ASSERT_EQ(event->header.event_size, 48); + ASSERT_EQ(event->header.log_pos, 397); + + auto table_event = std::static_pointer_cast(event); + ASSERT_TRUE(table_event); + ASSERT_EQ(table_event->table_id, 7566); + ASSERT_EQ(table_event->flags, 1); + ASSERT_EQ(table_event->schema_len, 2u); + ASSERT_EQ(table_event->schema, "db"); + ASSERT_EQ(table_event->table_len, 1u); + ASSERT_EQ(table_event->table, "a"); + ASSERT_EQ(table_event->column_count, 4); + std::vector column_type = {3u, 3u, 3u, 3u}; + ASSERT_EQ(table_event->column_type, column_type); + std::vector column_meta = {0, 0, 0, 0}; + ASSERT_EQ(table_event->column_meta, column_meta); + std::vector column_charset = {}; + ASSERT_EQ(table_event->column_charset, column_charset); + ASSERT_EQ(table_event->default_charset, 255u); + } + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, WRITE_ROWS_EVENT_V2); + ASSERT_EQ(event->header.timestamp, 1651475081); + ASSERT_EQ(event->header.event_size, 52); + ASSERT_EQ(event->header.log_pos, 449); + + ASSERT_EQ(event->type(), MYSQL_UNPARSED_ROWS_EVENT); + event = std::static_pointer_cast(event)->parse(); + + ASSERT_TRUE(event); + auto write_event = std::static_pointer_cast(event); + ASSERT_TRUE(write_event); + ASSERT_EQ(write_event->number_columns, 4); + ASSERT_EQ(write_event->schema, "db"); + ASSERT_EQ(write_event->table, "a"); + ASSERT_EQ(write_event->rows.size(), 1); + ASSERT_EQ(write_event->rows[0].getType(), Field::Types::Tuple); + auto row_data = write_event->rows[0].get(); + ASSERT_EQ(row_data.size(), 4u); + ASSERT_EQ(row_data[0].get(), 1u); + ASSERT_EQ(row_data[1].get(), 1u); + ASSERT_EQ(row_data[2].get(), 1u); + ASSERT_EQ(row_data[3].get(), 1u); + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, XID_EVENT); + ASSERT_EQ(event->header.timestamp, 1651475081); + ASSERT_EQ(event->header.event_size, 31); + ASSERT_EQ(event->header.log_pos, 480); + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, GTID_EVENT); + ASSERT_EQ(event->header.timestamp, 1651475244); + ASSERT_EQ(event->header.event_size, 79); + ASSERT_EQ(event->header.log_pos, 559); + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, QUERY_EVENT); + ASSERT_EQ(event->header.timestamp, 1651475244); + ASSERT_EQ(event->header.event_size, 82); + ASSERT_EQ(event->header.log_pos, 641); + + if (!filtered) + { + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, TABLE_MAP_EVENT); + ASSERT_EQ(event->header.timestamp, 1651475244); + ASSERT_EQ(event->header.event_size, 48); + ASSERT_EQ(event->header.log_pos, 689); + } + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.timestamp, 1651475244); + ASSERT_EQ(event->header.event_size, 70); + ASSERT_EQ(event->header.type, UPDATE_ROWS_EVENT_V2); + ASSERT_EQ(event->header.log_pos, 759); + + ASSERT_EQ(event->type(), MYSQL_UNPARSED_ROWS_EVENT); + event = std::static_pointer_cast(event)->parse(); + + ASSERT_TRUE(event); + auto update_event = std::static_pointer_cast(event); + ASSERT_TRUE(update_event); + ASSERT_EQ(update_event->number_columns, 4); + ASSERT_EQ(update_event->schema, "db"); + ASSERT_EQ(update_event->table, "a"); + ASSERT_EQ(update_event->rows.size(), 2); + ASSERT_EQ(update_event->rows[0].getType(), Field::Types::Tuple); + row_data = update_event->rows[0].get(); + ASSERT_EQ(row_data.size(), 4u); + ASSERT_EQ(row_data[0].get(), 1u); + ASSERT_EQ(row_data[1].get(), 1u); + ASSERT_EQ(row_data[2].get(), 1u); + ASSERT_EQ(row_data[3].get(), 1u); + row_data = update_event->rows[1].get(); + ASSERT_EQ(row_data.size(), 4u); + ASSERT_EQ(row_data[0].get(), 1u); + ASSERT_EQ(row_data[1].get(), 2u); + ASSERT_EQ(row_data[2].get(), 1u); + ASSERT_EQ(row_data[3].get(), 1u); + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, XID_EVENT); + ASSERT_EQ(event->header.timestamp, 1651475244); + ASSERT_EQ(event->header.event_size, 31); + ASSERT_EQ(event->header.log_pos, 790); + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, GTID_EVENT); + ASSERT_EQ(event->header.timestamp, 1651482394); + ASSERT_EQ(event->header.event_size, 79); + ASSERT_EQ(event->header.log_pos, 869); + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, QUERY_EVENT); + ASSERT_EQ(event->header.timestamp, 1651482394); + ASSERT_EQ(event->header.event_size, 82); + ASSERT_EQ(event->header.log_pos, 951); + + if (!filtered) + { + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, TABLE_MAP_EVENT); + ASSERT_EQ(event->header.timestamp, 1651482394); + ASSERT_EQ(event->header.event_size, 48); + ASSERT_EQ(event->header.log_pos, 999); + } + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, UPDATE_ROWS_EVENT_V2); + ASSERT_EQ(event->header.timestamp, 1651482394); + ASSERT_EQ(event->header.event_size, 70); + ASSERT_EQ(event->header.log_pos, 1069); + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, XID_EVENT); + ASSERT_EQ(event->header.timestamp, 1651482394); + ASSERT_EQ(event->header.event_size, 31); + ASSERT_EQ(event->header.log_pos, 1100); + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, GTID_EVENT); + ASSERT_EQ(event->header.timestamp, 1651483072); + ASSERT_EQ(event->header.event_size, 79); + ASSERT_EQ(event->header.log_pos, 1179); + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, QUERY_EVENT); + ASSERT_EQ(event->header.timestamp, 1651483072); + ASSERT_EQ(event->header.event_size, 82); + ASSERT_EQ(event->header.log_pos, 1261); + + if (!filtered) + { + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, TABLE_MAP_EVENT); + ASSERT_EQ(event->header.timestamp, 1651483072); + ASSERT_EQ(event->header.event_size, 48); + ASSERT_EQ(event->header.log_pos, 1309); + } + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, UPDATE_ROWS_EVENT_V2); + ASSERT_EQ(event->header.timestamp, 1651483072); + ASSERT_EQ(event->header.event_size, 70); + ASSERT_EQ(event->header.log_pos, 1379); + + ASSERT_EQ(binlog.getPosition().gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:87828-87830"); + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, XID_EVENT); + ASSERT_EQ(event->header.timestamp, 1651483072); + ASSERT_EQ(event->header.event_size, 31); + ASSERT_EQ(event->header.log_pos, 1410); + + ASSERT_EQ(binlog.getPosition().gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:87828-87831"); + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, GTID_EVENT); + ASSERT_EQ(event->header.timestamp, 1651483336); + ASSERT_EQ(event->header.event_size, 79); + ASSERT_EQ(event->header.log_pos, 1489); + gtid_event = std::static_pointer_cast(event); + ASSERT_TRUE(gtid_event); + ASSERT_EQ(gtid_event->commit_flag, 0); + gtid_expected = {}; + gtid_expected.parse("a9d88f83-c14e-11ec-bb36-244bfedf7766:87832"); + gtid_actual = {}; + gtid_actual.update(gtid_event->gtid); + ASSERT_EQ(gtid_actual.toString(), gtid_expected.toString()); + + ASSERT_EQ(binlog.getPosition().gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:87828-87831"); + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, QUERY_EVENT); + ASSERT_EQ(event->header.timestamp, 1651483336); + ASSERT_EQ(event->header.event_size, 82); + ASSERT_EQ(event->header.log_pos, 1571); + + if (!filtered) + { + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, TABLE_MAP_EVENT); + ASSERT_EQ(event->header.timestamp, 1651483336); + ASSERT_EQ(event->header.event_size, 48); + ASSERT_EQ(event->header.log_pos, 1619); + } + + int total_count = filtered ? 37 : 48; + for (; count < total_count; ++count) + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, ROTATE_EVENT); + ASSERT_EQ(event->header.timestamp, 1651528821); + ASSERT_EQ(event->header.event_size, 44); + ASSERT_EQ(event->header.log_pos, 3091); + ASSERT_EQ(count, total_count); + ASSERT_FALSE(binlog.tryReadEvent(event, 10)); + + auto position = binlog.getPosition(); + ASSERT_EQ(position.binlog_pos, 4); + ASSERT_EQ(position.binlog_name, "binlog.001391"); + ASSERT_EQ(position.gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:87828-87836"); +} + +TEST_F(MySQLBinlog, binlogFromFile1) +{ + BinlogFromFile binlog; + binlog.open(getTestDataPath("binlog.001390")); + testFile1(binlog, timeout); +} + +TEST_F(MySQLBinlog, binlogFromFactory1) +{ + auto f = std::make_shared(getTestDataPath("binlog.001390")); + auto binlog = f->createBinlog(""); + + testFile1(*binlog, timeout); +} + +TEST_F(MySQLBinlog, binlogFromFactory1ExecutedGtidSet) +{ + auto f = std::make_shared(getTestDataPath("binlog.001390")); + BinlogEventPtr event; + + auto binlog = f->createBinlog("a9d88f83-c14e-11ec-bb36-244bfedf7766:1-87828"); + + ASSERT_TRUE(binlog->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, GTID_EVENT); + ASSERT_EQ(event->header.log_pos, 559); + + auto gtid_event = std::static_pointer_cast(event); + ASSERT_TRUE(gtid_event); + GTIDSets gtid_expected; + gtid_expected.parse("a9d88f83-c14e-11ec-bb36-244bfedf7766:87829"); + GTIDSets gtid_actual; + gtid_actual.update(gtid_event->gtid); + ASSERT_EQ(gtid_actual.toString(), gtid_expected.toString()); + + for (int count = 8; count < 48; ++count) + ASSERT_TRUE(binlog->tryReadEvent(event, timeout)); + + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, ROTATE_EVENT); + auto position = binlog->getPosition(); + ASSERT_EQ(position.binlog_pos, 4); + ASSERT_EQ(position.binlog_name, "binlog.001391"); + ASSERT_EQ(position.gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1-87836"); + ASSERT_FALSE(binlog->tryReadEvent(event, 10)); + + binlog = f->createBinlog("a9d88f83-c14e-11ec-bb36-244bfedf7766:1-87829"); + + ASSERT_TRUE(binlog->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, GTID_EVENT); + ASSERT_EQ(event->header.log_pos, 869); + + gtid_event = std::static_pointer_cast(event); + ASSERT_TRUE(gtid_event); + gtid_expected = {}; + gtid_expected.parse("a9d88f83-c14e-11ec-bb36-244bfedf7766:87830"); + gtid_actual = {}; + gtid_actual.update(gtid_event->gtid); + ASSERT_EQ(gtid_actual.toString(), gtid_expected.toString()); + + for (int count = 13; count < 48; ++count) + ASSERT_TRUE(binlog->tryReadEvent(event, timeout)); + + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, ROTATE_EVENT); + position = binlog->getPosition(); + ASSERT_EQ(position.binlog_pos, 4); + ASSERT_EQ(position.binlog_name, "binlog.001391"); + ASSERT_EQ(position.gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1-87836"); + ASSERT_FALSE(binlog->tryReadEvent(event, 10)); + + binlog = f->createBinlog("a9d88f83-c14e-11ec-bb36-244bfedf7766:1-87834"); + + ASSERT_TRUE(binlog->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, GTID_EVENT); + ASSERT_EQ(event->header.log_pos, 2443); + + gtid_event = std::static_pointer_cast(event); + ASSERT_TRUE(gtid_event); + gtid_expected = {}; + gtid_expected.parse("a9d88f83-c14e-11ec-bb36-244bfedf7766:87835"); + gtid_actual = {}; + gtid_actual.update(gtid_event->gtid); + ASSERT_EQ(gtid_actual.toString(), gtid_expected.toString()); + + for (int count = 38; count < 48; ++count) + ASSERT_TRUE(binlog->tryReadEvent(event, timeout)); + + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, ROTATE_EVENT); + position = binlog->getPosition(); + ASSERT_EQ(position.binlog_pos, 4); + ASSERT_EQ(position.binlog_name, "binlog.001391"); + ASSERT_EQ(position.gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1-87836"); + ASSERT_FALSE(binlog->tryReadEvent(event, 10)); +} + +TEST_F(MySQLBinlog, binlogFromDispatcher1) +{ + auto f = std::make_shared(getTestDataPath("binlog.001390")); + BinlogEventsDispatcher d; + auto b = d.start(f->createBinlog("")); + testFile1(*b, timeout, true); + ASSERT_EQ(d.getPosition().gtid_sets.toString(), b->getPosition().gtid_sets.toString()); +} + +static void testFile2(IBinlog & binlog, UInt64 timeout, bool filtered = false) +{ + BinlogEventPtr event; + int count = 0; + + if (!filtered) + { + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, FORMAT_DESCRIPTION_EVENT); + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, PREVIOUS_GTIDS_EVENT); + } + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, GTID_EVENT); + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, QUERY_EVENT); + + if (!filtered) + { + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, TABLE_MAP_EVENT); + } + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, UPDATE_ROWS_EVENT_V2); + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, XID_EVENT); + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, GTID_EVENT); + ASSERT_EQ(event->header.log_pos, 608); + + auto gtid_event = std::static_pointer_cast(event); + ASSERT_TRUE(gtid_event); + ASSERT_EQ(gtid_event->commit_flag, 0); + GTIDSets gtid_expected; + gtid_expected.parse("a9d88f83-c14e-11ec-bb36-244bfedf7766:1059"); + GTIDSets gtid_actual; + gtid_actual.update(gtid_event->gtid); + ASSERT_EQ(gtid_actual.toString(), gtid_expected.toString()); + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, QUERY_EVENT); + ASSERT_EQ(event->header.log_pos, 701); + + if (!filtered) + { + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, TABLE_MAP_EVENT); + ASSERT_EQ(event->header.log_pos, 760); + } + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, UPDATE_ROWS_EVENT_V2); + ASSERT_EQ(event->header.log_pos, 830); + + ASSERT_EQ(event->type(), MYSQL_UNPARSED_ROWS_EVENT); + event = std::static_pointer_cast(event)->parse(); + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, XID_EVENT); + ASSERT_EQ(event->header.log_pos, 861); + + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + ++count; + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, GTID_EVENT); + ASSERT_EQ(event->header.log_pos, 940); + gtid_event = std::static_pointer_cast(event); + ASSERT_TRUE(gtid_event); + ASSERT_EQ(gtid_event->commit_flag, 0); + gtid_expected = {}; + gtid_expected.parse("a9d88f83-c14e-11ec-bb36-244bfedf7766:1060"); + gtid_actual = {}; + gtid_actual.update(gtid_event->gtid); + ASSERT_EQ(gtid_actual.toString(), gtid_expected.toString()); + + int total_count = filtered ? 13 : 18; + for (; count < total_count; ++count) + ASSERT_TRUE(binlog.tryReadEvent(event, timeout)); + + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, ROTATE_EVENT); + ASSERT_EQ(event->header.log_pos, 1237); + ASSERT_EQ(count, total_count); + ASSERT_FALSE(binlog.tryReadEvent(event, 10)); + + auto position = binlog.getPosition(); + ASSERT_EQ(position.binlog_pos, 4); + ASSERT_EQ(position.binlog_name, "binlog.000017"); + ASSERT_EQ(binlog.getPosition().gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1058-1060"); +} + +TEST_F(MySQLBinlog, binlogFromFile2) +{ + BinlogFromFile binlog; + binlog.open(getTestDataPath("binlog.000016")); + testFile2(binlog, timeout); +} + +TEST_F(MySQLBinlog, binlogFromDispatcher2) +{ + auto f = std::make_shared(getTestDataPath("binlog.000016")); + BinlogEventsDispatcher d; + auto b = d.start(f->createBinlog("")); + testFile2(*b, timeout, true); + ASSERT_EQ(d.getPosition().gtid_sets.toString(), b->getPosition().gtid_sets.toString()); +} + +TEST_F(MySQLBinlog, binlogsFromOneFile) +{ + auto f = std::make_shared(getTestDataPath("binlog.000016")); + auto d1 = std::make_shared("d1"); + auto d2 = std::make_shared("d2"); + auto b1 = d1->start(f->createBinlog("")); + auto b2 = d2->start(f->createBinlog("")); + + testFile2(*b1, timeout, true); + testFile2(*b2, timeout, true); + + ASSERT_EQ(b1->getPosition().gtid_sets.toString(), b2->getPosition().gtid_sets.toString()); + ASSERT_EQ(b1->getPosition().gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1058-1060"); + ASSERT_EQ(b1->getPosition().binlog_pos, b2->getPosition().binlog_pos); + ASSERT_EQ(b1->getPosition().binlog_pos, 4); +} + +TEST_F(MySQLBinlog, empty) +{ + auto f = std::make_shared(getTestDataPath("binlog.000016")); + auto d1 = std::make_shared("d1"); + ASSERT_TRUE(d1->getDispatcherMetadata().binlogs.empty()); +} + +TEST_F(MySQLBinlog, binlogsAfterStart) +{ + auto f = std::make_shared(getTestDataPath("binlog.000016")); + auto d1 = std::make_shared("d1"); + + auto b1 = d1->start(f->createBinlog("")); + auto b2 = d1->start(f->createBinlog("")); + ASSERT_FALSE(b2); + + testFile2(*b1, timeout, true); + ASSERT_EQ(b1->getPosition().gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1058-1060"); +} + +TEST_F(MySQLBinlog, metadata) +{ + auto f = std::make_shared(getTestDataPath("binlog.000016")); + auto d1 = std::make_shared("d1"); + ASSERT_TRUE(d1->getDispatcherMetadata().binlogs.empty()); + ASSERT_EQ(d1->getDispatcherMetadata().name, "d1"); + ASSERT_TRUE(d1->getDispatcherMetadata().position.gtid_sets.sets.empty()); + + auto b1 = d1->start(f->createBinlog("")); + ASSERT_TRUE(b1); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs.size(), 1); + ASSERT_FALSE(d1->start(f->createBinlog(""))); + + TRY_ASSERT_TRUE(!d1->getDispatcherMetadata().position.gtid_sets.sets.empty(), timeout); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs.size(), 1); + + testFile2(*b1, timeout, true); + + ASSERT_EQ(b1->getPosition().gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1058-1060"); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs.size(), 1); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs[0].position_read.gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1058-1060"); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs[0].position_write.gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1058-1060"); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs[0].size, 0); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs[0].bytes, 0); +} + +TEST_F(MySQLBinlog, catchingUp) +{ + auto f = std::make_shared(getTestDataPath("binlog.000016")); + auto d1 = std::make_shared("d1"); + auto d2 = std::make_shared("d2"); + ASSERT_TRUE(d1->getDispatcherMetadata().binlogs.empty()); + ASSERT_TRUE(d2->getDispatcherMetadata().binlogs.empty()); + + d2->syncTo(d1); + + auto b1 = d1->start(f->createBinlog("")); + auto b2 = d2->start(f->createBinlog("")); + ASSERT_TRUE(b1); + ASSERT_TRUE(b2); + TRY_ASSERT_EQ(d1->getDispatcherMetadata().binlogs.size(), 2, timeout); + ASSERT_FALSE(d1->getDispatcherMetadata().position.gtid_sets.sets.empty()); + ASSERT_EQ(d2->getDispatcherMetadata().binlogs.size(), 0); + ASSERT_FALSE(d2->getDispatcherMetadata().position.gtid_sets.sets.empty()); + ASSERT_FALSE(d2->start(f->createBinlog(""))); + + testFile2(*b1, timeout, true); + testFile2(*b2, timeout, true); + + ASSERT_EQ(b1->getPosition().gtid_sets.toString(), b2->getPosition().gtid_sets.toString()); + ASSERT_EQ(b1->getPosition().gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1058-1060"); + ASSERT_EQ(b1->getPosition().binlog_pos, b2->getPosition().binlog_pos); + ASSERT_EQ(b1->getPosition().binlog_pos, 4); + ASSERT_EQ(d2->getDispatcherMetadata().binlogs.size(), 0); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs.size(), 2); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs[0].position_read.gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1058-1060"); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs[0].position_write.gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1058-1060"); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs[0].position_read.binlog_pos, 4); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs[0].position_write.binlog_pos, 4); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs[0].size, 0); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs[0].bytes, 0); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs[1].position_read.gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1058-1060"); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs[1].position_write.gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1058-1060"); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs[1].position_read.binlog_pos, 4); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs[1].position_write.binlog_pos, 4); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs[1].size, 0); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs[1].bytes, 0); +} + +TEST_F(MySQLBinlog, catchingUpFastMaster) +{ + auto f = std::make_shared(getTestDataPath("binlog.000016")); + auto d1 = std::make_shared("d1"); + auto d2 = std::make_shared("d2"); + + d2->syncTo(d1); + + auto b1 = d1->start(f->createBinlog("")); + std::this_thread::sleep_for(std::chrono::milliseconds(50)); + auto b2 = d2->start(f->createBinlog("")); + + testFile2(*b1, timeout, true); + testFile2(*b2, timeout, true); + + ASSERT_EQ(b1->getPosition().gtid_sets.toString(), b2->getPosition().gtid_sets.toString()); + ASSERT_EQ(b1->getPosition().gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1058-1060"); + ASSERT_EQ(b1->getPosition().binlog_pos, b2->getPosition().binlog_pos); + ASSERT_EQ(b1->getPosition().binlog_pos, 4); + ASSERT_EQ(d2->getDispatcherMetadata().binlogs.size(), 0); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs.size(), 2); +} + +TEST_F(MySQLBinlog, catchingUpFastSlave) +{ + auto f = std::make_shared(getTestDataPath("binlog.000016")); + auto d1 = std::make_shared("d1"); + auto d2 = std::make_shared("d2"); + + d2->syncTo(d1); + + auto b2 = d2->start(f->createBinlog("")); + std::this_thread::sleep_for(std::chrono::milliseconds(50)); + auto b1 = d1->start(f->createBinlog("")); + + TRY_ASSERT_EQ(d1->getDispatcherMetadata().binlogs.size(), 2, timeout); + ASSERT_EQ(d2->getDispatcherMetadata().binlogs.size(), 0); + ASSERT_FALSE(d1->getDispatcherMetadata().position.gtid_sets.sets.empty()); + ASSERT_FALSE(d2->getDispatcherMetadata().position.gtid_sets.sets.empty()); + + testFile2(*b1, timeout, true); + testFile2(*b2, timeout, true); + + ASSERT_EQ(b1->getPosition().gtid_sets.toString(), b2->getPosition().gtid_sets.toString()); + ASSERT_EQ(b1->getPosition().gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1058-1060"); + ASSERT_EQ(b1->getPosition().binlog_pos, b2->getPosition().binlog_pos); + ASSERT_EQ(b1->getPosition().binlog_pos, 4); + ASSERT_EQ(d2->getDispatcherMetadata().binlogs.size(), 0); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs.size(), 2); +} + +TEST_F(MySQLBinlog, catchingUpWithoutWaiting) +{ + auto f = std::make_shared(getTestDataPath("binlog.000016")); + auto d1 = std::make_shared("d1"); + auto d2 = std::make_shared("d2"); + + d2->syncTo(d1); + + auto b1 = d1->start(f->createBinlog("")); + auto b2 = d2->start(f->createBinlog("")); + + testFile2(*b1, timeout, true); + testFile2(*b2, timeout, true); + + ASSERT_EQ(b1->getPosition().gtid_sets.toString(), b2->getPosition().gtid_sets.toString()); + ASSERT_EQ(b1->getPosition().gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1058-1060"); + ASSERT_EQ(b1->getPosition().binlog_pos, b2->getPosition().binlog_pos); + ASSERT_EQ(b1->getPosition().binlog_pos, 4); + TRY_ASSERT_EQ(d2->getDispatcherMetadata().binlogs.size(), 0, timeout); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs.size(), 2); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs[0].position_read.gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1058-1060"); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs[0].position_read.binlog_pos, 4); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs[1].position_read.gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1058-1060"); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs[1].position_read.binlog_pos, 4); +} + +TEST_F(MySQLBinlog, catchingUpManyToOne) +{ + auto f = std::make_shared(getTestDataPath("binlog.000016")); + auto d0 = std::make_shared("d0"); + std::vector ds; + int n = 10; + for (int i = 0; i < n; ++i) + { + auto d = std::make_shared("r" + std::to_string(i)); + d->syncTo(d0); + ds.push_back(d); + } + + for (int i = 0; i < n; ++i) + ASSERT_TRUE(ds[i]->getDispatcherMetadata().binlogs.empty()); + + auto b0 = d0->start(f->createBinlog(""), "b"); + ASSERT_EQ(d0->getDispatcherMetadata().binlogs.size(), 1); + ASSERT_EQ(d0->getDispatcherMetadata().binlogs[0].position_read.binlog_pos, 0); + std::vector bs; + bs.resize(n); + for (int i = 0; i < n; ++i) + bs[i] = ds[i]->start(f->createBinlog(""), "b" + std::to_string(i)); + + TRY_ASSERT_EQ(d0->getDispatcherMetadata().binlogs.size(), n + 1, timeout); + ASSERT_FALSE(d0->getDispatcherMetadata().position.gtid_sets.sets.empty()); + for (int i = 0; i < n; ++i) + { + ASSERT_EQ(ds[i]->getDispatcherMetadata().binlogs.size(), 0); + ASSERT_FALSE(ds[i]->getDispatcherMetadata().position.gtid_sets.sets.empty()); + } + + testFile2(*b0, timeout, true); + for (int i = 0; i < n; ++i) + testFile2(*bs[i], timeout, true); + + ASSERT_EQ(b0->getPosition().gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1058-1060"); + ASSERT_EQ(b0->getPosition().binlog_pos, 4); + + for (int i = 0; i < n; ++i) + { + ASSERT_EQ(bs[i]->getPosition().gtid_sets.toString(), b0->getPosition().gtid_sets.toString()); + ASSERT_EQ(bs[i]->getPosition().binlog_pos, b0->getPosition().binlog_pos); + } + + for (int i = 0; i < n; ++i) + ASSERT_EQ(ds[i]->getDispatcherMetadata().binlogs.size(), 0); + + ASSERT_EQ(d0->getDispatcherMetadata().binlogs.size(), n + 1); + for (int i = 0; i < n + 1; ++i) + { + ASSERT_EQ(d0->getDispatcherMetadata().binlogs[i].position_read.gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1058-1060"); + ASSERT_EQ(d0->getDispatcherMetadata().binlogs[i].position_write.gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1058-1060"); + ASSERT_EQ(d0->getDispatcherMetadata().binlogs[i].position_read.binlog_pos, 4); + ASSERT_EQ(d0->getDispatcherMetadata().binlogs[i].position_write.binlog_pos, 4); + ASSERT_EQ(d0->getDispatcherMetadata().binlogs[i].size, 0); + ASSERT_EQ(d0->getDispatcherMetadata().binlogs[i].bytes, 0); + } +} + +TEST_F(MySQLBinlog, catchingUpStopApplier) +{ + auto f = std::make_shared(getTestDataPath("binlog.000016")); + auto d1 = std::make_shared("d1"); + auto d2 = std::make_shared("d2"); + + d2->syncTo(d1); + + auto b1 = d1->start(f->createBinlog("")); + ASSERT_TRUE(b1); + d1 = nullptr; + + auto b2 = d2->start(f->createBinlog("")); + ASSERT_TRUE(b2); + testFile2(*b2, timeout, true); + ASSERT_EQ(b2->getPosition().gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1058-1060"); +} + +TEST_F(MySQLBinlog, catchingUpOneToAllPrevious) +{ + auto f = std::make_shared(getTestDataPath("binlog.000016")); + std::vector ds; + int n = 10; + for (int i = 0; i < n; ++i) + { + auto d = std::make_shared("d" + std::to_string(i)); + for (int j = 0; j < i; ++j) + d->syncTo(ds[j]); + ds.push_back(d); + } + + for (int i = 0; i < n; ++i) + ASSERT_TRUE(ds[i]->getDispatcherMetadata().binlogs.empty()); + + std::vector bs; + bs.resize(n); + for (int i = 0; i < n; ++i) + bs[i] = ds[i]->start(f->createBinlog(""), "b" + std::to_string(i)); + + auto check_dispatchers = [&] + { + int not_empty_count = 0; + int ii = 0; + for (int i = 0; i < n; ++i) + { + if (!ds[i]->getDispatcherMetadata().binlogs.empty()) + { + ++not_empty_count; + ii = i; + } + } + return not_empty_count == 1 && ds[ii]->getDispatcherMetadata().binlogs.size() == n; + }; + + for (int i = 0; i < n; ++i) + testFile2(*bs[i], timeout, true); + + TRY_ASSERT_TRUE(check_dispatchers(), timeout); + + for (int i = 1; i < n; ++i) + { + ASSERT_EQ(bs[i]->getPosition().gtid_sets.toString(), bs[0]->getPosition().gtid_sets.toString()); + ASSERT_EQ(bs[i]->getPosition().binlog_pos, bs[0]->getPosition().binlog_pos); + } + + int i = 0; + for (int j = 0; j < n; ++j) + { + auto bs_ = ds[j]->getDispatcherMetadata().binlogs; + for (; i < bs_.size(); ++i) + { + ASSERT_EQ(bs_[i].position_read.gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1058-1060"); + ASSERT_EQ(bs_[i].position_write.gtid_sets.toString(), bs_[i].position_write.gtid_sets.toString()); + ASSERT_EQ(bs_[i].position_read.binlog_pos, 4); + ASSERT_EQ(bs_[i].position_write.binlog_pos, 4); + ASSERT_EQ(bs_[i].size, 0); + ASSERT_EQ(bs_[i].bytes, 0); + } + } + ASSERT_EQ(i, n); +} + +TEST_F(MySQLBinlog, catchingUpMaxBytes) +{ + auto f = std::make_shared(getTestDataPath("binlog.000016")); + auto d1 = std::make_shared("d1"); + auto d2 = std::make_shared("d2"); + + d2->syncTo(d1); + + auto b1 = d1->start(f->createBinlog(""), "big"); + auto b2 = d2->start(f->createBinlog(""), "small", {}, 1, 10000); + + testFile2(*b2, timeout, true); + TRY_ASSERT_EQ(d1->getDispatcherMetadata().binlogs.size(), 2, timeout); + ASSERT_EQ(d1->getDispatcherMetadata().position.gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1058-1060"); + testFile2(*b1, timeout, true); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs[0].position_write.gtid_sets.toString(), d1->getDispatcherMetadata().position.gtid_sets.toString()); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs[0].position_read.gtid_sets.toString(), d1->getDispatcherMetadata().position.gtid_sets.toString()); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs[1].position_write.gtid_sets.toString(), d1->getDispatcherMetadata().position.gtid_sets.toString()); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs[1].position_read.gtid_sets.toString(), d1->getDispatcherMetadata().position.gtid_sets.toString()); +} + +TEST_F(MySQLBinlog, filterEvents) +{ + auto f = std::make_shared(getTestDataPath("binlog.001390")); + auto d1 = std::make_shared("d1"); + auto d2 = std::make_shared("d2"); + auto b1 = d1->start(f->createBinlog(""), "b1", {"db"}); + auto b2 = d2->start(f->createBinlog(""), "b2", {"unknown_database"}); + + BinlogEventPtr event; + for (int i = 0; i < 37; ++i) + { + ASSERT_TRUE(b1->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + switch (event->header.type) + { + case WRITE_ROWS_EVENT_V1: + case WRITE_ROWS_EVENT_V2: + case DELETE_ROWS_EVENT_V1: + case DELETE_ROWS_EVENT_V2: + case UPDATE_ROWS_EVENT_V1: + case UPDATE_ROWS_EVENT_V2: + if (event->type() == MYSQL_UNPARSED_ROWS_EVENT) + { + ASSERT_EQ(std::static_pointer_cast(event)->schema, "db"); + } + break; + default: + break; + } + } + + ASSERT_FALSE(b1->tryReadEvent(event, 0)); + ASSERT_FALSE(event); + + for (int i = 0; i < 37; ++i) + { + ASSERT_TRUE(b2->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + switch (event->header.type) + { + case ROTATE_EVENT: + case XID_EVENT: + case QUERY_EVENT: + case GTID_EVENT: + break; + default: + if (event->type() != MYSQL_UNHANDLED_EVENT) + FAIL() << "Unexpected event: " << magic_enum::enum_name(event->header.type); + break; + } + } + + ASSERT_FALSE(b2->tryReadEvent(event, 0)); + ASSERT_FALSE(event); + + ASSERT_EQ(b1->getPosition().gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:87828-87836"); + ASSERT_EQ(b1->getPosition().binlog_pos, 4); + ASSERT_EQ(b2->getPosition().gtid_sets.toString(), b1->getPosition().gtid_sets.toString()); + ASSERT_EQ(b2->getPosition().binlog_pos, b1->getPosition().binlog_pos); + ASSERT_FALSE(b2->tryReadEvent(event, 0)); +} + +TEST_F(MySQLBinlog, filterEventsMultipleDatabases) +{ + auto f = std::make_shared(getTestDataPath("binlog.001390")); + auto d1 = std::make_shared("d1"); + auto d2 = std::make_shared("d2"); + auto d3 = std::make_shared("d3"); + auto d4 = std::make_shared("d4"); + auto d5 = std::make_shared("d5"); + auto all_dbs = d1->start(f->createBinlog(""), "all_dbs"); + auto db = d2->start(f->createBinlog(""), "db", {"db"}); + auto aborted = d3->start(f->createBinlog(""), "aborted_full_sync", {"aborted_full_sync"}); + auto db_and_aborted = d4->start(f->createBinlog(""), "db_and_aborted", {"db", "aborted_full_sync"}); + auto unknown = d5->start(f->createBinlog(""), "unknown", {"unknown1", "unknown2"}); + + BinlogEventPtr event; + for (int i = 0; i < 37; ++i) + { + ASSERT_TRUE(all_dbs->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + switch (event->header.type) + { + case WRITE_ROWS_EVENT_V1: + case WRITE_ROWS_EVENT_V2: + case DELETE_ROWS_EVENT_V1: + case DELETE_ROWS_EVENT_V2: + case UPDATE_ROWS_EVENT_V1: + case UPDATE_ROWS_EVENT_V2: + ASSERT_EQ(event->type(), MYSQL_UNPARSED_ROWS_EVENT); + break; + default: + break; + } + } + + ASSERT_FALSE(all_dbs->tryReadEvent(event, 0)); + ASSERT_FALSE(event); + + for (int i = 0; i < 37; ++i) + { + ASSERT_TRUE(db->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + switch (event->header.type) + { + case WRITE_ROWS_EVENT_V1: + case WRITE_ROWS_EVENT_V2: + case DELETE_ROWS_EVENT_V1: + case DELETE_ROWS_EVENT_V2: + case UPDATE_ROWS_EVENT_V1: + case UPDATE_ROWS_EVENT_V2: + if (event->type() == MYSQL_UNPARSED_ROWS_EVENT) + { + ASSERT_EQ(std::static_pointer_cast(event)->schema, "db"); + } + break; + default: + break; + } + } + + ASSERT_FALSE(db->tryReadEvent(event, 0)); + ASSERT_FALSE(event); + + for (int i = 0; i < 37; ++i) + { + ASSERT_TRUE(aborted->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + switch (event->header.type) + { + case WRITE_ROWS_EVENT_V1: + case WRITE_ROWS_EVENT_V2: + case DELETE_ROWS_EVENT_V1: + case DELETE_ROWS_EVENT_V2: + case UPDATE_ROWS_EVENT_V1: + case UPDATE_ROWS_EVENT_V2: + if (event->type() == MYSQL_UNPARSED_ROWS_EVENT) + { + ASSERT_EQ(std::static_pointer_cast(event)->schema, "aborted_full_sync"); + } + break; + default: + break; + } + } + + ASSERT_FALSE(aborted->tryReadEvent(event, 0)); + ASSERT_FALSE(event); + + for (int i = 0; i < 37; ++i) + { + ASSERT_TRUE(db_and_aborted->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + switch (event->header.type) + { + case WRITE_ROWS_EVENT_V1: + case WRITE_ROWS_EVENT_V2: + case DELETE_ROWS_EVENT_V1: + case DELETE_ROWS_EVENT_V2: + case UPDATE_ROWS_EVENT_V1: + case UPDATE_ROWS_EVENT_V2: + { + ASSERT_EQ(event->type(), MYSQL_UNPARSED_ROWS_EVENT); + auto schema = std::static_pointer_cast(event)->schema; + ASSERT_TRUE(schema == "db" || schema == "aborted_full_sync"); + } break; + default: + break; + } + } + + ASSERT_FALSE(db_and_aborted->tryReadEvent(event, 0)); + ASSERT_FALSE(event); + + for (int i = 0; i < 37; ++i) + { + ASSERT_TRUE(unknown->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + switch (event->header.type) + { + case ROTATE_EVENT: + case XID_EVENT: + case QUERY_EVENT: + case GTID_EVENT: + break; + default: + ASSERT_EQ(event->type(), MYSQL_UNHANDLED_EVENT); + break; + } + } + + ASSERT_FALSE(unknown->tryReadEvent(event, 0)); + ASSERT_FALSE(event); +} + +TEST_F(MySQLBinlog, dispatcherStop) +{ + auto f = std::make_shared(getTestDataPath("binlog.000016")); + auto d1 = std::make_shared("d1"); + auto b1 = d1->start(f->createBinlog("")); + ASSERT_TRUE(b1); + d1 = nullptr; + BinlogEventPtr event; + EXPECT_THROW(for (int i = 0; i < 18 + 1; ++i) b1->tryReadEvent(event, timeout), DB::Exception); +} + +TEST_F(MySQLBinlog, executedGTIDSet) +{ + auto f = std::make_shared(getTestDataPath("binlog.000016")); + auto d1 = std::make_shared("d1"); + auto b1 = d1->start(f->createBinlog("a9d88f83-c14e-11ec-bb36-244bfedf7766:1-1058"), "b1"); + + BinlogEventPtr event; + ASSERT_TRUE(b1->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, GTID_EVENT); + ASSERT_EQ(event->header.log_pos, 608); + + ASSERT_TRUE(b1->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, QUERY_EVENT); + ASSERT_EQ(event->header.log_pos, 701); + + for (int i = 0; i < 7; ++i) + ASSERT_TRUE(b1->tryReadEvent(event, timeout)); + + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, ROTATE_EVENT); + ASSERT_EQ(event->header.log_pos, 1237); + ASSERT_EQ(d1->getPosition().gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1-1060"); + ASSERT_EQ(b1->getPosition().gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1-1060"); + ASSERT_FALSE(b1->tryReadEvent(event, 0)); +} + +TEST_F(MySQLBinlog, client) +{ + auto f = std::make_shared(getTestDataPath("binlog.000016")); + auto e = std::make_shared(f); + + auto b1 = e->createBinlog("", "b1"); + auto b2 = e->createBinlog("", "b2"); + testFile2(*b1, timeout, true); + testFile2(*b2, timeout, true); + + auto b3 = e->createBinlog("", "b3"); + + testFile2(*b3, timeout, true); + + b1 = nullptr; + b2 = nullptr; + + auto b4 = e->createBinlog("", "b4"); + testFile2(*b4, timeout, true); + + b3 = nullptr; + b4 = e->createBinlog("", "b4 2"); + testFile2(*b4, timeout, true); + + b1 = e->createBinlog("", "b1 2"); + b2 = e->createBinlog("", "b2 2"); + testFile2(*b1, timeout, true); + + b3 = e->createBinlog("", "b3 2"); + testFile2(*b2, timeout, true); + + b4 = e->createBinlog("", "b4 3"); + testFile2(*b3, timeout, true); + testFile2(*b4, timeout, true); + + b1 = nullptr; + b2 = nullptr; + b3 = nullptr; + b4 = nullptr; + b1 = e->createBinlog("", "b1 3"); + b2 = e->createBinlog("", "b2 3"); + b3 = e->createBinlog("", "b3 3"); + b4 = e->createBinlog("", "b4 4"); + testFile2(*b4, timeout, true); + testFile2(*b3, timeout, true); + testFile2(*b2, timeout, true); + testFile2(*b1, timeout, true); + + f = std::make_shared(getTestDataPath("binlog.000016")); + e = std::make_shared(f); + + b4 = e->createBinlog("", "b4 5"); + b3 = e->createBinlog("", "b3 4"); + testFile2(*b4, timeout, true); + b2 = e->createBinlog("", "b2 4"); + b1 = e->createBinlog("", "b1 4"); + testFile2(*b3, timeout, true); + testFile2(*b1, timeout, true); + testFile2(*b2, timeout, true); + + b1 = e->createBinlog("", "b1 5"); + b2 = e->createBinlog("", "b2 5"); + testFile2(*b1, timeout, true); + testFile2(*b2, timeout, true); + b1 = e->createBinlog("", "b1 6"); + testFile2(*b1, timeout, true); + b1 = e->createBinlog("", "b1 7"); + testFile2(*b1, timeout, true); + + b3 = nullptr; + b4 = nullptr; + b1 = e->createBinlog("", "b1 8"); + b4 = e->createBinlog("", "b4 6"); + b3 = e->createBinlog("", "b3 5"); + testFile2(*b4, timeout, true); + testFile2(*b3, timeout, true); + testFile2(*b1, timeout, true); + + b2 = nullptr; + b3 = nullptr; + b4 = nullptr; + b1 = nullptr; + b1 = e->createBinlog("", "b1 9"); + testFile2(*b1, timeout, true); +} + +TEST_F(MySQLBinlog, createBinlog) +{ + auto f = std::make_shared(getTestDataPath("binlog.001390")); + auto d1 = std::make_shared("d1"); + auto b1 = d1->start(f->createBinlog(""), "b1"); + ASSERT_TRUE(b1); + ASSERT_FALSE(d1->start(f->createBinlog(""))); + testFile1(*b1, timeout, true); + ASSERT_FALSE(d1->start(f->createBinlog(""))); + b1 = nullptr; + ASSERT_FALSE(d1->start(f->createBinlog(""))); +} + +TEST_F(MySQLBinlog, createBinlogAttach1) +{ + auto f = std::make_shared(getTestDataPath("binlog.001390")); + auto d1 = std::make_shared("d1"); + auto b1_ = d1->start(f->createBinlog("a9d88f83-c14e-11ec-bb36-244bfedf7766:1-87828"), "b1_"); + ASSERT_TRUE(b1_); + auto b1 = d1->attach("a9d88f83-c14e-11ec-bb36-244bfedf7766:1-87831", "b1"); + if (b1) + { + BinlogEventPtr event; + ASSERT_EQ(b1->getPosition().gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1-87831"); + ASSERT_TRUE(b1->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, GTID_EVENT); + ASSERT_EQ(b1->getPosition().gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1-87831"); + + ASSERT_TRUE(b1->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, QUERY_EVENT); + ASSERT_EQ(b1->getPosition().gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1-87831"); + + ASSERT_TRUE(b1->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, UPDATE_ROWS_EVENT_V2); + ASSERT_EQ(b1->getPosition().gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1-87831"); + + ASSERT_TRUE(b1->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, XID_EVENT); + ASSERT_EQ(b1->getPosition().gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1-87832"); + for (int i = 0; i < 17; ++i) + ASSERT_TRUE(b1->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, ROTATE_EVENT); + ASSERT_FALSE(b1->tryReadEvent(event, 10)); + ASSERT_EQ(b1->getPosition().binlog_pos, 4); + ASSERT_EQ(b1->getPosition().binlog_name, "binlog.001391"); + ASSERT_EQ(b1->getPosition().gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1-87836"); + for (int i = 0; i < 33; ++i) + ASSERT_TRUE(b1_->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, ROTATE_EVENT); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs.size(), 2); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs[0].bytes, 0); + ASSERT_EQ(d1->getDispatcherMetadata().binlogs[1].bytes, 0); + } +} + +TEST_F(MySQLBinlog, createBinlogAttach2) +{ + BinlogEventPtr event; + auto f = std::make_shared(getTestDataPath("binlog.001390")); + auto d1 = std::make_shared("d1"); + auto d2 = std::make_shared("d2"); + auto d3 = std::make_shared("d3"); + auto d4 = std::make_shared("d4"); + + auto b1 = d1->start(f->createBinlog("a9d88f83-c14e-11ec-bb36-244bfedf7766:1-87828"), "b1"); + ASSERT_TRUE(b1); + ASSERT_TRUE(b1->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, GTID_EVENT); + + auto b2_ = d2->start(f->createBinlog("a9d88f83-c14e-11ec-bb36-244bfedf7766:1-87828"), "b2_"); + ASSERT_TRUE(b2_); + auto b2 = d2->attach("a9d88f83-c14e-11ec-bb36-244bfedf7766:1-87831", "b2"); + + auto b3_ = d3->start(f->createBinlog("a9d88f83-c14e-11ec-bb36-244bfedf7766:1-87828"), "b3_"); + ASSERT_TRUE(b3_); + auto b3 = d3->attach("a9d88f83-c14e-11ec-bb36-244bfedf7766:1-87835", "b3"); + + auto b4_ = d4->start(f->createBinlog("a9d88f83-c14e-11ec-bb36-244bfedf7766:1-87828"), "b4_"); + ASSERT_TRUE(b4_); + auto b4 = d4->attach("a9d88f83-c14e-11ec-bb36-244bfedf7766:1-87828", "b4"); + + /// There is a race with dispatcher thread + if (b2) + { + ASSERT_TRUE(b2->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, GTID_EVENT); + + ASSERT_TRUE(b2->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, QUERY_EVENT); + + ASSERT_TRUE(b2->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, UPDATE_ROWS_EVENT_V2); + for (int i = 0; i < 18; ++i) + ASSERT_TRUE(b2->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, ROTATE_EVENT); + ASSERT_FALSE(b2->tryReadEvent(event, 10)); + ASSERT_EQ(b2->getPosition().binlog_pos, 4); + ASSERT_EQ(b2->getPosition().binlog_name, "binlog.001391"); + ASSERT_EQ(b2->getPosition().gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1-87836"); + for (int i = 0; i < 33; ++i) + ASSERT_TRUE(b2_->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, ROTATE_EVENT); + ASSERT_EQ(d2->getDispatcherMetadata().binlogs.size(), 2); + ASSERT_EQ(d2->getDispatcherMetadata().binlogs[0].bytes, 0); + ASSERT_EQ(d2->getDispatcherMetadata().binlogs[1].bytes, 0); + } + + if (b4) + { + ASSERT_TRUE(b4->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, GTID_EVENT); + + ASSERT_TRUE(b4->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, QUERY_EVENT); + + ASSERT_TRUE(b4->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, WRITE_ROWS_EVENT_V2); + for (int i = 0; i < 10; ++i) + ASSERT_TRUE(b4->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, ROTATE_EVENT); + ASSERT_FALSE(b2->tryReadEvent(event, 10)); + ASSERT_EQ(b4->getPosition().gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1-87836"); + for (int i = 0; i < 33; ++i) + ASSERT_TRUE(b4_->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, ROTATE_EVENT); + ASSERT_EQ(d4->getDispatcherMetadata().binlogs.size(), 2); + ASSERT_EQ(d4->getDispatcherMetadata().binlogs[0].bytes, 0); + ASSERT_EQ(d4->getDispatcherMetadata().binlogs[1].bytes, 0); + } + + if (b3) + { + ASSERT_TRUE(b3->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, GTID_EVENT); + + ASSERT_TRUE(b3->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, QUERY_EVENT); + for (int i = 0; i < 3; ++i) + ASSERT_TRUE(b3->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, ROTATE_EVENT); + ASSERT_FALSE(b3->tryReadEvent(event, 10)); + ASSERT_EQ(b3->getPosition().gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:1-87836"); + for (int i = 0; i < 33; ++i) + ASSERT_TRUE(b3_->tryReadEvent(event, timeout)); + ASSERT_TRUE(event); + ASSERT_EQ(event->header.type, ROTATE_EVENT); + ASSERT_EQ(d3->getDispatcherMetadata().binlogs.size(), 2); + ASSERT_EQ(d3->getDispatcherMetadata().binlogs[0].bytes, 0); + ASSERT_EQ(d3->getDispatcherMetadata().binlogs[1].bytes, 0); + } +} + +TEST_F(MySQLBinlog, factoryThreads) +{ + auto f = std::make_shared(getTestDataPath("binlog.001390")); + auto func1 = [&] + { + auto b1 = f->createBinlog(""); + auto b2 = f->createBinlog(""); + auto b3 = f->createBinlog(""); + testFile1(*b1, timeout); + testFile1(*b2, timeout); + b2 = f->createBinlog(""); + testFile1(*b2, timeout); + b1 = f->createBinlog(""); + testFile1(*b1, timeout); + b1 = nullptr; + b2 = f->createBinlog(""); + testFile1(*b2, timeout); + b1 = f->createBinlog(""); + testFile1(*b1, timeout); + testFile1(*b3, timeout); + }; + + auto func2 = [&] + { + auto b1 = f->createBinlog(""); + auto b2 = f->createBinlog(""); + testFile1(*b2, timeout); + testFile1(*b1, timeout); + b1 = f->createBinlog(""); + testFile1(*b1, timeout); + b2 = f->createBinlog(""); + testFile1(*b2, timeout); + b1 = f->createBinlog(""); + b2 = f->createBinlog(""); + testFile1(*b1, timeout); + b2 = nullptr; + b1 = f->createBinlog(""); + testFile1(*b1, timeout); + b1 = nullptr; + }; + + int n = 4; + std::vector ts1, ts2; + for (int i = 0; i < n; ++i) + { + ts1.emplace_back(std::thread(func1)); + ts2.emplace_back(std::thread(func2)); + } + for (int i = 0; i < n; ++i) + { + ts1[i].join(); + ts2[i].join(); + } +} + +TEST_F(MySQLBinlog, clientThreads) +{ + auto f = std::make_shared(getTestDataPath("binlog.001390")); + auto e = std::make_shared(f); + auto func1 = [&] + { + auto b1 = e->createBinlog(""); + auto b2 = e->createBinlog(""); + testFile1(*b1, timeout, true); + testFile1(*b2, timeout, true); + b1 = nullptr; + b2 = nullptr; + b2 = e->createBinlog(""); + testFile1(*b2, timeout, true); + b1 = e->createBinlog(""); + testFile1(*b1, timeout, true); + b1 = nullptr; + b2 = e->createBinlog(""); + testFile1(*b2, timeout, true); + b2 = nullptr; + b1 = e->createBinlog(""); + testFile1(*b1, timeout, true); + }; + + auto func2 = [&] + { + auto b1 = e->createBinlog(""); + testFile1(*b1, timeout, true); + auto b2 = e->createBinlog(""); + testFile1(*b2, timeout, true); + b2 = e->createBinlog(""); + b1 = e->createBinlog(""); + testFile1(*b1, timeout, true); + testFile1(*b2, timeout, true); + b1 = nullptr; + b2 = nullptr; + b1 = e->createBinlog(""); + testFile1(*b1, timeout, true); + b2 = e->createBinlog(""); + testFile1(*b2, timeout, true); + }; + + int n = 4; + std::vector ts1, ts2; + for (int i = 0; i < n; ++i) + { + ts1.emplace_back(std::thread(func1)); + ts2.emplace_back(std::thread(func2)); + } + for (int i = 0; i < n; ++i) + { + ts1[i].join(); + ts2[i].join(); + } + + // All dispatchers synced and finished + // No dispatchers and no binlogs are alive here + ASSERT_EQ(e->getMetadata().dispatchers.size(), 0); + + // Creates new dispatcher + auto b1 = e->createBinlog("", "b1 1"); + testFile1(*b1, timeout, true); + + auto md = e->getMetadata().dispatchers; + ASSERT_EQ(md.size(), 1); + ASSERT_EQ(md[0].position.gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:87828-87836"); + ASSERT_EQ(md[0].binlogs.size(), 1); + ASSERT_EQ(md[0].binlogs[0].position_read.gtid_sets.toString(), "a9d88f83-c14e-11ec-bb36-244bfedf7766:87828-87836"); + ASSERT_EQ(md[0].binlogs[0].size, 0); + ASSERT_EQ(md[0].binlogs[0].bytes, 0); + + // Creates new dispatcher + auto b1_2 = e->createBinlog("", "b1 2"); + + // Should sync to the first dispatcher + TRY_ASSERT_EQ(e->getMetadata().dispatchers.size(), 1, timeout); + // If there is no CPU available, + // it possible to catch in the middle of the transform between dispatchers. + // Checking again to make sure that catching up is finished. + TRY_ASSERT_EQ(e->getMetadata().dispatchers.size(), 1, timeout); + b1 = nullptr; + md = e->getMetadata().dispatchers; + ASSERT_EQ(md.size(), 1); + ASSERT_EQ(md[0].binlogs.size(), 1); + // Did not read any events yet + ASSERT_EQ(md[0].binlogs[0].position_read.gtid_sets.toString(), ""); + ASSERT_EQ(md[0].binlogs[0].position_read.binlog_pos, 0); + + auto b2 = e->createBinlog("", "b2"); + + BinlogEventPtr event; + // Read only one event + ASSERT_TRUE(b2->tryReadEvent(event, timeout)); + // Waits before all binlogs are moved to main dispatcher + TRY_ASSERT_EQ(e->getMetadata().dispatchers[0].binlogs.size(), 2, timeout); + + // One dispatcher is alive + md = e->getMetadata().dispatchers; + ASSERT_EQ(md.size(), 1); + ASSERT_EQ(md[0].binlogs.size(), 2); + ASSERT_EQ(md[0].binlogs[0].position_read.gtid_sets.toString(), ""); + ASSERT_EQ(md[0].binlogs[1].position_read.gtid_sets.toString(), ""); + ASSERT_EQ(md[0].binlogs[0].position_read.binlog_pos, md[0].binlogs[0].name == "b2" ? 276 : 0); // Read one event + ASSERT_EQ(md[0].binlogs[1].position_read.binlog_pos, md[0].binlogs[0].name == "b2" ? 0 : 276); +} diff --git a/src/Storages/System/StorageSystemMySQLBinlogs.cpp b/src/Storages/System/StorageSystemMySQLBinlogs.cpp new file mode 100644 index 000000000000..32648d22ee8c --- /dev/null +++ b/src/Storages/System/StorageSystemMySQLBinlogs.cpp @@ -0,0 +1,164 @@ +#include + +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +NamesAndTypesList StorageSystemMySQLBinlogs::getNamesAndTypes() +{ + return { + {"binlog_client_name", std::make_shared()}, + {"name", std::make_shared()}, + {"mysql_binlog_name", std::make_shared()}, + {"mysql_binlog_pos", std::make_shared()}, + {"mysql_binlog_timestamp", std::make_shared()}, + {"mysql_binlog_executed_gtid_set", std::make_shared()}, + {"dispatcher_name", std::make_shared()}, + {"dispatcher_mysql_binlog_name", std::make_shared()}, + {"dispatcher_mysql_binlog_pos", std::make_shared()}, + {"dispatcher_mysql_binlog_timestamp", std::make_shared()}, + {"dispatcher_mysql_binlog_executed_gtid_set", std::make_shared()}, + {"size", std::make_shared()}, + {"bytes", std::make_shared()}, + {"max_bytes", std::make_shared()}, + {"max_waiting_ms", std::make_shared()}, + {"dispatcher_events_read_per_sec", std::make_shared()}, + {"dispatcher_bytes_read_per_sec", std::make_shared()}, + {"dispatcher_events_flush_per_sec", std::make_shared()}, + {"dispatcher_bytes_flush_per_sec", std::make_shared()}, + }; +} + +StorageSystemMySQLBinlogs::StorageSystemMySQLBinlogs(const StorageID & storage_id_) + : IStorage(storage_id_) +{ + StorageInMemoryMetadata storage_metadata; + ColumnsDescription columns(getNamesAndTypes()); + storage_metadata.setColumns(columns); + setInMemoryMetadata(storage_metadata); +} + +class MetadataSource : public ISource +{ +public: + using DispatcherMetadata = MySQLReplication::BinlogEventsDispatcher::DispatcherMetadata; + using BinlogMetadata = MySQLReplication::BinlogEventsDispatcher::BinlogMetadata; + + MetadataSource(Block block_header_, const std::vector & clients_) + : ISource(block_header_) + , block_to_fill(std::move(block_header_)) + , clients(clients_) + {} + + String getName() const override { return "MySQLBinlogClient"; } + +protected: + Chunk generate() override + { + if (clients.empty()) + return {}; + + Columns columns; + columns.reserve(block_to_fill.columns()); + + size_t total_size = 0; + auto create_column = [&](auto && column, const std::function & field) + { + size_t size = 0; + for (const auto & client : clients) + { + for (const auto & d : client.dispatchers) + { + for (const auto & b : d.binlogs) + { + column->insert(field(client.binlog_client_name, d, b)); + ++size; + } + } + } + if (!total_size) + total_size = size; + return std::forward(column); + }; + + for (const auto & elem : block_to_fill) + { + if (elem.name == "binlog_client_name") + columns.emplace_back(create_column(ColumnString::create(), [](auto n, auto, auto) { return Field(n); })); + else if (elem.name == "name") + columns.emplace_back(create_column(ColumnString::create(), [](auto, auto, auto b) { return Field(b.name); })); + else if (elem.name == "mysql_binlog_name") + columns.emplace_back(create_column(ColumnString::create(), [](auto, auto, auto b) { return Field(b.position_read.binlog_name); })); + else if (elem.name == "mysql_binlog_pos") + columns.emplace_back(create_column(ColumnUInt64::create(), [](auto, auto, auto b) { return Field(b.position_read.binlog_pos); })); + else if (elem.name == "mysql_binlog_timestamp") + columns.emplace_back(create_column(ColumnUInt64::create(), [](auto, auto, auto b) { return Field(b.position_read.timestamp); })); + else if (elem.name == "mysql_binlog_executed_gtid_set") + columns.emplace_back(create_column(ColumnString::create(), [](auto, auto, auto b) { return Field(b.position_read.gtid_sets.toString()); })); + else if (elem.name == "dispatcher_name") + columns.emplace_back(create_column(ColumnString::create(), [](auto, auto d, auto) { return Field(d.name); })); + else if (elem.name == "dispatcher_mysql_binlog_name") + columns.emplace_back(create_column(ColumnString::create(), [](auto, auto d, auto) { return Field(d.position.binlog_name); })); + else if (elem.name == "dispatcher_mysql_binlog_pos") + columns.emplace_back(create_column(ColumnUInt64::create(), [](auto, auto d, auto) { return Field(d.position.binlog_pos); })); + else if (elem.name == "dispatcher_mysql_binlog_timestamp") + columns.emplace_back(create_column(ColumnUInt64::create(), [](auto, auto d, auto) { return Field(d.position.timestamp); })); + else if (elem.name == "dispatcher_mysql_binlog_executed_gtid_set") + columns.emplace_back(create_column(ColumnString::create(), [](auto, auto d, auto) { return Field(d.position.gtid_sets.toString()); })); + else if (elem.name == "size") + columns.emplace_back(create_column(ColumnUInt64::create(), [](auto, auto, auto b) { return Field(b.size); })); + else if (elem.name == "bytes") + columns.emplace_back(create_column(ColumnUInt64::create(), [](auto, auto, auto b) { return Field(b.bytes); })); + else if (elem.name == "max_bytes") + columns.emplace_back(create_column(ColumnUInt64::create(), [](auto, auto, auto b) { return Field(b.max_bytes); })); + else if (elem.name == "max_waiting_ms") + columns.emplace_back(create_column(ColumnUInt64::create(), [](auto, auto, auto b) { return Field(b.max_waiting_ms); })); + else if (elem.name == "dispatcher_events_read_per_sec") + columns.emplace_back(create_column(ColumnFloat32::create(), [](auto, auto d, auto) { return Field(d.events_read_per_sec); })); + else if (elem.name == "dispatcher_bytes_read_per_sec") + columns.emplace_back(create_column(ColumnFloat32::create(), [](auto, auto d, auto) { return Field(d.bytes_read_per_sec); })); + else if (elem.name == "dispatcher_events_flush_per_sec") + columns.emplace_back(create_column(ColumnFloat32::create(), [](auto, auto d, auto) { return Field(d.events_flush_per_sec); })); + else if (elem.name == "dispatcher_bytes_flush_per_sec") + columns.emplace_back(create_column(ColumnFloat32::create(), [](auto, auto d, auto) { return Field(d.bytes_flush_per_sec); })); + } + + clients.clear(); + return {std::move(columns), total_size}; + } + +private: + Block block_to_fill; + std::vector clients; +}; + +Pipe StorageSystemMySQLBinlogs::read( + const Names & column_names_, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & /* query_info_ */, + ContextPtr /*context_ */, + QueryProcessingStage::Enum /*processed_stage*/, + size_t /*max_block_size*/, + size_t /*num_streams*/) +{ + storage_snapshot->check(column_names_); + const ColumnsDescription & our_columns = storage_snapshot->getDescriptionForColumns(column_names_); + Block block_header; + for (const auto & name : column_names_) + { + const auto & name_type = our_columns.get(name); + MutableColumnPtr column = name_type.type->createColumn(); + block_header.insert({std::move(column), name_type.type, name_type.name}); + } + + return Pipe{std::make_shared(block_header, MySQLReplication::BinlogClientFactory::instance().getMetadata())}; +} + +} diff --git a/src/Storages/System/StorageSystemMySQLBinlogs.h b/src/Storages/System/StorageSystemMySQLBinlogs.h new file mode 100644 index 000000000000..a627137f4952 --- /dev/null +++ b/src/Storages/System/StorageSystemMySQLBinlogs.h @@ -0,0 +1,29 @@ +#pragma once + +#include + +namespace DB +{ + +class StorageSystemMySQLBinlogs final : public IStorage +{ +public: + explicit StorageSystemMySQLBinlogs(const StorageID & storage_id_); + + std::string getName() const override { return "MySQLBinlogs"; } + + Pipe read( + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + size_t num_streams) override; + + bool isSystemStorage() const override { return true; } + + static NamesAndTypesList getNamesAndTypes(); +}; + +} diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index 01a41fc4b407..3ebe2a2d9634 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -106,6 +106,10 @@ #include #endif +#if USE_MYSQL +#include +#endif + namespace DB { @@ -172,6 +176,9 @@ void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, b #if USE_ROCKSDB attach(context, system_database, "rocksdb"); #endif +#if USE_MYSQL + attach(context, system_database, "mysql_binlogs"); +#endif attach(context, system_database, "parts"); attach(context, system_database, "projection_parts"); diff --git a/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py b/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py index 286a2d295418..97c8b65f15dc 100644 --- a/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py +++ b/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py @@ -2714,3 +2714,698 @@ def table_with_indexes(clickhouse_node, mysql_node, service_name): mysql_node.query(f"DROP DATABASE IF EXISTS {db}") clickhouse_node.query(f"DROP DATABASE IF EXISTS {db}") + + +def binlog_client_test(clickhouse_node, mysql_node, replication): + db = "binlog_client_test" + replication.create_db_mysql(db) + + mysql_node.query( + f"CREATE TABLE {db}.t(id INT PRIMARY KEY AUTO_INCREMENT, score int, create_time DATETIME DEFAULT NOW())" + ) + replication.insert_data(db, "t", 100000, column="score") + replication.create_db_ch(f"{db}1", from_mysql_db=db, settings="use_binlog_client=1") + check_query( + clickhouse_node, + f"SHOW TABLES FROM {db}1 FORMAT TSV", + "t\n", + ) + + replication.insert_data(db, "t", 100000, column="score") + + num_rows = replication.inserted_rows + check_query( + clickhouse_node, + f"/* expect: {num_rows} */ SELECT count() FROM {db}1.t", + f"{num_rows}\n", + interval_seconds=1, + retry_count=30, + ) + + replication.create_db_ch(f"{db}2", from_mysql_db=db, settings="use_binlog_client=1") + check_query( + clickhouse_node, + f"SHOW TABLES FROM {db}2 FORMAT TSV", + "t\n", + ) + + replication.insert_data(db, "t", 100000, column="score") + num_rows = replication.inserted_rows + + check_query( + clickhouse_node, + f"/* expect: 1 */ SELECT count() FROM system.mysql_binlogs WHERE name = '{db}1'", + "1\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: 1 */ SELECT count() FROM system.mysql_binlogs WHERE name = '{db}2'", + "1\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: {num_rows} */ SELECT count() FROM {db}1.t", + f"{num_rows}\n", + interval_seconds=1, + retry_count=60, + ) + check_query( + clickhouse_node, + f"/* expect: {num_rows} */ SELECT count() FROM {db}2.t", + f"{num_rows}\n", + interval_seconds=1, + retry_count=60, + ) + # Catch up + check_query( + clickhouse_node, + f"/* expect: 1 */ SELECT COUNT(DISTINCT(dispatcher_name)) FROM system.mysql_binlogs WHERE name LIKE '{db}%'", + "1\n", + interval_seconds=1, + retry_count=30, + ) + + replication.drop_dbs_ch() + replication.create_db_ch( + f"{db}1", + from_mysql_db=db, + settings="use_binlog_client=1, max_bytes_in_binlog_queue=10", + ) + replication.create_db_ch( + f"{db}2", + from_mysql_db=db, + settings="use_binlog_client=1, max_bytes_in_binlog_queue=10", + ) + replication.insert_data(db, "t", 10000, column="score") + check_query( + clickhouse_node, + f"SHOW TABLES FROM {db}1 FORMAT TSV", + "t\n", + ) + + replication.insert_data(db, "t", 100000, column="score") + check_query( + clickhouse_node, + f"SHOW TABLES FROM {db}2 FORMAT TSV", + "t\n", + ) + + replication.insert_data(db, "t", 10000, column="score") + + num_rows = replication.inserted_rows + + check_query( + clickhouse_node, + f"/* expect: 1 */ SELECT count() FROM system.mysql_binlogs WHERE name = '{db}1'", + "1\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: 1 */ SELECT count() FROM system.mysql_binlogs WHERE name = '{db}2'", + "1\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: {num_rows} */ SELECT count() FROM {db}1.t", + f"{num_rows}\n", + interval_seconds=1, + retry_count=60, + ) + check_query( + clickhouse_node, + f"/* expect: {num_rows} */ SELECT count() FROM {db}2.t", + f"{num_rows}\n", + interval_seconds=1, + retry_count=60, + ) + check_query( + clickhouse_node, + f"/* expect: 1 */ SELECT COUNT(DISTINCT(dispatcher_name)) FROM system.mysql_binlogs WHERE name LIKE '{db}%'", + "1\n", + interval_seconds=1, + retry_count=30, + ) + + replication.create_db_ch( + f"{db}3", + from_mysql_db=db, + settings="use_binlog_client=1", + ) + + mysql_node.query(f"UPDATE {db}.t SET score = score + 1") + + check_query( + clickhouse_node, + f"/* expect: 1 */ SELECT COUNT(DISTINCT(dispatcher_name)) FROM system.mysql_binlogs WHERE name LIKE '{db}%'", + "1\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: 0 */ SELECT size FROM system.mysql_binlogs WHERE name = '{db}1'", + "0\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: 0 */ SELECT size FROM system.mysql_binlogs WHERE name = '{db}2'", + "0\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: 0 */ SELECT size FROM system.mysql_binlogs WHERE name = '{db}3'", + "0\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: {num_rows} */ SELECT count() FROM {db}1.t", + f"{num_rows}\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: {num_rows} */ SELECT count() FROM {db}2.t", + f"{num_rows}\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: {num_rows} */ SELECT count() FROM {db}3.t", + f"{num_rows}\n", + interval_seconds=1, + retry_count=30, + ) + + mysql_crc32 = mysql_node.query_and_get_data( + f"SELECT bit_xor(cast(crc32(concat(id, score, create_time)) AS unsigned)) AS checksum FROM {db}.t" + )[0][0] + column = "bit_xor(cast(crc32(concat(toString(assumeNotNull(id)), toString(assumeNotNull(score)), toString(assumeNotNull(create_time)))) AS UInt32)) AS checksum" + check_query( + clickhouse_node, + f"/* expect: {mysql_crc32} */ SELECT {column} FROM {db}1.t", + f"{mysql_crc32}\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: {mysql_crc32} */ SELECT {column} FROM {db}2.t", + f"{mysql_crc32}\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: {mysql_crc32} */ SELECT {column} FROM {db}3.t", + f"{mysql_crc32}\n", + interval_seconds=1, + retry_count=30, + ) + + clickhouse_node.query(f"DROP DATABASE IF EXISTS {db}1") + check_query( + clickhouse_node, + f"/* expect: 0 */ SELECT COUNT() FROM system.mysql_binlogs WHERE name = '{db}1'", + "0\n", + interval_seconds=1, + retry_count=10, + ) + + +def binlog_client_timeout_test(clickhouse_node, mysql_node, replication): + db = "binlog_client_timeout_test" + replication.create_db_mysql(db) + mysql_node.query( + f"CREATE TABLE {db}.t(id INT PRIMARY KEY AUTO_INCREMENT, score int, create_time DATETIME DEFAULT NOW())" + ) + replication.insert_data(db, "t", 10000, column="score") + num_rows = replication.inserted_rows + + replication.create_db_ch( + f"{db}1", + from_mysql_db=db, + settings="use_binlog_client=1, max_bytes_in_binlog_queue=100000000, max_milliseconds_to_wait_in_binlog_queue=60000", + ) + replication.create_db_ch( + f"{db}2", + from_mysql_db=db, + settings="use_binlog_client=1, max_bytes_in_binlog_queue=10", + ) + replication.create_db_ch( + f"{db}3", + from_mysql_db=db, + settings="use_binlog_client=1, max_bytes_in_binlog_queue=10, max_milliseconds_to_wait_in_binlog_queue=100", + ) + replication.create_db_ch( + f"{db}4", + from_mysql_db=db, + settings="use_binlog_client=1, max_bytes_in_binlog_queue=10, max_milliseconds_to_wait_in_binlog_queue=10", + ) + + # After incremental sync + check_query( + clickhouse_node, + f"/* expect: 100000000, 60000 */ SELECT max_bytes, max_waiting_ms FROM system.mysql_binlogs WHERE name = '{db}1'", + f"100000000\t60000\n", + interval_seconds=1, + retry_count=10, + ) + check_query( + clickhouse_node, + f"/* expect: 10 */ SELECT max_bytes FROM system.mysql_binlogs WHERE name = '{db}2'", + f"10\n", + interval_seconds=2, + retry_count=10, + ) + check_query( + clickhouse_node, + f"/* expect: {num_rows} */ SELECT count() FROM {db}1.t", + f"{num_rows}\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: {num_rows} */ SELECT count() FROM {db}2.t", + f"{num_rows}\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: {num_rows} */ SELECT count() FROM {db}3.t", + f"{num_rows}\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: {num_rows} */ SELECT count() FROM {db}4.t", + f"{num_rows}\n", + interval_seconds=1, + retry_count=30, + ) + + clickhouse_node.query(f"DROP DATABASE {db}3") + replication.create_db_ch( + f"{db}3", + from_mysql_db=db, + settings="use_binlog_client=1, max_bytes_in_binlog_queue=10, max_milliseconds_to_wait_in_binlog_queue=10", + ) + check_query( + clickhouse_node, + f"SHOW TABLES FROM {db}3 FORMAT TSV", + "t\n", + ) + + clickhouse_node.query(f"DROP DATABASE {db}4") + replication.create_db_ch( + f"{db}4", + from_mysql_db=db, + settings="use_binlog_client=1, max_bytes_in_binlog_queue=10, max_milliseconds_to_wait_in_binlog_queue=50", + ) + check_query( + clickhouse_node, + f"SHOW TABLES FROM {db}4 FORMAT TSV", + "t\n", + ) + + mysql_node.query( + f"UPDATE {db}.t SET create_time='2021-01-01' WHERE id > 1000 AND id < 100000" + ) + mysql_node.query(f"UPDATE {db}.t SET create_time='2021-11-11' WHERE score > 1000") + mysql_node.query( + f"UPDATE {db}.t SET create_time=now() WHERE create_time='2021-01-01'" + ) + + check_query( + clickhouse_node, + f"/* expect: 0 */ SELECT COUNT() FROM {db}1.t WHERE toDate(create_time)='2021-01-01'", + "0\n", + interval_seconds=1, + retry_count=300, + ) + check_query( + clickhouse_node, + f"/* expect: 0 */ SELECT COUNT() FROM {db}2.t WHERE toDate(create_time)='2021-01-01'", + "0\n", + interval_seconds=1, + retry_count=300, + ) + check_query( + clickhouse_node, + f"/* expect: 0 */ SELECT COUNT() FROM {db}3.t WHERE toDate(create_time)='2021-01-01'", + "0\n", + interval_seconds=1, + retry_count=300, + ) + check_query( + clickhouse_node, + f"/* expect: 0 */ SELECT COUNT() FROM {db}4.t WHERE toDate(create_time)='2021-01-01'", + "0\n", + interval_seconds=1, + retry_count=300, + ) + check_query( + clickhouse_node, + f"/* expect: 0 */ SELECT size FROM system.mysql_binlogs WHERE name = '{db}1'", + "0\n", + interval_seconds=1, + retry_count=300, + ) + check_query( + clickhouse_node, + f"/* expect: 0 */ SELECT size FROM system.mysql_binlogs WHERE name = '{db}2'", + "0\n", + interval_seconds=1, + retry_count=300, + ) + check_query( + clickhouse_node, + f"/* expect: 0 */ SELECT size FROM system.mysql_binlogs WHERE name = '{db}3'", + "0\n", + interval_seconds=1, + retry_count=300, + ) + check_query( + clickhouse_node, + f"/* expect: 0 */ SELECT size FROM system.mysql_binlogs WHERE name = '{db}4'", + "0\n", + interval_seconds=1, + retry_count=300, + ) + check_query( + clickhouse_node, + f"/* expect: {num_rows} */ SELECT count() FROM {db}1.t", + f"{num_rows}\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: {num_rows} */ SELECT count() FROM {db}2.t", + f"{num_rows}\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: {num_rows} */ SELECT count() FROM {db}3.t", + f"{num_rows}\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: {num_rows} */ SELECT count() FROM {db}4.t", + f"{num_rows}\n", + interval_seconds=1, + retry_count=30, + ) + + mysql_crc32 = mysql_node.query_and_get_data( + f"SELECT bit_xor(cast(crc32(concat(id, score, create_time)) AS unsigned)) AS checksum FROM {db}.t" + )[0][0] + column = "bit_xor(cast(crc32(concat(toString(assumeNotNull(id)), toString(assumeNotNull(score)), toString(assumeNotNull(create_time)))) AS UInt32)) AS checksum" + check_query( + clickhouse_node, + f"/* expect: {mysql_crc32} */ SELECT {column} FROM {db}1.t", + f"{mysql_crc32}\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: {mysql_crc32} */ SELECT {column} FROM {db}2.t", + f"{mysql_crc32}\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: {mysql_crc32} */ SELECT {column} FROM {db}3.t", + f"{mysql_crc32}\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: {mysql_crc32} */ SELECT {column} FROM {db}4.t", + f"{mysql_crc32}\n", + interval_seconds=1, + retry_count=30, + ) + + +def wrong_password_test(clickhouse_node, mysql_node, replication): + db = "wrong_password_test" + replication.create_db_mysql(db) + mysql_node.query( + f"CREATE TABLE {db}.t(id INT PRIMARY KEY AUTO_INCREMENT, score int, create_time DATETIME DEFAULT NOW())" + ) + replication.insert_data(db, "t", 100, column="score") + with pytest.raises(Exception) as exc: + clickhouse_node.query( + f"CREATE DATABASE {db} ENGINE = MaterializedMySQL('{replication.mysql_host}:3306', '{db}', 'root', 'wrong_password') SETTINGS use_binlog_client=1" + ) + + replication.create_db_ch(db, settings="use_binlog_client=1") + check_query( + clickhouse_node, + f"SHOW TABLES FROM {db} FORMAT TSV", + "t\n", + ) + + replication.insert_data(db, "t", 100, column="score") + check_query( + clickhouse_node, + f"/* expect: 200 */ SELECT COUNT() FROM {db}.t ", + "200\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: root@{replication.mysql_host}:3306 */ SELECT binlog_client_name FROM system.mysql_binlogs WHERE name = '{db}'", + f"root@{replication.mysql_host}:3306\n", + interval_seconds=1, + retry_count=30, + ) + + +def dispatcher_buffer_test(clickhouse_node, mysql_node, replication): + db = "dispatcher_buffer_test" + replication.create_db_mysql(db) + mysql_node.query( + f"CREATE TABLE {db}.t(id INT PRIMARY KEY AUTO_INCREMENT, score int, create_time DATETIME DEFAULT NOW())" + ) + replication.insert_data(db, "t", 100, column="score") + rows_count = 100 + replication.create_db_ch( + db, + settings="use_binlog_client=1, max_bytes_in_binlog_dispatcher_buffer=0, max_flush_milliseconds_in_binlog_dispatcher=0", + ) + check_query( + clickhouse_node, + f"SHOW TABLES FROM {db} FORMAT TSV", + "t\n", + ) + + replication.insert_data(db, "t", 100000, column="score") + rows_count += 100000 + + mysql_node.query( + f"UPDATE {db}.t SET create_time='2021-01-01' WHERE id > 10000 AND id < 50000" + ) + mysql_node.query( + f"UPDATE {db}.t SET create_time=now() WHERE create_time='2021-01-01'" + ) + + mysql_crc32 = mysql_node.query_and_get_data( + f"SELECT bit_xor(cast(crc32(concat(id, score, create_time)) AS unsigned)) AS checksum FROM {db}.t" + )[0][0] + column = "bit_xor(cast(crc32(concat(toString(assumeNotNull(id)), toString(assumeNotNull(score)), toString(assumeNotNull(create_time)))) AS UInt32)) AS checksum" + check_query( + clickhouse_node, + f"/* expect: {mysql_crc32} */ SELECT {column} FROM {db}.t", + f"{mysql_crc32}\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: {rows_count} */ SELECT COUNT() FROM {db}.t", + f"{rows_count}\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: 0 */ SELECT COUNT() FROM {db}.t WHERE toDate(create_time)='2021-01-01'", + "0\n", + interval_seconds=1, + retry_count=30, + ) + + clickhouse_node.query(f"DROP DATABASE {db}") + replication.create_db_ch( + f"{db}", + from_mysql_db=db, + settings="use_binlog_client=1, max_bytes_in_binlog_dispatcher_buffer=1000, max_flush_milliseconds_in_binlog_dispatcher=1000", + ) + check_query( + clickhouse_node, + f"SHOW TABLES FROM {db} FORMAT TSV", + "t\n", + ) + + replication.insert_data(db, "t", 10000, column="score") + rows_count += 10000 + + mysql_node.query(f"UPDATE {db}.t SET create_time='2021-11-11' WHERE score > 10000") + mysql_node.query( + f"UPDATE {db}.t SET create_time='2021-01-01' WHERE id > 10000 AND id < 50000" + ) + mysql_node.query( + f"UPDATE {db}.t SET create_time=now() WHERE create_time='2021-01-01'" + ) + mysql_node.query( + f"UPDATE {db}.t SET create_time=now() WHERE create_time='2021-11-01'" + ) + + mysql_crc32 = mysql_node.query_and_get_data( + f"SELECT bit_xor(cast(crc32(concat(id, score, create_time)) AS unsigned)) AS checksum FROM {db}.t" + )[0][0] + check_query( + clickhouse_node, + f"/* expect: {mysql_crc32} */ SELECT {column} FROM {db}.t", + f"{mysql_crc32}\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: {rows_count} */ SELECT COUNT() FROM {db}.t", + f"{rows_count}\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: 0 */ SELECT COUNT() FROM {db}.t WHERE toDate(create_time)='2021-11-01'", + "0\n", + interval_seconds=1, + retry_count=30, + ) + + replication.create_db_ch( + db, + settings="use_binlog_client=1, max_bytes_in_binlog_dispatcher_buffer=100000000, max_flush_milliseconds_in_binlog_dispatcher=1000", + ) + check_query( + clickhouse_node, + f"SHOW TABLES FROM {db} FORMAT TSV", + "t\n", + ) + + replication.insert_data(db, "t", 100000, column="score") + rows_count += 100000 + + mysql_node.query(f"UPDATE {db}.t SET create_time='2021-11-11' WHERE score > 10000") + mysql_node.query( + f"UPDATE {db}.t SET create_time='2021-01-01' WHERE id > 10000 AND id < 50000" + ) + mysql_node.query( + f"UPDATE {db}.t SET create_time=now() WHERE create_time='2021-01-01'" + ) + mysql_node.query( + f"UPDATE {db}.t SET create_time=now() WHERE create_time='2021-11-01'" + ) + + check_query( + clickhouse_node, + f"/* expect: 1 */ SELECT COUNT() FROM system.mysql_binlogs WHERE name = '{db}' AND (dispatcher_events_read_per_sec > 0 OR dispatcher_bytes_read_per_sec > 0 OR dispatcher_events_flush_per_sec > 0 OR dispatcher_bytes_flush_per_sec > 0)", + f"1\n", + interval_seconds=1, + retry_count=30, + ) + + mysql_crc32 = mysql_node.query_and_get_data( + f"SELECT bit_xor(cast(crc32(concat(id, score, create_time)) AS unsigned)) AS checksum FROM {db}.t" + )[0][0] + check_query( + clickhouse_node, + f"/* expect: {mysql_crc32} */ SELECT {column} FROM {db}.t", + f"{mysql_crc32}\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: {rows_count} */ SELECT COUNT() FROM {db}.t", + f"{rows_count}\n", + interval_seconds=1, + retry_count=30, + ) + check_query( + clickhouse_node, + f"/* expect: 0 */ SELECT COUNT() FROM {db}.t WHERE toDate(create_time)='2021-11-01'", + "0\n", + interval_seconds=1, + retry_count=30, + ) + + +def gtid_after_attach_test(clickhouse_node, mysql_node, replication): + db = "gtid_after_attach_test" + replication.create_db_mysql(db) + mysql_node.query( + f"CREATE TABLE {db}.t(id INT PRIMARY KEY AUTO_INCREMENT, score int, create_time DATETIME DEFAULT NOW())" + ) + + db_count = 6 + for i in range(db_count): + replication.create_db_ch( + f"{db}{i}", + from_mysql_db=db, + settings="use_binlog_client=1", + ) + check_query( + clickhouse_node, + f"SHOW TABLES FROM {db}0 FORMAT TSV", + "t\n", + ) + for i in range(int(db_count / 2)): + clickhouse_node.query(f"DETACH DATABASE {db}{i}") + + mysql_node.query(f"USE {db}") + rows = 10000 + for i in range(100): + mysql_node.query(f"ALTER TABLE t ADD COLUMN (e{i} INT)") + replication.insert_data(db, "t", rows, column="score") + + clickhouse_node.restart_clickhouse(stop_start_wait_sec=120) + + check_query( + clickhouse_node, + f"/* expect: 1 */ SELECT COUNT(DISTINCT(dispatcher_name)) FROM system.mysql_binlogs WHERE name LIKE '{db}%'", + "1\n", + interval_seconds=1, + retry_count=300, + ) diff --git a/tests/integration/test_materialized_mysql_database/test.py b/tests/integration/test_materialized_mysql_database/test.py index f3369e25d945..727188a4b864 100644 --- a/tests/integration/test_materialized_mysql_database/test.py +++ b/tests/integration/test_materialized_mysql_database/test.py @@ -1,3 +1,4 @@ +import os import time import pymysql.cursors import pytest @@ -142,6 +143,145 @@ def clickhouse_node(): yield node_db +class ReplicationHelper: + def __init__(self, clickhouse, mysql, mysql_host=None): + self.clickhouse = clickhouse + self.mysql = mysql + self.created_mysql_dbs = [] + self.created_clickhouse_dbs = [] + self.base_mysql_settings = os.getenv("TEST_BASE_MYSQL_SETTINGS", "") + self.base_ch_settings = os.getenv("TEST_BASE_CH_SETTINGS", "") + self.mysql_host = mysql_host if mysql_host is not None else cluster.mysql8_host + self.created_insert_procedures = {} + self.inserted_rows_per_sp = {} + self.inserted_rows = 0 + + def create_dbs(self, db_name, ch_settings="", mysql_settings=""): + self.create_db_mysql(db_name, settings=mysql_settings) + self.create_db_ch(db_name, settings=ch_settings) + + def create_db_mysql(self, db_name, settings=""): + self.mysql.query(f"DROP DATABASE IF EXISTS {db_name}") + self.mysql.query( + f"CREATE DATABASE {db_name} {self.base_mysql_settings} {settings}" + ) + self.created_mysql_dbs.append(db_name) + + def create_db_ch( + self, db_name, from_mysql_db=None, settings="", table_overrides="" + ): + if from_mysql_db is None: + from_mysql_db = db_name + self.clickhouse.query(f"DROP DATABASE IF EXISTS {db_name}") + all_settings = "" + create_query = f"CREATE DATABASE {db_name} ENGINE = MaterializedMySQL('{self.mysql_host}:3306', '{from_mysql_db}', 'root', 'clickhouse')" + if self.base_ch_settings or settings: + separator = ", " if self.base_ch_settings and settings else "" + create_query += f" SETTINGS {self.base_ch_settings}{separator}{settings}" + if table_overrides: + create_query += f" {table_overrides}" + self.clickhouse.query(create_query) + self.created_clickhouse_dbs.append(db_name) + + def drop_dbs_mysql(self): + for db_name in self.created_mysql_dbs: + self.mysql.query(f"DROP DATABASE IF EXISTS {db_name}") + self.created_mysql_dbs = [] + self.created_insert_procedures = {} + self.inserted_rows_per_sp = {} + self.inserted_rows = 0 + + def drop_dbs_ch(self): + for db_name in self.created_clickhouse_dbs: + self.clickhouse.query(f"DROP DATABASE IF EXISTS {db_name}") + self.created_clickhouse_dbs = [] + + def drop_dbs(self): + self.drop_dbs_mysql() + self.drop_dbs_ch() + + def create_stored_procedure(self, db, table, column): + sp_id = f"{db}_{table}_{column}" + if sp_id in self.created_insert_procedures: + return sp_id + self.mysql.query(f"DROP PROCEDURE IF EXISTS {db}.insert_test_data_{sp_id}") + self.mysql.query( + f""" +CREATE PROCEDURE {db}.insert_test_data_{sp_id}(IN num_rows INT, IN existing_rows INT) +BEGIN + DECLARE i INT; + SET i = existing_rows; + SET @insert = concat("INSERT INTO {table} ({column}) VALUES "); + SET @exedata = ""; + WHILE i < (num_rows + existing_rows) DO + SET @exedata=concat(@exedata, ",(", i , ")"); + SET i = i + 1; + IF i % 1000 = 0 + THEN + SET @exedata = SUBSTRING(@exedata, 2); + SET @exesql = concat(@insert, @exedata); + PREPARE stmt FROM @exesql; + EXECUTE stmt; + DEALLOCATE PREPARE stmt; + SET @exedata = ""; + END IF; + END WHILE; + IF length(@exedata) > 0 + THEN + SET @exedata = SUBSTRING(@exedata, 2); + SET @exesql = concat(@insert, @exedata); + PREPARE stmt FROM @exesql; + EXECUTE stmt; + DEALLOCATE PREPARE stmt; + END IF; +END""" + ) + self.created_insert_procedures[sp_id] = True + self.inserted_rows_per_sp[sp_id] = 0 + return sp_id + + def insert_data(self, db, table, num_rows, column="id"): + """Inserts num_rows into db.table, into the column `column` (which must be INT)""" + sp_id = self.create_stored_procedure(db, table, column) + self.mysql.query( + f"CALL {db}.insert_test_data_{sp_id}({num_rows}, {self.inserted_rows_per_sp[sp_id]})" + ) + self.inserted_rows_per_sp[sp_id] += num_rows + self.inserted_rows += num_rows + + def wait_for_sync_to_catch_up( + self, database: str = "", retry_count=30, interval_seconds=1 + ): + if database == "": + database = self.created_clickhouse_dbs[-1] + mysql_gtid = self.mysql.query_and_get_data("SELECT @@GLOBAL.gtid_executed")[0][ + 0 + ] + materialized_with_ddl.check_query( + self.clickhouse, + f"SELECT executed_gtid_set /* expect: {mysql_gtid} */ FROM system.materialized_mysql_databases WHERE name = '{database}'", + f"{mysql_gtid}\n", + retry_count=retry_count, + interval_seconds=interval_seconds, + ) + + +@pytest.fixture(scope="function") +def replication(started_mysql_8_0, request): + try: + replication = ReplicationHelper(node_db, started_mysql_8_0) + yield replication + finally: + if hasattr(request.session, "testsfailed") and request.session.testsfailed: + logging.warning(f"tests failed - not dropping databases") + else: + # drop databases only if the test succeeds - so we can inspect the database after failed tests + try: + replication.drop_dbs() + except Exception as e: + logging.warning(f"replication.drop_dbs() failed: {e}") + + def test_materialized_database_dml_with_mysql_5_7( started_cluster, started_mysql_5_7, clickhouse_node: ClickHouseInstance ): @@ -556,3 +696,21 @@ def test_table_with_indexes(started_cluster, started_mysql_8_0, clickhouse_node) materialized_with_ddl.table_with_indexes( clickhouse_node, started_mysql_8_0, "mysql80" ) + + +def test_binlog_client(started_cluster, started_mysql_8_0, replication): + materialized_with_ddl.binlog_client_test(node_db, started_mysql_8_0, replication) + replication.drop_dbs() + materialized_with_ddl.binlog_client_timeout_test( + node_db, started_mysql_8_0, replication + ) + replication.drop_dbs() + materialized_with_ddl.wrong_password_test(node_db, started_mysql_8_0, replication) + replication.drop_dbs() + materialized_with_ddl.dispatcher_buffer_test( + node_db, started_mysql_8_0, replication + ) + replication.drop_dbs() + materialized_with_ddl.gtid_after_attach_test( + node_db, started_mysql_8_0, replication + ) diff --git a/utils/check-mysql-binlog/main.cpp b/utils/check-mysql-binlog/main.cpp index d1f868eba976..484dd46a90cc 100644 --- a/utils/check-mysql-binlog/main.cpp +++ b/utils/check-mysql-binlog/main.cpp @@ -1,173 +1,98 @@ -#include - +#include +#include #include +#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include +bool quit = false; +void signal_handler(int) +{ + quit = true; +} -static DB::MySQLCharsetPtr charset = std::make_shared(); -static DB::MySQLReplication::BinlogEventPtr parseSingleEventBody( - DB::MySQLReplication::EventHeader & header, DB::ReadBuffer & payload, - std::shared_ptr & last_table_map_event, bool exist_checksum) +static void processBinlogFromFile(const std::string & bin_path, bool disable_checksum) { - DB::MySQLReplication::BinlogEventPtr event; - DB::ReadBufferPtr limit_read_buffer = std::make_shared(payload, header.event_size - 19, - /* trow_exception */ false, /* exact_limit */ std::nullopt); - DB::ReadBufferPtr event_payload = std::make_shared(*limit_read_buffer, exist_checksum ? 4 : 0); + DB::MySQLReplication::BinlogFromFile binlog; + binlog.open(bin_path); + binlog.setChecksum(disable_checksum ? DB::MySQLReplication::IBinlog::NONE : DB::MySQLReplication::IBinlog::CRC32); - switch (header.type) + DB::MySQLReplication::BinlogEventPtr event; + while (binlog.tryReadEvent(event, /*timeout*/ 0) && !quit) { - case DB::MySQLReplication::FORMAT_DESCRIPTION_EVENT: - { - event = std::make_shared(std::move(header)); - event->parseEvent(*event_payload); - break; - } - case DB::MySQLReplication::ROTATE_EVENT: - { - event = std::make_shared(std::move(header)); - event->parseEvent(*event_payload); - break; - } - case DB::MySQLReplication::QUERY_EVENT: - { - event = std::make_shared(std::move(header)); - event->parseEvent(*event_payload); - - auto query = std::static_pointer_cast(event); - switch (query->typ) - { - case DB::MySQLReplication::QUERY_EVENT_MULTI_TXN_FLAG: - case DB::MySQLReplication::QUERY_EVENT_XA: - { - event = std::make_shared(std::move(query->header)); - break; - } - default: - break; - } - break; - } - case DB::MySQLReplication::XID_EVENT: - { - event = std::make_shared(std::move(header)); - event->parseEvent(*event_payload); - break; - } - case DB::MySQLReplication::TABLE_MAP_EVENT: - { - DB::MySQLReplication::TableMapEventHeader map_event_header; - map_event_header.parse(*event_payload); - event = std::make_shared(std::move(header), map_event_header, charset); - event->parseEvent(*event_payload); - last_table_map_event = std::static_pointer_cast(event); - break; - } - case DB::MySQLReplication::WRITE_ROWS_EVENT_V1: - case DB::MySQLReplication::WRITE_ROWS_EVENT_V2: - { - DB::MySQLReplication::RowsEventHeader rows_header(header.type); - rows_header.parse(*event_payload); - event = std::make_shared(last_table_map_event, std::move(header), rows_header); - event->parseEvent(*event_payload); - break; - } - case DB::MySQLReplication::DELETE_ROWS_EVENT_V1: - case DB::MySQLReplication::DELETE_ROWS_EVENT_V2: - { - DB::MySQLReplication::RowsEventHeader rows_header(header.type); - rows_header.parse(*event_payload); - event = std::make_shared(last_table_map_event, std::move(header), rows_header); - event->parseEvent(*event_payload); - break; - } - case DB::MySQLReplication::UPDATE_ROWS_EVENT_V1: - case DB::MySQLReplication::UPDATE_ROWS_EVENT_V2: - { - DB::MySQLReplication::RowsEventHeader rows_header(header.type); - rows_header.parse(*event_payload); - event = std::make_shared(last_table_map_event, std::move(header), rows_header); - event->parseEvent(*event_payload); - break; - } - case DB::MySQLReplication::GTID_EVENT: - { - event = std::make_shared(std::move(header)); - event->parseEvent(*event_payload); - break; - } - default: - { - event = std::make_shared(std::move(header)); - event->parseEvent(*event_payload); - break; - } + DB::WriteBufferFromOStream cout(std::cout); + event->dump(cout); + binlog.getPosition().dump(cout); + cout.finalize(); } - - return event; } -static int checkBinLogFile(const std::string & bin_path, bool exist_checksum) +static void processBinlogFromSocket(const std::string & host, int port, const std::string & user, const std::string & password, const std::string & executed_gtid_set, bool disable_checksum) { - DB::ReadBufferFromFile in(bin_path); - DB::assertString("\xfe\x62\x69\x6e", in); /// magic number + DB::MySQLReplication::BinlogFromSocket binlog; + binlog.setChecksum(disable_checksum ? DB::MySQLReplication::IBinlog::NONE : DB::MySQLReplication::IBinlog::CRC32); - DB::MySQLReplication::BinlogEventPtr last_event; - std::shared_ptr last_header; - std::shared_ptr table_map; + binlog.connect(host, port, user, password); + binlog.start(/*unique number*/ 42, executed_gtid_set); + DB::MySQLReplication::BinlogEventPtr event; - try + while (!quit) { - while (!in.eof()) + if (binlog.tryReadEvent(event, /*timeout*/ 100)) { - last_header = std::make_shared(); - last_header->parse(in); - last_event = parseSingleEventBody(*last_header, in, table_map, exist_checksum); + if (event->header.type != DB::MySQLReplication::HEARTBEAT_EVENT) + { + DB::WriteBufferFromOStream cout(std::cout); + event->dump(cout); + binlog.getPosition().dump(cout); + cout.finalize(); + } } } - catch (...) - { - DB::WriteBufferFromOStream cerr(std::cerr); - cerr << "Unable to parse MySQL binlog event. Code: " << DB::getCurrentExceptionCode() << ", Exception message: " - << DB::getCurrentExceptionMessage(false) << '\n' << ", Previous event: " << '\n'; - last_event->dump(cerr); - cerr << '\n' << ", Event header: " << '\n'; - last_header->dump(cerr); - cerr << '\n'; - return DB::getCurrentExceptionCode(); - } - - DB::WriteBufferFromOStream cout(std::cout); - cout << "Check passed. " << '\n' << "No exception was thrown." << '\n' << "The last binlog event: " << '\n'; - last_event->dump(cout); - cout << '\n'; - return 0; } - int main(int argc, char ** argv) { + (void)signal(SIGINT, signal_handler); boost::program_options::options_description desc("Allowed options"); - desc.add_options()("help,h", "Produce help message"); - desc.add_options()("disable_checksum", "Disable checksums in binlog files."); - boost::program_options::variables_map options; - boost::program_options::store(boost::program_options::parse_command_line(argc, argv, desc), options); + std::string host = "127.0.0.1"; + int port = 3306; + std::string user = "root"; + std::string password; + std::string gtid; + + desc.add_options() + ("help", "Produce help message") + ("disable_checksum", "Disable checksums in binlog files.") + ("binlog", boost::program_options::value(), "Binlog file") + ("host", boost::program_options::value(&host)->default_value(host), "Host to connect") + ("port", boost::program_options::value(&port)->default_value(port), "Port number to connect") + ("user", boost::program_options::value(&user)->default_value(user), "User") + ("password", boost::program_options::value(&password), "Password") + ("gtid", boost::program_options::value(>id), "Executed gtid set"); + + try + { + boost::program_options::variables_map options; + boost::program_options::store(boost::program_options::parse_command_line(argc, argv, desc), options); + boost::program_options::notify(options); + + if (options.count("help") || (!options.count("binlog") && !options.count("gtid"))) + { + std::cout << "Usage: " << argv[0] << std::endl; + std::cout << desc << std::endl; + return EXIT_FAILURE; + } - if (options.count("help") || argc < 2) + if (options.count("binlog")) + processBinlogFromFile(options["binlog"].as(), options.count("disable_checksum")); + else + processBinlogFromSocket(host, port, user, password, gtid, options.count("disable_checksum")); + } + catch (std::exception & ex) { - std::cout << "Usage: " << argv[0] << " mysql_binlog_file" << std::endl; - std::cout << desc << std::endl; - return 1; + std::cerr << ex.what() << std::endl; + return EXIT_FAILURE; } - return checkBinLogFile(argv[argc - 1], !options.count("disable_checksum")); + return EXIT_SUCCESS; }