diff --git a/Docs/replication/binlog.md b/Docs/replication/binlog.md
new file mode 100644
index 0000000000000..cea6ac9e7e8b2
--- /dev/null
+++ b/Docs/replication/binlog.md
@@ -0,0 +1,151 @@
+# New binlog implementation
+
+This document describes the new binlog implementation that is enabled using
+the `--binlog-storage-engine` option.
+
+The new binlog uses a more efficient on-disk format that is integrated with
+the InnoDB write-ahead log. This provides two main benefits:
+
+1. The binlog will always be recovered into a consistent state after a crash. This makes it possible to use the options `--innodb-flush-log-at-trx-commit=0` or `--innodb-flush-log-at-trx-commit=2`, which can give a huge performance improvement depending on disk speed and transaction concurrency.
+2. When using the option `--innodb-flush-log-at-trx-commit=1`, commits are more efficient since there is no expensive two-phase commit between the binlog and the InnoDB storage engine.
+
+## Using the new binlog
+
+To use the new binlog, configure the server with the following two options:
+1. `log_bin`
+2. `binlog_storage_engine=innodb`
+
+Note that the `log_bin` option must be specified like that, without any argument; the option is not an on-off-switch.
+
+Optionally, the directory in which to store binlog files can be specified with `binlog_directory=
`. By default, the data directory is used.
+
+Note that using the new binlog is mutually exclusive with the traditional binlog format. Configuring an existing server to use the new binlog format will effectively ignore any old binlog files. This limitation may be relaxed in a future version of MariaDB.
+
+## Replicating with the new binlog
+
+Configuration of replication from a master using the new binlog is done in the usual way. Slaves must be configured to use Global Transaction ID (GTID) to connect to the master (this is the default). The old filename/offset-based replication position is not available when using the new binlog implementation on the master.
+
+## Working with the binlog files
+
+The binlog files will be written to the data directory (or to the directory configured with `--binlog-directory`). The files are named `binlog-000000.ibb`, `binlog-000001.ibb`, ... and so on.
+
+The size of each binlog file is determined by the value of `max_binlog_size` (by default 1 GB). The binlog files are pre-allocated, so they will always have the configured size, with the last one or two files being possibly partially empty. The exception is when the command `FLUSH BINARY LOGS` is used; then the last active binlog file will be truncated to the used part of it, and binlog writes will continue in the following file.
+
+The list of current binlog files can be obtained with the command `SHOW BINLOG EVENTS`. Note that there is no binlog index file (`.index`) like with the traditional binlog format, nor are there any GTID index files (`.idx`) or GTID state (`.state`) file (`.state`).
+
+Instead of the GTID index and state files, the new binlog periodically writes GTID *state records* into the binlog containing the equivalent information. When a slave connects to the master, as well as when the server starts up, the binlog will be scanned from the last GTID state record to find or recover the correct GTID position. The `--innodb-binlog-state-interval` configuration option controls the interval (in bytes) between writing a state record. Thus, this option can be increased to reduce the overhead of state records, or decreased to speed up finding the initial GTID position at slave connect. The overhead however is small either way, and normally there is little reason to change the default. The status variables `binlog_gtid_index_hit` and `binlog_gtid_index_miss` are not used with the new binlog implementation.
+
+Binlog files can be purged (removed) automatically after a configured time or disk space usage, provided they are no longer needed by active replication slaves or for possible crash recovery. This is configured using the options `binlog_expire_log_seconds`, `binlog_expire_log_days`, `max_binlog_total_size`, and `slave_connections_needed_for_purge`.
+
+The contents of binlog files can be inspected in two ways:
+1. From within the server, using the command `SHOW BINLOG EVENTS`.
+2. Independent of the server, using the `mariadb-binlog` command-line program.
+
+Unlike in the traditional binlog format, one binlog event can be stored in multiple different binlog files, and different parts of individual events can be interleaved with one another in each file. The `mariadb-binlog` program will coalesce the different parts of each event as necessary, so the output of the program is a consistent, non-interleaved stream of events. To obtain a correct seequnce of events across multiple binlog files, all binlog files should be passed to the `mariadb-binlog` program at once in correct order; this ensures that events that cross file boundaries are included in the output exactly once.
+
+When using the `--start-position` and `--stop-position` options of `mariadb-binlog`, it is recommended to use GTID positions. The event file offsets used in the tranditional binlog format are not used in the new binlog, and will mostly be reported as zero.
+
+The `--binlog-checksum` option is no longer used with the new binlog implementation. The binlog files are always checksummed, with a CRC32 at the end of each page. To have checksum of the data sent on the network between the master and the slave (in addition to the normal TCP checksums), use the `MASTER_SSL` option for `CHANGE MASTER` to make the connection use SSL.
+
+## Using the new binlog with mariadb-backup
+
+The `mariadb-backup` program will by default back up the binlog files together with the rest of the server data. This fixes a long-standing limitation of the old binlog that it is missing from backups made with `mariadb-backup`.
+
+The binlog files are backed up in a transactionally consistent way, just like other InnoDB data. This means that a restored backup can be used to setup a new slave simply by using the `MASTER_DEMOTE_TO_SLAVE=1` option of `CHANGE MASTER`.
+
+The server being backed up is not blocked during the copy of the binlog files; only `RESET MASTER`, `PURGE BINARY LOGS` and `FLUSH BINARY LOGS` are blocked by default. This blocking can be disabled with the option `--no-lock` option.
+
+To omit the binlog files from the backup (ie. to save space in the backup when the binlog files are known to be not needed), use the `--skip-binlog` option on both the `mariadb-backup --backup` and `mariadb-backup --prepare` step. Note that when binlog files are omitted from the backup, the restored server will behave as if `RESET MASTER` was run on it just at the point of the backup. Also note that any transactions that were prepared, but not yet committed, at the time of the backup will be rolled back when the restored server starts up for the first time.
+
+## `FLUSH BINARY LOGS`
+
+Binlog files are pre-allocated for efficiency. When binlog file N is filled up, any remainder event data continues in file N+1, and and empty file N+2 is pre-allocated in the background. This means that binlog files are always exactly `--max-binlog-size` bytes long; and if the server restarts, binlog writing continues at the point reached before shutdown.
+
+The exception is when the `FLUSH BINARY LOGS` command is run. This pads the current binlog file up to the next page boundary, truncates the file, and switches to the next file. This can thus leave the binlog file smaller than `--max-binlog-size` (but always a multiple of the binlog page size).
+
+The `FLUSH BINARY LOGS DELETE_DOMAIN_ID=N` can be used to remove an old domain id from the `@@gtid_binlog_pos`. This requires that the domain is not in use in any existing binlog files; a combination of running `FLUSH BINARY LOGS` and `PURGE BINARY LOGS TO` can help ensure this. If the domain id N is already deleted, a warning is issues but the `FLUSH BINARY LOGS` operation is still run (this is for consistency, but is different from the old binlog implementation, where the `FLUSH` is skipped if the domain id was already deleted.
+
+## Upgrading
+
+When switching an existing server to use the new binlog format, the old binlog files will not be available after the switch, as the two formats are mutually exclusive.
+
+If the old binlog files are not needed after the transition, no special actions are needed. Just stop the server and restart with the configuration `--log-bin --binlog-storage-engine=innodb`. The new binlog will start empty. The old binlog files can be removed manually afterwards.
+
+Optionally, note down the value of `@@binlog_gtid_state` and execute `SET GLOBAL binlog_gtid_state=` as the first thing after starting up, to preserve the GTID state. This can be used to migrate a replication setup. First stop all writes to the master, and let all slaves catch up. Then note down the value of `@@binlog_gtid_state`, restart the master with `--binlog-storage-engine=innodb`, and restore `@@binlog_gtid_state`. Then the slaves will be able to connect and continue from where they left off.
+
+Alternatively, live migration can be done by switching a slave first. Restart a slave with the new `binlog-storage-engine=innodb` option and let the slave replicate for a while until it has sufficient binlog data in the new binlog format. Then promote the slave as the new master. The other slaves can then be stopped, switched to the new binlog, and restarted, as convenient.
+
+When using the new binlog format for a new installation, nothing special is needed. Just configure `--binlog-storage-engine=innodb` on the new server installation.
+
+When the new binlog format is enabled on a master, the slaves should be upgraded to at least MariaDB version 12.3 first. The slave can be switched to the new binlog format without upgrading the master first. The master and slave can use the old or the new binlog format independently of one another.
+
+## Using the new binlog with 3rd-party programs
+
+The new binlog uses a different on-disk format than the traditional binlog. The format of individual replication events is the same; however the files stored on disk are page-based, and each page has some encapsulation of event data to support splitting events in multiple pieces etc.
+
+This means that existing 3rd-party programs that read the binlog files directly will need to be modified to support the new format. Until then, such programs will require using the traditional binlog format.
+
+The protocol for reading binlog data from a running server (eg. for a connecting slave) is however mostly unchanged. This means existing programs that read binlog events from a running server may be able to function unmodified with the new binlog. Similarly, `mariadb-binlog` with the `--read-from-remote-server` option works as usual.
+
+A difference is that file offsets and file bondaries are no longer meaningful and no longer reported to the connecting client. There are no rotate events at the end of a file to specify the name of the following file, nor is there a new format description event at the start of each new file. Effectively, the binlog appears as a single unbroken stream of events to clients. The position from which to start receiving binlog events from the server should be specified using a GTID position; specifying a filename and file offset is not available.
+
+### Documentation of the binlog file format
+
+A binlog file consists of a sequence of pages. The page size is currently fixed at 16kByte. The size of a binlog file is set with the `--max-binlog-size` option. Each page has a CRC32 in the last 4 bytes; all remaining bytes are used for data.
+
+Numbers are stored in little-endian format. Some numbers are stored as compressed integers. A compressed integer consists of 1-9 bytes. The lower 3 bits determine the number of bytes used. The number of bytes is one more than the value in the lower 3 bits, except that a value of 7 means that 9 bytes are used. The value of the number stored is the little-endian value of the used bytes, right-shifted by 3.
+
+The first page in each binlog file is a file header page, with the following format:
+
+| Offset | Size | Description |
+| -----: | ---: | :---------- |
+| 0 | 4 | 4-byte MAGIC value 0x010dfefe to identify the file as a binlog file. |
+| 4 | 4 | The log-2 of the page size, currently fixed at 14 for a 16kByte page size. |
+| 8 | 4 | Major file version, currently 1. A new major version is not readable by older server versions. |
+| 12 | 4 | Minor file version, currently 0. New minor versions are backwards-compatible with older server versions. |
+| 16 | 8 | The file number (same as the number in the `binlog-NNNNNN.ibb` file name), for consistency check. |
+| 24 | 8 | The size of the file, in pages. |
+| 32 | 8 | The InnoDB LSN corresponding to the start of the file, used for crash recovery. |
+| 40 | 8 | The value of `--innodb-binlog-state-interval` used in the file. |
+| 48 | 8 | The file number of the earliest file into which this file may contain references into (such references occur when a large event group is split into multiple pieces, called out-of-band or oob records, and are used to locate all the pieces of event data from the final commit record). Used to prevent purging binlog files that contain data that may still be needed. |
+| 56 | 8 | The file number of the earliest file containing pending XA transactions that may still be active. |
+| 64 | 448 | Unused. |
+| 512 | 4 | Extra CRC32. This is used for future expansion to support configurable binlog page size. The header page can be read and checksummed as a 512-byte page, after which the real page size can be determined from the value at offset 4. |
+
+Remaining pages in the file are data pages. Data is structured as a sequence of *records*; each record consists of one or more chunks. A page contains one or more chunks; a record can span multiple pages, but a chunk always fits within one page. Chunks are a minumum of 4 bytes long; any remaining 1-3 bytes of data in a page are filled with the byte 0xff. Unused bytes in a page are set to 0x00.
+
+A chunk consists of a *type byte*, two *length bytes* (little endian), and the *data bytes*. The length bytes count only the data bytes, so if the length bytes are 0x0001, then the total size of the chunk is 4 bytes.
+
+The high two bits of the type byte are used to collect chunks into records:
+- Bit 7 is clear for the first chunk in a record, and set for all following chunks in the record.
+- Bit 6 is set for the last chunk in a record, and clear for any prior chunks.
+
+These are the chunk types used:
+
+| Type | Description |
+| ---: | :---------- |
+| 0 | (not a real type, 0 is an unused byte and denotes end-of-file in the current binlog file). |
+| 1 | A commit record, containing binlog event data. First a compressed integer of the number of oob records referenced by the commit record, if any; then if non-zero, four more compressed integers of the file number and offset of the first and last such reference. This is followed by another similar 1 or 5 compressed integers, only used in the special case where transactional and non-transactional updates are mixed in a single event group. The remainder bytes are the payload data. |
+| 2 | This is a GTID state record, written every `--innodb-binlog-state-interval` bytes. It consists of a sequence of compressed integers. The first is the number of GTIDs in the GTID state stored in the record. The second is one more than the earliest file number containing possibly still active XA transactions (used for crash recovery), or 0 for none. After this comes N*3 compressed integers, each representing a GTID in the GTID state. |
+| 3 | This is an out-of-band (oob) data record. It is used to split large event groups into smaller pieces, organized as a forest of perfect binary trees. The record starts with 5 compressed integers: A node index (starts at 0 and increments for each oob record in an event group); the file number and offset of the left child oob node; and the file number and offset of the right child oob node. Remainder bytes are the payload event data. |
+| 4 | This is a filler record, it is used to pad out the last page of a binlog file which is truncated due to `FLUSH BINARY LOGS`. |
+| 5 | This is an XA PREPARE record, used for consistent crash recovery of user XA transactions. It starts with 1 byte counting the number of storage engines participating in the transaction. Then follows the XID (4 byte formatID; 1 byte gtrid length; 1 byte bqual length; and the characters of the XID name). Finally 5 compressed integers: the number of referenced oob nodes; the file number and offset of the first one; and the file number and offset of the last one. |
+| 6 | This is an XA complete record, used for recovery purposes for internal 2-phase commit transactions and user XA. The first byte is a type byte, which is 0 for commit and 1 for XA rollback. Then follows 6-134 bytes of the XID, in the same format as for the XA PREPARE record. |
+
+## Not supported
+
+A few things are not supported with the new binlog implementation. Some of
+these should be supported in a later version of MariaDB. Some of these are
+legacy stuff that fundamentally works poorly or is otherwise undesirable,
+and are intentionally removed in the new binlog implementation.
+
+- Old-style filename/offset replication positions are not available with the new binlog. Slaves must be configured to use GTID (this is the default). Event offsets are generally reported as zero. `MASTER_POS_WAIT()` is not available, `MASTER_GTID_WAIT()` should be used instead. Similarly, `BINLOG_GTID_POS()` is not available.
+- Using savepoints inside triggers is not supported. This is because of bugs and inconsistencies like in MDEV38465. Now executing a `SAVEPOINT` or `ROLLBACK TO SAVEPOINT` statement in a trigger will consistently error and roll back the entire statement.
+- Semi-synchronous replication is not supported in the first version. It will be supported as normal eventually using the `AFTER_COMMIT` option. The `AFTER_SYNC` option cannot be supported, as the expensive two-phase commit between binlog and engine is no longer needed (`AFTER_SYNC` waits for slave acknowledgement in the middle of the two-phase commit). Likewise, `--init-rpl-role` is not supported.
+- The new binlog implementation cannot be used with Galera.
+- In the initial version, only InnoDB is available as an engine for the binlog (`--binlog-storage-engine=innodb`). It the future, other transactional storage engines could implement storing the binlog themselves (performance is best when the binlog is implemented in the same engine as the tables that are updated).
+- The `sync_binlog` option is no longer needed and is effectively ignored. Since the binlog files are now crash-safe without needing any syncing. The durability of commits is now controlled solely by the `--innodb-flush-log-at-trx-commit` option, which now applies to both binlog files and InnoDB table data.
+- The command `RESET MASTER TO` is not available with the new binlog.
+- The `--tc-heuristic-recover` option is not needed with the new binlog and cannot be used. Any pending prepared transactions will always be rolled back or committed to be consistent with the binlog. If the binlog is empty (ie. has been deleted manually), pending transactions will be rolled back.
+- Binlog encryption is not available. It is suggested to use filesystem-level encryption facilities of the operating system instead, and/or use SSL for the slave's connection to the master.
+- SHOW BINLOG EVENTS FROM will not give an error for a position that starts in the middle of an event group. Instead, it will start from the first GTID event following the position (or return empty, if the position is past the end).
diff --git a/client/CMakeLists.txt b/client/CMakeLists.txt
index b21402ad9305e..8c3718975cdb7 100644
--- a/client/CMakeLists.txt
+++ b/client/CMakeLists.txt
@@ -70,7 +70,7 @@ TARGET_LINK_LIBRARIES(mariadb-show ${CLIENT_LIB})
MYSQL_ADD_EXECUTABLE(mariadb-plugin mysql_plugin.c)
TARGET_LINK_LIBRARIES(mariadb-plugin ${CLIENT_LIB})
-MYSQL_ADD_EXECUTABLE(mariadb-binlog mysqlbinlog.cc)
+MYSQL_ADD_EXECUTABLE(mariadb-binlog mysqlbinlog.cc mysqlbinlog-engine.cc)
TARGET_LINK_LIBRARIES(mariadb-binlog ${CLIENT_LIB} mysys_ssl)
MYSQL_ADD_EXECUTABLE(mariadb-admin mysqladmin.cc ../sql/password.c)
diff --git a/client/mysqlbinlog-engine.cc b/client/mysqlbinlog-engine.cc
new file mode 100644
index 0000000000000..93829609a09b3
--- /dev/null
+++ b/client/mysqlbinlog-engine.cc
@@ -0,0 +1,1185 @@
+/* Copyright (c) 2025, Kristian Nielsen.
+
+ This program is free software; you can redistribute it and/or modify
+ it under the terms of the GNU General Public License as published by
+ the Free Software Foundation; version 2 of the License.
+
+ This program is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU General Public License for more details.
+
+ You should have received a copy of the GNU General Public License
+ along with this program; if not, write to the Free Software
+ Foundation, Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1335 USA */
+
+/*
+ Code for reading engine-implemented binlog from the mysqlbinlog client
+ program.
+*/
+
+#include
+#include
+#include
+#include "client_priv.h"
+#include "mysqlbinlog-engine.h"
+#include "my_compr_int.h"
+#include "my_dir.h"
+
+
+const char *INNODB_BINLOG_MAGIC= "\xfe\xfe\x0d\x01";
+static constexpr uint32_t INNODB_BINLOG_FILE_VERS_MAJOR= 1;
+
+static uint32_t binlog_page_size;
+
+/*
+ Some code here is copied from storage/innobase/handler/innodb_binlog.cc and
+ storage/innodb_binlog/fsp/fsp_binlog.cc and modified for use in the
+ mysqlbinlog command-line client.
+
+ Normally it is desirable to share code rather than copy/modify it, but
+ special considerations apply here:
+
+ - Normally, it is desirable to share the code so that modifications to the
+ logic are automatically kept in sync between the two use cases. However
+ in the case of the binlog, non-backwards compatible changes are highly
+ undesirable, and having a separate reader implementation in mysqlbinlog
+ is actually useful to detect any unintended or non-desirable changes to
+ the format that prevent old code from reading it. The binlog files
+ should remain readable to old mysqlbinlog versions if at all possible,
+ as well as to any other 3rd-party readers.
+
+ - The main purpose of the code inside InnoDB is to very efficiently allow
+ reading of binlog data concurrently with active writing threads and
+ concurrently with page fifo asynchroneous flushing. In contrast, the
+ purpose of the mysqlclient code is to have a simple stand-alone command
+ line reader of the files. These two use cases are sufficiently
+ different, and the code frameworks used for storage/innobase/ and
+ client/ likewise sufficiently different, that code-sharing seems more
+ troublesome than beneficial here.
+*/
+
+static constexpr uint32_t BINLOG_PAGE_SIZE_MAX= 65536;
+#define BINLOG_PAGE_DATA 0
+#define BINLOG_PAGE_CHECKSUM 4
+#define BINLOG_PAGE_DATA_END BINLOG_PAGE_CHECKSUM
+
+#define BINLOG_NAME_BASE "binlog-"
+#define BINLOG_NAME_EXT ".ibb"
+
+enum fsp_binlog_chunk_types {
+ /* Zero means no data, effectively EOF. */
+ FSP_BINLOG_TYPE_EMPTY= 0,
+ /* A binlogged committed event group. */
+ FSP_BINLOG_TYPE_COMMIT= 1,
+ /* A binlog GTID state record. */
+ FSP_BINLOG_TYPE_GTID_STATE= 2,
+ /* Out-of-band event group data. */
+ FSP_BINLOG_TYPE_OOB_DATA= 3,
+ /* Dummy record, use to fill remainder of page (eg. FLUSH BINARY LOGS). */
+ FSP_BINLOG_TYPE_DUMMY= 4,
+ /* Must be one more than the last type. */
+ FSP_BINLOG_TYPE_END,
+
+ /* Padding data at end of page. */
+ FSP_BINLOG_TYPE_FILLER= 0xff
+};
+static constexpr uint32_t FSP_BINLOG_FLAG_BIT_CONT= 7;
+static constexpr uint32_t FSP_BINLOG_FLAG_CONT= (1 << FSP_BINLOG_FLAG_BIT_CONT);
+static constexpr uint32_t FSP_BINLOG_FLAG_BIT_LAST= 6;
+static constexpr uint32_t FSP_BINLOG_FLAG_LAST= (1 << FSP_BINLOG_FLAG_BIT_LAST);
+static constexpr uint32_t FSP_BINLOG_TYPE_MASK=
+ ~(FSP_BINLOG_FLAG_CONT | FSP_BINLOG_FLAG_LAST);
+static constexpr uint64_t ALLOWED_NESTED_RECORDS=
+ /* GTID STATE at start of page can occur in the middle of other record. */
+ ((uint64_t)1 << FSP_BINLOG_TYPE_GTID_STATE) |
+ /* DUMMY data at tablespace end can occur in the middle of other record. */
+ ((uint64_t)1 << FSP_BINLOG_TYPE_DUMMY)
+ ;
+
+
+static char binlog_dir[FN_REFLEN + 1];
+
+
+class chunk_reader_mysqlbinlog {
+public:
+ enum chunk_reader_status {
+ CHUNK_READER_ERROR= -1,
+ CHUNK_READER_EOF= 0,
+ CHUNK_READER_FOUND= 1
+ };
+
+ /*
+ Current state, can be obtained from save_pos() and later passed to
+ restore_pos().
+ */
+ struct saved_position {
+ /* Current position file. */
+ uint64_t file_no;
+ /* Current position page. */
+ uint32_t page_no;
+ /* Start of current chunk inside page. */
+ uint32_t in_page_offset;
+ /*
+ The length of the current chunk, once the chunk type has been read.
+ If 0, it means the chunk type (and length) has not yet been read.
+ */
+ uint32_t chunk_len;
+ /* The read position inside the current chunk. */
+ uint32_t chunk_read_offset;
+ uchar chunk_type;
+ /* When set, read will skip the current chunk, if any. */
+ bool skip_current;
+ /* Set while we are in the middle of reading a record. */
+ bool in_record;
+ } s;
+
+ /* Length of the currently open file, valid if cur_file_handle != -1. */
+ uint64_t cur_file_length;
+ /* Buffer for reading a page from a binlog file. */
+ uchar *page_buffer;
+ /* Open file handle to tablespace file_no, or -1. */
+ File cur_file_handle;
+ /*
+ Flag used to skip the rest of any partial chunk we might be starting in
+ the middle of.
+ */
+ bool skipping_partial;
+ /* If the s.file_no / s.page_no is loaded in the page buffer. */
+ bool page_loaded;
+
+ chunk_reader_mysqlbinlog();
+ void set_page_buf(uchar *in_page_buf) { page_buffer= in_page_buf; }
+ ~chunk_reader_mysqlbinlog();
+
+ /* Current type, or FSP_BINLOG_TYPE_FILLER if between records. */
+ uchar cur_type() { return (uchar)(s.chunk_type & FSP_BINLOG_TYPE_MASK); }
+ bool cur_is_cont() { return (s.chunk_type & FSP_BINLOG_FLAG_CONT) != 0; }
+ bool end_of_record() { return !s.in_record; }
+ bool is_end_of_page() noexcept
+ {
+ return s.in_page_offset >= binlog_page_size - (BINLOG_PAGE_DATA_END + 3);
+ }
+ bool is_end_of_file() noexcept
+ {
+ return current_pos() + (BINLOG_PAGE_DATA_END + 3) >= cur_file_length;
+ }
+ static int read_error_corruption(uint64_t file_no, uint64_t page_no,
+ const char *msg);
+ int read_error_corruption(const char *msg)
+ {
+ return read_error_corruption(s.file_no, s.page_no, msg);
+ }
+ enum chunk_reader_status fetch_current_page();
+ /*
+ Try to read max_len bytes from a record into buffer.
+
+ If multipage is true, will move across pages to read following
+ continuation chunks, if any, to try and read max_len total bytes. Only if
+ the record ends before max_len bytes is less amount of bytes returned.
+
+ If multipage is false, will read as much is available on one page (up to
+ max of max_len), and then return.
+
+ Returns number of bytes read, or -1 for error.
+ Returns 0 if the chunk_reader is pointing to start of a chunk at the end
+ of the current binlog (ie. end-of-file).
+ */
+ int read_data(uchar *buffer, int max_len, bool multipage);
+ /* Read the file header of current file_no. */
+ int parse_file_header();
+
+ /* Save current position, and restore it later. */
+ void save_pos(saved_position *out_pos) { *out_pos= s; }
+ void restore_pos(saved_position *pos);
+ void seek(uint64_t file_no, uint64_t offset);
+
+ /*
+ Make next read_data() skip any data from the current chunk (if any), and
+ start reading data only from the beginning of the next chunk. */
+ void skip_current() { if (s.in_record) s.skip_current= true; }
+ /*
+ Used initially, after seeking potentially into the middle of a (commit)
+ record, to skip any continuation chunks until we reach the start of the
+ first real record.
+ */
+ void skip_partial(bool skip) { skipping_partial= skip; }
+ uint64_t current_pos() {
+ return (s.page_no * binlog_page_size) + s.in_page_offset;
+ }
+ void set_fd(File fd);
+};
+
+
+class oob_reader_mysqlbinlog {
+ enum oob_states {
+ /* The initial state, about to visit the node for the first time. */
+ ST_initial,
+ /* State of leaf node while traversing the prior trees in the forest. */
+ ST_traversing_prior_trees,
+ /* State of non-leaf node while traversing its left sub-tree. */
+ ST_traversing_left_child,
+ /* State of non-leaf node while traversing its right sub-tree. */
+ ST_traversing_right_child,
+ /* State of node while reading out its data. */
+ ST_self
+ };
+
+ /*
+ Stack entry for one node currently taking part in post-order traversal.
+ We maintain a stack of pending nodes during the traversal, as the traversal
+ happens in a state machine rather than by recursion.
+ */
+ struct stack_entry {
+ /* Saved position after reading header. */
+ chunk_reader_mysqlbinlog::saved_position saved_pos;
+ /* The location of this node's OOB record. */
+ uint64_t file_no;
+ uint64_t offset;
+ /* Right child, to be traversed after left child. */
+ uint64_t right_file_no;
+ uint64_t right_offset;
+ /* Offset of real data in this node, after header. */
+ uint32_t header_len;
+ /* Amount of data read into rd_buf, and amount used to parse header. */
+ uint32_t rd_buf_len;
+ uint32_t rd_buf_sofar;
+ /* Current state in post-order traversal state machine. */
+ enum oob_states state;
+ /* Buffer for reading header. */
+ uchar rd_buf[5*COMPR_INT_MAX64];
+ /*
+ True when the node is reached using only left child pointers, false
+ otherwise. Used to identify the left-most leaf in a tree which points to
+ a prior tree that must be traversed first.
+ */
+ bool is_leftmost;
+ };
+ std::vectorstack;
+
+ /* State machine current state. */
+ enum oob_states state;
+
+public:
+ oob_reader_mysqlbinlog();
+ ~oob_reader_mysqlbinlog();
+
+ void start_traversal(uint64_t file_no, uint64_t offset);
+ bool oob_traversal_done() { return stack.empty(); }
+ int read_data(chunk_reader_mysqlbinlog *chunk_rd, uchar *buf, int max_len);
+
+private:
+ void push_state(enum oob_states state, uint64_t file_no, uint64_t offset,
+ bool is_leftmost);
+};
+
+
+class binlog_reader_innodb : public handler_binlog_reader {
+ enum reader_states {
+ ST_read_next_event_group, ST_read_oob_data, ST_read_commit_record
+ };
+
+ chunk_reader_mysqlbinlog chunk_rd;
+ oob_reader_mysqlbinlog oob_reader;
+ chunk_reader_mysqlbinlog::saved_position saved_commit_pos;
+
+ /* Out-of-band data to read after commit record, if any. */
+ uint64_t oob_count;
+ uint64_t oob_last_file_no;
+ uint64_t oob_last_offset;
+ /* Any secondary out-of-band data to be also read. */
+ uint64_t oob_count2;
+ uint64_t oob_last_file_no2;
+ uint64_t oob_last_offset2;
+ /*
+ The starting file_no. We stop once we've read the last record in this file
+ (which may span into the next file).
+ */
+ uint64_t start_file_no;
+ /* Buffer to hold a page read directly from the binlog file. */
+ uchar *page_buf;
+ /* Keep track of pending bytes in the rd_buf. */
+ uint32_t rd_buf_len;
+ uint32_t rd_buf_sofar;
+ /* State for state machine reading chunks one by one. */
+ enum reader_states state;
+
+ /* Used to read the header of the commit record. */
+ uchar rd_buf[5*COMPR_INT_MAX64];
+private:
+ int read_data(uchar *buf, uint32_t len);
+
+public:
+ binlog_reader_innodb();
+ virtual ~binlog_reader_innodb();
+ virtual int read_binlog_data(uchar *buf, uint32_t len) final;
+ virtual bool data_available() final;
+ virtual bool wait_available(THD *thd, const struct timespec *abstime) final;
+ virtual int init_gtid_pos(THD *thd, slave_connection_state *pos,
+ rpl_binlog_state_base *state) final;
+ virtual int init_legacy_pos(THD *thd, const char *filename,
+ ulonglong offset) final;
+ virtual void enable_single_file() final;
+ bool is_valid() { return page_buf != nullptr; }
+ bool init_from_fd_pos(File fd, ulonglong start_position);
+};
+
+
+static int
+read_page_mysqlbinlog(File fd, uchar *buf, uint32_t page_no) noexcept
+{
+ size_t read= my_pread(fd, buf, binlog_page_size,
+ (my_off_t)page_no * binlog_page_size, MYF(0));
+ int res= 1;
+ if (likely(read == binlog_page_size))
+ {
+ const uint32_t payload= (uint32_t)binlog_page_size - BINLOG_PAGE_CHECKSUM;
+ uint32_t crc32= uint4korr(buf + payload);
+ /* Allow a completely zero (empty) page as well. */
+ if (unlikely(crc32 != my_crc32c(0, buf, payload)) &&
+ (buf[0] != 0 || 0 != memcmp(buf, buf+1, binlog_page_size - 1)))
+ {
+ res= -1;
+ my_errno= EIO;
+ }
+ }
+ else if (read == (size_t)-1)
+ res= -1;
+ else
+ res= 0;
+
+ return res;
+}
+
+
+chunk_reader_mysqlbinlog::chunk_reader_mysqlbinlog()
+ : s { 0, 0, 0, 0, 0, FSP_BINLOG_TYPE_FILLER, false, false },
+ cur_file_length(0),
+ cur_file_handle((File)-1),
+ skipping_partial(false), page_loaded(false)
+{
+}
+
+
+chunk_reader_mysqlbinlog::~chunk_reader_mysqlbinlog()
+{
+ if (cur_file_handle >= (File)0)
+ my_close(cur_file_handle, MYF(0));
+}
+
+
+int
+chunk_reader_mysqlbinlog::read_error_corruption(uint64_t file_no, uint64_t page_no,
+ const char *msg)
+{
+ error("Corrupt InnoDB binlog found on page %" PRIu64 " in binlog number "
+ "%" PRIu64 ": %s", page_no, file_no, msg);
+ return -1;
+}
+
+
+int
+chunk_reader_mysqlbinlog::read_data(uchar *buffer, int max_len, bool multipage)
+{
+ uint32_t size;
+ int sofar= 0;
+
+read_more_data:
+ if (max_len == 0)
+ return sofar;
+
+ if (!page_loaded)
+ {
+ enum chunk_reader_status res= fetch_current_page();
+ if (res == CHUNK_READER_EOF)
+ return 0;
+ if (res == CHUNK_READER_ERROR)
+ return -1;
+ }
+
+ if (s.chunk_len == 0)
+ {
+ uchar type;
+ /*
+ This code gives warning "comparison of unsigned expression in ‘< 0’ is
+ always false" when BINLOG_PAGE_DATA is 0.
+
+ So use a static assert for now; if it ever triggers, replace it with this
+ code:
+
+ if (s.in_page_offset < BINLOG_PAGE_DATA)
+ s.in_page_offset= BINLOG_PAGE_DATA;
+ */
+ if (0)
+ static_assert(BINLOG_PAGE_DATA == 0,
+ "Replace static_assert with code from above comment");
+
+ /* Check for end-of-file. */
+ if ((s.page_no * binlog_page_size) + s.in_page_offset >= cur_file_length)
+ return sofar;
+
+ if (s.in_page_offset >= binlog_page_size - (BINLOG_PAGE_DATA_END + 3) ||
+ page_buffer[s.in_page_offset] == FSP_BINLOG_TYPE_FILLER)
+ {
+ DBUG_ASSERT(s.in_page_offset >= binlog_page_size - BINLOG_PAGE_DATA_END ||
+ page_buffer[s.in_page_offset] == FSP_BINLOG_TYPE_FILLER);
+ goto go_next_page;
+ }
+
+ type= page_buffer[s.in_page_offset];
+ if (type == 0)
+ return 0;
+
+ /*
+ Consistency check on the chunks. A record must consist in a sequence of
+ chunks of the same type, all but the first must have the
+ FSP_BINLOG_FLAG_BIT_CONT bit set, and the final one must have the
+ FSP_BINLOG_FLAG_BIT_LAST bit set.
+ */
+ if (!s.in_record)
+ {
+ if (type & FSP_BINLOG_FLAG_CONT && !s.skip_current)
+ {
+ if (skipping_partial)
+ {
+ s.chunk_len= page_buffer[s.in_page_offset + 1] |
+ ((uint32_t)page_buffer[s.in_page_offset + 2] << 8);
+ s.skip_current= true;
+ goto skip_chunk;
+ }
+ else
+ return read_error_corruption(s.file_no, s.page_no, "Binlog record "
+ "starts with continuation chunk");
+ }
+ }
+ else
+ {
+ if ((type ^ s.chunk_type) & FSP_BINLOG_TYPE_MASK)
+ {
+ /*
+ As a special case, we must allow a GTID state to appear in the
+ middle of a record.
+ */
+ if (((uint64_t)1 << (type & FSP_BINLOG_TYPE_MASK)) &
+ ALLOWED_NESTED_RECORDS)
+ {
+ s.chunk_len= page_buffer[s.in_page_offset + 1] |
+ ((uint32_t)page_buffer[s.in_page_offset + 2] << 8);
+ goto skip_chunk;
+ }
+ /* Chunk type changed in the middle. */
+ return read_error_corruption(s.file_no, s.page_no, "Binlog record missing "
+ "end chunk");
+ }
+ if (!(type & FSP_BINLOG_FLAG_CONT))
+ {
+ /* START chunk without END chunk. */
+ return read_error_corruption(s.file_no, s.page_no, "Binlog record missing "
+ "end chunk");
+ }
+ }
+
+ s.skip_current= false;
+ s.chunk_type= type;
+ s.in_record= true;
+ s.chunk_len= page_buffer[s.in_page_offset + 1] |
+ ((uint32_t)page_buffer[s.in_page_offset + 2] << 8);
+ s.chunk_read_offset= 0;
+ }
+
+ /* Now we have a chunk available to read data from. */
+ DBUG_ASSERT(s.chunk_read_offset < s.chunk_len);
+ if (s.skip_current &&
+ (s.chunk_read_offset > 0 || (s.chunk_type & FSP_BINLOG_FLAG_CONT)))
+ {
+ /*
+ Skip initial continuation chunks.
+ Used to be able to start reading potentially in the middle of a record,
+ ie. at a GTID state point.
+ */
+ s.chunk_read_offset= s.chunk_len;
+ }
+ else
+ {
+ size= std::min((uint32_t)max_len, s.chunk_len - s.chunk_read_offset);
+ memcpy(buffer, page_buffer + s.in_page_offset + 3 + s.chunk_read_offset, size);
+ buffer+= size;
+ s.chunk_read_offset+= size;
+ max_len-= size;
+ sofar+= size;
+ }
+
+ if (s.chunk_len > s.chunk_read_offset)
+ {
+ DBUG_ASSERT(max_len == 0 /* otherwise would have read more */);
+ return sofar;
+ }
+
+ /* We have read all of the chunk. Move to next chunk or end of the record. */
+skip_chunk:
+ s.in_page_offset+= 3 + s.chunk_len;
+ s.chunk_len= 0;
+ s.chunk_read_offset= 0;
+
+ if (s.chunk_type & FSP_BINLOG_FLAG_LAST)
+ {
+ s.in_record= false; /* End of record. */
+ s.skip_current= false;
+ }
+
+ if (s.in_page_offset >= binlog_page_size - (BINLOG_PAGE_DATA_END + 3) &&
+ (s.page_no * binlog_page_size) + s.in_page_offset < cur_file_length)
+ {
+go_next_page:
+ /* End of page reached, move to the next page. */
+ ++s.page_no;
+ page_loaded= false;
+ s.in_page_offset= 0;
+
+ if (cur_file_handle >= (File)0 &&
+ (s.page_no * binlog_page_size) >= cur_file_length)
+ {
+ /* Move to the next file. */
+ my_close(cur_file_handle, MYF(0));
+ cur_file_handle= (File)-1;
+ cur_file_length= ~(uint64_t)0;
+ ++s.file_no;
+ s.page_no= 1; /* Skip the header page. */
+ }
+ }
+
+ if (sofar > 0 && (!multipage || !s.in_record))
+ return sofar;
+
+ goto read_more_data;
+}
+
+
+oob_reader_mysqlbinlog::oob_reader_mysqlbinlog()
+{
+ /* Nothing. */
+}
+
+
+oob_reader_mysqlbinlog::~oob_reader_mysqlbinlog()
+{
+ /* Nothing. */
+}
+
+
+void
+oob_reader_mysqlbinlog::push_state(enum oob_states state, uint64_t file_no,
+ uint64_t offset, bool is_leftmost)
+{
+ stack_entry new_entry;
+ new_entry.state= state;
+ new_entry.file_no= file_no;
+ new_entry.offset= offset;
+ new_entry.is_leftmost= is_leftmost;
+ stack.emplace_back(std::move(new_entry));
+}
+
+
+void
+oob_reader_mysqlbinlog::start_traversal(uint64_t file_no, uint64_t offset)
+{
+ stack.clear();
+ push_state(ST_initial, file_no, offset, true);
+}
+
+
+/*
+ Read from out-of-band event group data.
+
+ Does a state-machine incremental traversal of the forest of perfect binary
+ trees of oob records in the event group. May read just the data available
+ on one page, thus returning less than the requested number of bytes (this
+ is to prefer to inspect each page only once, returning data page-by-page as
+ long as reader asks for at least a full page worth of data).
+*/
+int
+oob_reader_mysqlbinlog::read_data(chunk_reader_mysqlbinlog *chunk_rd,
+ uchar *buf, int len)
+{
+ stack_entry *e;
+ uint64_t chunk_idx;
+ uint64_t left_file_no;
+ uint64_t left_offset;
+ int res;
+ const uchar *p_end;
+ const uchar *p;
+ std::pair v_and_p;
+ int size;
+
+ if (stack.empty())
+ {
+ DBUG_ASSERT(0 /* Should not call when no more oob data to read. */);
+ return 0;
+ }
+
+again:
+ e= &(stack[stack.size() - 1]);
+ switch (e->state)
+ {
+ case ST_initial:
+ chunk_rd->seek(e->file_no, e->offset);
+ static_assert(sizeof(e->rd_buf) == 5*COMPR_INT_MAX64,
+ "rd_buf size must match code using it");
+ res= chunk_rd->read_data(e->rd_buf, 5*COMPR_INT_MAX64, true);
+ if (res < 0)
+ return -1;
+ if (chunk_rd->cur_type() != FSP_BINLOG_TYPE_OOB_DATA)
+ return chunk_rd->read_error_corruption("Wrong chunk type");
+ if (res == 0)
+ return chunk_rd->read_error_corruption("Unexpected EOF, expected "
+ "oob chunk");
+ e->rd_buf_len= res;
+ p_end= e->rd_buf + res;
+ v_and_p= compr_int_read(e->rd_buf);
+ p= v_and_p.second;
+ if (p > p_end)
+ return chunk_rd->read_error_corruption("Short chunk");
+ chunk_idx= v_and_p.first;
+ (void)chunk_idx;
+
+ v_and_p= compr_int_read(p);
+ p= v_and_p.second;
+ if (p > p_end)
+ return chunk_rd->read_error_corruption("Short chunk");
+ left_file_no= v_and_p.first;
+ v_and_p= compr_int_read(p);
+ p= v_and_p.second;
+ if (p > p_end)
+ return chunk_rd->read_error_corruption("Short chunk");
+ left_offset= v_and_p.first;
+
+ v_and_p= compr_int_read(p);
+ p= v_and_p.second;
+ if (p > p_end)
+ return chunk_rd->read_error_corruption("Short chunk");
+ e->right_file_no= v_and_p.first;
+ v_and_p= compr_int_read(p);
+ p= v_and_p.second;
+ if (p > p_end)
+ return chunk_rd->read_error_corruption("Short chunk");
+ e->right_offset= v_and_p.first;
+ e->rd_buf_sofar= (uint32_t)(p - e->rd_buf);
+ if (left_file_no == 0 && left_offset == 0)
+ {
+ /* Leaf node. */
+ if (e->is_leftmost && !(e->right_file_no == 0 && e->right_offset == 0))
+ {
+ /* Traverse the prior tree(s) in the forst. */
+ e->state= ST_traversing_prior_trees;
+ chunk_rd->save_pos(&e->saved_pos);
+ push_state(ST_initial, e->right_file_no, e->right_offset, true);
+ }
+ else
+ e->state= ST_self;
+ }
+ else
+ {
+ e->state= ST_traversing_left_child;
+ chunk_rd->save_pos(&e->saved_pos);
+ push_state(ST_initial, left_file_no, left_offset, e->is_leftmost);
+ }
+ goto again;
+
+ case ST_traversing_prior_trees:
+ chunk_rd->restore_pos(&e->saved_pos);
+ e->state= ST_self;
+ goto again;
+
+ case ST_traversing_left_child:
+ e->state= ST_traversing_right_child;
+ push_state(ST_initial, e->right_file_no, e->right_offset, false);
+ goto again;
+
+ case ST_traversing_right_child:
+ chunk_rd->restore_pos(&e->saved_pos);
+ e->state= ST_self;
+ goto again;
+
+ case ST_self:
+ size= 0;
+ if (e->rd_buf_len > e->rd_buf_sofar)
+ {
+ /* Use any excess data from when the header was read. */
+ size= std::min((int)(e->rd_buf_len - e->rd_buf_sofar), len);
+ memcpy(buf, e->rd_buf + e->rd_buf_sofar, size);
+ e->rd_buf_sofar+= size;
+ len-= size;
+ buf+= size;
+ }
+
+ if (len > 0 && !chunk_rd->end_of_record())
+ {
+ res= chunk_rd->read_data(buf, len, false);
+ if (res < 0)
+ return -1;
+ size+= res;
+ }
+
+ if (chunk_rd->end_of_record())
+ {
+ /* This oob record done, pop the state. */
+ DBUG_ASSERT(!stack.empty());
+ stack.erase(stack.end() - 1, stack.end());
+ }
+ return size;
+
+ default:
+ DBUG_ASSERT(0);
+ return -1;
+ }
+}
+
+
+binlog_reader_innodb::binlog_reader_innodb()
+ : oob_count(0), oob_last_file_no(0), oob_last_offset(0),
+ oob_count2(0), oob_last_file_no2(0), oob_last_offset2(0),
+ start_file_no(~(uint64_t)0),
+ rd_buf_len(0), rd_buf_sofar(0), state(ST_read_next_event_group)
+{
+ page_buf= (uchar *)
+ my_malloc(PSI_NOT_INSTRUMENTED, BINLOG_PAGE_SIZE_MAX, MYF(MY_WME));
+ chunk_rd.set_page_buf(page_buf);
+}
+
+
+binlog_reader_innodb::~binlog_reader_innodb()
+{
+ my_free(page_buf);
+}
+
+
+void
+chunk_reader_mysqlbinlog::set_fd(File fd)
+{
+ if (cur_file_handle != (File)-1)
+ {
+ my_close(cur_file_handle, MYF(0));
+ cur_file_length= ~(uint64_t)0;
+ page_loaded= false;
+ }
+ cur_file_handle= fd;
+ my_off_t old_pos= my_tell(fd, MYF(0));
+ if (old_pos != (my_off_t)-1)
+ {
+ /* Will be ~0 if we cannot seek the file. */
+ cur_file_length= my_seek(fd, 0, SEEK_END, MYF(0));
+ my_seek(fd, old_pos, SEEK_SET, MYF(0));
+ }
+}
+
+
+bool
+binlog_reader_innodb::data_available()
+{
+ DBUG_ASSERT(0 /* Should not be used in mysqlbinlog. */);
+ return true;
+}
+
+
+bool
+binlog_reader_innodb::wait_available(THD *thd, const struct timespec *abstime)
+{
+ DBUG_ASSERT(0 /* Should not be used in mysqlbinlog. */);
+ return true;
+}
+
+
+int
+binlog_reader_innodb::init_gtid_pos(THD *thd, slave_connection_state *pos,
+ rpl_binlog_state_base *state)
+{
+ DBUG_ASSERT(0 /* Should not be used in mysqlbinlog. */);
+ return 1;
+}
+
+
+int
+binlog_reader_innodb::init_legacy_pos(THD *thd, const char *filename,
+ ulonglong offset)
+{
+ DBUG_ASSERT(0 /* Should not be used in mysqlbinlog. */);
+ return 1;
+}
+
+
+void
+binlog_reader_innodb::enable_single_file()
+{
+ DBUG_ASSERT(0 /* Should not be used in mysqlbinlog. */);
+}
+
+
+int
+binlog_reader_innodb::read_binlog_data(uchar *buf, uint32_t len)
+{
+ int res= read_data(buf, len);
+ return res;
+}
+
+
+bool
+binlog_reader_innodb::init_from_fd_pos(File fd, ulonglong start_position)
+{
+ chunk_rd.set_fd(fd);
+ if (chunk_rd.parse_file_header())
+ return true;
+ uint64_t prev_start_file_no= start_file_no;
+ start_file_no= chunk_rd.s.file_no;
+ if (prev_start_file_no != ~(uint64_t)0 &&
+ prev_start_file_no + 1 == chunk_rd.s.file_no)
+ {
+ /* Continuing in the file following the previous one. */
+ }
+ else
+ {
+ if (start_position < binlog_page_size)
+ start_position= binlog_page_size;
+ chunk_rd.seek(chunk_rd.s.file_no, (uint64_t)start_position);
+ chunk_rd.skip_partial(true);
+ }
+ return false;
+}
+
+
+int binlog_reader_innodb::read_data(uchar *buf, uint32_t len)
+{
+ int res;
+ const uchar *p_end;
+ const uchar *p;
+ std::pair v_and_p;
+ int sofar= 0;
+
+again:
+ switch (state)
+ {
+ case ST_read_next_event_group:
+ if (chunk_rd.s.file_no > start_file_no ||
+ (chunk_rd.s.file_no == start_file_no && chunk_rd.is_end_of_file()))
+ {
+ /*
+ We have read the entire file, return EOF.
+ If the user specified to read the following file also, we may
+ continue where we left in that file later.
+ */
+ return sofar;
+ }
+ static_assert(sizeof(rd_buf) == 5*COMPR_INT_MAX64,
+ "rd_buf size must match code using it");
+ res= chunk_rd.read_data(rd_buf, 5*COMPR_INT_MAX64, true);
+ if (res < 0)
+ return res;
+ if (res == 0)
+ return sofar;
+ if (chunk_rd.cur_type() != FSP_BINLOG_TYPE_COMMIT)
+ {
+ chunk_rd.skip_current();
+ goto again;
+ }
+ /* Found the start of a commit record. */
+ chunk_rd.skip_partial(false);
+
+ /* Read the header of the commit record to see if there's any oob data. */
+ rd_buf_len= res;
+ p_end= rd_buf + res;
+ v_and_p= compr_int_read(rd_buf);
+ p= v_and_p.second;
+ if (p > p_end)
+ return chunk_rd.read_error_corruption("Short chunk");
+ oob_count= v_and_p.first;
+ oob_count2= 0;
+
+ if (oob_count > 0)
+ {
+ /* Skip the pointer to first chunk. */
+ v_and_p= compr_int_read(p);
+ p= v_and_p.second;
+ if (p > p_end)
+ return chunk_rd.read_error_corruption("Short chunk");
+ v_and_p= compr_int_read(p);
+ p= v_and_p.second;
+ if (p > p_end)
+ return chunk_rd.read_error_corruption("Short chunk");
+
+ v_and_p= compr_int_read(p);
+ p= v_and_p.second;
+ if (p > p_end)
+ return chunk_rd.read_error_corruption("Short chunk");
+ oob_last_file_no= v_and_p.first;
+ v_and_p= compr_int_read(p);
+ p= v_and_p.second;
+ if (p > p_end)
+ return chunk_rd.read_error_corruption("Short chunk");
+ oob_last_offset= v_and_p.first;
+
+ /* Check for any secondary oob data. */
+ v_and_p= compr_int_read(p);
+ p= v_and_p.second;
+ if (p > p_end)
+ return chunk_rd.read_error_corruption("Short chunk");
+ oob_count2= v_and_p.first;
+
+ if (oob_count2 > 0)
+ {
+ /* Skip the pointer to first chunk. */
+ v_and_p= compr_int_read(p);
+ p= v_and_p.second;
+ if (p > p_end)
+ return chunk_rd.read_error_corruption("Short chunk");
+ v_and_p= compr_int_read(p);
+ p= v_and_p.second;
+ if (p > p_end)
+ return chunk_rd.read_error_corruption("Short chunk");
+
+ v_and_p= compr_int_read(p);
+ p= v_and_p.second;
+ if (p > p_end)
+ return chunk_rd.read_error_corruption("Short chunk");
+ oob_last_file_no2= v_and_p.first;
+ v_and_p= compr_int_read(p);
+ p= v_and_p.second;
+ if (p > p_end)
+ return chunk_rd.read_error_corruption("Short chunk");
+ oob_last_offset2= v_and_p.first;
+ }
+ }
+
+ rd_buf_sofar= (uint32_t)(p - rd_buf);
+ state= ST_read_commit_record;
+ goto again;
+
+ case ST_read_commit_record:
+ if (rd_buf_len > rd_buf_sofar)
+ {
+ /* Use any excess data from when the header was read. */
+ int size= std::min((int)(rd_buf_len - rd_buf_sofar), (int)len);
+ memcpy(buf, rd_buf + rd_buf_sofar, size);
+ rd_buf_sofar+= size;
+ len-= size;
+ buf+= size;
+ sofar+= size;
+ }
+
+ if (len > 0 && !chunk_rd.end_of_record())
+ {
+ res= chunk_rd.read_data(buf, len, false);
+ if (res < 0)
+ return -1;
+ len-= res;
+ buf+= res;
+ sofar+= res;
+ }
+
+ if (rd_buf_sofar == rd_buf_len && chunk_rd.end_of_record())
+ {
+ if (oob_count == 0)
+ {
+ state= ST_read_next_event_group;
+ if (len > 0 && !chunk_rd.is_end_of_page())
+ {
+ /*
+ Let us try to read more data from this page. The goal is to read
+ from each page only once, as long as caller passes in a buffer at
+ least as big as our page size. Though commit record header that
+ spans a page boundary or oob records can break this property.
+ */
+ goto again;
+ }
+ }
+ else
+ {
+ oob_reader.start_traversal(oob_last_file_no, oob_last_offset);
+ chunk_rd.save_pos(&saved_commit_pos);
+ state= ST_read_oob_data;
+ }
+ if (sofar == 0)
+ goto again;
+ }
+
+ return sofar;
+
+ case ST_read_oob_data:
+ res= oob_reader.read_data(&chunk_rd, buf, len);
+ if (res < 0)
+ return -1;
+ if (oob_reader.oob_traversal_done())
+ {
+ if (oob_count2 > 0)
+ {
+ /* Switch over to secondary oob data. */
+ oob_count= oob_count2;
+ oob_count2= 0;
+ oob_last_file_no= oob_last_file_no2;
+ oob_last_offset= oob_last_offset2;
+ oob_reader.start_traversal(oob_last_file_no, oob_last_offset);
+ state= ST_read_oob_data;
+ }
+ else
+ {
+ chunk_rd.restore_pos(&saved_commit_pos);
+ state= ST_read_next_event_group;
+ }
+ }
+ if (res == 0)
+ {
+ DBUG_ASSERT(0 /* Should have had oob_traversal_done() last time then. */);
+ if (sofar == 0)
+ goto again;
+ }
+ return sofar + res;
+
+ default:
+ DBUG_ASSERT(0);
+ return -1;
+ }
+}
+
+
+int
+chunk_reader_mysqlbinlog::parse_file_header()
+{
+ binlog_page_size= BINLOG_HEADER_PAGE_SIZE; // Until we get the real page size
+ if (read_page_mysqlbinlog(cur_file_handle, page_buffer, 0) <= 0)
+ {
+ error("Cannot read first page of InnoDB binlog file");
+ return -1;
+ }
+ const uint32_t payload= BINLOG_HEADER_PAGE_SIZE - BINLOG_PAGE_CHECKSUM;
+ uint32_t crc32= uint4korr(page_buffer + payload);
+ if (crc32 != my_crc32c(0, page_buffer, payload))
+ {
+ error("Invalid checksum on first page, cannot read binlog file");
+ return -1;
+ }
+ uint32_t vers_major= uint4korr(page_buffer + 8);
+ if (vers_major > INNODB_BINLOG_FILE_VERS_MAJOR)
+ {
+ error("Unsupported version of InnoDB binlog file, cannot read");
+ return -1;
+ }
+ binlog_page_size= 1 << uint4korr(page_buffer + 4);
+ s.file_no= uint8korr(page_buffer + 16);
+ return 0;
+}
+
+
+enum chunk_reader_mysqlbinlog::chunk_reader_status
+chunk_reader_mysqlbinlog::fetch_current_page()
+{
+ uint64_t offset;
+ page_loaded= false;
+ for (;;)
+ {
+ if (cur_file_handle < (File)0)
+ {
+ char filename[FN_REFLEN + 1];
+ MY_STAT stat_buf;
+
+ snprintf(filename, FN_REFLEN,
+ "%s/" BINLOG_NAME_BASE "%06" PRIu64 BINLOG_NAME_EXT,
+ binlog_dir, s.file_no);
+ cur_file_handle= my_open(filename, O_RDONLY | O_BINARY, MYF(MY_WME));
+ if (cur_file_handle < (File)0) {
+ cur_file_handle= (File)-1;
+ cur_file_length= ~(uint64_t)0;
+ /*
+ For mysqlbinlog where the user specifies the file, treat a missing
+ file as EOF, on the idea that we read as much as possible from what
+ the user supplied. But still use MY_WME in the my_open() to give
+ some indication that we stopped due to a missing file.
+ */
+ return errno == ENOENT ? CHUNK_READER_EOF : CHUNK_READER_ERROR;
+ }
+ if (my_fstat(cur_file_handle, &stat_buf, MYF(0))) {
+ error("Cannot stat() file '%s', errno: %d", filename, errno);
+ my_close(cur_file_handle, MYF(0));
+ cur_file_handle= (File)-1;
+ cur_file_length= ~(uint64_t)0;
+ return CHUNK_READER_ERROR;
+ }
+ cur_file_length= stat_buf.st_size;
+ }
+
+ offset= (s.page_no * binlog_page_size) | s.in_page_offset;
+ if (offset >= cur_file_length) {
+ /* End of this file, move to the next one. */
+ goto_next_file:
+ if (cur_file_handle >= (File)0)
+ {
+ my_close(cur_file_handle, MYF(0));
+ cur_file_handle= (File)-1;
+ cur_file_length= ~(uint64_t)0;
+ }
+ ++s.file_no;
+ s.page_no= 1; /* Skip the header page. */
+ continue;
+ }
+ break;
+ }
+
+ int res= read_page_mysqlbinlog(cur_file_handle, page_buffer, s.page_no);
+ if (res < 0)
+ return CHUNK_READER_ERROR;
+ if (res == 0)
+ goto goto_next_file;
+ page_loaded= true;
+ return CHUNK_READER_FOUND;
+}
+
+
+void
+chunk_reader_mysqlbinlog::restore_pos(chunk_reader_mysqlbinlog::saved_position *pos)
+{
+ if (cur_file_handle != (File)-1 && pos->file_no != s.file_no)
+ {
+ /* Seek to a different file than currently open, close it. */
+ my_close(cur_file_handle, MYF(0));
+ cur_file_handle= (File)-1;
+ cur_file_length= ~(uint64_t)0;
+ }
+ s= *pos;
+ page_loaded= false;
+}
+
+
+void
+chunk_reader_mysqlbinlog::seek(uint64_t file_no, uint64_t offset)
+{
+ saved_position pos {
+ file_no, (uint32_t)(offset / binlog_page_size),
+ (uint32_t)(offset % binlog_page_size),
+ 0, 0, FSP_BINLOG_TYPE_FILLER, false, false };
+ restore_pos(&pos);
+}
+
+
+bool
+open_engine_binlog(handler_binlog_reader *generic_reader,
+ ulonglong start_position,
+ const char *filename, IO_CACHE *opened_cache)
+{
+ binlog_reader_innodb *reader= (binlog_reader_innodb *)generic_reader;
+ if (!reader->is_valid())
+ {
+ error("Out of memory allocating page buffer");
+ return true;
+ }
+ static_assert(sizeof(binlog_dir) >= FN_REFLEN + 1,
+ "dirname_part() needs up to FN_REFLEN char buffer");
+ size_t dummy;
+ dirname_part(binlog_dir, filename, &dummy);
+ if (!strlen(binlog_dir))
+ strncpy(binlog_dir, ".", sizeof(binlog_dir) - 1);
+ return reader->init_from_fd_pos(dup(opened_cache->file), start_position);
+}
+
+
+handler_binlog_reader *
+get_binlog_reader_innodb()
+{
+ return new binlog_reader_innodb();
+}
diff --git a/client/mysqlbinlog-engine.h b/client/mysqlbinlog-engine.h
new file mode 100644
index 0000000000000..1438abeb48069
--- /dev/null
+++ b/client/mysqlbinlog-engine.h
@@ -0,0 +1,32 @@
+/* Copyright (c) 2025, Kristian Nielsen.
+
+ This program is free software; you can redistribute it and/or modify
+ it under the terms of the GNU General Public License as published by
+ the Free Software Foundation; version 2 of the License.
+
+ This program is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU General Public License for more details.
+
+ You should have received a copy of the GNU General Public License
+ along with this program; if not, write to the Free Software
+ Foundation, Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1335 USA */
+
+#include
+#include
+
+#include "handler_binlog_reader.h"
+
+
+static constexpr uint32_t BINLOG_HEADER_PAGE_SIZE= 512;
+extern const char *INNODB_BINLOG_MAGIC;
+
+extern handler_binlog_reader *get_binlog_reader_innodb();
+extern bool open_engine_binlog(handler_binlog_reader *reader,
+ ulonglong start_position,
+ const char *filename, IO_CACHE *opened_cache);
+
+
+/* Shared functions defined in mysqlbinlog.cc */
+extern void error(const char *format, ...) ATTRIBUTE_FORMAT(printf, 1, 2);
diff --git a/client/mysqlbinlog.cc b/client/mysqlbinlog.cc
index c0414676c5ef8..ed1867b5da8e8 100644
--- a/client/mysqlbinlog.cc
+++ b/client/mysqlbinlog.cc
@@ -43,6 +43,8 @@
#include "sql_priv.h"
#include "sql_basic_types.h"
#include
+#include "handler_binlog_reader.h"
+#include "mysqlbinlog-engine.h"
#include "log_event.h"
#include "compat56.h"
#include "sql_common.h"
@@ -109,7 +111,7 @@ static const char *load_groups[]=
{ "mysqlbinlog", "mariadb-binlog", "client", "client-server", "client-mariadb",
0 };
-static void error(const char *format, ...) ATTRIBUTE_FORMAT(printf, 1, 2);
+void error(const char *format, ...) ATTRIBUTE_FORMAT(printf, 1, 2);
static void warning(const char *format, ...) ATTRIBUTE_FORMAT(printf, 1, 2);
static bool one_database=0, one_table=0, to_last_remote_log= 0, disable_log_bin= 0;
@@ -202,6 +204,14 @@ enum Exit_status {
OK_EOF,
};
+
+static enum Binlog_format {
+ ORIGINAL_BINLOG_FORMAT, INNODB_BINLOG_FORMAT
+} binlog_format= ORIGINAL_BINLOG_FORMAT;
+
+static handler_binlog_reader *engine_binlog_reader;
+
+
/**
Pointer to the last read Annotate_rows_log_event. Having read an
Annotate_rows event, we should not print it immediately because all
@@ -689,13 +699,15 @@ static bool print_base64(PRINT_EVENT_INFO *print_event_info, Log_event *ev)
{
/*
These events must be printed in base64 format, if printed.
+ In the original binlog format (no --binlog-storage-engine),
base64 format requires a FD event to be safe, so if no FD
event has been printed, we give an error. Except if user
passed --short-form, because --short-form disables printing
row events.
*/
- if (!print_event_info->printed_fd_event && !short_form &&
+ if (binlog_format == ORIGINAL_BINLOG_FORMAT &&
+ !print_event_info->printed_fd_event && !short_form &&
opt_base64_output_mode != BASE64_OUTPUT_DECODE_ROWS &&
opt_base64_output_mode != BASE64_OUTPUT_NEVER)
{
@@ -896,7 +908,7 @@ Exit_status process_event(PRINT_EVENT_INFO *print_event_info, Log_event *ev,
/*
Run time estimation of the output window configuration.
- Do not validate GLLE information is start position is provided as a file
+ Do not validate GLLE information if start position is provided as a file
offset.
*/
if (ev_type == GTID_LIST_EVENT && ev->when)
@@ -1753,7 +1765,7 @@ static void error_or_warning(const char *format, va_list args, const char *msg)
@param format Printf-style format string, followed by printf
varargs.
*/
-static void error(const char *format,...)
+void error(const char *format,...)
{
va_list args;
va_start(args, format);
@@ -1849,6 +1861,7 @@ static void cleanup()
delete_dynamic(&binlog_events);
delete_dynamic(&events_in_stmt);
}
+ delete engine_binlog_reader;
DBUG_VOID_RETURN;
}
@@ -2525,6 +2538,7 @@ static Exit_status check_master_version()
if (position_gtid_filter &&
position_gtid_filter->get_num_start_gtids() > 0)
{
+ to_last_remote_log= TRUE;
char str_buf[256];
String query_str(str_buf, sizeof(str_buf), system_charset_info);
query_str.length(0);
@@ -2979,7 +2993,35 @@ static Exit_status check_header(IO_CACHE* file,
error("Failed reading header; probably an empty file.");
return ERROR_STOP;
}
- if (memcmp(header, BINLOG_MAGIC, sizeof(header)))
+ if (0 == memcmp(header, INNODB_BINLOG_MAGIC, sizeof(header)))
+ {
+ binlog_format= INNODB_BINLOG_FORMAT;
+ if (!engine_binlog_reader)
+ {
+ engine_binlog_reader= get_binlog_reader_innodb();
+ if (!engine_binlog_reader)
+ {
+ error("Out of memory setting up reader for InnoDB-implemented binlog.");
+ return ERROR_STOP;
+ }
+ }
+ /*
+ New engine-implemented binlog always does checksum verification on the
+ page level.
+ */
+ opt_verify_binlog_checksum= 0;
+ /*
+ New engine-implemented binlog does not contain format description
+ events.
+ */
+ goto end;
+ }
+ else if (header[0] == '\0' && !memcmp(header, header+1, sizeof(header)-1))
+ {
+ /* This is an empty InnoDB binlog file, pre-allocated but not yet used. */
+ return OK_EOF;
+ }
+ else if (memcmp(header, BINLOG_MAGIC, sizeof(header)))
{
error("File is not a binary log file.");
return ERROR_STOP;
@@ -3089,6 +3131,7 @@ static Exit_status check_header(IO_CACHE* file,
break;
}
}
+end:
my_b_seek(file, pos);
return OK_CONTINUE;
}
@@ -3126,8 +3169,19 @@ static Exit_status dump_local_log_entries(PRINT_EVENT_INFO *print_event_info,
my_close(fd, MYF(MY_WME));
return ERROR_STOP;
}
- if ((retval= check_header(file, print_event_info, logname)) != OK_CONTINUE)
+ retval= check_header(file, print_event_info, logname);
+ if (retval != OK_CONTINUE)
+ {
+ if (retval == OK_EOF)
+ {
+ /*
+ Empty InnoDB-implemented binlog file. Just skip it (but still
+ continue with any following files user specified).
+ */
+ retval= OK_CONTINUE;
+ }
goto end;
+ }
}
else
{
@@ -3153,8 +3207,13 @@ static Exit_status dump_local_log_entries(PRINT_EVENT_INFO *print_event_info,
error("Failed to init IO cache.");
return ERROR_STOP;
}
- if ((retval= check_header(file, print_event_info, logname)) != OK_CONTINUE)
+ retval= check_header(file, print_event_info, logname);
+ if (retval != OK_CONTINUE)
+ {
+ if (retval == OK_EOF)
+ retval= OK_CONTINUE;
goto end;
+ }
if (start_position)
{
/* skip 'start_position' characters from stdin */
@@ -3183,15 +3242,56 @@ static Exit_status dump_local_log_entries(PRINT_EVENT_INFO *print_event_info,
error("Failed reading from file.");
goto err;
}
+ if (binlog_format == INNODB_BINLOG_FORMAT)
+ {
+ if (open_engine_binlog(engine_binlog_reader, start_position, logname, file))
+ goto err;
+ }
for (;;)
{
char llbuff[21];
my_off_t old_off = my_b_tell(file);
int read_error;
+ Log_event* ev;
- Log_event* ev = Log_event::read_log_event(file, &read_error,
- glob_description_event,
- opt_verify_binlog_checksum);
+ if (binlog_format == INNODB_BINLOG_FORMAT)
+ {
+ String packet;
+ int res= engine_binlog_reader->read_log_event(&packet, 0, MAX_MAX_ALLOWED_PACKET);
+ if (res == LOG_READ_EOF)
+ {
+ ev= nullptr;
+ read_error= 0;
+ }
+ else if (res < 0)
+ {
+ ev= nullptr;
+ read_error= -1;
+ }
+ else
+ {
+ const char *errmsg= nullptr;
+ ev= Log_event::read_log_event((uchar *)packet.ptr(), packet.length(),
+ &errmsg, glob_description_event,
+ FALSE, FALSE);
+ if (!ev)
+ {
+ error("Error reading event: %s", errmsg);
+ read_error= -1;
+ }
+ else
+ {
+ ev->register_temp_buf((uchar *)packet.release(), true);
+ read_error= 0;
+ }
+ }
+ }
+ else
+ {
+ ev= Log_event::read_log_event(file, &read_error,
+ glob_description_event,
+ opt_verify_binlog_checksum);
+ }
if (!ev)
{
/*
@@ -3217,6 +3317,7 @@ static Exit_status dump_local_log_entries(PRINT_EVENT_INFO *print_event_info,
the size of the event, unless the event is encrypted.
*/
DBUG_ASSERT(
+ binlog_format == INNODB_BINLOG_FORMAT ||
((ev->get_type_code() == UNKNOWN_EVENT &&
((Unknown_log_event *) ev)->what == Unknown_log_event::ENCRYPTED)) ||
old_off + ev->data_written == my_b_tell(file));
diff --git a/extra/mariabackup/backup_copy.cc b/extra/mariabackup/backup_copy.cc
index 22a40e5fb1042..33cba0e3f2e22 100644
--- a/extra/mariabackup/backup_copy.cc
+++ b/extra/mariabackup/backup_copy.cc
@@ -61,6 +61,7 @@ Street, Fifth Floor, Boston, MA 02110-1335 USA
#include "backup_debug.h"
#include "backup_mysql.h"
#include
+#include
#ifdef _WIN32
#include /* rmdir */
#endif
@@ -1628,7 +1629,10 @@ class Copy_back_dst_dir
return mysql_data_home;
if (is_absolute_path(path))
return path;
- return buf.assign(mysql_data_home).append(path).c_str();
+ buf.assign(mysql_data_home);
+ if (!buf.empty() && buf.back() != '/' IF_WIN(&& buf.back() != '\\', ))
+ buf.append("/");
+ return buf.append(path).c_str();
}
};
@@ -1672,6 +1676,7 @@ copy_back()
datadir_iter_t *it = NULL;
datadir_node_t node;
const char *dst_dir;
+ ds_ctxt *ds_binlogs = NULL;
memset(&node, 0, sizeof(node));
@@ -1797,6 +1802,10 @@ copy_back()
ds_destroy(ds_tmp);
+ /* Prepare destination directory for any InnoDB binlog files. */
+ dst_dir = dst_dir_buf.make(opt_binlog_directory);
+ ds_binlogs = ds_create(dst_dir, DS_TYPE_LOCAL);
+
/* copy the rest of tablespaces */
ds_tmp = ds_create(mysql_data_home, DS_TYPE_LOCAL);
@@ -1858,6 +1867,16 @@ copy_back()
filename = base_name(node.filepath);
+ /* Copy InnoDB binlog files into --binlog-directory. */
+ uint64_t file_no;
+ if (is_binlog_name(filename, &file_no)) {
+ if (!(ret = copy_or_move_file(ds_binlogs, filename, filename,
+ dst_dir, 1))) {
+ goto cleanup;
+ }
+ continue;
+ }
+
/* skip .qp files */
if (filename_matches(filename, ext_list)) {
continue;
@@ -1918,6 +1937,11 @@ copy_back()
ds_tmp = NULL;
+ if (ds_binlogs != NULL) {
+ ds_destroy(ds_binlogs);
+ ds_binlogs = NULL;
+ }
+
return(ret);
}
diff --git a/extra/mariabackup/backup_mysql.cc b/extra/mariabackup/backup_mysql.cc
index 01228f6ec304b..6c71bed5fc54d 100644
--- a/extra/mariabackup/backup_mysql.cc
+++ b/extra/mariabackup/backup_mysql.cc
@@ -385,6 +385,7 @@ bool get_mysql_vars(MYSQL *connection)
char *aria_log_dir_path_var= NULL;
char *page_zip_level_var= NULL;
char *ignore_db_dirs= NULL;
+ char *binlog_directory_var= NULL;
char *endptr;
ulong server_version= mysql_get_server_version(connection);
@@ -411,6 +412,7 @@ bool get_mysql_vars(MYSQL *connection)
{"innodb_compression_level", &page_zip_level_var},
{"ignore_db_dirs", &ignore_db_dirs},
{"aria_log_dir_path", &aria_log_dir_path_var},
+ {"binlog_directory", &binlog_directory_var},
{NULL, NULL}};
read_mysql_variables(connection, "SHOW VARIABLES", mysql_vars, true);
@@ -547,6 +549,13 @@ bool get_mysql_vars(MYSQL *connection)
if (ignore_db_dirs)
xb_load_list_string(ignore_db_dirs, ",", register_ignore_db_dirs_filter);
+ if (free_opt_binlog_directory)
+ my_free(const_cast(opt_binlog_directory));
+ opt_binlog_directory= my_strdup(PSI_NOT_INSTRUMENTED,
+ (binlog_directory_var ? binlog_directory_var : ""),
+ MYF(MY_FAE));
+ free_opt_binlog_directory= true;
+
out:
return (ret);
diff --git a/extra/mariabackup/common_engine.cc b/extra/mariabackup/common_engine.cc
index a360f63d84f9f..366e2da38ca6a 100644
--- a/extra/mariabackup/common_engine.cc
+++ b/extra/mariabackup/common_engine.cc
@@ -9,6 +9,9 @@
#include
#include
+#include "innodb_binlog.h"
+
+
namespace common_engine {
class Table {
@@ -298,17 +301,21 @@ class BackupImpl {
}
bool copy_log_tables(bool finalize);
bool copy_stats_tables();
+ bool copy_engine_binlogs(const char *binlog_dir, lsn_t backup_lsn);
bool wait_for_finish();
bool close_log_tables();
private:
void process_table_job(Table *table, bool no_lock, bool delete_table,
bool finalize, unsigned thread_num);
+ void process_binlog_job(std::string src, std::string dst,
+ lsn_t backup_lsn, unsigned thread_num);
const char *m_datadir_path;
ds_ctxt_t *m_ds;
std::vector &m_con_pool;
TasksGroup m_process_table_jobs;
+ std::unique_ptr m_page_buf;
post_copy_table_hook_t m_table_post_copy_hook;
std::unordered_map> m_log_tables;
@@ -337,6 +344,35 @@ void BackupImpl::process_table_job(Table *table, bool no_lock,
m_process_table_jobs.finish_task(result);
}
+void BackupImpl::process_binlog_job(std::string src, std::string dst,
+ lsn_t backup_lsn, unsigned thread_num) {
+ int result = 0;
+ const char *c_src= src.c_str();
+ bool is_empty= true;
+ lsn_t start_lsn;
+ int binlog_found;
+
+ if (!m_process_table_jobs.get_result())
+ goto exit;
+
+ binlog_found= get_binlog_header(c_src, m_page_buf.get(), start_lsn, is_empty);
+ if (binlog_found > 0 && !is_empty && start_lsn <= backup_lsn) {
+ // Test binlog_in_engine.mariabackup_binlogs will try to inject
+ // RESET MASTER and PURGE BINARY LOGS here.
+ DBUG_EXECUTE_IF("binlog_copy_sleep_2",
+ if (src.find("binlog-000002.ibb") !=
+ std::string::npos)
+ my_sleep(2000000););
+ if (!m_ds->copy_file(c_src, dst.c_str(), thread_num))
+ goto exit;
+ }
+
+ result = 1;
+
+exit:
+ m_process_table_jobs.finish_task(result);
+}
+
bool BackupImpl::scan(const std::unordered_set &exclude_tables,
std::unordered_set *out_processed_tables, bool no_lock,
bool collect_log_and_stats) {
@@ -461,6 +497,26 @@ bool BackupImpl::copy_stats_tables() {
return true;
}
+bool BackupImpl::copy_engine_binlogs(const char *binlog_dir, lsn_t backup_lsn) {
+ std::vectorfiles;
+ std::string dir(binlog_dir && binlog_dir[0] ? binlog_dir : m_datadir_path);
+ foreach_file_in_datadir(dir.c_str(),
+ [&](const char *name)->bool {
+ uint64_t file_no;
+ if (is_binlog_name(name, &file_no))
+ files.emplace_back(name);
+ return true;
+ });
+ m_page_buf.reset(new byte [ibb_page_size]);
+ for (auto &file : files) {
+ std::string path(dir + "/" + file);
+ m_process_table_jobs.push_task(
+ std::bind(&BackupImpl::process_binlog_job, this, path,
+ file, backup_lsn, std::placeholders::_1));
+ }
+ return true;
+}
+
bool BackupImpl::wait_for_finish() {
/* Wait for threads to exit */
return m_process_table_jobs.wait_for_finish();
@@ -499,6 +555,10 @@ bool Backup::copy_stats_tables() {
return m_backup_impl->copy_stats_tables();
}
+bool Backup::copy_engine_binlogs(const char *binlog_dir, lsn_t backup_lsn) {
+ return m_backup_impl->copy_engine_binlogs(binlog_dir, backup_lsn);
+}
+
bool Backup::wait_for_finish() {
return m_backup_impl->wait_for_finish();
}
diff --git a/extra/mariabackup/common_engine.h b/extra/mariabackup/common_engine.h
index 6f5d8062e5040..3e6035f47f368 100644
--- a/extra/mariabackup/common_engine.h
+++ b/extra/mariabackup/common_engine.h
@@ -28,6 +28,7 @@ class Backup {
bool no_lock, bool collect_log_and_stats);
bool copy_log_tables(bool finalize);
bool copy_stats_tables();
+ bool copy_engine_binlogs(const char *binlog_dir, lsn_t backup_lsn);
bool wait_for_finish();
bool close_log_tables();
void set_post_copy_table_hook(const post_copy_table_hook_t &hook);
diff --git a/extra/mariabackup/xtrabackup.cc b/extra/mariabackup/xtrabackup.cc
index 18fd030e644b4..ffc5e32bac526 100644
--- a/extra/mariabackup/xtrabackup.cc
+++ b/extra/mariabackup/xtrabackup.cc
@@ -375,6 +375,7 @@ my_bool opt_log_innodb_page_corruption;
my_bool opt_lock_ddl_per_table = FALSE;
static my_bool opt_check_privileges;
+my_bool opt_backup_binlog= TRUE;
extern const char *innodb_checksum_algorithm_names[];
extern TYPELIB innodb_checksum_algorithm_typelib;
@@ -398,6 +399,7 @@ char *opt_incremental_history_uuid;
char *opt_user;
const char *opt_password;
bool free_opt_password;
+bool free_opt_binlog_directory= false;
char *opt_host;
char *opt_defaults_group;
char *opt_socket;
@@ -1457,7 +1459,9 @@ enum options_xtrabackup
OPT_XB_IGNORE_INNODB_PAGE_CORRUPTION,
OPT_INNODB_FORCE_RECOVERY,
OPT_INNODB_CHECKPOINT,
- OPT_ARIA_LOG_DIR_PATH
+ OPT_ARIA_LOG_DIR_PATH,
+ OPT_BINLOG,
+ OPT_BINLOG_DIRECTORY
};
struct my_option xb_client_options[]= {
@@ -2122,6 +2126,24 @@ struct my_option xb_server_options[] =
(G_PTR *) &xtrabackup_help, (G_PTR *) &xtrabackup_help, 0,
GET_BOOL, NO_ARG, 0, 0, 0, 0, 0, 0},
+ {"binlog", OPT_BINLOG,
+ "Backup the server binary logs. Only applies to server configured with "
+ "--binlog-storage-engine, old-style binlog is not backed up. Enabled by "
+ "default, specify --skip-binlog to not backup the binlog files. The "
+ "--skip-binlog option, if used, must be specified with both --backup and "
+ "--prepare",
+ (G_PTR*)&opt_backup_binlog,
+ (G_PTR*)&opt_backup_binlog,
+ 0, GET_BOOL, OPT_ARG, 1, 0, 0, 0, 0, 0},
+
+ {"binlog-directory", OPT_BINLOG_DIRECTORY,
+ "The directory into which to copy any binlog files in the backup. This can "
+ "be used to put binlog files in the correct location if the restored "
+ "server is to be configured with a non-default --binlog-directory. Only "
+ "used with --copy-back",
+ &opt_binlog_directory, &opt_binlog_directory,
+ 0, GET_STR, REQUIRED_ARG, 0, 0, 0, 0, 0, 0 },
+
{ 0, 0, 0, 0, 0, 0, GET_NO_ARG, NO_ARG, 0, 0, 0, 0, 0, 0}
};
@@ -2436,6 +2458,10 @@ xb_get_one_option(const struct my_option *opt,
if (my_handle_options_init_variables)
fprintf(stderr, "Obsolete option: %s. Ignored\n", opt->name);
break;
+ case OPT_BINLOG_DIRECTORY:
+
+ ADD_PRINT_PARAM_OPT(opt_binlog_directory);
+ break;
#define MYSQL_CLIENT
#include "sslopt-case.h"
#undef MYSQL_CLIENT
@@ -2603,6 +2629,13 @@ static bool innodb_init_param()
srv_undo_dir = (char*) ".";
}
+ if (!opt_binlog_directory || !xtrabackup_backup) {
+ if (free_opt_binlog_directory)
+ my_free(const_cast(opt_binlog_directory));
+ opt_binlog_directory = ".";
+ free_opt_binlog_directory= false;
+ }
+
compile_time_assert(SRV_FORCE_IGNORE_CORRUPT == 1);
/*
@@ -5394,6 +5427,29 @@ class BackupStages {
return res;
}
+ bool do_backup_binlogs() {
+ // Copy InnoDB binlog files.
+ // Going to BACKUP STAGE START protects against RESET
+ // MASTER deleting files during the copy, or FLUSH
+ // BINARY LOGS truncating them.
+ if (!opt_no_lock)
+ xb_mysql_query(mysql_connection, "BACKUP STAGE START",
+ false, false);
+ if (!m_common_backup.copy_engine_binlogs(opt_binlog_directory,
+ recv_sys.lsn)) {
+ msg("Error on copy InnoDB binlog files");
+ return false;
+ }
+ if (!m_common_backup.wait_for_finish()) {
+ msg("InnoDB binlog file backup process is finished with error");
+ return false;
+ }
+ if (!opt_no_lock)
+ xb_mysql_query(mysql_connection, "BACKUP STAGE END",
+ false, false);
+ return true;
+ }
+
bool stage_end(Backup_datasinks &backup_datasinks) {
msg("Starting BACKUP STAGE END");
/* release all locks */
@@ -5414,6 +5470,11 @@ class BackupStages {
nullptr);
);
+ if (opt_backup_binlog) {
+ if (!do_backup_binlogs())
+ return false;
+ }
+
backup_finish(backup_datasinks.m_data);
return true;
}
@@ -7690,6 +7751,8 @@ int main(int argc, char **argv)
my_free((char*) opt_password);
plugin_shutdown();
free_list(opt_plugin_load_list_ptr);
+ if (free_opt_binlog_directory)
+ my_free(const_cast(opt_binlog_directory));
mysql_server_end();
sys_var_end();
diff --git a/extra/mariabackup/xtrabackup.h b/extra/mariabackup/xtrabackup.h
index 38d7e5fdd03d1..d7f3435871f4c 100644
--- a/extra/mariabackup/xtrabackup.h
+++ b/extra/mariabackup/xtrabackup.h
@@ -151,6 +151,7 @@ extern char *opt_incremental_history_uuid;
extern char *opt_user;
extern const char *opt_password;
+extern bool free_opt_binlog_directory;
extern char *opt_host;
extern char *opt_defaults_group;
extern char *opt_socket;
@@ -178,6 +179,7 @@ extern const char *opt_history;
enum binlog_info_enum { BINLOG_INFO_OFF, BINLOG_INFO_LOCKLESS, BINLOG_INFO_ON,
BINLOG_INFO_AUTO};
+extern bool backup_binlog;
extern ulong opt_binlog_info;
extern ulong xtrabackup_innodb_force_recovery;
diff --git a/include/handler_binlog_reader.h b/include/handler_binlog_reader.h
new file mode 100644
index 0000000000000..be97c14111fa4
--- /dev/null
+++ b/include/handler_binlog_reader.h
@@ -0,0 +1,97 @@
+#ifndef HANDLER_BINLOG_READER_INCLUDED
+#define HANDLER_BINLOG_READER_INCLUDED
+
+/* Copyright (c) 2025, Kristian Nielsen.
+
+ This program is free software; you can redistribute it and/or modify
+ it under the terms of the GNU General Public License as published by
+ the Free Software Foundation; version 2 of the License.
+
+ This program is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU General Public License for more details.
+
+ You should have received a copy of the GNU General Public License
+ along with this program; if not, write to the Free Software
+ Foundation, Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1335 USA */
+
+
+class String;
+class THD;
+struct slave_connection_state;
+struct rpl_binlog_state_base;
+
+
+/*
+ Class for reading a binlog implemented in an engine.
+*/
+class handler_binlog_reader {
+public:
+ /*
+ Approximate current position (from which next call to read_binlog_data()
+ will need to read). Updated by the engine. Used to know which binlog files
+ the active dump threads are currently reading from, to avoid purging
+ actively used binlogs.
+ */
+ uint64_t cur_file_no;
+ uint64_t cur_file_pos;
+
+private:
+ /* Position and length of any remaining data in buf[]. */
+ uint32_t buf_data_pos;
+ uint32_t buf_data_remain;
+ /* Buffer used when reading data out via read_binlog_data(). */
+ static constexpr size_t BUF_SIZE= 32768;
+ uchar *buf;
+
+public:
+ handler_binlog_reader()
+ : cur_file_no(~(uint64_t)0), cur_file_pos(~(uint64_t)0),
+ buf_data_pos(0), buf_data_remain(0)
+ {
+ buf= (uchar *)my_malloc(PSI_INSTRUMENT_ME, BUF_SIZE, MYF(0));
+ }
+ virtual ~handler_binlog_reader() {
+ my_free(buf);
+ };
+ virtual int read_binlog_data(uchar *buf, uint32_t len) = 0;
+ virtual bool data_available()= 0;
+ /*
+ Wait for data to be available to read, for kill, or for timeout.
+ Returns true in case of timeout reached, false otherwise.
+ Caller should check for kill before calling again (to avoid busy-loop).
+ */
+ virtual bool wait_available(THD *thd, const struct timespec *abstime) = 0;
+ /*
+ This initializes the current read position to the point of the slave GTID
+ position passed in as POS. It is permissible to start at a position a bit
+ earlier in the binlog, only cost is the extra read cost of reading not
+ needed event data.
+
+ If position is found, must return the corresponding binlog state in the
+ STATE output parameter and initialize cur_file_no and cur_file_pos members.
+
+ Returns:
+ -1 Error
+ 0 The requested GTID position not found, needed binlogs have been purged
+ 1 Ok, position found and returned.
+ */
+ virtual int init_gtid_pos(THD *thd, slave_connection_state *pos,
+ rpl_binlog_state_base *state) = 0;
+ /*
+ Initialize to a legacy-type position (filename, offset). This mostly to
+ support legacy SHOW BINLOG EVENTS.
+ */
+ virtual int init_legacy_pos(THD *thd, const char *filename,
+ ulonglong offset) = 0;
+ /*
+ Can be called after init_gtid_pos() or init_legacy_pos() to make the reader
+ stop (return EOF) at the end of the binlog file. Used for SHOW BINLOG
+ EVENTS, which has a file-based interface based on legacy file name.
+ */
+ virtual void enable_single_file() = 0;
+ int read_log_event(String *packet, uint32_t ev_offset, size_t max_allowed);
+};
+
+#endif /* HANDLER_BINLOG_READER_INCLUDED */
diff --git a/include/my_bit.h b/include/my_bit.h
index e86c3ec883fd3..adfc458f4dc69 100644
--- a/include/my_bit.h
+++ b/include/my_bit.h
@@ -221,4 +221,72 @@ static inline uint my_find_first_bit(ulonglong n)
}
C_MODE_END
+/*
+The helper function my_nlz(x) calculates the number of leading zeros
+in the binary representation of the number "x", either using a
+built-in compiler function or a substitute trick based on the use
+of the multiplication operation and a table indexed by the prefix
+of the multiplication result:
+
+Moved to mysys from ha_innodb.cc to be able to use in non-InnoDB code.
+*/
+#ifdef __GNUC__
+#define my_nlz(x) __builtin_clzll(x)
+#elif defined(_MSC_VER) && !defined(_M_CEE_PURE) && \
+ (defined(_M_IX86) || defined(_M_X64) || defined(_M_ARM64))
+#ifndef __INTRIN_H_
+#pragma warning(push, 4)
+#pragma warning(disable: 4255 4668)
+#include
+#pragma warning(pop)
+#endif
+__forceinline unsigned int my_nlz (unsigned long long x)
+{
+#if defined(_M_IX86) || defined(_M_X64)
+ unsigned long n;
+#ifdef _M_X64
+ _BitScanReverse64(&n, x);
+ return (unsigned int) n ^ 63;
+#else
+ unsigned long y = (unsigned long) (x >> 32);
+ unsigned int m = 31;
+ if (y == 0)
+ {
+ y = (unsigned long) x;
+ m = 63;
+ }
+ _BitScanReverse(&n, y);
+ return (unsigned int) n ^ m;
+#endif
+#elif defined(_M_ARM64)
+ return _CountLeadingZeros64(x);
+#endif
+}
+#else
+inline unsigned int my_nlz (unsigned long long x)
+{
+ static unsigned char table [48] = {
+ 32, 6, 5, 0, 4, 12, 0, 20,
+ 15, 3, 11, 0, 0, 18, 25, 31,
+ 8, 14, 2, 0, 10, 0, 0, 0,
+ 0, 0, 0, 21, 0, 0, 19, 26,
+ 7, 0, 13, 0, 16, 1, 22, 27,
+ 9, 0, 17, 23, 28, 24, 29, 30
+ };
+ unsigned int y= (unsigned int) (x >> 32);
+ unsigned int n= 0;
+ if (y == 0) {
+ y= (unsigned int) x;
+ n= 32;
+ }
+ y = y | (y >> 1); // Propagate leftmost 1-bit to the right.
+ y = y | (y >> 2);
+ y = y | (y >> 4);
+ y = y | (y >> 8);
+ y = y & ~(y >> 16);
+ y = y * 0x3EF5D037;
+ return n + table[y >> 26];
+}
+#endif
+
#endif /* MY_BIT_INCLUDED */
diff --git a/include/my_compr_int.h b/include/my_compr_int.h
new file mode 100644
index 0000000000000..5e58ba5366369
--- /dev/null
+++ b/include/my_compr_int.h
@@ -0,0 +1,72 @@
+/*****************************************************************************
+
+Copyright (c) 2024 Kristian Nielsen.
+
+This program is free software; you can redistribute it and/or modify it under
+the terms of the GNU General Public License as published by the Free Software
+Foundation; version 2 of the License.
+
+This program is distributed in the hope that it will be useful, but WITHOUT
+ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or FITNESS
+FOR A PARTICULAR PURPOSE. See the GNU General Public License for more details.
+
+You should have received a copy of the GNU General Public License along with
+this program; if not, write to the Free Software Foundation, Inc.,
+51 Franklin Street, Fifth Floor, Boston, MA 02110-1335 USA
+
+*****************************************************************************/
+
+/*
+ Reading and writing of compressed integers.
+
+ Created 2024-10-01 Kristian Nielsen
+*/
+
+#ifndef MY_COMPR_INT_H
+#define MY_COMPR_INT_H
+
+#include "my_bit.h"
+#include
+#include
+
+
+/*
+ Read and write compressed (up to) 64-bit integers.
+
+ A 64-bit number is encoded with 1-9 bytes. The 3 first bits stores a tag
+ that determines the number of bytes used, and the encoding is written in
+ little-endian format as (TAG | (NUMBER << 3)). The tag is the number of
+ bytes used minus 1, except that 7 denotes 9 bytes used (numbers are never
+ encoded with 8 bytes). For example:
+
+ Number Encoding
+ 0 0x00
+ 0x1f 0xf8 (0 | (0x1f << 3))
+ 0x20 0x01 0x01
+ 0xf6 0xb1 0x07
+ 0xd34a 0x52 0x9a 0x06
+ 0x1fffffffffffff 0xfe 0xff 0xff 0xff 0xff 0xff 0xff
+ 0x20000000000000 0x07 0x00 0x00 0x00 0x00 0x00 0x00 0x01 0x00
+ 0xffffffffffffffff 0xff 0xff 0xff 0xff 0xff 0xff 0xff 0xff 0x07
+
+ The main advantage over something like base-128 compression (also called
+ varint) is that the encoding and decoding can happen with just a single
+ conditional jump to determine if one or two 64-bit words are involved (or
+ even no or only well-predicted conditional jump if unaligned reads/writes
+ and buffer padding can be assumed).
+*/
+
+#define COMPR_INT_MAX32 5
+#define COMPR_INT_MAX64 9
+#define COMPR_INT_MAX COMPR_INT_MAX64
+
+/* Write compressed unsigned integer */
+extern unsigned char *compr_int_write(unsigned char *p, uint64_t v);
+/*
+ Read compressed integer.
+ Returns a pair of the value read and the incremented pointer.
+*/
+extern std::pair
+ compr_int_read(const unsigned char *p);
+
+#endif /* MY_COMPR_INT_H */
diff --git a/include/rpl_gtid_base.h b/include/rpl_gtid_base.h
new file mode 100644
index 0000000000000..e197c71a4f6ab
--- /dev/null
+++ b/include/rpl_gtid_base.h
@@ -0,0 +1,130 @@
+/* Copyright (c) 2013,2024, Kristian Nielsen and MariaDB Services Ab.
+
+ This program is free software; you can redistribute it and/or modify
+ it under the terms of the GNU General Public License as published by
+ the Free Software Foundation; version 2 of the License.
+
+ This program is distributed in the hope that it will be useful,
+ but WITHOUT ANY WARRANTY; without even the implied warranty of
+ MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ GNU General Public License for more details.
+
+ You should have received a copy of the GNU General Public License
+ along with this program; if not, write to the Free Software
+ Foundation, Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1335 USA */
+
+#ifndef RPL_GTID_BASE_H
+#define RPL_GTID_BASE_H
+
+#include "hash.h"
+
+
+/* Definitions for MariaDB global transaction ID (GTID). */
+
+struct slave_connection_state;
+
+struct rpl_gtid
+{
+ uint32 domain_id;
+ uint32 server_id;
+ uint64 seq_no;
+};
+
+/*
+ Binlog state.
+
+ A binlog state records the last GTID written to the binlog for every
+ distinct (domain_id, server_id) pair. Thus, each point in the binlog
+ corresponds to a specific binlog state.
+
+ When starting replication from a specific GTID position, the starting point
+ is identified as the most recent one where the binlog state has no higher
+ seq_no than the GTID position for any (domain_id, server_id) combination.
+
+ We also remember the most recent logged GTID for every domain_id. This is
+ used to know where to start when a master is changed to a slave. As a side
+ effect, it also allows to skip a hash lookup in the very common case of
+ logging a new GTID with same server id as last GTID.
+
+ This base class rpl_binlog_state_base contains just be basic data operations
+ to insert/update GTIDs, and is used eg. from Gtid_index_*.
+*/
+struct rpl_binlog_state_base
+{
+ struct element {
+ uint32 domain_id;
+ HASH hash; /* Containing all server_id for one domain_id */
+ /* The most recent entry in the hash. */
+ rpl_gtid *last_gtid;
+ /* Counter to allocate next seq_no for this domain. */
+ uint64 seq_no_counter;
+
+ int update_element(const rpl_gtid *gtid);
+ };
+
+ /* Mapping from domain_id to collection of elements. */
+ HASH hash;
+ my_bool initialized;
+
+ rpl_binlog_state_base() : initialized(0) {}
+ ~rpl_binlog_state_base();
+ void init();
+ void reset_nolock();
+ void free();
+ bool load_nolock(struct rpl_gtid *list, uint32 count);
+ bool load_nolock(rpl_binlog_state_base *orig_state);
+ int update_nolock(const struct rpl_gtid *gtid);
+ int alloc_element_nolock(const rpl_gtid *gtid);
+ uint32 count_nolock();
+ int get_gtid_list_nolock(rpl_gtid *gtid_list, uint32 list_size);
+ rpl_gtid *find_nolock(uint32 domain_id, uint32 server_id);
+ bool is_before_pos(slave_connection_state *pos);
+
+ /*
+ Inline iterator over a binlog state, most recent GTID comes last for each
+ domain just like get_gtid_list_nolock().
+
+ The ITERATOR_FUNC should have signature bool f(const rpl_gtid *), and
+ return true in case of error (in which case iterate() aborts and also
+ returns true).
+
+ Intended to do custom GTID state processing without requiring the overhead
+ of an intermediate list, and where the extra code generation is justified.
+ */
+ template bool iterate(F iterator_func)
+ {
+ uint32 i, j;
+ ulong outer_records= hash.records;
+
+ for (i= 0; i < outer_records; ++i)
+ {
+ element *e= (element *)my_hash_element(&hash, i);
+ ulong inner_records= e->hash.records;
+ const rpl_gtid *last_gtid= e->last_gtid;
+ if (unlikely(!last_gtid))
+ {
+ DBUG_ASSERT(inner_records==0);
+ continue;
+ }
+ for (j= 0; j <= inner_records; ++j)
+ {
+ const rpl_gtid *gtid;
+ if (j < inner_records)
+ {
+ gtid= (rpl_gtid *)my_hash_element(&e->hash, j);
+ if (gtid == last_gtid)
+ continue;
+ }
+ else
+ gtid= e->last_gtid;
+ if (iterator_func(gtid))
+ return true;
+ }
+ }
+
+ return false; // No error
+ }
+};
+
+
+#endif /* RPL_GTID_BASE_H */
diff --git a/mysql-test/include/assert_grep.inc b/mysql-test/include/assert_grep.inc
index c4f184405a297..a6abc35c1953f 100644
--- a/mysql-test/include/assert_grep.inc
+++ b/mysql-test/include/assert_grep.inc
@@ -58,7 +58,7 @@ if ($assert_match == '')
{
if ($assert_count == '')
{
- --die !!!ERROR IN TEST: you must set either $assert_match or $assert_count
+ --die !!!ERROR IN TEST: you must set either \$assert_match or \$assert_count
}
}
if ($assert_match != '')
@@ -66,7 +66,7 @@ if ($assert_match != '')
if ($assert_count != '')
{
--echo assert_text='$assert_text' assert_count='$assert_count'
- --die !!!ERROR IN TEST: you must set only one of $assert_match or $assert_count
+ --die !!!ERROR IN TEST: you must set only one of \$assert_match or \$assert_count
}
}
diff --git a/mysql-test/include/crash_mysqld.inc b/mysql-test/include/crash_mysqld.inc
index 89bc8ced4161f..6d2e1c522ceee 100644
--- a/mysql-test/include/crash_mysqld.inc
+++ b/mysql-test/include/crash_mysqld.inc
@@ -1,6 +1,6 @@
# Crash mysqld hard and wait until it's restarted
---source include/have_debug_sync.inc
+--source include/have_debug.inc
--source include/not_embedded.inc
# Write file to make mysql-test-run.pl expect crash and restart
diff --git a/mysql-test/include/have_innodb_binlog.inc b/mysql-test/include/have_innodb_binlog.inc
new file mode 100644
index 0000000000000..c841fece70297
--- /dev/null
+++ b/mysql-test/include/have_innodb_binlog.inc
@@ -0,0 +1 @@
+--source include/have_innodb.inc
diff --git a/mysql-test/include/have_innodb_binlog.opt b/mysql-test/include/have_innodb_binlog.opt
new file mode 100644
index 0000000000000..67fff2dd613e7
--- /dev/null
+++ b/mysql-test/include/have_innodb_binlog.opt
@@ -0,0 +1 @@
+--log-bin --binlog-storage-engine=innodb --max-binlog-size=256K
diff --git a/mysql-test/include/kill_binlog_dump_threads.inc b/mysql-test/include/kill_binlog_dump_threads.inc
index 38e3e2a7c989c..7c8e63fd52912 100644
--- a/mysql-test/include/kill_binlog_dump_threads.inc
+++ b/mysql-test/include/kill_binlog_dump_threads.inc
@@ -35,10 +35,17 @@ let $success= 0;
while ($wait_counter)
{
dec $wait_counter;
- let $_tid= `SELECT id FROM information_schema.processlist WHERE command = 'Binlog Dump' LIMIT 1`;
+ # Tricky here. The binlog dump thread will normally be identified by the
+ # command name "Binlog Dump". But if it was killed, but didn't have time
+ # to react on the killed yet, it will be 'Killed'. It can also be 'Busy'
+ # if the code fails to obtain the LOCK_thd_data mutex.
+ let $_tid= `SELECT IF(command='Binlog Dump', id, -1) FROM information_schema.processlist WHERE command IN ('Binlog Dump', 'Killed', 'Busy') LIMIT 1`;
if ($_tid)
{
- eval KILL QUERY $_tid;
+ if ($_tid > 0) {
+ --error 0,ER_NO_SUCH_THREAD
+ eval KILL CONNECTION $_tid;
+ }
}
if (!$_tid)
{
@@ -55,6 +62,14 @@ if (!$success)
SHOW FULL PROCESSLIST;
--die Timeout while waiting for binlog dump threads to disappear.
}
+# This an attempt to get more info about a rare sporadic test failure where
+# RESET MASTER still fails with ER_BINLOG_IN_USE after this has run.
+--let $sanity_check= `SELECT COUNT(*) FROM information_schema.processlist WHERE command = 'Binlog Dump'`
+if ($sanity_check > 0) {
+ SHOW FULL PROCESSLIST;
+ --echo ERROR: still $sanity_check dump thread(s) found!
+ --die ERROR: still $sanity_check dump thread(s) found
+}
--enable_query_log
diff --git a/mysql-test/include/reset_master.inc b/mysql-test/include/reset_master.inc
new file mode 100644
index 0000000000000..b4036ec31be0f
--- /dev/null
+++ b/mysql-test/include/reset_master.inc
@@ -0,0 +1,58 @@
+# ==== Purpose ====
+#
+# Execute a RESET MASTER on the current connection, first terminating any
+# lingering binlog dump threads that might still be sitting idle and would
+# block the RESTE MASTER.
+#
+# Note that any configured IO threads on other servers must be stopped before
+# calling this, as RESET MASTER cannot be run while there is a slave connected.
+#
+#
+# ==== Usage ====
+#
+# [--let $kill_timeout= NUMBER]
+# [--let $reset_master_retries= NUMBER]
+# --source include/reset_master.inc
+#
+# Parameters:
+# $kill_timeout
+# Maximum number of seconds to wait for dump threads to disappear.
+# $reset_master_retries
+# Maximum number of times RESET MASTER can get ER_BINLOG_IN_USE before
+# giving up.
+
+--let $include_filename= reset_master.inc
+--source include/begin_include_file.inc
+
+--disable_query_log
+
+let $_retries= 10;
+if ($reset_master_retries)
+{
+ let $_retries= $reset_master_retries;
+}
+
+let $_success= 0;
+let $_i= 0;
+while ($_i < $_retries)
+{
+ inc $_i;
+ --source include/kill_binlog_dump_threads.inc
+ --let $errno= 0
+ --error 0,ER_BINLOG_IN_USE
+ RESET MASTER;
+ if (!$errno) {
+ let $_success= 1;
+ let $_i = $_retries;
+ }
+}
+if (!$_success)
+{
+ SHOW FULL PROCESSLIST;
+ --die Timeout while trying to remove dump threads and run RESET MASTER.
+}
+
+--enable_query_log
+
+--let $include_filename= reset_master.inc
+--source include/end_include_file.inc
diff --git a/mysql-test/include/reset_master_slave.inc b/mysql-test/include/reset_master_slave.inc
index af66da2bb8bfb..a468ee873be39 100644
--- a/mysql-test/include/reset_master_slave.inc
+++ b/mysql-test/include/reset_master_slave.inc
@@ -21,6 +21,7 @@ while ($con_name != 'No such row')
--let $con_name = query_get_value(show all slaves status, Connection_name, 1)
}
+--source include/kill_binlog_dump_threads.inc
--error 0,ER_FLUSH_MASTER_BINLOG_CLOSED
reset master;
set global gtid_slave_pos='';
diff --git a/mysql-test/include/rpl_clone_slave_using_mariadb-backup.inc b/mysql-test/include/rpl_clone_slave_using_mariadb-backup.inc
index 5961e3cc09621..e73fcb054bddb 100644
--- a/mysql-test/include/rpl_clone_slave_using_mariadb-backup.inc
+++ b/mysql-test/include/rpl_clone_slave_using_mariadb-backup.inc
@@ -293,6 +293,7 @@ if ($cnf == "galera2_to_mariadb")
--connection master
set global wsrep_on=OFF;
+ --source include/kill_binlog_dump_threads.inc
RESET MASTER;
set global wsrep_on=ON;
}
diff --git a/mysql-test/include/rpl_heartbeat.inc b/mysql-test/include/rpl_heartbeat.inc
new file mode 100644
index 0000000000000..e564f8d55dbc8
--- /dev/null
+++ b/mysql-test/include/rpl_heartbeat.inc
@@ -0,0 +1,175 @@
+# Testing master to slave heartbeat protocol
+# Shared between rpl.rpl_heartbeat and binlog_in_engine.rpl_heartbeat.
+#
+# Including:
+# - user interface, grammar, checking the range and warnings about
+# unreasonable values for the heartbeat period;
+# - no rotation of relay log if heartbeat is less that slave_net_timeout
+# - SHOW STATUS like 'Slave_received_heartbeats' action
+# - SHOW STATUS like 'Slave_heartbeat_period' report
+
+connection slave;
+-- source include/stop_slave.inc
+
+connection master;
+--source include/kill_binlog_dump_threads.inc
+reset master;
+
+connection slave;
+set @restore_slave_net_timeout= @@global.slave_net_timeout;
+--disable_warnings
+set @@global.slave_net_timeout= 10;
+--enable_warnings
+
+--enable_prepare_warnings
+###
+### Checking the range
+###
+
+#
+# default period slave_net_timeout/2
+#
+--replace_result $MASTER_MYPORT MASTER_PORT
+eval change master to master_host='127.0.0.1',master_port=$MASTER_MYPORT, master_user='root';
+--query_vertical show status like 'Slave_heartbeat_period';
+
+#
+# the max for the period is ULONG_MAX/1000; an attempt to exceed it is denied
+#
+--replace_result $MASTER_MYPORT MASTER_PORT
+--error ER_SLAVE_HEARTBEAT_VALUE_OUT_OF_RANGE
+eval change master to master_host='127.0.0.1',master_port=$MASTER_MYPORT, master_user='root', master_heartbeat_period= 4294968;
+--query_vertical show status like 'Slave_heartbeat_period';
+
+#
+# the min value for the period is 1 millisecond an attempt to assign a
+# lesser will be warned with treating the value as zero
+#
+connection slave;
+--replace_result $MASTER_MYPORT MASTER_PORT
+### 5.1 mtr does not have --warning ER_SLAVE_HEARTBEAT_VALUE_OUT_OF_RANGE
+eval change master to master_host='127.0.0.1',master_port=$MASTER_MYPORT, master_user='root', master_heartbeat_period= 0.0009999;
+--query_vertical show status like 'Slave_heartbeat_period';
+
+#
+# the actual max and min must be accepted
+#
+--replace_result $MASTER_MYPORT MASTER_PORT
+eval change master to master_host='127.0.0.1',master_port=$MASTER_MYPORT, master_user='root', master_heartbeat_period= 4294967;
+--query_vertical show status like 'Slave_heartbeat_period';
+
+--replace_result $MASTER_MYPORT MASTER_PORT
+eval change master to master_host='127.0.0.1',master_port=$MASTER_MYPORT, master_user='root', master_heartbeat_period= 0.001;
+--query_vertical show status like 'Slave_heartbeat_period';
+
+reset slave;
+
+#
+# A warning if period greater than slave_net_timeout
+#
+set @@global.slave_net_timeout= 5;
+--replace_result $MASTER_MYPORT MASTER_PORT
+eval change master to master_host='127.0.0.1',master_port=$MASTER_MYPORT, master_user='root', master_heartbeat_period= 5.001;
+--query_vertical show status like 'Slave_heartbeat_period';
+
+reset slave;
+
+#
+# A warning if slave_net_timeout is set to less than the current HB period
+#
+set @@global.slave_net_timeout= 5;
+--replace_result $MASTER_MYPORT MASTER_PORT
+eval change master to master_host='127.0.0.1',master_port=$MASTER_MYPORT, master_user='root', master_heartbeat_period= 4;
+--query_vertical show status like 'Slave_heartbeat_period';
+set @@global.slave_net_timeout= 3 /* must be a warning */;
+
+reset slave;
+
+
+###
+### checking no rotation
+###
+
+connection master;
+--disable_warnings
+drop table if exists t1;
+--enable_warnings
+#
+# Even though master_heartbeat_period= 0.5 is 20 times less than
+# @@global.slave_net_timeout= 10 in some circumstances master will
+# not be able to send any heartbeat during the slave's net timeout
+# and slave's relay log will rotate.
+# The probability for such a scenario is pretty small so the following
+# part is almost deterministic.
+#
+
+connection slave;
+set @@global.slave_net_timeout= 10;
+--replace_result $MASTER_MYPORT MASTER_PORT
+# no error this time but rather a warning
+eval change master to master_host='127.0.0.1',master_port=$MASTER_MYPORT, master_user='root', master_ssl_verify_server_cert=0, master_heartbeat_period= 0.5;
+--query_vertical show status like 'Slave_heartbeat_period';
+
+start slave;
+
+connection master;
+create table t1 (f1 int);
+
+connection master;
+--source include/save_master_gtid.inc
+connection slave;
+--source include/sync_with_master_gtid.inc
+let $slave_param= Relay_Log_File;
+let $slave_param_value= query_get_value(SHOW SLAVE STATUS, Relay_Log_File, 1);
+
+# there is an explicit sleep lasting longer than slave_net_timeout
+# to ensure that nothing will come to slave from master for that period.
+# That would cause reconnecting and relaylog rotation w/o the fix i.e
+# without a heartbeat received.
+
+real_sleep 15;
+
+# check (compare with the previous show's results) that no rotation happened
+source include/check_slave_param.inc;
+
+###
+### SHOW STATUS like 'Slave_heartbeat_period' and 'Slave_received_heartbeats'
+###
+
+--query_vertical show status like 'Slave_heartbeat_period';
+
+#
+# proof that there has been received at least one heartbeat;
+# The exact number of received heartbeat is an indeterministic value
+# and therefore it's not recorded into results.
+#
+
+let $slave_wait_param_counter= 300;
+let $slave_value= query_get_value("SHOW STATUS like 'Slave_received_heartbeats'", Value, 1);
+# Checking the fact that at least one heartbeat is received
+while (!$slave_value)
+{
+ dec $slave_wait_param_counter;
+ if (!$slave_wait_param_counter)
+ {
+ --echo ERROR: failed while waiting for slave parameter $slave_param: $slave_param_value
+ query_vertical show slave status;
+ SHOW STATUS like 'Slave_received_heartbeats';
+ exit;
+ }
+ sleep 0.1;
+ let $slave_value= query_get_value("SHOW STATUS like 'Slave_received_heartbeats'", Value, 1);
+}
+--echo A heartbeat has been received by the slave
+# cleanup
+
+connection master;
+drop table t1;
+
+connection master;
+--source include/save_master_gtid.inc
+connection slave;
+--source include/sync_with_master_gtid.inc
+set @@global.slave_net_timeout= @restore_slave_net_timeout;
+
+--disable_prepare_warnings
diff --git a/mysql-test/include/rpl_init.inc b/mysql-test/include/rpl_init.inc
index 26ce4e1b57fd8..671943340b35a 100644
--- a/mysql-test/include/rpl_init.inc
+++ b/mysql-test/include/rpl_init.inc
@@ -162,6 +162,7 @@ while ($_rpl_server)
USE test;
if (!$rpl_skip_reset_master_and_slave)
{
+ --source include/kill_binlog_dump_threads.inc
if (!$rpl_server_skip_log_bin)
{
--error 0
diff --git a/mysql-test/include/rpl_ip_mix.inc b/mysql-test/include/rpl_ip_mix.inc
index d547d77a3ccb1..01db8b4d77d8b 100644
--- a/mysql-test/include/rpl_ip_mix.inc
+++ b/mysql-test/include/rpl_ip_mix.inc
@@ -1,6 +1,7 @@
connect (master,$IPv6,root,,test,$MASTER_MYPORT);
connect (slave,127.0.0.1,root,,test,$SLAVE_MYPORT);
connection master;
+--source include/kill_binlog_dump_threads.inc
reset master;
source include/show_master_status.inc;
save_master_pos;
diff --git a/mysql-test/include/rpl_ipv6.inc b/mysql-test/include/rpl_ipv6.inc
index 69f282e22d98f..762e51a82e987 100644
--- a/mysql-test/include/rpl_ipv6.inc
+++ b/mysql-test/include/rpl_ipv6.inc
@@ -1,5 +1,6 @@
connect (master,$IPv6,root,,test,$MASTER_MYPORT);
connect (slave,$IPv6,root,,test,$SLAVE_MYPORT);
+--source include/kill_binlog_dump_threads.inc
connection master;
reset master;
source include/show_master_status.inc;
diff --git a/mysql-test/include/rpl_multi_engine2.inc b/mysql-test/include/rpl_multi_engine2.inc
index 24154220cb098..e7cd7ef38d318 100644
--- a/mysql-test/include/rpl_multi_engine2.inc
+++ b/mysql-test/include/rpl_multi_engine2.inc
@@ -9,6 +9,7 @@ STOP SLAVE;
RESET SLAVE;
connection master;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
connection slave;
diff --git a/mysql-test/include/rpl_parallel_temptable.inc b/mysql-test/include/rpl_parallel_temptable.inc
new file mode 100644
index 0000000000000..28ff0a408eb07
--- /dev/null
+++ b/mysql-test/include/rpl_parallel_temptable.inc
@@ -0,0 +1,302 @@
+--echo *** MDEV-6321: close_temporary_tables() in format description event not serialised correctly ***
+
+--connection server_2
+SET @old_parallel_threads=@@GLOBAL.slave_parallel_threads;
+--source include/stop_slave.inc
+SET GLOBAL slave_parallel_threads=5;
+--source include/start_slave.inc
+
+--connection server_1
+CREATE TABLE t1 (a INT PRIMARY KEY, b VARCHAR(100) CHARACTER SET utf8);
+--source include/save_master_gtid.inc
+
+--connection server_2
+--source include/sync_with_master_gtid.inc
+--source include/stop_slave.inc
+
+
+--connection server_1
+SET gtid_domain_id= 1;
+INSERT INTO t1 VALUES (1, 0);
+
+CREATE TEMPORARY TABLE t2 (a int);
+
+--connection default
+SET gtid_domain_id= 2;
+CREATE TEMPORARY TABLE t3 (a INT PRIMARY KEY);
+CREATE TEMPORARY TABLE t4 (a int);
+INSERT INTO t3 VALUES (100);
+INSERT INTO t4 SELECT a+1 FROM t3;
+
+--connection server_1
+INSERT INTO t2 VALUES (2), (4), (6), (8), (10), (12), (14), (16), (18), (20);
+INSERT INTO t2 VALUES (3), (6), (9), (12), (15), (18);
+INSERT INTO t2 VALUES (4), (8), (12), (16), (20);
+
+--connection default
+INSERT INTO t3 SELECT a+2 FROM t4;
+INSERT INTO t4 SELECT a+4 FROM t3;
+
+--connection server_1
+INSERT INTO t2 VALUES (5), (10), (15), (20);
+INSERT INTO t2 VALUES (6), (12), (18);
+INSERT INTO t2 VALUES (7), (14);
+INSERT INTO t2 VALUES (8), (16);
+INSERT INTO t2 VALUES (9), (18);
+INSERT INTO t2 VALUES (10), (20);
+
+--connection default
+INSERT INTO t3 SELECT a+8 FROM t4;
+INSERT INTO t4 SELECT a+16 FROM t3;
+
+--connection server_1
+INSERT INTO t2 VALUES (11);
+INSERT INTO t2 VALUES (12);
+INSERT INTO t2 VALUES (13);
+
+--connection default
+INSERT INTO t3 SELECT a+32 FROM t4;
+
+--connection server_1
+INSERT INTO t2 VALUES (14);
+INSERT INTO t2 VALUES (15);
+INSERT INTO t2 VALUES (16);
+
+--connection default
+INSERT INTO t4 SELECT a+64 FROM t3;
+
+--connection server_1
+INSERT INTO t2 VALUES (17);
+INSERT INTO t2 VALUES (18);
+INSERT INTO t2 VALUES (19);
+
+--connection default
+INSERT INTO t3 SELECT a+128 FROM t4;
+
+--connection server_1
+INSERT INTO t2 VALUES (20);
+
+--connection default
+INSERT INTO t1 SELECT a, a MOD 7 FROM t3;
+INSERT INTO t1 SELECT a, a MOD 7 FROM t4;
+
+--connection server_1
+INSERT INTO t1 SELECT a, COUNT(*) FROM t2 GROUP BY a;
+
+# Crash the master server, so that temporary tables are implicitly dropped.
+--write_file $MYSQLTEST_VARDIR/tmp/mysqld.1.expect
+wait
+EOF
+
+FLUSH TABLES;
+SET SESSION debug_dbug="+d,crash_dispatch_command_before";
+--error 2006,2013
+SELECT 1;
+
+--source include/wait_until_disconnected.inc
+--connection default
+--source include/wait_until_disconnected.inc
+
+--append_file $MYSQLTEST_VARDIR/tmp/mysqld.1.expect
+restart
+EOF
+
+--connection default
+--enable_reconnect
+--source include/wait_until_connected_again.inc
+
+--connection server_1
+--enable_reconnect
+--source include/wait_until_connected_again.inc
+
+INSERT INTO t1 VALUES (0, 1);
+--source include/save_master_gtid.inc
+
+--connection server_2
+# Start the slave replicating the events.
+# The bug was that the format description event written after the crash could
+# be fetched ahead of the execution of the temporary table events and executed
+# out-of-band. This would cause drop of all temporary tables and thus failure
+# for execution of remaining events.
+
+--source include/start_slave.inc
+--source include/sync_with_master_gtid.inc
+
+SELECT * FROM t1 WHERE a <= 20 ORDER BY a;
+SELECT COUNT(*) FROM t1 WHERE a BETWEEN 100+0 AND 100+256;
+SHOW STATUS LIKE 'Slave_open_temp_tables';
+
+
+--echo *** Test that if master logged partial event group before crash, we finish that group correctly before executing format description event ***
+
+--source include/stop_slave.inc
+
+--connection server_1
+CALL mtr.add_suppression("Statement accesses nontransactional table as well as transactional or temporary table, and writes to any of them");
+SET gtid_domain_id= 1;
+DELETE FROM t1;
+ALTER TABLE t1 ENGINE=InnoDB;
+CREATE TEMPORARY TABLE t2 (a INT PRIMARY KEY);
+INSERT INTO t2 VALUES (1);
+INSERT INTO t2 VALUES (2);
+
+--connection default
+SET gtid_domain_id= 2;
+CREATE TEMPORARY TABLE t3 (a INT PRIMARY KEY);
+INSERT INTO t3 VALUES (10);
+INSERT INTO t3 VALUES (20);
+
+--connection server_1
+INSERT INTO t1 SELECT a, 'server_1' FROM t2;
+
+--connection default
+INSERT INTO t1 SELECT a, 'default' FROM t3;
+
+--connection server_1
+INSERT INTO t1 SELECT a+2, '+server_1' FROM t2;
+
+# Crash the master server in the middle of writing an event group.
+--write_file $MYSQLTEST_VARDIR/tmp/mysqld.1.expect
+wait
+EOF
+
+# Don't run this part of the test when testing binlog-in-engine.
+# Because with binlog-in-engine, partial event groups cannot result
+# from server crash, binlog writes are atomic and crash-recovered.
+if (!$binlog_in_engine) {
+ FLUSH TABLES;
+ SET SESSION debug_dbug="+d,crash_before_writing_xid";
+ --error 2006,2013
+ INSERT INTO t1 SELECT a+4, '++server_1' FROM t2;
+
+--source include/wait_until_disconnected.inc
+--connection default
+--source include/wait_until_disconnected.inc
+}
+
+--append_file $MYSQLTEST_VARDIR/tmp/mysqld.1.expect
+restart
+EOF
+
+if (!$binlog_in_engine) {
+--connection default
+--enable_reconnect
+--source include/wait_until_connected_again.inc
+
+--connection server_1
+--enable_reconnect
+--source include/wait_until_connected_again.inc
+}
+if ($binlog_in_engine) {
+ # Drop the temporary tables manually, as we did not crash the server.
+ --connection default
+ DROP TEMPORARY TABLE t3;
+ --connection server_1
+ DROP TEMPORARY TABLE t2;
+}
+
+INSERT INTO t1 VALUES (0, 1);
+--source include/save_master_gtid.inc
+
+--connection server_2
+# Start the slave replicating the events.
+# The main thing to test here is that the slave will know that it
+# needs to abort the partially received event group, so that the
+# execution of format_description event will not wait infinitely
+# for a commit of the incomplete group that never happens.
+
+# Apart from the suppression, MDEV-27697 refinement to the original test needs
+# an allowance to one time accept malformed event group.
+set @@sql_log_bin=0;
+call mtr.add_suppression("Unexpected break of being relay-logged GTID 1-1-32 event group by the current GTID event 0-1-4");
+set @@sql_log_bin=1;
+set @@global.debug_dbug="+d,slave_discard_xid_for_gtid_0_x_1000";
+--source include/start_slave.inc
+--source include/sync_with_master_gtid.inc
+set @@global.debug_dbug="";
+
+SELECT * FROM t1 ORDER BY a;
+SHOW STATUS LIKE 'Slave_open_temp_tables';
+
+--connection server_1
+# This FLUSH can be removed once MDEV-6608 is fixed.
+FLUSH LOGS;
+
+--echo *** MDEV-7936: Assertion `!table || table->in_use == _current_thd()' failed on parallel replication in optimistic mode ***
+
+--connection server_1
+CREATE TEMPORARY TABLE t4 (a INT PRIMARY KEY) ENGINE=InnoDB;
+SET @old_dbug= @@SESSION.debug_dbug;
+SET SESSION debug_dbug="+d,binlog_force_commit_id";
+SET @commit_id= 10000;
+INSERT INTO t4 VALUES (30);
+INSERT INTO t4 VALUES (31);
+SET SESSION debug_dbug= @old_dbug;
+INSERT INTO t1 SELECT a, "conservative" FROM t4;
+DROP TEMPORARY TABLE t4;
+SELECT * FROM t1 WHERE a >= 30 ORDER BY a;
+--source include/save_master_gtid.inc
+
+--connection server_2
+--source include/sync_with_master_gtid.inc
+
+SELECT * FROM t1 WHERE a >= 30 ORDER BY a;
+
+--source include/stop_slave.inc
+SET @old_mode= @@GLOBAL.slave_parallel_mode;
+SET GLOBAL slave_parallel_mode=optimistic;
+
+--connection server_1
+CREATE TEMPORARY TABLE t4 (a INT PRIMARY KEY) ENGINE=InnoDB;
+INSERT INTO t4 VALUES (32);
+INSERT INTO t4 VALUES (33);
+INSERT INTO t1 SELECT a, "optimistic" FROM t4;
+DROP TEMPORARY TABLE t4;
+
+SELECT * FROM t1 WHERE a >= 30 ORDER BY a;
+--source include/save_master_gtid.inc
+
+--connection server_2
+--source include/start_slave.inc
+--source include/sync_with_master_gtid.inc
+
+SELECT * FROM t1 WHERE a >= 30 ORDER BY a;
+
+--source include/stop_slave.inc
+SET GLOBAL slave_parallel_mode=@old_mode;
+--source include/start_slave.inc
+
+
+--echo *** MDEV33426: Memory allocation accounting incorrect for replicated temptable
+--connection server_1
+CREATE TEMPORARY TABLE t5 (a int) ENGINE=Aria;
+CREATE TEMPORARY TABLE t6 (a int) ENGINE=Heap;
+INSERT INTO t5 VALUES (1);
+INSERT INTO t6 VALUES (2);
+--source include/save_master_gtid.inc
+
+--connection server_2
+--source include/sync_with_master_gtid.inc
+--source include/stop_slave.inc
+
+--connection server_1
+INSERT INTO t1 SELECT a+40, 5 FROM t5;
+INSERT INTO t1 SELECT a+40, 6 FROM t6;
+DROP TABLE t5, t6;
+
+--source include/save_master_gtid.inc
+
+--connection server_2
+--source include/start_slave.inc
+--source include/sync_with_master_gtid.inc
+SELECT * FROM t1 WHERE a>=40 ORDER BY a;
+
+# Clean up.
+
+--connection server_2
+--source include/stop_slave.inc
+SET GLOBAL slave_parallel_threads=@old_parallel_threads;
+--source include/start_slave.inc
+
+--connection server_1
+DROP TABLE t1;
diff --git a/mysql-test/include/rpl_reset.inc b/mysql-test/include/rpl_reset.inc
index 53ed1de98749d..82961c2e0e0e9 100644
--- a/mysql-test/include/rpl_reset.inc
+++ b/mysql-test/include/rpl_reset.inc
@@ -31,6 +31,9 @@
# Set the timeout when waiting for slave threads to stop and
# start, respectively. See include/wait_for_slave_param.inc
#
+# $rpl_skip_sync
+# Don't sync the slaves before resetting them.
+#
# Note:
# This script will fail if Last_SQL_Error or Last_IO_Error is
# nonempty. If you expect an error in the SQL thread, you should
@@ -50,7 +53,10 @@ if (!$rpl_debug)
}
---source include/rpl_sync.inc
+if (!$rpl_skip_sync)
+{
+ --source include/rpl_sync.inc
+}
if ($rpl_debug)
@@ -63,11 +69,26 @@ while ($_rpl_server)
--let $rpl_connection_name= server_$_rpl_server
--source include/rpl_connection.inc
- RESET MASTER;
# Check if this server is configured to have a master
if (`SELECT SUBSTRING('$rpl_master_list', 1 + ($_rpl_server - 1) * $rpl_server_count_length, $rpl_server_count_length) != ''`)
{
--source include/stop_slave.inc
+ }
+ --dec $_rpl_server
+}
+
+--let $_rpl_server= $rpl_server_count
+while ($_rpl_server)
+{
+ --let $rpl_connection_name= server_$_rpl_server
+ --source include/rpl_connection.inc
+
+ --source include/kill_binlog_dump_threads.inc
+ RESET MASTER;
+
+ # Check if this server is configured to have a master
+ if (`SELECT SUBSTRING('$rpl_master_list', 1 + ($_rpl_server - 1) * $rpl_server_count_length, $rpl_server_count_length) != ''`)
+ {
--source include/reset_slave.inc
}
--dec $_rpl_server
diff --git a/mysql-test/include/rpl_restart_server.inc b/mysql-test/include/rpl_restart_server.inc
index 5df2c67d3da23..c2b6427390374 100644
--- a/mysql-test/include/rpl_restart_server.inc
+++ b/mysql-test/include/rpl_restart_server.inc
@@ -6,6 +6,7 @@
# ==== Usage ====
#
# --let $rpl_server_number= N
+# [--let $shutdown_timeout= 60]
# [--let $rpl_server_parameters= --flag1 --flag2 ...]
# [--let $rpl_debug= 1]
# --source include/rpl_restart_server.inc
diff --git a/mysql-test/include/rpl_stmt_seq.inc b/mysql-test/include/rpl_stmt_seq.inc
index b23178ee5343a..372473cfb7f52 100644
--- a/mysql-test/include/rpl_stmt_seq.inc
+++ b/mysql-test/include/rpl_stmt_seq.inc
@@ -81,7 +81,10 @@ if ($show_binlog)
--let $binlog_file= master-bin.$_log_num_s
--source include/show_binlog_events.inc
}
-sync_slave_with_master;
+--connection master
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
connection slave;
# results before DDL(to be tested)
@@ -106,7 +109,10 @@ if ($show_binlog)
--let $binlog_file= master-bin.$_log_num_s
--source include/show_binlog_events.inc
}
-sync_slave_with_master;
+--connection master
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
connection slave;
# results after DDL(to be tested)
@@ -139,7 +145,10 @@ if ($show_binlog)
--let $binlog_file= master-bin.$_log_num_s
--source include/show_binlog_events.inc
}
-sync_slave_with_master;
+--connection master
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
connection slave;
# results after final ROLLBACK
@@ -173,7 +182,10 @@ connection master;
flush logs;
# sleep 1;
# eval SHOW BINLOG EVENTS IN 'master-bin.$_log_num_s';
-sync_slave_with_master;
+--connection master
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
connection slave;
# the final content of the binary log
diff --git a/mysql-test/include/rpl_stop_server.inc b/mysql-test/include/rpl_stop_server.inc
index 1e4a64cca250d..ee331d49cad7a 100644
--- a/mysql-test/include/rpl_stop_server.inc
+++ b/mysql-test/include/rpl_stop_server.inc
@@ -5,6 +5,7 @@
# ==== Usage ====
#
# --let $rpl_server_number= N
+# [--let $shutdown_timeout= 60]
# [--let $rpl_debug= 1]
# --source include/rpl_stop_server.inc
#
@@ -50,6 +51,10 @@ if ($rpl_debug)
# it 60 seconds (of mysqltest's default) to die before zapping it
let $rpl_shutdown_timeout= `select 60*(1+9*count(*)) from information_schema.system_variables where variable_name='have_sanitizer' and global_value like "MSAN%"`;
+if ($shutdown_timeout != '') {
+ --let $rpl_shutdown_timeout= $shutdown_timeout
+}
+
shutdown_server $rpl_shutdown_timeout;
--source include/wait_until_disconnected.inc
diff --git a/mysql-test/include/rpl_sync.inc b/mysql-test/include/rpl_sync.inc
index 83d4a2e4628bc..64275fade63f1 100644
--- a/mysql-test/include/rpl_sync.inc
+++ b/mysql-test/include/rpl_sync.inc
@@ -10,6 +10,7 @@
# [--let $rpl_only_running_threads= 1]
# [--let $rpl_debug= 1]
# [--let $slave_timeout= NUMBER]
+# [--let $disable_gtid= 1]
# --source include/rpl_sync.inc
#
# Parameters:
@@ -23,6 +24,11 @@
# - If only SQL thread is running, sync SQL thread with IO thread.
# - If no thread is running, don't sync.
#
+# $disable_gtid
+# Use old-style file/offset for syncing, even if slave is configured
+# to connect using GTID. (Useful for tests that run without properly
+# configuring domain_id, like rpl.rpl_circular_for_4_hosts).
+#
# $slave_timeout
# Set the timeout when waiting for threads to sync. See
# include/wait_for_slave_param.inc
@@ -75,6 +81,17 @@ while ($_rpl_i) {
if ($_rpl_server)
{
+ disable_query_log;
+ --connection server_$_rpl_server
+ enable_query_log;
+ # By default, sync using GTID (when GTID is enabled), as GTID is now
+ # on by default. But allow to fallback explicitly to old-style file/pos
+ # for tests that need this.
+ --let $_rpl_gtid_mode= query_get_value(SHOW SLAVE STATUS, Using_Gtid, 1)
+ if ($disable_gtid)
+ {
+ --let $_rpl_gtid_mode= No
+ }
if ($rpl_debug)
{
--echo [sync server_$_rpl_prev_server -> server_$_rpl_server]
@@ -99,15 +116,30 @@ while ($_rpl_i) {
enable_query_log;
if ($_rpl_slave_sql_running)
{
- if ($rpl_debug)
+ if ($_rpl_gtid_mode == 'No')
+ {
+ if ($rpl_debug)
+ {
+ --let $_rpl_master_file= query_get_value("SHOW MASTER STATUS", File, 1)
+ --let $_rpl_master_pos= query_get_value("SHOW MASTER STATUS", Position, 1)
+ --echo syncing master_file='$_rpl_master_file' master_pos='$_rpl_master_pos'
+ }
+ disable_connect_log;
+ --sync_slave_with_master server_$_rpl_server
+ enable_connect_log;
+ }
+ if ($_rpl_gtid_mode != 'No')
{
- --let $_rpl_master_file= query_get_value("SHOW MASTER STATUS", File, 1)
- --let $_rpl_master_pos= query_get_value("SHOW MASTER STATUS", Position, 1)
- --echo syncing master_file='$_rpl_master_file' master_pos='$_rpl_master_pos'
+ --source include/save_master_gtid.inc
+ disable_connect_log;
+ --connection server_$_rpl_server
+ if ($rpl_debug)
+ {
+ --echo syncing with GTID position '$master_pos'
+ }
+ --source include/sync_with_master_gtid.inc
+ enable_connect_log;
}
- disable_connect_log;
- --sync_slave_with_master server_$_rpl_server
- enable_connect_log;
}
if (!$_rpl_slave_sql_running)
{
@@ -127,13 +159,25 @@ while ($_rpl_i) {
{
disable_connect_log;
--connection server_$_rpl_prev_server
- if ($rpl_debug)
- {
- --let $_rpl_master_file= query_get_value("SHOW MASTER STATUS", File, 1)
- --let $_rpl_master_pos= query_get_value("SHOW MASTER STATUS", Position, 1)
- --echo syncing master_file='$_rpl_master_file' master_pos='$_rpl_master_pos'
+ if ($_rpl_gtid_mode == 'No') {
+ if ($rpl_debug)
+ {
+ --let $_rpl_master_file= query_get_value("SHOW MASTER STATUS", File, 1)
+ --let $_rpl_master_pos= query_get_value("SHOW MASTER STATUS", Position, 1)
+ --echo syncing master_file='$_rpl_master_file' master_pos='$_rpl_master_pos'
+ }
+ --sync_slave_with_master server_$_rpl_server
+ }
+ if ($_rpl_gtid_mode != 'No') {
+ --source include/save_master_gtid.inc
+ disable_connect_log;
+ --connection server_$_rpl_server
+ if ($rpl_debug)
+ {
+ --echo syncing with GTID position'$master_pos'
+ }
+ --source include/sync_with_master_gtid.inc
}
- --sync_slave_with_master server_$_rpl_server
enable_connect_log;
}
}
diff --git a/mysql-test/include/test_fieldsize.inc b/mysql-test/include/test_fieldsize.inc
index 1ce846eed0b49..cbf69c2df80c4 100644
--- a/mysql-test/include/test_fieldsize.inc
+++ b/mysql-test/include/test_fieldsize.inc
@@ -17,6 +17,7 @@ eval $test_table_slave;
connection master;
eval $test_table_master;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
eval $test_insert;
@@ -33,6 +34,7 @@ STOP SLAVE;
--source include/reset_slave.inc
connection master;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
connection slave;
diff --git a/mysql-test/include/wait_for_engine_binlog.inc b/mysql-test/include/wait_for_engine_binlog.inc
new file mode 100644
index 0000000000000..040d1de6279ce
--- /dev/null
+++ b/mysql-test/include/wait_for_engine_binlog.inc
@@ -0,0 +1,77 @@
+# include/wait_for_engine_binlog.inc
+#
+# SUMMARY
+#
+# Waits until a specific (engine-implemented) binlog file is seen with
+# the specified size in SHOW BINARY LOGS.
+# Used to avoid sporadic failures due to races with the binlog
+# pre-allocation thread.
+#
+# USAGE
+#
+# --let $binlog_name= binlog-000002.ibb
+# --let $binlog_size= 262144
+# --source include/wait_for_engine_binlog.inc
+#
+# OPTIONALLY:
+#
+# let $wait_timeout= 60; # Override default 30 seconds with 60.
+# let $wait_notfound= 1; # Wait until specified binlog _not_ found.
+#
+# EXAMPLE
+# binlog_in_engine.binlog_flush_purge.test
+#
+
+--disable_query_log
+
+let $_wait_counter= 300;
+if ($wait_timeout)
+{
+ let $_wait_counter= `SELECT $wait_timeout * 10`;
+}
+# Reset $wait_timeout so that its value won't be used on subsequent
+# calls, and default will be used instead.
+let $wait_timeout= 0;
+
+--let $_expect= 1
+--let $_message= exist
+if ($wait_notfound) {
+ --let $_expect= 0
+ --let $_message= no longer exist
+}
+let $wait_notfound= 0;
+
+--let $_done= 0
+--let $_i= 0
+while (!$_done) {
+ --let $_j= 1
+ --let $_end= 0
+ --let $_found= 0
+ while (!$_end) {
+ --let $_x= query_get_value(SHOW BINARY LOGS, Log_name, $_j)
+ if ($_x == No such row) {
+ --let $_end= 1
+ }
+ if ($_x == $binlog_name) {
+ --let $_y= query_get_value(SHOW BINARY LOGS, File_size, $_j)
+ if ($_y == $binlog_size) {
+ --let $_found= 1
+ --let $_end= 1
+ }
+ }
+ inc $_j;
+ }
+ if ($_found == $_expect) {
+ --let $_done= 1
+ }
+ if (!$_done) {
+ sleep 0.1;
+ inc $_i;
+ if ($_i >= $_wait_counter) {
+ SHOW BINARY LOGS;
+ --die Timeout waiting for binlog '$binlog_name' to $_message
+ }
+ }
+}
+
+--enable_query_log
diff --git a/mysql-test/lib/v1/mtr_cases.pl b/mysql-test/lib/v1/mtr_cases.pl
index cc190cb39f792..b85c3db00d8c0 100644
--- a/mysql-test/lib/v1/mtr_cases.pl
+++ b/mysql-test/lib/v1/mtr_cases.pl
@@ -864,6 +864,7 @@ ($$$$$$$$$)
"binlog_formats", ["row", "statement"]],
["include/have_innodb.inc", "innodb_test", 1],
+ ["include/have_innodb_binlog.inc", "innodb_test", 1],
["include/have_log_bin.inc", "need_binlog", 1],
["include/big_test.inc", "big_test", 1],
["include/have_debug.inc", "need_debug", 1],
diff --git a/mysql-test/main/mysqld--help.result b/mysql-test/main/mysqld--help.result
index 7d8e9c08e2858..0064470586f6a 100644
--- a/mysql-test/main/mysqld--help.result
+++ b/mysql-test/main/mysqld--help.result
@@ -82,6 +82,10 @@ The following specify which files/extra groups are read (specified before remain
tables such as in the statement INSERT INTO t_myisam
SELECT * FROM t_innodb; otherwise, slaves may diverge
from the master.
+ --binlog-directory=name
+ Directory path (absolute or relative to datadir) where
+ binlog files are stored. If this is used, must not
+ specify a directory path for --log-bin
--binlog-do-db=name Tells the master it should log updates for the specified
database, and exclude all others not explicitly
mentioned.
@@ -162,6 +166,10 @@ The following specify which files/extra groups are read (specified before remain
non-transactional engines for the binary log. If you
often use statements updating a great number of rows, you
can increase this to get more performance.
+ --binlog-storage-engine=name
+ Use a more efficient binlog implementation integrated
+ with the storage engine. Only available for supporting
+ engines
--block-encryption-mode=name
Default block encryption mode for AES_ENCRYPT() and
AES_DECRYPT() functions. One of: aes-128-ecb, aes-192-ecb,
@@ -1639,6 +1647,7 @@ binlog-checksum CRC32
binlog-commit-wait-count 0
binlog-commit-wait-usec 100000
binlog-direct-non-transactional-updates FALSE
+binlog-directory (No default value)
binlog-expire-logs-seconds 0
binlog-file-cache-size 16384
binlog-format MIXED
@@ -1652,6 +1661,7 @@ binlog-row-image FULL
binlog-row-metadata NO_LOG
binlog-space-limit 0
binlog-stmt-cache-size 32768
+binlog-storage-engine (No default value)
block-encryption-mode aes-128-ecb
bulk-insert-buffer-size 8388608
character-set-client-handshake TRUE
diff --git a/mysql-test/main/temp_table.result b/mysql-test/main/temp_table.result
index bf59a95660555..8d869255a6276 100644
--- a/mysql-test/main/temp_table.result
+++ b/mysql-test/main/temp_table.result
@@ -700,3 +700,24 @@ DROP TABLE t1;
#
# End of 10.6 tests
#
+#
+# Failed CREATE TEMPORARY ... SELECT must not roll back transaction
+#
+CREATE TABLE t (pk INT PRIMARY KEY) ENGINE=InnoDB;
+START TRANSACTION;
+INSERT INTO t VALUES (0);
+INSERT INTO t VALUES (1);
+CREATE TEMPORARY TABLE tmp2 (a INT) ENGINE=InnoDB SELECT 'x' AS a;
+ERROR 22007: Incorrect integer value: 'x' for column `test`.`tmp2`.`a` at row 1
+SELECT * FROM t ORDER BY pk;
+pk
+0
+1
+INSERT INTO t VALUES (1);
+ERROR 23000: Duplicate entry '1' for key 'PRIMARY'
+COMMIT;
+SELECT * FROM t ORDER BY pk;
+pk
+0
+1
+DROP TABLE t;
diff --git a/mysql-test/main/temp_table.test b/mysql-test/main/temp_table.test
index 529ce4a6af603..fd008d3dc1b20 100644
--- a/mysql-test/main/temp_table.test
+++ b/mysql-test/main/temp_table.test
@@ -768,3 +768,20 @@ DROP TABLE t1;
--echo #
--echo # End of 10.6 tests
--echo #
+
+--echo #
+--echo # Failed CREATE TEMPORARY ... SELECT must not roll back transaction
+--echo #
+
+CREATE TABLE t (pk INT PRIMARY KEY) ENGINE=InnoDB;
+START TRANSACTION;
+INSERT INTO t VALUES (0);
+INSERT INTO t VALUES (1);
+--error ER_TRUNCATED_WRONG_VALUE_FOR_FIELD
+CREATE TEMPORARY TABLE tmp2 (a INT) ENGINE=InnoDB SELECT 'x' AS a;
+SELECT * FROM t ORDER BY pk;
+--error ER_DUP_ENTRY
+INSERT INTO t VALUES (1);
+COMMIT;
+SELECT * FROM t ORDER BY pk;
+DROP TABLE t;
diff --git a/mysql-test/mariadb-test-run.pl b/mysql-test/mariadb-test-run.pl
index ee3412c9cd154..6d2e41b9564f5 100755
--- a/mysql-test/mariadb-test-run.pl
+++ b/mysql-test/mariadb-test-run.pl
@@ -182,6 +182,7 @@ END
atomic-
binlog-
binlog_encryption-
+ binlog_in_engine-
client-
csv-
compat/oracle-
diff --git a/mysql-test/suite/binlog/include/mysqlbinlog_gtid_window_test_cases.inc b/mysql-test/suite/binlog/include/mysqlbinlog_gtid_window_test_cases.inc
index 2830f3f665dbb..5b992e18d47e1 100644
--- a/mysql-test/suite/binlog/include/mysqlbinlog_gtid_window_test_cases.inc
+++ b/mysql-test/suite/binlog/include/mysqlbinlog_gtid_window_test_cases.inc
@@ -457,7 +457,8 @@ if ($is_remote == 1)
{
--echo #
--echo # Reset server state
- RESET MASTER;
+ # Use the include script to kill any lingering dump threads first.
+ --source include/reset_master.inc
--exec $MYSQL_BINLOG $MYSQLD_DATADIR/$BINLOG_FILE.orig | $MYSQL
FLUSH LOGS;
--exec $MYSQL_BINLOG $MYSQLD_DATADIR/$BINLOG_FILE2.orig | $MYSQL
diff --git a/mysql-test/suite/binlog/r/binlog_base64_flag.result b/mysql-test/suite/binlog/r/binlog_base64_flag.result
index 4b75b712aee81..0d90fde009441 100644
--- a/mysql-test/suite/binlog/r/binlog_base64_flag.result
+++ b/mysql-test/suite/binlog/r/binlog_base64_flag.result
@@ -6,16 +6,6 @@ select * from t1;
a
1
1
-==== Test BINLOG statement w/o FD event ====
-BINLOG '
-SVtYRxMBAAAAKQAAADQBAAAAABAAAAAAAAAABHRlc3QAAnQxAAEDAAE=
-SVtYRxcBAAAAIgAAAFYBAAAQABAAAAAAAAEAAf/+AgAAAA==
-';
-ERROR HY000: The BINLOG statement of type Table_map was not preceded by a format description BINLOG statement
-select * from t1;
-a
-1
-1
==== Test BINLOG statement with FD event ====
BINLOG '
ODdYRw8BAAAAZgAAAGoAAAABAAQANS4xLjIzLXJjLWRlYnVnLWxvZwAAAAAAAAAAAAAAAAAAAAAA
diff --git a/mysql-test/suite/binlog/r/binlog_checkpoint.result b/mysql-test/suite/binlog/r/binlog_checkpoint.result
index a00b1c0c1a379..3b863926f9301 100644
--- a/mysql-test/suite/binlog/r/binlog_checkpoint.result
+++ b/mysql-test/suite/binlog/r/binlog_checkpoint.result
@@ -134,17 +134,15 @@ SET GLOBAL innodb_flush_log_at_trx_commit= @old_innodb_flush_log_at_trx_commit;
# restart
*** MDEV-7402: 'reset master' hangs, waits for signalled COND_xid_list ***
connect con3,localhost,root,,;
-SET debug_sync="reset_logs_after_set_reset_master_pending SIGNAL reset_master_ready WAIT_FOR reset_master_cont";
+SET debug_sync="reset_logs_after_set_reset_master_pending SIGNAL reset_master_ready WAIT_FOR reset_master_cont TIMEOUT 1";
RESET MASTER;
connection default;
-SET @old_dbug= @@global.DEBUG_DBUG;
-SET GLOBAL debug_dbug="+d,inject_binlog_background_thread_before_mark_xid_done";
SET debug_sync="now WAIT_FOR reset_master_ready";
RESET MASTER;
-SET debug_sync="now WAIT_FOR injected_binlog_background_thread";
-SET GLOBAL debug_dbug=@old_dbug;
SET debug_sync="now SIGNAL reset_master_cont";
connection con3;
+Warnings:
+Warning 1639 debug sync point wait timed out
connection default;
SET debug_sync = 'reset';
*** MDEV-24660: MYSQL_BIN_LOG::cleanup(): Assertion `b->xid_count == 0' failed in MYSQL_BIN_LOG::cleanup
diff --git a/mysql-test/suite/binlog/r/binlog_mysqlbinlog_gtid_window.result b/mysql-test/suite/binlog/r/binlog_mysqlbinlog_gtid_window.result
index fce5f95ce82d5..f8a63f190970e 100644
--- a/mysql-test/suite/binlog/r/binlog_mysqlbinlog_gtid_window.result
+++ b/mysql-test/suite/binlog/r/binlog_mysqlbinlog_gtid_window.result
@@ -391,7 +391,7 @@ DROP TABLE t1;
# Gtid_list_log_event state
#
# Reset server state
-RESET MASTER;
+include/reset_master.inc
FLUSH LOGS;
FLUSH LOGS;
PURGE BINARY LOGS TO "master-bin.000002";
diff --git a/mysql-test/suite/binlog/r/binlog_mysqlbinlog_warn_stop_datetime.result b/mysql-test/suite/binlog/r/binlog_mysqlbinlog_warn_stop_datetime.result
index 88a3c6cc8acaf..a82e1b53acacf 100644
--- a/mysql-test/suite/binlog/r/binlog_mysqlbinlog_warn_stop_datetime.result
+++ b/mysql-test/suite/binlog/r/binlog_mysqlbinlog_warn_stop_datetime.result
@@ -3,6 +3,7 @@ SET TIMESTAMP= UNIX_TIMESTAMP('2024-12-01 10:20:30.123456');
# Clear the existing binary log state, and start fresh using
# the timestamp variable set above
#
+include/kill_binlog_dump_threads.inc
RESET MASTER;
create table t1 (a int);
insert into t1 values (1);
diff --git a/mysql-test/suite/binlog/t/binlog_base64_flag.test b/mysql-test/suite/binlog/t/binlog_base64_flag.test
index 6935f69ba3606..46d6c4950e789 100644
--- a/mysql-test/suite/binlog/t/binlog_base64_flag.test
+++ b/mysql-test/suite/binlog/t/binlog_base64_flag.test
@@ -16,20 +16,6 @@ select * from t1;
# Test that a BINLOG statement encoding a row event fails unless a
# Format_description_event as been supplied with an earlier BINLOG
# statement.
---echo ==== Test BINLOG statement w/o FD event ====
-
-# This is a binlog statement consisting of one Table_map_log_event and
-# one Write_rows_log_event. Together, they correspond to the
-# following query:
-# INSERT INTO TABLE test.t1 VALUES (2)
-
-error ER_NO_FORMAT_DESCRIPTION_EVENT_BEFORE_BINLOG_STATEMENT;
-BINLOG '
-SVtYRxMBAAAAKQAAADQBAAAAABAAAAAAAAAABHRlc3QAAnQxAAEDAAE=
-SVtYRxcBAAAAIgAAAFYBAAAQABAAAAAAAAEAAf/+AgAAAA==
-';
-# The above line should fail and 2 should not be in the table
-select * from t1;
# Test that it works to read a Format_description_log_event with a
diff --git a/mysql-test/suite/binlog/t/binlog_checkpoint.test b/mysql-test/suite/binlog/t/binlog_checkpoint.test
index 4237f33c8af4d..029f084764e1c 100644
--- a/mysql-test/suite/binlog/t/binlog_checkpoint.test
+++ b/mysql-test/suite/binlog/t/binlog_checkpoint.test
@@ -156,16 +156,15 @@ connect(con3,localhost,root,,);
# flag set by another RESET MASTER; this could cause the wakeup from the
# binlog background thread not to be sent, and thus the second RESET MASTER
# to wait infinitely.
-SET debug_sync="reset_logs_after_set_reset_master_pending SIGNAL reset_master_ready WAIT_FOR reset_master_cont";
+# After MDEV-34705, two RESET MASTERs are protected by MDL_BACKUP_START from
+# running concurrently; we test this by having a debug_sync point time
+# out while a second RESET MASTER is waiting for the first one.
+SET debug_sync="reset_logs_after_set_reset_master_pending SIGNAL reset_master_ready WAIT_FOR reset_master_cont TIMEOUT 1";
send RESET MASTER;
--connection default
-SET @old_dbug= @@global.DEBUG_DBUG;
-SET GLOBAL debug_dbug="+d,inject_binlog_background_thread_before_mark_xid_done";
SET debug_sync="now WAIT_FOR reset_master_ready";
RESET MASTER;
-SET debug_sync="now WAIT_FOR injected_binlog_background_thread";
-SET GLOBAL debug_dbug=@old_dbug;
SET debug_sync="now SIGNAL reset_master_cont";
--connection con3
diff --git a/mysql-test/suite/binlog/t/binlog_mysqlbinlog_warn_stop_datetime.test b/mysql-test/suite/binlog/t/binlog_mysqlbinlog_warn_stop_datetime.test
index f862f0e7887c9..9ccb367f9ecae 100644
--- a/mysql-test/suite/binlog/t/binlog_mysqlbinlog_warn_stop_datetime.test
+++ b/mysql-test/suite/binlog/t/binlog_mysqlbinlog_warn_stop_datetime.test
@@ -29,6 +29,7 @@
--echo # Clear the existing binary log state, and start fresh using
--echo # the timestamp variable set above
--echo #
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
--let $binlog_f1= query_get_value(SHOW MASTER STATUS, File, 1)
diff --git a/mysql-test/suite/binlog_encryption/rpl_binlog_errors.result b/mysql-test/suite/binlog_encryption/rpl_binlog_errors.result
index 442b2a5da1d55..dddf26aa0b91b 100644
--- a/mysql-test/suite/binlog_encryption/rpl_binlog_errors.result
+++ b/mysql-test/suite/binlog_encryption/rpl_binlog_errors.result
@@ -13,6 +13,7 @@ call mtr.add_suppression("Could not use master-bin for logging");
SET @old_debug= @@global.debug_dbug;
SELECT repeat('x',8192) INTO OUTFILE 'MYSQLTEST_VARDIR/tmp/bug_46166.data';
SELECT repeat('x',10) INTO OUTFILE 'MYSQLTEST_VARDIR/tmp/bug_46166-2.data';
+include/kill_binlog_dump_threads.inc
RESET MASTER;
###################### TEST #1
FLUSH LOGS;
@@ -273,5 +274,6 @@ Note 1255 Slave already has been stopped
RESET SLAVE;
Warnings:
Note 4190 RESET SLAVE is implicitly changing the value of 'Using_Gtid' from 'No' to 'Slave_Pos'
+include/kill_binlog_dump_threads.inc
RESET MASTER;
include/rpl_end.inc
diff --git a/mysql-test/suite/binlog_encryption/rpl_checksum.result b/mysql-test/suite/binlog_encryption/rpl_checksum.result
index 503f359f72644..0fbb751dff7c9 100644
--- a/mysql-test/suite/binlog_encryption/rpl_checksum.result
+++ b/mysql-test/suite/binlog_encryption/rpl_checksum.result
@@ -134,6 +134,7 @@ set @@global.binlog_checksum= IF(floor((rand()*1000)%2), "CRC32", "NONE");
flush logs;
connection master;
set @@global.binlog_checksum= CRC32;
+include/kill_binlog_dump_threads.inc
reset master;
flush logs;
create table t3 (a int, b char(5));
@@ -148,6 +149,7 @@ include/stop_slave.inc
change master to master_host='127.0.0.1',master_port=MASTER_PORT, master_user='root';
connection master;
flush logs;
+include/kill_binlog_dump_threads.inc
reset master;
insert into t3 value (1, @@global.binlog_checksum);
connection slave;
diff --git a/mysql-test/suite/binlog_encryption/rpl_gtid_basic.result b/mysql-test/suite/binlog_encryption/rpl_gtid_basic.result
index 997218131596d..4c79bce26b728 100644
--- a/mysql-test/suite/binlog_encryption/rpl_gtid_basic.result
+++ b/mysql-test/suite/binlog_encryption/rpl_gtid_basic.result
@@ -197,6 +197,7 @@ connection server_1;
SET @old_state= @@GLOBAL.gtid_binlog_state;
SET GLOBAL gtid_binlog_state = '';
ERROR HY000: This operation is not allowed if any GTID has been logged to the binary log. Run RESET MASTER first to erase the log
+include/kill_binlog_dump_threads.inc
RESET MASTER;
SET GLOBAL gtid_binlog_state = '';
FLUSH LOGS;
diff --git a/mysql-test/suite/binlog_encryption/rpl_packet.result b/mysql-test/suite/binlog_encryption/rpl_packet.result
index bb6269607fed0..3f62e585ad54f 100644
--- a/mysql-test/suite/binlog_encryption/rpl_packet.result
+++ b/mysql-test/suite/binlog_encryption/rpl_packet.result
@@ -65,6 +65,7 @@ Last_IO_Error = 'Got a packet bigger than 'slave_max_allowed_packet' bytes'
STOP SLAVE;
RESET SLAVE;
connection master;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
SET @max_allowed_packet_0= @@session.max_allowed_packet;
SHOW BINLOG EVENTS;
diff --git a/mysql-test/suite/binlog_encryption/rpl_semi_sync.result b/mysql-test/suite/binlog_encryption/rpl_semi_sync.result
index 03e3443b31ed4..307d54ee78b66 100644
--- a/mysql-test/suite/binlog_encryption/rpl_semi_sync.result
+++ b/mysql-test/suite/binlog_encryption/rpl_semi_sync.result
@@ -20,7 +20,7 @@ include/reset_slave.inc
set global rpl_semi_sync_master_enabled= 0;
set global rpl_semi_sync_slave_enabled= 0;
connection master;
-reset master;
+include/reset_master.inc
set global rpl_semi_sync_master_enabled= 0;
set global rpl_semi_sync_slave_enabled= 0;
#
@@ -61,7 +61,7 @@ Rpl_semi_sync_master_status OFF
show status like 'Rpl_semi_sync_master_yes_tx';
Variable_name Value
Rpl_semi_sync_master_yes_tx 0
-reset master;
+include/reset_master.inc
connection slave;
[ default state of semi-sync on slave should be OFF ]
show variables like 'rpl_semi_sync_slave_enabled';
@@ -301,9 +301,10 @@ connection master;
create table t1 (a int) engine = ENGINE_TYPE;
drop table t1;
connection slave;
+include/stop_slave.inc
[ test reset master ]
connection master;
-reset master;
+include/reset_master.inc
show status like 'Rpl_semi_sync_master_status';
Variable_name Value
Rpl_semi_sync_master_status ON
@@ -314,7 +315,6 @@ show status like 'Rpl_semi_sync_master_yes_tx';
Variable_name Value
Rpl_semi_sync_master_yes_tx 0
connection slave;
-include/stop_slave.inc
include/reset_slave.inc
include/kill_binlog_dump_threads.inc
connection slave;
@@ -349,7 +349,7 @@ connection slave;
include/stop_slave.inc
include/reset_slave.inc
connection master;
-reset master;
+include/reset_master.inc
include/kill_binlog_dump_threads.inc
set sql_log_bin=0;
grant replication slave on *.* to rpl@127.0.0.1 identified by 'rpl_password';
diff --git a/mysql-test/suite/binlog_in_engine/binlog_file_pos.result b/mysql-test/suite/binlog_in_engine/binlog_file_pos.result
new file mode 100644
index 0000000000000..f473a65daed7e
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/binlog_file_pos.result
@@ -0,0 +1,71 @@
+include/master-slave.inc
+[connection master]
+connection slave;
+include/stop_slave.inc
+connection master;
+include/reset_master.inc
+connection slave;
+include/start_slave.inc
+connection master;
+CREATE TABLE t1 (a INT PRIMARY KEY, b VARCHAR(4096)) ENGINE=InnoDB;
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection master;
+Slave current file: binlog-000000.ibb / binlog-000000.ibb
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection master;
+Slave current file: binlog-000001.ibb / binlog-000001.ibb
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection master;
+Slave current file: binlog-000001.ibb / binlog-000001.ibb
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection master;
+Slave current file: binlog-000002.ibb / binlog-000002.ibb
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection master;
+Slave current file: binlog-000002.ibb / binlog-000002.ibb
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection master;
+Slave current file: binlog-000003.ibb / binlog-000003.ibb
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection master;
+Slave current file: binlog-000003.ibb / binlog-000003.ibb
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection master;
+Slave current file: binlog-000004.ibb / binlog-000004.ibb
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection master;
+Slave current file: binlog-000004.ibb / binlog-000004.ibb
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection master;
+Slave current file: binlog-000005.ibb / binlog-000005.ibb
+SELECT COUNT(*) FROM t1;
+COUNT(*)
+300
+connection slave;
+SELECT COUNT(*) FROM t1;
+COUNT(*)
+300
+connection master;
+DROP TABLE t1;
+include/rpl_end.inc
diff --git a/mysql-test/suite/binlog_in_engine/binlog_file_pos.test b/mysql-test/suite/binlog_in_engine/binlog_file_pos.test
new file mode 100644
index 0000000000000..a961ade1a16d4
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/binlog_file_pos.test
@@ -0,0 +1,42 @@
+--source include/master-slave.inc
+--source include/have_binlog_format_row.inc
+--source include/have_innodb_binlog.inc
+
+--connection slave
+--source include/stop_slave.inc
+--connection master
+--source include/reset_master.inc
+--connection slave
+--source include/start_slave.inc
+--connection master
+
+CREATE TABLE t1 (a INT PRIMARY KEY, b VARCHAR(4096)) ENGINE=InnoDB;
+
+# Cycle through a few binlog files and see the position updated on the slave.
+--connection master
+--let $i= 0
+while ($i < 10) {
+ --let $j= 0
+ --disable_query_log
+ while ($j < 30) {
+ eval INSERT INTO t1 VALUES ($i*100+$j, repeat('.', 4000));
+ inc $j;
+ }
+ --enable_query_log
+ --source include/save_master_gtid.inc
+ --connection slave
+ --source include/sync_with_master_gtid.inc
+ --let $i_file= query_get_value(SHOW SLAVE STATUS, Master_Log_File, 1)
+ --let $s_file= query_get_value(SHOW SLAVE STATUS, Relay_Master_Log_File, 1)
+ --connection master
+ --echo Slave current file: $i_file / $s_file
+ inc $i;
+}
+
+SELECT COUNT(*) FROM t1;
+--connection slave
+SELECT COUNT(*) FROM t1;
+
+--connection master
+DROP TABLE t1;
+--source include/rpl_end.inc
diff --git a/mysql-test/suite/binlog_in_engine/binlog_flush_purge.result b/mysql-test/suite/binlog_in_engine/binlog_flush_purge.result
new file mode 100644
index 0000000000000..e5eabe79a8759
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/binlog_flush_purge.result
@@ -0,0 +1,183 @@
+include/reset_master.inc
+CREATE TABLE t1 (a INT PRIMARY KEY) ENGINE=InnoDB;
+CREATE TABLE t2 (a INT PRIMARY KEY, b VARCHAR(2048)) ENGINE=InnoDB;
+INSERT INTO t1 VALUES (1);
+BEGIN;
+INSERT INTO t1 VALUES (2);
+INSERT INTO t1 VALUES (3);
+COMMIT;
+INSERT INTO t2 VALUES (0, REPEAT("x", 2048));
+INSERT INTO t2 SELECT a+1, b FROM t2;
+INSERT INTO t2 SELECT a+2, b FROM t2;
+INSERT INTO t2 SELECT a+4, b FROM t2;
+INSERT INTO t2 SELECT a+8, b FROM t2;
+SHOW BINARY LOGS;
+Log_name File_size
+binlog-000000.ibb 262144
+binlog-000001.ibb 262144
+FLUSH BINARY LOGS;
+SHOW BINARY LOGS;
+Log_name File_size
+binlog-000000.ibb 65536
+binlog-000001.ibb 262144
+binlog-000002.ibb 262144
+SET STATEMENT sql_log_bin=0 FOR
+CALL mtr.add_suppression("InnoDB: Page corruption in binlog tablespace file page number 1");
+FLUSH BINARY LOGS;
+FLUSH BINARY LOGS;
+SHOW BINLOG EVENTS IN 'binlog-000000.ibb' LIMIT 1;
+ERROR HY000: Replication event checksum verification failed while reading from a log file
+include/reset_master.inc
+SHOW BINARY LOGS;
+Log_name File_size
+binlog-000000.ibb 262144
+binlog-000001.ibb 262144
+INSERT INTO t1 VALUES (100);
+INSERT INTO t2 VALUES (100, 'xyzzy');
+DROP TABLE t1, t2;
+CREATE TABLE t1 (a INT PRIMARY KEY, b VARCHAR(2048)) ENGINE=InnoDB;
+SET @old_min_slaves= @@GLOBAL.slave_connections_needed_for_purge;
+SET GLOBAL slave_connections_needed_for_purge= 1;
+PURGE BINARY LOGS TO 'binlog-000001.ibb';
+ERROR HY000: A purgeable log is in use, will not purge
+SHOW WARNINGS;
+Level Code Message
+Note 1375 Binary log 'binlog-000000.ibb' is not purged because less than 'slave_connections_needed_for_purge' slaves have processed it
+Error 1378 A purgeable log is in use, will not purge
+SET GLOBAL slave_connections_needed_for_purge= 0;
+PURGE BINARY LOGS TO 'binlog-000001.ibb';
+ERROR HY000: A purgeable log is in use, will not purge
+SHOW WARNINGS;
+Level Code Message
+Note 1375 Binary log 'binlog-000000.ibb' is not purged because the binlog file is in active use
+Error 1378 A purgeable log is in use, will not purge
+SET @old_max_total= @@GLOBAL.max_binlog_total_size;
+SET GLOBAL max_binlog_total_size= 4*@@GLOBAL.max_binlog_size;
+SET SESSION binlog_format= ROW;
+*** Do 1500 transactions ...
+SHOW BINARY LOGS;
+Log_name File_size
+binlog-000011.ibb 262144
+binlog-000012.ibb 262144
+binlog-000013.ibb 262144
+binlog-000014.ibb 262144
+*** Test purge by date.
+SET GLOBAL max_binlog_total_size= 0;
+SET @old_expire= @@GLOBAL.binlog_expire_logs_seconds;
+SET GLOBAL binlog_expire_logs_seconds= 1;
+*** Do 187 inserts ...
+SET GLOBAL binlog_expire_logs_seconds= 0;
+*** Do 1000 transactions ...
+SHOW BINARY LOGS;
+Log_name File_size
+binlog-000013.ibb 262144
+binlog-000014.ibb 262144
+binlog-000015.ibb 262144
+binlog-000016.ibb 262144
+binlog-000017.ibb 262144
+binlog-000018.ibb 262144
+binlog-000019.ibb 262144
+binlog-000020.ibb 262144
+binlog-000021.ibb 262144
+binlog-000022.ibb 262144
+binlog-000023.ibb 262144
+binlog-000024.ibb 262144
+binlog-000025.ibb 262144
+SET @now= NOW();
+*** Do 124 inserts ...
+PURGE BINARY LOGS BEFORE @now;
+SHOW BINARY LOGS;
+Log_name File_size
+binlog-000024.ibb 262144
+binlog-000025.ibb 262144
+binlog-000026.ibb 262144
+*** Test PURGE BINARY LOGS TO
+PURGE BINARY LOGS TO 'binlog-000026.ibb';
+ERROR HY000: A purgeable log is in use, will not purge
+SHOW WARNINGS;
+Level Code Message
+Note 1375 Binary log 'binlog-000025.ibb' is not purged because the binlog file is in active use
+Error 1378 A purgeable log is in use, will not purge
+SHOW BINARY LOGS;
+Log_name File_size
+binlog-000025.ibb 262144
+binlog-000026.ibb 262144
+*** Do 436 inserts ...
+SHOW BINARY LOGS;
+Log_name File_size
+binlog-000025.ibb 262144
+binlog-000026.ibb 262144
+binlog-000027.ibb 262144
+binlog-000028.ibb 262144
+binlog-000029.ibb 262144
+binlog-000030.ibb 262144
+PURGE BINARY LOGS TO 'binlog-000026.ibb';
+SHOW BINARY LOGS;
+Log_name File_size
+binlog-000026.ibb 262144
+binlog-000027.ibb 262144
+binlog-000028.ibb 262144
+binlog-000029.ibb 262144
+binlog-000030.ibb 262144
+PURGE BINARY LOGS TO 'binlog-999999.ibb';
+ERROR HY000: Target log not found in binlog index
+SHOW WARNINGS;
+Level Code Message
+Error 1373 Target log not found in binlog index
+*** Test purging logs when setting the maximum size.
+SET GLOBAL max_binlog_total_size= ceil(1.5*@@GLOBAL.max_binlog_size);
+Warnings:
+Note 1375 Binary log 'binlog-000029.ibb' is not purged because the binlog file is in active use
+SHOW BINARY LOGS;
+Log_name File_size
+binlog-000029.ibb 262144
+binlog-000030.ibb 262144
+SET SESSION binlog_format= MIXED;
+DROP TABLE t1;
+SET GLOBAL max_binlog_total_size= @old_max_total;
+SET GLOBAL binlog_expire_logs_seconds= @old_expire;
+*** Test FLUSH BINARY LOGS DELETE_DOMAIN_ID.
+FLUSH BINARY LOGS DELETE_DOMAIN_ID=(101);
+Warnings:
+Warning 1076 The gtid domain being deleted ('101') is not in the current binlog state
+SET SESSION gtid_domain_id= 1;
+SET SESSION gtid_seq_no= 1000;
+CREATE TABLE t1 (a INT PRIMARY KEY, b INT) ENGINE=InnoDB;
+INSERT INTO t1 VALUES (1, 0);
+INSERT INTO t1 VALUES (2, 2), (3, 0), (4, 5), (5, 0), (6, 3), (7, 4), (8, 8);
+SET SESSION gtid_domain_id= 2;
+SET SESSION gtid_seq_no= 100;
+ALTER TABLE t1 ADD INDEX b_idx(b);
+SET SESSION gtid_domain_id= 1;
+INSERT INTO t1 VALUES (10, 0), (11, 0), (12, 0);
+SELECT @@GLOBAL.gtid_binlog_state;
+@@GLOBAL.gtid_binlog_state
+0-1-2508,1-1-1003,2-1-100
+FLUSH BINARY LOGS DELETE_DOMAIN_ID=(2);
+ERROR HY000: Could not delete gtid domain. Reason: binlog files may contain gtids from the domain ('2') being deleted. Make sure to first purge those files.
+SELECT @@GLOBAL.gtid_binlog_state;
+@@GLOBAL.gtid_binlog_state
+0-1-2508,1-1-1003,2-1-100
+FLUSH BINARY LOGS;
+PURGE BINARY LOGS TO 'binlog-000031.ibb';
+FLUSH BINARY LOGS DELETE_DOMAIN_ID=(101, 2);
+Warnings:
+Warning 1076 The gtid domain being deleted ('101') is not in the current binlog state
+SELECT @@GLOBAL.gtid_binlog_state;
+@@GLOBAL.gtid_binlog_state
+0-1-2508,1-1-1003
+# restart
+SELECT @@GLOBAL.gtid_binlog_state;
+@@GLOBAL.gtid_binlog_state
+0-1-2508,1-1-1003
+SET @old_max_size= @@GLOBAL.max_binlog_size;
+SET GLOBAL max_binlog_size= 1048576;
+FLUSH BINARY LOGS;
+SHOW BINARY LOGS;
+Log_name File_size
+binlog-000031.ibb 32768
+binlog-000032.ibb 32768
+binlog-000033.ibb 262144
+binlog-000034.ibb 1048576
+SET GLOBAL max_binlog_size= @old_max_size;
+DROP TABLE t1;
diff --git a/mysql-test/suite/binlog_in_engine/binlog_flush_purge.test b/mysql-test/suite/binlog_in_engine/binlog_flush_purge.test
new file mode 100644
index 0000000000000..77a67bbae9367
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/binlog_flush_purge.test
@@ -0,0 +1,268 @@
+--source include/have_binlog_format_row.inc
+--source include/have_innodb_binlog.inc
+
+--source include/reset_master.inc
+
+CREATE TABLE t1 (a INT PRIMARY KEY) ENGINE=InnoDB;
+CREATE TABLE t2 (a INT PRIMARY KEY, b VARCHAR(2048)) ENGINE=InnoDB;
+INSERT INTO t1 VALUES (1);
+BEGIN;
+INSERT INTO t1 VALUES (2);
+INSERT INTO t1 VALUES (3);
+COMMIT;
+INSERT INTO t2 VALUES (0, REPEAT("x", 2048));
+INSERT INTO t2 SELECT a+1, b FROM t2;
+INSERT INTO t2 SELECT a+2, b FROM t2;
+INSERT INTO t2 SELECT a+4, b FROM t2;
+INSERT INTO t2 SELECT a+8, b FROM t2;
+
+--let $binlog_name= binlog-000001.ibb
+--let $binlog_size= 262144
+--source include/wait_for_engine_binlog.inc
+SHOW BINARY LOGS;
+FLUSH BINARY LOGS;
+--let $binlog_name= binlog-000002.ibb
+--let $binlog_size= 262144
+--source include/wait_for_engine_binlog.inc
+SHOW BINARY LOGS;
+
+# Flush couple logs so we are sure the first file is on disk.
+# Corrupt one bit in the first data page of the first file to test that crc32
+# mismatch is caught.
+SET STATEMENT sql_log_bin=0 FOR
+ CALL mtr.add_suppression("InnoDB: Page corruption in binlog tablespace file page number 1");
+FLUSH BINARY LOGS;
+FLUSH BINARY LOGS;
+--let $file= binlog-000000.ibb
+--let $datadir= `SELECT @@datadir`
+--let BINLOG_FILE= $datadir/$file
+perl;
+my $pos= 16384 + 50; # Early byte in page 1 (page 0 is file header).
+open F, '+<', $ENV{BINLOG_FILE} or die $!;
+sysseek F, $pos, 0 or die $!;
+my $x;
+sysread F, $x, 1 or die $!;
+$x= chr(ord($x) ^ (1 <<3));
+sysseek F, $pos, 0 or die $!;
+syswrite F, $x, 1 or die $!;
+EOF
+
+--error ER_BINLOG_READ_EVENT_CHECKSUM_FAILURE
+eval SHOW BINLOG EVENTS IN '$file' LIMIT 1;
+
+
+--source include/reset_master.inc
+--let $binlog_name= binlog-000001.ibb
+--let $binlog_size= 262144
+--source include/wait_for_engine_binlog.inc
+SHOW BINARY LOGS;
+
+INSERT INTO t1 VALUES (100);
+INSERT INTO t2 VALUES (100, 'xyzzy');
+
+DROP TABLE t1, t2;
+
+# Test purge by size
+CREATE TABLE t1 (a INT PRIMARY KEY, b VARCHAR(2048)) ENGINE=InnoDB;
+
+SET @old_min_slaves= @@GLOBAL.slave_connections_needed_for_purge;
+SET GLOBAL slave_connections_needed_for_purge= 1;
+--error ER_LOG_IN_USE
+PURGE BINARY LOGS TO 'binlog-000001.ibb';
+SHOW WARNINGS;
+SET GLOBAL slave_connections_needed_for_purge= 0;
+--error ER_LOG_IN_USE
+PURGE BINARY LOGS TO 'binlog-000001.ibb';
+SHOW WARNINGS;
+SET @old_max_total= @@GLOBAL.max_binlog_total_size;
+SET GLOBAL max_binlog_total_size= 4*@@GLOBAL.max_binlog_size;
+SET SESSION binlog_format= ROW;
+--let $num_trx= 1500
+--echo *** Do $num_trx transactions ...
+--disable_query_log
+--let $i= 0
+while ($i < $num_trx) {
+ eval INSERT INTO t1 VALUES ($i+100000, REPEAT("x", 2048));
+ inc $i;
+}
+--enable_query_log
+# The precise point at which we move to the next binlog file depends on the
+# exact size of binlogged transactions, which might change as server code is
+# developed, and then this test will fail with a different set of binlog files
+# appearing in SHOW BINARY LOGS.
+#
+# In this case, just check that the general structure of the present binlogs
+# is similar, and then update the $binlog_name waited for and the .result file.
+--let $binlog_name= binlog-000010.ibb
+--let $binlog_size= 262144
+--let $wait_notfound= 1
+--source include/wait_for_engine_binlog.inc
+--let $binlog_name= binlog-000014.ibb
+--let $binlog_size= 262144
+--source include/wait_for_engine_binlog.inc
+SHOW BINARY LOGS;
+
+--echo *** Test purge by date.
+SET GLOBAL max_binlog_total_size= 0;
+SET @old_expire= @@GLOBAL.binlog_expire_logs_seconds;
+SET GLOBAL binlog_expire_logs_seconds= 1;
+--sleep 2
+--let $num_insert= `SELECT floor(256*1.5*1024/2100)`
+--echo *** Do $num_insert inserts ...
+--disable_query_log
+BEGIN;
+--let $i= 0
+while ($i < $num_insert) {
+ eval INSERT INTO t1 VALUES ($i+200000, REPEAT("x", 2048));
+ inc $i;
+}
+COMMIT;
+--enable_query_log
+--let $binlog_name= binlog-000012.ibb
+--let $binlog_size= 262144
+--let $wait_notfound= 1
+--source include/wait_for_engine_binlog.inc
+# SHOW BINARY LOGS here will not be stable.
+# We can wait for the log before the --sleep 2 to expire.
+# But the next log might also expire, if there is a random delay sufficiently
+# long before the automatic purge runs.
+#SHOW BINARY LOGS;
+
+SET GLOBAL binlog_expire_logs_seconds= 0;
+
+--let $num_trx= 1000
+--echo *** Do $num_trx transactions ...
+--disable_query_log
+--let $i= 0
+while ($i < $num_trx) {
+ eval INSERT INTO t1 VALUES ($i+300000, REPEAT("x", 2048));
+ inc $i;
+}
+--enable_query_log
+
+--let $binlog_name= binlog-000025.ibb
+--let $binlog_size= 262144
+--source include/wait_for_engine_binlog.inc
+SHOW BINARY LOGS;
+--sleep 1
+SET @now= NOW();
+--sleep 1
+--let $num_insert= `SELECT floor(256*1.0*1024/2100)`
+--echo *** Do $num_insert inserts ...
+--disable_query_log
+BEGIN;
+--let $i= 0
+while ($i < $num_insert) {
+ eval INSERT INTO t1 VALUES ($i+400000, REPEAT("x", 2048));
+ inc $i;
+}
+COMMIT;
+--enable_query_log
+# We need to wait for 26 to be pre-allocated here, so we know that 24 has been
+# fully written to disk. Otherwise 24 may still be in the buffer pool, and the
+# file date can be older than @now and then the PURGE ... BEFORE @now below
+# fails.
+--let $binlog_name= binlog-000026.ibb
+--let $binlog_size= 262144
+--source include/wait_for_engine_binlog.inc
+PURGE BINARY LOGS BEFORE @now;
+--let $binlog_name= binlog-000023.ibb
+--let $binlog_size= 262144
+--let $wait_notfound= 1
+--source include/wait_for_engine_binlog.inc
+SHOW BINARY LOGS;
+
+--echo *** Test PURGE BINARY LOGS TO
+--let $current= query_get_value(SHOW BINARY LOGS, Log_name, 3)
+--error ER_LOG_IN_USE
+eval PURGE BINARY LOGS TO '$current';
+SHOW WARNINGS;
+SHOW BINARY LOGS;
+--let $num_insert= `SELECT floor(256*3.5*1024/2100)`
+--echo *** Do $num_insert inserts ...
+--disable_query_log
+BEGIN;
+--let $i= 0
+while ($i < $num_insert) {
+ eval INSERT INTO t1 VALUES ($i+500000, REPEAT("x", 2048));
+ inc $i;
+}
+COMMIT;
+--enable_query_log
+--let $binlog_name= binlog-000030.ibb
+--let $binlog_size= 262144
+--source include/wait_for_engine_binlog.inc
+SHOW BINARY LOGS;
+eval PURGE BINARY LOGS TO '$current';
+--let $binlog_name= binlog-000025.ibb
+--let $binlog_size= 262144
+--let $wait_notfound= 1
+--source include/wait_for_engine_binlog.inc
+SHOW BINARY LOGS;
+
+
+--error ER_UNKNOWN_TARGET_BINLOG
+PURGE BINARY LOGS TO 'binlog-999999.ibb';
+SHOW WARNINGS;
+
+--echo *** Test purging logs when setting the maximum size.
+SET GLOBAL max_binlog_total_size= ceil(1.5*@@GLOBAL.max_binlog_size);
+SHOW BINARY LOGS;
+
+SET SESSION binlog_format= MIXED;
+DROP TABLE t1;
+
+SET GLOBAL max_binlog_total_size= @old_max_total;
+SET GLOBAL binlog_expire_logs_seconds= @old_expire;
+
+
+--echo *** Test FLUSH BINARY LOGS DELETE_DOMAIN_ID.
+
+# Test that deleting only non-existing domain succeeds with a warning.
+FLUSH BINARY LOGS DELETE_DOMAIN_ID=(101);
+
+SET SESSION gtid_domain_id= 1;
+SET SESSION gtid_seq_no= 1000;
+CREATE TABLE t1 (a INT PRIMARY KEY, b INT) ENGINE=InnoDB;
+INSERT INTO t1 VALUES (1, 0);
+INSERT INTO t1 VALUES (2, 2), (3, 0), (4, 5), (5, 0), (6, 3), (7, 4), (8, 8);
+SET SESSION gtid_domain_id= 2;
+SET SESSION gtid_seq_no= 100;
+ALTER TABLE t1 ADD INDEX b_idx(b);
+SET SESSION gtid_domain_id= 1;
+INSERT INTO t1 VALUES (10, 0), (11, 0), (12, 0);
+SELECT @@GLOBAL.gtid_binlog_state;
+--error ER_BINLOG_CANT_DELETE_GTID_DOMAIN
+FLUSH BINARY LOGS DELETE_DOMAIN_ID=(2);
+SELECT @@GLOBAL.gtid_binlog_state;
+FLUSH BINARY LOGS;
+--let $binlog_name= binlog-000032.ibb
+--let $binlog_size= 262144
+--source include/wait_for_engine_binlog.inc
+PURGE BINARY LOGS TO 'binlog-000031.ibb';
+FLUSH BINARY LOGS DELETE_DOMAIN_ID=(101, 2);
+SELECT @@GLOBAL.gtid_binlog_state;
+
+# Test that deletion of domains in the state got persisted to disk.
+--let $binlog_name= binlog-000033.ibb
+--let $binlog_size= 262144
+--source include/wait_for_engine_binlog.inc
+--source include/restart_mysqld.inc
+SELECT @@GLOBAL.gtid_binlog_state;
+
+
+# Test dynamically changing the max binlog size.
+SET @old_max_size= @@GLOBAL.max_binlog_size;
+--let $binlog_size= 1048576
+eval SET GLOBAL max_binlog_size= $binlog_size;
+FLUSH BINARY LOGS;
+--let $binlog_name= binlog-000034.ibb
+--source include/wait_for_engine_binlog.inc
+SHOW BINARY LOGS;
+SET GLOBAL max_binlog_size= @old_max_size;
+
+
+DROP TABLE t1;
+
+# No need to restore @@GLOBAL.slave_connections_needed_for_purge, as we
+# restarted the server.
diff --git a/mysql-test/suite/binlog_in_engine/binlog_in_engine.result b/mysql-test/suite/binlog_in_engine/binlog_in_engine.result
new file mode 100644
index 0000000000000..d920e67891ea3
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/binlog_in_engine.result
@@ -0,0 +1,45 @@
+include/reset_master.inc
+SELECT @@GLOBAL.binlog_storage_engine;
+@@GLOBAL.binlog_storage_engine
+innodb
+CREATE TABLE t1 (a INT PRIMARY KEY) ENGINE=InnoDB;
+INSERT INTO t1 VALUES (1);
+BEGIN;
+INSERT INTO t1 VALUES (2);
+INSERT INTO t1 VALUES (3);
+COMMIT;
+SELECT * FROM t1 ORDER BY a;
+a
+1
+2
+3
+DROP TABLE t1;
+SELECT @@GLOBAL.binlog_checksum;
+@@GLOBAL.binlog_checksum
+NONE
+SHOW MASTER STATUS;
+File Position Binlog_Do_DB Binlog_Ignore_DB
+binlog-000000.ibb #
+include/show_binlog_events.inc
+Log_name Pos Event_type Server_id End_log_pos Info
+binlog-000000.ibb # Gtid # # GTID #-#-#
+binlog-000000.ibb # Query # # use `test`; CREATE TABLE t1 (a INT PRIMARY KEY) ENGINE=InnoDB
+binlog-000000.ibb # Gtid # # BEGIN GTID #-#-#
+binlog-000000.ibb # Query # # use `test`; INSERT INTO t1 VALUES (1)
+binlog-000000.ibb # Xid # # COMMIT /* XID */
+binlog-000000.ibb # Gtid # # BEGIN GTID #-#-#
+binlog-000000.ibb # Query # # use `test`; INSERT INTO t1 VALUES (2)
+binlog-000000.ibb # Query # # use `test`; INSERT INTO t1 VALUES (3)
+binlog-000000.ibb # Xid # # COMMIT /* XID */
+binlog-000000.ibb # Gtid # # GTID #-#-#
+binlog-000000.ibb # Query # # use `test`; DROP TABLE `t1` /* generated by server */
+include/show_binlog_events.inc
+Log_name Pos Event_type Server_id End_log_pos Info
+binlog-000000.ibb # Gtid # # BEGIN GTID #-#-#
+binlog-000000.ibb # Query # # use `test`; INSERT INTO t1 VALUES (1)
+binlog-000000.ibb # Xid # # COMMIT /* XID */
+CREATE TABLE t2 (a INT PRIMARY KEY, b VARCHAR(2048)) ENGINE=InnoDB;
+SET SESSION binlog_format= ROW;
+*** Do 1500 transactions ...
+SET SESSION binlog_format= MIXED;
+DROP TABLE t2;
diff --git a/mysql-test/suite/binlog_in_engine/binlog_in_engine.test b/mysql-test/suite/binlog_in_engine/binlog_in_engine.test
new file mode 100644
index 0000000000000..441cdb779c56a
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/binlog_in_engine.test
@@ -0,0 +1,57 @@
+--source include/have_binlog_format_mixed.inc
+# For now, this has to come _after_ have_log_bin.inc (or
+# have_binlog_format_*.inc), to override --log-bin=master-bin with empty
+# --log-bin, as engine does not allow to set the binlog name.
+# Alternatively, maybe could have separate have_innodb_binlog_format_*.inc
+# files and only need to include the one.
+--source include/have_innodb_binlog.inc
+
+--source include/reset_master.inc
+
+SELECT @@GLOBAL.binlog_storage_engine;
+
+CREATE TABLE t1 (a INT PRIMARY KEY) ENGINE=InnoDB;
+--let $gtid_pos= `SELECT @@last_gtid`
+INSERT INTO t1 VALUES (1);
+BEGIN;
+INSERT INTO t1 VALUES (2);
+INSERT INTO t1 VALUES (3);
+COMMIT;
+SELECT * FROM t1 ORDER BY a;
+DROP TABLE t1;
+SELECT @@GLOBAL.binlog_checksum;
+
+# This output will depend on exact size of events, which might change over time.
+# If it changes, make the test abort after the SHOW MASTER STATUS, and check
+# that the reported new binlog position is correct, it should be the point
+# where the file starts containing only zero bytes.
+#
+# If this gets too annoying to do, we can replace this with something that
+# checks that the reported file and position is within some reasonable range
+# of the value left by current code.
+--replace_column 2 #
+SHOW MASTER STATUS;
+--let $binlog_file= binlog-000000.ibb
+--let $binlog_start= 0
+--source include/show_binlog_events.inc
+--let $binlog_file=
+--let $binlog_limit= 2, 3
+--source include/show_binlog_events.inc
+
+CREATE TABLE t2 (a INT PRIMARY KEY, b VARCHAR(2048)) ENGINE=InnoDB;
+
+SET SESSION binlog_format= ROW;
+--let num_trx= 1500
+--echo *** Do $num_trx transactions ...
+--disable_query_log
+--let $i= 0
+while ($i < $num_trx) {
+ eval INSERT INTO t2 VALUES ($i, REPEAT("x", 2048));
+ inc $i;
+}
+--enable_query_log
+SET SESSION binlog_format= MIXED;
+
+--exec $MYSQL_BINLOG --read-from-remote-server --user=root --host=127.0.0.1 --port=$MASTER_MYPORT master-bin.000001 --start-position=$gtid_pos > $MYSQLTEST_VARDIR/tmp/mysqlbinlog.txt
+
+DROP TABLE t2;
diff --git a/mysql-test/suite/binlog_in_engine/binlog_in_engine2.result b/mysql-test/suite/binlog_in_engine/binlog_in_engine2.result
new file mode 100644
index 0000000000000..a0716f4e75a1e
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/binlog_in_engine2.result
@@ -0,0 +1,13 @@
+CREATE TABLE sbtest1(
+id INTEGER NOT NULL AUTO_INCREMENT,
+k INTEGER DEFAULT '0' NOT NULL,
+c CHAR(120) DEFAULT '' NOT NULL,
+pad CHAR(60) DEFAULT '' NOT NULL,
+PRIMARY KEY (id)
+) ENGINE = innodb;
+*** Test bug where a large event in trx cache would fail to flush to disk
+*** the last part of the IO_CACHE after reinit_io_cache()
+SELECT COUNT(*) FROM sbtest1;
+COUNT(*)
+1000
+DROP TABLE sbtest1;
diff --git a/mysql-test/suite/binlog_in_engine/binlog_in_engine2.test b/mysql-test/suite/binlog_in_engine/binlog_in_engine2.test
new file mode 100644
index 0000000000000..262ddfd14b353
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/binlog_in_engine2.test
@@ -0,0 +1,25 @@
+--source include/have_binlog_format_mixed.inc
+--source include/have_innodb_binlog.inc
+
+# RESET MASTER deliberately omitted here for now; the test should be able to
+# work on top of any existing binlog.
+
+CREATE TABLE sbtest1(
+ id INTEGER NOT NULL AUTO_INCREMENT,
+ k INTEGER DEFAULT '0' NOT NULL,
+ c CHAR(120) DEFAULT '' NOT NULL,
+ pad CHAR(60) DEFAULT '' NOT NULL,
+ PRIMARY KEY (id)
+) ENGINE = innodb;
+
+--echo *** Test bug where a large event in trx cache would fail to flush to disk
+--echo *** the last part of the IO_CACHE after reinit_io_cache()
+--disable_query_log
+SET SESSION binlog_format= ROW;
+INSERT INTO sbtest1(k, c, pad) VALUES(429, '83868641912-28773972837-60736120486-75162659906-27563526494-20381887404-41576422241-93426793964-56405065102-33518432330', '67847967377-48000963322-62604785301-91415491898-96926520291'),(697, '38014276128-25250245652-62722561801-27818678124-24890218270-18312424692-92565570600-36243745486-21199862476-38576014630', '23183251411-36241541236-31706421314-92007079971-60663066966'),(927, '33973744704-80540844748-72700647445-87330233173-87249600839-07301471459-22846777364-58808996678-64607045326-48799346817', '38615512647-91458489257-90681424432-95014675832-60408598704'),(702, '37002370280-58842166667-00026392672-77506866252-09658311935-56926959306-83464667271-94685475868-28264244556-14550208498', '63947013338-98809887124-59806726763-79831528812-45582457048'),(482, '44257470806-17967007152-32809666989-26174672567-29883439075-95767161284-94957565003-35708767253-53935174705-16168070783', '34551750492-67990399350-81179284955-79299808058-21257255869'),(559, '37216201353-39109531021-11197415756-87798784755-02463049870-83329763120-57551308766-61100580113-80090253566-30971527105', '05161542529-00085727016-35134775864-52531204064-98744439797'),(638, '33071042495-29920376648-91343430102-79082003121-73317691963-02846712788-88069761578-14885283975-44409837760-90760298045', '91798303270-64988107984-08161247972-12116454627-22996445111'),(494, '73754818686-04889373966-18668178968-56957589012-31352882173-91882653509-59577900152-88962682169-52981807259-62646890059', '76460662325-41613089656-42706083314-81833284991-17063140920'),(144, '26482547570-00155460224-12388481921-23289186371-78242522654-77998886134-73270876420-50821093220-31442690639-11588920653', '30508501104-50823269125-88107014550-70202920684-95842308929'),(523, '80707408529-50567701755-94710751896-99750913740-12893433455-71449705205-06190682370-44407762850-72484044178-50518730145', '62779785163-72948938250-41369758259-80996497836-62655463951'),(456, '36136545002-86991679216-03626334357-76004089615-79374967509-18534984815-05132458464-89989437682-83510632788-12432287646', '48573666000-08738799548-77636362790-42210907101-93959032950'),(992, '09541621871-07745451544-86066369281-11914125493-19870176899-61795004975-23290516920-73643807748-74786437698-52733733610', '96342150431-06343701782-81892924588-00981118830-88451196139'),(57, '01877680294-77233412754-51137257355-53666692787-52764669315-17631954384-86640167063-19920714065-41378429761-91537156169', '48448536455-20468184091-91553509501-43838086179-53174613680'),(793, '33740308365-39166188520-01682550323-83007641809-43240305797-03977684459-82195791961-50424044606-62545300617-65290792867', '78807652152-78313318846-42431475196-47944829954-29585828954'),(914, '34970221859-99698108169-07142919560-98903199891-66744672791-96334514604-49208641542-88329768174-54414112722-66897927419', '74534326291-11150330256-87932479390-72544188925-11601655822'),(922, '74631933003-43390307278-63258399899-26720471559-03326667493-40796193207-09735926091-39669580589-48659364378-15084254276', '03876524453-13729490888-27847301519-41380944459-32695775352'),(80, '93052222322-14980867581-16008150406-96939000857-33201893419-58487428638-54528010818-29164157635-52981124037-43290732502', '76677112276-46213345089-38444960911-16410303610-51807024013'),(879, '56783011387-27116056038-48067486493-29455758906-20469605654-72212829918-34540271079-60725552557-97929156311-02224932189', '76341025872-59718392188-01679600590-23868062697-82489882203'),(243, '38902235195-81188119393-70390525549-27200428859-46388097114-24752507752-17476733382-82083956099-59533464585-17058432249', '21897048450-35863093719-54537798623-51894566432-59241353840'),(805, '68605640865-88310163057-64728225453-65726328661-93025739623-80884820952-94113467186-89069937514-29117949824-56004288798', '31505394960-75815870051-12245603725-48581079925-89471354580'),(649, '89587818381-90525476703-57946403400-02867568273-40842702527-10456508553-75521637295-91043232942-31831513223-56802023249', '62600275636-56515929504-77178050118-05310509871-10081566965'),(271, '05367364711-04520098425-77020103115-09828150502-96765567520-46235568606-22186424607-75166230231-19346825581-09828591952', '22914805293-81182025714-99637159002-87047921232-57898241792'),(678, '26783945024-40690970343-77538221714-41026161012-38784848460-34880526720-68951206442-51386544462-96917581616-36423683349', '00404634916-36048479510-74303434286-36276961457-21185502236'),(96, '46586956624-27361601845-23279467667-59012909118-33400879590-70052418454-49470443485-18603852106-33142017101-11151421091', '20002318311-57516375099-95416789724-53132534187-35972020646'),(128, '95948605971-90998281356-35255674379-94936292158-18736994589-92251390945-49883666084-82446461467-59178903814-30449849367', '65588202835-08118148183-52878400500-64726831266-43987867449'),(216, '51156598121-57157148626-12980106354-97713178060-67465525383-00281694676-19709429726-49773955979-45050429248-29589430840', '16587515464-26715954775-54463282350-68000910777-50289417415'),(540, '17134293738-60578689370-88303503798-16918968646-83352830786-52771798281-47771263754-15575565298-46960252021-82626815215', '98660262840-57539654497-67858185333-60364378071-65140512744'),(607, '79744948403-26092962428-55828522335-01393214195-91584855492-02586502555-40355958534-92520300996-28700873209-89422141101', '70671223637-90471263842-83168809424-58899546675-63362224391'),(23, '13620685397-00912825882-04237013399-19646595366-58663237122-52645892420-54424624132-20864537915-40743333263-46214240430', '90934248452-94833593490-10068226752-03736369406-08390384137'),(845, '92479749452-92786403535-21095579768-68965304225-16437280581-93247330520-60453166181-54422120569-18792937007-67856041115', '46793841527-79985704433-47030434638-01182279652-38435806593'),(634, '13692123886-56365348074-81949789215-49210116377-04450963936-40764296490-76722606840-29551100741-67026376754-81744276227', '61854382117-29645023805-74280251863-61629770407-65581679258'),(997, '15481904507-70393711810-42152713009-35149398404-29847624509-39763850463-91712162537-11315287156-31719083726-10642987434', '83241037779-13528024410-66802232530-91989347566-07010888838'),(674, '47000976441-55336898721-99114451022-78866786590-69804055253-03196318789-87430103627-13613832277-34539304412-43981244922', '46450481888-37230724501-76096269430-37504876342-51108336287'),(285, '52367230428-85660247721-31044034248-48983804711-82257212656-16306588806-83604911354-04744806383-40077320499-44914917340', '17359906163-76440450970-88154985927-14558353179-85473630491'),(400, '64512565939-75803778443-97158137838-80518193263-88218707051-89103591678-23753301687-59881648117-49588633166-73886889657', '09844230753-69819523413-35612261393-40566646210-56460413211'),(758, '45776809762-48335707021-40415814289-36930849535-21663424086-40666683320-92053369094-60024632274-84567636250-39140244728', '61705517227-99651863135-99403912407-90653300293-89125022956'),(217, '31088400544-10310017472-06035248342-14279944207-76929904720-70530763987-97378856219-86693143243-27016492497-68360289531', '39079173214-47460327540-94764441995-08346569800-39400716471'),(777, '52065011519-33878081747-36597965566-83186721131-90868737291-31731898014-91237173733-10348726096-00774871327-80833011153', '23492880902-73876531274-84828041248-57474865543-91157914134'),(677, '41834504364-96483871556-34760862108-18621966272-65978349104-80142566639-24866045933-69473983741-80937613662-58061503480', '28449291917-90027988058-24675450761-54975421618-19247815441'),(709, '96923778645-27584329163-35303605405-80059796373-94549481415-22453500939-48591695814-94660167251-25708064226-24093923729', '85987269334-61409520021-28479928904-49645871258-40427435864'),(376, '34600172982-61258877004-02456216612-93017353068-68948947187-65320412979-86812414667-04055433428-95674959566-59630142193', '63453128243-68849021141-08672523473-85610846672-27849382792'),(863, '73171405107-31449034460-74143529906-15657516993-20446302671-00649349958-00486571786-06593628871-00502676497-79441845551', '84206451650-59021631767-66042550926-42600881052-26542069234'),(616, '38152232797-63459919141-65106036837-21876373100-26591250271-57392076940-95603701734-08565505986-87051553137-81120733162', '57107811348-35782613440-51421129832-20817432001-32453984143'),(186, '74378354846-03255911878-68277176000-20011632277-90875640627-82063765310-72011728517-16864452781-34466450328-44548619673', '22020807635-49130877636-71492553875-72418760842-87470578602'),(854, '42682187921-09333153655-69326153876-48192480342-63348719007-68317338251-48954879470-76098420955-08539634355-11596436919', '98471974300-72043726335-52501797568-37134487554-18933192878'),(39, '22708576232-92979722900-14660200913-98837526562-69122690845-07426253364-06562981372-92478636812-72736397176-94207056534', '10624910505-72714677269-21372747759-01167762129-12735807236'),(270, '30707983652-05592076082-25895195703-89360513610-89272686179-31973745971-00720731078-03147186803-82047075427-38081222754', '77993952459-48779056775-65160057087-17510573230-45146421664'),(752, '71178939564-61986321127-05518056013-46983555946-03679940305-96864905650-85127739243-40585093836-86496106425-75256937745', '42572984743-64723813065-98611580103-59913493399-12220910979'),(214, '55636022617-55984885848-71943796267-73543083637-69449771813-80370780941-78241936731-31870744959-31153498284-12237395877', '57850112739-95668586102-81658798318-31796020891-40926069961'),(757, '10923999163-62456209892-81857278263-70143250239-15606134521-44083140664-95363395979-86940476660-36354495334-20191950871', '82717304796-41029052399-21412402204-35610683286-07059026822'),(46, '13832266544-54924860176-52009114478-48578374278-59559422656-81996543408-74426065693-02148581443-71986982342-12200286154', '19358162861-10795678404-47426458051-46764960233-52880211987'),(352, '50421227177-66313729012-84800122898-17589641308-83210134303-57245245662-84582139166-65388732569-02643847468-22046847042', '33006691413-34496612858-69491722321-67577296208-03525017498'),(526, '24374708598-60526737335-41639277970-45569293800-82896181833-51775256153-68315266016-03914045989-19442924186-38194995645', '23428622188-45236023984-20650699718-59701011682-72367968712'),(933, '11296321669-03597635816-09064628485-66299541590-34449342586-05002664293-01518752837-34882467363-85524446649-03847685622', '39926522608-75593556598-13968659841-12550038902-85972215005'),(22, '65571967167-49305176301-37972608086-67065412804-26250418697-06939744173-97536036881-59886316063-69008422496-92031246418', '07356759576-04377368780-48781033656-61622665741-38400852323'),(611, '32517516048-14597050546-65213934969-14501637043-58281918884-27786554550-78062847764-72895749345-03999754927-05425777725', '12546106939-79030553833-80795658844-27950514708-50473069685'),(127, '55639956038-45890868477-01842547846-71800827834-74460081435-90415832204-09393377735-27176408410-26302298341-10429283466', '28438657875-26870809773-47644984453-13636600779-06149917923'),(28, '08016030547-98815627423-27086919761-48989051901-47759019021-10795501790-22479028038-50376884568-48668871979-76988313544', '75646832333-05562214211-59909843039-18565693434-93785238947'),(336, '01300675253-25989066262-32319399586-82358133519-22401355009-28750091708-91301771728-12779486175-28978872985-74650188480', '41868128001-01794420965-83778110735-07673043452-62671196838'),(25, '11165250365-75317820485-56254989127-06575447498-20583165742-86110818757-65535876327-45881972151-47949800340-28221482968', '46780265042-67495189972-46506728829-38828871696-43425708055'),(643, '15967969602-78009534020-46265194058-40324015158-46460126315-08548098094-80007369023-27128860548-70141397329-96163005173', '39669293820-24912323814-34223505656-26753877309-13862832587'),(639, '61193328856-42407318410-64011810225-63038772840-69089923800-69377872766-97331408027-15511091611-52005940871-10377291243', '88714744963-81528762393-80143568030-07271525690-69458642738'),(409, '54120489991-96823506976-08377697808-35277700939-30252740126-33601444998-98136516340-40720841758-43889077857-62427149079', '94647246822-75668008637-67699480909-51027234450-77143755848'),(579, '97001210895-16716920934-07358417380-26726913031-45007117445-02014094300-04821191162-92008614500-90985860510-32524589956', '77233448372-05753550346-23156426140-87400607286-48000836565'),(853, '67519673891-79121082774-88458325390-31080662576-30252357729-32941544384-18106627558-10962968032-94985739921-99572514536', '63263226964-68022100061-05070750236-87607509177-98257300388'),(290, '71069937072-47060207459-67747755073-01941177772-81470984643-41734424655-55660875871-64188056064-39284883388-14468035474', '78547838620-43535220809-60261383056-84956545047-58114176180'),(200, '88763635220-20884460620-36896181278-14510947876-85236374788-25271072036-80112912550-16243682887-29338493311-31552282337', '06281248760-86712150669-81315919186-91881766540-29615057258'),(222, '98904449818-60263910819-17586356846-72668214611-92646055591-11396179897-48552526934-35243005370-71023863785-26256920182', '46675431726-86419774923-50175581015-49050838534-79385080475'),(640, '74691174676-62936769003-44086568916-18448904526-62291298996-47818364319-44621599588-32455670517-46830563185-70790918511', '84046505561-49325646945-41156374283-26758255021-29130198890'),(645, '91638189693-86796082010-28955250335-79631996988-80609800396-40964408158-07944503996-85499647462-12429303347-48801080524', '21910325260-47762184696-59321889370-62919536138-12067503413'),(155, '52337951630-49943307357-91147850685-87842187567-89420377810-73014547129-00359539134-94483602420-99343552472-95712871920', '79281844838-83003773048-57424027086-02352897083-72907847074'),(293, '16915984303-57448700400-91776903132-18209803214-07927130344-77900249870-51788335558-79612016279-10432218408-76152658913', '95559049901-82171953523-68970632040-97161679545-50076290399'),(739, '21590460676-42248850703-94925142646-57425637189-47355808517-04104496021-31977459747-14638126960-87212472043-42074181945', '45295355896-39586904006-59300128236-06608433928-24667161516'),(595, '16079097686-58054635862-01152052013-40467836689-54055912782-84542140831-44919306158-94484775157-17075866904-30455306554', '07436944582-08037067664-13189624247-19412661969-48150948808'),(441, '45381712113-27386841333-34572348773-25827236852-75807745051-43086014653-59449764348-71234239681-19297042034-12573458541', '96555609780-90050383241-49011498710-46359678603-70029594366'),(433, '12455514316-65583967608-34602673881-58841400024-00133307960-15790337885-26520912193-29851691222-95878366132-28859946336', '99500743565-93262213644-82718229594-36429835574-16458066383'),(214, '53188828302-49569859282-19206363327-46428997999-20358907060-07802053698-00877169374-61748651972-47477651695-93121542175', '96680788568-10814505293-43049032612-35051649009-54489445564'),(33, '30313558444-96378872736-09861986629-74852458937-75923359663-86150760942-25692716815-20632872635-08331299517-73639859023', '80393303626-73520830488-73613143551-81380370596-75037819952'),(190, '86383720237-03116953064-32668172038-17584678933-04467409659-75157883718-68954418845-14290794597-47873284633-68740420917', '46799504796-76426226199-39676629392-25073389823-77184539021'),(648, '45404810760-15246167246-47774891419-54126159610-45953842903-26566125690-71866297482-84513983994-78461794594-98631076095', '50138968682-99242733815-78475419304-79134152225-67687922297'),(870, '87697261658-17559977945-56223504918-52380757699-90671446914-83338058277-56849199063-65371883997-36669598967-79347237201', '07301644888-42094343006-74288522653-62504805149-63314004238'),(730, '45136504585-61938096769-61679381516-26628663103-53574199649-93757935884-32770455200-27636424731-65424394518-94098447777', '85643656229-41536870081-02267728192-78743261136-75824766052'),(891, '30308252682-06283522175-67902706867-95966094805-28999420394-46808248134-06401085028-41395189068-00633223126-93636231108', '86232323422-48745011360-06194245039-33818719935-78110439894'),(825, '64219961948-10106774621-93951257637-55496996399-97937280742-62140129159-58769024448-81049341410-39833509187-40566209827', '96694152671-73275428503-91656282620-27214636219-34071860320'),(227, '87362051055-60373185500-71087786925-02690352356-50586308319-06956055081-70152118862-57371395801-88051503677-93353150364', '34748514379-15541086271-32391095560-28402149030-13481762512'),(562, '91535512693-60864150432-19910091705-69271944991-35551495759-13173331357-07398353622-39552805676-35850230666-40497448275', '79873413393-92660820582-76148474723-57464846184-82954519017'),(95, '84619012702-76689635643-17482119520-74107178123-34123970169-79760965599-70221102492-84950711871-08756636493-04270668313', '18928916572-69562296188-75981236712-35154598105-31494626273'),(269, '65394189087-77203122491-22449281706-75863058355-63344410620-60198218469-80629073478-77602325477-38298744909-79273638052', '84578664386-75533426501-65239238774-91333014229-55607575003'),(41, '06907688570-65812933246-55485404005-74272747696-66249413223-96176220869-99747585872-57376406160-56916665979-51457529282', '90875426495-37925911061-18708549767-91443660289-39355132818'),(391, '15982269997-85452048224-30878914768-10169791604-13951779968-36163866759-47184812873-32234587581-55509175799-21522566496', '50055242034-51338956180-39643170407-17654889668-65960060487'),(134, '90091047146-88678190837-78280582816-03392002095-22710393228-01140158993-73859720408-17755235561-97805513411-07318782143', '36310139846-18816420355-96229479586-54836685013-23239943175'),(58, '35867524122-71117885362-06899946074-99140845836-05831175297-24084666413-01687239459-01778477009-78320930928-65607596366', '46965385812-61691382879-04883553561-42030053890-50802592773'),(203, '03458474530-80875043588-83908154712-82220654239-73792406134-19481736981-77153526872-32344516194-04014591329-64703527492', '30341111879-53857138605-88851508566-42334309417-11782890879'),(315, '44814618838-98322687269-19027278152-97126057606-26207406474-96693460005-26285631395-74918172028-22478511471-16826951693', '13919332016-78398109634-07042852913-31834140851-66985147063'),(70, '44895688261-04200538094-90297548873-31458778216-75470936246-37351682477-54121389464-20330899142-84520190905-83430472032', '32749894144-27669422775-00524810168-57188660496-35809566347'),(657, '09284207701-56520570284-02088443370-40551174062-65473131557-25947191563-32453254793-87208257112-30397233881-51719549160', '36221695628-46410927622-80002531728-52309791338-57147076791'),(786, '45922863033-29975022739-78532400107-69535571433-64235308792-79538408739-54378893044-48496056635-15538690555-24910345880', '43147662529-83533836870-19100031671-53650111240-40706923310'),(218, '66165490282-83710912131-23685758964-24796639748-27275218624-97423205719-81780865810-41432863701-68813588559-90160197028', '21007415114-10151448428-22640902737-27931829276-33666357913'),(607, '53543561685-75845045099-31985976632-54988703813-62644574665-50083126453-54183906176-90038790522-83730071484-25166259478', '65684246915-08273396463-57800272169-60571921580-86569788999'),(235, '46009775381-40009026913-06061490478-49538434039-55433053624-99603504523-44844162884-58045431465-96910277417-63279867775', '06528947288-74823590168-24259276923-34513238653-61245311862'),(637, '84799357060-08246296095-89499266454-17076407426-31859220564-74748953168-38063371408-21307958592-29508682054-14728354361', '62054552009-85570869611-80828838129-72545575584-05198066223'),(85, '38505051551-69832576748-49925789573-52299332075-60842713721-73539179290-75642472060-72288527617-82691567875-60462903577', '47576064667-47216390439-20963312312-38472650187-89191065661'),(249, '15642583632-50714417850-18558868597-70714958584-82231079577-80411640881-66634834243-76823262910-64093045793-13763978742', '07069922640-08115750983-90180029592-80863319638-44970935170'),(121, '10773230219-37187051373-46505170448-93685450504-56561471426-84916973337-52976183802-87725231892-08447833154-97702315429', '70534525654-31702813562-68235394798-28443173126-93856865544'),(622, '29747670170-22446123034-22741195137-51634870266-50340232143-68248189484-89482144826-86047391060-08593704239-71016792853', '32212055134-98353664568-46906825151-98247802524-26987266417'),(295, '15564706983-30039935661-97679153004-13846993792-20684637709-91725548027-73606633555-55649556481-25182447147-97539902751', '75121087264-30680914051-05214396712-97924214587-69506820729'),(350, '86302150599-75246605759-73007679011-47289364223-49574462991-15341670683-52883519944-06649360197-11331531853-91056553102', '88783140279-30508876520-66370896178-34639556671-64464158851'),(644, '88051117471-75719192977-89591055324-92958958268-02708867590-17565123571-96717992068-61485897426-84879683611-66847333202', '97261064295-37675488196-64654881986-96057005057-07768408102'),(622, '17969532628-80224635487-59379948168-75517627117-33034777368-60703951303-47719945727-88739464864-30724112436-69243179656', '42300455221-98172648433-96117120506-16004241582-71720819463'),(168, '77526166800-48870942198-55930748915-62188939868-60136932484-61167371417-89989076864-02984035481-35775062884-94963103708', '53464309684-58745853419-21498752216-04782806685-09307236711'),(144, '45950170742-04290329420-85547105541-83921665112-06951338046-84350729668-19771294040-98778189900-97735928733-78379052124', '97689006217-49535518040-69025364058-10952469381-26138381902'),(51, '15100726330-65214967315-92009111562-23224267895-60639812480-75051303611-69341406784-97884697460-25694101971-99560761717', '75207946707-60441690569-63816630149-23290950507-98615348230'),(381, '64075855988-23497534654-92590778917-53148676784-38547267306-16072837796-75075037568-22649495417-30012325977-44526883306', '90712046928-00204548230-56125382450-01718291661-58772192781'),(135, '12687845402-11591222221-42244497411-59977994935-44240717316-63028302869-54463925318-53911073122-97326680451-80802339875', '49804536999-35947541929-70053089314-71866559984-32705563703'),(822, '64401307673-32702680865-29049925486-70185390476-49458645885-17785401617-94908869389-66341275822-14479256383-44939888721', '00653634363-75779004367-86095935940-77623703758-69756597140'),(827, '53423218314-77868119295-48883848277-77588131932-23622736439-54210007565-55495089045-62775392986-53094946340-46654396528', '82349571899-16027567760-81229129406-84109949534-18784840589'),(42, '66176823439-56869195593-46268886504-10302709655-14538987632-05472227368-26390887412-42200082575-95599334376-69040383103', '90927550139-76263386353-15604677312-78394765122-34771275301'),(284, '82373707674-94745549380-63815331146-65188196169-48918434926-61299185050-43889490555-78528936559-15575642423-04716452712', '44973059588-07046094483-58830613271-52382431239-95478864085'),(565, '19306170777-77084267946-80536184656-23271765068-49385835563-49471778618-91244111731-73179984637-70441121546-20013731578', '89924600558-61887592476-83569510907-85053734912-23348220184'),(739, '69079340397-99950906143-73096391876-78039881822-85502486577-52810970925-72162940673-22095243524-20894988107-28126770094', '86096336916-32359524923-43863128337-99047353276-42968543904'),(243, '58694641865-59491370159-50051944270-15463534128-88750000944-11013643769-39886261801-36330056067-64711432706-17073744468', '59142750986-76299290656-48018266943-78747071462-75691437838'),(909, '65462753551-88621742846-23884277513-54943598742-77275903022-70135832955-16903346032-67855817143-67298195072-37815248284', '69347961276-70819496533-19189301516-73584785984-81315652650'),(783, '30934949325-12962596412-87818775905-37814331286-67161937987-32853550170-84561494080-82865082778-66253827060-99633708227', '33314736501-82871324591-94869450544-16514169284-18196079301'),(418, '59670431829-48900273743-74359269994-13914317569-17579219484-34024467707-95939210650-06340024534-19755810728-25176625355', '96142610679-81954738256-92988850708-80973327393-38916864129'),(908, '78474369681-87632607594-83866887820-01607872040-28686701080-56138646128-69541483215-95822086291-94765723200-36696070311', '90142322947-25226899020-66615740284-77262816763-94380476091'),(42, '26316483168-05692683928-62402999093-53723076473-44984135261-14261500501-96771331789-44512562106-27997797500-22189383169', '48727243415-68027968385-79422443431-91329856118-79531651922'),(786, '27515648273-07366206556-75542136490-57708302891-22918468822-27132191221-03762064072-95167540846-12758731730-12660106942', '23498932375-57428776552-70784342239-27288121657-56092072461'),(330, '46987988184-20174181774-93456503749-05717531734-41787928782-10575853789-08951034211-49258602812-17463020108-08103327888', '40316678090-79310731861-21414311728-37800705634-35057927659'),(858, '79520595388-46028029873-01549213820-40401281665-52391450434-68324885477-03060197924-40036343431-98266888466-00992778752', '19374732248-57129004263-14478195001-65631255705-69009002190'),(553, '82381706179-53125069435-45857015612-27559284461-89903414076-61245162051-10523040465-87524924643-82409716744-06019474138', '41976714775-27016860469-70217155185-25614786647-96516911129'),(992, '77889626313-65039617493-39506883496-66325319444-27062636008-75136373283-93997991315-78300250312-72548350264-45173927261', '96914673037-28042324442-86098799135-72565429128-81913140887'),(499, '84503833337-59643782839-39047934798-11885407288-42839902050-12452693952-73538837181-40719629079-26247056117-50437282650', '16234070685-05602726968-59498506133-70426488580-42742605739'),(811, '59284990362-48291237585-81600764408-53087744117-38240405789-23033456102-30898816010-95992003128-68792520360-99741089948', '13867896678-77139123188-38758930374-62430197984-17623516227'),(749, '32934904461-08622898221-80277671335-07319905934-60785524722-60280039873-89292703652-03547508746-38866272183-69159267094', '82242276094-65104413609-95012687307-33243699175-76593241147'),(390, '33530289915-11003238283-39490481191-12570479306-73090771650-77738194292-82673765972-98327260497-33343328872-91436056311', '03007375342-74069906779-47469416896-07558469551-86801767181'),(859, '16717924488-15231082425-54770322007-01500905344-68355196555-94529401294-30352264352-26955757760-91381673222-70182268952', '13853586838-36424564832-23010095198-99418968870-44349711146'),(944, '94223333615-73476185379-08438761262-03063570856-50812636998-64653451758-10359675964-95527643194-57601891580-49253487637', '44156261664-06725294660-08791333159-17588263905-91277791767'),(976, '13327171359-73975223339-09317707866-36364858327-26207049423-73086702993-12251396800-03265534357-55917511088-36838451935', '33522062690-74578854936-06782258980-90455279156-92829372618'),(720, '66785078827-68336668078-40071721519-38216530829-62133716702-38816486167-58619287366-88524638986-84877735816-91606176889', '07536381938-30015065173-61095524289-89090759202-69637454510'),(433, '62538765518-46521857187-55925250512-98810216410-23568819597-18215343311-85777814445-24577874001-67590519331-80999845022', '70625176179-71900800789-50210567127-77152326543-44910422464'),(566, '14866698899-38313557179-44628465953-84697202896-25514866380-59345860172-84130477816-20550075968-75387419324-28347229038', '13177088132-22082016593-66402114678-66792768853-42006340313'),(268, '72082498965-70532703697-59126630563-25254792248-48936239330-49182528248-09761247972-12498467124-67312780696-17549367626', '87398164773-62017023269-31053988635-38076291983-97388964245'),(865, '34719512678-00938129280-38541378698-65478037982-24211812255-61986405856-65204838739-64260289137-40872248445-51824942849', '74163741554-57984336870-78463254061-76647500616-58987177572'),(228, '51149383572-85683061868-39689267217-89704452726-06949331501-86786007074-51131373149-37400929256-00017928200-72904204961', '39861548162-42753188853-08931437629-36154024846-67302943322'),(17, '37424722863-07015288773-37614381506-79528500715-25457817152-93681862117-46361444921-61665013940-44514368903-74138841391', '37234786149-45784041662-80626640723-94929953290-30087093265'),(439, '20729618242-21416803444-31689633634-32718108398-63461204143-33359086278-24987659551-21545733156-75664892378-49916727703', '38950159311-21669514653-93706164906-16484565641-42686172624'),(599, '94860350527-35464985790-31730070436-83668119182-02790077891-82735693203-51586327036-18440983170-43502058030-83981402618', '87355170983-75861395572-24407754655-66664968620-04700193887'),(59, '72281661235-29848905860-32422095268-53742449586-92117796423-81464976240-90836647964-11906781573-46475729112-26036494133', '62741982231-19387405993-52733342381-03477199585-09635257038'),(64, '62387525611-38307496784-57699845781-03810575348-92936344711-56472762234-48928759979-89339845034-88752749381-92013840337', '81228206081-62075597759-29257902137-68921915568-70261419352'),(141, '12153472987-07099639517-39559123039-64895618787-58315289945-96752322712-81394605851-53278790482-07007516862-91632233596', '86093461502-56953238592-93582176937-25202958055-35419763623'),(726, '90544150860-92847657954-16591565829-95069125205-34351782143-92254126423-19739219237-46199061822-94613274860-87044671310', '67954360026-72839471418-77827175167-42213564354-39889613231'),(481, '17029130404-90712192891-12970958964-11277753572-25122080260-29020358991-76103615238-14297612979-90306932274-15540559725', '98201958375-44666410770-00068162028-42931688694-80976477643'),(304, '76743329892-00489968432-46984996865-90085611709-93791878583-86384655696-46337909372-10291906088-06564198664-19053124959', '53757260208-39884692127-71122877855-92635943859-60457475228'),(292, '16930521679-85949678454-89203437021-04207956182-77510309203-93656375866-66736580154-82419451612-62725711791-69205303914', '24740214085-10831805471-09487189845-07344679341-58675129335'),(688, '51541351969-38751889252-46674210836-80371996106-70120484786-68449144713-71047244671-03527586881-44869205954-57075407546', '08616915042-56812523678-40924072258-70919414480-89175804704'),(786, '29122431609-24970403807-82228097284-27797399143-84781894286-57439962970-53460545635-05350506456-31364585630-92224449106', '87938204384-21574452012-60222577936-26334408934-92500480558'),(98, '20998562278-10657020476-01418604387-47571997810-78489675776-20507248031-74697788757-12978646483-40244541881-93103925006', '15215464867-83291036957-98045369395-77367221864-72197615808'),(133, '93489726291-98684241017-63912348781-61959736848-08676565747-27297915855-78931538204-15538134108-45784403137-52341129780', '47721314259-38049208108-82399338099-72519469914-77366902367'),(182, '11021458155-20814773766-10805943829-20570440438-83679329955-69751656164-11423455418-28917907209-33916549098-03692883517', '21436363022-37578341006-53267278047-30157026161-65436022011'),(829, '54222125347-87974308583-86146791141-34959921880-60969814570-17637924169-43611635697-29604967414-67616157452-95343419689', '85411353262-81952279307-82656584537-86921090627-63482294495'),(363, '77304077587-08808358351-81752399797-33429432236-50483883980-09306380585-32077836936-79547176317-97165039677-50131915928', '06092798071-47190806041-71656710734-29590966023-91945484176'),(720, '61499797912-95345459303-47902826265-96211428221-69870576166-97933035196-95562106612-19984840185-77961913975-68557777664', '86206556925-94372511936-41152617219-91583953064-39252686979'),(686, '97083540076-10544325655-94766996454-73155884997-22177912997-70906096068-67439646105-98680039833-76623955113-33313087305', '87524920150-26567862913-44531927263-11297187653-68929445805'),(538, '12437392301-76948177250-94568194140-53632062796-22547649730-46955633034-41842869138-13981050485-38656493076-44940759328', '97676387635-27398188269-78203577336-62243701452-53164514947'),(870, '39987692326-64859792375-05763187584-03368705122-80059243229-54101297508-77993082718-18879321924-53125376082-48831162172', '32457669482-20290757483-51963166352-40562104765-15324052602'),(179, '84031204267-69671520210-58718468696-82211214987-99127186656-36779059268-39059558450-67776938510-71042694568-71680550856', '56975608747-61951875703-63653286220-95299703942-14674245558'),(319, '77760113605-10056565977-31956112381-13422126819-85702979293-65334377975-32624576157-64466425071-33670582599-23299274140', '37920279163-89174622908-48058593049-36799307737-69330096539'),(961, '66916197303-66027018550-46676905887-96195582044-37036768219-56635152134-66444640580-19706934316-28838939743-20962985194', '37722385794-12993346451-71003998811-37538676064-11037118767'),(584, '53586492569-38559761938-91406805327-04524648532-72855782951-81403672658-11145636919-58021339051-96440098402-73010828746', '15486507078-85006535035-24437065897-16873992537-76657290209'),(331, '29568309809-32127609896-17814581303-58481458906-47973593224-88403257700-92460471512-29488423099-01728406675-33362212352', '11493001067-45811865503-03727647114-85443943800-84280845683'),(363, '21448210513-88854364312-59889760404-62834862476-27806659125-84718704276-85708698421-12820559895-47109746985-47005162481', '37542847705-03154407068-43152609085-14298276384-41120095145'),(439, '87632056333-02346159866-66859090897-90841567079-25701236098-06495630939-87486077178-52271740666-69739854220-88439449503', '39320428337-47974184786-69979667337-23714555738-13141605578'),(982, '19313673708-12628795125-62124709362-65761180447-76459009149-26745581602-55595713454-00937531923-71308556122-36171514222', '59991111112-15784236608-26822368340-64348020606-14571893611'),(253, '82848800075-82955221313-85625811766-09903222556-34611001406-15518775088-11599022588-58788868812-51442494789-23255938567', '53227726945-49733081684-89493433095-25149335817-09629805954'),(791, '68327322730-30198041048-01592194356-20742877633-56483881096-78209599670-25942768079-78773388612-41342556938-52680155358', '47131364979-25971446501-94630857171-39466347339-68930815752'),(749, '32941339450-05468000153-21722530887-28014402285-44600967348-28700432928-13397153249-63888317515-53987348729-85989043810', '18485754041-73932144171-74225199352-28217179790-34894843105'),(700, '82023388981-01821936579-82155057456-03113808960-35121329938-81853270874-49861007705-18370994510-56080167014-78101301045', '84779402358-83713142217-55802516511-34807470794-21679509985'),(489, '31780236865-73594679371-01891820391-54101702719-05026252990-24459786159-42500227392-09308395541-22574898862-71389140013', '76656109928-04235157543-73165438269-83854986867-80892771597'),(98, '72005021421-25686974662-76373448035-02925077145-65415682690-21178564612-23923771372-81069700700-78399485497-33481787604', '67293291632-31941977906-73677582902-22217523943-70861736667'),(481, '12169949471-72883744181-92999249689-10409833178-92257870258-29795536325-64747677330-02721387893-60136243240-52963742520', '41705074902-73145526528-88576990261-18237204207-93234616177'),(564, '93815746746-09526652748-35158020280-76320675163-64484741742-77726469075-74331519894-46968132550-34417678672-49063513211', '27873247473-57444382716-73173514547-27506870994-11647545425'),(961, '02689134602-73721312256-74094551016-04527085910-48490532387-81456492097-23743419006-82060404767-11593678751-97606507105', '17140142877-12568098330-83632345934-45843982031-82724776323'),(71, '42293124003-46912285509-35899857764-70719596290-93683481423-61487693761-03706223336-30254132747-41539548219-57521120416', '95221313182-81924054776-36622984823-56245985776-71824872041'),(978, '23812656545-58504897387-17767050292-66097519385-54581543650-46765335690-02810583908-93630855966-43817336646-44327596116', '59966842718-04841965599-03887699871-07830148122-06577929034'),(895, '97193214516-98768072660-40811038684-26845112370-53728969748-73026259799-33558235682-72333437808-20274159826-80849380994', '99505572228-21455237008-49171096314-64439078790-71612457099'),(897, '94493365037-80422215733-21118238061-77364437801-26156026787-08175672444-86200645633-33096164788-04210390507-27682005504', '15737057038-89571946830-96056366589-70031136626-79258746336'),(214, '70320943158-09007490382-69796227532-83900393168-51938194717-24947476042-00664446339-73918771491-80824514572-07473969604', '98965610406-48395008641-68899929754-86673648478-25039856572'),(968, '87644029856-97427719472-22961829071-54408507970-70977092214-58867710884-59763681370-20999812566-40400141192-71955791291', '54846780094-03057651860-55020457238-28023994520-48168209360'),(905, '28204406845-10343134278-71856915529-32353414899-30659703665-01417021560-26029003962-05377108165-95050450952-36369950358', '80649873169-81540297164-67956297425-49628043360-50540745270'),(436, '16231212779-26308543826-11047599436-90551721962-71838270365-06462338917-87930222801-20235413646-88082239492-62221644536', '37715919543-84632674745-96476202735-74293068559-31481683301'),(626, '00595634310-91021932195-00295518925-32844959061-87092332938-11344425183-38980253433-52554055843-37183170455-11165742251', '34403763173-30945953377-00803474566-61644102781-17662712458'),(163, '10475316644-04085329437-43503726346-66980220687-31742637572-09558304246-19935932314-36961217399-22869162726-79023370516', '74975878910-61888900680-43543824978-37176788273-48762148113'),(755, '50921557746-81140483647-89486738558-00173898101-37065929439-56314833185-21380968169-74290688311-74014836923-80496429321', '67895672919-14329466367-80708024808-71462632348-91266166385'),(609, '53822630625-62422491779-09044660571-71338977650-31625624030-33716554219-05207084582-67528506672-46414887775-41814293266', '36988119217-09902890459-95050337490-62148861694-38460553383'),(456, '58711742271-16005802230-76283664787-08213593928-78910602597-35643969320-01494313499-85217735648-77790967791-39509451972', '28840569754-07412079152-74059917904-88984471287-49723131081'),(721, '43456762277-00882094413-32013533588-52464597678-44386905418-16246495155-48885804827-43357761411-45574613852-50955804223', '93175374758-93239969232-42359616365-08895589976-18496650581'),(177, '49480145753-63166404442-38520430139-98531202063-57255664039-29143689222-87196577538-84267321691-98813775322-81313812107', '03385227416-92999378275-20610780443-74310543355-11903084120'),(897, '32175668794-70375624488-77666881709-30125093945-88060912043-12282288822-05647033627-35696550983-31090404682-83154579810', '33666214791-28055170647-40653518853-45455886128-31429042143'),(884, '32496404183-70580819061-52988008303-95942978979-47187364064-35023429013-70228265321-65884600797-72742168293-87176673501', '48821771090-13604654287-88006013030-39339011659-34742016657'),(72, '08109376442-05581422205-60566122766-81731002235-77366200738-46552668082-62919093994-66071063559-14741833814-94331740432', '87499214943-70153091646-98163805266-43712101788-40325792539'),(235, '71892171482-04264867267-05359428694-95128065113-27378936131-68577069398-95156022147-25942921416-21472280617-94763726658', '40466671587-70500443358-99724847822-49551938894-51024242148'),(398, '68863397415-66695810491-26651893434-37378617829-12707193468-11119326109-22649614563-06860174621-76706672835-09917691630', '75975244273-89815201314-77710712700-90244587660-60729177632'),(145, '68676187192-58432420107-60027800986-85211063253-93555066654-00150982539-96947377444-98787942946-53333561315-95468074192', '16038150407-14250990505-73977868758-09339650404-10049660816'),(866, '50792262475-86024002208-90322653052-61571160882-15512558031-30770387678-94184620900-73219211099-94646563389-59965548056', '81020778068-82473568745-39049875615-11577747066-97405988711'),(509, '63344185739-58748121698-25186387467-84687860779-82858423765-78938138240-75698854654-56254537951-89053628535-97502020057', '32493031912-21618906259-75168187249-63009281048-26160811505'),(789, '58793449884-75494464424-06543222415-31436786452-44952549311-38789174729-42340545218-20112190454-02922278713-60389417886', '99489974070-54798897074-65961191180-79447683927-53991324663'),(255, '80652130697-05641566468-44422868052-68505073729-68832897355-46298158974-76284603182-61159277302-08854546194-25888885444', '78402904827-93803424438-66727769815-55985517122-42033587268'),(304, '29145152273-66093626271-43595975739-15325164271-84740973477-80086102204-89747154783-26350772993-19671708117-02908575306', '65677708393-73782169781-66964546260-68457803007-81629383115'),(339, '41278658148-77396990670-64233544349-41492791258-42388571138-67323081258-44385151968-79454093122-76929586960-48225973431', '99076707327-88459128914-28673150712-87918102929-11482078433'),(257, '63297051309-35927199378-90645144092-39291954051-09814841375-90022400593-64994440535-82898227090-87602499107-21371926556', '59854780516-79107159646-20857961282-75726348771-26598340867'),(181, '75025054293-52033622124-79944520365-19344007381-71272134515-41556215059-47844482475-61677919968-36830562172-92301838763', '85671371153-75719434060-57335689867-14245733223-73175829113'),(202, '77687377226-55186812376-55025722469-04039319191-30260040687-32440714987-76808765736-93330639476-41529510847-59764340887', '65606680988-16036453701-29283944627-72444548096-67689272137'),(208, '68682482254-13637755912-21171556317-97200359865-19682354650-36038528530-93812007646-31010871927-37779576809-07231441015', '16986014522-63627201920-32038876634-43878088577-00697486347'),(614, '27317759617-77398667123-51650432258-64614831347-88590996027-94737769448-50246120705-58819652859-57927597198-68848475814', '00983561247-29763129425-39134309571-30367136047-88323751405'),(546, '39867913420-61549741420-92614708761-86904797375-74241893067-47047591048-58240391123-92961293568-44958529784-50739039289', '89548863483-06898292340-88820721126-55492457645-67702859610'),(574, '46226460075-02455772984-65876621294-86181657028-06631529435-17559294543-33847975232-39672964424-35714062185-72705740462', '59934694997-47032867236-78766450695-72646805569-38440900439'),(467, '88264555448-41913883470-68377302409-03589650137-30829655266-51172029123-19838546778-30692397842-65698090547-10799130865', '13941671579-00612813901-87629291599-95663692329-36452407554'),(883, '74731251512-50098275567-18428443229-21222671404-88743457214-87324878076-05424087330-80994813291-18740833413-28524994200', '00608022890-92817482471-47638742079-73736203700-41568702058'),(14, '49488719258-30650071989-85617478621-62183648309-51832544572-51228091742-15716645663-40175649729-33022483304-08019207951', '62063357711-02406111614-00345565720-45027970127-60067640684'),(984, '34846333948-11276010449-28256675308-41828789856-39317879881-39112372338-96924702453-86116236440-58017607714-89850061508', '95536862308-44259645910-98232724691-41856566107-17973659657'),(581, '70261362588-17285735501-51776772958-59167506739-60612851873-80810782761-84816838818-35383172375-29053987817-85441959342', '17802999599-45884969492-57770116650-96566942152-32188652695'),(550, '89578846406-21169711872-16252122816-52023949975-15028692941-09111217536-50947772286-22139131578-97715447688-05790846091', '19116742858-57408570683-85601187142-37523033658-54957102007'),(805, '59090788355-57244613685-32460077994-00643916818-55704122434-89501165733-61264025010-48174001289-65059952319-97332151964', '13381821184-15423666315-58588441542-39312720387-07272126918'),(46, '83396245620-61491249494-32471559712-98920824244-46022441730-65429453946-14521401176-48996874584-91899910193-92286785937', '02128381733-76966081745-88802986287-08057238208-76129876135'),(305, '52484761020-61676093803-45905925599-17091682554-90896837031-35792225101-99920554341-11515256405-28428877496-94465240632', '24579242614-51062238590-89287601768-63212295762-00582130497'),(431, '77911947398-51109503192-22350633443-76956723673-00398281957-90159711881-77310531110-28230839691-54977276136-00996040643', '11395300431-09168811202-88091688185-34666982806-65684362235'),(246, '64079543457-75530495422-73276072230-47093104577-01875033207-49995087218-64367974029-23624279895-43252354570-25452073098', '92945963978-38202371285-23013418450-28359564431-29145728433'),(706, '76875442980-24943228560-85182845064-32243094800-64283406516-80070213633-04570787481-67450824367-74320449228-29182548432', '86737910457-80007415115-31174452501-94290012596-23627722236'),(983, '85901829022-09612653842-10129160791-35075297136-97889802706-76620533912-47960120168-14228468673-22292466175-47785128321', '50845653641-76758555982-83527856790-43654558937-01534725877'),(298, '07787595048-67790034503-42972561879-35397937488-81840585845-77364843967-75328987642-22228951015-78893860636-89028105408', '27922907422-58990440686-24359809402-32461006104-37053481413'),(715, '11945557575-76533274557-98711860900-35185440859-16840041320-26077898662-87597634077-59081832455-76836920195-49513305052', '99059223328-87814374440-45291856741-09608679825-10019594190'),(706, '82035475483-04888957312-16660646239-90558062925-69085243510-82993074911-65406541935-92823465629-89781363494-64582034397', '38651073063-63813361153-94769473120-79606520822-23342415078'),(254, '09892895698-99319064277-64139339744-00177425801-38515618743-85890842967-27560985416-85268947887-39733788324-31408751627', '79445436897-68741764624-84971893690-85177893999-13629131195'),(105, '17787598683-61065982098-78949631469-21513577742-17975350098-22057313828-09358371038-98581079325-84143642327-23814223127', '50854124716-99799516645-81251946216-99314912088-25996346694'),(964, '87482371372-40550217434-11101000479-06963663057-08832035278-33692116851-22077418868-86330441905-98240846069-50955233551', '37208787468-71527232099-87405580842-89959557293-97868668468'),(71, '56297095331-33650667928-46856806686-22249369810-37044661789-18043883829-56973804896-80305490268-84256724461-01369040734', '72785816919-27871768386-94640477778-19936076560-58536358618'),(73, '36157643425-74943415855-20234030656-67665624865-48413781338-36831203628-47320825910-69447861013-25587667581-22154522449', '38238461694-00463732033-06031385589-61385543547-79483918927'),(921, '31606883425-40936074678-34274719873-40478576045-09106128472-94916205263-21120655666-84507520384-40263034008-46574291261', '02144569840-88450618592-32090584298-96952734221-62814806096'),(378, '81567528515-54774459817-01939608533-35595901297-94794550320-57773754697-88223837750-02077364070-36449832671-96982192401', '86427429897-34376402220-59237991169-23368585016-02029885419'),(546, '03788558174-68354569109-38929799035-43879044891-66441911344-32207932159-48096959190-52747458086-30320616934-31765353805', '99256309384-63593652466-66786284048-59469285586-61591596221'),(503, '98622050036-92646554212-91084143446-43609063193-39656392279-11243874801-06681781002-29523570063-90088259361-50186000713', '46868795872-34540898170-02617128791-53759104160-63751419517'),(610, '86845287522-28526042860-76430413572-26521901380-81434583688-71876348182-69087321079-98302618389-32325283553-44375351497', '12408641052-93941213063-54924311050-96044882507-14986116151'),(787, '34658322536-84642953161-67322996255-50616889040-85793904922-09789016057-52494298847-08535783833-98291435946-64935539588', '93305075478-44567698645-60624128496-84076330546-14557455039'),(738, '16125435704-95395710006-63132493654-01695671254-79940446892-01915506876-08683308717-39744961413-67375226148-20978516635', '43179483270-82073248411-00323098089-38388699143-88660070531'),(487, '01531149677-88488815175-94237179484-59179204292-54822028293-92799100531-08834266957-04967119397-56821926182-36102707072', '71097235723-87478103059-50748147096-64566727147-27486410067'),(659, '34264379052-50570221543-40432611837-94863706791-65963724388-20763748886-04951305414-08198500280-38360922923-47293814745', '22575893857-39816964010-82434865943-48149170268-51922241193'),(923, '15527031618-47508936219-04392288079-28711925066-05854392191-44810027957-64078496127-48093135045-30764691198-63894471724', '10762818645-13316603626-77111917291-81895442777-02254738066'),(267, '66170280521-30184972934-24068153373-27695261689-34650083267-99551873453-89942573254-97893582837-50447490074-30929253914', '11129380736-24269408571-46401057234-83154478283-00991692167'),(210, '67689121679-37001589496-58512804702-47352422404-04283516570-65486094928-41054476855-79751158499-97746220969-41627561679', '84616708686-78033900315-60459456322-98027365882-66714749519'),(207, '25428308942-38578738782-98546031520-29494575050-29866329926-78813181187-55399375118-88320206022-71146893626-42109787125', '19203868417-49152634578-98901186906-31417378143-90684647883'),(99, '57148177950-14296169431-80166090340-10291357390-95849718852-94440773703-46210888682-36188895861-07625523958-62979700082', '61968983472-53658980710-54893276178-50753412030-61101180931'),(123, '75820505191-38708835171-42873652218-07880817148-81789314212-24438596360-25857373538-39648203142-36918194619-18679795155', '68989671908-69387387350-28805974466-44563117901-93267243904'),(4, '90642701094-84034198553-99264893716-54228020711-04419039485-33383295566-26443210232-90324590746-62208143527-41148092290', '02603111275-19742041150-71019301863-60971066365-82022942868'),(181, '84576716454-16250022302-27403046124-81206055262-68243916367-78848413751-35139462408-25632150579-81217559232-63770908140', '06424091095-54666139552-40594371084-72135310524-01227663107'),(559, '92319109783-21904833004-85701499334-03340503530-14450866261-56942327526-55738713510-95828040721-37387700123-16684278108', '65727315813-73163474313-06618712924-47325918934-97203135887'),(980, '05131973747-49856788456-05696740888-88801967688-06295620377-02140661599-43975778911-71136464889-47944212811-54296073334', '61817178787-48601997939-01317536879-39446213940-44471966529'),(92, '02372711764-06148220495-08949586048-88386005013-48992271989-10129232863-89987662976-08796297733-65522057061-78841168270', '24974488605-82465211093-48698617199-93912686534-27955683903'),(530, '11790813954-03100446313-14437744736-09145511849-16674012958-55274863503-03879229766-33114154190-34714637233-03827203066', '36152260328-40408909480-98619930590-66644723400-86765758269'),(907, '40744389537-06061833420-67625016845-32125237262-48951127514-06395697217-59791902907-22537930335-07796334887-80932335986', '99396667231-27448526829-63766616428-99623508128-49378800436'),(428, '29899672068-72794287243-08404445188-28963425022-26413552317-76458623816-48255427092-07019070925-74754294389-96771917799', '05552877611-68708638229-79676784977-74762393137-52731133188'),(990, '96633175064-96029475087-72997528346-97393226767-30767930072-11257020050-09310664619-88346451263-23522688535-27445665723', '72931678082-23750262542-60485321115-85372830062-67659561817'),(207, '09417185334-22450059703-68000147984-27390122389-69068293221-77163490080-39151177644-26316599926-25672401331-93538730875', '51661395908-42914832564-09713822646-53852643276-76629644769'),(963, '64554059576-29002581793-89273229147-62356112226-38520840014-53250294590-03980922317-59295230683-66688863462-67043681431', '11438137578-46869760162-95884027177-89513599456-21002003886'),(493, '67248728945-58398988761-84609629616-55383482770-47143154700-26288078526-38137250414-31870374460-66154729375-35450974328', '30099577602-66212009136-67579322381-70407399622-08070564879'),(348, '21244518353-48839650484-34186605384-06723395471-83440515396-80483181147-14071537150-52740968280-59194012848-70844516287', '77943912622-19697692144-92163480220-95427910119-49160941798'),(256, '68871581860-98550178198-68864666079-18902203742-76998713513-34405534285-07598700040-28547536578-26077932938-89421971066', '68975855135-54350011723-07771320092-48855132505-54308426809'),(190, '79493524540-05332287716-12089319170-21673096386-61332474721-71754255515-26207314581-98602795820-09365837887-72987450470', '73683471738-78617592696-08567962011-31339066586-77826801162'),(41, '53407936500-38029147815-48062354538-95758511173-42077086528-81209107791-65676093574-35426703002-28742810576-20521666262', '15872195001-84581640884-00246381716-53126576697-48674067232'),(33, '09116353949-37155320136-03304564280-04944834896-38224960554-74518681190-64337543143-53769336723-51003390696-26626054753', '12291716951-27748774247-32922881619-61529263767-27872545223'),(859, '05452139266-97643647829-32047923344-59282577887-40777367513-17773682674-11811333725-14084472158-47381152078-50534253983', '48418425667-14016664201-21190401406-47331133569-37754061019'),(889, '60139220582-97070839745-39838876349-24854476692-18137971586-82436041740-18415919889-72979101691-97986525574-39731777910', '90373139543-60959407676-42663036341-96925291009-11589293047'),(169, '45567795468-09860279698-30758711147-92630237868-33326462031-83027641149-36762002845-63007965717-15153707455-23183648582', '25444271688-37076749477-45226475599-92512846281-08520513498'),(800, '60787571644-65266259813-11031570735-28320997278-83248848791-65052059625-80311835700-69328336080-26111118068-85445636910', '58831513623-65040127292-69595845128-81957907039-26072893604'),(254, '72856985321-60306661248-67808360477-17503845437-41110594101-99968375090-54466545566-25678558706-06182238745-74854492155', '36149904326-56863464089-91902952814-18887441347-03713318641'),(397, '82096053011-91996437192-99596012496-84189672650-41367932518-22705675115-12178064560-64463302761-85032420068-79706705314', '25243513111-94235081649-59590769678-21940302542-77054282123'),(359, '68092312439-10719353758-11768611693-53697954684-10049104222-75464821143-32688983315-37649173388-87616481359-49328784333', '20173505124-05701795243-43949695119-23311601303-71153008133'),(168, '33073108985-32313574144-79982791294-26304112005-28295644980-86009171487-56741637576-11295903325-72291914801-02464265044', '64731166808-74239710908-84225848039-93347167210-72990832125'),(52, '95054500094-86700755733-74446621544-34914577577-38249440088-96567420015-16665154606-26348763355-11593225710-19705308604', '03040145147-05008011243-00270477374-20410293702-22089613406'),(551, '66412577202-31460076261-36140207873-59355521334-39461827802-86909106997-74239462647-10799555941-68056797484-74072335494', '94469741276-86813110561-69315787193-78898260535-15179372474'),(310, '32913721652-86553749429-44418903252-63936161882-16699600253-91041886004-26819836224-42254492326-45042037728-01240236471', '62716348762-85789758062-98632318537-21742355230-93560762023'),(45, '38765143286-26427882414-69607691651-21946009801-29322874033-37647346689-34420485793-20307316538-80846348952-35007914217', '08104053431-46840659635-00225491885-29997043226-90454614054'),(832, '93319788522-12840660206-09723640622-99859411324-56827406341-91259414119-21886351874-28060300364-21176425699-83781245156', '57897979718-93574091860-22168844883-57701278082-31992665617'),(13, '21079457936-31803077786-53849090553-65246516956-74929723204-83392793511-79486957789-64638512806-22857180367-39740846724', '58029043044-66285975002-33131082956-51965917937-04420065267'),(515, '89255131510-40045011266-67084110923-82602591705-48043428091-68954693046-78043818408-00053525353-39838331998-02106260137', '12529479489-06882555182-71705527051-77964097633-14725817877'),(754, '13970669682-85898127929-29749698483-57616231013-42497498741-26853056761-42096938543-57272679243-89634552276-71848394396', '59374310649-41127266409-71275005269-68462410734-45842477581'),(181, '77333008719-78742002552-41284749978-88552509221-41351729137-70407725474-71433038468-48162411160-15575177764-73957824991', '71061858520-24255729180-46499005737-48767116749-80716992505'),(88, '87672916780-12047675168-13100468044-21245061555-14478405875-01888116195-61055575580-80883515206-68585032430-65414320797', '85256117278-62821149866-42750069037-63620146382-04407602186'),(588, '07127949486-36756271057-07376596298-42024361986-26099514303-50220044864-66268628825-23317176368-79336100740-87556869918', '80703820588-73946816873-18960908688-45499015549-82546573524'),(48, '66387887340-45348705817-46881274691-36273813990-03399563698-06102278091-55220157976-78149201127-49397544196-51674695632', '83110471633-59628489583-44550711005-78484109230-31329692472'),(876, '39817204207-81507395298-91213369197-80074716191-87516745874-41906016353-11400423991-85274239894-97347126556-81010380632', '14434899004-08633244682-76982812226-53441662055-67402072124'),(513, '22322754873-47919830856-12512874985-18072942115-97060375910-55840656551-43112223535-86149113138-29658252631-69281684499', '11447746803-17793034450-73526630938-32726444065-09558628800'),(226, '79547474365-42628963891-91379083612-94149482084-75273464527-70323822407-68562324747-49563641313-85779995857-19288211763', '47158197306-59163295614-10233159445-99012479096-63276156209'),(562, '53222060229-04635160833-96731668245-45606925248-41830307358-74088078971-44932484647-08681791981-58418587279-43158112535', '26635573924-57128396218-38585544082-53463372590-49711623265'),(347, '16685872445-77819761334-98196304400-62979589091-97441070692-51113844480-35705249329-10718855567-62337316282-75332406849', '67713483314-10276311408-50886254740-88811415584-10379213791'),(843, '85608844976-31423880802-17784811277-05735487574-45921891860-23251198514-11431247297-53862827603-31979337312-43538909510', '85170364103-26427967495-68876111717-72381981204-95072653173'),(358, '30824152188-97414173702-66145318530-62084535003-64629764002-51006707796-73576989513-07832796162-04271276006-27988811837', '53522188559-92000485577-69891275895-38410580292-83230790744'),(717, '05516938350-60466947583-63614860928-44096487021-78236437936-78774768516-01771688883-53190080639-94339002093-50131939322', '86770076010-72839023470-45743281751-16322854132-19016914804'),(748, '10929156656-14640871877-85677325825-75116684988-93048278260-91060985588-76527749900-37377508772-57507809356-59251728349', '44038950915-77568006546-47689607909-02322931979-51387683206'),(269, '62080643676-39332627261-25126345696-69163790960-80804137839-73303246940-89551168076-04695924110-25780503132-27300675881', '80771617108-01833992416-40452332574-91844701764-25426757964'),(521, '21729957603-84890414326-86935470324-06697108233-02605545517-64455003533-16288522848-62051199659-11633370273-83472261348', '39941754905-98961582096-36360288409-95231937115-77474611994'),(689, '32697279104-73783047577-66615504084-89516168733-66383724534-76958142211-09783763829-30640814490-55571072827-88177372309', '49510720811-10487264908-86969672837-44554749600-20221273267'),(921, '85576069194-07568004352-32942703380-44664139773-53409559965-05204149085-72950209890-99744295030-31063564595-28701651994', '24883577082-11798285745-78872129810-57342081470-46802213724'),(653, '71182954052-87915676369-75731768201-91096711268-09081407907-32265449348-09334279172-62002177878-98473065446-33340515352', '10271998883-27689409219-10415081405-46783198935-71919463582'),(559, '29055890016-02270320337-09017368424-34285837386-97450171981-72141332812-84842969871-76434497232-54512158797-25671101723', '23639773324-82081450449-61517004309-67820535053-44248205850'),(689, '83113515563-70322133437-26431468915-07040150961-34517344379-44622898286-48969462064-02888338119-10818128403-55450039831', '06766392470-38748607336-36322629985-03825251482-30030408808'),(890, '28931683147-22024674062-63332487947-36363567416-64982063615-52210486324-59673373322-95020702949-32597633369-45531283278', '22460296233-81301208504-09989394683-46603233650-48062751878'),(743, '20251127335-95621587964-76244816163-74993081845-27816294760-09076673767-12191076964-56151014310-44395320018-99100893048', '84354692511-16991324215-47052649520-69235952403-37894890362'),(104, '20619068423-50547343099-39449369337-12130198053-99393544520-26174816788-28142740397-68102211208-75367559740-97577827596', '10856185143-22192867682-74274237091-70507919511-38207317337'),(37, '08398434123-95556740588-89939147135-79970057113-37012118268-62607790926-92517073509-56705958052-78970208851-06428257917', '59396074337-30593639403-26157353300-55653807356-78201172170'),(617, '23439303329-14222937693-38987498925-47648416991-98372740767-89023061468-85351513575-76257457017-49507639562-43531320769', '57593209106-62338042132-69808169726-45111280733-24865792296'),(975, '58092416570-34450446863-99913642180-75022991996-98351930238-66895914139-36827282431-42746507275-81083993387-29729476305', '28593708531-78722207792-49076543546-75632455203-12324648586'),(411, '15152726549-53353769537-28002057908-93587421508-90345440540-48235218329-35022475436-39335994043-50721864512-41096894711', '11522929716-92177963556-28350278938-16818728629-81450043156'),(358, '40908518746-88575243836-63033817915-23943440401-55044503433-62532775356-54829354820-68064012244-52845199756-32864703157', '35987263620-10444905971-62354640794-15283142050-16068286709'),(423, '88126447150-78626167056-60558753301-59212455224-27197620829-06145027921-61217381751-15108417542-68641520718-89801990571', '69718275451-53624792556-34632770157-44282892583-97579722052'),(355, '78161502164-55469252641-56881362426-99677985677-56475584647-82343161962-09089088092-59915619301-62083558281-09748388161', '28858951005-31774759761-92843638236-29759438117-60830141872'),(894, '36808777730-35351673009-06648332421-80125208345-31333417297-54664555025-73265183796-87469768731-84575894754-73644561469', '70838671754-06992392753-33381849178-77558025653-95542187557'),(883, '67992440163-57192659823-16095393470-79951538798-63063761907-35723677962-58285414266-59538499094-34043060622-10521235604', '22955799085-23009983779-43734952759-13442795137-49895122695'),(194, '60289684097-16128541450-17138428779-49610491578-28665682084-44420771431-17981671102-12600058789-86469806170-38493020710', '47907783600-38244328684-63833222416-37201459412-96699039928'),(477, '46561549406-05507942854-10102720684-01320359781-12408369248-29038599496-40945649587-32846681484-43128207164-93744339472', '38039806598-70945989390-34335695056-12823275335-22236843837'),(108, '22516712783-24577962754-24915234217-36856694680-29430283605-96438871647-23210095863-77334581670-25225477637-84935816500', '80090088367-79090613924-07790226417-93758311167-84950259311'),(563, '67494968694-05719255161-51863196257-21040900211-76581667778-46491474235-43773479823-62229458822-00955736719-89117232879', '84405999653-00239499024-07619894673-30712239012-77526259529'),(730, '40801832729-86951579624-71733219512-84869349835-71343933368-97064717446-75202740555-92780068509-84533800040-73546929678', '86854898832-00548846968-62756320099-19551343271-55731847090'),(617, '24880272030-20361311003-40792641998-76649730879-51674323984-97572136617-34361749911-56635034599-82764614539-15028998266', '79710765101-52713221461-78248856449-24448949501-70497582356'),(898, '76110852905-01831742972-78087565238-21167230915-35153733266-99523553679-33933162269-84740531601-96225923254-94399233874', '45769787143-51890027717-23851492090-02455730624-06895434489'),(246, '39749426671-11916658821-01944433317-95772497848-58356849560-28402008942-52877729919-28784366194-71990266202-23772975274', '97202331039-65943920366-14637265975-27737583142-13513488770'),(181, '91507676958-88744475935-45398125651-12040520627-62207127921-44304162794-09881356272-12018110431-10560508920-31202916179', '20650061289-92808878253-52441629806-21928392080-47044061287'),(94, '44073641206-16347253631-37425874769-68167317243-01664047896-36832210849-68421603774-66483604589-54399276369-50950582755', '01793123118-29660189334-27250083691-54718251758-85010359296'),(741, '90235150301-42567638886-21125093102-17725131036-55863316355-49066947724-48235265246-44605674710-88681586282-49724499144', '77949872530-91177093562-54713946028-18913723509-19814709193'),(678, '25087106756-05358861945-28639810730-70293660170-58309876130-54681200436-23663683142-92420314539-18642450369-82863665005', '39869284856-57048363201-54479788494-88842253993-52056631753'),(414, '31914064645-08099207467-26964490520-11089050424-57567436212-14455579252-70591757534-68614910900-22646065508-47627072383', '75851551701-96935704290-04336607733-24270897419-82426146804'),(510, '76754538171-51867710569-79680104359-36285928840-03051532796-11002705757-31771682547-91996364136-59357127224-44356691797', '43870405601-37090522869-11595183570-44956672673-49594497287'),(216, '76531004094-51801789866-90949169311-01588040017-81670541236-89863833257-15011871499-78192208217-82904796727-48669649471', '09248828281-05176762248-93622926664-34367950309-45772676093'),(785, '66003794498-80831902385-50754321148-59464768036-48233126433-18419280866-74619415402-65793487832-47540616857-13394788754', '20381042086-43691131716-34869967898-18263422400-88353829489'),(766, '06134485884-13206757457-22931925476-91521063927-15463159640-21907590821-98663922825-22987721859-65536151665-67734419289', '67292441740-36596764837-32380787868-75657832842-75211509694'),(536, '13002509216-18976093448-53560891403-98314430126-50171771652-40533024199-88842805588-97614642030-10267682554-91272848594', '11096962504-85683373413-47909712678-63278399670-57287106589'),(521, '56581557745-89412930726-88308625804-16347388225-14193080611-57654908211-68579846661-86826631038-59961112604-95768123635', '15794072242-50589516473-37385753315-35704095285-74198715220'),(9, '34720142722-86774642262-36169926117-68525884917-00669004796-43964127988-13929926473-17450700675-77980881963-47923580993', '22016824166-15600613975-11048160412-78899260655-74870367887'),(513, '97002180064-71059686507-73579199911-53430203247-58650120683-74037173491-63840541822-55438625540-54800198346-01708155648', '83030727063-42587240678-95740048738-02675180231-28759197330'),(956, '40139723249-29672269262-53813203112-99591616809-14476455738-46322819976-91626081022-62566859577-37931877427-58734868208', '22916642125-61509832929-30994135759-39751358240-16771791055'),(848, '37787604269-69029099082-51754422176-20282531746-25097726803-09988492709-31449286298-57570377821-89393066731-69880511023', '61616189989-92719047615-20855634567-89414905355-89470059302'),(897, '71213202381-38507614453-23838237071-20579852108-12684119110-24620080978-94714061481-47656014923-80454729136-73066159543', '80341785702-61614499886-10291704038-27129014651-25107698306'),(291, '02081872791-10438887852-15663458163-47302440220-96591037766-81472852574-52822859661-15920658587-71315363224-75234776398', '26227770680-14794369158-26753190079-51569032536-78009756143'),(986, '51901700058-80874057188-23082044709-78378865508-33637408833-23495186415-89250245690-82738536371-26011868260-03838810581', '08297119946-52031040195-11394044256-43393988895-67081227717'),(535, '94773699754-69539872617-13876627693-64398418633-29396756702-67629905291-63650782570-33791588708-42677769868-70384217328', '62957600043-20092357377-84865036285-69302448960-75688961195'),(114, '04545715352-30701031388-39062174017-94133122992-19111930305-75698489198-10328772901-69716608766-54853884162-17267975067', '14186575096-32579177693-05959866964-91283198809-72806099362'),(458, '76644409278-29308090652-51849009884-88065026106-02828518665-63804292578-21669171723-78735868943-41888489801-83021774579', '09656739660-09706135299-71479220754-44597687667-53327977324'),(450, '73345625341-97816006274-56181524785-01670725681-51442092494-43886728031-36988181626-64400987620-22700578733-52048196675', '80023957736-04335285848-12393330205-28982658072-22113419391'),(145, '69924802617-32385918335-43027701503-91762643386-73447981782-07573524861-35895382388-41081331334-18716685188-00708711377', '34359907503-94780699794-60487120471-18305687039-37411151062'),(51, '00284458679-58760530072-11119515938-20632335538-21888943140-35437900707-76775160748-60435386728-66364651160-50806844315', '26709922940-03677519347-27493154221-45631144326-06968029147'),(505, '95755976675-78831294880-56065729321-60441515335-60756661404-86030032949-54216606593-19116411386-63516225203-05795713311', '60173904909-15097942118-12742013281-27652789209-31587558809'),(327, '16432291122-93176839322-79404747975-35936573623-73012917341-80881696284-80385172794-11412745782-09640843660-64893789734', '45157173798-82515969620-63669315590-01641535848-40824630044'),(142, '14352830899-98707568455-66367915291-24363054068-89858660435-71192559035-89236535781-49379620149-00935446185-82763754993', '07988083514-83907919400-92064766930-91132118853-55324049201'),(830, '24089838316-60090406867-83259782538-38605979309-65142002680-59705344339-61879290455-19086274901-10188929278-11667456389', '00992065916-39478436513-28713381033-81810952147-33151154968'),(124, '15446403096-77173216315-73731442250-59374136675-42402984655-81653501109-47141852282-46195976235-44890105023-82690608454', '56011801375-88384964831-63829610306-20370632669-90695797382'),(853, '83365814120-87419469442-88660287228-37111754262-23473268388-21139041028-92668407691-30393326494-54369686470-56614787851', '12269014395-83852957907-91381375543-79110420201-68265217298'),(833, '57664356042-01948913389-37170307658-86255420736-91251933874-36980880089-61321077597-91872561996-04340450386-60641050459', '04552436430-91513771950-54089865555-75362691868-32433414455'),(540, '31017648426-63853403436-03645878866-32404881051-56094870518-27671439344-15458182212-28044747740-67984340520-94828547820', '37757260162-31487105039-73319307460-00678953433-72863776099'),(813, '10323128127-73998955887-46799970755-74469533479-95416699031-51825936732-90701227979-87735046000-07838359663-65817931950', '25759144642-80904610591-68625383128-42888920957-83234862078'),(896, '75453209307-64224815154-78584418474-13900052669-80031223542-45587323817-10984852447-10935941584-38894401018-85539664269', '02866207081-44746059403-82865638573-94937925801-00074004880'),(921, '10612620335-86900047528-75852265924-86289333385-82110447678-28641486891-31937481762-42750109427-16969579575-45283372917', '39764968094-50516831590-99925438705-29496562733-73090704719'),(440, '13965852677-10709990549-73447426251-02821027211-58170752889-47362851876-42249583400-13585308069-07515592270-93130338531', '33313329805-76762201223-51678319990-43666423703-46982758704'),(320, '65714772113-49845456556-15712554202-80197255289-08352514565-42357620902-85694846582-20507370153-71763946842-63584234186', '30909413219-73554776860-68654821343-05686324250-66530013182'),(74, '51946033405-56469592643-12681117526-15390460592-06493392382-49884987588-54054475680-25980670585-10990687832-11458922430', '18236761951-10211237902-28108831461-78383184852-68658452590'),(893, '90697627693-54736242323-03944103307-06821604428-68038736361-75782888442-46213269398-77076670207-51524460827-74454764404', '41327346237-55941029671-80576085763-46614516106-51742964562'),(932, '73116490752-25971033774-28200888790-29035891805-56561607368-15690348349-31268026929-16358547271-91921812150-08944718325', '74312938394-44388116092-97345071692-93012927862-44436606678'),(569, '75716434830-49797087970-08151646141-35156830581-23015250375-10189817712-50941231823-64923262392-47267492938-18759239867', '74961566550-24244099315-45007944353-63209560362-68933545543'),(992, '63858420708-61190790007-11118670492-58288367679-95208686083-30603966120-78128798062-98765431715-43098872964-67666844134', '89959804786-57287125313-81340890538-81322298751-70238726982'),(208, '90269685150-49377015235-64046042432-71007986460-35875019923-06653342220-44390834480-51448119798-25175809609-46553280342', '81033945268-90345022035-18102956514-74492116667-40639982635'),(898, '40606750475-47715056513-03490819214-03271231492-89562653560-47839924927-67480395503-60031191501-55743795348-22847274676', '76112960338-26789527734-58433369693-72398253121-71824496179'),(696, '90513415426-63996962935-21295276008-45154936930-90181237903-84580451736-44797433127-12997273317-48603308873-59046279140', '89688704195-41356360497-24229240048-41864096344-79481200269'),(845, '61557084343-59247091641-20999935710-13349770851-81108612501-84711674878-80152731196-67694976967-28954995750-58205904984', '53903055017-18015946750-83387420295-48388117479-27780062335'),(539, '89899945572-77133393289-77165605163-21102728034-01076844286-06207756957-43081607795-46370052675-97194546094-81618415721', '91864788266-85682751939-67853586483-32870315425-53103926744'),(401, '00740663918-89068843604-18098812086-06711478959-04060739746-57940961990-92846605282-42323620465-39465752705-97790545367', '72131641973-05154636611-66357958691-48810795538-94278127702'),(782, '38199388308-39059460117-91410916365-50205338341-78805843302-05499933668-43025403804-84569435493-94735207260-68136006929', '04087119557-48496581107-47394674553-11129697480-63599584584'),(421, '58297498180-74850056054-57228824746-28474717748-31554524572-75263351893-78410691928-59981793606-46290577302-14620184035', '05341676941-43063639032-34115531208-12562674120-73163598975'),(724, '27236214331-61895969156-94308691249-03562499779-59993084781-53133424884-54122466840-52741230777-79507422096-41039114874', '78689823474-93117709449-76845415986-13169332215-16699601518'),(943, '82414322904-75822546407-12960892548-53659449082-84590074187-28310040198-92686794939-25702426121-18365237029-95897176232', '98944125703-63451453877-10923459901-87668312092-58544224699'),(782, '19655399601-76895445558-86832212016-80105793111-89574625738-93665342332-33146534156-20131482103-05865996059-20555993064', '64070676109-48606713382-36157514111-88380929234-68279587814'),(881, '67755528826-17031146436-70941213816-05679880237-02249156656-72154769737-34954486064-05784506318-33826203020-78345213265', '07504444449-15795625832-66030093229-45364813579-28831596180'),(474, '27309324038-50205496988-23958453492-57135624639-00532904021-21691382862-57978281389-94070330627-91209863970-38431123391', '05020510107-99908851248-39804320163-46183619238-19708583513'),(123, '92630458942-77402943496-04247353003-14597494615-58107796987-97028149625-57841069941-86917225946-41251268540-52958215574', '88033882935-45101392652-89317330320-89823661137-70703232657'),(286, '79424477777-77725527169-99755385202-07952125987-78355976752-99161469833-64497630820-64628838282-01576288819-12365929892', '06639669252-08221158190-14825538746-75997118818-81537365078'),(560, '61703222295-11068948136-67999137248-96924039422-87203691139-44415358175-73481254633-44181036146-16988789758-23078315394', '57467685033-72994411268-06966984848-31319455161-89108681164'),(987, '09895745895-38109491240-97255898034-78157738987-23640031251-21468538722-72538014325-90415922934-37079967296-45519747084', '38405989082-11987141398-47077717144-60616410098-96764609525'),(87, '39698240573-77851441471-46396429902-35824776391-45121104282-44437295969-61470656929-64988949015-71878315679-93490794528', '33191996073-71609268062-19277151802-78556399985-51561735836'),(973, '75195951890-69844543027-91566456372-12370913749-95983458939-15177537006-32698432733-93075411845-58916802789-50088273835', '00974601788-72115965571-58981806558-60763357359-19273736360'),(301, '27318875687-69462318352-58807689623-32442337222-04668535672-11008154898-08828208209-03058473531-57060985332-51377700115', '17749003820-00189847350-64572871599-79977668618-78010091806'),(236, '91279909389-53493372046-32447915713-57040269496-45116124043-02738502749-74732157230-15696091679-00532448944-37804882181', '97317854961-15597136634-31807898287-71563028255-85392224933'),(878, '59468018226-97844849650-01989522590-44404645496-27496319466-08117917920-48912020365-61222031442-00990987429-70654035362', '73169133690-26377848306-21272167107-04263154267-06594903631'),(861, '81754751068-25242986522-11461329001-23675996442-64292476198-64915715848-84379917809-02733788140-78445499217-00316322198', '39519661198-37596526457-00140551896-91454746327-28168601321'),(506, '58830372112-36022680419-83393684906-12828593570-62173640417-38704502703-17772831185-10688209445-28862717919-45296468365', '73560221162-82982148973-38592964230-37560489666-88483099639'),(229, '11891116814-50793516939-02563418006-07425995122-03766970450-04767299442-11695669630-59525765948-21435880219-98930218510', '52493413387-75182724453-62262176991-63178650630-62751959546'),(795, '55985884258-82955767857-55074064731-61692574206-84513331095-27314887362-00335754437-52675658731-70356712386-82063410960', '59704155080-60105106588-15446600604-50820146476-93647076151'),(977, '93737763042-25720429149-70446536979-28096176036-17920946258-23293648420-11210689592-96050608979-68215827051-63268114933', '14951516256-29145297827-39029544465-76089140002-25302573164'),(35, '13937130551-42236567795-89701925400-05001059489-72547715625-47493011939-01211079626-57483980559-53792528021-91353617700', '36360456197-67573810253-42766845097-40722772239-99316385600'),(640, '25399633221-77666143649-47832018421-14446371887-61506597714-98599993672-52437353938-54888616548-47636509419-05706294609', '90192709335-07686095526-43919553355-51095192616-79242848519'),(664, '45014704941-57265975319-92841673839-28683535715-33216304956-85704487759-53622385208-97910761619-43424226025-35981683145', '08777903350-73452633755-68648654966-92412581357-53480484298'),(157, '99905836096-72393191253-07634853043-34171040063-29064935068-69998513990-91344511504-10749216137-58710945749-91920248865', '33754008052-35320169134-01072452116-60874810007-56834057707'),(241, '16572544365-12927422448-38049525567-70406748215-54121979828-17662105239-84057081373-98217598953-52881329167-44015975987', '06142730090-21445290375-25892586508-33782862701-75301512081'),(791, '96134408923-99809597181-75597834907-10521385234-51327388780-28589794904-94987779277-61771890072-90516610257-77648008967', '64145433631-98592094364-47228974047-42457817004-98277969995'),(784, '09829421958-51828376375-57698043560-52569961278-65573628330-03971567109-98873341635-61943252772-26698031703-14004752608', '31359172180-46603114699-19898092674-10875737472-98331940566'),(590, '14601890656-00297732339-62515901146-82581746661-72110858758-33169543320-10657348527-64079005067-26336601456-69083433848', '73229678010-31790219849-04873237968-65480383391-83179142920'),(361, '59108802459-14938614990-47084037800-92170788847-62600550219-57044678896-64731620147-75684713386-11478365599-36822472412', '09934136111-96289595587-37358302888-77580014625-61663223040'),(700, '70165892781-65329977816-06990507701-17645574620-81166023492-37930221423-74265945556-67106519959-21789934415-82702672327', '86452174937-77603007720-89552166299-10051007245-28744001289'),(568, '92822042654-99145552241-49401248207-18991747962-64841533560-85047081954-14498512246-75816327361-67979941571-81186721443', '85901151630-86390185102-94514256728-84375175767-26634055490'),(529, '45564592372-07243810898-78397208308-29240535575-72526674231-44477655310-48675898769-20754152291-05350396097-99378237645', '54829050055-22773842160-25616426247-28732742065-20651565461'),(374, '86553471240-84458130667-63875442765-48452920792-41804889575-05197422865-97547472448-15482969678-20488870467-51747393760', '99615804772-72037339981-61682635802-37495538324-49691820765'),(400, '12145628081-08041560179-57584441144-64515544277-84693208001-76895080164-91573326306-95820650440-38659585482-99836496835', '44027733691-85679039188-98597198005-05782740704-91638895619'),(806, '63956598684-41800417263-95404437911-14770785054-33723913667-42091641518-09840175740-69266032096-15094096675-07858733551', '63497908073-63252287361-80800028696-38949186423-99603773595'),(545, '57578503554-74530992499-42754506859-77610663074-41201620795-24449836167-60080690325-65002499025-51519651562-19090742690', '04987136090-51633207148-38822423488-45634263700-56399280668'),(439, '48967402873-36329075227-86828541869-20037851721-65401544638-87667506616-42140655874-51921786375-90092372180-45380197838', '72864346959-62678853120-35097622920-91847943343-54955207549'),(308, '83315603484-53377130871-39459413446-90423264177-15541052357-26308053212-27120015136-87593360293-35346960296-20887532702', '79095273986-46960380932-15513733784-12929182618-08253913537'),(911, '85364576721-39939353923-25724371574-33769936696-73122338058-16328257142-42132549414-67410196765-22559622419-61867649176', '17313469903-29279508577-40776269089-13532596230-03785899137'),(461, '97871450593-21113477296-74656031607-25142902479-03313392966-81584101165-54091745225-82050403143-15354656880-58556931320', '25528079010-38676512027-26090660054-13509739290-70249235598'),(369, '36030665546-73501229642-94068795087-71049448918-51732572612-79933920567-62993929091-97365434003-89647157523-37543983360', '56750686234-46918557724-98330727566-75022305022-82000050441'),(968, '14031409897-55773285371-61330185076-09222191468-58832957627-34274891944-02711715241-59323418071-37421140218-04938203963', '65867824728-75062289456-86406806241-11931314756-97858850807'),(442, '10715306736-19169577456-12838705017-20526825697-60704881342-30168598688-22963976000-68960938359-57710108771-92174262156', '61872853362-04948084811-06149634182-57975283981-87097483925'),(242, '09975241816-22613346368-73887241655-17477083056-54261095885-97389803837-34328042608-35267783948-89621008777-26445847155', '14337933254-48846707208-18606251571-73007402670-99331074980'),(467, '45191598347-78472433831-42829538623-65599209048-74553787197-59231807173-75467981234-62516190279-85185537087-23146080398', '40052240908-43836594700-10561206252-86457936706-25757942759'),(535, '10615754103-08445145577-96681030884-24914226735-94504498848-02260646641-46690990200-96982870790-94384903437-66424803341', '23802705844-63336888920-41477932419-28484616145-94523704867'),(141, '92279396654-90041573588-43212966535-11612261636-81101676143-40164469845-15509131842-05571695986-55659936466-35515756046', '42628354389-73073292117-88853966530-33335096657-43406191062'),(640, '42406680142-31009429163-84946658627-89399674029-38633789183-35373032137-66983802899-12393119525-88349441430-76416238516', '89560834252-05124735704-45222197486-39768397102-31634034935'),(953, '10221901104-09758342554-28197745716-06736537760-58682926605-99118667387-52803205100-47021190852-17105259158-89580253963', '13189169416-51233704056-55909177007-58423049006-30132124128'),(679, '02257988695-88917213020-79819427546-10693808725-98745661706-59090834093-95722888514-83690806460-89628951102-16294165262', '61860569271-20133338693-77687262671-84412437538-69766773777'),(350, '64134667050-14501832745-13980927370-41949146357-65730444687-87432201649-67445299921-66071164602-54872688367-01871440039', '61600294026-14646715196-63665316492-87348634965-38927408048'),(618, '68904926608-65860448569-55422498954-07098518646-47614626622-49209791765-02393828941-15650208579-13762802218-17778125057', '87666631293-04566257566-65060859415-14614189731-80377786884'),(113, '71473075967-44105021515-27783937915-94065315770-86903766438-39603984222-34572177635-07277587844-87014441456-11256843176', '15195060936-66221680165-94985570827-06264441728-42745162451'),(375, '72004027135-45776761812-98488003844-40157342585-62569122582-85357306321-32066763739-87996880306-21364872292-65100404173', '46446865222-81055077831-39627391899-18961580700-13988778465'),(427, '65105297456-69844751170-33316815083-77709621720-24758380218-72212619531-88646811402-08010111263-61786453788-81498033831', '99345064178-65776917024-81052128950-51240010827-54174560998'),(228, '50952929082-40283662872-37150801355-89766573977-43707218573-60153090664-42808723129-27700586445-64750287808-41740530396', '05618718163-64504460880-24896861545-41304487609-93263703364'),(6, '78210579989-02325541727-55882152165-24896657483-97299046405-04642606779-50549413927-27750452921-31739153540-85089157756', '91691265865-19757678876-16225038900-49646553566-28653368628'),(677, '20733263355-62680155138-59202662796-22240912815-66020137659-34664213229-37475047132-41236721520-35535094158-75606241461', '94429426200-23979873590-33095427835-06798615418-09446033410'),(622, '21234019257-26118597142-23202789169-45787020325-62737229590-70385398137-92248613968-35475163272-41482501938-00123087114', '11413762899-28352903738-22593112073-02567916247-77685300825'),(86, '41032353003-48423688934-82561370011-32643392092-58790753660-14715844647-64223511949-71043288162-88486231489-63008281545', '20853750703-90737976572-42646606317-62750798837-24144257817'),(871, '91543752240-13982663959-11527995911-87196243619-92910782799-64541547430-37251343508-84956504106-44743593186-88982046175', '96060118231-20515164394-45443546723-63640456459-84319692564'),(562, '91437549181-21629696238-30158088280-64729276960-52988128317-65971268007-73048842733-04670840860-87353413833-41107432631', '02984001382-92737882742-60020555330-65407661408-02882880596'),(524, '56173061225-72622621750-86481570273-37229303200-27785316870-76282514275-89774264015-45600555430-77147333950-29118235824', '77627796064-10526078979-96275997905-98529032779-71975206029'),(592, '52441700708-92749917251-53255429192-96022918762-93383763727-50540766203-32307362297-72210485915-87863153763-26226608448', '49710886671-66193617754-35801675299-34059485449-49332211699'),(196, '73698325560-58522245706-85357754436-26422633712-83854027898-74598987156-05043728768-33146832833-15650010550-48976772283', '67570385152-58765926859-33723963556-96995519736-18068989355'),(716, '46092563193-27346193567-02731212593-21572828831-19823667295-67446589277-05716179099-09498830690-00959467779-67080601625', '98712194986-11791872722-21719494338-67086931267-67990743066'),(819, '64347834388-67694023774-84186899270-65205730391-10683221455-51938351737-06256291070-92383423919-96399238663-93246570810', '44529942165-40605717369-03260635659-35567299518-87504061610'),(395, '07419762432-72045751007-52582538206-29847496384-14337385893-57025966390-52478161931-46859331645-97194647642-58072058381', '23288574537-55178676293-81201203116-31640464681-07271417169'),(767, '83154113735-42417041440-44192791504-87668723473-78875230530-29592958132-64888424775-75489485758-28741103239-93373387377', '72585249558-27440547954-99235587158-25617390802-70194931427'),(549, '94264068667-44115543054-79888631111-14572117863-60397114969-84276730460-59788608589-41570839205-45633897379-01359483038', '93394728969-62511962022-53300574267-73764370233-18676441794'),(752, '37358501266-13051980323-56743668316-50989813029-58677308553-84514912359-50025683603-47097990281-90502438066-54432884163', '46216357214-83182921507-87973763460-92159331288-32284407354'),(574, '78557291065-68714211611-11736842057-80908791638-19830036871-04398727874-60740468255-59682024508-52891267093-90414223231', '59108101854-64412436279-99666620744-16885810548-28040719997'),(397, '36946122616-08681935965-49030944205-51289063359-58133568781-56292875703-49189040400-89026624515-64082356779-49778668657', '69772861051-14391109185-66261255446-96486436662-72529893590'),(303, '04737892417-41420483091-31918595572-28548925807-27319363827-33556777138-14462783723-57452702812-30133070106-65607541992', '46164105438-68564272715-40773380553-94681079558-16067188403'),(399, '83558904663-72519168148-69624563163-43476716442-55073660160-68071101628-09878210411-54865355275-30896791860-18945333456', '31578529519-22106668701-03122226508-57926518525-91000580948'),(565, '55393616761-44811100681-05102099000-16266505456-70213985513-74498949680-27475694735-78045782247-39735909722-58436793002', '31053553669-33778188071-10002594217-74502044209-57958248862'),(478, '89657925452-48940954750-07284971003-01641905782-52191059158-47794201081-42332898249-60368335654-74899430163-96702070592', '63664697855-14088983065-95720109448-74910996260-35440209308'),(325, '54478770431-92482426045-82680195451-95607417167-49455097301-17697186455-15421617816-58896421875-11426798721-14405503945', '00532205676-23841163136-58679202420-02975724557-50770124487'),(893, '56557921418-10358975710-45660372385-72924872441-07912865469-50927897468-54742343316-23233964380-32153603505-54336297658', '45223112109-38523419265-50876596646-61369524573-65422280944'),(574, '05688297310-87610755507-82172887273-31791158646-42778348337-96136611003-93616002945-63666239198-15910388767-83086245919', '28530760357-44893898345-04477716600-08784525076-45318824710'),(907, '80422828586-29927234153-68292737543-30319629783-78503582145-93722769752-90413937834-01618489995-96523008271-57432663713', '04183681120-11144158466-71603402671-29074563158-91531409666'),(876, '67792874668-29327890605-23564567277-23072012615-74103909631-84114727065-68916593149-21561029338-56240943980-85367834258', '79237940326-84153658161-16677283736-28214474360-15315687784'),(843, '98947472692-61812298008-84550529319-30657674144-97509099250-48319910292-37120015686-50121127787-96290277070-40563654949', '63798743905-09227561953-54580808902-30038823769-96036593766'),(833, '45635109368-23426285937-10066231998-80128599979-38895935984-69465691497-67057541708-54926560902-12540112930-67497563629', '64038052264-88916885506-11332920278-37132300763-00738420309'),(431, '31659224899-59626503715-02229657536-66821788565-50114829681-47954339955-88268597277-70426882966-47990364813-09585115149', '16738488839-58677865590-66192651847-50392361791-21937825862'),(970, '65760420748-98638962274-39239613025-35044517096-61112322875-95840356346-57169528936-90221834231-66546438860-14826133977', '93782331806-08083625275-21589327716-62872174570-05847231344'),(194, '22160182257-50402921060-88007319901-50056711860-56473236014-64681088821-28464245640-43533211226-16346813014-29816571511', '59907840455-85912012227-35008358179-04817614658-73028938583'),(721, '10718174896-08735068583-64819213309-75170568488-44159083993-76226352830-52203023538-00960942751-21950664871-75675642927', '25424468738-12336365542-37874507849-54063972893-18464973870'),(752, '07142107131-98105885481-24045570457-03078073893-59358860014-15125469505-63074281932-64008463356-78508693082-41535964577', '92932173648-36656955554-82244188212-22019833858-96183153374'),(934, '82674369929-62660017381-24863169119-59174121721-33473027840-71185972303-21458143308-44798271798-39217353703-98333829202', '58063192021-29490330179-22409743156-50626006640-20516888534'),(314, '38852140205-19516988704-15847351925-96787740257-07289369196-32690973331-55126126487-02718231009-90132781288-76977763118', '62524138365-14692897507-45027625254-34291662550-30799083123'),(203, '41685178454-47112265023-52609073922-45803721270-46535422571-39217524257-96501485437-47371031121-25439213668-14646365771', '66131342279-77427867009-38390956264-00050873889-39255639195'),(680, '60307865783-67351991851-59497866128-53451635773-03242489069-72204123690-68021263397-07912804383-67107532914-61992812394', '59149496197-54997445581-06791272873-70824246652-48382344715'),(525, '31351452473-01213227914-89992787418-86947418385-31577164034-04574291706-34754531507-93860746403-11891016875-75746635003', '68716597123-52208450599-04763702460-20501071468-96164082609'),(319, '79565054797-23904349491-79536081100-31607920414-81036865720-66070985344-46566062863-84095594363-49938040153-02805259678', '20720277993-64891338173-86978608740-91748709987-58443079436'),(78, '42524850623-24885682956-33434394396-77265577388-99637376276-47166445278-77480769854-82689936806-33904232071-76660119405', '38756773129-05785475209-21062226022-32807722096-14724526517'),(617, '55866205277-88925732970-33777227387-89628541732-90730077008-53794629330-90039213737-37106436282-47222918274-48377531134', '44530269194-84080829661-32513048356-32046415198-01054525288'),(486, '99228289375-64623185945-60117490905-20493732843-28165292565-96085397622-13104676666-47418076793-74300290074-44350547555', '01562074799-28702078112-29494514766-30736008653-50021011438'),(708, '97167711588-54827068206-93619617981-99417890945-22838716414-44303293453-67484872994-50390443479-66981402209-29609596313', '29497834838-52702629749-54566202324-13398510182-73834567575'),(495, '07693105350-36313502946-06066320274-84004079588-41161908240-80558842486-62733838047-30461696140-53435994337-49045484258', '41510841226-98807534278-54483744846-29062224897-87520632368'),(930, '58736632685-15149490201-59680480478-87143512171-39571795060-94589707119-45344703125-12738955169-67998363039-53371728948', '39544970793-42899754529-02411348970-37473559136-81276222690'),(170, '06433157483-36858809025-47342054262-13991562440-36057697608-31795034401-99698665253-19981841288-61849701970-68907596438', '64597255215-22683885222-49863969254-57113772557-60501665229'),(940, '15573333717-44239591232-14448854350-36801645534-07798713864-85033983057-41703920570-28514945463-43428076257-42192096353', '85331522577-95916287671-39571271323-97406054437-13504483777'),(783, '09331898128-08586631154-16009730320-60961102797-55589855132-87664055848-80420614923-68773930400-52785606070-21668062819', '99731269770-10720282309-44431762382-38796543685-31188745784'),(813, '33940577335-04889624915-47280823094-69527120270-64601211173-41018744455-79065563644-66981197256-95105901258-83944556163', '27330741615-49465654797-60041302244-04470416181-91549744016'),(262, '08319245968-16988172043-62133466989-21887175361-27790836956-13724422624-33526517335-43276845666-65501080111-77843677260', '27494136308-87555185970-29430698687-65467748430-24944173835'),(176, '31280584862-99950250355-21899464614-86780018159-67660163811-53086226447-68760574390-78200546805-95113625154-92326213118', '79962810245-35898616406-14345191629-99019633007-64596995282'),(757, '97859305402-82362027980-34568005685-88264225965-56125602621-66123986614-61386582466-98907928990-47247328536-53359375415', '25373832388-27208781842-15697440406-08432333834-02953389740'),(562, '26325543773-04337033886-98821746574-84981881710-21243952810-36598830208-90455310067-00130577587-01285500805-65304127103', '39647060693-63370010878-13926546635-97012480239-70953836583'),(137, '67583065899-70172604102-82206640641-65297792015-03564504352-53370471232-71930940337-75509994070-00348556528-66096786080', '48092396365-67112698910-35917319159-64301791595-02235016775'),(927, '19412932745-81020231818-56305055182-97855881158-31228488508-34267168761-51689555590-03144321281-96213272619-31120127726', '03921621667-53066862722-34518414556-87505210016-73315830295'),(250, '16614773688-88737529764-64388271052-36314327923-79782306080-32558959926-19504872047-10233181764-37349277437-70137547754', '01314879631-92392033016-31699421970-01064933462-09907977626'),(667, '50039284407-97527539798-12553428522-10130563158-86975296635-81990620745-66583125168-99661804979-46062772160-50612354782', '36860985072-66111009632-09457544231-47885263523-21519872314'),(535, '27284969578-02770755608-47224654398-81977463170-01590095923-17343483392-43334274875-48350813186-37540768144-97589003197', '10884876557-22536182351-53059895910-67792737262-64713616017'),(828, '71563020968-76973736486-73364581114-61401015857-50056838355-64326455326-62958904246-12006781654-04170890699-40512080829', '18964916803-87104447716-93991908678-35153484293-18847373411'),(20, '22848833307-31871950624-88899978202-41492799128-51904225106-61381021701-04714305136-42621480375-55121379422-91010141328', '48531667268-04976405080-57730930973-49765492367-95557400340'),(471, '90327973021-76002649504-26895763520-28709262690-49344930434-59761111629-03136747597-23174426645-63591267178-56204082567', '55396861128-82565988230-77465613055-59774500530-27556909760'),(864, '87565467301-97396839979-70144685637-59647327652-18799802888-56578763900-16230708100-71521986398-99349507555-77760183049', '47134649518-21247767495-91410611132-58196179886-07404575044'),(766, '96297496167-79784669374-78991299498-41903534411-18764038961-26887044218-13159789677-28377575586-28531446693-25642716674', '48087449266-96809128144-61055814671-06752552426-14546632728'),(301, '30488803739-31194747536-33364077099-13310305050-03719417766-63389287951-67853986207-62868907453-34722146524-51088731825', '32050490996-02619908216-49972878238-40925283206-57334932854'),(653, '21297922434-85543875353-36466950908-97044136252-17334436800-40571600480-83230629479-49743635751-23799285062-43117508391', '90903104134-48951156762-85379196789-33315138431-76412479992'),(667, '30812299356-14153158501-66935014668-96936794309-15097214699-77209090523-85129937191-55178784393-74894620172-06136237403', '19686810744-73467811985-03853115436-42987378607-84369616332'),(61, '86106102637-58589594424-75505228500-77106111755-17690384037-98840226606-70693258806-35360640041-56430220928-49773731300', '92580797595-13853926157-89405521144-20799199927-99518143977'),(689, '58612408168-25109440310-37623547220-83972521889-06506441060-64585692681-06643497575-98695671604-48346630295-51576101025', '99273581444-31443718051-37971363566-84758028426-26745723331'),(655, '59948102145-44474617675-11153603764-29779955619-85418934156-87126062135-16515412423-77120291252-77490078332-11050219423', '66978168625-25147071415-56049494211-17170064056-20735808358'),(743, '33585309651-93713840672-91696210974-82175065262-25915820571-91883096545-03893919710-95718325718-72103879124-49603444462', '77016487824-49980169781-81047808758-63379429930-75335436714'),(587, '73013350209-55440032705-63575121719-02634159221-38551485665-45398373171-30800486932-41907648426-59531859820-01541120765', '94335209240-07104423925-71813540583-82979972359-15761656548'),(368, '70488983235-06151783341-94235790292-62663033438-97364278898-57714097627-50044733990-56395254419-98476138467-81650380270', '60998040317-82954549007-31595478933-46347001144-22100875172'),(219, '38294031070-23358211997-54089581416-39133805561-65695013919-11457942293-31529785527-72970989345-88793902164-97865990868', '60984612441-83151077541-33841753831-04180765530-99395019999'),(73, '69958403385-29879971923-06074240634-35359321322-11083928428-13821229576-46958009355-22400539679-55880654566-83066702413', '22657021706-17343812788-79484590577-20082851837-60365321590'),(569, '22980380982-73675166742-00027799864-48551391618-47150040114-65528572560-79813486518-56599538885-30549604252-72444666223', '37071186540-20588213463-99496639161-07887674478-22252039811'),(982, '48026090242-93122432224-60214415678-30882402805-84690841517-87023774881-11900423989-79658641566-93746111069-82098388479', '00777555626-15848488874-10552881740-17328234376-34261600797'),(36, '16326698683-05545807069-00994321525-57047042916-46059274266-81675592114-98540077870-63721346672-28605748781-01069706574', '12325337532-79797976666-95626303854-12780643403-77676711789'),(391, '18775222245-42002709537-66471532203-70379836356-68806397554-37240038363-55497257843-75865213650-83074459216-57633736263', '52286063399-58124333802-57401322503-10574026244-25030276708'),(537, '85804059791-71904085963-45584284198-57123548804-97577881061-52383643366-37788592228-55079780012-52132963418-80976634645', '74545761793-25388809470-07943948628-93821952054-45300967612'),(595, '61851894727-52884628047-50824104999-97684081733-80197847151-43680890304-56093382697-21003728077-02161477055-28245867256', '81847058435-72768320683-72045138955-88868153699-01242573719'),(728, '35685466972-05067467175-08928855119-31101066074-85781724853-92999254126-26253501433-61720532035-46733763146-44733490961', '19944663370-39130029056-76145721139-16506587960-23835923145'),(547, '80535602169-54707634843-62860274897-89547789493-68495626013-68278299601-47155248505-84373019868-91198072725-31233916975', '04669330466-98012223099-80310836037-91652943154-83965721045'),(110, '46704309405-53376480274-91386588943-94901465396-96544988131-20299250148-08590845299-35627541710-19995792624-67678275887', '03126131966-54476636180-86854227036-97115693127-39247146836'),(134, '87306491161-21295202544-98588161532-62383964373-55651043424-76224738334-96641904211-55637683137-59059476008-21781509632', '41794091466-63267132505-64281682561-39059031549-16635446566'),(536, '55755887108-38305801729-93019128986-08030646434-19454568512-71867865464-32621775449-17186138292-89635991361-20225952280', '20433081505-63391914956-99472218427-92974136014-28990723602'),(957, '56533782808-52685962513-75388589122-86206078637-94536157320-38642989780-86099391154-35204186845-11183982408-30526636163', '47469824827-03464983176-30095259773-22521797061-02736848126'),(722, '86720577303-96627754818-34607135872-77478989961-29786406791-41085774908-11345181290-87174058613-76338803585-45945496501', '88656541901-77852846258-67532818942-46823143415-67879310976'),(210, '75632493050-06449563057-12244156387-77589061577-55915139470-88972816730-62100764172-51535805601-65826066830-43894045370', '32562310104-57189334488-81402542953-60953834629-37390297045'),(92, '21874510176-84350676397-00486310448-89000574464-85706471055-76589556812-41999646558-65033321429-12568249987-11973281078', '46066674370-78837258160-71985416414-15377303807-61527375135'),(729, '99993899693-82722392157-72641065672-01392848996-08740352136-14431975686-96491044606-84150862920-67002404978-21032918132', '99763505813-42468851055-24324101763-04705122436-15298054474'),(576, '07100519289-08348910124-26420712985-95534462764-80069993395-15028494095-37807710484-91404368565-79635445695-82504590666', '00369177803-16472815220-14176491269-22062408814-72581583338'),(558, '15037895892-16668781349-50119256939-60445173528-41037595682-68519894126-03111039295-55199998158-54410695883-48606017078', '86726170232-84278475490-17772402366-25695817957-00507915347'),(432, '62559575930-24137471614-45351205696-75220407045-28955139074-92391327513-65714829953-03265369873-61384826089-32207384393', '42690868588-09359359726-81266557326-75471492795-51953043972'),(172, '61419549433-79850729126-92464881126-55100633087-00052305956-31350793818-79590079455-00005906543-36623421335-36673156859', '91322626310-10679279646-39640028579-46960637589-94241587673'),(619, '75880179614-75923585805-79369442296-40634617638-98136904452-45538148904-88203432508-73118220210-16443821727-60951665374', '29244243412-73350946904-60747845577-04778348714-02547985140'),(360, '06243982837-47275028795-33086894707-29964535507-08090361702-88889908242-57819681369-15560722424-26967317689-01485090681', '45225966698-08636008199-12554143707-41576382363-32424406550'),(493, '13133783936-31084520891-94549507813-15916083727-28605477290-91675532540-07307837324-31566908756-91235281006-16604523735', '82349684868-06322739869-48715940322-56093792592-66476907759'),(319, '74989120791-65350711306-80484312249-29284312880-84271638212-65859544207-08773568355-16600363595-06234953406-54822906018', '45455961634-21678830156-88805865873-17052252610-94448076229'),(22, '01551609821-42640253449-55780858125-02120580920-27700914218-93623125565-87138643518-97171258022-95515630721-76882589334', '86377819779-71358547203-52628560672-29449286320-95216316765'),(371, '54732742265-65330591934-60748840638-37512728816-59742832074-04722366790-10618836179-26965768683-56147098304-29847812006', '39209545259-37492259275-77581403032-53076580851-15661551343'),(387, '78247607910-71435000559-18682641014-02682488506-39621534822-35808708974-27042437202-04882499694-13728650651-14448174715', '78935029146-47944245967-48657979816-75348744168-09172987915'),(116, '62589243453-42439251492-17325550685-14566497136-86755107871-73384596146-52522169739-94968808614-24251220400-44724813524', '74935570389-27759440207-85731874859-28972391928-34383265120'),(136, '21250203111-39136263996-46290608908-29680757099-68565206912-25283391952-54335090125-98461867672-33131201615-09523750898', '58547548160-54760135492-65777757313-98699768562-71605835314'),(670, '12627892415-81401929538-19603280724-35318340324-99906233469-22470619505-09506137953-16973117404-32985454067-92693053756', '56712051179-88716611124-87144836179-84550362963-76209780995'),(236, '97454483195-25996794156-29551106413-40062202451-64827179459-85049017428-10664072361-75997249213-23015474100-15549275685', '33036274845-33757241904-17622055852-61264591167-36763998575'),(886, '76833895347-29272353107-71513020197-07607435226-15825724877-54509343561-85157602219-78640480413-24543764791-30676759331', '80297184313-58894797795-70773895841-71912369693-80212098165'),(998, '57492935241-82444660262-07639718685-30406786102-21987919048-41735280197-73060953262-55435560570-96813517675-93467618545', '95956543719-96534134934-84122834989-99359907733-56984190084'),(194, '30059253222-18697951546-26598571858-86032467890-43162234024-41040755878-72300162439-27240051464-90481439689-50524744940', '33882739732-32901955256-00498320016-29734697745-69617188705'),(843, '80543196235-59013419463-47867867147-70987183905-48734771777-23354102690-02268945852-11697442883-33961044440-71066668056', '01498716771-52246199305-79578790423-44231545003-40865903700'),(897, '95894790490-08629880055-53305899264-58609825227-36216317364-57459466717-59818854468-47764973764-89115549381-65389608122', '04860326618-01088273188-62394183566-48355200553-67233639496'),(306, '22596857609-99306587322-38742741014-09682079958-43283078631-47518600765-87936213624-37480641119-97719892599-56207072138', '11332211088-74611599122-61861292959-68429297521-42831115498'),(930, '03134228652-29520900089-70678199145-26859838481-11488465052-25559592436-20344706662-22341985723-03614183865-22650713186', '42706647293-48579616133-73160553616-80917236745-28678347545'),(323, '11864832957-17348102690-96423062212-74124344947-19434037630-79465746201-36242359106-40507232791-83922512084-74909297818', '91228396828-50405743771-15478689169-12668824995-64726618937'),(428, '31269983670-04821652466-83039914349-27849474864-70649512994-71690136640-85375338308-54694632823-58538008761-03911134736', '94455033162-46245535244-47942663883-55515795112-82538551016'),(774, '74835412215-56155821632-37941853102-45468073907-29901994629-67431325060-90926025391-66241293257-92494799115-15794519002', '29244251611-48379181402-05485008085-84601141195-37640393818'),(28, '53125997361-50221270131-24853573593-61777223729-71615028221-72155777723-57041620036-20097193092-92617047105-48296734368', '58293124894-44543638444-92589614150-66570017487-97029046576'),(189, '58638040471-48454266446-78941071606-58647342256-72634781777-50922592118-63933126446-97653427880-84951736165-53326161784', '77738660506-97381414598-17839416399-63127809908-11818090370'),(46, '88841750029-40774083174-71473186059-81181622009-93809240378-15481882349-58673571635-22257715749-24763436551-79797457357', '47241073426-21613589465-80694980115-12031014637-03445184334'),(236, '02409931213-37099409232-36085583908-53583446811-22218421835-10375020627-18098908721-88747278232-84854309245-54862107522', '44046629439-11473040721-27106089682-12568408836-73209523754'),(2, '58698532456-51552010966-01352107974-42071347301-85335252729-74322017423-19667764348-88558090075-77332837004-66602538620', '08700896654-51539087211-36879834404-51974584214-12954777183'),(478, '61938754723-78132699407-43403345904-17669648489-74423688463-28484561366-28457136559-91003444368-45274068289-62115597191', '52170670937-09350007147-15409489666-34337899374-20470929871'),(512, '23526800620-53559586832-36842799113-20579467695-85602974742-07974298349-98155638067-05280510507-38572148720-30112650963', '31176928023-39383247059-39782960530-86631024949-36941202313'),(556, '73221253443-79943944312-75739331437-50016054351-28753497567-33434235203-04918746943-21483745722-82733260249-28877193932', '77368284726-04650276286-83555247968-74228139078-93202406270'),(121, '88708912126-90985531046-33046648306-47330974947-46321101164-43717825253-40537121640-77679408899-75967432800-02950491376', '43674443245-96482933745-21887105060-05701406339-05111102333'),(721, '59529231998-06186044326-89243501897-35837432989-42068681895-32108198856-31993208823-26175855152-90788691933-66385532500', '55153037914-12601350885-33701033385-02948766244-26921212503'),(310, '49104281084-14955759586-99421667409-05433581680-30071787017-41952472631-32579066441-26587025163-89523067307-07563076329', '53026130131-01794999911-05438054092-34524967179-55150516035'),(122, '57554045845-98041406771-71958922312-19388344448-57329454741-09533249670-43393619295-62082005335-21657808037-42901810400', '49054246528-67699012398-95824747955-70485310637-26766632556'),(829, '85850938174-37757004777-59499326284-89220334921-76737681421-74846385902-65109429028-22587185098-40370820466-69169844701', '52706049652-07602284055-10462289016-73368004739-72385591058'),(652, '49531554256-63693039077-04364374233-80485709908-02295715730-86078088911-34972197061-95223380096-70989391803-97556750148', '68967084797-07778630513-13078443896-52484904093-24257627723'),(91, '62841965395-87383636302-16709524170-62912271540-55810264762-98363810019-28223329363-79426058805-95700468501-16402456271', '20104766274-50354644875-51539820381-04645388569-76047806591'),(790, '77212673677-74308531588-06398613994-32614480780-99404078764-27922553141-35696947657-45109290220-10426522580-84803738736', '41789059246-91188351200-86745947370-02197805564-00073681466'),(878, '89357947512-83664855038-38905097813-17266035533-08522423173-08983692165-34755380187-80361123492-44824374219-70507182734', '53056349738-13696120862-71966016687-39771856493-92241615862'),(281, '45202458566-34549640717-77642280880-28995221252-21110876928-31660608019-68524053302-85203257003-08363599548-12939181959', '86163980852-45882255380-53025497241-91554808331-79652892952'),(291, '09529181157-74651450008-58885593842-15154717878-94945847334-40841570258-37834701906-65343687256-18435570037-84084939332', '80707626679-87009111835-18550386998-98872792986-76621932723'),(64, '22084621514-12828713810-92211782613-82236077552-79428997360-07683200412-41581459313-25840330679-20430853749-98124513907', '62510765735-87611472486-89450512394-36642897133-58482134232'),(510, '13683664707-47712320165-22701207926-37213835570-58087660120-24189678780-72254940409-99235541726-24547025242-44597659831', '17530463506-29563303220-19345463127-69398040160-10466891106'),(675, '03704256058-19295414477-11843978024-04317452842-93481843144-50312115921-34726607172-33126218308-95273767653-45103735516', '82372553071-87227501420-36177657469-31805690048-99961623727'),(920, '26932437154-29331942074-93521657267-54272363667-17031213088-44225999348-52156392347-07021487014-80784467815-25454303438', '86733723530-80841377408-17175593048-71638597988-38215896740'),(156, '74130110720-32768513369-95880522884-70744130630-77141546410-39622488194-75382122995-61596104788-31931523960-28370503924', '31366859570-47239550791-08036220400-76085173718-69326356519'),(928, '48532017537-68341739111-13163713867-10943981789-74236792691-74041621950-19846536077-48428041308-61386141513-87968562756', '88776918867-61968534899-28471781145-30303105226-59239909551'),(453, '11333386915-42603281245-40082636564-42093810185-64144824479-41831333456-53851701874-80861763716-50066632568-21067496281', '64371724790-82987826889-45285929336-18683441884-05910443754'),(573, '06992882648-98335438348-85601992596-34943220108-10265453836-31319774671-54081834838-04383611043-37823268479-96283726172', '98017377536-34001491677-07307828401-24418104469-84430207646'),(406, '35020358720-82469905498-18713126354-08709233839-56379822017-27215121713-85311748446-91786577624-98955875232-02439157847', '87004299175-99151517291-42093701573-89011328067-39595766944'),(380, '21293956654-02296559086-22591842036-71096123567-92543704337-22341289897-48808228002-33827009277-43720754831-36868214866', '68587897173-72820966690-64210714747-21369441225-89029169956'),(864, '61430976930-34600215094-77165378836-43014797132-05079878264-96156192292-06066229231-27315580092-21109964144-18463550431', '94161969057-03728997196-66634820330-95141562760-95961464016'),(734, '26398238910-60847438841-99116870396-18474053652-87555281906-56776534660-45277052586-38687261652-01081662916-63617285729', '25506361925-94195059981-09522958411-37999504095-22846243841'),(315, '91733782832-90728012445-95690930886-33842445617-30086814310-09131251630-55593344177-74938720940-08172911878-99550645451', '17215439644-29252363059-31224181779-00234020714-90962496674'),(637, '48941444692-59123721629-34342211499-58259168514-49158773182-38510108901-03523158420-37325294111-58403165161-23315184130', '64312513891-36759299801-01441606586-17379841597-49113527710'),(325, '67215345663-80002148544-00656486568-49321539732-19806925263-17346785781-16450136576-81697813628-30847546687-23689754532', '52243458714-40942902267-06509544984-01207811218-69438261116'),(262, '97474759467-23981040098-75622568339-77112056729-50179515828-01310036824-29557733297-43526987887-82346664496-15856514723', '64851998192-45566095245-42309035036-87402345330-07324618331'),(424, '78008138505-12651247383-12271518592-06052484817-85901370390-07981042612-54540573784-82055867935-82109683224-42651209572', '09324933078-42425888967-48448537202-52098981665-84877034775'),(697, '18333267069-69167836375-76890539053-85604485135-73879919017-85921004874-83687535209-14030046430-20826057304-28647943081', '89543260200-60557884625-95241434512-65637744856-79173048473'),(715, '92256830466-34810169079-07257807305-52172436187-25402021784-03255431981-67795457485-35534159318-49000234501-61897063732', '50515779490-18543787508-63843577921-32460836374-12528530183'),(715, '34165965630-43879091387-12396227731-88307401776-24048447809-92028331048-92468910465-31195009480-49541531050-32784659577', '15900873833-50715950900-97896512351-11328003417-79857924739'),(789, '57060549466-62973709098-04206142432-99078975245-55131538437-93905075766-97238191512-48097460360-39252074753-49620396596', '79838434744-28769292689-67036280997-14588421647-40541758301'),(612, '65155034856-92118094474-75939317312-49134755902-86831404906-94841840882-32617890037-60465005027-56839858414-86771116352', '26293909006-07289153678-05342215625-32848893819-81955150384'),(497, '48487529878-62726906406-68953709504-68990822593-13522556058-52627956622-35170576148-41673144003-36796178997-44881649740', '74908447752-78235399426-59433015427-44437222002-70781510811'),(791, '84995612127-83255276969-95932324771-50754821779-69694910042-42384558896-57954371341-10246648865-46443305163-11486834266', '86159449473-32754726569-93096617473-33370275014-22618813656'),(545, '48217161931-75050301450-09337042410-74340400896-29782634359-26426222359-20645239526-80595105463-78848879291-18152984280', '01194672134-91017142388-01390834050-36533876640-98998187421'),(58, '52003711364-35405020005-19969006026-36078642527-56123869763-63567738161-52672181029-82864126943-62015867193-14003904247', '71384863366-61214510210-57805423167-06979221071-28268047710'),(353, '62826155646-43435050586-79410571970-09882504617-65620999577-95613955616-66059046831-04480171001-98012860927-59594968753', '99125459097-82492642440-96360940886-96408474681-54334137265'),(616, '11832951444-43399074187-92078406620-38430191171-88393997774-85458742248-69078669448-54271656835-73409437007-13247475250', '64963598514-02601682800-69615555584-86106553403-64051552921'),(151, '18693984604-04511085504-83676404827-53200495043-95153699408-15163005065-51306933362-89911210816-22291081588-61852439262', '84031693105-84861961967-49803894483-16389493814-25052695298'),(18, '70791621166-01689389380-34645949457-02596582778-17663555978-77887869872-54727203545-90035518172-68046063850-12437512291', '74078949735-16462644013-46465023513-68915854987-96592547263'),(873, '02540457780-58821554238-68684873908-28768159678-67500833015-44464016612-72167893714-58109437492-02441106238-12721503973', '08417642914-90974966034-22182389210-67001570810-69655283414'),(793, '49424217486-81755039929-92581179002-76738748232-38722035831-83859913073-85332990762-58447108491-54742376880-68022953252', '26869955035-47033112105-16830559687-29749105160-69351021950'),(892, '77071576731-11352056680-54844988858-77664835887-72017665296-18652507948-21372187594-92081557118-60695401057-53000341428', '66945781813-08901024086-48046515369-58551368669-29211327034'),(194, '52920381606-92770866198-77873300666-29941810388-17636074231-88711735742-45095256864-79163942377-45802151974-82065383140', '16682325432-94878500122-55186818951-31864071185-72417515304'),(175, '58439460742-17779048395-88534910662-77371477731-81478277747-69686128483-64931441955-84270795392-57299258527-44441951121', '14483426144-21751730534-64154189185-62456729903-07361646689'),(608, '79136133656-17625263007-39019722091-44162735298-97619939432-16943767020-86662461248-37268930291-56506058596-54705842223', '58934900683-61452084302-14540237594-45555236816-39983820445'),(814, '56701554157-86881196524-99165132601-36532044673-40505364523-05439713120-85595197875-69306937883-34828197569-33340710941', '15373828549-96906208858-52591394129-95787015062-06490745283'),(20, '12900390636-85497678660-79634208591-71323888349-76399200697-55027424478-60536058608-73164043648-99358676486-98425925603', '14385572330-20487164424-75775021126-86493745528-37722325991'),(322, '00727801983-77376233602-96072916379-98253834415-96117645402-60651069720-70997578359-23208141569-97355005038-82686139442', '10627816079-83473094675-54995339545-69261001979-32962754018'),(610, '20585404383-22716746666-29707456677-65448455802-12326117280-71780683320-32366581981-63808695394-82374626864-40670496103', '53295292613-76217894958-50390904868-71612060880-08894482039'),(497, '18002776149-79210865136-94914784309-23246304950-13756620654-28222843849-50060782368-31058003539-06568377855-41051866364', '03251425852-48178968715-12708280137-40071613107-64497209681'),(188, '42233050184-17378850566-69243053564-28817890977-46109850817-67908413986-28578106005-86545812623-72362675522-59871994557', '00352648508-11890944764-11184817130-38516197464-07660319654'),(181, '24964685103-67774598987-48687080376-72668886337-13062402269-38157760267-33774150747-94308314871-40614422990-13764079712', '73837514589-40966291796-77966124113-64978033941-03900025586'),(844, '17983313828-12443430478-48817904466-74190622781-34741303337-85152564808-05950079829-97304860903-66023146375-19174441434', '56673831065-05809960817-84025308600-81279441876-59487493300'),(756, '51039515245-87268723852-33460620820-77614018002-68225362739-94667820926-41208129304-20239587781-78042084299-51129630076', '10792620335-03542079965-13383321452-32026237476-32384943179'),(539, '87692141008-23191814826-89047338053-80677117365-64239600604-81876714366-33166209839-88954494107-69185982871-92543196618', '63743646795-87714982565-57683057446-54604259618-72835850929'),(878, '46559418543-93196799826-81448146129-38245754806-03026885110-77741948346-71732047887-14798735027-63965507766-08674247416', '37692925886-30809029517-10766071198-23124796888-46979105585'),(858, '96315985299-20481345787-75718377527-25388891320-27906884506-10800938224-07418938233-46988399133-37721870540-58544872122', '32508512810-65247521335-82470966294-26206715458-78487778402'),(617, '39429294189-10932439404-59275256106-20667363942-60631433512-50114527587-10265593810-49277914877-34153027838-63354741144', '32203482574-13177072998-13699055874-01696781360-11533972063'),(65, '92256333542-98700685008-20680290364-21317348026-84948267585-75785009564-70176520992-22774882953-26776524939-15431892740', '56649892813-34261270286-38371574554-69624713530-06508253503'),(661, '49004703950-99871433185-01801119032-00276220294-77148867855-31834425585-93315021977-10394499434-67424389198-18536053700', '43436410211-57431732304-61221916806-17699364916-40727805675'),(253, '37974952240-68539023988-30502118359-23168577129-44326090837-93369516434-68181256433-74360515941-03656990090-28458224538', '67574303157-58272180535-64067952317-12736677949-52022344281'),(37, '41807299819-44613319788-03183252628-49186745663-57854302201-33696571536-03384523597-00247541911-53483582536-74753647419', '71873717612-93349082492-21682814301-55693081851-67789693153'),(801, '31463995853-73800835393-47200548939-86068500088-63498590242-54308360763-85319111229-05762048945-62069370225-19103298951', '84896378013-60394020661-77498059190-11961751593-61754838611'),(98, '36016958104-15182028238-01298522591-34184558059-88827434013-14422183739-48816217158-49642259754-37366034170-51515608945', '11346952398-10025038003-02715945443-01341645415-26589201335'),(549, '62430460215-92448821300-96125287143-77925529873-32882077993-37349662905-15558966998-25615916372-47979080227-07029378977', '08240040956-72500382775-93517930635-28619401302-63147854073'),(307, '98970394830-48014660514-73218018521-08106291185-23611225011-84220180681-55611854797-84841040758-45343469789-28835840628', '74838578723-75862360552-52238356489-62444990636-46697323268'),(920, '53261572671-05411394594-60654998571-57126704800-33453821653-59562285079-72958202553-94821935436-99283793037-52308100277', '15160009803-39997518001-68643435416-34882978755-54804794944'),(53, '47815295771-73461444018-49415452548-85727228165-92905377056-46291734000-55395544513-40153939380-09845017726-87159220993', '66898968880-77788817405-00182121888-11668166759-21940474493'),(967, '46421611757-55465152397-40287098564-60516908208-73205632949-99432266798-46884448812-72784393058-75917369371-35738816656', '02900435089-77770981511-69764853297-56055712379-58062424706'),(815, '60237756941-96952716125-02074516394-27231957698-45667273901-08562960517-75365494036-47237726277-03407836662-36432667546', '88753601974-55090999526-10249830473-54195127392-15669371555'),(560, '01073293493-55510395016-78723597641-87225435618-29567354933-48994310723-77320376794-68584963679-39787650211-17516406100', '29525174252-82207122075-26943817462-88940538226-09650228062'),(200, '54858652712-44385317549-61605518256-09976097913-85540381584-10517259937-63514887635-75510189086-82250009080-95402961489', '96125712624-45996331141-04630384181-87872034266-49926220423'),(29, '60802806903-80791250673-40556697806-57790584681-72723718333-88064107400-30292280299-33105172538-05622724858-01940063293', '59976153774-90332606264-54101459709-64939162699-59469541721'),(223, '21204374580-49819393569-62229097450-31434363400-80024506390-27559565340-12832325664-20996610765-55057920137-80155773192', '08748781866-24151484709-18520501978-88888609631-05115644595'),(547, '32248921520-22541113439-83791077458-07147588468-53429401710-22654560163-95356431582-76843656999-03888129496-20492318380', '95083205988-52698163838-32900348404-08897442087-63184795886'),(922, '91438855509-66461827294-41670365388-92504738092-49078018343-95387306940-19210018293-80797533926-11789545961-95345657510', '12638306749-06872765720-87657725981-21603433556-07408046714'),(441, '32200777173-28725000803-99173520381-10667455931-02447105491-33309090522-28992768033-53672490754-19284833864-00400050602', '42300513296-92930557166-53729671991-10367263573-80788043832'),(805, '53288498712-13406487876-26063008909-57865665461-70546499008-16885800597-05833819414-02262952984-95888193947-06864612291', '22843872932-34334538158-54347444102-25397516950-61487739461'),(845, '29155546295-15808640128-65609971332-32620811200-22111752818-73370268951-57043473787-19549830682-63459849581-49784192044', '31314006391-68173590668-88225616907-01596527142-88425293531'),(759, '12871263987-21383308289-57308965565-99534064001-13608839805-78597306432-48396392257-95649928759-62696099029-91907495247', '47950222020-21106328311-91771633634-52053133908-42586705736'),(652, '11358656502-94808499077-62603305567-31422556848-79578879397-27295612462-25046060327-35315225133-11800460928-25138664615', '09357798549-58054401186-23143903777-53720049754-70143918650'),(745, '34552658057-87497549727-92018143293-56202356550-13317868535-80464626650-28038102245-79573865396-89181567114-32800448808', '41094517732-34331906871-37217962373-46376445824-77585483581'),(840, '13725043854-45068807923-93041118384-23163485954-04031596709-95849223327-00855304629-77815945351-52931347986-35552163977', '13249776502-79783677591-63086524973-93580214790-04826339230'),(979, '64506785673-09394697048-17930820718-68187588376-75872337611-64555575375-34841433737-46045379640-77481084728-86498619912', '40544230296-83475795899-97111126075-50972580341-28054763397'),(843, '07616018903-50742346733-24170188349-91399899640-89269845186-97513854633-64464399176-17505059303-20546936245-07799796648', '84009421222-39959293543-67945215579-03884699647-62299984687'),(455, '77552061542-57845182816-79434312565-62701522273-44436721183-59080442999-41435628655-59361224150-68209977691-43260818531', '63610367767-81569253044-39302760061-85504875703-08995457713'),(117, '34871058443-69455735862-48522096223-09565564487-30989029362-52044870448-45288857535-30093354393-78294838311-99598769553', '60853717779-22855456658-46625289198-73685373521-78242228005'),(100, '27459879212-11970924691-18221279807-04113355423-32489139029-27941286009-18280528554-38756289142-29173805109-63389021809', '95489998223-09509465958-24446267756-27561306772-00008557857'),(968, '21100440917-59704542753-06857177131-26661732277-33857747257-48986982810-31045269823-62632872333-01584559645-57945854139', '82158523169-85996775720-87219459262-39561214005-52459600605'),(248, '28797886797-90748535316-77559099757-11366430012-57605880858-69136944712-62468056398-35031027561-23687255126-90002801154', '41476832141-23597432887-04356912414-14973876167-98524977994'),(276, '60363112801-46085490385-68923818518-90918342582-72507726005-58018953436-54654717278-69354533648-40822633387-47900919292', '27244411763-85527009835-28207993706-38928955703-16155429751'),(254, '78330401745-53589513957-91898421425-23519009092-96933016964-30336749484-45421967489-28571188280-66119822715-06867528982', '23974604327-45239868233-20868714425-19173241566-24497355431'),(12, '63527791457-15414971352-72920306458-96626255568-82808174859-91081003626-26217226507-66871150927-99265055316-00746601113', '51143162398-95811452205-50401927784-01725089120-48266105582'),(809, '63404298899-15964893708-86667233099-73316611070-92382593257-12122543353-48578929885-21478407645-81150415027-62520008310', '86275832315-50354681612-86707045893-38146495947-46199533586'),(859, '17989330934-24444193250-80396946488-93113508663-96162083629-37792787233-43015687127-67271249020-72911598314-66510199952', '25969592667-87460448407-18594106955-92532817023-40578947349'),(792, '49975056185-06661938930-51763030961-22428251006-12261574038-43795104182-85710923421-25227432813-95652864300-70407226796', '26898548413-90349852458-34013064748-14488492897-99280482077'),(361, '89711487301-53830447448-91259014199-20616924760-27175723961-63875954403-34776526336-87763194719-76376976151-20119507247', '95967859839-43442361893-53304266000-88899875538-91987720294'),(35, '15215622001-78406063166-67352086415-31628386377-56097545371-76061619870-68796353170-02493324120-95260391979-12020700665', '93799412087-75770498536-38878650480-41223473307-79138195744'),(75, '35262979980-68736021264-97747128373-64608469975-99875735539-46975643136-41493764703-64949552596-33846282465-46879845889', '02322333389-21450304121-33603149825-98737748776-61062182241'),(167, '54173402991-88350291508-46142993105-15529039648-36494179545-67318080679-78615559334-55693264305-18024500035-74896208892', '97442865964-08905635096-07892389567-77615530059-90071697720'),(733, '39719721097-07702499609-48140311131-20877089697-03979369856-18670634434-62600850190-75697981097-19839140781-33890163237', '18592076528-60354560456-25754614491-56990736661-43171850237'),(766, '40396652425-57727684810-88770358944-13157786650-07556594449-59511023510-76915716224-51131300461-65859280739-95837017171', '34729984948-57407870688-95285310079-87789500880-44912852361'),(711, '21135198464-38048809646-43833193156-61356732782-82631497416-07121671085-01126246411-27369131711-50470761213-04341389493', '81639336647-67548046650-02672797017-10070144597-76367705146'),(290, '33268918998-94363441039-87571208832-53126448630-30753604323-72803439583-43663774901-45007804751-03157864305-19935895045', '33594151673-80441091702-30506819964-15529127005-98471837867'),(585, '19996778900-50208504589-56169299306-72210649533-37189442616-36351032611-79399105873-74738643846-35494589955-93673090803', '51414709325-12065305550-31979802609-70685649981-30549498769'),(193, '35272991416-34017286412-00939029153-14024907300-84060906492-48764947067-22431124350-97043897811-27470119228-58678418644', '57554356576-50798837954-82045940853-87552118074-44545626612'),(58, '65512100300-59489693178-07438124730-50227138695-40703460847-20416143325-85473482288-13331745748-90310704074-55680042480', '40249499859-62469593824-17824768752-86149131596-57522784922'),(986, '10461879226-86199232552-18525088136-99458258212-24417654045-51170624147-88295317462-81352914887-61893955749-68970866059', '88914614869-79179927616-13179337231-29616740839-98782713478'),(411, '72490188207-42834550093-62760902374-78153417250-26885503427-15926787482-94100997676-64471453763-12479962567-05226441516', '77170933784-54362788492-10974417641-57190438688-81950738830'),(434, '28667261445-20530564085-50598906928-62162494758-54943976463-80107836754-64582871413-30965602641-18967935480-03618988392', '10807283301-11817272024-97627148141-15382187353-66772032923'),(972, '97079341725-32060674627-10633553741-90308391449-36852976450-14177619290-01245834474-98358289462-33883298939-12522796662', '50424907909-52584204974-31989477276-35057955577-96430715189'),(527, '70872726960-11020043882-27783505615-82688457504-72099435425-48714096747-59974829736-99230005181-80560861798-54705110346', '03425942704-24523050033-27023957998-89192261794-70979572869'),(805, '33425562442-14879735689-49963008732-44568931521-86609979618-49304527011-21147190496-33019166961-57437563694-28313926827', '65616940337-24682467900-89052947049-04735680277-58072423467'),(641, '60512899276-96575172031-11085024712-97002392463-16036647949-92144858257-88916654705-04419168093-50277053143-18631189932', '88812420881-56654631573-73579404207-31698655854-40169072611'),(936, '49126572628-28638526383-06343184403-48726836131-10750491640-48278802240-56496805683-09816896195-53386296926-27459370601', '81556649547-48059237055-25070612538-67402473666-06437717519'),(745, '09901562313-67422308116-24580106111-71732118428-21882262131-07849068604-34949143409-23118672097-41345798535-37004953337', '77772459007-95852600085-40569658269-65817563981-70355776123'),(881, '18658360853-87641086870-62325663824-88619850022-37061278939-20510104599-24239179223-06205631811-07597168263-30370679172', '59955474419-77037522432-97450163993-72641894601-81795907027'),(665, '58272879325-80094627060-93184620777-72319260286-89405667253-87707478420-83429508812-61665685616-94754195389-50232599491', '80771780776-23718150480-60774048604-38044734076-57563310138'),(634, '68532919532-98276866621-37021854522-86693362789-87928596195-71211063354-56634723749-63321231008-07568288183-52164704187', '43400213342-41426379473-27952285002-16774773419-97084573134'),(319, '66958315143-30730150886-46933687040-62246416126-57289930776-29124165333-03319111459-68938644897-59681058237-73214025121', '66454020780-75877847401-07580803091-29475597610-85773214992'),(106, '34734513793-98245991613-63799973043-52614710318-76987519501-51621109004-99833628287-27523648451-22829904059-36151730812', '99968338091-85705587057-00035017961-38999112133-97491305202'),(710, '96531223022-11876292976-39359892635-56614292687-86944932018-13691523670-10965906068-58495921850-65324667278-91783652172', '45558840257-36550645326-28462223043-91699616800-75467977096'),(136, '54256753334-86157543672-97275222811-78906723815-31524550017-02768322378-69008710379-32570992945-57258237072-47843906429', '42151902437-56770407516-79827518392-02698615311-72249752128'),(1, '03342564466-26301970039-71350976676-36811065496-94666684051-43905694430-75738607763-68527282571-01929544686-48232583383', '37019403949-78512948757-20188118680-17207645356-59265297178'),(262, '75712825488-27887787462-09464441556-56561282534-28283788053-40238218196-90175077156-06388772955-09377309061-26135039763', '10678844044-82237316464-74671458821-37200196161-16161112218'),(233, '47499524803-96834337956-11140942405-08437853173-93611225777-72442911378-62176502111-53626731319-88294847527-88708482039', '19667644510-94864610186-90101545474-98029034522-56381282689'),(118, '33743215778-36727946760-05317432635-68375912064-43939097181-93997853811-65510135562-81200387510-59149751358-20181919497', '62926831723-05306152021-67088281942-98640555627-32328494199'),(945, '28865939723-43576404152-12026343558-53367821483-11676480739-49456884136-93264674914-63271095972-78781452728-11740228718', '41694703662-90962523231-29293133083-23831438333-73605116528'),(478, '85126300277-67013111817-14178076237-44973251357-76739986530-23355402257-75724826952-43666664483-54367488488-30663639084', '10412328532-16250880600-05895893051-17227262573-93941331217'),(871, '62828608812-85202857357-02311174879-31147741752-82245579287-32227769586-91459114688-63700808533-51126766110-23326735870', '10847308490-13816923554-37625339603-26429448472-81285578448'),(674, '43462432498-30933729219-49717764898-66138247859-52668926416-77035176161-72771586001-10518140285-28332187078-36275630034', '03752974182-72557687036-15462369152-37269080535-15014372498'),(949, '82868331555-44301717502-67936034134-11695671795-20919784969-85841043076-35185959817-76103040543-47440707298-30934648649', '22222136465-92744071234-39874350649-64178844096-26937309240'),(312, '40701132029-67458564284-46620290398-75204734548-03987142325-04466390146-15167649544-86755671933-16220104755-70595595071', '01604919650-94887015214-34253911812-93469169450-03729446033'),(613, '83431285245-72696896967-45503285995-70986630226-96637648365-07243131980-76242524913-09417495698-25005376396-88051071341', '23391053265-05934503319-47956430388-23559634347-75844996755'),(182, '86144954307-91205899085-83787574480-64584328544-29989089017-90773180420-37597557116-23240018220-42943203869-56358861074', '12253606949-52616685540-57025449502-08659091819-99350654253'),(666, '46177158076-77829356107-75497742711-22744929942-02508880660-64706335381-85929121921-52025894371-81466812238-46198422990', '50397307854-11012174149-84015713853-10395636357-48124033549'),(76, '53582160736-25196756050-40508576892-87096532940-15202529014-43508759608-17856315508-97657219441-81763039558-07170678076', '90415925484-74914914816-50257268464-23221673121-34456544942'),(845, '30635469607-41645427851-67119178387-42790453606-08992672631-08155604830-11343180718-70479564528-25019240959-03880533559', '01689393310-00951592915-61013261240-78651492715-53027162264'),(508, '08450342921-26720989661-46760556095-62584630151-42542947692-15250447333-06897261728-82335455487-66064998513-02708704480', '68131823456-17503685888-31027779231-06225775602-73340416753'),(704, '07910271014-97805500689-61685307201-84166816468-80240657402-44439408514-38968923305-33979388860-19818719915-52581293335', '52204708821-89280195752-24322176135-12012458927-81210841600'),(50, '48566538158-87919746518-60419523836-27021752622-46954013397-59646697076-09491827561-69222107127-61223390165-69979248586', '82131492440-92968578685-47990370602-93656240362-24756430569'),(509, '55793646769-12500821411-76563412808-66597297046-83112081050-09879560229-60014166431-17729367509-29300381794-16610131445', '93551416695-20591270145-66042095033-94514017131-09074725892'),(656, '07920340527-86959138732-39827478238-83377357187-16377741498-06235039682-60957011968-12321622732-69005859733-28053324097', '28485575277-55438505416-67353301901-16163460579-26072721771'),(407, '45663039795-75551972158-49506637233-12717395399-55051331172-76787605705-38333303413-93055543070-21571411474-93810617546', '51011056069-89773323684-04016713158-95157078376-91403187841'),(449, '42917766352-58709868158-64784829920-34955538728-53740694025-00002637627-87403913117-44122517150-90673756882-49301756484', '73175442967-91047811413-39249189779-66794265888-52948128460'),(768, '27436070651-70444688204-81698121769-20593186082-21151158019-33774049882-78162015022-16162891982-41961015671-13971891882', '86846131426-16237142390-27439551157-88455958498-46336730569'),(407, '18768918096-69957124039-51820870295-45421458580-28138042098-97565250133-22272802386-49269426413-17659357550-70254112766', '34934905896-97824788366-86767814391-31482094203-66761975940'),(105, '21690271081-99818878126-17254559998-72627073684-75821793512-34548763726-07264880475-72878127258-99337965531-04497186950', '30499240431-24145029111-20130461659-48285522960-29479297805'),(611, '48848156922-57593511833-92268500835-29949327058-51371947998-15244006575-86593969249-50822694871-74010013054-97640564309', '70720831737-66617559275-73081085972-45042430258-68069373726'),(416, '95705226864-34954159750-42176272535-83430727317-92841422810-89601660466-70419951288-43833585994-37079228200-30826684489', '36225063224-09465979106-74440446690-64772787950-62183725918'),(634, '66015419586-83108867902-28703608725-38347562642-22313913722-10938058475-96017442539-08458876736-77127751032-56233273457', '83263866138-11857670723-71896286256-36996705060-38733457270'),(169, '22554557656-64058926976-13426474749-54680857414-86141277509-19717405619-59839188356-75121733824-92590837323-86218443932', '77713919271-29004196088-22232427936-11671700630-39040788666'),(250, '90587286721-43867754451-20553008745-38055057817-57739183479-96783798915-24446189282-83376293493-05177385431-66013710837', '52211539613-98228238343-15313941742-24267630762-79722954283'),(170, '89907573390-30745684372-53577093963-97931655372-14664136761-29849092193-91845723620-71505326579-77870015114-22625626099', '22094112286-04972722376-87695654218-41699368551-55196490878'),(596, '90133980835-57548618278-78350020112-19152243072-29102294801-29075252636-47966863571-67234425829-94465654191-56923748496', '93375725128-13483451085-02102494050-32866407598-40360597030'),(24, '77416032556-14611596483-28818616758-10621086302-95562746357-60296425254-50784775673-40643696175-27810257372-29806334057', '18886877415-33974748716-17793808381-65554120408-92389432810'),(866, '70596387241-04061689984-80090382286-87186921957-96198668601-19168456786-92663725454-18624994943-83879853179-09480017922', '89319935621-07778810599-27548119907-92919209039-29749597401'),(19, '89880646340-95700555342-10274689555-26338151501-94421848995-61545828826-65325135987-27357115182-62060144711-77025945544', '22564684275-85230157498-07702135369-31746471258-48265985121'),(956, '41789594031-07997176270-82691552664-69716888212-51522836127-14565735950-78494735538-47199141707-40062516741-86500661414', '15859243822-41400034485-18708871694-90603076543-35099490257'),(182, '07192153568-51024724889-92415093073-12570804870-69892278183-73031749957-71365568057-25907910360-18022150435-34451192333', '12151107456-45362328111-88537673885-38015990336-66930904156'),(585, '16281526983-05041646851-25802225400-38065606311-52335847842-51387517852-93632323689-09137510230-71770135236-65054370018', '64053489091-14288043883-66268584023-91786174746-70585307884'),(533, '09099844304-22910238966-27316994920-87518688709-82921438043-23784863094-19350499504-46599397092-09972169928-72284281789', '43463884500-04633854925-75983097696-83338255469-48365118231'),(692, '48311868103-47509474944-43838115290-02363088369-00440055875-33345289103-35664115932-29811772380-68157426776-19432076120', '79554268557-91089764289-51102634847-06797055016-35799234283'),(988, '29946668917-53127747734-86561578120-05218046104-49749790231-78719014800-52787809935-70869663189-60052402071-43533685105', '16992258658-06537278032-53315721116-14480423261-93785825843'),(971, '82499809212-18132247346-70216783392-17822218246-80015428204-56540795499-52066652794-33459156840-43233521923-43402123003', '41452519521-57443347791-67173895081-38700599376-20729676077'),(614, '88036062133-75824582634-73882595508-23803301595-06374191492-04051857187-92232276688-15199202475-97599024660-24590685752', '58939618981-90917203210-41952785858-26621964818-98149862167'),(171, '29816233783-14026860877-58160537187-42654841842-39938609170-53387634583-86573675589-66124380696-50390545464-69621393382', '57650315021-12654753146-85958529956-50999444963-40527877120'),(589, '66789335168-86133655087-67153649199-05703382011-53254765095-17233549023-46819943697-18931852531-46154491945-14048159520', '98702589268-69140935466-41957749116-56711700390-27618046133'),(645, '68367382376-85421551902-44170664928-95705507107-35616435812-98969712934-94479393377-02920441975-51739502157-40454041333', '43178360701-76771138287-90870927327-40929116926-31649682690'),(794, '96271598257-33171625451-67108543210-56877353311-81460456328-38773357267-05345442462-55436692817-85076049207-61344717341', '82428314818-74633660649-81736610409-27777054254-89053206397'),(80, '03714153595-10060821823-08967149507-72909703048-90356762492-68676383740-85276285734-46057719049-74434884487-14116380177', '22336720975-48634504716-89076779852-51762247252-45830023404'),(655, '37787843818-17792288406-69980794030-78500021491-43896856091-36486745115-73519753269-27029082860-57873177485-62286323634', '53622193977-89055404687-00757808281-35101858793-26999283907'),(383, '86525256592-68384434641-87004171191-07268018700-11701932027-39766879849-82840256619-17814397247-90921015316-66308306432', '57299809036-49526786257-03482377626-68770679184-31322848751'),(858, '23675650166-87600324218-78279371189-97897419935-62815489828-84364145357-50217623813-91041448394-16074778162-80518672324', '14415401709-46493362606-35980380658-08214468641-59976017639'),(944, '73260234254-04791320878-95004751461-32985290740-27005770219-60974700718-87018899646-46305907433-80732131027-83916864721', '97405147502-50897531763-67535945729-35042589033-43493574985'),(937, '47283753021-98472562624-30654738434-96959137877-72042448606-36171187366-40214013759-11874799046-33560512335-43465716231', '05209740572-58227306953-51985554545-38107825342-71996332697'),(160, '78525060800-94955068542-23833459991-97871506331-98030058812-57530922290-70840078390-94563784761-71523239404-07558943175', '66911731158-46392329970-45269410591-00970744139-18933472922'),(911, '79662620676-98651141609-09365512562-43302388193-84956520197-22512206244-99821289398-00537005371-88922438632-05568785881', '01707451854-82853877610-06172355387-61452333304-51049222823'),(439, '22822372244-86430253172-60870009195-85747173058-74990075397-59218271319-07377981701-25666714562-22241772100-00708101063', '09945465158-21393021003-22728353969-20199323582-80556586072'),(734, '19275486480-26778263659-42020225684-82799607266-40302485257-81566281356-32640225188-54862718477-96748694462-24170946482', '21135143051-51561744319-14255601597-02009924532-57532594328'),(275, '97879028834-19309337869-71128518452-90282765524-42119570656-41477500983-43758246896-82398804882-11943471193-49218585711', '16468289409-40232320544-55613216750-97659146724-27592820386'),(778, '29495090139-54812220435-18771117998-63411859674-97265617608-16371385503-86607792640-42194680403-81422352652-94359712009', '09774532052-71988833577-92404985324-65658045743-90615710343'),(451, '16801183592-63470222694-90164944043-49014343181-23925072707-46185855515-69160137968-67899468982-68499922946-28659077552', '42985360464-70503764374-16997641089-30033588157-12641014384'),(411, '83780449938-29686077406-03060547498-15131589279-10954183460-16618482510-08614802507-61888877704-67950723964-78289813724', '49929246350-25229773042-70911554435-14870665571-48810475222'),(795, '78905573784-98849898817-19738227361-23449262275-92664213188-60134475753-26210978274-14758138965-27023838734-99610145977', '24994633651-77132456254-43020922936-96209725522-20384117767'),(633, '29791257523-06675266818-08447428628-22304946958-39421602767-78631439048-97759245574-34953319317-00023420442-88339527909', '05908243055-07688705544-06942128784-30105182320-18513830068'),(363, '82137236649-34477074407-19007803868-53979260546-51571293786-98355560192-64917516499-81585010589-83422079427-04520352993', '23994084885-51658981727-71531855977-87527301041-12611810516'),(852, '67660017758-21945815274-56733850271-02628466131-35517588948-68825340848-00768036804-70526578195-19469336797-83169827081', '70191797461-97803525521-27436174409-91946581803-87482346540'),(856, '47677571111-61672949471-67675558329-87554357525-23669386846-90828366446-23703614733-14048493589-15447654726-05328619155', '90827324902-80697882485-12922577689-47250210812-63725244349'),(5, '08817806532-83923689507-35099210734-24319960740-63323578366-08740704224-07829764362-32741916200-68151791486-95785283216', '88455330288-78329379503-11542731021-31443865584-39312828822'),(788, '84182499872-15490804117-26250826226-31355250957-69725334608-44234562013-93048621875-96231937550-09311720641-91028317266', '25180755840-49275122099-37344433746-15454672891-15935245743'),(713, '01698516490-94018248299-83230379570-34486174553-20393853097-59501050219-48693553906-97171357539-09988926916-57220355407', '37910428608-46575649021-66033607893-45249513689-55510613828'),(380, '15934757775-93603973510-39814908889-56990292754-80807604028-99734943239-97943917898-85274309441-92001726800-34467814106', '29249947259-72624395545-35006646967-43212252029-27274598125'),(194, '32639369671-27125550368-32428666560-76680886770-60177256103-56260424955-86339664527-12701598188-46774101448-35400459670', '56336289637-14461114472-46974840979-91313544778-44688767732'),(806, '83408420039-45521433312-51995530022-27404339938-38458984274-63933979706-27313848685-65617075405-89165025486-12375473897', '08795896975-58138522443-20086384224-04006368781-30971084869'),(973, '37657164258-56426559016-14350724277-36691472790-82617177830-17182821703-09873811371-31090379624-37556121886-14649485974', '75853941679-09195927043-85487086356-00402790942-75402717407'),(982, '15306623905-38952431288-91509522445-58826410466-75187251940-03206608206-62886656718-19797932717-28221820744-44571676042', '64024544730-09773647796-75998461507-94080236840-85923869096'),(848, '74723989293-98687198894-80835831065-35192115998-73514897524-00286326106-11297193267-26552180186-34816130795-75974208481', '06972236995-32520359591-26100188967-55476468935-18857647736'),(447, '54003905719-31650734395-16700163221-47165969123-76840647643-49930190117-18987443964-00924158901-27434991331-69950013442', '94506790998-86308693209-87623131177-29125665913-26370085963'),(605, '80663050324-74232219439-54487365729-82698522439-85567066447-32748935158-18493871422-52007544916-20505937217-91782376726', '16183708440-12061480906-52474091458-34950493481-50246366257'),(444, '31554857833-87798457081-72269281243-22286166317-31385569716-19035741310-85078385082-19851724832-99895091290-48708363100', '78925088292-07661478873-79858708030-04160734600-20298727515'),(474, '88506366664-74220036048-97993423551-58536908213-56581978004-86828119504-56405576920-22268887731-20471841781-80526223216', '54998645512-85763248184-35298575473-65675741915-34914015852'),(630, '13936137614-53355323969-15966883294-94051020088-07107708770-84279366178-98537826438-06760775902-75777975794-66048706115', '57250367147-47520871046-40176612205-86545204757-81562072384'),(343, '72073661139-01794932725-82848098170-52024376790-02123776466-00758115778-06905683392-06224275894-35522548958-09389082835', '73590812094-62971427504-06337070191-14784803840-83524548553'),(18, '52408350915-15246179845-20303160911-45761154974-04805810508-90336178849-74130118631-42352647945-58002440384-56716073936', '84012492816-56359153792-31391647502-95561248105-36076315687'),(47, '00941328593-18683759224-26445638717-54349525949-83031224600-18819384044-81500493590-76606392829-74549105510-51041790570', '09070897445-17269714891-76864301339-45536065870-44426698812'),(683, '19506288419-43676736713-11574000603-20281545764-68062080770-29713682981-63880090792-66706716643-47999445193-94205726574', '79959716089-54425219759-89137806809-99367331431-37526988133'),(142, '54614716089-94873774566-70745255657-35144643977-50672808526-20433762132-01898425040-84099867846-68054827053-96832478921', '29686695831-09798410514-72699543811-86420924511-75971176614'),(517, '83084179674-14124670865-56127792226-96652448903-51385338929-92105760806-48679320948-04342193747-10068349822-95160544730', '09453315581-58846004162-76388527361-42326454716-02390083726'),(714, '03250674224-42484096267-73689574492-13445790375-72058313503-52693331073-90490354459-52766127054-30801523768-20254884203', '04050211898-25337303895-81633334242-71693378383-21100788959'),(516, '26367878047-41967455542-65445116753-18804472290-77532951859-31284688459-30725477474-69276392327-34132804700-37446149378', '74245923386-50826629712-45449702033-58191270728-05374273032'),(856, '24165240846-08412942106-94261312929-43762257063-13701634611-48970985297-53972098260-22628365747-30330074348-59701158973', '31840593423-27182432427-04092320239-60334261982-62709431816'),(440, '28185235414-73363160062-18037263886-54047398007-99130917691-78344560794-31785453589-86188534468-17195336234-53250957669', '65278453726-12912990037-60890518646-17789177396-89961868201'),(957, '16556975127-12265301918-29608916702-39140107507-68816915755-15418159693-42318561040-43646047201-99505473783-98962233106', '79283973490-07772552738-22831082136-40601485264-01488405971'),(595, '79967823351-61673014226-72770676879-34464474654-55855699561-74679702858-61076817528-26450341596-09990275996-57778325373', '99295017025-98416970905-88875620522-78920990793-28535739971'),(876, '17947974929-35143935473-13647046394-01770344059-50895475352-05434910711-98662605221-28702678847-62626410565-03639049518', '26232593288-16862690625-50320514639-20448323170-15399607958'),(425, '63894233777-31886759539-08719858430-27320803406-10353618544-32610589288-90056172144-84191541769-47238528574-58297773270', '68352590123-47325254933-96257668845-24972663177-73553111580'),(48, '78819778466-81491581728-06607354525-22236119945-82439384140-16028429364-39951453009-51057509101-67935464264-61241952069', '42644960314-81824545533-24726702292-50694206184-06309797781'),(716, '11054743157-49936506334-86318316162-80101784239-58135650023-75370585560-38594289906-53699735365-52103183750-98538995826', '50280290443-24721133730-41760488016-95159609995-34318608024'),(443, '53481021503-71968066085-24090599682-04154363631-86461459344-52952930511-00567777136-83738488308-42583494381-11118086275', '69339010591-45345514019-26990314651-69918037377-83577970762'),(996, '84964491861-30796863638-65304852657-02128706716-81036300975-49502309502-96123394930-58177157158-21469456673-77746702465', '98713551822-73218310387-87765455883-61335058444-60068126589'),(993, '52196024763-94111102736-79010662098-49787002017-88640435648-64171687206-69703017669-04657771171-24888112176-66171982201', '89086374589-83463306504-92733691483-14992597774-67892169324'),(217, '22644457769-63802000323-32689276791-41440614418-03561297236-99960611124-44467865824-54614846459-96779619199-21944458292', '88615630748-85543993525-97327654161-94089421546-39238472028'),(817, '43419030369-51312015566-38670542037-99795677748-97499787352-96873463550-23577108697-59442208987-79948028504-13901327563', '50799466537-36256311739-08450212247-74893037342-21801501282'),(852, '66986431004-53589836602-73917695861-46564727338-12839721658-28548064504-81038653424-19454390584-72225071086-96386011752', '16044127642-20840759725-57714996073-84612684363-11295032998'),(55, '11100211066-09996665078-13730151431-39572220198-04854349853-79737507286-40495578830-62953433541-71334250537-39732021788', '44048843676-10490290708-32426663418-33672185132-72183918469'),(833, '88743624554-55128014449-42463332101-45913985935-04583964387-10698525934-47968838923-09868747703-96459058780-41071529628', '52359597741-43694547062-74365568130-35947951752-67528836371'),(559, '19998154821-65857662471-92344170294-50126680582-66673077561-15532801875-32211299702-13521701620-38879679234-29018533454', '09870338164-24041800826-35216659633-01275010984-88531733108'),(367, '05404481063-86492626644-18646030872-66476939035-24532786573-30746200751-75130939678-01437011370-06383838115-92013034999', '12446068337-24490820008-86395573375-42834596541-15815768320'),(511, '21549538211-57916095901-10719381829-72614920131-79532547511-03102317483-91900925768-06330660869-67698415703-14811157871', '52950013335-93784877014-78269056901-94672585132-17809627738'),(553, '06657502319-63490829519-64445558849-40512358302-94349381668-00645919326-08860055621-41690903274-02049409625-60523466806', '45778770808-91974494131-77797267572-22967786218-87770834242'),(83, '55473432507-08936548770-29138952820-73735484976-00147438110-02903251705-33195435330-32178985474-09808413814-19964960625', '53347994074-12436036883-06746494643-72340437633-28327563858'),(146, '18027740415-13869175144-36246462268-88571313637-04802310942-65548442345-70357741274-69871136189-68976363103-61167341257', '40702498481-70598818306-33584622609-17702053463-73091084302'),(780, '52323419256-02091542908-49795286944-52802590566-18957358152-92551474262-18209981889-95999165026-64819297849-63648355479', '33976331523-35465747413-89785472180-89931543749-60213950795'),(108, '29595292390-96718263890-79811363830-34003978903-16099258907-40285400989-26665182756-50608536909-29779750937-56016101125', '99250530820-48005166836-63508719393-90704891119-08106707292'),(486, '86231407930-71419226654-56136875123-70900703684-78410368841-49057329118-20354856478-80728025967-58056256214-29123032669', '57837472209-73820084721-24947351012-29914554738-13134577035'),(218, '64553156958-12711557551-09026979244-95328927022-25425764675-45299459125-34741904600-94571860112-53752890211-65944611854', '63035009310-00443267443-36800757746-15832160638-58188083549'),(405, '32631448954-81948279551-36623621784-33038587159-39966028683-67287067640-22550418983-16374682644-23350664192-73053111174', '07948744808-76712341842-94194059519-83695391112-62929060786'),(778, '09951085435-24929848523-34695137219-79235889364-59948291549-33907464802-94467107518-87988951249-62536028399-15400384352', '90890237304-31672978026-08968197737-55003235759-65455391859'),(406, '72418290724-90203252471-30740701068-05959795396-87178149664-10292401949-11721960228-40085182752-26847537222-32057705502', '41772667548-59567956649-68270814375-62852347560-40863159607'),(632, '10227297414-14475868288-83925319652-41662368257-06708523548-40372296576-56939244105-27707643342-29863346144-42230946736', '51922373641-45346536016-61974842150-81796540121-63696056764'),(508, '76222092813-29198471254-53781736985-13438734795-12544804508-03351418276-54933383370-26306057085-93465649395-44143090683', '92546779484-67010075381-42140404819-14899698354-19996886153'),(761, '15763475490-96029694290-80062736962-95331268356-57572642010-61678880388-72983397954-76517601004-06221726308-81699729678', '87084657901-59794746158-60691909571-56427177288-15106473194'),(760, '65648958528-37369738739-96651630838-08436659482-11978534024-02846236632-75031835266-53557092608-49895431568-58052229637', '15038093819-96703734966-33138509284-57491343805-03593264046'),(33, '95659846373-13347813476-56689711708-80102825532-74861262639-05863843151-87561051919-02604349920-59079534179-35923715394', '87002459077-10495594406-48528916491-32097651110-55766782426'),(98, '13144224971-85787062009-31715555257-11868677187-14011984850-02614075893-94397165103-64091774046-31468192405-84406610082', '78999869444-77415794324-62250569600-26513352453-09500543237'),(951, '22735861047-26417236926-96311360376-71051024515-71579676787-88088209452-71585013488-63592295882-64162725408-17553083798', '81848270031-92415256339-35451613144-06421147816-69184073143'),(780, '24063636917-74298339875-14792910300-45632736876-53325216094-50237221260-47945838396-52378379969-19209438103-86753945633', '11282204262-15986075989-51245581635-35726191650-54433521685'),(912, '11320775875-00307198947-31128343927-63365443570-94320163087-94861943564-23181291102-04628793188-81568188374-55087065147', '89173936893-92656408054-40244732386-60408783592-25893105359'),(811, '75439785429-35320559947-89775669223-23442669447-16021749096-91985816706-24707646466-93141533140-52074137930-60813160663', '70045045674-22832750567-19449085512-52933502209-28436705493'),(280, '77586706839-93931683653-90002703738-57449460536-62154230088-91656141340-53624748874-32284739624-17594251293-31080625966', '79699272791-47956253574-13514017819-03967080997-10079901631'),(612, '55330086631-53191330430-75465665429-25161075201-54051350388-54662673008-83465266761-06817179018-79846904865-54811764703', '07656021841-67868529106-91713948044-06467029623-93823468524'),(335, '14531139743-19685467872-62640794996-43408727913-57298107953-84246643297-40531631779-57243128486-80920274306-71396451942', '18614825029-91220415056-06871121197-37907318948-64483231511'),(751, '97106919276-39446401283-75937856189-96369964129-99229315078-41046052610-44900845465-04004962512-35673116153-76404051054', '78716143467-71921487260-05306366324-22521023561-96472349549'),(3, '60615759742-42693872872-43417156926-16375056051-47559204570-65527831476-42100483220-63766563040-43613037779-91354997009', '49988519458-46067996899-45299349927-82102526117-08266526174'),(55, '47943457203-36954695377-39607403153-45330238367-15605069393-86091885754-66635029693-13029290401-15527530511-37931241437', '58284754616-78299119046-00887980177-32968445041-50965623984'),(940, '26552270838-97170567396-41675891462-88890971354-35805989287-27125287582-47300017901-94639800229-55145780251-28829425598', '71190676212-90471268910-90087857992-58352397416-76361267586'),(311, '31355966535-16850060005-83727190239-60765405218-01032779261-62332206877-02002042012-85475302801-76784776455-45306560895', '77315034941-03559245382-28048038326-02854677814-26555540162'),(215, '26438275857-47523201341-97693726750-54421444771-38942717117-13590404701-00560741953-90382073649-50742347840-18124209530', '01387523866-13629160937-39441683742-31977124262-23056147316'),(139, '21975730033-68531730418-44940521412-09610058959-95390837424-13856342199-42898729525-30253837581-62008481987-84635408427', '26076898244-44352039045-54196374280-62247319967-87121680191'),(508, '76890857411-31869209123-78932303993-47943848912-65797821284-06859480448-85686522147-25006566961-15355043285-00813909066', '49544093151-45776323048-56165259244-15495414473-48374688631'),(881, '99634101543-07888742414-75224707543-95401293459-84525861845-65903811109-18369403595-78669392845-17977877814-33962927185', '00937061146-16216822641-53267715528-73154905120-13188453261'),(572, '25476197046-51278249831-50690605483-10814274232-62189950201-12588702036-67572949150-80875171068-39450448043-85217333982', '62713026799-11366405784-62694408555-84518257905-82223218429'),(500, '32198028283-51418233745-50466320023-04868428947-00987585558-08901833564-37399958432-00721261973-81294098189-29109247380', '15291850018-23755912492-76614727681-32451905292-08269140630'),(416, '04769506283-92306261340-84581884637-01759903217-45571175466-24522784998-02227749001-09568040212-01307977352-71891698358', '76207065322-26735647780-88730432627-61285655795-35270382243'),(234, '35952731132-59166270056-37493409412-95545941837-50543779985-96691491635-96973492804-32399634419-83581675814-48844659379', '97407926364-64719676600-71720662658-61598674884-76077837609'),(947, '42077533674-87913540163-35157697804-37399806673-88491782960-13480967240-61294152945-27440983640-77057197804-48758204822', '18911612457-10911174878-95982630834-08616095649-56235833465'),(612, '46901339723-26198557491-49171217362-92432702879-53986682192-75254378578-70356998284-79346614746-44813487459-21630710191', '53682094098-00866985941-50506946380-66480189812-43722386928'),(587, '49784785273-80255093653-62082830721-38392435973-07323298245-29114664983-95075655411-90523720767-12641614252-38951142789', '36226811712-80665343834-99136053933-59932899807-73013738378'),(3, '78627858697-22208201785-36869445657-77457697725-04276140782-95460732328-25300337751-27334070528-24896378003-18738561072', '05817480376-07951157137-36756419521-43061738690-73345644971'),(208, '16889514216-22285163935-77240601618-38884296121-42006885543-79285119507-95615916356-37238349418-09314888287-00687618143', '16942249801-37401438468-26069287404-29617704224-33200832134'),(416, '17226384089-11858760411-85925350605-90760537571-83593242196-88889407691-98548581134-36258060977-24326840081-45746996638', '60077542491-83929688006-24704582252-40067065347-24579343441'),(391, '87736611175-83121204874-30322046004-03465921963-63866768340-56037844331-12656952953-01476819543-53224912255-72540088319', '86539236885-64302488982-91580009361-29687234758-34674378623'),(866, '00248652384-84522184912-97268970706-08435449479-40192341554-39619800524-03256101746-66126259588-75979784550-76276607098', '45476053875-55423576478-92518369273-23442654329-92639839202'),(969, '38842826388-33057818278-78495852454-38389552097-96634703282-37660931135-77553747233-77507731218-42707471829-13768357626', '61682975135-44090591971-16840288127-69845061796-12063161565'),(364, '36486661256-94988485876-70295176646-23835363894-04802299563-45297791393-69186750569-31841833382-15526334889-55699681007', '48736810902-78227074136-90692592726-68041024247-12546808283'),(790, '22508499918-84791494955-94162981981-70363861913-88812623562-40043933412-59633917673-75998727398-03084192447-02327944733', '41624687455-80290558797-75493406677-91133239117-70414258896'),(714, '60606120476-54933959400-47309452105-32712132437-43366473969-32055219086-40282324314-34583401808-84755661874-67590695246', '22965262046-32436192070-23796331760-02253960359-98678252425'),(824, '73249835530-20434169392-93214800599-76162265929-64795936651-62099473599-41351308085-40077556865-19590579655-34381747704', '91831904162-59547000498-22216590544-46502592840-47942518716'),(569, '97739198444-33381583208-70299073993-66343956339-16051552912-37805960792-22456364754-91454025417-99731376312-05980001798', '81160954464-54526771673-71880181506-48146298228-74157323990'),(186, '22684964251-99823724350-02936134576-74650710016-50532193370-58845027721-23781759738-61833922949-64656642102-53604791263', '32869854643-83871899960-36241003188-95148011443-67459145782'),(707, '06689146564-05291603814-94568263483-47198771390-22306632899-37520877670-92807761792-17264418769-61126878685-12991462385', '73848386815-01450921712-53043642873-91856711329-64767346979'),(109, '08718232865-70758870689-48897828639-59772527811-61739799339-35835948648-07270110777-32716545325-17780909391-11879141236', '29735214550-54140165721-80064839293-98086186866-21017435708'),(982, '87408593456-33237586560-05010494903-96826171347-28134160495-79802721224-87367389670-41399618249-11768589248-94463474947', '17541527221-55555237187-46156580436-59414328696-71491960308'),(152, '21942540472-88421421218-52162018850-05977864782-26025467920-47539518042-21208138970-68407537048-23815774696-11425847645', '45951782255-32199111306-24651207290-02131937536-04883578867'),(136, '97913212313-25944028604-97303415297-97736105466-00552669214-32172264354-87926190139-36314109411-57993004170-30360375412', '51217527459-52523729014-31931223735-40055146528-57798687453'),(295, '76758889166-06435972263-70070874017-46826525655-26528359313-16791799090-83207101035-49240686611-96463962508-34690628897', '10022843318-15958297666-99857928406-57664683332-73881444970'),(759, '00538670031-26272451432-62924603463-02618106972-07967191226-56462761183-47821468811-62338398408-76689655329-25879083685', '28101385272-08519084869-42497414931-85242631430-04703671115'),(902, '69733589763-45967279015-73547580763-18726704395-02497155114-61733190898-61768023955-68211568794-38029518678-43774201506', '64616724200-24756221435-38047842007-45016903519-51212270638'),(401, '18015654835-96163619811-16505815888-84257205860-80243486831-64645753884-71647437852-13230735176-37680684738-78283459484', '84971108682-75878530595-24150760265-69474309335-74246989704'),(360, '34326977611-57658797850-29342210739-61839691488-15994076706-67517410275-87740938778-99647704982-60999518121-20092748134', '75461136995-49876430395-42928773921-92261660077-32957821595'),(153, '00620141922-59216802159-96922602315-31751084086-50597732339-14426470403-62213887862-19704939194-33227579824-66227330831', '16237432172-05066917084-32839991044-32737010998-56652763724'),(58, '29457254900-25201914131-84125745371-27007023947-25345130665-85012184563-18363921403-73069133302-95939934764-38136403877', '08896438583-45483092904-41986085264-82039741225-31553736539'),(79, '93500825004-78004886777-56589571533-24732874747-12824253827-12474801926-09086668069-81827828050-55308970907-77206084330', '29358608299-43429429188-03326869740-63534652593-57947247701'),(762, '69002004802-50692483574-93710735691-26069752427-36286454405-10853245939-29818874418-83477454416-66090906999-64757755544', '15566649931-15751966104-17097064555-47410249631-62629347419'),(993, '26470547150-84335902588-87830007505-79910698395-46087963144-72785765791-51927425170-17677166014-05071297959-52297877907', '81288998846-68041212326-27691442247-12458927119-57784996387'),(416, '65170610380-92758220188-20482325541-45180888204-94902710015-96473354344-23434789549-96286142130-97167091039-07405333546', '36134749623-22255071697-84553547317-11581734309-67207608919'),(689, '16466913849-74214522362-67468987217-85763824191-85732376522-21269845564-92655609606-03456179930-42496535107-54933661910', '71071574473-79435203958-59472041237-39209030870-03960299392'),(531, '68506292169-86913425686-24353172805-94695187841-11542417158-86054386161-64299470044-27255330253-19316813708-60375421651', '01425057938-41088451393-45054542263-63948570887-45017576408'),(555, '60139814838-30244870090-04197621740-69627526747-85752373308-46505242264-87158801397-35810780191-10712339418-16453296873', '98420987958-28341353759-15653118912-66370889047-06776662251'),(993, '36584540880-85639559730-97331799895-33183003588-86049856612-68336827593-27653087237-11552609231-44198511587-12544973422', '63437957676-46333397265-69291313259-87399468040-65138698364'),(826, '56999176749-12280323789-63632314313-27924496359-19197578388-53322426466-66064998089-22797361174-36033516757-20330605950', '61677038016-45766476530-77574657719-20631107617-68402189764'),(430, '15936457210-62869630024-50705019801-58502305720-83131113415-21336115823-78113887678-46070785974-22641638672-93375094109', '23807156672-96857820422-45630722051-47022917701-45318524909'),(666, '73565892586-67509155328-76412309193-91066694900-52040606608-42032394286-25993306230-98866477394-17906293356-65510958021', '86643801127-48631604886-37079023724-42394038844-97119877062'),(531, '99067419595-96410953185-90145320016-22509797874-02728445240-60161097492-10905222215-45841433013-23348924716-97114256692', '88649864581-70878317027-85107399743-96607147074-46784425395'),(344, '27394716413-71519460415-94597833973-00669741849-22416533730-61738359718-42203804441-17662436114-67569357242-63164940361', '27388074158-11087263594-48674468540-66283616024-69706819230'),(378, '74912192689-98143854454-51726729901-82660969397-16709615309-99338455898-58156372663-99654458633-24282057350-87154813136', '49752368626-64499225301-83492649656-53766213958-71893132442'),(812, '81722662216-20849692877-65325851383-28370789857-96997875243-27679963737-87421127719-22436490423-71249398458-09197197877', '37624387179-95378635823-70632376247-12594273004-94871630538'),(692, '91515339279-12583829708-10561139914-50640285414-13250271599-85873436860-81900729327-18065606449-48457329124-55828275243', '98113055815-10684001080-41009504718-78006357076-36826416057'),(978, '37479607376-79006868708-72797955342-30528668463-27919004009-93019538788-23176505007-43730609018-62114398382-71422436818', '28666381313-90980328581-03889593580-77016044524-64600334598'),(946, '12898008320-98453789730-88594498710-88044834780-42166681313-74830321261-79029807450-87378599132-83055339464-43709500898', '57944673364-14256334678-68997332285-46952365459-24735071444'),(415, '88824892359-69492716398-89634507709-59807133185-45148592232-89128559776-28870517211-75322578003-76575534094-78813476167', '34403541516-30226585461-26869407628-82607843231-79079997964'),(375, '14202882902-99734172821-38600935279-40490221401-38554367128-34201323174-90758124713-54186647530-20238470637-18837055941', '63780506972-18071338688-42894112744-66132326859-66392628715'),(457, '66158070753-33274865868-11095925387-27708094576-24772616074-62797227769-92390842870-05175432583-45807665277-79082809539', '68783132497-46927732272-71065043636-56942783860-91893429722'),(404, '33607104361-28850374174-89407620441-83393173253-18339040222-72418859239-53769828076-83928311453-30095911966-11438452427', '19570305365-15033595518-39750880176-23230831509-64602221184'),(162, '67350772625-96929822028-59798041656-87881514972-05973740338-42666940481-74106243301-98560804427-63967572602-74291507182', '62459323894-54587375116-66387913353-08518842805-59310826489'),(706, '21048524600-14889919653-59468851838-00883596101-31406458086-68418360056-92418442017-70796424220-04407073709-47402172193', '19020103893-70079384557-88493044306-09613182803-03688109902'),(9, '41552353406-01333705372-02529305804-72006358775-44338131868-30730066748-31472789882-93170483482-85987214443-88328865566', '99919178213-16603378717-19247107097-41954687856-33561300120'),(208, '01758637212-42046442497-48634499291-87220688055-62262680033-72049456584-29485829174-59340801865-66775758732-38233802620', '82578310205-54722914201-94819661550-38370679258-44417074041'),(780, '38534672919-52373468876-83095762076-62121955906-92785400609-00164034748-97573031185-14444727291-41551890417-35722806936', '46232057671-35289555628-23878118423-45843961314-65985344454'),(503, '18848589572-80709687719-68592121342-76455846784-65891346169-31524700920-29101438864-12975623732-57251406394-75595211002', '56480343307-00077247544-87958543603-79078801753-14642276352'),(537, '75481764677-78496917875-81999532787-21523212057-42229576835-22546535150-28586758364-31121606538-54041724957-80676518500', '06910275401-20590490918-74828271065-66334034180-79020297861'),(328, '75422221593-78569654953-03393234647-09483552837-29984390801-19889348522-16923300145-26240017529-61634073983-36109737998', '40353531844-98132446625-75004047621-82079356358-86740999730'),(198, '82068675760-98130129552-14983754300-41501852561-72914499823-69358555060-60900317718-46698188216-02962708442-48460140902', '44255105833-02171212898-04226314567-66627647428-32549820504'),(494, '31009936907-65822421349-21312734590-54532024042-52411231955-25710430517-58704274846-78763841369-17328974517-37142523957', '32805990730-30953277257-17986600441-03918973556-44819692691'),(467, '47045790745-93301839839-71023313181-77310418796-93727586746-22321270385-32560672284-32893596175-49538265302-91475604389', '94184890641-24249170668-24639089431-95360080227-71002007487'),(456, '49863265420-69799518996-63924564772-73137627587-67872237228-32057309840-73176097168-17528620068-30579697693-37188000875', '91474096315-53909458722-98025853121-43229143428-24145236951'),(92, '77143414057-90869259601-88506436133-62830483552-43824344482-64227517077-33823709706-80015260097-93105323632-15514533136', '20948212437-40023285200-21529432791-29282095961-63475258723'),(180, '21909136830-53512542204-47918963152-95170893801-23228839546-25364128182-94649935907-67675077651-93291626262-27447363261', '52860951140-09120408794-06992599980-15684559028-43790743229'),(653, '30993732889-68627237540-57522611349-66044060004-00488872156-80721152160-69574212375-27823972190-59338855552-97926407918', '71205409254-58857807342-94849079564-54545821906-80746611612'),(762, '29012078388-42029472669-18365358348-24090665443-41797586564-43475486361-75826477581-96824345854-66494156871-57348636879', '15672638678-21420243319-55538280308-56895390967-82316421475'),(129, '24965206048-47064895711-50503299799-28429100853-66846398281-79808296693-14959965326-58547539625-06066574512-56934684355', '41726178350-37453272534-53985097820-97394246941-67777438178'),(849, '89590681495-67579582240-97798770820-11011698641-13302153707-25483991203-20212432232-64103308531-56881792384-10614019961', '15557199903-41760301757-78899798417-79962843845-83877575910'),(175, '99815118325-42556161697-46749713447-94663593949-16869729281-84705892371-86076838543-62711146707-56275317329-19119265273', '91683686944-08698994128-23282861712-40275616997-42026397635'),(91, '21796254825-67267786213-06303495512-12659697002-06899496616-74484362585-53944939366-20130384597-02902304394-52741271387', '05810816953-74478736619-47163337280-66931465996-88977255713'),(715, '81933846726-96496977662-94557989389-30738712102-13422819574-96106413509-79567314617-87128663790-69184955529-87584430517', '49978289166-63509531111-00586553041-03556704998-62399419206'),(94, '71385535424-56284601947-41172903770-51785237710-12390179688-52057228303-23781681591-52969454168-27651685384-27289200522', '77029145761-84342563071-06968783545-25554495465-01942083637'),(698, '23553765568-69406373159-27282129960-45851795719-65947441098-81662940783-12030160181-69002898463-47961515186-25069638536', '34132770233-04494801261-43401823587-99167567969-49252674051'),(944, '65112396173-86670398568-03277955843-38956274038-36339718807-69303413427-42491180578-45248080376-86721930479-47223932348', '41756413879-86998575948-81325895031-24337721865-69809966594'),(803, '06367347795-14144160085-25163883323-40234061287-75932700674-11196534722-29354621268-90570821028-84251693257-75499450420', '37081064084-68192734892-64231807331-73039216496-69772330076'),(945, '99538412743-82011442064-00364709718-27455997042-29927129082-25985816313-29328094866-59880730661-30369219751-51225709113', '87192121542-56793831197-04534405564-71647109726-65785281776'),(726, '58414682073-46133249891-53511384655-53130257984-16049777626-68449814330-81100197967-49845443093-30020932155-55398277362', '20223677393-00653114984-16514715691-98775808710-15487436784'),(669, '78006687702-58864270843-98955970802-92834158055-57895103953-65882260520-09377337092-97583472787-53385364026-46592474256', '82747434157-35582811381-30262136348-31439657104-96803985822'),(404, '21418069219-39943873223-10417497214-79222754323-16832554249-23528021498-87994819752-62444535945-74936686019-60499925931', '57815277716-83512390670-83610529249-42107501547-12123409968'),(776, '69783391319-63716634465-56274383094-06593879842-01717834489-56014461892-38891158668-40895176139-20588205905-93161937578', '82040322868-32426290319-32656154331-60522179391-25458098162'),(914, '33869497409-38299541250-22306243700-75841273228-85173698058-16044195199-10765560354-13075559562-73162813956-07749610208', '42655754880-11606046717-12182635654-48247018533-54898911747'),(288, '71873975670-07385158291-19501649661-59429306857-44666877340-15704684833-86004992685-43752774996-01015127659-53760917030', '39389867119-11416478248-30649328249-57646784994-98781714482'),(176, '34625943404-57922886647-95005350813-46785001062-93925010797-38335449900-02312971968-68206544381-40098159810-11030941590', '65588089915-34116288709-32802472651-09094849105-35536865341'),(403, '07871248665-39258763262-55425938653-66327218114-64128398199-35892814263-93240477332-18527203167-29701717846-70556891832', '71271316894-60870425519-75416019832-74903875828-92634917268'),(225, '65914371565-53765540043-54070338867-06156110624-00049662133-38602433588-72358446698-10575140745-58463553177-10444818015', '65702726820-27295019141-73442614360-80654735242-16587466175'),(790, '56756614439-23372928546-34777069296-23465222908-35366877266-83044409417-95206250641-14313628079-29401409410-26133452160', '42101585755-57574599855-71513750042-86159459539-17687802488'),(387, '44643191492-19826594508-63141843385-76797176766-81765843808-12531463110-63960864783-25488338817-43127666486-24977785147', '53754918323-94613024918-46955060964-54744359596-08327940392'),(102, '52118788249-99018905292-51060012136-12183964485-90185160488-45996246319-53984933899-37120329861-93904493651-48261336697', '15510045499-43191507242-20007047702-00864351014-20348014663'),(495, '68671181304-56259256956-23189251686-71471397506-15223166437-58379004228-85054638845-08720850267-61759994613-16911701387', '50041732342-14677099544-87738133532-11531709172-38912216224'),(99, '38501458857-20334191921-71488071104-76118326744-17154003496-94323998555-03210136165-74131063646-23774572049-35636460107', '75916918241-92345427216-36296126088-37884781645-56193405963'),(903, '76714314234-24953014595-66620548047-70923497391-75841661755-66236919041-59158634424-91411796905-57289030345-24826238219', '45901692126-58593329230-53213141531-79073445889-54562636873'),(705, '91564885384-87374824565-08767080998-75093357886-68992281828-86989431513-33589758670-97859237038-58663391421-21751968151', '44609594801-21184238980-43544963638-58521260093-27350395697'),(906, '72944748770-70603372554-97775239554-19148220624-96728935839-88594823685-15084621211-12540718248-58897526211-19765718713', '09187931512-99558250410-75487789335-83105262438-43771548938'),(606, '25994246300-05540385930-18626112678-44785598318-84466460464-84535010783-18830215973-19302186837-09319604890-24821133327', '20667935345-16073165477-80058216462-00850539253-58716406094'),(465, '85185215152-26886678854-34747437378-96647788590-11462875858-62216352637-80386272736-14733798377-88617804481-17958602363', '47955604642-87848008290-73300821038-09023987281-72886127329'),(542, '98273802235-23821329242-69223046300-90583952777-38961507949-89975800708-90930891664-98341157758-55190411501-60309991944', '67153978540-71070050990-58510899155-89306020611-90459639846'),(152, '86136087490-54142464814-42119117616-50102210329-62225082080-25100178140-76058211352-17640253797-53676365950-21888852905', '74162624622-84333235043-37054645573-70515119572-09438687650'),(702, '58953426860-36297833184-93692936929-01544002092-57152612973-19937404900-12866587478-34205776362-94713611309-12775165474', '22614095224-33973562125-51847802503-30230043183-05822861547'),(41, '72947257133-44373405023-16819613273-65885025526-14053209567-82626240012-12390648509-72033172347-44977481453-93058418661', '51827787842-20635181476-94221825599-22807314273-29968369834'),(723, '72607100175-08191996027-20238004890-63948343601-83807452241-02212294806-97545899973-37361342748-08182738493-77873469040', '26756615669-71130880759-97600805728-38044233565-36691927525'),(339, '34682545364-63056163129-13281560205-64830162743-31330731612-80869481230-15197650193-80478138788-70136918600-04122896553', '73300559298-65968642689-86863208377-69463472480-88816817697'),(472, '36272859947-42145394379-93818038460-85709787698-55776749738-77742345715-15775595192-00882934506-25338309361-31088552417', '65389137899-48327179805-82672580338-04825957624-40900733972'),(525, '08450906719-20057853050-95932653644-23995591415-33994050955-75983589833-53379023689-83460247266-39258187853-61345269070', '13452854888-43479832962-39940662289-31986434347-93043816673'),(845, '10202348121-83595075668-15563297183-24193774710-04097190633-53855752153-27198432684-94454906054-70122659478-99715609871', '78238031490-79426827355-10042984564-96517078005-56864764386'),(958, '10782330004-52637830130-60604048472-31092906715-51213257465-40291729430-02316044134-47893951187-20847678141-07455942694', '59433649854-74761783075-29452440811-75309302203-80779972894'),(605, '02124185347-83744950431-28528176262-98869889520-24980752701-79904194265-98245333231-86540966026-49124447535-09268169471', '16521295485-54011197438-24589109979-52964100599-05120648219'),(304, '30463274018-27367143176-49436793347-25020514518-86613279130-84481598352-86821377701-90029823238-12506446104-71970469484', '61417087626-14508990194-38690857986-11706762251-13932474981'),(659, '58776777159-69455342600-89776303366-41158383050-05331016213-05795189238-89675221496-16104941691-07216014044-85518854954', '03661329651-84859693349-66880026071-90136279403-17930705524'),(354, '69065792350-75497161279-69404151112-83146014826-76152556353-98985885355-70628223549-40278805341-31952176407-18053723469', '11271744922-51054991293-14169488194-34513159989-64915691182'),(25, '70192249638-52065085596-99019573316-27102630577-70988996689-39615743039-56376921557-86372731379-94612525830-04972652683', '12999242643-36883765647-06460138357-10459587672-45875806618'),(886, '72261974084-41888161514-94952790962-33338008160-01754921651-02282765947-46165865339-40430300002-73137972146-08641581584', '56760460913-19622800663-35737985657-39939796471-67885987086'),(642, '63726796339-39709449104-74372696561-43419884239-43922367082-36481294352-97782329937-01472311806-20841867603-78970576510', '97831478285-24431418405-33909689660-68719153304-52041564767'),(571, '69766455897-03001020621-81373079883-79230179835-14719072914-44327505952-16744555346-78780925250-68982942943-26869691555', '32827112674-87078593964-54524666196-87607084013-61420115271'),(414, '16432216248-44147627353-51119828450-96419332561-24810060505-49464907801-30334571275-49292501877-82067636543-80370594122', '56438540839-13974405692-62384516104-84429575590-57392176290'),(41, '09364206289-22815311616-38311985131-29716614411-12567516158-11080249283-26798412405-96039944699-80672619040-94994634110', '12981579659-97550282506-84885433246-44797515839-38915071020'),(207, '99235422506-96561000580-98103683242-98444474305-08150207751-06527343162-67831974870-81956284398-85123237928-30616495176', '06585384176-21445401961-47563054734-84774390845-01192215459'),(617, '28078823582-53008039413-85086611883-25320702940-91594931789-16453828242-24178413393-29538155543-66110601759-93960009820', '30321579184-97893422704-38789409466-95121734586-21133091266'),(624, '06671466142-49877906374-30296266180-94265994727-71546909894-76360641814-69997406109-03717453983-10676634692-85619457916', '58829167061-96253457026-04859687366-32492958072-55872922562'),(613, '32852256977-80768461609-83729011119-08420689382-67274007006-22800650679-78165776298-23205729768-82172550266-24316147801', '13562775079-67779908198-35281359001-78590377874-57309031907'),(170, '89201028474-36304394831-80495655883-09312623889-37900869177-27020343759-28983439396-02439430499-24078868763-42062151133', '21976965407-26796301520-61051373509-27390191950-81894739916'),(795, '50535189349-99033193844-83547105505-76515254418-33345472121-81897295954-04720996364-38807082673-22950883034-17300363911', '66079584515-47283794983-17467084307-71497420039-26776976072'),(244, '05877841538-60635884519-51303346591-96733433762-84809407657-91116922790-92139725904-12474599062-42794260621-17471657481', '13299639207-66444695816-97097794037-70686554100-42246241707'),(94, '63010117103-39065502173-23793591284-44581056772-44672306954-39096576724-63573997799-56288944400-38780780636-29414581709', '78465644577-08783462501-32650273003-79188644854-59002027916'),(442, '48586048669-04798370738-12898251575-92423830144-30041694943-51527021294-70144178325-77120349662-12320565016-16303276800', '82558828619-00971059830-57994957881-34899590342-99119291735'),(361, '07107808206-12097584353-26061471289-18517744066-54573702348-67545414218-81082932557-16735238772-71434133985-38662541570', '62728208461-89454383669-94731985767-87672512388-10125586396'),(91, '42898790962-86843220566-76643955294-05879831056-14993328291-25906366916-16361780391-28250235958-94835445807-72539237762', '08590577784-65851081475-45888781697-73822934900-83119855716'),(609, '50946940236-06906659558-29052149112-93147298039-96639147865-01319131318-74532452116-13752423935-23893275405-87123219094', '50397598559-60043518981-60884080557-67293034924-10207002880'),(876, '30507546841-65373727471-85197340072-62666339895-95415848794-83583906628-10246487352-96367113271-74144445364-10715781593', '62022199650-28215718399-68197782387-21134859456-27541940211'),(682, '62493829769-48306297338-76838275176-48952635092-27445105252-46392210028-81724730510-16718801922-14694794210-82279601371', '88153632793-00257865149-54989222437-74207337397-15891292055'),(497, '48496206097-57414120577-05066945146-89813608485-41728886452-63844939098-71025484343-33517694117-65945059533-24338891787', '26164901337-92987936231-97582075030-71483634794-98442264292'),(799, '17111122823-18919397172-31758935727-68714048437-97555315264-02028214543-65073653136-91787391510-91245383675-85717512953', '45587710770-00097597037-99921800240-50223403497-02840383743'),(223, '58197962618-39883545793-04626280920-86894241389-71804439959-53805217981-97144401613-78386010768-78595886376-84978230685', '18229308609-45943665700-41264506311-58070288465-78837908797'),(778, '84288148980-60500236126-91843433211-46574269415-58326235895-61490119784-31136107874-19873964223-63059865356-00333981557', '77345640891-68925953295-28469538009-35904568599-25638815575'),(135, '02780246796-82548641845-05238637912-11519490146-96501998883-67720981769-79522825400-84076731323-82404375135-21368915000', '89186288630-80778239770-15825051857-59511351567-08673111221'),(794, '33232126456-48284529140-10904575533-28897459454-17441803739-99731743134-77568559311-96950474079-19068615364-76245454269', '94935311896-48146470072-51850159530-61511568743-56417956166'),(952, '03429809439-55422532551-45136548503-48624136460-43110618374-08846072835-18945551083-90692380856-80434338659-39507619514', '95686438337-72726896591-67055039265-61341577207-97504669785'),(331, '44798215622-12947089928-71451886821-46346691920-53343898520-88811473373-18825483035-36542353462-94281903848-57484772745', '01362302325-65792956478-66305326203-16346503002-77805636706'),(519, '97256646774-29992851381-77693138822-80949699313-52979351966-06189055826-15359289686-28688696388-79195806311-61420074430', '33082010830-06294938744-67394755358-14344394556-64939453134'),(540, '10382563601-81554506757-24885977766-12223188338-56895720594-55800231514-16630550126-47138888913-27909951231-65692910810', '46097994005-15474824121-67613974638-85777051445-56969780402'),(462, '28634827902-98663281481-02612768544-55802328730-58425439211-39909748562-87212326126-21047030966-27551963174-11371840637', '36242222528-47284399355-29098622258-74640269069-11776848790'),(188, '92065451024-22233266476-73089100606-91618694862-70459062136-97748177656-32005591643-44704701951-18775999647-16698837802', '64016268290-87389316822-18263237550-37844673533-21506842205'),(556, '61947027459-34360501132-60676149325-64495017627-31535907912-96860857636-31137339865-58526456436-16135823542-92283992601', '05902384908-66350673727-37562876800-10570941676-26439118194'),(200, '51636737747-83928177488-56700945337-18759927289-73210798776-22683255394-20663743678-54971485754-10724776003-43695802315', '26302056129-50716309849-04282921485-93524093683-33627171669'),(934, '29880211022-36812864261-31692615896-66422565478-49778075742-66191062934-09434967385-30603293096-18229149286-47886904758', '43884320589-09549006134-60126122539-72379726163-47974013790'),(576, '51700527229-59432176028-02654118781-60392708790-20989959062-69781764038-74824352037-21477432957-16364780690-34392889005', '94056619782-23642126643-02026578564-02242380519-55128227944'),(670, '50764517799-43205370386-48615482281-03835665916-45868945045-19538634543-73807717186-28872711528-15801331820-46780812236', '61763850595-66289595937-56559392466-38196881365-31689134999'),(87, '29832985346-78042194011-03611434435-89768223499-02883586190-08078362903-57118444300-26340124740-43977951098-30822124401', '61294949525-86507376584-04675602720-23009423289-07405436072'),(723, '29756300630-04840504889-65293203593-07612842082-96835134192-62665258779-50072499397-09783551306-06471867282-27092460378', '53898403337-86785152526-01003377514-09214235876-65303674697'),(361, '32255585932-70469081364-76297911244-82836974809-09947002731-40120574537-27206267359-94127228589-92755489330-23543798271', '21376251572-77152123875-54274662794-46718995147-64923104563'),(39, '83372305309-05672414653-81259441261-21123137531-17608096378-22800423527-86729852555-09759695456-46027743447-55158441290', '69739188199-04509171277-38261964383-42310085622-23723994601'),(182, '61416439647-47835370221-57889455270-96792286546-08248006550-74210376435-33941503704-82772018968-96399888205-52941765758', '57785983218-07843764340-65749563673-18980293987-73998822913'),(943, '23275885456-04658564476-15009359662-52145956169-82852196888-08302715785-76913270798-18405574137-02889827598-79051320149', '06087879781-12548662208-46321174598-50526123301-23974441335'),(365, '68385427845-97592313392-36252127794-05347328211-74306291821-66317495800-53143489707-39030602025-05523140410-47223878203', '17285760171-74921572215-31912383559-84702255519-48623297816'),(894, '76158279727-62817900033-92666437317-30960080445-46577281488-70677478641-29311048793-62070759446-96781810378-35199150890', '64638154266-67696529555-87334836955-11222563524-12007585482'),(875, '44376648769-53673202447-02378986859-81868561163-32027008786-84348070961-52340204511-68230150334-35557866929-80639903283', '27013854888-54767651533-05563985289-76629869819-26968513924'),(384, '81538140787-88119513459-38147505471-81365833877-86277743590-30487127982-25408429076-19032980605-77661404704-03112676538', '47962837346-46990269543-84940082605-63409581499-75678007274'),(534, '21357660898-75405677603-42151578407-62517284936-83680391670-94423842371-79793307311-31937185159-23060926375-52136930438', '17702440415-46598618145-93768007270-31259809037-23365533866'),(857, '79268762063-60030360020-08229768160-96622270879-30153624737-05850213216-58103136573-24238976997-94530774334-67568955062', '64280481788-03080822274-94837586174-24505190247-58940856893'),(63, '20109162003-96770597831-37533839086-13342181430-60018358674-02003154953-26151224364-53135402302-34393690904-42807199353', '29311454595-93796162544-46441304909-53299789802-76525615521'),(467, '81501910133-83360475096-61632681115-59351463317-76242594342-23917550218-61163649119-40131331386-47798843169-38055326453', '34591801788-66060559770-39763465899-92584881246-28174977479'),(519, '45228333217-55320708654-16432253601-14283375041-92297705754-91849431879-51834239272-64026200750-02195655392-94987331173', '61052112726-65251829706-58689558965-15719451770-27803863110'),(874, '77128950873-51250830804-92670262237-20516506949-20500241842-86792471119-50092720044-12133387885-87591263379-41602789528', '79354041407-25621479203-66514203560-84256878723-60621851657'),(152, '00425270723-77309797425-46548622950-98841329136-88056655776-65150290838-70856356348-24843319200-63395953874-41667465963', '71377065977-81479173561-28066885053-07573803021-88250330066'),(274, '79398364508-04922660953-38762620569-08216875070-20819378668-12084564806-80697616769-89831855161-83792541728-69782760957', '36067788315-54014246456-61232703095-44842004744-57842171200'),(603, '28494347276-01505549945-81724981811-64166302760-09643661807-08354952152-49105935830-98957996578-78037428434-97355992811', '08157156488-32547092680-30846579261-57255014088-72417515259'),(501, '17041692977-97064061517-26018348906-69440020093-30863780645-98804395906-83539299273-01725767299-17105232692-07457832001', '42106292205-46969978601-33221425844-35250164790-05251158057'),(881, '58427649582-05890552627-94492135585-84312223485-38620371760-75006794297-26778657210-39135974821-10395892885-00850873970', '70088763170-94398079363-27256251811-07063402322-63217980566'),(172, '36055627153-51389838806-79140955295-82130371495-84820101420-96132210289-01027924942-55125410403-04974841830-77346392676', '43733104951-51666225050-20294435264-36582276036-28128765327'),(327, '56871604970-47284006212-75945144493-39627094719-49152138330-77018603431-15125589949-85255590337-74574219106-47337777319', '23300926044-86880802907-99622356956-64486422498-81030517652'),(871, '43675984995-73103691590-92393033819-68738508321-93003071626-15522631009-33508075171-98202495179-26185534278-50547432289', '23795835829-29312674580-64927205632-10068616819-16415919175'),(227, '98118616291-61208892092-73473238445-56178816139-78337313306-85023528490-55921893328-28989244054-97794976722-81007514861', '36436970802-65690866879-16445675070-60285748269-64033413861'),(853, '91179192832-89603250568-41894741127-52856011098-39625156076-52705315562-22039172020-35880465222-72979392567-37906286634', '40490764079-08000042781-63969956664-00457548135-94949683364'),(721, '07607358239-59180779259-72472521191-42805638862-27959117936-44850963967-95681307740-60094586159-31870958996-70432330235', '89053697649-51185826585-99747180957-11097499209-58454919866'),(333, '09103681576-94248345504-70466346060-06288171819-06517185669-18782286584-18787488393-67206999497-50363827556-64246148671', '46543312736-94902397790-69489972196-33288786914-64942993892'),(642, '69970161063-40647070482-20203505126-83778589125-05612383985-97380483562-13901786242-22318131941-36010631024-14741410620', '92778695913-93731363261-78349094290-68801603980-96097806033'),(84, '08165353118-02024273586-11662025729-14301189534-10315224294-21652239505-34612845651-03766454406-50213646877-99595984827', '51997471378-82747635328-45024574456-89047775257-65270307751'),(311, '82311218678-07389376378-48226968218-32814489135-45859641406-39642601540-63850460676-28662019835-45401055670-91526210290', '31764884227-95469848255-34664570688-43665929087-18866414231'),(166, '17951260097-49862151590-24110394929-12672689573-37077616299-83307739416-76916331443-15691963692-71769716909-03020354522', '52179679122-17505260345-93423268071-91143950210-72337965028'),(772, '04000343724-01387113021-72282824898-24380588168-38457502432-08774808895-94766115190-61231095222-69471668081-35896616882', '89825633640-91741315226-05432224607-37642031512-07683455526'),(870, '96191248304-21576454621-70856686125-75401558606-17090639317-67450694630-67759104365-18551112382-03928600227-05786399258', '28523848988-64899776636-79822395069-56294608647-44804802979');
+SET SESSION binlog_format= DEFAULT;
+--enable_query_log
+
+SELECT COUNT(*) FROM sbtest1;
+
+DROP TABLE sbtest1;
diff --git a/mysql-test/suite/binlog_in_engine/binlog_in_engine_restart.opt b/mysql-test/suite/binlog_in_engine/binlog_in_engine_restart.opt
new file mode 100644
index 0000000000000..fa1aa8c5f34ea
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/binlog_in_engine_restart.opt
@@ -0,0 +1,2 @@
+--innodb-binlog-state-interval=65536
+--binlog-directory=binlogs
diff --git a/mysql-test/suite/binlog_in_engine/binlog_in_engine_restart.result b/mysql-test/suite/binlog_in_engine/binlog_in_engine_restart.result
new file mode 100644
index 0000000000000..e66ef9e3636b0
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/binlog_in_engine_restart.result
@@ -0,0 +1,58 @@
+include/reset_master.inc
+CREATE TABLE t1 (a INT PRIMARY KEY, b INT) ENGINE=InnoDB;
+INSERT INTO t1 VALUES (1, 0);
+INSERT INTO t1 SELECT seq+1000000+100000*0, seq FROM seq_1_to_4000;
+INSERT INTO t1 SELECT seq+1000000+100000*1, seq FROM seq_1_to_4000;
+INSERT INTO t1 SELECT seq+1000000+100000*2, seq FROM seq_1_to_4000;
+INSERT INTO t1 SELECT seq+1000000+100000*3, seq FROM seq_1_to_4000;
+INSERT INTO t1 SELECT seq+1000000+100000*4, seq FROM seq_1_to_4000;
+INSERT INTO t1 SELECT seq+1000000+100000*5, seq FROM seq_1_to_4000;
+INSERT INTO t1 SELECT seq+1000000+100000*6, seq FROM seq_1_to_4000;
+INSERT INTO t1 SELECT seq+1000000+100000*7, seq FROM seq_1_to_4000;
+INSERT INTO t1 SELECT seq+1000000+100000*8, seq FROM seq_1_to_4000;
+INSERT INTO t1 SELECT seq+1000000+100000*9, seq FROM seq_1_to_4000;
+SELECT @@GLOBAL.gtid_binlog_state;
+@@GLOBAL.gtid_binlog_state
+0-1-12
+# restart
+SELECT @@GLOBAL.gtid_binlog_state;
+@@GLOBAL.gtid_binlog_state
+0-1-12
+INSERT INTO t1 VALUES (2, 0);
+INSERT INTO t1 SELECT seq+2000000+100000*0, seq FROM seq_1_to_4000;
+INSERT INTO t1 SELECT seq+2000000+100000*1, seq FROM seq_1_to_4000;
+INSERT INTO t1 SELECT seq+2000000+100000*2, seq FROM seq_1_to_4000;
+INSERT INTO t1 SELECT seq+2000000+100000*3, seq FROM seq_1_to_4000;
+INSERT INTO t1 SELECT seq+2000000+100000*4, seq FROM seq_1_to_4000;
+INSERT INTO t1 SELECT seq+2000000+100000*5, seq FROM seq_1_to_4000;
+INSERT INTO t1 SELECT seq+2000000+100000*6, seq FROM seq_1_to_4000;
+INSERT INTO t1 SELECT seq+2000000+100000*7, seq FROM seq_1_to_4000;
+INSERT INTO t1 SELECT seq+2000000+100000*8, seq FROM seq_1_to_4000;
+INSERT INTO t1 SELECT seq+2000000+100000*9, seq FROM seq_1_to_4000;
+SELECT @@GLOBAL.gtid_binlog_state;
+@@GLOBAL.gtid_binlog_state
+0-1-23
+# restart
+SELECT @@GLOBAL.gtid_binlog_state;
+@@GLOBAL.gtid_binlog_state
+0-1-23
+INSERT INTO t1 VALUES (3, 0);
+INSERT INTO t1 SELECT seq+3000000+100000*0, seq FROM seq_1_to_4000;
+INSERT INTO t1 SELECT seq+3000000+100000*1, seq FROM seq_1_to_4000;
+INSERT INTO t1 SELECT seq+3000000+100000*2, seq FROM seq_1_to_4000;
+INSERT INTO t1 SELECT seq+3000000+100000*3, seq FROM seq_1_to_4000;
+INSERT INTO t1 SELECT seq+3000000+100000*4, seq FROM seq_1_to_4000;
+INSERT INTO t1 SELECT seq+3000000+100000*5, seq FROM seq_1_to_4000;
+INSERT INTO t1 SELECT seq+3000000+100000*6, seq FROM seq_1_to_4000;
+INSERT INTO t1 SELECT seq+3000000+100000*7, seq FROM seq_1_to_4000;
+INSERT INTO t1 SELECT seq+3000000+100000*8, seq FROM seq_1_to_4000;
+INSERT INTO t1 SELECT seq+3000000+100000*9, seq FROM seq_1_to_4000;
+SHOW BINARY LOGS;
+Log_name File_size
+binlog-000000.ibb 262144
+binlog-000001.ibb 262144
+binlog-000002.ibb 262144
+binlog-000003.ibb 262144
+binlog-000004.ibb 262144
+binlog-000005.ibb 262144
+DROP TABLE t1;
diff --git a/mysql-test/suite/binlog_in_engine/binlog_in_engine_restart.test b/mysql-test/suite/binlog_in_engine/binlog_in_engine_restart.test
new file mode 100644
index 0000000000000..d6330c2fb66eb
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/binlog_in_engine_restart.test
@@ -0,0 +1,50 @@
+--source include/big_test.inc
+--source include/have_binlog_format_row.inc
+--source include/have_innodb_binlog.inc
+--source include/have_sequence.inc
+
+# Note: This test also tests the --binlog-directory option by putting it
+# in binlog_in_engine_restart.opt .
+
+--source include/reset_master.inc
+
+CREATE TABLE t1 (a INT PRIMARY KEY, b INT) ENGINE=InnoDB;
+INSERT INTO t1 VALUES (1, 0);
+let $i= 0;
+while ($i < 10) {
+ eval INSERT INTO t1 SELECT seq+1000000+100000*$i, seq FROM seq_1_to_4000;
+ inc $i;
+}
+SELECT @@GLOBAL.gtid_binlog_state;
+--exec $MYSQL_BINLOG --start-position=42-42-42 --gtid-strict-mode=0 --read-from-remote-server --user=root --host=127.0.0.1 --port=$MASTER_MYPORT master-bin.000001 > $MYSQLTEST_VARDIR/tmp/mysqlbinlog1.txt
+
+--source include/restart_mysqld.inc
+
+SELECT @@GLOBAL.gtid_binlog_state;
+--let $gtid_pos2= `SELECT @@GLOBAL.gtid_binlog_pos`
+INSERT INTO t1 VALUES (2, 0);
+let $i= 0;
+while ($i < 10) {
+ eval INSERT INTO t1 SELECT seq+2000000+100000*$i, seq FROM seq_1_to_4000;
+ inc $i;
+}
+SELECT @@GLOBAL.gtid_binlog_state;
+--exec $MYSQL_BINLOG --start-position=$gtid_pos2 --read-from-remote-server --user=root --host=127.0.0.1 --port=$MASTER_MYPORT master-bin.000001 > $MYSQLTEST_VARDIR/tmp/mysqlbinlog2.txt
+
+--source include/restart_mysqld.inc
+
+SELECT @@GLOBAL.gtid_binlog_state;
+--let $gtid_pos3= `SELECT @@GLOBAL.gtid_binlog_pos`
+INSERT INTO t1 VALUES (3, 0);
+let $i= 0;
+while ($i < 10) {
+ eval INSERT INTO t1 SELECT seq+3000000+100000*$i, seq FROM seq_1_to_4000;
+ inc $i;
+}
+
+--let $binlog_name= binlog-000005.ibb
+--let $binlog_size= 262144
+--source include/wait_for_engine_binlog.inc
+SHOW BINARY LOGS;
+
+DROP TABLE t1;
diff --git a/mysql-test/suite/binlog_in_engine/binlog_legacy_pos.result b/mysql-test/suite/binlog_in_engine/binlog_legacy_pos.result
new file mode 100644
index 0000000000000..fbfc2b27c7cc2
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/binlog_legacy_pos.result
@@ -0,0 +1,98 @@
+include/reset_master.inc
+CREATE TABLE t1 (a INT PRIMARY KEY, b INT) ENGINE=InnoDB;
+INSERT INTO t1 VALUES (1, 0);
+connect con1,localhost,root,,;
+START TRANSACTION WITH CONSISTENT SNAPSHOT;
+connect con2,localhost,root,,;
+*** Connection sees current position by default.
+connection default;
+INSERT INTO t1 VALUES (2, 0);
+FLUSH BINARY LOGS;
+INSERT INTO t1 VALUES (3, 0);
+INSERT INTO t1 VALUES (4, 0);
+SELECT * FROM t1 ORDER BY a;
+a b
+1 0
+2 0
+3 0
+4 0
+include/show_binlog_events.inc
+Log_name Pos Event_type Server_id End_log_pos Info
+binlog-000001.ibb # Gtid # # BEGIN GTID #-#-#
+binlog-000001.ibb # Query # # use `test`; INSERT INTO t1 VALUES (4, 0)
+binlog-000001.ibb # Xid # # COMMIT /* XID */
+*** START TRANSACTION WITH CONSISTENT SNAPSHOT sees position consistent with read view.
+connection con1;
+SELECT * FROM t1 ORDER BY a;
+a b
+1 0
+include/show_binlog_events.inc
+Log_name Pos Event_type Server_id End_log_pos Info
+binlog-000000.ibb # Gtid # # BEGIN GTID #-#-#
+binlog-000000.ibb # Query # # use `test`; INSERT INTO t1 VALUES (2, 0)
+binlog-000000.ibb # Xid # # COMMIT /* XID */
+*** Connection with no active transaction sees the current position.
+connection con2;
+connection default;
+INSERT INTO t1 VALUES (5, 0);
+connection con2;
+SELECT * FROM t1 ORDER BY a;
+a b
+1 0
+2 0
+3 0
+4 0
+5 0
+include/show_binlog_events.inc
+Log_name Pos Event_type Server_id End_log_pos Info
+binlog-000001.ibb # Gtid # # BEGIN GTID #-#-#
+binlog-000001.ibb # Query # # use `test`; INSERT INTO t1 VALUES (5, 0)
+binlog-000001.ibb # Xid # # COMMIT /* XID */
+*** Test some various conditions around SHOW BINLOG EVENTS.
+SHOW BINLOG EVENTS IN 'binlog-000003.ibb';
+ERROR HY000: Error when executing command SHOW BINLOG EVENTS: Could not find target log
+SHOW BINLOG EVENTS IN 'binlog-000003.ibb' FROM 1234567890;
+ERROR HY000: Error when executing command SHOW BINLOG EVENTS: Could not find target log
+SHOW BINLOG EVENTS IN 'binlog-000000.ibb' FROM 0 LIMIT 1;
+Log_name Pos Event_type Server_id End_log_pos Info
+binlog-000000.ibb 0 Gtid 1 0 GTID 0-1-1
+SHOW BINLOG EVENTS IN 'binlog-000000.ibb' FROM 4 LIMIT 1;
+Log_name Pos Event_type Server_id End_log_pos Info
+binlog-000000.ibb 0 Gtid 1 0 GTID 0-1-1
+SHOW BINLOG EVENTS IN 'binlog-000000.ibb' FROM 42 LIMIT 1;
+Log_name Pos Event_type Server_id End_log_pos Info
+binlog-000000.ibb 0 Gtid 1 0 GTID 0-1-1
+SHOW BINLOG EVENTS IN 'binlog-000000.ibb' FROM 3000 LIMIT 1;
+Log_name Pos Event_type Server_id End_log_pos Info
+binlog-000000.ibb 0 Gtid 1 0 GTID 0-1-1
+SHOW BINLOG EVENTS IN 'binlog-000000.ibb' FROM 16384 LIMIT 1;
+Log_name Pos Event_type Server_id End_log_pos Info
+binlog-000000.ibb 0 Gtid 1 0 GTID 0-1-1
+SHOW BINLOG EVENTS IN 'binlog-000001.ibb' FROM 16384 LIMIT 1;
+Log_name Pos Event_type Server_id End_log_pos Info
+binlog-000001.ibb 0 Gtid 1 0 BEGIN GTID 0-1-4
+SHOW BINLOG EVENTS IN 'binlog-000001.ibb' FROM 16400 LIMIT 1;
+Log_name Pos Event_type Server_id End_log_pos Info
+binlog-000001.ibb 0 Gtid 1 0 BEGIN GTID 0-1-5
+SHOW BINLOG EVENTS IN 'binlog-000000.ibb' FROM 32768;
+Log_name Pos Event_type Server_id End_log_pos Info
+SHOW BINLOG EVENTS IN 'binlog-000000.ibb' FROM 32769;
+Log_name Pos Event_type Server_id End_log_pos Info
+SHOW BINLOG EVENTS IN 'binlog-000000.ibb' FROM 262144;
+Log_name Pos Event_type Server_id End_log_pos Info
+SHOW BINLOG EVENTS IN 'binlog-000000.ibb' FROM 2000000000;
+Log_name Pos Event_type Server_id End_log_pos Info
+SHOW BINLOG EVENTS IN 'binlog--000012.ibb';
+ERROR HY000: Target log not found in binlog index
+SHOW BINLOG EVENTS IN 'abadcafe.ibb';
+ERROR HY000: Target log not found in binlog index
+SET @old_needed= @@GLOBAL.slave_connections_needed_for_purge;
+SET GLOBAL slave_connections_needed_for_purge= 0;
+PURGE BINARY LOGS TO 'binlog-000001.ibb';
+SET GLOBAL slave_connections_needed_for_purge= @old_needed;
+SHOW BINLOG EVENTS IN 'binlog-000000.ibb' LIMIT 10;
+ERROR HY000: File './binlog-000000.ibb' not found (Errcode: 2 "No such file or directory")
+connection default;
+disconnect con1;
+disconnect con2;
+DROP TABLE t1;
diff --git a/mysql-test/suite/binlog_in_engine/binlog_legacy_pos.test b/mysql-test/suite/binlog_in_engine/binlog_legacy_pos.test
new file mode 100644
index 0000000000000..2ed3a1f2aaf25
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/binlog_legacy_pos.test
@@ -0,0 +1,85 @@
+--source include/have_binlog_format_mixed.inc
+--source include/have_innodb_binlog.inc
+
+--source include/reset_master.inc
+CREATE TABLE t1 (a INT PRIMARY KEY, b INT) ENGINE=InnoDB;
+INSERT INTO t1 VALUES (1, 0);
+
+--connect(con1,localhost,root,,)
+START TRANSACTION WITH CONSISTENT SNAPSHOT;
+
+--connect(con2,localhost,root,,)
+
+--echo *** Connection sees current position by default.
+--connection default
+INSERT INTO t1 VALUES (2, 0);
+FLUSH BINARY LOGS;
+INSERT INTO t1 VALUES (3, 0);
+--let $binlog_file= query_get_value(SHOW STATUS LIKE 'binlog_snapshot_file', Value, 1)
+--let $binlog_start= query_get_value(SHOW STATUS LIKE 'binlog_snapshot_position', Value, 1)
+--let $binlog_limit= 0, 3
+INSERT INTO t1 VALUES (4, 0);
+SELECT * FROM t1 ORDER BY a;
+--source include/show_binlog_events.inc
+
+--echo *** START TRANSACTION WITH CONSISTENT SNAPSHOT sees position consistent with read view.
+--connection con1
+--let $binlog_file= query_get_value(SHOW STATUS LIKE 'binlog_snapshot_file', Value, 1)
+--let $binlog_start= query_get_value(SHOW STATUS LIKE 'binlog_snapshot_position', Value, 1)
+SELECT * FROM t1 ORDER BY a;
+--source include/show_binlog_events.inc
+
+--echo *** Connection with no active transaction sees the current position.
+--connection con2
+--let $binlog_file= query_get_value(SHOW STATUS LIKE 'binlog_snapshot_file', Value, 1)
+--let $binlog_start= query_get_value(SHOW STATUS LIKE 'binlog_snapshot_position', Value, 1)
+--connection default
+INSERT INTO t1 VALUES (5, 0);
+--connection con2
+SELECT * FROM t1 ORDER BY a;
+--source include/show_binlog_events.inc
+
+--echo *** Test some various conditions around SHOW BINLOG EVENTS.
+--let $binlog_name= binlog-000002.ibb
+--let $binlog_size= 262144
+--source include/wait_for_engine_binlog.inc
+
+# Non-existing file.
+--error ER_ERROR_WHEN_EXECUTING_COMMAND
+SHOW BINLOG EVENTS IN 'binlog-000003.ibb';
+--error ER_ERROR_WHEN_EXECUTING_COMMAND
+SHOW BINLOG EVENTS IN 'binlog-000003.ibb' FROM 1234567890;
+
+# In-between offset starts from first following GTID.
+SHOW BINLOG EVENTS IN 'binlog-000000.ibb' FROM 0 LIMIT 1;
+SHOW BINLOG EVENTS IN 'binlog-000000.ibb' FROM 4 LIMIT 1;
+SHOW BINLOG EVENTS IN 'binlog-000000.ibb' FROM 42 LIMIT 1;
+SHOW BINLOG EVENTS IN 'binlog-000000.ibb' FROM 3000 LIMIT 1;
+SHOW BINLOG EVENTS IN 'binlog-000000.ibb' FROM 16384 LIMIT 1;
+SHOW BINLOG EVENTS IN 'binlog-000001.ibb' FROM 16384 LIMIT 1;
+SHOW BINLOG EVENTS IN 'binlog-000001.ibb' FROM 16400 LIMIT 1;
+
+# Offset past end of file shows empty.
+SHOW BINLOG EVENTS IN 'binlog-000000.ibb' FROM 32768;
+SHOW BINLOG EVENTS IN 'binlog-000000.ibb' FROM 32769;
+SHOW BINLOG EVENTS IN 'binlog-000000.ibb' FROM 262144;
+SHOW BINLOG EVENTS IN 'binlog-000000.ibb' FROM 2000000000;
+
+# Non-binlog file name
+--error ER_UNKNOWN_TARGET_BINLOG
+SHOW BINLOG EVENTS IN 'binlog--000012.ibb';
+--error ER_UNKNOWN_TARGET_BINLOG
+SHOW BINLOG EVENTS IN 'abadcafe.ibb';
+
+# Purged binlog file.
+SET @old_needed= @@GLOBAL.slave_connections_needed_for_purge;
+SET GLOBAL slave_connections_needed_for_purge= 0;
+PURGE BINARY LOGS TO 'binlog-000001.ibb';
+SET GLOBAL slave_connections_needed_for_purge= @old_needed;
+--error 29
+SHOW BINLOG EVENTS IN 'binlog-000000.ibb' LIMIT 10;
+
+--connection default
+--disconnect con1
+--disconnect con2
+DROP TABLE t1;
diff --git a/mysql-test/suite/binlog_in_engine/binlog_oob_reset_purge.result b/mysql-test/suite/binlog_in_engine/binlog_oob_reset_purge.result
new file mode 100644
index 0000000000000..cd4e70ef11528
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/binlog_oob_reset_purge.result
@@ -0,0 +1,28 @@
+include/reset_master.inc
+CREATE TABLE t1(a INT PRIMARY KEY, b INT, c LONGBLOB) ENGINE=InnoDB;
+*** Test that an active transaction with OOB data prevents PURGE and RESET MASTER.
+connect con1,localhost,root,,;
+BEGIN;
+INSERT INTO t1 VALUES (1, 1, REPEAT('$', 100000));
+connection default;
+INSERT INTO t1 VALUES (2, 0, 'abd');
+FLUSH BINARY LOGS;
+INSERT INTO t1 VALUES (3, 0, 'def');
+FLUSH BINARY LOGS;
+INSERT INTO t1 VALUES (4, 0, 'ghi');
+FLUSH BINARY LOGS;
+SET @old_min_slaves= @@GLOBAL.slave_connections_needed_for_purge;
+SET GLOBAL slave_connections_needed_for_purge= 0;
+PURGE BINARY LOGS TO 'binlog-000001.ibb';
+ERROR HY000: A purgeable log is in use, will not purge
+RESET MASTER;
+ERROR HY000: Cannot execute RESET MASTER as the binlog is in use by an active transaction
+connection con1;
+COMMIT;
+disconnect con1;
+connection default;
+PURGE BINARY LOGS TO 'binlog-000001.ibb';
+RESET MASTER;
+connection default;
+SET GLOBAL slave_connections_needed_for_purge= @old_min_slaves;
+DROP TABLE t1;
diff --git a/mysql-test/suite/binlog_in_engine/binlog_oob_reset_purge.test b/mysql-test/suite/binlog_in_engine/binlog_oob_reset_purge.test
new file mode 100644
index 0000000000000..453b93b9c753c
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/binlog_oob_reset_purge.test
@@ -0,0 +1,44 @@
+--source include/have_binlog_format_row.inc
+--source include/have_innodb_binlog.inc
+
+--source include/reset_master.inc
+
+CREATE TABLE t1(a INT PRIMARY KEY, b INT, c LONGBLOB) ENGINE=InnoDB;
+
+--echo *** Test that an active transaction with OOB data prevents PURGE and RESET MASTER.
+
+--connect con1,localhost,root,,
+# Create an active transaction large enough to generate OOB data.
+BEGIN;
+INSERT INTO t1 VALUES (1, 1, REPEAT('$', 100000));
+
+--connection default
+INSERT INTO t1 VALUES (2, 0, 'abd');
+FLUSH BINARY LOGS;
+INSERT INTO t1 VALUES (3, 0, 'def');
+FLUSH BINARY LOGS;
+INSERT INTO t1 VALUES (4, 0, 'ghi');
+FLUSH BINARY LOGS;
+
+# PURGE and RESET MASTER are blocked because con1 has pending references to
+# binlog-000000.ibb
+SET @old_min_slaves= @@GLOBAL.slave_connections_needed_for_purge;
+SET GLOBAL slave_connections_needed_for_purge= 0;
+--error ER_LOG_IN_USE
+PURGE BINARY LOGS TO 'binlog-000001.ibb';
+--error ER_BINLOG_IN_USE_TRX
+RESET MASTER;
+
+# Committing the transaction allows the PURGE and RESET to succeed.
+--connection con1
+COMMIT;
+--disconnect con1
+
+--connection default
+PURGE BINARY LOGS TO 'binlog-000001.ibb';
+RESET MASTER;
+
+--connection default
+
+SET GLOBAL slave_connections_needed_for_purge= @old_min_slaves;
+DROP TABLE t1;
diff --git a/mysql-test/suite/binlog_in_engine/binlog_restart.result b/mysql-test/suite/binlog_in_engine/binlog_restart.result
new file mode 100644
index 0000000000000..3e2be45330bbd
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/binlog_restart.result
@@ -0,0 +1,21 @@
+CREATE TABLE t1(a INT PRIMARY KEY, b INT, c LONGBLOB) ENGINE=InnoDB;
+*** Test restarting the server on an existing binlog
+include/reset_master.inc
+SET SESSION binlog_annotate_row_events= 0;
+INSERT INTO t1 VALUES (1, 0, REPEAT('@', 31760));
+INSERT INTO t1 VALUES (2, 0, REPEAT('@', 100));
+RESET MASTER;
+INSERT INTO t1 VALUES (3, 0, REPEAT('&', 31760));
+INSERT INTO t1 VALUES (4, 0, REPEAT('&', $length));
+# restart
+SELECT @@GLOBAL.max_binlog_size;
+@@GLOBAL.max_binlog_size
+262144
+RESET MASTER;
+INSERT INTO t1 VALUES (5, 0, REPEAT('@', 243700));
+INSERT INTO t1 VALUES (6, 0, REPEAT('@', 100));
+RESET MASTER;
+INSERT INTO t1 VALUES (7, 0, REPEAT('&', 243700));
+INSERT INTO t1 VALUES (8, 0, REPEAT('&', $length));
+# restart
+DROP TABLE t1;
diff --git a/mysql-test/suite/binlog_in_engine/binlog_restart.test b/mysql-test/suite/binlog_in_engine/binlog_restart.test
new file mode 100644
index 0000000000000..5ff4ae9337c76
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/binlog_restart.test
@@ -0,0 +1,63 @@
+--source include/have_binlog_format_row.inc
+--source include/have_innodb_binlog.inc
+
+CREATE TABLE t1(a INT PRIMARY KEY, b INT, c LONGBLOB) ENGINE=InnoDB;
+
+--echo *** Test restarting the server on an existing binlog
+
+# Reset the binlogs here to get maximally predictable binlog offsets
+--source include/reset_master.inc
+
+# Try to hit exactly the end of page 2 before restart.
+# There was a bug where restarting the binlog on an exact page boundary
+# would initialize the page fifo incorrectly.
+# Note a page has 16384 - 4 = 16380 bytes of data available due to CRC.
+--let $target= `SELECT 16384*3 - 4`
+--let $initial= `SELECT 2*16380 - 1000`
+--let $length= 100
+
+# First try some value a bit too small and see where we end.
+SET SESSION binlog_annotate_row_events= 0;
+eval INSERT INTO t1 VALUES (1, 0, REPEAT('@', $initial));
+eval INSERT INTO t1 VALUES (2, 0, REPEAT('@', $length));
+--let $boundary=query_get_value(SHOW MASTER STATUS, Position, 1)
+
+# Now calculate the length required to hit exactly the end of the third
+# page (page_no=2). We aim for 1 byte before the end, which will then
+# fill up with one filler byte.
+--let $length= `SELECT $length + ($target - $boundary) - 1`
+
+# Now try again with the calculated length; should make the binlog end
+# exactly at the page boundary.
+RESET MASTER;
+eval INSERT INTO t1 VALUES (3, 0, REPEAT('&', $initial));
+evalp INSERT INTO t1 VALUES (4, 0, REPEAT('&', $length));
+
+# Now restart the server and let it continue at the current point of
+# the binlog. There was a bug when we restarted exactly on a page boundary
+# that would initialize the page fifo incorrectly and assert.
+--source include/restart_mysqld.inc
+
+
+# Now try to fill up exactly one binlog file, and restart with the
+# current position at the end of one file / start of the next one.
+# The test is designed to run with binlog size 256K which is 16 pages.
+SELECT @@GLOBAL.max_binlog_size;
+RESET MASTER;
+--let $target= `SELECT 16384*16 - 4`
+--let $initial= `SELECT 15*16380 - 2000`
+--let $length= 100
+
+eval INSERT INTO t1 VALUES (5, 0, REPEAT('@', $initial));
+eval INSERT INTO t1 VALUES (6, 0, REPEAT('@', $length));
+--let $boundary=query_get_value(SHOW MASTER STATUS, Position, 1)
+
+--let $length= `SELECT $length + ($target - $boundary) - 1`
+
+RESET MASTER;
+eval INSERT INTO t1 VALUES (7, 0, REPEAT('&', $initial));
+evalp INSERT INTO t1 VALUES (8, 0, REPEAT('&', $length));
+
+--source include/restart_mysqld.inc
+
+DROP TABLE t1;
diff --git a/mysql-test/suite/binlog_in_engine/crash_safe_slave-master.opt b/mysql-test/suite/binlog_in_engine/crash_safe_slave-master.opt
new file mode 100644
index 0000000000000..19ee75da5470c
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/crash_safe_slave-master.opt
@@ -0,0 +1 @@
+--innodb-log-file-mmap=OFF
diff --git a/mysql-test/suite/binlog_in_engine/crash_safe_slave.result b/mysql-test/suite/binlog_in_engine/crash_safe_slave.result
new file mode 100644
index 0000000000000..4288b7b248593
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/crash_safe_slave.result
@@ -0,0 +1,25 @@
+include/master-slave.inc
+[connection master]
+*** Test that slave doesn't get ahead of a non-durable master that crashes.
+connection master;
+SET GLOBAL innodb_flush_log_at_trx_commit= 0;
+CREATE TABLE t1 (a INT PRIMARY KEY, b INT, c TEXT) ENGINE=InnoDB;
+SELECT COUNT(*) FROM t1;
+COUNT(*)
+630
+SET SESSION debug_dbug="+d,crash_dispatch_command_before";
+SELECT 1;
+Got one of the listed errors
+connection master1;
+connection server_1;
+connection default;
+connection master;
+include/rpl_reconnect.inc
+SET STATEMENT gtid_domain_id= 1 FOR INSERT INTO t1 VALUES (9, 9, 'extra');
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+# Asserted this: Row count should match on master and slave (no extra rows on slave)
+connection master;
+DROP TABLE t1;
+include/rpl_end.inc
diff --git a/mysql-test/suite/binlog_in_engine/crash_safe_slave.test b/mysql-test/suite/binlog_in_engine/crash_safe_slave.test
new file mode 100644
index 0000000000000..ad768bbfb67b0
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/crash_safe_slave.test
@@ -0,0 +1,90 @@
+--source include/not_embedded.inc
+--source include/not_valgrind.inc
+--source include/have_debug.inc
+--source include/have_binlog_format_row.inc
+--source include/master-slave.inc
+--source include/have_innodb_binlog.inc
+
+-- echo *** Test that slave doesn't get ahead of a non-durable master that crashes.
+--connection master
+--let $old_flatc= `SELECT @@GLOBAL.innodb_flush_log_at_trx_commit`
+SET GLOBAL innodb_flush_log_at_trx_commit= 0;
+
+CREATE TABLE t1 (a INT PRIMARY KEY, b INT, c TEXT) ENGINE=InnoDB;
+
+# Create a bunch of transactions, when --innodb-flush-log-at-trx-commit=0
+# we should lose some of them in master crash.
+# Note though that this is ineffective when running in /dev/shm/ (./mtr --mem).
+# Because InnoDB is hard-coded to simulate PMEM in this case and forces
+# mmap on the log file even though we have --innodb-log-file-mmap=OFF in our
+# .opt file. Then the memory mapping gets updated immediately when the mtr
+# commits, and kill -9 cannot lose any transactions. The test will still pass,
+# but no transactions are lost on the master so nothing much is tested.
+
+--disable_query_log
+--let loop= 0
+while ($loop < 10) {
+ eval INSERT INTO t1 VALUES ($loop*1000, 0, '');
+ BEGIN;
+ --let $i= 0
+ while ($i < 20) {
+ eval INSERT INTO t1 VALUES ($loop*1000 + $i*10 + 1, 1, REPEAT('a', 1000));
+ eval INSERT INTO t1 VALUES ($loop*1000 + $i*10 + 2, 2, REPEAT('z', 1000));
+ eval INSERT INTO t1 VALUES ($loop*1000 + $i*10 + 3, 3, REPEAT('#', 1000));
+ inc $i;
+ }
+ COMMIT;
+# CREATE TABLE t2 (a INT PRIMARY KEY, b INT) ENGINE=InnoDB;
+ eval INSERT INTO t1 VALUES ($loop*1000 + 5, 5, 'zyxxy');
+# INSERT INTO t2 VALUES (1, 0);
+ eval INSERT INTO t1 VALUES ($loop*1000 + 6, 6, 'the quick brown fox');
+# DROP TABLE t2;
+ inc $loop;
+}
+--enable_query_log
+
+# Crash the master
+--write_file $MYSQLTEST_VARDIR/tmp/mysqld.1.expect
+wait-recovery.test
+EOF
+
+SELECT COUNT(*) FROM t1;
+SET SESSION debug_dbug="+d,crash_dispatch_command_before";
+--error 2006,2013
+SELECT 1;
+--source include/wait_until_disconnected.inc
+--connection master1
+--source include/wait_until_disconnected.inc
+--connection server_1
+--source include/wait_until_disconnected.inc
+--connection default
+--source include/wait_until_disconnected.inc
+
+--connection master
+--append_file $MYSQLTEST_VARDIR/tmp/mysqld.1.expect
+restart
+EOF
+--let $rpl_server_number= 1
+--source include/rpl_reconnect.inc
+
+# After the crash, transactions can be lost and the table row count
+# may be smaller than before the crash.
+--let $master_count= `SELECT COUNT(*) FROM t1`
+SET STATEMENT gtid_domain_id= 1 FOR INSERT INTO t1 VALUES (9, 9, 'extra');
+--source include/save_master_gtid.inc
+
+--connection slave
+--source include/sync_with_master_gtid.inc
+# The slave should have the same amount of rows as the master (ie. not have
+# any extra transactions that were lost on the master by the crash).
+--let $slave_count= `SELECT COUNT(*) FROM t1`
+--let $assert_text= Row count should match on master and slave (no extra rows on slave)
+--let $assert_cond= $master_count + 1 = $slave_count
+--source include/rpl_assert.inc
+
+--connection master
+--disable_query_log
+eval SET GLOBAL innodb_flush_log_at_trx_commit= $old_flatc;
+--enable_query_log
+DROP TABLE t1;
+--source include/rpl_end.inc
diff --git a/mysql-test/suite/binlog_in_engine/crash_safe_slave2-master.opt b/mysql-test/suite/binlog_in_engine/crash_safe_slave2-master.opt
new file mode 100644
index 0000000000000..1d10f302791fb
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/crash_safe_slave2-master.opt
@@ -0,0 +1 @@
+--loose-innodb-binlog-state-interval=65536 --loose-max-binlog-size=524288
diff --git a/mysql-test/suite/binlog_in_engine/crash_safe_slave2.result b/mysql-test/suite/binlog_in_engine/crash_safe_slave2.result
new file mode 100644
index 0000000000000..416e6ab07d498
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/crash_safe_slave2.result
@@ -0,0 +1,87 @@
+include/master-slave.inc
+[connection master]
+*** Test that slave is not allowed to find a GTID starting position that is ahead of where the binlog is durable.
+SELECT @@GLOBAL.max_binlog_size;
+@@GLOBAL.max_binlog_size
+524288
+SELECT @@GLOBAL.innodb_binlog_state_interval;
+@@GLOBAL.innodb_binlog_state_interval
+65536
+CREATE TABLE t1 (a LONGBLOB) ENGINE=InnoDB;
+INSERT INTO t1 VALUES ('initial');
+INSERT INTO t1 VALUES (REPEAT(CHR(97 + 0), 8192));
+INSERT INTO t1 VALUES (REPEAT(CHR(97 + 1), 8192));
+INSERT INTO t1 VALUES (REPEAT(CHR(97 + 2), 8192));
+INSERT INTO t1 VALUES (REPEAT(CHR(97 + 3), 8192));
+INSERT INTO t1 VALUES (REPEAT(CHR(97 + 4), 8192));
+INSERT INTO t1 VALUES (REPEAT(CHR(97 + 5), 8192));
+INSERT INTO t1 VALUES (REPEAT(CHR(97 + 6), 8192));
+INSERT INTO t1 VALUES (REPEAT(CHR(97 + 7), 8192));
+INSERT INTO t1 VALUES (REPEAT(CHR(97 + 8), 8192));
+INSERT INTO t1 VALUES (REPEAT(CHR(97 + 9), 8192));
+INSERT INTO t1 VALUES (REPEAT(CHR(97 + 10), 8192));
+INSERT INTO t1 VALUES (REPEAT(CHR(97 + 11), 8192));
+INSERT INTO t1 VALUES (REPEAT(CHR(97 + 12), 8192));
+INSERT INTO t1 VALUES (REPEAT(CHR(97 + 13), 8192));
+INSERT INTO t1 VALUES (REPEAT(CHR(97 + 14), 8192));
+INSERT INTO t1 VALUES (REPEAT(CHR(97 + 15), 8192));
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+include/stop_slave.inc
+connection master;
+SET @old_dbug= @@GLOBAL.debug_dbug;
+SET GLOBAL debug_dbug= '+d,block_binlog_durable';
+INSERT INTO t1 VALUES (REPEAT(CHR(65 + 0), 8192));
+INSERT INTO t1 VALUES (REPEAT(CHR(65 + 1), 8192));
+INSERT INTO t1 VALUES (REPEAT(CHR(65 + 2), 8192));
+INSERT INTO t1 VALUES (REPEAT(CHR(65 + 3), 8192));
+INSERT INTO t1 VALUES (REPEAT(CHR(65 + 4), 8192));
+INSERT INTO t1 VALUES (REPEAT(CHR(65 + 5), 8192));
+INSERT INTO t1 VALUES (REPEAT(CHR(65 + 6), 8192));
+INSERT INTO t1 VALUES (REPEAT(CHR(65 + 7), 8192));
+INSERT INTO t1 VALUES (REPEAT(CHR(65 + 8), 8192));
+INSERT INTO t1 VALUES (REPEAT(CHR(65 + 9), 8192));
+INSERT INTO t1 VALUES (REPEAT(CHR(65 + 10), 8192));
+INSERT INTO t1 VALUES (REPEAT(CHR(65 + 11), 8192));
+INSERT INTO t1 VALUES (REPEAT(CHR(65 + 12), 8192));
+INSERT INTO t1 VALUES (REPEAT(CHR(65 + 13), 8192));
+INSERT INTO t1 VALUES (REPEAT(CHR(65 + 14), 8192));
+INSERT INTO t1 VALUES (REPEAT(CHR(65 + 15), 8192));
+INSERT INTO t1 VALUES (REPEAT(CHR(65 + 16), 8192));
+INSERT INTO t1 VALUES (REPEAT(CHR(65 + 17), 8192));
+INSERT INTO t1 VALUES (REPEAT(CHR(65 + 18), 8192));
+INSERT INTO t1 VALUES (REPEAT(CHR(65 + 19), 8192));
+INSERT INTO t1 VALUES (REPEAT(CHR(65 + 20), 8192));
+INSERT INTO t1 VALUES (REPEAT(CHR(65 + 21), 8192));
+INSERT INTO t1 VALUES (REPEAT(CHR(65 + 22), 8192));
+INSERT INTO t1 VALUES (REPEAT(CHR(65 + 23), 8192));
+INSERT INTO t1 VALUES (REPEAT(CHR(65 + 24), 8192));
+INSERT INTO t1 VALUES (REPEAT(CHR(65 + 25), 8192));
+INSERT INTO t1 VALUES (REPEAT(CHR(65 + 26), 8192));
+INSERT INTO t1 VALUES (REPEAT(CHR(65 + 27), 8192));
+INSERT INTO t1 VALUES (REPEAT(CHR(65 + 28), 8192));
+INSERT INTO t1 VALUES (REPEAT(CHR(65 + 29), 8192));
+INSERT INTO t1 VALUES (REPEAT(CHR(65 + 30), 8192));
+INSERT INTO t1 VALUES (REPEAT(CHR(65 + 31), 8192));
+INSERT INTO t1 VALUES ('middle');
+include/save_master_gtid.inc
+connection slave;
+SET GLOBAL gtid_slave_pos= '0-1-50';
+START SLAVE;
+SELECT '0-1-50' as START_POS, @@GLOBAL.gtid_slave_pos as CURRENT_POS;
+START_POS CURRENT_POS
+0-1-50 0-1-50
+connection master;
+SET GLOBAL debug_dbug= @old_dbug;
+INSERT INTO t1 VALUES (REPEAT(CHR(48 + 0), 8192));
+INSERT INTO t1 VALUES (REPEAT(CHR(48 + 1), 8192));
+INSERT INTO t1 VALUES (REPEAT(CHR(48 + 2), 8192));
+INSERT INTO t1 VALUES (REPEAT(CHR(48 + 3), 8192));
+INSERT INTO t1 VALUES ('final');
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection master;
+DROP TABLE t1;
+include/rpl_end.inc
diff --git a/mysql-test/suite/binlog_in_engine/crash_safe_slave2.test b/mysql-test/suite/binlog_in_engine/crash_safe_slave2.test
new file mode 100644
index 0000000000000..39f21d18281a4
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/crash_safe_slave2.test
@@ -0,0 +1,88 @@
+--source include/have_debug.inc
+--source include/have_binlog_format_row.inc
+--source include/master-slave.inc
+--source include/have_innodb_binlog.inc
+
+--echo *** Test that slave is not allowed to find a GTID starting position that is ahead of where the binlog is durable.
+
+# The bug was a somewhat tricky race. The connecting slave will start at
+# the most recent GTID state record it can find in the binlog that is before
+# it's starting position. The code did not properly check that this state
+# record had become durably redo logged, so the dump thread could end up
+# being ahead of the allowed durable position, and assert due to this.
+#
+# The GTID state records are written every --innodb-binlog-state-interval
+# bytes, so the .opt of the test sets up some know values so the testcase is
+# independent of changes to these in the testsuite framework:
+#
+# innodb_binlog_state_interval= 16384 * 4
+# max_binlog_size= 16384 * 4 * 8
+SELECT @@GLOBAL.max_binlog_size;
+SELECT @@GLOBAL.innodb_binlog_state_interval;
+
+CREATE TABLE t1 (a LONGBLOB) ENGINE=InnoDB;
+INSERT INTO t1 VALUES ('initial');
+
+# First fill in approximately 1/4 of the binlog file.
+--let $i= 0
+while ($i < 16) {
+ eval INSERT INTO t1 VALUES (REPEAT(CHR(97 + $i), 8192));
+ inc $i;
+}
+--source include/save_master_gtid.inc
+
+--connection slave
+--source include/sync_with_master_gtid.inc
+--source include/stop_slave.inc
+
+--connection master
+# Temporarily block binlogged data from being marked durable, thus
+# blocking the slave from receiving them.
+SET @old_dbug= @@GLOBAL.debug_dbug;
+SET GLOBAL debug_dbug= '+d,block_binlog_durable';
+
+# Then fill in up to approximately 3/4 of the binlog, blocking it from
+# becoming marked as durable.
+
+--let $i= 0
+while ($i < 32) {
+ eval INSERT INTO t1 VALUES (REPEAT(CHR(65 + $i), 8192));
+ inc $i;
+}
+
+--let $gtid= `SELECT @@gtid_binlog_pos`
+INSERT INTO t1 VALUES ('middle');
+--source include/save_master_gtid.inc
+
+# Then connect the slave starting at a GTID in the non-durable part of the binlog file.
+--connection slave
+eval SET GLOBAL gtid_slave_pos= '$gtid';
+START SLAVE;
+
+# Here, the slave will be blocked from replicating.
+# Give it a small amount of time to hit the potential race.
+--sleep 0.5
+
+# Check that the GTID pos could not move yet.
+eval SELECT '$gtid' as START_POS, @@GLOBAL.gtid_slave_pos as CURRENT_POS;
+
+--connection master
+# Now release the durability block, allowing the slave to continue.
+SET GLOBAL debug_dbug= @old_dbug;
+
+--let $i= 0
+while ($i < 4) {
+ eval INSERT INTO t1 VALUES (REPEAT(CHR(48 + $i), 8192));
+ inc $i;
+}
+INSERT INTO t1 VALUES ('final');
+--source include/save_master_gtid.inc
+
+--connection slave
+--source include/sync_with_master_gtid.inc
+
+--connection master
+
+DROP TABLE t1;
+
+--source include/rpl_end.inc
diff --git a/mysql-test/suite/binlog_in_engine/disk_full.result b/mysql-test/suite/binlog_in_engine/disk_full.result
new file mode 100644
index 0000000000000..a2f1aa26f72d7
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/disk_full.result
@@ -0,0 +1,28 @@
+*** Test handling of disk full, waiting for space to be freed.
+CREATE TABLE t1 (a INT NOT NULL, b INT NOT NULL, c LONGBLOB, PRIMARY KEY(a, b)) ENGINE=InnoDB;
+INSERT INTO t1 VALUES (0, 0, 'Start');
+INSERT INTO t1 VALUES (1, 0, REPEAT('a', 1000000));
+connection default;
+SET @old_dbug= @@GLOBAL.debug_dbug;
+SET GLOBAL debug_dbug= '+d,ib_alloc_file_disk_full';
+connect con1,localhost,root,,;
+SET SESSION debug_dbug= '+d,dummy_keyword_just_to_not_enable_dbug_output';
+SET SESSION debug_dbug= '-d,ib_alloc_file_disk_full';
+INSERT INTO t1 VALUES (2, 0, REPEAT('b', 1000000));
+connection default;
+SELECT a, b, LENGTH(c) FROM t1 ORDER BY a, b;
+a b LENGTH(c)
+0 0 5
+1 0 1000000
+SET GLOBAL debug_dbug= @old_dbug;
+connection con1;
+connection default;
+disconnect con1;
+SELECT a, b, LENGTH(c) FROM t1 ORDER BY a, b;
+a b LENGTH(c)
+0 0 5
+1 0 1000000
+2 0 1000000
+include/assert_grep.inc [Check that binlog pre-allocation is retried on ENOSPC]
+DROP TABLE t1;
+CALL mtr.add_suppression('InnoDB: Unable to allocate file.* "No space left on device"');
diff --git a/mysql-test/suite/binlog_in_engine/disk_full.test b/mysql-test/suite/binlog_in_engine/disk_full.test
new file mode 100644
index 0000000000000..5347418bd87e2
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/disk_full.test
@@ -0,0 +1,47 @@
+--source include/not_windows.inc
+--source include/have_debug.inc
+--source include/have_binlog_format_row.inc
+--source include/have_innodb_binlog.inc
+
+--echo *** Test handling of disk full, waiting for space to be freed.
+CREATE TABLE t1 (a INT NOT NULL, b INT NOT NULL, c LONGBLOB, PRIMARY KEY(a, b)) ENGINE=InnoDB;
+INSERT INTO t1 VALUES (0, 0, 'Start');
+
+# A normal operation that spans couple binlog files without failing the
+# pre-alloocation of the tablespace.
+INSERT INTO t1 VALUES (1, 0, REPEAT('a', 1000000));
+
+# Now simulate a disk-full error.
+--connection default
+SET @old_dbug= @@GLOBAL.debug_dbug;
+SET GLOBAL debug_dbug= '+d,ib_alloc_file_disk_full';
+
+--connect con1,localhost,root,,
+SET SESSION debug_dbug= '+d,dummy_keyword_just_to_not_enable_dbug_output';
+SET SESSION debug_dbug= '-d,ib_alloc_file_disk_full';
+send INSERT INTO t1 VALUES (2, 0, REPEAT('b', 1000000));
+
+--connection default
+# Test that the insert gets delayed.
+--sleep 1
+SELECT a, b, LENGTH(c) FROM t1 ORDER BY a, b;
+
+# Wait a bit more, and then simulate providing space.
+# The DBUG injection reduces the wait time to 2 seconds.
+--sleep 7
+SET GLOBAL debug_dbug= @old_dbug;
+
+--connection con1
+reap;
+--connection default
+--disconnect con1
+SELECT a, b, LENGTH(c) FROM t1 ORDER BY a, b;
+
+--let $assert_text= Check that binlog pre-allocation is retried on ENOSPC
+--let $assert_file= $MYSQLTEST_VARDIR/log/mysqld.1.err
+--let $assert_select= InnoDB: Unable to allocate file
+--let $assert_match= $assert_select
+--source include/assert_grep.inc
+
+DROP TABLE t1;
+CALL mtr.add_suppression('InnoDB: Unable to allocate file.* "No space left on device"');
diff --git a/mysql-test/suite/binlog_in_engine/gtid_state.result b/mysql-test/suite/binlog_in_engine/gtid_state.result
new file mode 100644
index 0000000000000..f45378f41fe58
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/gtid_state.result
@@ -0,0 +1,42 @@
+include/master-slave.inc
+[connection master]
+*** Test large gtid_state record, > 1 page.
+connection master;
+CREATE TABLE t1 (a INT PRIMARY KEY, b INT) ENGINE=InnoDB;
+INSERT INTO t1 VALUES (0, 0);
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+include/stop_slave.inc
+connection master;
+include/save_master_gtid.inc
+SELECT COUNT(*) AS midway_count FROM t1;
+midway_count
+4001
+INSERT INTO t1 VALUES (-1, 2);
+FLUSH BINARY LOGS;
+INSERT INTO t1 VALUES (-2, 3);
+SELECT COUNT(*) AS full_count FROM t1;
+full_count
+8003
+include/rpl_restart_server.inc [server_number=1]
+OK, binlog state preserved across server restart.
+connection slave;
+Warnings:
+Note 1278 It is recommended to use --skip-slave-start when doing step-by-step replication with START SLAVE UNTIL; otherwise, you will get problems if you get an unexpected slave's mariadbd restart
+include/sync_with_master_gtid.inc
+SELECT COUNT(*) FROM t1;
+COUNT(*)
+4001
+include/wait_for_slave_to_stop.inc
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/start_slave.inc
+include/sync_with_master_gtid.inc
+SELECT COUNT(*) FROM t1;
+COUNT(*)
+8003
+connection master;
+DROP TABLE t1;
+include/rpl_end.inc
diff --git a/mysql-test/suite/binlog_in_engine/gtid_state.test b/mysql-test/suite/binlog_in_engine/gtid_state.test
new file mode 100644
index 0000000000000..219619d6d4ec1
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/gtid_state.test
@@ -0,0 +1,83 @@
+--source include/have_binlog_format_row.inc
+--source include/master-slave.inc
+--source include/have_innodb_binlog.inc
+
+--echo *** Test large gtid_state record, > 1 page.
+
+--connection master
+CREATE TABLE t1 (a INT PRIMARY KEY, b INT) ENGINE=InnoDB;
+INSERT INTO t1 VALUES (0, 0);
+--source include/save_master_gtid.inc
+
+--connection slave
+--source include/sync_with_master_gtid.inc
+--source include/stop_slave.inc
+
+--connection master
+# Generate a large amount of domains, more than will fit in one page.
+# We use large domain_id/server_id so they will not compress to few bytes.
+--let $NUM= 8000
+--let $HALFWAY= `SELECT FLOOR($NUM/2)`
+--let $i = 1
+--disable_query_log
+while ($i <= $NUM) {
+ eval SET SESSION server_id= 4294967295 - $i;
+ eval SET SESSION gtid_domain_id= 4294967295 - $i;
+ eval INSERT INTO t1 VALUES ($i, 1);
+ if ($i == $HALFWAY) {
+ --let $gtid_mid= `SELECT @@gtid_binlog_pos`
+ --source include/save_master_gtid.inc
+ --enable_query_log
+ SELECT COUNT(*) AS midway_count FROM t1;
+ --disable_query_log
+ }
+ inc $i;
+}
+SET SESSION server_id= 1;
+SET SESSION gtid_domain_id= 0;
+--enable_query_log
+INSERT INTO t1 VALUES (-1, 2);
+
+FLUSH BINARY LOGS;
+INSERT INTO t1 VALUES (-2, 3);
+SELECT COUNT(*) AS full_count FROM t1;
+
+# Test that we can recover a large GTID state after server restart.
+--let $before_state= `SELECT @@gtid_binlog_state`
+--let $rpl_server_number= 1
+--source include/rpl_restart_server.inc
+--let $after_state= `SELECT @@gtid_binlog_state`
+if ($before_state == $after_state) {
+ --echo OK, binlog state preserved across server restart.
+}
+if ($before_state != $after_state) {
+ --echo NOT ok, binlog state NOT preserved across server restart.
+ --echo Before state:
+ --echo $before_state
+ --echo After state:
+ --echo $after_state
+ --die Test failed due to GTID state not preserved across server restart
+}
+
+--connection slave
+--disable_query_log
+eval START SLAVE UNTIL Master_gtid_pos='$gtid_mid';
+--enable_query_log
+--source include/sync_with_master_gtid.inc
+SELECT COUNT(*) FROM t1;
+--source include/wait_for_slave_to_stop.inc
+
+# Test that the dump thread can find the GTID position to start even when
+# the GTID state record spans multiple pages.
+--connection master
+--source include/save_master_gtid.inc
+--connection slave
+--source include/start_slave.inc
+--source include/sync_with_master_gtid.inc
+SELECT COUNT(*) FROM t1;
+
+
+# Cleanup.
+--connection master
+DROP TABLE t1;
+--source include/rpl_end.inc
diff --git a/mysql-test/suite/binlog_in_engine/include/mariabackup_slave_provision.inc b/mysql-test/suite/binlog_in_engine/include/mariabackup_slave_provision.inc
new file mode 100644
index 0000000000000..be67c55f7567a
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/include/mariabackup_slave_provision.inc
@@ -0,0 +1,118 @@
+--source include/have_innodb_binlog.inc
+# Test does a lot of queries that take a lot of CPU under Valgrind.
+--source include/not_valgrind.inc
+
+--source include/reset_master.inc
+
+let $basedir=$MYSQLTEST_VARDIR/tmp/backup;
+
+CREATE TABLE t1(a varchar(60) PRIMARY KEY, b VARCHAR(60)) ENGINE INNODB;
+INSERT INTO t1 VALUES(1, NULL);
+CREATE TABLE t2 (val INT) ENGINE=InnoDB;
+INSERT INTO t2 VALUES (0);
+
+--disable_query_log
+--delimiter //
+CREATE PROCEDURE gen_load()
+ MODIFIES SQL DATA
+ BEGIN
+ DECLARE i INT;
+ DECLARE flag TYPE OF t2.val;
+ SET i = 0;
+ load_loop: LOOP
+ SELECT val INTO flag FROM t2;
+ IF NOT (flag=0) THEN
+ LEAVE load_loop;
+ END IF;
+ START TRANSACTION;
+ INSERT INTO t1 VALUES (CONCAT("AbAdCaFe", LPAD(i, 6, "0")), @@SESSION.last_gtid);
+ COMMIT;
+ SET i = i + 1;
+ END LOOP;
+ END
+//
+--delimiter ;
+--enable_query_log
+
+connect (con1,localhost,root,,);
+--echo *** Start a background load...
+send CALL gen_load();
+
+--connection default
+--echo *** Doing backup...
+--exec $XTRABACKUP --defaults-file=$MYSQLTEST_VARDIR/my.cnf --backup --target-dir=$basedir $backup_args
+--echo *** Doing prepare...
+--exec $XTRABACKUP --defaults-file=$MYSQLTEST_VARDIR/my.cnf --prepare --target-dir=$basedir $prepare_args
+
+--echo *** Stop the background load...
+UPDATE t2 SET val=1;
+--connection con1
+reap;
+--connection default
+disconnect con1;
+--let $count_master= `SELECT COUNT(*) FROM t1`
+
+--echo *** Provision a new slave from the backup
+--connect (server2,127.0.0.1,root,,,$SERVER_MYPORT_2)
+--let $datadir_2= `SELECT @@datadir`
+
+--echo *** Stopping provisioned server
+--source include/shutdown_mysqld.inc
+
+--echo *** Removing old datadir for provisioned server
+--rmdir $datadir_2
+
+--echo *** Provision new server from backup
+--exec $XTRABACKUP --copy-back --datadir=$datadir_2 --target-dir=$basedir $copy_back_args
+
+--let $restart_parameters= --skip-slave-start
+--source include/start_mysqld.inc
+
+if (!$nobinlog) {
+ --let $gtid_pos= `SELECT @@GLOBAL.gtid_binlog_pos`
+}
+
+if ($nobinlog) {
+ CREATE TABLE t3 (file VARCHAR(255), pos INT, gtid VARCHAR(255)) ENGINE=InnoDB;
+ --replace_result $basedir BASEDIR
+ --disable_warnings
+ eval LOAD DATA LOCAL INFILE "$basedir/mariadb_backup_binlog_info"
+ INTO TABLE t3 FIELDS ESCAPED BY '' (file, pos, gtid);
+ --enable_warnings
+ --let $gtid_pos= `SELECT gtid FROM t3`
+ DROP TABLE t3;
+}
+
+--replace_result $gtid_pos GTID_POS
+eval SET GLOBAL gtid_slave_pos= '$gtid_pos';
+--replace_result $SERVER_MYPORT_1 SERVER_MYPORT_1
+eval CHANGE MASTER TO
+ master_ssl_verify_server_cert=0,
+ master_port=$SERVER_MYPORT_1, master_host='127.0.0.1', master_user='root',
+ master_use_gtid= Slave_pos;
+START SLAVE;
+
+--connection default
+--source include/save_master_gtid.inc
+
+--connection server2
+--source include/sync_with_master_gtid.inc
+--let $count_slave= `SELECT COUNT(*) FROM t1`
+if ($count_master != $count_slave) {
+ --echo *** ERROR: Table on master has $count_master rows, but table on provisioned slave has $count_slave rows
+ --die Row difference on provisioned slave.
+}
+
+# Cleanup
+
+--connection server2
+STOP SLAVE;
+RESET SLAVE ALL;
+DROP PROCEDURE gen_load;
+DROP TABLE t1, t2;
+
+--connection default
+DROP PROCEDURE gen_load;
+DROP TABLE t1, t2;
+
+rmdir $basedir;
diff --git a/mysql-test/suite/binlog_in_engine/include/xa_crash.inc b/mysql-test/suite/binlog_in_engine/include/xa_crash.inc
new file mode 100644
index 0000000000000..7629bdd367a4f
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/include/xa_crash.inc
@@ -0,0 +1,143 @@
+--source include/reset_master.inc
+
+--echo *** Test XA PREPARE, COMMIT, ROLLBACK with crashes at various points.
+eval CREATE TABLE t1(a INT NOT NULL, b INT NOT NULL, c LONGBLOB, PRIMARY KEY(a, b)) ENGINE=$engine;
+
+INSERT INTO t1 VALUES (0, 1, '');
+INSERT INTO t1 VALUES (0, 2, '');
+INSERT INTO t1 VALUES (0, 3, '');
+
+--echo *** x1: prepared in binlog but not in engine/tables.
+--connect con1,localhost,root,,
+XA BEGIN 'x1';
+INSERT INTO t1 VALUES (1, 1, '');
+INSERT INTO t1 VALUES (1, 2, REPEAT('', 10000));
+
+--echo *** x2: prepared in binlog and engine.
+--connect con2,localhost,root,,
+XA BEGIN 'x2';
+INSERT INTO t1 VALUES (2, 1, REPEAT('', 10000));
+
+--echo *** x3: committed in binlog, not in engine.
+--connect con3,localhost,root,,
+XA BEGIN 'x3';
+INSERT INTO t1 VALUES (3, 1, '');
+
+--connection default
+FLUSH BINARY LOGS;
+
+--echo *** x4: committed in binlog and in engine.
+--connect con4,localhost,root,,
+XA BEGIN 'x4';
+INSERT INTO t1 VALUES (4, 1, REPEAT('', 10000));
+
+--echo *** x5: rolled back in binlog, not in engine.
+--connect con5,localhost,root,,
+XA BEGIN 'x5';
+INSERT INTO t1 VALUES (5, 1, '');
+
+--echo *** x6: rolled back in binlog and in engine.
+--connect con6,localhost,root,,
+XA BEGIN 'x6';
+INSERT INTO t1 VALUES (6, 1, REPEAT('', 10000));
+
+--connection default
+FLUSH BINARY LOGS;
+
+--connection con3
+INSERT INTO t1 VALUES (3, 2, '');
+XA END 'x3';
+XA PREPARE 'x3';
+SET debug_sync= 'ibb_after_commit_redo_log SIGNAL con3_rdy WAIT_FOR crash';
+send XA COMMIT 'x3';
+
+--connection default
+SET debug_sync= 'now WAIT_FOR con3_rdy';
+
+--connection con1
+INSERT INTO t1 VALUES (1, 3, '');
+XA END 'x1';
+SET debug_sync= 'ibb_after_prepare_redo_log SIGNAL con1_rdy WAIT_FOR crash';
+send XA PREPARE 'x1';
+
+--connection default
+SET debug_sync= 'now WAIT_FOR con1_rdy';
+
+--connection con6
+INSERT INTO t1 VALUES (6, 2, '');
+XA END 'x6';
+XA PREPARE 'x6';
+SET debug_sync= 'rollback_handlerton_after SIGNAL con6_rdy WAIT_FOR crash';
+send XA ROLLBACK 'x6';
+
+--connection default
+SET debug_sync= 'now WAIT_FOR con6_rdy';
+
+--connection con5
+INSERT INTO t1 VALUES (5, 2, '');
+XA END 'x5';
+XA PREPARE 'x5';
+SET debug_sync= 'ibb_after_rollback_redo_log SIGNAL con5_rdy WAIT_FOR crash';
+send XA ROLLBACK 'x5';
+
+--connection default
+SET debug_sync= 'now WAIT_FOR con5_rdy';
+
+FLUSH BINARY LOGS;
+
+--connection con2
+INSERT INTO t1 VALUES (2, 2, '');
+INSERT INTO t1 VALUES (2, 3, REPEAT('', 10001));
+XA END 'x2';
+SET debug_sync= 'at_unlog_xa_prepare SIGNAL con2_rdy WAIT_FOR crash';
+send XA PREPARE 'x2';
+
+--connection default
+SET debug_sync= 'now WAIT_FOR con2_rdy';
+
+--connection con4
+INSERT INTO t1 VALUES (4, 2, '');
+XA END 'x4';
+XA PREPARE 'x4';
+SET debug_sync= 'commit_handlerton_after SIGNAL con4_rdy WAIT_FOR crash';
+send XA COMMIT 'x4';
+
+--connection default
+SET debug_sync= 'now WAIT_FOR con4_rdy';
+
+--let $shutdown_timeout=0
+--source include/restart_mysqld.inc
+
+--disconnect con1
+--disconnect con2
+--disconnect con3
+--disconnect con4
+--disconnect con5
+--disconnect con6
+
+--connection default
+--sorted_result
+XA RECOVER;
+
+XA COMMIT 'x2';
+--let $binlog_file= binlog-000000.ibb
+--let $binlog_start= 4
+--source include/show_binlog_events.inc
+--let $binlog_file= binlog-000001.ibb
+--source include/show_binlog_events.inc
+--let $binlog_file= binlog-000002.ibb
+--source include/show_binlog_events.inc
+FLUSH BINARY LOGS;
+--let $binlog_file= binlog-000003.ibb
+--source include/show_binlog_events.inc
+
+SELECT a, b, LENGTH(c) FROM t1 ORDER BY a, b;
+
+--echo *** Test that we can purge, no leaked XA/OOB refs.
+SET @old_min_slaves= @@GLOBAL.slave_connections_needed_for_purge;
+SET GLOBAL slave_connections_needed_for_purge= 0;
+FLUSH BINARY LOGS;
+PURGE BINARY LOGS TO 'binlog-000004.ibb';
+SET GLOBAL slave_connections_needed_for_purge= @old_min_slaves;
+
+DROP TABLE t1;
diff --git a/mysql-test/suite/binlog_in_engine/innodb_log_check.result b/mysql-test/suite/binlog_in_engine/innodb_log_check.result
new file mode 100644
index 0000000000000..ce8e469a45a2f
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/innodb_log_check.result
@@ -0,0 +1,16 @@
+*** Test that binlogging throttles large writes, waiting for InnoDB checkpoints as needed
+CREATE TABLE t1 (a INT, b INT, c LONGTEXT, PRIMARY KEY(a, b)) ENGINE=InnoDB;
+SET @old_max_packet= @@GLOBAL.max_allowed_packet;
+SET GLOBAL max_allowed_packet=128*1024*1024;
+SET @old_max_binlog= @@GLOBAL.max_binlog_size;
+SET GLOBAL max_binlog_size= 16*1024*1024;
+connect con1,localhost,root;
+INSERT INTO t1 VALUES (1, 1, REPEAT('x', 64*1024*1024));
+disconnect con1;
+connection default;
+SELECT a, b, LENGTH(c) FROM t1 ORDER BY a, b;
+a b LENGTH(c)
+1 1 67108864
+SET GLOBAL max_allowed_packet= @old_max_packet;
+SET GLOBAL max_binlog_size= @old_max_binlog;
+DROP TABLE t1;
diff --git a/mysql-test/suite/binlog_in_engine/innodb_log_check.test b/mysql-test/suite/binlog_in_engine/innodb_log_check.test
new file mode 100644
index 0000000000000..824f95b5e3110
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/innodb_log_check.test
@@ -0,0 +1,30 @@
+--source include/have_binlog_format_row.inc
+--source include/have_innodb_binlog.inc
+
+--echo *** Test that binlogging throttles large writes, waiting for InnoDB checkpoints as needed
+
+CREATE TABLE t1 (a INT, b INT, c LONGTEXT, PRIMARY KEY(a, b)) ENGINE=InnoDB;
+
+# Do a large amount of binlogging in a single operation. The binlog write
+# must throttle the amount of data and wait for InnoDB checkpoints as
+# approrpiate to avoid overwriting the head of the cyclic InnoDB write-ahead
+# log. Otherwise it will cause an error in the error log that will make the
+# test fail:
+#
+# [ERROR] InnoDB: Crash recovery is broken due to insufficient innodb_log_file_size
+
+SET @old_max_packet= @@GLOBAL.max_allowed_packet;
+SET GLOBAL max_allowed_packet=128*1024*1024;
+SET @old_max_binlog= @@GLOBAL.max_binlog_size;
+SET GLOBAL max_binlog_size= 16*1024*1024;
+
+--connect (con1,localhost,root)
+INSERT INTO t1 VALUES (1, 1, REPEAT('x', 64*1024*1024));
+--disconnect con1
+--connection default
+
+SELECT a, b, LENGTH(c) FROM t1 ORDER BY a, b;
+
+SET GLOBAL max_allowed_packet= @old_max_packet;
+SET GLOBAL max_binlog_size= @old_max_binlog;
+DROP TABLE t1;
diff --git a/mysql-test/suite/binlog_in_engine/mariabackup_binlog_dir.result b/mysql-test/suite/binlog_in_engine/mariabackup_binlog_dir.result
new file mode 100644
index 0000000000000..c83a150161aff
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/mariabackup_binlog_dir.result
@@ -0,0 +1,12 @@
+include/reset_master.inc
+#
+# MDEV-38432: Issues with mariabackup's binlog-directory option
+#
+create table t (a int) engine=InnoDB;
+flush binary logs;
+insert into t values (1);
+flush binary logs;
+binlog-000000.ibb
+binlog-000001.ibb
+binlog-000002.ibb
+drop table t;
diff --git a/mysql-test/suite/binlog_in_engine/mariabackup_binlog_dir.test b/mysql-test/suite/binlog_in_engine/mariabackup_binlog_dir.test
new file mode 100644
index 0000000000000..b83faeab90100
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/mariabackup_binlog_dir.test
@@ -0,0 +1,31 @@
+--source include/not_embedded.inc
+--source include/have_innodb_binlog.inc
+
+--source include/reset_master.inc
+
+--echo #
+--echo # MDEV-38432: Issues with mariabackup's binlog-directory option
+--echo #
+
+create table t (a int) engine=InnoDB;
+flush binary logs;
+insert into t values (1);
+flush binary logs;
+
+--let $binlog_name= binlog-000002.ibb
+--let $binlog_size= 262144
+--source include/wait_for_engine_binlog.inc
+
+--exec $XTRABACKUP --backup --target-dir=$MYSQL_TMP_DIR/mbackup --socket=$MASTER_MYSOCK --user=root --binlog-directory=backup_binlogs > $MYSQL_TMP_DIR/mbackup.log 2>&1
+--exec $XTRABACKUP --prepare --target-dir=$MYSQL_TMP_DIR/mbackup > $MYSQL_TMP_DIR/mprepare.log 2>&1
+--exec $XTRABACKUP --copy-back --target-dir=$MYSQL_TMP_DIR/mbackup --binlog-directory=binlogs --datadir=$MYSQL_TMP_DIR/mcopyback > $MYSQL_TMP_DIR/mcopyback.log 2>&1
+# The bug was a missing slash, so the binlogs would be in mcopybackbinlogs/
+--list_files $MYSQL_TMP_DIR/mcopyback/binlogs/ *
+
+drop table t;
+
+--remove_file $MYSQL_TMP_DIR/mbackup.log
+--remove_file $MYSQL_TMP_DIR/mprepare.log
+--remove_file $MYSQL_TMP_DIR/mcopyback.log
+--rmdir $MYSQL_TMP_DIR/mbackup
+--rmdir $MYSQL_TMP_DIR/mcopyback
diff --git a/mysql-test/suite/binlog_in_engine/mariabackup_binlogs.result b/mysql-test/suite/binlog_in_engine/mariabackup_binlogs.result
new file mode 100644
index 0000000000000..4c86df8850098
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/mariabackup_binlogs.result
@@ -0,0 +1,73 @@
+include/reset_master.inc
+*** Test mariabackup concurrent with RESET MASTER and FLUSH BINARY LOGS.
+CREATE TABLE t1(a INT PRIMARY KEY) ENGINE=InnoDB;
+INSERT INTO t1 VALUES (1);
+INSERT INTO t1 SELECT a+1 FROM t1;
+INSERT INTO t1 SELECT a+2 FROM t1;
+INSERT INTO t1 SELECT a+4 FROM t1;
+INSERT INTO t1 SELECT a+8 FROM t1;
+FLUSH BINARY LOGS;
+INSERT INTO t1 SELECT a+16 FROM t1;
+INSERT INTO t1 SELECT a+32 FROM t1;
+FLUSH BINARY LOGS;
+INSERT INTO t1 SELECT a+64 FROM t1;
+INSERT INTO t1 SELECT a+128 FROM t1;
+SET @old_needed= @@GLOBAL.slave_connections_needed_for_purge;
+SET GLOBAL slave_connections_needed_for_purge=0;
+connect con$i,localhost,root,,;
+BEGIN NOT ATOMIC
+SELECT SLEEP(0.9);
+FLUSH BINARY LOGS;
+END //
+connection default;
+connection con1;
+SLEEP(0.9)
+0
+disconnect con1;
+connection default;
+FLUSH BINARY LOGS;
+INSERT INTO t1 SELECT a+256 FROM t1;
+connect con$i,localhost,root,,;
+BEGIN NOT ATOMIC
+SELECT SLEEP(1.1);
+PURGE BINARY LOGS TO 'binlog-000003.ibb';
+END //
+connection default;
+connection con2;
+SLEEP(1.1)
+0
+disconnect con2;
+connection default;
+RESET MASTER;
+INSERT INTO t1 SELECT a+512 FROM t1 WHERE a <= 256;
+FLUSH BINARY LOGS;
+INSERT INTO t1 SELECT a+768 FROM t1 WHERE a <= 256;
+FLUSH BINARY LOGS;
+connect con$i,localhost,root,,;
+BEGIN NOT ATOMIC
+SELECT SLEEP(0.1);
+RESET MASTER;
+END //
+connection default;
+connection con3;
+SLEEP(0.1)
+0
+disconnect con3;
+connection default;
+INSERT INTO t1 SELECT a+1024 FROM t1 WHERE a <= 256;
+FLUSH BINARY LOGS;
+INSERT INTO t1 SELECT a+1280 FROM t1 WHERE a <= 256;
+FLUSH BINARY LOGS;
+connect con$i,localhost,root,,;
+BEGIN NOT ATOMIC
+SELECT SLEEP(1.1);
+RESET MASTER;
+END //
+connection default;
+connection con4;
+SLEEP(1.1)
+0
+disconnect con4;
+connection default;
+DROP TABLE t1;
+SET GLOBAL slave_connections_needed_for_purge= @old_needed;
diff --git a/mysql-test/suite/binlog_in_engine/mariabackup_binlogs.test b/mysql-test/suite/binlog_in_engine/mariabackup_binlogs.test
new file mode 100644
index 0000000000000..eeef77d1e132d
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/mariabackup_binlogs.test
@@ -0,0 +1,118 @@
+--source include/not_embedded.inc
+--source include/have_debug.inc
+--source include/have_innodb_binlog.inc
+
+--source include/reset_master.inc
+
+let $basedir= $MYSQLTEST_VARDIR/tmp/backup;
+let $datadir_2= $MYSQLTEST_VARDIR/tmp/restore;
+
+--echo *** Test mariabackup concurrent with RESET MASTER and FLUSH BINARY LOGS.
+CREATE TABLE t1(a INT PRIMARY KEY) ENGINE=InnoDB;
+INSERT INTO t1 VALUES (1);
+INSERT INTO t1 SELECT a+1 FROM t1;
+INSERT INTO t1 SELECT a+2 FROM t1;
+INSERT INTO t1 SELECT a+4 FROM t1;
+INSERT INTO t1 SELECT a+8 FROM t1;
+FLUSH BINARY LOGS;
+INSERT INTO t1 SELECT a+16 FROM t1;
+INSERT INTO t1 SELECT a+32 FROM t1;
+FLUSH BINARY LOGS;
+INSERT INTO t1 SELECT a+64 FROM t1;
+INSERT INTO t1 SELECT a+128 FROM t1;
+
+# Attempt to run a RESET MASTER, a FLUSH BINARY LOGS, and a PURGE concurrent
+# with the copy of the binlog files, to test that backup locks will prevent
+# the binlog files from changing during the copy.
+# We test this by putting a sleep in mariabackup after getting the list
+# of binlog files and before copying them. And then sending paralle
+# RESET/FLUSH/PURGE with a delay that is shorter than the sleep but longer
+# than the typical time to run the backup.
+# This gives a good chance to hit the potential race, and missing it is
+# not critical, it will at most cause a false negative, but never false
+# positive.
+SET @old_needed= @@GLOBAL.slave_connections_needed_for_purge;
+SET GLOBAL slave_connections_needed_for_purge=0;
+
+--let $i= 1
+while ($i <= 4) {
+ --connect con$i,localhost,root,,
+ --delimiter //
+
+ if ($i == 1) {
+ # A FLUSH BINARY LOGS that truncates binlog-000002.ibb at around the same
+ # time that mariabackup tries to copy it.
+ send BEGIN NOT ATOMIC
+ SELECT SLEEP(0.9);
+ FLUSH BINARY LOGS;
+ END //
+ }
+
+ if ($i == 2) {
+ # A PURGE BINARY LOGS that removes binlog-000002.ibb at around the same
+ # time that mariabackup tries to copy it.
+ send BEGIN NOT ATOMIC
+ SELECT SLEEP(1.1);
+ PURGE BINARY LOGS TO 'binlog-000003.ibb';
+ END //
+ }
+
+ if ($i == 3) {
+ # A RESET MASTER that removes binlog-000002.ibb early during the first
+ # stage of backup.
+ send BEGIN NOT ATOMIC
+ SELECT SLEEP(0.1);
+ RESET MASTER;
+ END //
+ }
+
+ if ($i == 4) {
+ # A RESET MASTER that removes binlog-000002.ibb at around the same
+ # time that mariabackup tries to copy it.
+ send BEGIN NOT ATOMIC
+ SELECT SLEEP(1.1);
+ RESET MASTER;
+ END //
+ }
+
+ --delimiter ;
+
+ --connection default
+ --exec $XTRABACKUP --defaults-file=$MYSQLTEST_VARDIR/my.cnf --backup --target-dir=$basedir --dbug=+d,binlog_copy_sleep_2
+ --exec $XTRABACKUP --defaults-file=$MYSQLTEST_VARDIR/my.cnf --prepare --target-dir=$basedir
+ --exec $XTRABACKUP --copy-back --datadir=$datadir_2 --target-dir=$basedir
+ --rmdir $basedir
+ --rmdir $datadir_2
+
+ --connection con$i
+ REAP;
+ --disconnect con$i
+ --connection default
+
+ if ($i == 1) {
+ # Another FLUSH, moving the active binlog file to binlog-000004.ibb, so
+ # that next round can PURGE to remove binlog-000002.ibb.
+ FLUSH BINARY LOGS;
+ INSERT INTO t1 SELECT a+256 FROM t1;
+ }
+
+ if ($i == 2) {
+ # Re-create the binlog-000002.ibb for the following RESET MASTER test.
+ RESET MASTER;
+ INSERT INTO t1 SELECT a+512 FROM t1 WHERE a <= 256;
+ FLUSH BINARY LOGS;
+ INSERT INTO t1 SELECT a+768 FROM t1 WHERE a <= 256;
+ FLUSH BINARY LOGS;
+ }
+ if ($i == 3) {
+ INSERT INTO t1 SELECT a+1024 FROM t1 WHERE a <= 256;
+ FLUSH BINARY LOGS;
+ INSERT INTO t1 SELECT a+1280 FROM t1 WHERE a <= 256;
+ FLUSH BINARY LOGS;
+ }
+
+ inc $i;
+}
+
+DROP TABLE t1;
+SET GLOBAL slave_connections_needed_for_purge= @old_needed;
diff --git a/mysql-test/suite/binlog_in_engine/mariabackup_slave_provision_binlog_dir.cnf b/mysql-test/suite/binlog_in_engine/mariabackup_slave_provision_binlog_dir.cnf
new file mode 100644
index 0000000000000..2e048897e6375
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/mariabackup_slave_provision_binlog_dir.cnf
@@ -0,0 +1,5 @@
+!include my.cnf
+
+[mysqld]
+binlog_directory=binlogs
+loose_innodb_log_file_size=256M
diff --git a/mysql-test/suite/binlog_in_engine/mariabackup_slave_provision_binlog_dir.result b/mysql-test/suite/binlog_in_engine/mariabackup_slave_provision_binlog_dir.result
new file mode 100644
index 0000000000000..7957270040289
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/mariabackup_slave_provision_binlog_dir.result
@@ -0,0 +1,40 @@
+include/reset_master.inc
+CREATE TABLE t1(a varchar(60) PRIMARY KEY, b VARCHAR(60)) ENGINE INNODB;
+INSERT INTO t1 VALUES(1, NULL);
+CREATE TABLE t2 (val INT) ENGINE=InnoDB;
+INSERT INTO t2 VALUES (0);
+connect con1,localhost,root,,;
+*** Start a background load...
+CALL gen_load();
+connection default;
+*** Doing backup...
+*** Doing prepare...
+*** Stop the background load...
+UPDATE t2 SET val=1;
+connection con1;
+connection default;
+disconnect con1;
+*** Provision a new slave from the backup
+connect server2,127.0.0.1,root,,,$SERVER_MYPORT_2;
+*** Stopping provisioned server
+*** Removing old datadir for provisioned server
+*** Provision new server from backup
+# restart: --skip-slave-start
+SET GLOBAL gtid_slave_pos= 'GTID_POS';
+CHANGE MASTER TO
+master_ssl_verify_server_cert=0,
+master_port=SERVER_MYPORT_1, master_host='127.0.0.1', master_user='root',
+master_use_gtid= Slave_pos;
+START SLAVE;
+connection default;
+include/save_master_gtid.inc
+connection server2;
+include/sync_with_master_gtid.inc
+connection server2;
+STOP SLAVE;
+RESET SLAVE ALL;
+DROP PROCEDURE gen_load;
+DROP TABLE t1, t2;
+connection default;
+DROP PROCEDURE gen_load;
+DROP TABLE t1, t2;
diff --git a/mysql-test/suite/binlog_in_engine/mariabackup_slave_provision_binlog_dir.test b/mysql-test/suite/binlog_in_engine/mariabackup_slave_provision_binlog_dir.test
new file mode 100644
index 0000000000000..43301c519387b
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/mariabackup_slave_provision_binlog_dir.test
@@ -0,0 +1,4 @@
+--source include/not_embedded.inc
+--let $backup_args=
+--let $copy_back_args= --binlog-directory=binlogs
+--source include/mariabackup_slave_provision.inc
diff --git a/mysql-test/suite/binlog_in_engine/mariabackup_slave_provision_no_binlog.cnf b/mysql-test/suite/binlog_in_engine/mariabackup_slave_provision_no_binlog.cnf
new file mode 100644
index 0000000000000..3c67b7c2b638e
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/mariabackup_slave_provision_no_binlog.cnf
@@ -0,0 +1,4 @@
+!include my.cnf
+
+[mysqld]
+loose_innodb_log_file_size=256M
diff --git a/mysql-test/suite/binlog_in_engine/mariabackup_slave_provision_no_binlog.result b/mysql-test/suite/binlog_in_engine/mariabackup_slave_provision_no_binlog.result
new file mode 100644
index 0000000000000..e8993f0371204
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/mariabackup_slave_provision_no_binlog.result
@@ -0,0 +1,44 @@
+include/reset_master.inc
+CREATE TABLE t1(a varchar(60) PRIMARY KEY, b VARCHAR(60)) ENGINE INNODB;
+INSERT INTO t1 VALUES(1, NULL);
+CREATE TABLE t2 (val INT) ENGINE=InnoDB;
+INSERT INTO t2 VALUES (0);
+connect con1,localhost,root,,;
+*** Start a background load...
+CALL gen_load();
+connection default;
+*** Doing backup...
+*** Doing prepare...
+*** Stop the background load...
+UPDATE t2 SET val=1;
+connection con1;
+connection default;
+disconnect con1;
+*** Provision a new slave from the backup
+connect server2,127.0.0.1,root,,,$SERVER_MYPORT_2;
+*** Stopping provisioned server
+*** Removing old datadir for provisioned server
+*** Provision new server from backup
+# restart: --skip-slave-start
+CREATE TABLE t3 (file VARCHAR(255), pos INT, gtid VARCHAR(255)) ENGINE=InnoDB;
+LOAD DATA LOCAL INFILE "BASEDIR/mariadb_backup_binlog_info"
+ INTO TABLE t3 FIELDS ESCAPED BY '' (file, pos, gtid);
+DROP TABLE t3;
+SET GLOBAL gtid_slave_pos= 'GTID_POS';
+CHANGE MASTER TO
+master_ssl_verify_server_cert=0,
+master_port=SERVER_MYPORT_1, master_host='127.0.0.1', master_user='root',
+master_use_gtid= Slave_pos;
+START SLAVE;
+connection default;
+include/save_master_gtid.inc
+connection server2;
+include/sync_with_master_gtid.inc
+connection server2;
+STOP SLAVE;
+RESET SLAVE ALL;
+DROP PROCEDURE gen_load;
+DROP TABLE t1, t2;
+connection default;
+DROP PROCEDURE gen_load;
+DROP TABLE t1, t2;
diff --git a/mysql-test/suite/binlog_in_engine/mariabackup_slave_provision_no_binlog.test b/mysql-test/suite/binlog_in_engine/mariabackup_slave_provision_no_binlog.test
new file mode 100644
index 0000000000000..19178a47c302c
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/mariabackup_slave_provision_no_binlog.test
@@ -0,0 +1,6 @@
+--source include/not_embedded.inc
+--let $backup_args= --skip-binlog
+--let $prepare_args= --skip-binlog
+--let $nobinlog= 1
+--let $copy_back_args= --binlog-directory=binlogs
+--source include/mariabackup_slave_provision.inc
diff --git a/mysql-test/suite/binlog_in_engine/mariabackup_slave_provision_nolock.cnf b/mysql-test/suite/binlog_in_engine/mariabackup_slave_provision_nolock.cnf
new file mode 100644
index 0000000000000..3c67b7c2b638e
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/mariabackup_slave_provision_nolock.cnf
@@ -0,0 +1,4 @@
+!include my.cnf
+
+[mysqld]
+loose_innodb_log_file_size=256M
diff --git a/mysql-test/suite/binlog_in_engine/mariabackup_slave_provision_nolock.result b/mysql-test/suite/binlog_in_engine/mariabackup_slave_provision_nolock.result
new file mode 100644
index 0000000000000..7957270040289
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/mariabackup_slave_provision_nolock.result
@@ -0,0 +1,40 @@
+include/reset_master.inc
+CREATE TABLE t1(a varchar(60) PRIMARY KEY, b VARCHAR(60)) ENGINE INNODB;
+INSERT INTO t1 VALUES(1, NULL);
+CREATE TABLE t2 (val INT) ENGINE=InnoDB;
+INSERT INTO t2 VALUES (0);
+connect con1,localhost,root,,;
+*** Start a background load...
+CALL gen_load();
+connection default;
+*** Doing backup...
+*** Doing prepare...
+*** Stop the background load...
+UPDATE t2 SET val=1;
+connection con1;
+connection default;
+disconnect con1;
+*** Provision a new slave from the backup
+connect server2,127.0.0.1,root,,,$SERVER_MYPORT_2;
+*** Stopping provisioned server
+*** Removing old datadir for provisioned server
+*** Provision new server from backup
+# restart: --skip-slave-start
+SET GLOBAL gtid_slave_pos= 'GTID_POS';
+CHANGE MASTER TO
+master_ssl_verify_server_cert=0,
+master_port=SERVER_MYPORT_1, master_host='127.0.0.1', master_user='root',
+master_use_gtid= Slave_pos;
+START SLAVE;
+connection default;
+include/save_master_gtid.inc
+connection server2;
+include/sync_with_master_gtid.inc
+connection server2;
+STOP SLAVE;
+RESET SLAVE ALL;
+DROP PROCEDURE gen_load;
+DROP TABLE t1, t2;
+connection default;
+DROP PROCEDURE gen_load;
+DROP TABLE t1, t2;
diff --git a/mysql-test/suite/binlog_in_engine/mariabackup_slave_provision_nolock.test b/mysql-test/suite/binlog_in_engine/mariabackup_slave_provision_nolock.test
new file mode 100644
index 0000000000000..95dbff47d1aa6
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/mariabackup_slave_provision_nolock.test
@@ -0,0 +1,4 @@
+--source include/not_embedded.inc
+--let $backup_args= --no-lock
+--let $copy_back_args=
+--source include/mariabackup_slave_provision.inc
diff --git a/mysql-test/suite/binlog_in_engine/my.cnf b/mysql-test/suite/binlog_in_engine/my.cnf
new file mode 100644
index 0000000000000..b6a2d13c4ead6
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/my.cnf
@@ -0,0 +1,29 @@
+!include include/default_my.cnf
+
+[mysqld.1]
+innodb
+binlog-storage-engine=innodb
+max_binlog_size=8M
+innodb-binlog-state-interval=128k
+log-basename= master
+
+[mysqld.2]
+#!use-slave-opt
+innodb
+binlog-storage-engine=innodb
+max_binlog_size=8M
+innodb-binlog-state-interval=128k
+log-slave-updates
+log-basename= slave
+
+[ENV]
+
+# We will adopt tests with master-slave setup as well as rpl_init setup,
+# so need both sets of variables
+MASTER_MYPORT= @mysqld.1.port
+SERVER_MYPORT_1= @mysqld.1.port
+SERVER_MYSOCK_1= @mysqld.1.socket
+
+SLAVE_MYPORT= @mysqld.2.port
+SERVER_MYPORT_2= @mysqld.2.port
+SERVER_MYSOCK_2= @mysqld.2.socket
diff --git a/mysql-test/suite/binlog_in_engine/mysqlbinlog-master.opt b/mysql-test/suite/binlog_in_engine/mysqlbinlog-master.opt
new file mode 100644
index 0000000000000..4d69f3359db5c
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/mysqlbinlog-master.opt
@@ -0,0 +1 @@
+--timezone=GMT-3
diff --git a/mysql-test/suite/binlog_in_engine/mysqlbinlog.result b/mysql-test/suite/binlog_in_engine/mysqlbinlog.result
new file mode 100644
index 0000000000000..efe7020f70e8a
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/mysqlbinlog.result
@@ -0,0 +1,103 @@
+include/reset_master.inc
+set TIMESTAMP= UNIX_TIMESTAMP("1970-01-21 15:32:22");
+CREATE TABLE t1 (a INT PRIMARY KEY, b INT) ENGINE=InnoDB;
+*** Test dumping all binlog files, including the last empty one.
+FLUSH BINARY LOGS;
+/*!50530 SET @@SESSION.PSEUDO_SLAVE_MODE=1*/;
+/*!40019 SET @@session.max_delayed_threads=0*/;
+/*!50003 SET @OLD_COMPLETION_TYPE=@@COMPLETION_TYPE,COMPLETION_TYPE=0*/;
+DELIMITER /*!*/;
+use `test`/*!*/;
+SET TIMESTAMP=1773142/*!*/;
+SET @@session.pseudo_thread_id=999999999/*!*/;
+SET @@session.foreign_key_checks=1, @@session.sql_auto_is_null=0, @@session.unique_checks=1, @@session.autocommit=1, @@session.check_constraint_checks=1, @@session.system_versioning_insert_history=0/*!*/;
+SET @@session.sql_mode=1411383296/*!*/;
+SET @@session.auto_increment_increment=1, @@session.auto_increment_offset=1/*!*/;
+/*!\C latin1 *//*!*/;
+SET @@session.character_set_client=X,@@session.collation_connection=X,@@session.collation_server=X/*!*/;
+SET @@session.lc_time_names=0/*!*/;
+SET @@session.collation_database=DEFAULT/*!*/;
+CREATE TABLE t1 (a INT PRIMARY KEY, b INT) ENGINE=InnoDB
+/*!*/;
+DELIMITER ;
+DELIMITER /*!*/;
+DELIMITER ;
+DELIMITER /*!*/;
+DELIMITER ;
+# End of log file
+ROLLBACK /* added by mysqlbinlog */;
+/*!50003 SET COMPLETION_TYPE=@OLD_COMPLETION_TYPE*/;
+/*!50530 SET @@SESSION.PSEUDO_SLAVE_MODE=0*/;
+INSERT INTO t1 VALUES (1, 0), (2, 0), (3, 0);
+UPDATE t1 SET b=1 WHERE a=1;
+DELETE FROM t1 WHERE a=2;
+REPLACE INTO t1 VALUES (3, 3);
+FLUSH BINARY LOGS;
+/*!50530 SET @@SESSION.PSEUDO_SLAVE_MODE=1*/;
+/*!40019 SET @@session.max_delayed_threads=0*/;
+/*!50003 SET @OLD_COMPLETION_TYPE=@@COMPLETION_TYPE,COMPLETION_TYPE=0*/;
+DELIMITER /*!*/;
+START TRANSACTION
+/*!*/;
+# Annotate_rows:
+#Q> INSERT INTO t1 VALUES (1, 0), (2, 0), (3, 0)
+COMMIT/*!*/;
+START TRANSACTION
+/*!*/;
+# Annotate_rows:
+#Q> UPDATE t1 SET b=1 WHERE a=1
+COMMIT/*!*/;
+START TRANSACTION
+/*!*/;
+# Annotate_rows:
+#Q> DELETE FROM t1 WHERE a=2
+COMMIT/*!*/;
+START TRANSACTION
+/*!*/;
+# Annotate_rows:
+#Q> REPLACE INTO t1 VALUES (3, 3)
+COMMIT/*!*/;
+DELIMITER ;
+# End of log file
+ROLLBACK /* added by mysqlbinlog */;
+/*!50003 SET COMPLETION_TYPE=@OLD_COMPLETION_TYPE*/;
+/*!50530 SET @@SESSION.PSEUDO_SLAVE_MODE=0*/;
+*** Test dumping the last, pre-allocated, empty file.
+/*!50530 SET @@SESSION.PSEUDO_SLAVE_MODE=1*/;
+/*!40019 SET @@session.max_delayed_threads=0*/;
+/*!50003 SET @OLD_COMPLETION_TYPE=@@COMPLETION_TYPE,COMPLETION_TYPE=0*/;
+DELIMITER /*!*/;
+DELIMITER ;
+# End of log file
+ROLLBACK /* added by mysqlbinlog */;
+/*!50003 SET COMPLETION_TYPE=@OLD_COMPLETION_TYPE*/;
+/*!50530 SET @@SESSION.PSEUDO_SLAVE_MODE=0*/;
+SELECT * FROM t1 ORDER BY a;
+a b
+1 1
+3 3
+SELECT * FROM t1 ORDER BY a;
+a b
+SELECT * FROM t1 ORDER BY a;
+a b
+1 1
+3 3
+SET SESSION binlog_format= MIXED;
+INSERT INTO t1 SELECT a+10, 10 FROM t1;
+UPDATE t1 SET b=11 WHERE a <= 12;
+SELECT * FROM t1 ORDER BY a;
+a b
+1 11
+3 11
+11 11
+13 10
+FLUSH BINARY LOGS;
+FLUSH BINARY LOGS;
+TRUNCATE t1;
+SELECT * FROM t1 ORDER BY a;
+a b
+1 11
+3 11
+11 11
+13 10
+DROP TABLE t1;
diff --git a/mysql-test/suite/binlog_in_engine/mysqlbinlog.test b/mysql-test/suite/binlog_in_engine/mysqlbinlog.test
new file mode 100644
index 0000000000000..fdcb55a6a82eb
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/mysqlbinlog.test
@@ -0,0 +1,75 @@
+--source include/have_binlog_format_row.inc
+--source include/have_innodb_binlog.inc
+
+--let $datadir= `SELECT @@datadir`
+
+--source include/reset_master.inc
+
+set TIMESTAMP= UNIX_TIMESTAMP("1970-01-21 15:32:22");
+CREATE TABLE t1 (a INT PRIMARY KEY, b INT) ENGINE=InnoDB;
+
+--echo *** Test dumping all binlog files, including the last empty one.
+FLUSH BINARY LOGS; # To make sure the CREATE TABLE is on disk
+--let $binlog_name= binlog-000002.ibb
+--let $binlog_size= 262144
+--source include/wait_for_engine_binlog.inc
+--replace_regex /collation_server=[0-9]+/collation_server=X/ /character_set_client=[a-zA-Z0-9]+/character_set_client=X/ /collation_connection=[0-9]+/collation_connection=X/
+--exec $MYSQL_BINLOG --short-form --base64-output=never $datadir/binlog-000000.ibb $datadir/binlog-000001.ibb $datadir/binlog-000002.ibb 2>&1
+
+--let $gtid_start= `SELECT @@gtid_binlog_pos`
+INSERT INTO t1 VALUES (1, 0), (2, 0), (3, 0);
+UPDATE t1 SET b=1 WHERE a=1;
+DELETE FROM t1 WHERE a=2;
+REPLACE INTO t1 VALUES (3, 3);
+--let $gtid_stop= `SELECT @@gtid_binlog_pos`
+
+# Force rotate the binlog to make sure the one we want to dump is written
+# to disk.
+FLUSH BINARY LOGS;
+--let $binlog_name= binlog-000003.ibb
+--let $binlog_size= 262144
+--source include/wait_for_engine_binlog.inc
+--replace_regex /collation_server=[0-9]+/collation_server=X/ /character_set_client=[a-zA-Z0-9]+/character_set_client=X/ /collation_connection=[0-9]+/collation_connection=X/
+--exec $MYSQL_BINLOG --short-form --base64-output=never $datadir/binlog-000001.ibb
+
+--echo *** Test dumping the last, pre-allocated, empty file.
+--replace_regex /collation_server=[0-9]+/collation_server=X/ /character_set_client=[a-zA-Z0-9]+/character_set_client=X/ /collation_connection=[0-9]+/collation_connection=X/
+--exec $MYSQL_BINLOG --short-form --base64-output=never $datadir/binlog-000003.ibb 2>&1
+
+SELECT * FROM t1 ORDER BY a;
+
+# Test flashback from files specified with directory path.
+--exec $MYSQL_BINLOG --flashback --start-position=$gtid_start --stop-position=$gtid_stop $datadir/binlog-000000.ibb $datadir/binlog-000001.ibb > $MYSQLTEST_VARDIR/tmp/mysqlbinlog_1.txt
+--exec $MYSQL --abort-source-on-error -e "source $MYSQLTEST_VARDIR/tmp/mysqlbinlog_1.txt;" test
+--remove_file $MYSQLTEST_VARDIR/tmp/mysqlbinlog_1.txt
+
+SELECT * FROM t1 ORDER BY a;
+
+# Test normal apply from binlog files in current directory
+--exec cd $datadir && $MYSQL_BINLOG --start-position=$gtid_start --stop-position=$gtid_stop binlog-000000.ibb binlog-000001.ibb > $MYSQLTEST_VARDIR/tmp/mysqlbinlog_2.txt
+--exec $MYSQL -e "source $MYSQLTEST_VARDIR/tmp/mysqlbinlog_2.txt;"
+--remove_file $MYSQLTEST_VARDIR/tmp/mysqlbinlog_2.txt
+
+SELECT * FROM t1 ORDER BY a;
+
+SET SESSION binlog_format= MIXED;
+INSERT INTO t1 SELECT a+10, 10 FROM t1;
+UPDATE t1 SET b=11 WHERE a <= 12;
+--let $gtid_stop= `SELECT @@gtid_binlog_pos`
+
+SELECT * FROM t1 ORDER BY a;
+
+FLUSH BINARY LOGS;
+FLUSH BINARY LOGS;
+
+# Test across multiple binlog files.
+# Need --gtid-strict-mode=0 here, as the previous replay duplicated the
+# original GTIDs.
+TRUNCATE t1;
+--exec $MYSQL_BINLOG --start-position=$gtid_start --stop-position=$gtid_stop --gtid-strict-mode=0 $datadir/binlog-000000.ibb $datadir/binlog-000001.ibb $datadir/binlog-000002.ibb > $MYSQLTEST_VARDIR/tmp/mysqlbinlog_3.txt
+--exec $MYSQL -e "source $MYSQLTEST_VARDIR/tmp/mysqlbinlog_3.txt;"
+--remove_file $MYSQLTEST_VARDIR/tmp/mysqlbinlog_3.txt
+
+SELECT * FROM t1 ORDER BY a;
+
+DROP TABLE t1;
diff --git a/mysql-test/suite/binlog_in_engine/nontrans.result b/mysql-test/suite/binlog_in_engine/nontrans.result
new file mode 100644
index 0000000000000..c5c74139a95e7
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/nontrans.result
@@ -0,0 +1,481 @@
+include/master-slave.inc
+[connection master]
+CREATE TABLE t1(a INT PRIMARY KEY, b INT, c LONGTEXT) ENGINE=InnoDB;
+CREATE TABLE t2(a INT PRIMARY KEY, b INT, c LONGTEXT) ENGINE=Aria;
+SET @c= REPEAT('*', 20);
+SET SESSION binlog_format=statement;
+SET SESSION binlog_direct_non_transactional_updates= 0;
+INSERT INTO t1 VALUES (1 + 0, 0, @c), (2 + 0, 0, @c), (3 + 0, 0, @c);
+INSERT INTO t2 VALUES (1 + 0, 1, @c), (2 + 0, 1, @c), (3 + 0, 1, @c);
+BEGIN;
+UPDATE t1 SET b=2, c=CONCAT('!', c) WHERE a=1+0;
+UPDATE t2 SET b=2, c=CONCAT('!', c) WHERE a=2+0;
+Warnings:
+Note 1592 Unsafe statement written to the binary log using statement format since BINLOG_FORMAT = STATEMENT. Statement is unsafe because it accesses a non-transactional table after accessing a transactional table within the same transaction
+UPDATE t1 SET b=2, c=CONCAT('!', c) WHERE a=3+0;
+INSERT INTO t2 VALUES (4 + 0, 2, @c);
+Warnings:
+Note 1592 Unsafe statement written to the binary log using statement format since BINLOG_FORMAT = STATEMENT. Statement is unsafe because it accesses a non-transactional table after accessing a transactional table within the same transaction
+COMMIT;
+SET SESSION binlog_direct_non_transactional_updates= 1;
+INSERT INTO t1 VALUES (1 + 10, 0, @c), (2 + 10, 0, @c), (3 + 10, 0, @c);
+INSERT INTO t2 VALUES (1 + 10, 1, @c), (2 + 10, 1, @c), (3 + 10, 1, @c);
+BEGIN;
+UPDATE t1 SET b=2, c=CONCAT('!', c) WHERE a=1+10;
+UPDATE t2 SET b=2, c=CONCAT('!', c) WHERE a=2+10;
+UPDATE t1 SET b=2, c=CONCAT('!', c) WHERE a=3+10;
+INSERT INTO t2 VALUES (4 + 10, 2, @c);
+COMMIT;
+SET SESSION binlog_format=row;
+SET SESSION binlog_direct_non_transactional_updates= 0;
+INSERT INTO t1 VALUES (1 + 100, 0, @c), (2 + 100, 0, @c), (3 + 100, 0, @c);
+INSERT INTO t2 VALUES (1 + 100, 1, @c), (2 + 100, 1, @c), (3 + 100, 1, @c);
+BEGIN;
+UPDATE t1 SET b=2, c=CONCAT('!', c) WHERE a=1+100;
+UPDATE t2 SET b=2, c=CONCAT('!', c) WHERE a=2+100;
+UPDATE t1 SET b=2, c=CONCAT('!', c) WHERE a=3+100;
+INSERT INTO t2 VALUES (4 + 100, 2, @c);
+COMMIT;
+SET SESSION binlog_direct_non_transactional_updates= 1;
+INSERT INTO t1 VALUES (1 + 110, 0, @c), (2 + 110, 0, @c), (3 + 110, 0, @c);
+INSERT INTO t2 VALUES (1 + 110, 1, @c), (2 + 110, 1, @c), (3 + 110, 1, @c);
+BEGIN;
+UPDATE t1 SET b=2, c=CONCAT('!', c) WHERE a=1+110;
+UPDATE t2 SET b=2, c=CONCAT('!', c) WHERE a=2+110;
+UPDATE t1 SET b=2, c=CONCAT('!', c) WHERE a=3+110;
+INSERT INTO t2 VALUES (4 + 110, 2, @c);
+COMMIT;
+SET @c= REPEAT('%', 1024);
+SET SESSION binlog_format=statement;
+SET SESSION binlog_direct_non_transactional_updates= 0;
+INSERT INTO t1 VALUES (1 + 1000, 0, @c), (2 + 1000, 0, @c), (3 + 1000, 0, @c);
+INSERT INTO t2 VALUES (1 + 1000, 1, @c), (2 + 1000, 1, @c), (3 + 1000, 1, @c);
+BEGIN;
+UPDATE t1 SET b=2, c=CONCAT('!', c) WHERE a=1+1000;
+UPDATE t2 SET b=2, c=CONCAT('!', c) WHERE a=2+1000;
+Warnings:
+Note 1592 Unsafe statement written to the binary log using statement format since BINLOG_FORMAT = STATEMENT. Statement is unsafe because it accesses a non-transactional table after accessing a transactional table within the same transaction
+UPDATE t1 SET b=2, c=CONCAT('!', c) WHERE a=3+1000;
+INSERT INTO t2 VALUES (4 + 1000, 2, @c);
+Warnings:
+Note 1592 Unsafe statement written to the binary log using statement format since BINLOG_FORMAT = STATEMENT. Statement is unsafe because it accesses a non-transactional table after accessing a transactional table within the same transaction
+COMMIT;
+SET SESSION binlog_direct_non_transactional_updates= 1;
+INSERT INTO t1 VALUES (1 + 1010, 0, @c), (2 + 1010, 0, @c), (3 + 1010, 0, @c);
+INSERT INTO t2 VALUES (1 + 1010, 1, @c), (2 + 1010, 1, @c), (3 + 1010, 1, @c);
+BEGIN;
+UPDATE t1 SET b=2, c=CONCAT('!', c) WHERE a=1+1010;
+UPDATE t2 SET b=2, c=CONCAT('!', c) WHERE a=2+1010;
+UPDATE t1 SET b=2, c=CONCAT('!', c) WHERE a=3+1010;
+INSERT INTO t2 VALUES (4 + 1010, 2, @c);
+COMMIT;
+SET SESSION binlog_format=row;
+SET SESSION binlog_direct_non_transactional_updates= 0;
+INSERT INTO t1 VALUES (1 + 1100, 0, @c), (2 + 1100, 0, @c), (3 + 1100, 0, @c);
+INSERT INTO t2 VALUES (1 + 1100, 1, @c), (2 + 1100, 1, @c), (3 + 1100, 1, @c);
+BEGIN;
+UPDATE t1 SET b=2, c=CONCAT('!', c) WHERE a=1+1100;
+UPDATE t2 SET b=2, c=CONCAT('!', c) WHERE a=2+1100;
+UPDATE t1 SET b=2, c=CONCAT('!', c) WHERE a=3+1100;
+INSERT INTO t2 VALUES (4 + 1100, 2, @c);
+COMMIT;
+SET SESSION binlog_direct_non_transactional_updates= 1;
+INSERT INTO t1 VALUES (1 + 1110, 0, @c), (2 + 1110, 0, @c), (3 + 1110, 0, @c);
+INSERT INTO t2 VALUES (1 + 1110, 1, @c), (2 + 1110, 1, @c), (3 + 1110, 1, @c);
+BEGIN;
+UPDATE t1 SET b=2, c=CONCAT('!', c) WHERE a=1+1110;
+UPDATE t2 SET b=2, c=CONCAT('!', c) WHERE a=2+1110;
+UPDATE t1 SET b=2, c=CONCAT('!', c) WHERE a=3+1110;
+INSERT INTO t2 VALUES (4 + 1110, 2, @c);
+COMMIT;
+SET @c= REPEAT('.', 18000);
+SET SESSION binlog_format=statement;
+SET SESSION binlog_direct_non_transactional_updates= 0;
+INSERT INTO t1 VALUES (1 + 2000, 0, @c), (2 + 2000, 0, @c), (3 + 2000, 0, @c);
+INSERT INTO t2 VALUES (1 + 2000, 1, @c), (2 + 2000, 1, @c), (3 + 2000, 1, @c);
+BEGIN;
+UPDATE t1 SET b=2, c=CONCAT('!', c) WHERE a=1+2000;
+UPDATE t2 SET b=2, c=CONCAT('!', c) WHERE a=2+2000;
+Warnings:
+Note 1592 Unsafe statement written to the binary log using statement format since BINLOG_FORMAT = STATEMENT. Statement is unsafe because it accesses a non-transactional table after accessing a transactional table within the same transaction
+UPDATE t1 SET b=2, c=CONCAT('!', c) WHERE a=3+2000;
+INSERT INTO t2 VALUES (4 + 2000, 2, @c);
+Warnings:
+Note 1592 Unsafe statement written to the binary log using statement format since BINLOG_FORMAT = STATEMENT. Statement is unsafe because it accesses a non-transactional table after accessing a transactional table within the same transaction
+COMMIT;
+SET SESSION binlog_direct_non_transactional_updates= 1;
+INSERT INTO t1 VALUES (1 + 2010, 0, @c), (2 + 2010, 0, @c), (3 + 2010, 0, @c);
+INSERT INTO t2 VALUES (1 + 2010, 1, @c), (2 + 2010, 1, @c), (3 + 2010, 1, @c);
+BEGIN;
+UPDATE t1 SET b=2, c=CONCAT('!', c) WHERE a=1+2010;
+UPDATE t2 SET b=2, c=CONCAT('!', c) WHERE a=2+2010;
+UPDATE t1 SET b=2, c=CONCAT('!', c) WHERE a=3+2010;
+INSERT INTO t2 VALUES (4 + 2010, 2, @c);
+COMMIT;
+SET SESSION binlog_format=row;
+SET SESSION binlog_direct_non_transactional_updates= 0;
+INSERT INTO t1 VALUES (1 + 2100, 0, @c), (2 + 2100, 0, @c), (3 + 2100, 0, @c);
+INSERT INTO t2 VALUES (1 + 2100, 1, @c), (2 + 2100, 1, @c), (3 + 2100, 1, @c);
+BEGIN;
+UPDATE t1 SET b=2, c=CONCAT('!', c) WHERE a=1+2100;
+UPDATE t2 SET b=2, c=CONCAT('!', c) WHERE a=2+2100;
+UPDATE t1 SET b=2, c=CONCAT('!', c) WHERE a=3+2100;
+INSERT INTO t2 VALUES (4 + 2100, 2, @c);
+COMMIT;
+SET SESSION binlog_direct_non_transactional_updates= 1;
+INSERT INTO t1 VALUES (1 + 2110, 0, @c), (2 + 2110, 0, @c), (3 + 2110, 0, @c);
+INSERT INTO t2 VALUES (1 + 2110, 1, @c), (2 + 2110, 1, @c), (3 + 2110, 1, @c);
+BEGIN;
+UPDATE t1 SET b=2, c=CONCAT('!', c) WHERE a=1+2110;
+UPDATE t2 SET b=2, c=CONCAT('!', c) WHERE a=2+2110;
+UPDATE t1 SET b=2, c=CONCAT('!', c) WHERE a=3+2110;
+INSERT INTO t2 VALUES (4 + 2110, 2, @c);
+COMMIT;
+SET @c= REPEAT('.', 40000);
+SET SESSION binlog_format=statement;
+SET SESSION binlog_direct_non_transactional_updates= 0;
+INSERT INTO t1 VALUES (1 + 3000, 0, @c), (2 + 3000, 0, @c), (3 + 3000, 0, @c);
+INSERT INTO t2 VALUES (1 + 3000, 1, @c), (2 + 3000, 1, @c), (3 + 3000, 1, @c);
+BEGIN;
+UPDATE t1 SET b=2, c=CONCAT('!', c) WHERE a=1+3000;
+UPDATE t2 SET b=2, c=CONCAT('!', c) WHERE a=2+3000;
+Warnings:
+Note 1592 Unsafe statement written to the binary log using statement format since BINLOG_FORMAT = STATEMENT. Statement is unsafe because it accesses a non-transactional table after accessing a transactional table within the same transaction
+UPDATE t1 SET b=2, c=CONCAT('!', c) WHERE a=3+3000;
+INSERT INTO t2 VALUES (4 + 3000, 2, @c);
+Warnings:
+Note 1592 Unsafe statement written to the binary log using statement format since BINLOG_FORMAT = STATEMENT. Statement is unsafe because it accesses a non-transactional table after accessing a transactional table within the same transaction
+COMMIT;
+SET SESSION binlog_direct_non_transactional_updates= 1;
+INSERT INTO t1 VALUES (1 + 3010, 0, @c), (2 + 3010, 0, @c), (3 + 3010, 0, @c);
+INSERT INTO t2 VALUES (1 + 3010, 1, @c), (2 + 3010, 1, @c), (3 + 3010, 1, @c);
+BEGIN;
+UPDATE t1 SET b=2, c=CONCAT('!', c) WHERE a=1+3010;
+UPDATE t2 SET b=2, c=CONCAT('!', c) WHERE a=2+3010;
+UPDATE t1 SET b=2, c=CONCAT('!', c) WHERE a=3+3010;
+INSERT INTO t2 VALUES (4 + 3010, 2, @c);
+COMMIT;
+SET SESSION binlog_format=row;
+SET SESSION binlog_direct_non_transactional_updates= 0;
+INSERT INTO t1 VALUES (1 + 3100, 0, @c), (2 + 3100, 0, @c), (3 + 3100, 0, @c);
+INSERT INTO t2 VALUES (1 + 3100, 1, @c), (2 + 3100, 1, @c), (3 + 3100, 1, @c);
+BEGIN;
+UPDATE t1 SET b=2, c=CONCAT('!', c) WHERE a=1+3100;
+UPDATE t2 SET b=2, c=CONCAT('!', c) WHERE a=2+3100;
+UPDATE t1 SET b=2, c=CONCAT('!', c) WHERE a=3+3100;
+INSERT INTO t2 VALUES (4 + 3100, 2, @c);
+COMMIT;
+SET SESSION binlog_direct_non_transactional_updates= 1;
+INSERT INTO t1 VALUES (1 + 3110, 0, @c), (2 + 3110, 0, @c), (3 + 3110, 0, @c);
+INSERT INTO t2 VALUES (1 + 3110, 1, @c), (2 + 3110, 1, @c), (3 + 3110, 1, @c);
+BEGIN;
+UPDATE t1 SET b=2, c=CONCAT('!', c) WHERE a=1+3110;
+UPDATE t2 SET b=2, c=CONCAT('!', c) WHERE a=2+3110;
+UPDATE t1 SET b=2, c=CONCAT('!', c) WHERE a=3+3110;
+INSERT INTO t2 VALUES (4 + 3110, 2, @c);
+COMMIT;
+SELECT a, b, length(c) FROM t1 ORDER BY a;
+a b length(c)
+1 2 21
+2 0 20
+3 2 21
+11 2 21
+12 0 20
+13 2 21
+101 2 21
+102 0 20
+103 2 21
+111 2 21
+112 0 20
+113 2 21
+1001 2 1025
+1002 0 1024
+1003 2 1025
+1011 2 1025
+1012 0 1024
+1013 2 1025
+1101 2 1025
+1102 0 1024
+1103 2 1025
+1111 2 1025
+1112 0 1024
+1113 2 1025
+2001 2 18001
+2002 0 18000
+2003 2 18001
+2011 2 18001
+2012 0 18000
+2013 2 18001
+2101 2 18001
+2102 0 18000
+2103 2 18001
+2111 2 18001
+2112 0 18000
+2113 2 18001
+3001 2 40001
+3002 0 40000
+3003 2 40001
+3011 2 40001
+3012 0 40000
+3013 2 40001
+3101 2 40001
+3102 0 40000
+3103 2 40001
+3111 2 40001
+3112 0 40000
+3113 2 40001
+SELECT a, b, length(c) FROM t2 ORDER BY a;
+a b length(c)
+1 1 20
+2 2 21
+3 1 20
+4 2 20
+11 1 20
+12 2 21
+13 1 20
+14 2 20
+101 1 20
+102 2 21
+103 1 20
+104 2 20
+111 1 20
+112 2 21
+113 1 20
+114 2 20
+1001 1 1024
+1002 2 1025
+1003 1 1024
+1004 2 1024
+1011 1 1024
+1012 2 1025
+1013 1 1024
+1014 2 1024
+1101 1 1024
+1102 2 1025
+1103 1 1024
+1104 2 1024
+1111 1 1024
+1112 2 1025
+1113 1 1024
+1114 2 1024
+2001 1 18000
+2002 2 18001
+2003 1 18000
+2004 2 18000
+2011 1 18000
+2012 2 18001
+2013 1 18000
+2014 2 18000
+2101 1 18000
+2102 2 18001
+2103 1 18000
+2104 2 18000
+2111 1 18000
+2112 2 18001
+2113 1 18000
+2114 2 18000
+3001 1 40000
+3002 2 40001
+3003 1 40000
+3004 2 40000
+3011 1 40000
+3012 2 40001
+3013 1 40000
+3014 2 40000
+3101 1 40000
+3102 2 40001
+3103 1 40000
+3104 2 40000
+3111 1 40000
+3112 2 40001
+3113 1 40000
+3114 2 40000
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+SELECT a, b, length(c) FROM t1 ORDER BY a;
+a b length(c)
+1 2 21
+2 0 20
+3 2 21
+11 2 21
+12 0 20
+13 2 21
+101 2 21
+102 0 20
+103 2 21
+111 2 21
+112 0 20
+113 2 21
+1001 2 1025
+1002 0 1024
+1003 2 1025
+1011 2 1025
+1012 0 1024
+1013 2 1025
+1101 2 1025
+1102 0 1024
+1103 2 1025
+1111 2 1025
+1112 0 1024
+1113 2 1025
+2001 2 18001
+2002 0 18000
+2003 2 18001
+2011 2 18001
+2012 0 18000
+2013 2 18001
+2101 2 18001
+2102 0 18000
+2103 2 18001
+2111 2 18001
+2112 0 18000
+2113 2 18001
+3001 2 40001
+3002 0 40000
+3003 2 40001
+3011 2 40001
+3012 0 40000
+3013 2 40001
+3101 2 40001
+3102 0 40000
+3103 2 40001
+3111 2 40001
+3112 0 40000
+3113 2 40001
+SELECT a, b, length(c) FROM t2 ORDER BY a;
+a b length(c)
+1 1 20
+2 2 21
+3 1 20
+4 2 20
+11 1 20
+12 2 21
+13 1 20
+14 2 20
+101 1 20
+102 2 21
+103 1 20
+104 2 20
+111 1 20
+112 2 21
+113 1 20
+114 2 20
+1001 1 1024
+1002 2 1025
+1003 1 1024
+1004 2 1024
+1011 1 1024
+1012 2 1025
+1013 1 1024
+1014 2 1024
+1101 1 1024
+1102 2 1025
+1103 1 1024
+1104 2 1024
+1111 1 1024
+1112 2 1025
+1113 1 1024
+1114 2 1024
+2001 1 18000
+2002 2 18001
+2003 1 18000
+2004 2 18000
+2011 1 18000
+2012 2 18001
+2013 1 18000
+2014 2 18000
+2101 1 18000
+2102 2 18001
+2103 1 18000
+2104 2 18000
+2111 1 18000
+2112 2 18001
+2113 1 18000
+2114 2 18000
+3001 1 40000
+3002 2 40001
+3003 1 40000
+3004 2 40000
+3011 1 40000
+3012 2 40001
+3013 1 40000
+3014 2 40000
+3101 1 40000
+3102 2 40001
+3103 1 40000
+3104 2 40000
+3111 1 40000
+3112 2 40001
+3113 1 40000
+3114 2 40000
+SELECT MASTER_POS_WAIT("binlog-000000.ibb", 4096);
+ERROR HY000: master_pos_wait() is not available when --binlog-storage-engine is enabled
+*** Test CREATE TABLE ... SELECT binlogging when it uses ROW mode.
+connection master;
+SET @old_max_size= @@GLOBAL.max_binlog_size;
+SET GLOBAL max_binlog_size= 1048576;
+FLUSH BINARY LOGS;
+FLUSH BINARY LOGS;
+ALTER TABLE t1 FORCE;
+SET binlog_format=MIXED;
+CREATE TABLE t3 AS SELECT * FROM t1 LIMIT 10;
+CREATE TABLE t4 AS SELECT * FROM t2 LIMIT 10;
+SET binlog_format=ROW;
+CREATE TABLE t5 AS SELECT * FROM t3;
+CREATE TABLE t6 AS SELECT * FROM t4;
+SET binlog_format=MIXED;
+CREATE TABLE t7 AS SELECT * FROM t3;
+CREATE TABLE t8 AS SELECT * FROM t4;
+show binlog events in 'binlog-000009.ibb' from ;
+Log_name Pos Event_type Server_id End_log_pos Info
+binlog-000009.ibb # Gtid 1 # BEGIN GTID #-#-#
+binlog-000009.ibb # Query 1 # use `test`; CREATE TABLE `t3` (
+ `a` int(11) NOT NULL,
+ `b` int(11) DEFAULT NULL,
+ `c` longtext DEFAULT NULL
+)
+binlog-000009.ibb # Annotate_rows 1 # CREATE TABLE t3 AS SELECT * FROM t1 LIMIT 10
+binlog-000009.ibb # Table_map 1 # table_id: # (test.t3)
+binlog-000009.ibb # Write_rows_v1 1 # table_id: # flags: STMT_END_F
+binlog-000009.ibb # Query 1 # COMMIT
+binlog-000009.ibb # Gtid 1 # BEGIN GTID #-#-#
+binlog-000009.ibb # Query 1 # use `test`; CREATE TABLE `t4` (
+ `a` int(11) NOT NULL,
+ `b` int(11) DEFAULT NULL,
+ `c` longtext DEFAULT NULL
+)
+binlog-000009.ibb # Annotate_rows 1 # CREATE TABLE t4 AS SELECT * FROM t2 LIMIT 10
+binlog-000009.ibb # Table_map 1 # table_id: # (test.t4)
+binlog-000009.ibb # Write_rows_v1 1 # table_id: # flags: STMT_END_F
+binlog-000009.ibb # Query 1 # COMMIT
+binlog-000009.ibb # Gtid 1 # BEGIN GTID #-#-#
+binlog-000009.ibb # Query 1 # use `test`; CREATE TABLE `t5` (
+ `a` int(11) NOT NULL,
+ `b` int(11) DEFAULT NULL,
+ `c` longtext DEFAULT NULL
+)
+binlog-000009.ibb # Annotate_rows 1 # CREATE TABLE t5 AS SELECT * FROM t3
+binlog-000009.ibb # Table_map 1 # table_id: # (test.t5)
+binlog-000009.ibb # Write_rows_v1 1 # table_id: # flags: STMT_END_F
+binlog-000009.ibb # Query 1 # COMMIT
+binlog-000009.ibb # Gtid 1 # BEGIN GTID #-#-#
+binlog-000009.ibb # Query 1 # use `test`; CREATE TABLE `t6` (
+ `a` int(11) NOT NULL,
+ `b` int(11) DEFAULT NULL,
+ `c` longtext DEFAULT NULL
+)
+binlog-000009.ibb # Annotate_rows 1 # CREATE TABLE t6 AS SELECT * FROM t4
+binlog-000009.ibb # Table_map 1 # table_id: # (test.t6)
+binlog-000009.ibb # Write_rows_v1 1 # table_id: # flags: STMT_END_F
+binlog-000009.ibb # Query 1 # COMMIT
+binlog-000009.ibb # Gtid 1 # GTID #-#-#
+binlog-000009.ibb # Query 1 # use `test`; CREATE TABLE t7 AS SELECT * FROM t3
+binlog-000009.ibb # Gtid 1 # GTID #-#-#
+binlog-000009.ibb # Query 1 # use `test`; CREATE TABLE t8 AS SELECT * FROM t4
+SET GLOBAL max_binlog_size= @old_max_size;
+DROP TABLE t3,t4,t5,t6,t7,t8;
+connection master;
+DROP TABLE t1, t2;
+CALL mtr.add_suppression('Statement is unsafe because it accesses a non-transactional table after accessing a transactional table');
+include/rpl_end.inc
diff --git a/mysql-test/suite/binlog_in_engine/nontrans.test b/mysql-test/suite/binlog_in_engine/nontrans.test
new file mode 100644
index 0000000000000..d9296318c4075
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/nontrans.test
@@ -0,0 +1,98 @@
+--source include/master-slave.inc
+--source include/have_binlog_format_mixed.inc
+--source include/have_innodb_binlog.inc
+
+CREATE TABLE t1(a INT PRIMARY KEY, b INT, c LONGTEXT) ENGINE=InnoDB;
+CREATE TABLE t2(a INT PRIMARY KEY, b INT, c LONGTEXT) ENGINE=Aria;
+
+--let $i= 0
+while ($i <= 3) {
+ if ($i == 0) {
+ SET @c= REPEAT('*', 20);
+ }
+ if ($i == 1) {
+ SET @c= REPEAT('%', 1024);
+ }
+ if ($i == 2) {
+ SET @c= REPEAT('.', 18000);
+ }
+ if ($i == 3) {
+ SET @c= REPEAT('.', 40000);
+ }
+
+ --let $f= 0
+ while ($f <= 1) {
+ if ($f == 0) {
+ SET SESSION binlog_format=statement;
+ }
+ if ($f == 1) {
+ SET SESSION binlog_format=row;
+ }
+
+ --let $s= 0
+ while ($s <= 1) {
+ --let $k = `SELECT $i*1000 + $f*100 + $s*10`
+ eval SET SESSION binlog_direct_non_transactional_updates= $s;
+
+ eval INSERT INTO t1 VALUES (1 + $k, 0, @c), (2 + $k, 0, @c), (3 + $k, 0, @c);
+ eval INSERT INTO t2 VALUES (1 + $k, 1, @c), (2 + $k, 1, @c), (3 + $k, 1, @c);
+
+ BEGIN;
+ eval UPDATE t1 SET b=2, c=CONCAT('!', c) WHERE a=1+$k;
+ eval UPDATE t2 SET b=2, c=CONCAT('!', c) WHERE a=2+$k;
+ eval UPDATE t1 SET b=2, c=CONCAT('!', c) WHERE a=3+$k;
+ eval INSERT INTO t2 VALUES (4 + $k, 2, @c);
+ COMMIT;
+ inc $s;
+ }
+ inc $f;
+ }
+ inc $i;
+}
+
+SELECT a, b, length(c) FROM t1 ORDER BY a;
+SELECT a, b, length(c) FROM t2 ORDER BY a;
+
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
+SELECT a, b, length(c) FROM t1 ORDER BY a;
+SELECT a, b, length(c) FROM t2 ORDER BY a;
+
+# Test that master_pos_wait() is disabled when master is binlog-in-engine.
+--error ER_NOT_AVAILABLE_WITH_ENGINE_BINLOG
+SELECT MASTER_POS_WAIT("binlog-000000.ibb", 4096);
+
+
+--echo *** Test CREATE TABLE ... SELECT binlogging when it uses ROW mode.
+--connection master
+# Rotate binlogs to a bigger size to make sure our SHOW BINLOG EVENTS output
+# fits in a single binlog file.
+SET @old_max_size= @@GLOBAL.max_binlog_size;
+SET GLOBAL max_binlog_size= 1048576;
+FLUSH BINARY LOGS;
+FLUSH BINARY LOGS;
+ALTER TABLE t1 FORCE;
+--let $binlog_file= query_get_value(SHOW MASTER STATUS, File, 1)
+--let $binlog_start= query_get_value(SHOW MASTER STATUS, Position, 1)
+SET binlog_format=MIXED;
+CREATE TABLE t3 AS SELECT * FROM t1 LIMIT 10;
+CREATE TABLE t4 AS SELECT * FROM t2 LIMIT 10;
+SET binlog_format=ROW;
+CREATE TABLE t5 AS SELECT * FROM t3;
+CREATE TABLE t6 AS SELECT * FROM t4;
+SET binlog_format=MIXED;
+CREATE TABLE t7 AS SELECT * FROM t3;
+CREATE TABLE t8 AS SELECT * FROM t4;
+--source include/show_binlog_events2.inc
+SET GLOBAL max_binlog_size= @old_max_size;
+
+DROP TABLE t3,t4,t5,t6,t7,t8;
+
+
+--connection master
+DROP TABLE t1, t2;
+
+CALL mtr.add_suppression('Statement is unsafe because it accesses a non-transactional table after accessing a transactional table');
+
+--source include/rpl_end.inc
diff --git a/mysql-test/suite/binlog_in_engine/not_implemented_yet.result b/mysql-test/suite/binlog_in_engine/not_implemented_yet.result
new file mode 100644
index 0000000000000..1228afe0aad67
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/not_implemented_yet.result
@@ -0,0 +1,23 @@
+SET GLOBAL rpl_semi_sync_master_enabled= 1;
+ERROR HY000: Semi-synchronous replication is not yet supported with --binlog-storage-engine
+SELECT BINLOG_GTID_POS("binlog-000000.ibb", 4096);
+ERROR HY000: BINLOG_GTID_POS() is not available when --binlog-storage-engine is enabled
+CREATE TABLE t (a INT) ENGINE=InnoDB;
+CREATE TRIGGER tr AFTER INSERT ON t FOR EACH ROW BEGIN SAVEPOINT A; DELETE FROM non_existing; END $
+START TRANSACTION;
+INSERT INTO t VALUES (1), (3);
+ERROR HY000: Using savepoints in triggers is not available when --binlog-storage-engine is enabled
+COMMIT;
+SELECT * FROM t ORDER BY a;
+a
+DROP TRIGGER tr;
+CREATE TRIGGER tr AFTER INSERT ON t FOR EACH ROW BEGIN ROLLBACK TO SAVEPOINT B; DELETE FROM non_existing; END $
+START TRANSACTION;
+SAVEPOINT B;
+INSERT INTO t VALUES (11), (13);
+ERROR HY000: Using savepoints in triggers is not available when --binlog-storage-engine is enabled
+COMMIT;
+SELECT * FROM t ORDER BY a;
+a
+DROP TRIGGER tr;
+DROP TABLE t;
diff --git a/mysql-test/suite/binlog_in_engine/not_implemented_yet.test b/mysql-test/suite/binlog_in_engine/not_implemented_yet.test
new file mode 100644
index 0000000000000..7351c36bc781d
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/not_implemented_yet.test
@@ -0,0 +1,34 @@
+--source include/have_binlog_format_mixed.inc
+--source include/have_innodb_binlog.inc
+
+--error ER_NOT_YET_SUPPORTED_ENGINE_BINLOG
+SET GLOBAL rpl_semi_sync_master_enabled= 1;
+
+# The BINLOG_GTID_POS() function is not available and will not be, we
+# want to get away from old problematic filename/offset coordinates.
+--error ER_NOT_AVAILABLE_WITH_ENGINE_BINLOG
+SELECT BINLOG_GTID_POS("binlog-000000.ibb", 4096);
+
+# MDEV-38465: Savepoint in trigger causes transactional inconsistency.
+CREATE TABLE t (a INT) ENGINE=InnoDB;
+--delimiter $
+CREATE TRIGGER tr AFTER INSERT ON t FOR EACH ROW BEGIN SAVEPOINT A; DELETE FROM non_existing; END $
+--delimiter ;
+START TRANSACTION;
+--error ER_NOT_AVAILABLE_WITH_ENGINE_BINLOG
+INSERT INTO t VALUES (1), (3);
+COMMIT;
+SELECT * FROM t ORDER BY a;
+DROP TRIGGER tr;
+
+--delimiter $
+CREATE TRIGGER tr AFTER INSERT ON t FOR EACH ROW BEGIN ROLLBACK TO SAVEPOINT B; DELETE FROM non_existing; END $
+--delimiter ;
+START TRANSACTION;
+SAVEPOINT B;
+--error ER_NOT_AVAILABLE_WITH_ENGINE_BINLOG
+INSERT INTO t VALUES (11), (13);
+COMMIT;
+SELECT * FROM t ORDER BY a;
+DROP TRIGGER tr;
+DROP TABLE t;
diff --git a/mysql-test/suite/binlog_in_engine/online_alter.result b/mysql-test/suite/binlog_in_engine/online_alter.result
new file mode 100644
index 0000000000000..fb3a1e3dff3fc
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/online_alter.result
@@ -0,0 +1,23 @@
+CREATE TABLE t1 (a INT PRIMARY KEY) ENGINE=InnoDB;
+INSERT INTO t1 VALUES (1), (2), (3), (4), (5);
+connect con1,localhost,root,,;
+SET debug_sync= 'alter_table_online_downgraded SIGNAL ready WAIT_FOR cont';
+ALTER TABLE t1 FORCE, ALGORITHM=COPY;
+connection default;
+SET debug_sync= 'now WAIT_FOR ready';
+connect con2,localhost,root,,;
+BEGIN;
+INSERT INTO t1 VALUES (6);
+connection default;
+SET debug_sync= 'now SIGNAL cont';
+connection con2;
+INSERT INTO t1 VALUES (7);
+COMMIT;
+INSERT INTO t1 VALUES (8);
+connection con1;
+SET debug_sync= 'RESET';
+connection con2;
+SELECT COUNT(*) FROM t1;
+COUNT(*)
+8
+DROP TABLE t1;
diff --git a/mysql-test/suite/binlog_in_engine/online_alter.test b/mysql-test/suite/binlog_in_engine/online_alter.test
new file mode 100644
index 0000000000000..e1e4339b76d20
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/online_alter.test
@@ -0,0 +1,35 @@
+--source include/have_binlog_format_mixed.inc
+--source include/have_debug_sync.inc
+--source include/have_innodb_binlog.inc
+
+CREATE TABLE t1 (a INT PRIMARY KEY) ENGINE=InnoDB;
+INSERT INTO t1 VALUES (1), (2), (3), (4), (5);
+
+--connect (con1,localhost,root,,)
+SET debug_sync= 'alter_table_online_downgraded SIGNAL ready WAIT_FOR cont';
+send ALTER TABLE t1 FORCE, ALGORITHM=COPY;
+
+--connection default
+SET debug_sync= 'now WAIT_FOR ready';
+
+--connect (con2,localhost,root,,)
+BEGIN;
+INSERT INTO t1 VALUES (6);
+
+--connection default
+SET debug_sync= 'now SIGNAL cont';
+
+--connection con2
+INSERT INTO t1 VALUES (7);
+COMMIT;
+send INSERT INTO t1 VALUES (8);
+
+--connection con1
+reap;
+SET debug_sync= 'RESET';
+
+--connection con2
+reap;
+
+SELECT COUNT(*) FROM t1;
+DROP TABLE t1;
diff --git a/mysql-test/suite/binlog_in_engine/purge_dump_thread-master.opt b/mysql-test/suite/binlog_in_engine/purge_dump_thread-master.opt
new file mode 100644
index 0000000000000..f139c2d921d3a
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/purge_dump_thread-master.opt
@@ -0,0 +1,5 @@
+--max-binlog-size=64k
+--innodb-binlog-state-interval=32k
+--log-bin
+--binlog-cache-size=8192
+--binlog-stmt-cache-size=8192
diff --git a/mysql-test/suite/binlog_in_engine/purge_dump_thread.result b/mysql-test/suite/binlog_in_engine/purge_dump_thread.result
new file mode 100644
index 0000000000000..93fc516af36bd
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/purge_dump_thread.result
@@ -0,0 +1,131 @@
+include/master-slave.inc
+[connection master]
+CREATE TABLE t1 (a INT NOT NULL, b INT NOT NULL, c TEXT, PRIMARY KEY(a, b)) ENGINE=InnoDB;
+INSERT INTO t1 VALUES (0, 0, 'Start');
+*** Test iteration, RESTART=0
+connection master;
+FLUSH BINARY LOGS;
+connection master1;
+*** Create a transaction with active OOB records (to be committed).
+connection default;
+*** Create a transaction with active OOB records (to be rolled back).
+connection master;
+FLUSH BINARY LOGS;
+*** Generating 10 large transactions in 5 interleaved connections
+connection master;
+FLUSH BINARY LOGS;
+INSERT INTO t1 VALUES (0 + 2, 0, "Park point 1 for dump thread");
+include/save_master_gtid.inc
+SELECT COUNT(*) FROM t1;
+COUNT(*)
+122
+connection slave;
+include/sync_with_master_gtid.inc
+SELECT COUNT(*) FROM t1;
+COUNT(*)
+122
+connection master;
+SET @old_dbug= @@global.debug_dbug;
+SET GLOBAL debug_dbug= "+d,dump_thread_wait_before_send_xid";
+INSERT INTO t1 VALUES (0 + 2, 1, "Transaction to pause dump thread");
+PURGE BINARY LOGS TO 'OOB1_START';
+PURGE BINARY LOGS TO 'OOB1_AFTER';
+ERROR HY000: A purgeable log is in use, will not purge
+PURGE BINARY LOGS TO 'OOB1_LATER';
+ERROR HY000: A purgeable log is in use, will not purge
+PURGE BINARY LOGS TO 'OOB1_CURRENT';
+ERROR HY000: A purgeable log is in use, will not purge
+connection master1;
+COMMIT;
+connection default;
+ROLLBACK;
+connection master;
+PURGE BINARY LOGS TO 'OOB1_CURRENT';
+ERROR HY000: A purgeable log is in use, will not purge
+*** Allow the dump thread to proceed, and see that purge is now possible.
+SET GLOBAL debug_dbug= @old_dbug;
+SET debug_sync= 'now SIGNAL signal.continue';
+FLUSH BINARY LOGS;
+INSERT INTO t1 VALUES (0 + 2, 2, 'Transaction to get dump thread to the next file');
+SELECT COUNT(*), SUM(a), SUM(b), SUM(LENGTH(c)) FROM t1;
+COUNT(*) SUM(a) SUM(b) SUM(LENGTH(c))
+134 120676 708 220262
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+SELECT COUNT(*), SUM(a), SUM(b), SUM(LENGTH(c)) FROM t1;
+COUNT(*) SUM(a) SUM(b) SUM(LENGTH(c))
+134 120676 708 220262
+connection master;
+SET debug_sync= 'RESET';
+PURGE BINARY LOGS TO 'OOB1_START';
+PURGE BINARY LOGS TO 'OOB1_AFTER';
+PURGE BINARY LOGS TO 'OOB1_LATER';
+PURGE BINARY LOGS TO 'OOB1_CURRENT';
+*** Test iteration, RESTART=1
+connection master;
+FLUSH BINARY LOGS;
+connection master1;
+*** Create a transaction with active OOB records (to be committed).
+connection default;
+*** Create a transaction with active OOB records (to be rolled back).
+connection master;
+FLUSH BINARY LOGS;
+*** Generating 10 large transactions in 5 interleaved connections
+connection master;
+FLUSH BINARY LOGS;
+INSERT INTO t1 VALUES (100000 + 2, 0, "Park point 1 for dump thread");
+include/save_master_gtid.inc
+SELECT COUNT(*) FROM t1;
+COUNT(*)
+255
+connection slave;
+include/sync_with_master_gtid.inc
+SELECT COUNT(*) FROM t1;
+COUNT(*)
+255
+connection slave;
+include/stop_slave.inc
+connection master1;
+COMMIT;
+connection default;
+ROLLBACK;
+connection master;
+include/rpl_restart_server.inc [server_number=1 parameters: --skip-slave-start]
+connection master;
+SET @old_dbug= @@global.debug_dbug;
+SET GLOBAL debug_dbug= "+d,dump_thread_wait_before_send_xid";
+INSERT INTO t1 VALUES (100000 + 2, 1, "Transaction to pause dump thread");
+connection slave;
+include/start_slave.inc
+connection master;
+PURGE BINARY LOGS TO 'OOB1_START';
+PURGE BINARY LOGS TO 'OOB1_AFTER';
+ERROR HY000: A purgeable log is in use, will not purge
+PURGE BINARY LOGS TO 'OOB1_LATER';
+ERROR HY000: A purgeable log is in use, will not purge
+PURGE BINARY LOGS TO 'OOB1_CURRENT';
+ERROR HY000: A purgeable log is in use, will not purge
+*** Allow the dump thread to proceed, and see that purge is now possible.
+SET GLOBAL debug_dbug= @old_dbug;
+SET debug_sync= 'now SIGNAL signal.continue';
+FLUSH BINARY LOGS;
+INSERT INTO t1 VALUES (100000 + 2, 2, 'Transaction to get dump thread to the next file');
+SELECT COUNT(*), SUM(a), SUM(b), SUM(LENGTH(c)) FROM t1;
+COUNT(*) SUM(a) SUM(b) SUM(LENGTH(c))
+267 13541352 1416 440519
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+SELECT COUNT(*), SUM(a), SUM(b), SUM(LENGTH(c)) FROM t1;
+COUNT(*) SUM(a) SUM(b) SUM(LENGTH(c))
+267 13541352 1416 440519
+connection master;
+SET debug_sync= 'RESET';
+PURGE BINARY LOGS TO 'OOB1_START';
+PURGE BINARY LOGS TO 'OOB1_AFTER';
+PURGE BINARY LOGS TO 'OOB1_LATER';
+PURGE BINARY LOGS TO 'OOB1_CURRENT';
+connection master;
+DROP TABLE t1;
+include/rpl_end.inc
diff --git a/mysql-test/suite/binlog_in_engine/purge_dump_thread.test b/mysql-test/suite/binlog_in_engine/purge_dump_thread.test
new file mode 100644
index 0000000000000..132b035774844
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/purge_dump_thread.test
@@ -0,0 +1,205 @@
+--source include/have_debug.inc
+--source include/have_debug_sync.inc
+--source include/have_binlog_format_row.inc
+--source include/master-slave.inc
+--source include/have_innodb_binlog.inc
+
+--let $NUM_CONNECTIONS= 5
+# $NUM_TRANSACTIONS is total, not per connection.
+--let $NUM_TRANSACTIONS=10
+--let $NUM_PIECES= 10
+--let $PIECE_SIZE= 2000
+
+# Test that PURGE BINARY LOGS avoids purging files containing OOB records
+# referenced from files that a dump thread is still active in.
+#
+# The test has --max-binlog-size=64k to have a larger number of binlog files
+# to test with. The --binlog-cache-size is set to 8k, so more event data than
+# that causes OOB binlogging.
+
+CREATE TABLE t1 (a INT NOT NULL, b INT NOT NULL, c TEXT, PRIMARY KEY(a, b)) ENGINE=InnoDB;
+INSERT INTO t1 VALUES (0, 0, 'Start');
+
+# Run twice. Once where the OOB references to earlier file numbers is kept
+# track of in-memory. And once, where server is restarted so the references
+# must be read from the file headers.
+--let $restart= 0
+while ($restart <= 1) {
+
+ --echo *** Test iteration, RESTART=$restart
+ --connection master
+ --let $D= `SELECT $restart*100000`
+ FLUSH BINARY LOGS;
+
+ # Start a transaction that will have OOB references in this specific binlog file.
+ --let $oob1_start= query_get_value(SHOW MASTER STATUS, File, 1)
+ --connection master1
+ --echo *** Create a transaction with active OOB records (to be committed).
+ --disable_query_log
+ BEGIN;
+ --let $i= 0
+ while ($i < 10) {
+ eval INSERT INTO t1 VALUES ($D+1, $i, REPEAT(CHR(65 + ($i MOD 26)), 2000));
+ inc $i;
+ }
+ --enable_query_log
+ # Leaving the transaction open, so the commit record will end up in a later
+ # binlog file and have a reference back that blocks purge.
+
+ # Also test an OOB record for a transaction that is later rolled back.
+ --connection default
+ --echo *** Create a transaction with active OOB records (to be rolled back).
+ --disable_query_log
+ BEGIN;
+ --let $i= 0
+ while ($i < 10) {
+ eval INSERT INTO t1 VALUES ($D+10, $i, REPEAT(CHR(65 + ($i MOD 26)), 2000));
+ inc $i;
+ }
+ --enable_query_log
+
+ --connection master
+ FLUSH BINARY LOGS;
+ --let $oob1_after= query_get_value(SHOW MASTER STATUS, File, 1)
+
+ # Generate a bunch of more transactions that contain OOB and flex the
+ # OOB refcounting.
+ --echo *** Generating $NUM_TRANSACTIONS large transactions in $NUM_CONNECTIONS interleaved connections
+ --disable_query_log
+ let $t= 0;
+ while ($t < $NUM_TRANSACTIONS) {
+ let $b= $t;
+ let $i= 1;
+ while ($i <= $NUM_CONNECTIONS) {
+ --connect(con$i,localhost,root,,)
+ START TRANSACTION;
+ eval INSERT INTO t1 VALUES ($D + 1000 + $b + $i, 0, 'Initial $i');
+ inc $i;
+ inc $t;
+ }
+
+ let $p= 1;
+ while ($p <= $NUM_PIECES) {
+ let $i= 1;
+ while ($i <= $NUM_CONNECTIONS) {
+ --connection con$i
+ eval INSERT INTO t1 VALUES ($D + 1000 + $b + $i, $p, REPEAT(CHR(65 + ($p + $i MOD 26)), $PIECE_SIZE));
+ inc $i;
+ }
+ inc $p;
+ }
+
+ let $i= 1;
+ while ($i <= $NUM_CONNECTIONS) {
+ --connection con$i
+ eval INSERT INTO t1 VALUES ($D + 1000 + $b + $i, $NUM_PIECES+1, 'Last $i');
+ COMMIT;
+ --disconnect con$i
+ inc $i;
+ }
+ }
+ --enable_query_log
+
+ --connection master
+ --let $oob1_later= query_get_value(SHOW MASTER STATUS, File, 1)
+ FLUSH BINARY LOGS;
+ eval INSERT INTO t1 VALUES ($D + 2, 0, "Park point 1 for dump thread");
+
+ # Now get the dump thread to the current point.
+ --source include/save_master_gtid.inc
+ SELECT COUNT(*) FROM t1;
+
+ --connection slave
+ --source include/sync_with_master_gtid.inc
+ SELECT COUNT(*) FROM t1;
+
+ if ($restart) {
+ --connection slave
+ --source include/stop_slave.inc
+
+ --connection master1
+ # Commit the transaction with OOB references back to an earlier binlog
+ # file, so that the reference will be there also after server restart.
+ COMMIT;
+ --connection default
+ # Roll back the other transaction with OOB.
+ ROLLBACK;
+
+ --connection master
+ --let $rpl_server_number=1
+ --let $rpl_server_parameters= --skip-slave-start
+ --source include/rpl_restart_server.inc
+ }
+
+ --connection master
+ SET @old_dbug= @@global.debug_dbug;
+ SET GLOBAL debug_dbug= "+d,dump_thread_wait_before_send_xid";
+ eval INSERT INTO t1 VALUES ($D + 2, 1, "Transaction to pause dump thread");
+ --let $oob1_current= query_get_value(SHOW MASTER STATUS, File, 1)
+
+ if ($restart) {
+ --connection slave
+ --source include/start_slave.inc
+ --connection master
+ }
+
+ let $wait_condition= SELECT COUNT(*) = 1 FROM INFORMATION_SCHEMA.PROCESSLIST
+ WHERE Command = 'Binlog Dump' AND State = 'debug sync point: now';
+ --source include/wait_condition.inc
+
+ # At this point, we have a dump thread active in $oob1_current. But we still
+ # have an active OOB record in $oob1_start, so neither of $oob1_start or
+ # any other prior to $oob1_current must be purged.
+ # The file before $oob1_start is allowed to be purged, though.
+ --replace_result $oob1_start OOB1_START
+ eval PURGE BINARY LOGS TO '$oob1_start';
+ --replace_result $oob1_after OOB1_AFTER
+ --error ER_LOG_IN_USE
+ eval PURGE BINARY LOGS TO '$oob1_after';
+ --replace_result $oob1_later OOB1_LATER
+ --error ER_LOG_IN_USE
+ eval PURGE BINARY LOGS TO '$oob1_later';
+ --replace_result $oob1_current OOB1_CURRENT
+ --error ER_LOG_IN_USE
+ eval PURGE BINARY LOGS TO '$oob1_current';
+ if (!$restart) {
+ --connection master1
+ COMMIT;
+ --connection default
+ ROLLBACK;
+ --connection master
+ --replace_result $oob1_current OOB1_CURRENT
+ --error ER_LOG_IN_USE
+ eval PURGE BINARY LOGS TO '$oob1_current';
+ }
+
+ --echo *** Allow the dump thread to proceed, and see that purge is now possible.
+ SET GLOBAL debug_dbug= @old_dbug;
+ SET debug_sync= 'now SIGNAL signal.continue';
+ FLUSH BINARY LOGS;
+ eval INSERT INTO t1 VALUES ($D + 2, 2, 'Transaction to get dump thread to the next file');
+ SELECT COUNT(*), SUM(a), SUM(b), SUM(LENGTH(c)) FROM t1;
+ --source include/save_master_gtid.inc
+ --connection slave
+ --source include/sync_with_master_gtid.inc
+ SELECT COUNT(*), SUM(a), SUM(b), SUM(LENGTH(c)) FROM t1;
+
+ --connection master
+ SET debug_sync= 'RESET';
+ # Now the dump thread is past $oob1_current, so all PURGE should succeed.
+ --replace_result $oob1_start OOB1_START
+ eval PURGE BINARY LOGS TO '$oob1_start';
+ --replace_result $oob1_after OOB1_AFTER
+ eval PURGE BINARY LOGS TO '$oob1_after';
+ --replace_result $oob1_later OOB1_LATER
+ eval PURGE BINARY LOGS TO '$oob1_later';
+ --replace_result $oob1_current OOB1_CURRENT
+ eval PURGE BINARY LOGS TO '$oob1_current';
+
+ inc $restart;
+}
+
+# Cleanup.
+--connection master
+DROP TABLE t1;
+--source include/rpl_end.inc
diff --git a/mysql-test/suite/binlog_in_engine/purge_locking.result b/mysql-test/suite/binlog_in_engine/purge_locking.result
new file mode 100644
index 0000000000000..a985ef25e86ef
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/purge_locking.result
@@ -0,0 +1,29 @@
+*** Test a binlog write running concurrently with a FLUSH BINARY LOGS.
+include/reset_master.inc
+CREATE TABLE t1(a INT PRIMARY KEY, b LONGBLOB) ENGINE=InnoDB;
+connect con1,localhost,root,,;
+SET SESSION debug_dbug= '+d,pause_binlog_write_after_release_page';
+INSERT INTO t1 VALUES(0, REPEAT('#', 20000));
+connection default;
+FLUSH NO_WRITE_TO_BINLOG BINARY LOGS;
+connection con1;
+include/show_binlog_events.inc
+Log_name Pos Event_type Server_id End_log_pos Info
+binlog-000000.ibb # Gtid # # GTID #-#-#
+binlog-000000.ibb # Query # # use `test`; CREATE TABLE t1(a INT PRIMARY KEY, b LONGBLOB) ENGINE=InnoDB
+binlog-000000.ibb # Gtid # # BEGIN GTID #-#-#
+binlog-000000.ibb # Annotate_rows # # INSERT INTO t1 VALUES(0, REPEAT('#', 20000))
+binlog-000000.ibb # Table_map # # table_id: # (test.t1)
+binlog-000000.ibb # Write_rows_v1 # # table_id: # flags: STMT_END_F
+binlog-000000.ibb # Xid # # COMMIT /* XID */
+*** Now the same thing for RESET MASTER.
+include/reset_master.inc
+connection con1;
+SET SESSION debug_dbug= '+d,pause_binlog_write_after_release_page';
+INSERT INTO t1 VALUES(1, REPEAT('#', 20000));
+connection default;
+RESET MASTER;
+connection con1;
+connection default;
+disconnect con1;
+DROP TABLE t1;
diff --git a/mysql-test/suite/binlog_in_engine/purge_locking.test b/mysql-test/suite/binlog_in_engine/purge_locking.test
new file mode 100644
index 0000000000000..19770bd7cc66a
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/purge_locking.test
@@ -0,0 +1,45 @@
+--source include/have_debug.inc
+--source include/have_binlog_format_row.inc
+--source include/have_innodb_binlog.inc
+
+--echo *** Test a binlog write running concurrently with a FLUSH BINARY LOGS.
+--source include/reset_master.inc
+CREATE TABLE t1(a INT PRIMARY KEY, b LONGBLOB) ENGINE=InnoDB;
+
+--connect con1,localhost,root,,
+# The dbug insertion injects a small sleep just after the binlog write has
+# written the first part of the record to the page and has released the page.
+# This helps the test trigger the condition where the FLUSH (which will be
+# blocked while the writer has a page latched) runs concurrently with the
+# write. A dbug_sync would not be effective here, since (as is usual for
+# race conditions) when the bug is fixed, the race can no longer occur and
+# the debug_sync would time out.
+SET SESSION debug_dbug= '+d,pause_binlog_write_after_release_page';
+send INSERT INTO t1 VALUES(0, REPEAT('#', 20000));
+
+--connection default
+--sleep 0.1
+FLUSH NO_WRITE_TO_BINLOG BINARY LOGS;
+
+--connection con1
+reap;
+--source include/show_binlog_events.inc
+
+--echo *** Now the same thing for RESET MASTER.
+--source include/reset_master.inc
+
+--connection con1
+SET SESSION debug_dbug= '+d,pause_binlog_write_after_release_page';
+send INSERT INTO t1 VALUES(1, REPEAT('#', 20000));
+
+--connection default
+--sleep 0.1
+RESET MASTER;
+
+--connection con1
+reap;
+
+# Clean up.
+--connection default
+--disconnect con1
+DROP TABLE t1;
diff --git a/mysql-test/suite/binlog_in_engine/purge_restart.result b/mysql-test/suite/binlog_in_engine/purge_restart.result
new file mode 100644
index 0000000000000..41a7c56ce65c3
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/purge_restart.result
@@ -0,0 +1,31 @@
+include/reset_master.inc
+SET GLOBAL slave_connections_needed_for_purge= 0;
+SET GLOBAL max_binlog_total_size= 128*1024;
+Warnings:
+Note 1375 Binary log 'binlog-000000.ibb' is not purged because the binlog file is in active use
+CREATE TABLE t1(a INT NOT NULL, b INT NOT NULL, c LONGBLOB, PRIMARY KEY(a, b)) ENGINE=InnoDB;
+INSERT INTO t1 VALUES (1, 0, REPEAT('x', 262144-8000));
+BEGIN;
+INSERT INTO t1 VALUES (2, 0, REPEAT('y', 40000));
+INSERT INTO t1 VALUES (2, 1, 'end');
+COMMIT;
+show binary logs;
+Log_name File_size
+binlog-000000.ibb #
+binlog-000001.ibb #
+binlog-000002.ibb #
+# restart
+SET @old_min_slaves= @@GLOBAL.slave_connections_needed_for_purge;
+SET GLOBAL slave_connections_needed_for_purge= 0;
+show binary logs;
+Log_name File_size
+binlog-000000.ibb #
+binlog-000001.ibb #
+binlog-000002.ibb #
+PURGE BINARY LOGS TO 'binlog-000001.ibb';
+show binary logs;
+Log_name File_size
+binlog-000001.ibb #
+binlog-000002.ibb #
+SET GLOBAL slave_connections_needed_for_purge= @old_min_slaves;
+DROP TABLE t1;
diff --git a/mysql-test/suite/binlog_in_engine/purge_restart.test b/mysql-test/suite/binlog_in_engine/purge_restart.test
new file mode 100644
index 0000000000000..9813dbc4a48ea
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/purge_restart.test
@@ -0,0 +1,45 @@
+--source include/have_binlog_format_row.inc
+--source include/have_innodb_binlog.inc
+
+--source include/reset_master.inc
+
+SET GLOBAL slave_connections_needed_for_purge= 0;
+
+# Set a very low value to force purge as early as possible.
+SET GLOBAL max_binlog_total_size= 128*1024;
+
+CREATE TABLE t1(a INT NOT NULL, b INT NOT NULL, c LONGBLOB, PRIMARY KEY(a, b)) ENGINE=InnoDB;
+
+# Test that refcounting works correctly when an OOB spill spans two binlog files.
+# First let's get the binlog position to shortly before the end of binlog-000000.ibb
+
+INSERT INTO t1 VALUES (1, 0, REPEAT('x', 262144-8000));
+
+# Now there are no outstanding oob references to binlog-000000.ibb.
+# Then spill a new OOB, which will span from binlog-000000.ibb into
+# binlog-000001.ibb. Then the header page of binlog-000001.ibb must
+# have oob reference file_no=0, not 1, as the _start_ of the new oob
+# is in file 0.
+
+BEGIN;
+INSERT INTO t1 VALUES (2, 0, REPEAT('y', 40000));
+INSERT INTO t1 VALUES (2, 1, 'end');
+COMMIT;
+
+--let $binlog_name= binlog-000002.ibb
+--let $binlog_size= 262144
+--source include/wait_for_engine_binlog.inc
+--source include/show_binary_logs.inc
+
+--source include/restart_mysqld.inc
+SET @old_min_slaves= @@GLOBAL.slave_connections_needed_for_purge;
+SET GLOBAL slave_connections_needed_for_purge= 0;
+
+--source include/show_binary_logs.inc
+PURGE BINARY LOGS TO 'binlog-000001.ibb';
+--source include/show_binary_logs.inc
+
+SET GLOBAL slave_connections_needed_for_purge= @old_min_slaves;
+# We do not need to restore max_binlog_total_size, as we restarted the server.
+
+DROP TABLE t1;
diff --git a/mysql-test/suite/binlog_in_engine/recover_concurrent_dml.result b/mysql-test/suite/binlog_in_engine/recover_concurrent_dml.result
new file mode 100644
index 0000000000000..ed6921957c9f8
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/recover_concurrent_dml.result
@@ -0,0 +1,129 @@
+include/master-slave.inc
+[connection master]
+connection master;
+CREATE TABLE t_strings(a INT NOT NULL, b INT NOT NULL, c VARCHAR(2048),
+PRIMARY KEY (a,b)) ENGINE=InnoDB;
+CREATE TABLE t_sum_lengths(a INT PRIMARY KEY, c INT, sum_len BIGINT) ENGINE=InnoDB;
+CREATE TABLE t_accounts(a INT PRIMARY KEY, balance INT) ENGINE=InnoDB;
+INSERT INTO t_accounts(a, balance) SELECT seq, 1000 FROM seq_1_to_500;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+include/stop_slave.inc
+SET STATEMENT sql_log_bin=0 FOR ALTER TABLE mysql.gtid_slave_pos ENGINE=InnoDB;
+SET @old_threads= @@GLOBAL.slave_parallel_threads;
+SET GLOBAL slave_parallel_threads= 10;
+*** Looping, killing server while running some parallel load...
+connection master;
+*** Loop: 1 ...
+connect con1,localhost,root,,;
+CALL gen_load1(10 + 1);;
+connect con2,localhost,root,,;
+CALL gen_load1(20 + 1);;
+connect con3,localhost,root,,;
+CALL gen_load1(30 + 1);;
+connect con4,localhost,root,,;
+CALL gen_load2(40 + 1);;
+connect con5,localhost,root,,;
+CALL gen_load2(50 + 1);;
+connection master;
+include/rpl_restart_server.inc [server_number=1]
+disconnect con1;
+disconnect con2;
+disconnect con3;
+disconnect con4;
+disconnect con5;
+*** Loop: 2 ...
+connect con1,localhost,root,,;
+CALL gen_load1(10 + 2);;
+connect con2,localhost,root,,;
+CALL gen_load1(20 + 2);;
+connect con3,localhost,root,,;
+CALL gen_load1(30 + 2);;
+connect con4,localhost,root,,;
+CALL gen_load2(40 + 2);;
+connect con5,localhost,root,,;
+CALL gen_load2(50 + 2);;
+connection master;
+include/rpl_restart_server.inc [server_number=1]
+disconnect con1;
+disconnect con2;
+disconnect con3;
+disconnect con4;
+disconnect con5;
+*** Loop: 3 ...
+connect con1,localhost,root,,;
+CALL gen_load1(10 + 3);;
+connect con2,localhost,root,,;
+CALL gen_load1(20 + 3);;
+connect con3,localhost,root,,;
+CALL gen_load1(30 + 3);;
+connect con4,localhost,root,,;
+CALL gen_load2(40 + 3);;
+connect con5,localhost,root,,;
+CALL gen_load2(50 + 3);;
+connection master;
+include/rpl_restart_server.inc [server_number=1]
+disconnect con1;
+disconnect con2;
+disconnect con3;
+disconnect con4;
+disconnect con5;
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/start_slave.inc
+include/sync_with_master_gtid.inc
+connection server_1;
+CREATE TEMPORARY TABLE t_check_len AS
+SELECT a, COUNT(b), SUM(LENGTH(c))
+FROM t_strings
+GROUP BY a;
+*** Must have no rows output from the two SELECT on server 1:
+(SELECT * FROM t_sum_lengths)
+EXCEPT
+(SELECT * FROM t_check_len);
+a c sum_len
+(SELECT * FROM t_check_len)
+EXCEPT
+(SELECT * FROM t_sum_lengths);
+a COUNT(b) SUM(LENGTH(c))
+*** Must have zero balance from the SELECT on server 1:
+SELECT SUM(balance) - 500 * 1000
+FROM t_accounts;
+SUM(balance) - 500 * 1000
+0
+connection server_2;
+CREATE TEMPORARY TABLE t_check_len AS
+SELECT a, COUNT(b), SUM(LENGTH(c))
+FROM t_strings
+GROUP BY a;
+*** Must have no rows output from the two SELECT on server 2:
+(SELECT * FROM t_sum_lengths)
+EXCEPT
+(SELECT * FROM t_check_len);
+a c sum_len
+(SELECT * FROM t_check_len)
+EXCEPT
+(SELECT * FROM t_sum_lengths);
+a COUNT(b) SUM(LENGTH(c))
+*** Must have zero balance from the SELECT on server 2:
+SELECT SUM(balance) - 500 * 1000
+FROM t_accounts;
+SUM(balance) - 500 * 1000
+0
+*** Check that slave data is identical to master
+include/diff_tables.inc [master:t_strings, slave:t_strings]
+include/diff_tables.inc [master:t_sum_lengths, slave:t_sum_lengths]
+include/diff_tables.inc [master:t_accounts, slave:t_accounts]
+connection slave;
+include/stop_slave.inc
+SET GLOBAL slave_parallel_threads= @old_threads;
+include/start_slave.inc
+connection master;
+DROP TABLE t_strings;
+DROP TABLE t_sum_lengths;
+DROP TABLE t_accounts;
+DROP PROCEDURE gen_load1;
+DROP PROCEDURE gen_load2;
+include/rpl_end.inc
diff --git a/mysql-test/suite/binlog_in_engine/recover_concurrent_dml.test b/mysql-test/suite/binlog_in_engine/recover_concurrent_dml.test
new file mode 100644
index 0000000000000..47f6da3a4d54b
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/recover_concurrent_dml.test
@@ -0,0 +1,153 @@
+--source include/not_embedded.inc
+# Test does a lot of queries that take a lot of CPU under Valgrind.
+--source include/big_test.inc
+--source include/not_valgrind.inc
+--source include/have_binlog_format_row.inc
+--source include/have_sequence.inc
+--source include/master-slave.inc
+--source include/have_innodb_binlog.inc
+
+--connection master
+CREATE TABLE t_strings(a INT NOT NULL, b INT NOT NULL, c VARCHAR(2048),
+ PRIMARY KEY (a,b)) ENGINE=InnoDB;
+CREATE TABLE t_sum_lengths(a INT PRIMARY KEY, c INT, sum_len BIGINT) ENGINE=InnoDB;
+--let $NUM_ACC= 500
+--let $INIT_AMOUNT= 1000
+CREATE TABLE t_accounts(a INT PRIMARY KEY, balance INT) ENGINE=InnoDB;
+eval INSERT INTO t_accounts(a, balance) SELECT seq, $INIT_AMOUNT FROM seq_1_to_$NUM_ACC;
+
+--disable_query_log
+--delimiter //
+CREATE PROCEDURE gen_load1(x INT)
+ MODIFIES SQL DATA
+ BEGIN
+ DECLARE i INT;
+ SET i = 0;
+ load_loop: LOOP
+ START TRANSACTION;
+ SET @val= REPEAT(@@SESSION.last_gtid, 1 + FLOOR(RAND() * 10));
+ INSERT INTO t_strings VALUES (x, i, @val);
+ INSERT INTO t_sum_lengths(a, c, sum_len) VALUES (x, 1, LENGTH(@val))
+ ON DUPLICATE KEY UPDATE c = c + 1, sum_len = sum_len + LENGTH(@val);
+ COMMIT;
+ SET i = i + 1;
+ END LOOP;
+ END
+//
+
+eval CREATE PROCEDURE gen_load2(x INT)
+ MODIFIES SQL DATA
+ BEGIN
+ DECLARE i INT;
+ SET i = 0;
+ load_loop: LOOP
+ START TRANSACTION;
+ SET @acc1= 1 + FLOOR(RAND() * $NUM_ACC);
+ SET @acc2= 1 + FLOOR(RAND() * $NUM_ACC);
+ SET @amount= 1 + FLOOR(RAND() * $INIT_AMOUNT / 10);
+ UPDATE t_accounts SET balance = balance - @amount WHERE a = @acc1;
+ UPDATE t_accounts SET balance = balance + @amount WHERE a = @acc2;
+ COMMIT;
+ SET i = i + 1;
+ END LOOP;
+ END
+//
+--delimiter ;
+--enable_query_log
+
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
+--source include/stop_slave.inc
+SET STATEMENT sql_log_bin=0 FOR ALTER TABLE mysql.gtid_slave_pos ENGINE=InnoDB;
+SET @old_threads= @@GLOBAL.slave_parallel_threads;
+SET GLOBAL slave_parallel_threads= 10;
+
+
+--echo *** Looping, killing server while running some parallel load...
+--connection master
+--let $loop= 1
+while ($loop <= 3) {
+ --echo *** Loop: $loop ...
+
+ --connect (con1,localhost,root,,)
+ --send_eval CALL gen_load1(10 + $loop);
+
+ --connect (con2,localhost,root,,)
+ --send_eval CALL gen_load1(20 + $loop);
+
+ --connect (con3,localhost,root,,)
+ --send_eval CALL gen_load1(30 + $loop);
+
+ --connect (con4,localhost,root,,)
+ --send_eval CALL gen_load2(40 + $loop);
+
+ --connect (con5,localhost,root,,)
+ --send_eval CALL gen_load2(50 + $loop);
+
+ --connection master
+ --sleep 0.8
+ --let $rpl_server_number= 1
+ --let $shutdown_timeout=0
+ --source include/rpl_restart_server.inc
+
+ --disconnect con1
+ --disconnect con2
+ --disconnect con3
+ --disconnect con4
+ --disconnect con5
+
+ inc $loop;
+}
+
+--connection master
+--source include/save_master_gtid.inc
+--connection slave
+--source include/start_slave.inc
+--source include/sync_with_master_gtid.inc
+
+# Check consistency, to verify that no transaction was
+# partially/incorrectly recovered.
+--let $i= 1
+while ($i <= 2) {
+ --connection server_$i
+ CREATE TEMPORARY TABLE t_check_len AS
+ SELECT a, COUNT(b), SUM(LENGTH(c))
+ FROM t_strings
+ GROUP BY a;
+ --echo *** Must have no rows output from the two SELECT on server $i:
+ (SELECT * FROM t_sum_lengths)
+ EXCEPT
+ (SELECT * FROM t_check_len);
+ (SELECT * FROM t_check_len)
+ EXCEPT
+ (SELECT * FROM t_sum_lengths);
+
+ --echo *** Must have zero balance from the SELECT on server $i:
+ eval SELECT SUM(balance) - $NUM_ACC * $INIT_AMOUNT
+ FROM t_accounts;
+
+ inc $i;
+}
+
+--echo *** Check that slave data is identical to master
+--let $diff_tables= master:t_strings, slave:t_strings
+--source include/diff_tables.inc
+--let $diff_tables= master:t_sum_lengths, slave:t_sum_lengths
+--source include/diff_tables.inc
+--let $diff_tables= master:t_accounts, slave:t_accounts
+--source include/diff_tables.inc
+
+# Clean up.
+--connection slave
+--source include/stop_slave.inc
+SET GLOBAL slave_parallel_threads= @old_threads;
+--source include/start_slave.inc
+
+--connection master
+DROP TABLE t_strings;
+DROP TABLE t_sum_lengths;
+DROP TABLE t_accounts;
+DROP PROCEDURE gen_load1;
+DROP PROCEDURE gen_load2;
+--source include/rpl_end.inc
diff --git a/mysql-test/suite/binlog_in_engine/recovery.opt b/mysql-test/suite/binlog_in_engine/recovery.opt
new file mode 100644
index 0000000000000..99097c033d34a
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/recovery.opt
@@ -0,0 +1 @@
+--loose-skip-stack-trace --skip-core-file --max-binlog-size=128K
diff --git a/mysql-test/suite/binlog_in_engine/recovery.result b/mysql-test/suite/binlog_in_engine/recovery.result
new file mode 100644
index 0000000000000..fb149d6910538
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/recovery.result
@@ -0,0 +1,57 @@
+include/reset_master.inc
+CREATE TABLE t1 (a INT PRIMARY KEY) ENGINE=InnoDB;
+INSERT INTO t1 VALUES (1);
+
+# Flush all dirty pages from buffer pool
+SET @no_checkpoint_save_pct= @@GLOBAL.innodb_max_dirty_pages_pct;
+SET @no_checkpoint_save_pct_lwm= @@GLOBAL.innodb_max_dirty_pages_pct_lwm;
+SET GLOBAL innodb_max_dirty_pages_pct_lwm=0.0;
+SET GLOBAL innodb_max_dirty_pages_pct=0.0;
+SET GLOBAL innodb_max_dirty_pages_pct= @no_checkpoint_save_pct;
+SET GLOBAL innodb_max_dirty_pages_pct_lwm= @no_checkpoint_save_pct_lwm;
+
+BEGIN;
+INSERT INTO t1 VALUES (2);
+INSERT INTO t1 VALUES (3);
+COMMIT;
+INSERT INTO t1 VALUES (4);
+INSERT INTO t1 VALUES (5);
+INSERT INTO t1 VALUES (6);
+INSERT INTO t1 VALUES (7);
+SELECT * FROM t1 ORDER BY a;
+a
+1
+2
+3
+4
+5
+6
+7
+SET SESSION debug_dbug="+d,crash_dispatch_command_before";
+SELECT 1;
+Got one of the listed errors
+include/show_binlog_events.inc
+Log_name Pos Event_type Server_id End_log_pos Info
+binlog-000000.ibb # Gtid # # GTID #-#-#
+binlog-000000.ibb # Query # # use `test`; CREATE TABLE t1 (a INT PRIMARY KEY) ENGINE=InnoDB
+binlog-000000.ibb # Gtid # # BEGIN GTID #-#-#
+binlog-000000.ibb # Query # # use `test`; INSERT INTO t1 VALUES (1)
+binlog-000000.ibb # Xid # # COMMIT /* XID */
+binlog-000000.ibb # Gtid # # BEGIN GTID #-#-#
+binlog-000000.ibb # Query # # use `test`; INSERT INTO t1 VALUES (2)
+binlog-000000.ibb # Query # # use `test`; INSERT INTO t1 VALUES (3)
+binlog-000000.ibb # Xid # # COMMIT /* XID */
+binlog-000000.ibb # Gtid # # BEGIN GTID #-#-#
+binlog-000000.ibb # Query # # use `test`; INSERT INTO t1 VALUES (4)
+binlog-000000.ibb # Xid # # COMMIT /* XID */
+binlog-000000.ibb # Gtid # # BEGIN GTID #-#-#
+binlog-000000.ibb # Query # # use `test`; INSERT INTO t1 VALUES (5)
+binlog-000000.ibb # Xid # # COMMIT /* XID */
+binlog-000000.ibb # Gtid # # BEGIN GTID #-#-#
+binlog-000000.ibb # Query # # use `test`; INSERT INTO t1 VALUES (6)
+binlog-000000.ibb # Xid # # COMMIT /* XID */
+binlog-000000.ibb # Gtid # # BEGIN GTID #-#-#
+binlog-000000.ibb # Query # # use `test`; INSERT INTO t1 VALUES (7)
+binlog-000000.ibb # Xid # # COMMIT /* XID */
+binlog-000000.ibb # Format_desc # # SERVER_VERSION, BINLOG_VERSION
+DROP TABLE t1;
diff --git a/mysql-test/suite/binlog_in_engine/recovery.test b/mysql-test/suite/binlog_in_engine/recovery.test
new file mode 100644
index 0000000000000..5d9a286501359
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/recovery.test
@@ -0,0 +1,62 @@
+--source include/not_embedded.inc
+--source include/not_valgrind.inc
+--source include/have_debug.inc
+--source include/have_binlog_format_mixed.inc
+--source include/have_innodb_binlog.inc
+
+--let $datadir= `SELECT @@datadir`
+--source include/reset_master.inc
+
+CREATE TABLE t1 (a INT PRIMARY KEY) ENGINE=InnoDB;
+INSERT INTO t1 VALUES (1);
+
+--let $no_checkpoint_flush= 1
+--let no_checkpoint_kill= 1
+--source ../../suite/innodb/include/no_checkpoint_start.inc
+--let $file= query_get_value(SHOW MASTER STATUS, File, 1)
+--let $pos= query_get_value(SHOW MASTER STATUS, Position, 1)
+
+
+BEGIN;
+INSERT INTO t1 VALUES (2);
+INSERT INTO t1 VALUES (3);
+COMMIT;
+INSERT INTO t1 VALUES (4);
+INSERT INTO t1 VALUES (5);
+INSERT INTO t1 VALUES (6);
+INSERT INTO t1 VALUES (7);
+SELECT * FROM t1 ORDER BY a;
+
+# Crash the server
+--write_file $MYSQLTEST_VARDIR/tmp/mysqld.1.expect
+wait-recovery.test
+EOF
+
+SET SESSION debug_dbug="+d,crash_dispatch_command_before";
+--error 2006,2013
+SELECT 1;
+--source include/wait_until_disconnected.inc
+
+--source ../../suite/innodb/include/no_checkpoint_end.inc
+
+# Overwrite the binlog file past the last checkpoint
+--let BINLOG_FILE= $datadir/$file
+--let OFFSET= $pos
+--let LEN=5000
+perl;
+open F, '+<', $ENV{BINLOG_FILE} or die $!;
+sysseek F, $ENV{OFFSET}, 0 or die $!;
+my $x= chr(0) x $ENV{LEN};
+syswrite F, $x, $ENV{LEN} or die $!;
+EOF
+
+--append_file $MYSQLTEST_VARDIR/tmp/mysqld.1.expect
+restart
+EOF
+--enable_reconnect
+--source include/wait_until_connected_again.inc
+
+--let $binlog_file=
+--let $binlog_start= 4
+--source include/show_binlog_events.inc
+DROP TABLE t1;
diff --git a/mysql-test/suite/binlog_in_engine/recovery_large.opt b/mysql-test/suite/binlog_in_engine/recovery_large.opt
new file mode 100644
index 0000000000000..915f0638bd214
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/recovery_large.opt
@@ -0,0 +1 @@
+--loose-skip-stack-trace --skip-core-file --max-binlog-size=64K --innodb-log-file-size=64M
diff --git a/mysql-test/suite/binlog_in_engine/recovery_large.result b/mysql-test/suite/binlog_in_engine/recovery_large.result
new file mode 100644
index 0000000000000..3f19651c9cd1e
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/recovery_large.result
@@ -0,0 +1,45 @@
+include/master-slave.inc
+[connection master]
+connection slave;
+include/stop_slave.inc
+SET GLOBAL gtid_slave_pos= '';
+connection master;
+include/reset_master.inc
+DROP TABLE IF EXISTS t1, t2;
+Warnings:
+Note 1051 Unknown table 'test.t1,test.t2'
+CREATE TABLE t1 (a INT PRIMARY KEY) ENGINE=InnoDB;
+CREATE TABLE t2 (a INT PRIMARY KEY AUTO_INCREMENT, b INT, c VARCHAR(4000)) ENGINE=InnoDB;
+INSERT INTO t1 VALUES (1);
+
+# Flush all dirty pages from buffer pool
+SET @no_checkpoint_save_pct= @@GLOBAL.innodb_max_dirty_pages_pct;
+SET @no_checkpoint_save_pct_lwm= @@GLOBAL.innodb_max_dirty_pages_pct_lwm;
+SET GLOBAL innodb_max_dirty_pages_pct_lwm=0.0;
+SET GLOBAL innodb_max_dirty_pages_pct=0.0;
+SET GLOBAL innodb_max_dirty_pages_pct= @no_checkpoint_save_pct;
+SET GLOBAL innodb_max_dirty_pages_pct_lwm= @no_checkpoint_save_pct_lwm;
+
+CHECKSUM TABLE t1, t2;
+Table Checksum
+test.t1 659068978
+test.t2 4017444020
+SET SESSION debug_dbug="+d,crash_dispatch_command_before";
+SELECT 1;
+Got one of the listed errors
+connection default;
+HULU
+BULU
+connection server_1;
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/start_slave.inc
+include/sync_with_master_gtid.inc
+CHECKSUM TABLE t1, t2;
+Table Checksum
+test.t1 659068978
+test.t2 4017444020
+connection master;
+DROP TABLE t1, t2;
+include/rpl_end.inc
diff --git a/mysql-test/suite/binlog_in_engine/recovery_large.test b/mysql-test/suite/binlog_in_engine/recovery_large.test
new file mode 100644
index 0000000000000..33ba8f65af8f0
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/recovery_large.test
@@ -0,0 +1,120 @@
+--source include/not_embedded.inc
+--source include/not_valgrind.inc
+--source include/have_debug.inc
+--source include/have_binlog_format_row.inc
+--source include/master-slave.inc
+--source include/have_innodb_binlog.inc
+
+--let $pre_count= 4
+--let $mid_count= 55
+
+--connection slave
+--source include/stop_slave.inc
+SET GLOBAL gtid_slave_pos= '';
+
+--connection master
+--let $datadir= `SELECT @@datadir`
+--source include/reset_master.inc
+
+DROP TABLE IF EXISTS t1, t2;
+CREATE TABLE t1 (a INT PRIMARY KEY) ENGINE=InnoDB;
+CREATE TABLE t2 (a INT PRIMARY KEY AUTO_INCREMENT, b INT, c VARCHAR(4000)) ENGINE=InnoDB;
+INSERT INTO t1 VALUES (1);
+
+--disable_query_log
+--let $i= 0
+while ($i < $pre_count) {
+ eval INSERT INTO t1 VALUES (100+$i);
+ eval INSERT INTO t2(b, c) VALUES ($i, repeat(chr(65 + ($i MOD 27)), 3500));
+ inc $i;
+}
+--enable_query_log
+
+--let $no_checkpoint_flush= 1
+--let $no_checkpoint_kill= 1
+--source ../../suite/innodb/include/no_checkpoint_start.inc
+--let $file= query_get_value(SHOW MASTER STATUS, File, 1)
+--let $pos= query_get_value(SHOW MASTER STATUS, Position, 1)
+
+
+--disable_query_log
+--let $i= 0
+while ($i < $mid_count) {
+ eval INSERT INTO t1 VALUES (1000+$i);
+ eval INSERT INTO t2(b, c) VALUES ($i, repeat(chr(65 + ($i MOD 27)), 3500));
+ inc $i;
+}
+--enable_query_log
+
+CHECKSUM TABLE t1, t2;
+
+# Crash the server
+# Set $_expect_file_name for no_checkpoint_end.inc which uses start_mysqld.inc
+--let $_expect_file_name= $MYSQLTEST_VARDIR/tmp/mysqld.1.expect
+--write_file $_expect_file_name
+wait-recovery.test
+EOF
+
+SET SESSION debug_dbug="+d,crash_dispatch_command_before";
+--error 2006,2013
+SELECT 1;
+--source include/wait_until_disconnected.inc
+--connection default
+--source include/wait_until_disconnected.inc
+
+--source ../../suite/innodb/include/no_checkpoint_end.inc
+
+--echo HULU
+
+# Overwrite the binlog file past the last checkpoint
+# But only in the two most recent files, the prior ones are durably flushed
+# and will not be recovered.
+--let BINLOG_FILE= $datadir/$file
+--let OFFSET= $pos
+--let LEN=5000
+perl;
+my $orig_file= $ENV{BINLOG_FILE};
+my $orig_pos= $ENV{OFFSET};
+$orig_file =~ m/(.*binlog-)([0-9]{6})\.ibb/ or die "Unexpected file name $file";
+my ($base, $seq)= ($1, $2);
+
+for (;;) {
+ my $file= $base . sprintf("%06d", $seq) . ".ibb";
+ last unless -f $file;
+ my $file_plus_2= $base . sprintf("%06d", $seq+2) . ".ibb";
+ ++$seq;
+ next if -f $file_plus_2;
+
+ my $pos= $file eq $orig_file ? $orig_pos : 0;
+ open F, '+<', $file or die $!;
+ sysseek F, $pos, 0 or die $!;
+ my $x= chr(0) x $ENV{LEN};
+ syswrite F, $x, $ENV{LEN} or die $!;
+}
+EOF
+
+--echo BULU
+
+--append_file $MYSQLTEST_VARDIR/tmp/mysqld.1.expect
+restart
+EOF
+--enable_reconnect
+--source include/wait_until_connected_again.inc
+--connection server_1
+--enable_reconnect
+--source include/wait_until_connected_again.inc
+--connection master
+--enable_reconnect
+--source include/wait_until_connected_again.inc
+
+--source include/save_master_gtid.inc
+
+--connection slave
+--source include/start_slave.inc
+--source include/sync_with_master_gtid.inc
+CHECKSUM TABLE t1, t2;
+
+--connection master
+DROP TABLE t1, t2;
+
+--source include/rpl_end.inc
diff --git a/mysql-test/suite/binlog_in_engine/rocksdb_internal_2pc.result b/mysql-test/suite/binlog_in_engine/rocksdb_internal_2pc.result
new file mode 100644
index 0000000000000..64d07a59ab23f
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/rocksdb_internal_2pc.result
@@ -0,0 +1,70 @@
+include/reset_master.inc
+CREATE TABLE t1 (a INT PRIMARY KEY, b INT) ENGINE=InnoDB;
+CREATE TABLE t2 (a INT PRIMARY KEY, b INT) ENGINE=RocksDB;
+*** A simple multi-engine transaction.
+BEGIN;
+INSERT INTO t1 VALUES (1, 0);
+INSERT INTO t2 VALUES (1, 0);
+COMMIT;
+include/show_binlog_events.inc
+Log_name Pos Event_type Server_id End_log_pos Info
+binlog-000000.ibb # Gtid # # BEGIN GTID #-#-#
+binlog-000000.ibb # Query # # use `test`; INSERT INTO t1 VALUES (1, 0)
+binlog-000000.ibb # Annotate_rows # # INSERT INTO t2 VALUES (1, 0)
+binlog-000000.ibb # Table_map # # table_id: # (test.t2)
+binlog-000000.ibb # Write_rows_v1 # # table_id: # flags: STMT_END_F
+binlog-000000.ibb # Xid # # COMMIT /* XID */
+*** A couple multi-engine transactions where we crash in the middle.
+connect con1,localhost,root,,;
+BEGIN;
+INSERT INTO t2 VALUES (2, 0);
+INSERT INTO t2 VALUES (20, 1);
+INSERT INTO t2 VALUES (21, 2);
+SET debug_sync= 'ibb_after_commit_redo_log SIGNAL con1_rdy WAIT_FOR crash';
+/* a */ COMMIT;
+connection default;
+SET debug_sync= 'now WAIT_FOR con1_rdy';
+connect con2,localhost,root,,;
+BEGIN;
+INSERT INTO t1 VALUES (3, 0);
+INSERT INTO t2 VALUES (3, 0);
+SET debug_sync= 'ibb_after_group_commit_redo_log SIGNAL con2_rdy WAIT_FOR crash';
+/* b */ COMMIT;
+connection default;
+SET debug_sync= 'now WAIT_FOR con2_rdy';
+# restart
+disconnect con1;
+disconnect con2;
+connection default;
+include/show_binlog_events.inc
+Log_name Pos Event_type Server_id End_log_pos Info
+binlog-000000.ibb # Gtid # # BEGIN GTID #-#-#
+binlog-000000.ibb # Annotate_rows # # INSERT INTO t2 VALUES (2, 0)
+binlog-000000.ibb # Table_map # # table_id: # (test.t2)
+binlog-000000.ibb # Write_rows_v1 # # table_id: # flags: STMT_END_F
+binlog-000000.ibb # Annotate_rows # # INSERT INTO t2 VALUES (20, 1)
+binlog-000000.ibb # Table_map # # table_id: # (test.t2)
+binlog-000000.ibb # Write_rows_v1 # # table_id: # flags: STMT_END_F
+binlog-000000.ibb # Annotate_rows # # INSERT INTO t2 VALUES (21, 2)
+binlog-000000.ibb # Table_map # # table_id: # (test.t2)
+binlog-000000.ibb # Write_rows_v1 # # table_id: # flags: STMT_END_F
+binlog-000000.ibb # Xid # # COMMIT /* XID */
+binlog-000000.ibb # Gtid # # BEGIN GTID #-#-#
+binlog-000000.ibb # Query # # use `test`; INSERT INTO t1 VALUES (3, 0)
+binlog-000000.ibb # Annotate_rows # # INSERT INTO t2 VALUES (3, 0)
+binlog-000000.ibb # Table_map # # table_id: # (test.t2)
+binlog-000000.ibb # Write_rows_v1 # # table_id: # flags: STMT_END_F
+binlog-000000.ibb # Xid # # COMMIT /* XID */
+binlog-000000.ibb # Format_desc # # SERVER_VERSION, BINLOG_VERSION
+SELECT * FROM t1 ORDER BY a;
+a b
+1 0
+3 0
+SELECT * FROM t2 ORDER BY a;
+a b
+1 0
+2 0
+3 0
+20 1
+21 2
+DROP TABLE t1, t2;
diff --git a/mysql-test/suite/binlog_in_engine/rocksdb_internal_2pc.test b/mysql-test/suite/binlog_in_engine/rocksdb_internal_2pc.test
new file mode 100644
index 0000000000000..d55bea321488a
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/rocksdb_internal_2pc.test
@@ -0,0 +1,56 @@
+--source include/have_debug_sync.inc
+--source include/have_rocksdb.inc
+--source include/have_binlog_format_mixed.inc
+--source include/have_innodb_binlog.inc
+
+--source include/reset_master.inc
+
+CREATE TABLE t1 (a INT PRIMARY KEY, b INT) ENGINE=InnoDB;
+CREATE TABLE t2 (a INT PRIMARY KEY, b INT) ENGINE=RocksDB;
+
+--echo *** A simple multi-engine transaction.
+--let $binlog_file= query_get_value(SHOW MASTER STATUS, File, 1)
+--let $binlog_start= query_get_value(SHOW MASTER STATUS, Position, 1)
+BEGIN;
+INSERT INTO t1 VALUES (1, 0);
+INSERT INTO t2 VALUES (1, 0);
+COMMIT;
+--source include/show_binlog_events.inc
+
+--echo *** A couple multi-engine transactions where we crash in the middle.
+--connect con1,localhost,root,,
+--let $binlog_start= query_get_value(SHOW MASTER STATUS, Position, 1)
+BEGIN;
+INSERT INTO t2 VALUES (2, 0);
+INSERT INTO t2 VALUES (20, 1);
+INSERT INTO t2 VALUES (21, 2);
+SET debug_sync= 'ibb_after_commit_redo_log SIGNAL con1_rdy WAIT_FOR crash';
+send /* a */ COMMIT;
+
+--connection default
+SET debug_sync= 'now WAIT_FOR con1_rdy';
+
+--connect con2,localhost,root,,
+BEGIN;
+INSERT INTO t1 VALUES (3, 0);
+INSERT INTO t2 VALUES (3, 0);
+SET debug_sync= 'ibb_after_group_commit_redo_log SIGNAL con2_rdy WAIT_FOR crash';
+send /* b */ COMMIT;
+
+--connection default
+SET debug_sync= 'now WAIT_FOR con2_rdy';
+
+--let $shutdown_timeout=0
+--source include/restart_mysqld.inc
+
+--disconnect con1
+--disconnect con2
+
+--connection default
+
+--source include/show_binlog_events.inc
+SELECT * FROM t1 ORDER BY a;
+SELECT * FROM t2 ORDER BY a;
+
+
+DROP TABLE t1, t2;
diff --git a/mysql-test/suite/binlog_in_engine/rpl_ddl_misf-master.opt b/mysql-test/suite/binlog_in_engine/rpl_ddl_misf-master.opt
new file mode 100644
index 0000000000000..ea0341e94e904
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/rpl_ddl_misf-master.opt
@@ -0,0 +1 @@
+--log-bin --binlog-storage-engine=innodb
diff --git a/mysql-test/suite/binlog_in_engine/rpl_ddl_misf-slave.opt b/mysql-test/suite/binlog_in_engine/rpl_ddl_misf-slave.opt
new file mode 100644
index 0000000000000..d814b853f05f8
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/rpl_ddl_misf-slave.opt
@@ -0,0 +1 @@
+--log-bin=slave-bin --loose-skip-innodb --binlog-storage-engine=
diff --git a/mysql-test/suite/binlog_in_engine/rpl_ddl_misf.result b/mysql-test/suite/binlog_in_engine/rpl_ddl_misf.result
new file mode 100644
index 0000000000000..7038ea8250eab
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/rpl_ddl_misf.result
@@ -0,0 +1,1765 @@
+include/master-slave.inc
+[connection master]
+set global sql_mode='';
+set local sql_mode='';
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection master;
+SET AUTOCOMMIT = 1;
+CREATE DATABASE mysqltest1;
+CREATE DATABASE mysqltest2;
+CREATE TABLE mysqltest1.t1 (f1 BIGINT) ENGINE=InnoDB;
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection master;
+INSERT INTO mysqltest1.t1 SET f1= 0;
+CREATE TABLE mysqltest1.t2 (f1 BIGINT) ENGINE=InnoDB;
+CREATE TABLE mysqltest1.t3 (f1 BIGINT) ENGINE=InnoDB;
+CREATE TABLE mysqltest1.t4 (f1 BIGINT) ENGINE=InnoDB;
+CREATE TABLE mysqltest1.t5 (f1 BIGINT) ENGINE=InnoDB;
+CREATE TABLE mysqltest1.t6 (f1 BIGINT) ENGINE=InnoDB;
+CREATE INDEX my_idx6 ON mysqltest1.t6(f1);
+CREATE TABLE mysqltest1.t7 (f1 BIGINT) ENGINE=InnoDB;
+INSERT INTO mysqltest1.t7 SET f1= 0;
+CREATE TABLE mysqltest1.t8 (f1 BIGINT) ENGINE=InnoDB;
+CREATE TABLE mysqltest1.t9 (f1 BIGINT) ENGINE=InnoDB;
+CREATE TABLE mysqltest1.t10 (f1 BIGINT) ENGINE=InnoDB;
+CREATE TABLE mysqltest1.t11 (f1 BIGINT) ENGINE=InnoDB;
+CREATE TABLE mysqltest1.t12 (f1 BIGINT) ENGINE=InnoDB;
+CREATE TABLE mysqltest1.t13 (f1 BIGINT) ENGINE=InnoDB;
+CREATE TABLE mysqltest1.t14 (f1 BIGINT) ENGINE=InnoDB;
+CREATE TABLE mysqltest1.t15 (f1 BIGINT) ENGINE=InnoDB;
+CREATE TABLE mysqltest1.t16 (f1 BIGINT) ENGINE=InnoDB;
+CREATE TABLE mysqltest1.t17 (f1 BIGINT) ENGINE=InnoDB;
+CREATE TABLE mysqltest1.t18 (f1 BIGINT) ENGINE=InnoDB;
+CREATE TABLE mysqltest1.t19 (f1 BIGINT) ENGINE=InnoDB;
+CREATE TEMPORARY TABLE mysqltest1.t23 (f1 BIGINT) ENGINE=MEMORY;
+SET AUTOCOMMIT = 0;
+use mysqltest1;
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SET AUTOCOMMIT = 1;
+use mysqltest1;
+connection master;
+
+######## SELECT 1 ########
+connection master;
+INSERT INTO t1 SET f1= 0 + 1;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+1
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+0
+connection master;
+SELECT 1;
+1
+1
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+1
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+0
+connection master;
+ROLLBACK;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+0
+
+TEST-INFO: MASTER: The INSERT is not committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+0
+
+TEST-INFO: SLAVE: The INSERT is not committed (Succeeded)
+connection master;
+
+######## SELECT COUNT(*) FROM t1 ########
+connection master;
+INSERT INTO t1 SET f1= 0 + 1;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+1
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+0
+connection master;
+SELECT COUNT(*) FROM t1;
+COUNT(*)
+2
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+1
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+0
+connection master;
+ROLLBACK;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+0
+
+TEST-INFO: MASTER: The INSERT is not committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+0
+
+TEST-INFO: SLAVE: The INSERT is not committed (Succeeded)
+connection master;
+
+######## COMMIT ########
+connection master;
+INSERT INTO t1 SET f1= 0 + 1;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+1
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+0
+connection master;
+COMMIT;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+1
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+1
+connection master;
+ROLLBACK;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+1
+
+TEST-INFO: MASTER: The INSERT is committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+1
+
+TEST-INFO: SLAVE: The INSERT is committed (Succeeded)
+connection master;
+
+######## ROLLBACK ########
+connection master;
+INSERT INTO t1 SET f1= 1 + 1;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+2
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+1
+connection master;
+ROLLBACK;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+1
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+1
+connection master;
+ROLLBACK;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+1
+
+TEST-INFO: MASTER: The INSERT is not committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+1
+
+TEST-INFO: SLAVE: The INSERT is not committed (Succeeded)
+connection master;
+
+######## SET AUTOCOMMIT=1 ########
+connection master;
+INSERT INTO t1 SET f1= 1 + 1;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+2
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+1
+connection master;
+SET AUTOCOMMIT=1;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+2
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+2
+connection master;
+ROLLBACK;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+2
+
+TEST-INFO: MASTER: The INSERT is committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+2
+
+TEST-INFO: SLAVE: The INSERT is committed (Succeeded)
+connection master;
+SET AUTOCOMMIT=0;
+
+######## START TRANSACTION ########
+connection master;
+INSERT INTO t1 SET f1= 2 + 1;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+3
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+2
+connection master;
+START TRANSACTION;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+3
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+3
+connection master;
+ROLLBACK;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+3
+
+TEST-INFO: MASTER: The INSERT is committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+3
+
+TEST-INFO: SLAVE: The INSERT is committed (Succeeded)
+connection master;
+
+######## BEGIN ########
+connection master;
+INSERT INTO t1 SET f1= 3 + 1;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+4
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+3
+connection master;
+BEGIN;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+4
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+4
+connection master;
+ROLLBACK;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+4
+
+TEST-INFO: MASTER: The INSERT is committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+4
+
+TEST-INFO: SLAVE: The INSERT is committed (Succeeded)
+connection master;
+
+######## DROP TABLE mysqltest1.t2 ########
+connection master;
+INSERT INTO t1 SET f1= 4 + 1;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+5
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+4
+connection master;
+DROP TABLE mysqltest1.t2;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+5
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+5
+connection master;
+ROLLBACK;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+5
+
+TEST-INFO: MASTER: The INSERT is committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+5
+
+TEST-INFO: SLAVE: The INSERT is committed (Succeeded)
+connection master;
+SHOW TABLES LIKE 't2';
+Tables_in_mysqltest1 (t2)
+connection slave;
+SHOW TABLES LIKE 't2';
+Tables_in_mysqltest1 (t2)
+connection master;
+
+######## DROP TEMPORARY TABLE mysqltest1.t23 ########
+connection master;
+INSERT INTO t1 SET f1= 5 + 1;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+6
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+5
+connection master;
+DROP TEMPORARY TABLE mysqltest1.t23;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+6
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+5
+connection master;
+ROLLBACK;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+5
+
+TEST-INFO: MASTER: The INSERT is not committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+6
+
+TEST-INFO: SLAVE: The INSERT is committed (Failed)
+connection master;
+SHOW TABLES LIKE 't23';
+Tables_in_mysqltest1 (t23)
+connection slave;
+SHOW TABLES LIKE 't23';
+Tables_in_mysqltest1 (t23)
+connection master;
+
+######## RENAME TABLE mysqltest1.t3 to mysqltest1.t20 ########
+connection master;
+INSERT INTO t1 SET f1= 5 + 1;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+6
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+6
+connection master;
+RENAME TABLE mysqltest1.t3 to mysqltest1.t20;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+6
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+6
+connection master;
+ROLLBACK;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+6
+
+TEST-INFO: MASTER: The INSERT is committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+6
+
+TEST-INFO: SLAVE: The INSERT is committed (Succeeded)
+connection master;
+SHOW TABLES LIKE 't20';
+Tables_in_mysqltest1 (t20)
+t20
+connection slave;
+SHOW TABLES LIKE 't20';
+Tables_in_mysqltest1 (t20)
+t20
+connection master;
+
+######## ALTER TABLE mysqltest1.t4 ADD column f2 BIGINT ########
+connection master;
+INSERT INTO t1 SET f1= 6 + 1;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+7
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+6
+connection master;
+ALTER TABLE mysqltest1.t4 ADD column f2 BIGINT;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+7
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+7
+connection master;
+ROLLBACK;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+7
+
+TEST-INFO: MASTER: The INSERT is committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+7
+
+TEST-INFO: SLAVE: The INSERT is committed (Succeeded)
+connection master;
+describe mysqltest1.t4;
+Field Type Null Key Default Extra
+f1 bigint(20) YES NULL
+f2 bigint(20) YES NULL
+connection slave;
+describe mysqltest1.t4;
+Field Type Null Key Default Extra
+f1 bigint(20) YES NULL
+f2 bigint(20) YES NULL
+connection master;
+
+######## CREATE TABLE mysqltest1.t21 (f1 BIGINT) ENGINE= InnoDB ########
+connection master;
+INSERT INTO t1 SET f1= 7 + 1;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+8
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+7
+connection master;
+CREATE TABLE mysqltest1.t21 (f1 BIGINT) ENGINE= InnoDB;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+8
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+8
+connection master;
+ROLLBACK;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+8
+
+TEST-INFO: MASTER: The INSERT is committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+8
+
+TEST-INFO: SLAVE: The INSERT is committed (Succeeded)
+connection master;
+
+######## CREATE TEMPORARY TABLE mysqltest1.t22 (f1 BIGINT) ENGINE=MEMORY ########
+connection master;
+INSERT INTO t1 SET f1= 8 + 1;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+9
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+8
+connection master;
+CREATE TEMPORARY TABLE mysqltest1.t22 (f1 BIGINT) ENGINE=MEMORY;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+9
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+8
+connection master;
+ROLLBACK;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+8
+
+TEST-INFO: MASTER: The INSERT is not committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+9
+
+TEST-INFO: SLAVE: The INSERT is committed (Failed)
+connection master;
+
+######## TRUNCATE TABLE mysqltest1.t7 ########
+connection master;
+INSERT INTO t1 SET f1= 8 + 1;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+9
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+9
+connection master;
+TRUNCATE TABLE mysqltest1.t7;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+9
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+9
+connection master;
+ROLLBACK;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+9
+
+TEST-INFO: MASTER: The INSERT is committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+9
+
+TEST-INFO: SLAVE: The INSERT is committed (Succeeded)
+connection master;
+SELECT * FROM mysqltest1.t7;
+f1
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT * FROM mysqltest1.t7;
+f1
+connection master;
+
+######## LOCK TABLES mysqltest1.t1 WRITE, mysqltest1.t8 READ ########
+connection master;
+INSERT INTO t1 SET f1= 9 + 1;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+10
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+9
+connection master;
+LOCK TABLES mysqltest1.t1 WRITE, mysqltest1.t8 READ;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+10
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+10
+connection master;
+ROLLBACK;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+10
+
+TEST-INFO: MASTER: The INSERT is committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+10
+
+TEST-INFO: SLAVE: The INSERT is committed (Succeeded)
+connection master;
+UNLOCK TABLES;
+
+######## UNLOCK TABLES ########
+connection master;
+INSERT INTO t1 SET f1= 10 + 1;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+11
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+10
+connection master;
+UNLOCK TABLES;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+11
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+10
+connection master;
+ROLLBACK;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+10
+
+TEST-INFO: MASTER: The INSERT is not committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+10
+
+TEST-INFO: SLAVE: The INSERT is not committed (Succeeded)
+connection master;
+LOCK TABLES mysqltest1.t1 READ;
+
+######## UNLOCK TABLES ########
+connection master;
+INSERT INTO t1 SET f1= 10 + 1;
+ERROR HY000: Table 't1' was locked with a READ lock and can't be updated
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+10
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+10
+connection master;
+UNLOCK TABLES;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+10
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+10
+connection master;
+ROLLBACK;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+10
+
+TEST-INFO: MASTER: The INSERT is not committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+10
+
+TEST-INFO: SLAVE: The INSERT is not committed (Succeeded)
+connection master;
+LOCK TABLES mysqltest1.t1 WRITE, mysqltest1.t8 READ;
+
+######## UNLOCK TABLES ########
+connection master;
+INSERT INTO t1 SET f1= 10 + 1;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+11
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+10
+connection master;
+UNLOCK TABLES;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+11
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+11
+connection master;
+ROLLBACK;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+11
+
+TEST-INFO: MASTER: The INSERT is committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+11
+
+TEST-INFO: SLAVE: The INSERT is committed (Succeeded)
+connection master;
+
+######## DROP INDEX my_idx6 ON mysqltest1.t6 ########
+connection master;
+INSERT INTO t1 SET f1= 11 + 1;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+12
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+11
+connection master;
+DROP INDEX my_idx6 ON mysqltest1.t6;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+12
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+12
+connection master;
+ROLLBACK;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+12
+
+TEST-INFO: MASTER: The INSERT is committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+12
+
+TEST-INFO: SLAVE: The INSERT is committed (Succeeded)
+connection master;
+SHOW INDEX FROM mysqltest1.t6;
+Table Non_unique Key_name Seq_in_index Column_name Collation Cardinality Sub_part Packed Null Index_type Comment Index_comment Ignored
+connection slave;
+SHOW INDEX FROM mysqltest1.t6;
+Table Non_unique Key_name Seq_in_index Column_name Collation Cardinality Sub_part Packed Null Index_type Comment Index_comment Ignored
+connection master;
+
+######## CREATE INDEX my_idx5 ON mysqltest1.t5(f1) ########
+connection master;
+INSERT INTO t1 SET f1= 12 + 1;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+13
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+12
+connection master;
+CREATE INDEX my_idx5 ON mysqltest1.t5(f1);
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+13
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+13
+connection master;
+ROLLBACK;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+13
+
+TEST-INFO: MASTER: The INSERT is committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+13
+
+TEST-INFO: SLAVE: The INSERT is committed (Succeeded)
+connection master;
+SHOW INDEX FROM mysqltest1.t5;
+Table Non_unique Key_name Seq_in_index Column_name Collation Cardinality Sub_part Packed Null Index_type Comment Index_comment Ignored
+t5 1 my_idx5 1 f1 A 0 NULL NULL YES BTREE NO
+connection slave;
+SHOW INDEX FROM mysqltest1.t5;
+Table Non_unique Key_name Seq_in_index Column_name Collation Cardinality Sub_part Packed Null Index_type Comment Index_comment Ignored
+t5 1 my_idx5 1 f1 A NULL NULL NULL YES BTREE NO
+connection master;
+
+######## DROP DATABASE mysqltest2 ########
+connection master;
+INSERT INTO t1 SET f1= 13 + 1;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+14
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+13
+connection master;
+DROP DATABASE mysqltest2;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+14
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+14
+connection master;
+ROLLBACK;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+14
+
+TEST-INFO: MASTER: The INSERT is committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+14
+
+TEST-INFO: SLAVE: The INSERT is committed (Succeeded)
+connection master;
+SHOW DATABASES LIKE "mysqltest2";
+Database (mysqltest2)
+connection slave;
+SHOW DATABASES LIKE "mysqltest2";
+Database (mysqltest2)
+connection master;
+
+######## CREATE DATABASE mysqltest3 ########
+connection master;
+INSERT INTO t1 SET f1= 14 + 1;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+15
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+14
+connection master;
+CREATE DATABASE mysqltest3;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+15
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+15
+connection master;
+ROLLBACK;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+15
+
+TEST-INFO: MASTER: The INSERT is committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+15
+
+TEST-INFO: SLAVE: The INSERT is committed (Succeeded)
+connection master;
+SHOW DATABASES LIKE "mysqltest3";
+Database (mysqltest3)
+mysqltest3
+connection slave;
+SHOW DATABASES LIKE "mysqltest3";
+Database (mysqltest3)
+mysqltest3
+connection master;
+
+######## CREATE PROCEDURE p1() READS SQL DATA SELECT "this is p1" ########
+connection master;
+INSERT INTO t1 SET f1= 15 + 1;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+16
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+15
+connection master;
+CREATE PROCEDURE p1() READS SQL DATA SELECT "this is p1";
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+16
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+16
+connection master;
+ROLLBACK;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+16
+
+TEST-INFO: MASTER: The INSERT is committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+16
+
+TEST-INFO: SLAVE: The INSERT is committed (Succeeded)
+connection master;
+SHOW PROCEDURE STATUS LIKE 'p1';
+Db mysqltest1
+Name p1
+Type PROCEDURE
+Definer root@localhost
+Modified #
+Created #
+Security_type DEFINER
+Comment
+character_set_client latin1
+collation_connection latin1_swedish_ci
+Database Collation latin1_swedish_ci
+connection slave;
+SHOW PROCEDURE STATUS LIKE 'p1';
+Db mysqltest1
+Name p1
+Type PROCEDURE
+Definer root@localhost
+Modified #
+Created #
+Security_type DEFINER
+Comment
+character_set_client latin1
+collation_connection latin1_swedish_ci
+Database Collation latin1_swedish_ci
+connection master;
+
+######## ALTER PROCEDURE p1 COMMENT "I have been altered" ########
+connection master;
+INSERT INTO t1 SET f1= 16 + 1;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+17
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+16
+connection master;
+ALTER PROCEDURE p1 COMMENT "I have been altered";
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+17
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+17
+connection master;
+ROLLBACK;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+17
+
+TEST-INFO: MASTER: The INSERT is committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+17
+
+TEST-INFO: SLAVE: The INSERT is committed (Succeeded)
+connection master;
+SHOW PROCEDURE STATUS LIKE 'p1';
+Db mysqltest1
+Name p1
+Type PROCEDURE
+Definer root@localhost
+Modified #
+Created #
+Security_type DEFINER
+Comment I have been altered
+character_set_client latin1
+collation_connection latin1_swedish_ci
+Database Collation latin1_swedish_ci
+connection slave;
+SHOW PROCEDURE STATUS LIKE 'p1';
+Db mysqltest1
+Name p1
+Type PROCEDURE
+Definer root@localhost
+Modified #
+Created #
+Security_type DEFINER
+Comment I have been altered
+character_set_client latin1
+collation_connection latin1_swedish_ci
+Database Collation latin1_swedish_ci
+connection master;
+
+######## DROP PROCEDURE p1 ########
+connection master;
+INSERT INTO t1 SET f1= 17 + 1;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+18
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+17
+connection master;
+DROP PROCEDURE p1;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+18
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+18
+connection master;
+ROLLBACK;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+18
+
+TEST-INFO: MASTER: The INSERT is committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+18
+
+TEST-INFO: SLAVE: The INSERT is committed (Succeeded)
+connection master;
+SHOW PROCEDURE STATUS LIKE 'p1';
+connection slave;
+SHOW PROCEDURE STATUS LIKE 'p1';
+connection master;
+
+######## CREATE OR REPLACE VIEW v1 as select * from t1 ########
+connection master;
+INSERT INTO t1 SET f1= 18 + 1;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+19
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+18
+connection master;
+CREATE OR REPLACE VIEW v1 as select * from t1;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+19
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+19
+connection master;
+ROLLBACK;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+19
+
+TEST-INFO: MASTER: The INSERT is committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+19
+
+TEST-INFO: SLAVE: The INSERT is committed (Succeeded)
+connection master;
+SHOW CREATE VIEW v1;
+View Create View character_set_client collation_connection
+v1 CREATE ALGORITHM=UNDEFINED DEFINER=`root`@`localhost` SQL SECURITY DEFINER VIEW `v1` AS select `t1`.`f1` AS `f1` from `t1` latin1 latin1_swedish_ci
+connection slave;
+SHOW CREATE VIEW v1;
+View Create View character_set_client collation_connection
+v1 CREATE ALGORITHM=UNDEFINED DEFINER=`root`@`localhost` SQL SECURITY DEFINER VIEW `v1` AS select `t1`.`f1` AS `f1` from `t1` latin1 latin1_swedish_ci
+connection master;
+
+######## ALTER VIEW v1 AS select f1 from t1 ########
+connection master;
+INSERT INTO t1 SET f1= 19 + 1;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+20
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+19
+connection master;
+ALTER VIEW v1 AS select f1 from t1;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+20
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+20
+connection master;
+ROLLBACK;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+20
+
+TEST-INFO: MASTER: The INSERT is committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+20
+
+TEST-INFO: SLAVE: The INSERT is committed (Succeeded)
+connection master;
+SHOW CREATE VIEW v1;
+View Create View character_set_client collation_connection
+v1 CREATE ALGORITHM=UNDEFINED DEFINER=`root`@`localhost` SQL SECURITY DEFINER VIEW `v1` AS select `t1`.`f1` AS `f1` from `t1` latin1 latin1_swedish_ci
+connection slave;
+SHOW CREATE VIEW v1;
+View Create View character_set_client collation_connection
+v1 CREATE ALGORITHM=UNDEFINED DEFINER=`root`@`localhost` SQL SECURITY DEFINER VIEW `v1` AS select `t1`.`f1` AS `f1` from `t1` latin1 latin1_swedish_ci
+connection master;
+
+######## DROP VIEW IF EXISTS v1 ########
+connection master;
+INSERT INTO t1 SET f1= 20 + 1;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+21
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+20
+connection master;
+DROP VIEW IF EXISTS v1;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+21
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+21
+connection master;
+ROLLBACK;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+21
+
+TEST-INFO: MASTER: The INSERT is committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+21
+
+TEST-INFO: SLAVE: The INSERT is committed (Succeeded)
+connection master;
+SHOW CREATE VIEW v1;
+ERROR 42S02: Table 'mysqltest1.v1' doesn't exist
+connection slave;
+SHOW CREATE VIEW v1;
+ERROR 42S02: Table 'mysqltest1.v1' doesn't exist
+connection master;
+
+######## CREATE TRIGGER trg1 BEFORE INSERT ON t1 FOR EACH ROW SET @a:=1 ########
+connection master;
+INSERT INTO t1 SET f1= 21 + 1;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+22
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+21
+connection master;
+CREATE TRIGGER trg1 BEFORE INSERT ON t1 FOR EACH ROW SET @a:=1;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+22
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+22
+connection master;
+ROLLBACK;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+22
+
+TEST-INFO: MASTER: The INSERT is committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+22
+
+TEST-INFO: SLAVE: The INSERT is committed (Succeeded)
+connection master;
+SHOW TRIGGERS;
+Trigger Event Table Statement Timing Created sql_mode Definer character_set_client collation_connection Database Collation
+trg1 INSERT t1 SET @a:=1 BEFORE # root@localhost latin1 latin1_swedish_ci latin1_swedish_ci
+connection slave;
+SHOW TRIGGERS;
+Trigger Event Table Statement Timing Created sql_mode Definer character_set_client collation_connection Database Collation
+trg1 INSERT t1 SET @a:=1 BEFORE # root@localhost latin1 latin1_swedish_ci latin1_swedish_ci
+connection master;
+
+######## DROP TRIGGER trg1 ########
+connection master;
+INSERT INTO t1 SET f1= 22 + 1;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+23
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+22
+connection master;
+DROP TRIGGER trg1;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+23
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+23
+connection master;
+ROLLBACK;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+23
+
+TEST-INFO: MASTER: The INSERT is committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+23
+
+TEST-INFO: SLAVE: The INSERT is committed (Succeeded)
+connection master;
+SHOW TRIGGERS;
+Trigger Event Table Statement Timing Created sql_mode Definer character_set_client collation_connection Database Collation
+connection slave;
+SHOW TRIGGERS;
+Trigger Event Table Statement Timing Created sql_mode Definer character_set_client collation_connection Database Collation
+connection master;
+
+######## CREATE USER user1@localhost ########
+connection master;
+INSERT INTO t1 SET f1= 23 + 1;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+24
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+23
+connection master;
+CREATE USER user1@localhost;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+24
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+24
+connection master;
+ROLLBACK;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+24
+
+TEST-INFO: MASTER: The INSERT is committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+24
+
+TEST-INFO: SLAVE: The INSERT is committed (Succeeded)
+connection master;
+SELECT user FROM mysql.user WHERE user = 'user1';
+User
+user1
+connection slave;
+SELECT user FROM mysql.user WHERE user = 'user1';
+User
+user1
+connection master;
+
+######## RENAME USER user1@localhost TO rename1@localhost ########
+connection master;
+INSERT INTO t1 SET f1= 24 + 1;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+25
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+24
+connection master;
+RENAME USER user1@localhost TO rename1@localhost;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+25
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+25
+connection master;
+ROLLBACK;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+25
+
+TEST-INFO: MASTER: The INSERT is committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+25
+
+TEST-INFO: SLAVE: The INSERT is committed (Succeeded)
+connection master;
+SELECT user FROM mysql.user WHERE user = 'rename1';
+User
+rename1
+connection slave;
+SELECT user FROM mysql.user WHERE user = 'rename1';
+User
+rename1
+connection master;
+
+######## DROP USER rename1@localhost ########
+connection master;
+INSERT INTO t1 SET f1= 25 + 1;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+26
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+25
+connection master;
+DROP USER rename1@localhost;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+26
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+26
+connection master;
+ROLLBACK;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+26
+
+TEST-INFO: MASTER: The INSERT is committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection slave;
+SELECT MAX(f1) FROM t1;
+MAX(f1)
+26
+
+TEST-INFO: SLAVE: The INSERT is committed (Succeeded)
+connection master;
+SELECT user FROM mysql.user WHERE user = 'rename1';
+User
+connection slave;
+SELECT user FROM mysql.user WHERE user = 'rename1';
+User
+use test;
+connection master;
+DROP TEMPORARY TABLE mysqltest1.t22;
+DROP DATABASE mysqltest1;
+DROP DATABASE mysqltest3;
+set global sql_mode=default;
+include/rpl_end.inc
diff --git a/mysql-test/suite/binlog_in_engine/rpl_ddl_misf.test b/mysql-test/suite/binlog_in_engine/rpl_ddl_misf.test
new file mode 100644
index 0000000000000..3d084668b4121
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/rpl_ddl_misf.test
@@ -0,0 +1,8 @@
+--source include/have_innodb.inc
+--source include/master-slave.inc
+let $engine_type= InnoDB;
+let $temp_engine_type= MEMORY;
+let $show_binlog = 0;
+let $manipulate = 0;
+--source ../rpl/include/rpl_ddl.test
+--source include/rpl_end.inc
diff --git a/mysql-test/suite/binlog_in_engine/rpl_dual_cache.result b/mysql-test/suite/binlog_in_engine/rpl_dual_cache.result
new file mode 100644
index 0000000000000..a127c1c8f5127
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/rpl_dual_cache.result
@@ -0,0 +1,108 @@
+include/master-slave.inc
+[connection master]
+*** Various tests for event groups using both stmt/trx cache.
+CREATE SEQUENCE s;
+CREATE TABLE t (a int PRIMARY KEY, b int DEFAULT (NEXTVAL(s))) ENGINE=InnoDB;
+INSERT INTO t (a) VALUES (1);
+show binlog events in 'binlog-000000.ibb' from ;
+Log_name Pos Event_type Server_id End_log_pos Info
+binlog-000000.ibb # Gtid 1 # BEGIN GTID #-#-#
+binlog-000000.ibb # Annotate_rows 1 # INSERT INTO t (a) VALUES (1)
+binlog-000000.ibb # Table_map 1 # table_id: # (test.s)
+binlog-000000.ibb # Write_rows_v1 1 # table_id: # flags: STMT_END_F
+binlog-000000.ibb # Table_map 1 # table_id: # (test.t)
+binlog-000000.ibb # Write_rows_v1 1 # table_id: # flags: STMT_END_F
+binlog-000000.ibb # Xid 1 # COMMIT /* XID */
+CREATE TABLE t1 (a INT PRIMARY KEY, b VARCHAR(2048)) ENGINE=MyISAM;
+CREATE TABLE t2 (a INT PRIMARY KEY, b VARCHAR(2048)) ENGINE=InnoDB;
+SET binlog_format=ROW;
+INSERT INTO t2 VALUES (1, '');
+INSERT INTO t1 VALUES (1, '');
+UPDATE t1, t2 SET t1.b='a', t2.b='b';
+show binlog events in 'binlog-000000.ibb' from ;
+Log_name Pos Event_type Server_id End_log_pos Info
+binlog-000000.ibb # Gtid 1 # BEGIN GTID #-#-#
+binlog-000000.ibb # Annotate_rows 1 # UPDATE t1, t2 SET t1.b='a', t2.b='b'
+binlog-000000.ibb # Table_map 1 # table_id: # (test.t1)
+binlog-000000.ibb # Update_rows_v1 1 # table_id: # flags: STMT_END_F
+binlog-000000.ibb # Table_map 1 # table_id: # (test.t2)
+binlog-000000.ibb # Update_rows_v1 1 # table_id: # flags: STMT_END_F
+binlog-000000.ibb # Xid 1 # COMMIT /* XID */
+INSERT INTO t1 SELECT seq, '' FROM seq_2_to_25;
+INSERT INTO t2 SELECT seq, '' FROM seq_2_to_25;
+UPDATE t1, t2 SET t1.b=REPEAT('x', 2000), t2.b=REPEAT('y',2000) WHERE t1.a=t2.a;
+show binlog events in 'binlog-000000.ibb' from ;
+Log_name Pos Event_type Server_id End_log_pos Info
+binlog-000000.ibb # Gtid 1 # BEGIN GTID #-#-#
+binlog-000000.ibb # Annotate_rows 1 # UPDATE t1, t2 SET t1.b=REPEAT('x', 2000), t2.b=REPEAT('y',2000) WHERE t1.a=t2.a
+binlog-000000.ibb # Table_map 1 # table_id: # (test.t1)
+binlog-000000.ibb # Update_rows_v1 1 # table_id: #
+binlog-000000.ibb # Update_rows_v1 1 # table_id: #
+binlog-000000.ibb # Update_rows_v1 1 # table_id: #
+binlog-000000.ibb # Update_rows_v1 1 # table_id: #
+binlog-000000.ibb # Update_rows_v1 1 # table_id: #
+binlog-000000.ibb # Update_rows_v1 1 # table_id: #
+binlog-000000.ibb # Update_rows_v1 1 # table_id: # flags: STMT_END_F
+binlog-000000.ibb # Table_map 1 # table_id: # (test.t2)
+binlog-000000.ibb # Update_rows_v1 1 # table_id: #
+binlog-000000.ibb # Update_rows_v1 1 # table_id: #
+binlog-000000.ibb # Update_rows_v1 1 # table_id: #
+binlog-000000.ibb # Update_rows_v1 1 # table_id: #
+binlog-000000.ibb # Update_rows_v1 1 # table_id: #
+binlog-000000.ibb # Update_rows_v1 1 # table_id: #
+binlog-000000.ibb # Update_rows_v1 1 # table_id: # flags: STMT_END_F
+binlog-000000.ibb # Xid 1 # COMMIT /* XID */
+SET @old_max_size= @@GLOBAL.max_binlog_size;
+SET GLOBAL max_binlog_size= 65536;
+FLUSH BINARY LOGS;
+FLUSH BINARY LOGS;
+INSERT INTO t1 SELECT seq, '' FROM seq_26_to_50;
+INSERT INTO t2 SELECT seq, '' FROM seq_26_to_50;
+UPDATE t1, t2 SET t1.b=REPEAT('\\', 2000), t2.b=REPEAT('/',2000) WHERE t1.a=t2.a;
+PURGE BINARY LOGS TO '$binlog_file';
+ERROR HY000: A purgeable log is in use, will not purge
+FLUSH BINARY LOGS;
+FLUSH BINARY LOGS;
+SET binlog_format=DEFAULT;
+SET GLOBAL max_binlog_size= @old_max_size;
+FLUSH BINARY LOGS;
+FLUSH BINARY LOGS;
+INSERT INTO t(a) SELECT 1+MAX(a) FROM t;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection master;
+PURGE BINARY LOGS TO '$binlog_file';
+SET binlog_format=ROW;
+CREATE SEQUENCE ss ENGINE=InnoDB;
+CREATE TABLE tt (a INT, b BIGINT DEFAULT(NEXTVAL(ss))) ENGINE=InnoDB;
+INSERT INTO tt (a) VALUES (1);
+CREATE TABLE xx ENGINE=InnoDB SELECT * FROM tt;
+SET binlog_format=DEFAULT;
+CREATE TABLE t3 (a INT PRIMARY KEY DEFAULT NEXTVAL(ss), b INT) ENGINE=InnoDB;
+INSERT INTO t3 VALUES (10000, 1);
+INSERT INTO t3 SELECT a+1, b+1 FROM t3;
+INSERT INTO t3 SELECT a+2, b+1 FROM t3;
+INSERT INTO t3 SELECT a+4, b+1 FROM t3;
+INSERT INTO t3 SELECT a+8, b+1 FROM t3;
+INSERT INTO t3 SELECT a+16, b+1 FROM t3;
+INSERT INTO t3 SELECT a+32, b+1 FROM t3;
+SELECT SETVAL(ss, 10005);
+SETVAL(ss, 10005)
+10005
+INSERT IGNORE INTO t3(b) VALUES (10);
+Warnings:
+Warning 1062 Duplicate entry '10006' for key 'PRIMARY'
+include/show_binlog_events.inc
+Log_name Pos Event_type Server_id End_log_pos Info
+binlog-000012.ibb # Gtid # # BEGIN GTID #-#-#
+binlog-000012.ibb # Annotate_rows # # INSERT IGNORE INTO t3(b) VALUES (10)
+binlog-000012.ibb # Table_map # # table_id: # (test.ss)
+binlog-000012.ibb # Write_rows_v1 # # table_id: # flags: STMT_END_F
+binlog-000012.ibb # Xid # # COMMIT /* XID */
+DROP TABLE t;
+DROP SEQUENCE s;
+DROP TABLE t1, t2, t3;
+DROP TABLE tt, xx;
+DROP SEQUENCE ss;
+include/rpl_end.inc
diff --git a/mysql-test/suite/binlog_in_engine/rpl_dual_cache.test b/mysql-test/suite/binlog_in_engine/rpl_dual_cache.test
new file mode 100644
index 0000000000000..ab3b8fb777bf3
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/rpl_dual_cache.test
@@ -0,0 +1,106 @@
+--source include/have_binlog_format_mixed.inc
+--source include/have_sequence.inc
+--source include/master-slave.inc
+--source include/have_innodb_binlog.inc
+
+--echo *** Various tests for event groups using both stmt/trx cache.
+
+--let $datadir= `SELECT @@datadir`
+
+# Implicit allocation from sequence gets binlogged as a non-transactional
+# part of a transactional INSERT.
+CREATE SEQUENCE s;
+CREATE TABLE t (a int PRIMARY KEY, b int DEFAULT (NEXTVAL(s))) ENGINE=InnoDB;
+--let $binlog_file= query_get_value(SHOW MASTER STATUS, File, 1)
+--let $binlog_start= query_get_value(SHOW MASTER STATUS, Position, 1)
+INSERT INTO t (a) VALUES (1);
+--source include/show_binlog_events2.inc
+
+# Multi-table update of both non-transactional and transactional table with
+# autocommit - this populates both the stmt and trx caches.
+
+CREATE TABLE t1 (a INT PRIMARY KEY, b VARCHAR(2048)) ENGINE=MyISAM;
+CREATE TABLE t2 (a INT PRIMARY KEY, b VARCHAR(2048)) ENGINE=InnoDB;
+SET binlog_format=ROW;
+INSERT INTO t2 VALUES (1, '');
+INSERT INTO t1 VALUES (1, '');
+--let $binlog_file= query_get_value(SHOW MASTER STATUS, File, 1)
+--let $binlog_start= query_get_value(SHOW MASTER STATUS, Position, 1)
+UPDATE t1, t2 SET t1.b='a', t2.b='b';
+--source include/show_binlog_events2.inc
+
+# Larger update so there will be oob data for both stmt and trx caches.
+INSERT INTO t1 SELECT seq, '' FROM seq_2_to_25;
+INSERT INTO t2 SELECT seq, '' FROM seq_2_to_25;
+--let $binlog_file= query_get_value(SHOW MASTER STATUS, File, 1)
+--let $binlog_start= query_get_value(SHOW MASTER STATUS, Position, 1)
+UPDATE t1, t2 SET t1.b=REPEAT('x', 2000), t2.b=REPEAT('y',2000) WHERE t1.a=t2.a;
+--source include/show_binlog_events2.inc
+
+# Test that oob refcount is working when both caches are in use.
+# Smaller binlog files to have oob refs across multiple binlog files.
+SET @old_max_size= @@GLOBAL.max_binlog_size;
+SET GLOBAL max_binlog_size= 65536;
+FLUSH BINARY LOGS;
+FLUSH BINARY LOGS;
+
+INSERT INTO t1 SELECT seq, '' FROM seq_26_to_50;
+INSERT INTO t2 SELECT seq, '' FROM seq_26_to_50;
+UPDATE t1, t2 SET t1.b=REPEAT('\\', 2000), t2.b=REPEAT('/',2000) WHERE t1.a=t2.a;
+--let $binlog_file= query_get_value(SHOW MASTER STATUS, File, 1)
+
+--error ER_LOG_IN_USE
+evalp PURGE BINARY LOGS TO '$binlog_file';
+FLUSH BINARY LOGS;
+FLUSH BINARY LOGS;
+# Test mysqlbinlog reading of dual-cache oob data.
+--exec $MYSQL_BINLOG $datadir/$binlog_file > $MYSQLTEST_VARDIR/tmp/mysqlbinlog.txt
+--let $binlog_file= query_get_value(SHOW MASTER STATUS, File, 1)
+SET binlog_format=DEFAULT;
+SET GLOBAL max_binlog_size= @old_max_size;
+FLUSH BINARY LOGS;
+FLUSH BINARY LOGS;
+
+# Make sure the slave is caught up so we can purge.
+INSERT INTO t(a) SELECT 1+MAX(a) FROM t;
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
+
+--connection master
+evalp PURGE BINARY LOGS TO '$binlog_file';
+
+# This asserted due to uninitialized handler_binlog_event_group_info->engine_ptr2
+SET binlog_format=ROW;
+CREATE SEQUENCE ss ENGINE=InnoDB;
+CREATE TABLE tt (a INT, b BIGINT DEFAULT(NEXTVAL(ss))) ENGINE=InnoDB;
+INSERT INTO tt (a) VALUES (1);
+CREATE TABLE xx ENGINE=InnoDB SELECT * FROM tt;
+SET binlog_format=DEFAULT;
+
+# A tricky case where a sequence is updated as part of a transactional
+# DML, but the DML ends up binlogging nothing. The code would put the GTID in
+# the stmt cache, but binlog the trx cache, which caused binlogging of a
+# garbage partial event group containing only annotate and table map event
+# but no GTID or rows event/commit event.
+CREATE TABLE t3 (a INT PRIMARY KEY DEFAULT NEXTVAL(ss), b INT) ENGINE=InnoDB;
+INSERT INTO t3 VALUES (10000, 1);
+INSERT INTO t3 SELECT a+1, b+1 FROM t3;
+INSERT INTO t3 SELECT a+2, b+1 FROM t3;
+INSERT INTO t3 SELECT a+4, b+1 FROM t3;
+INSERT INTO t3 SELECT a+8, b+1 FROM t3;
+INSERT INTO t3 SELECT a+16, b+1 FROM t3;
+INSERT INTO t3 SELECT a+32, b+1 FROM t3;
+
+SELECT SETVAL(ss, 10005);
+--let $binlog_file= query_get_value(SHOW MASTER STATUS, File, 1)
+--let $binlog_start= query_get_value(SHOW MASTER STATUS, Position, 1)
+INSERT IGNORE INTO t3(b) VALUES (10);
+--source include/show_binlog_events.inc
+
+DROP TABLE t;
+DROP SEQUENCE s;
+DROP TABLE t1, t2, t3;
+DROP TABLE tt, xx;
+DROP SEQUENCE ss;
+--source include/rpl_end.inc
diff --git a/mysql-test/suite/binlog_in_engine/rpl_group_commit.result b/mysql-test/suite/binlog_in_engine/rpl_group_commit.result
new file mode 100644
index 0000000000000..1b7044bedaca1
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/rpl_group_commit.result
@@ -0,0 +1,183 @@
+include/master-slave.inc
+[connection master]
+*** Test that the GTID state record is written correctly when inside a group commit.
+SELECT @@GLOBAL.max_binlog_size;
+@@GLOBAL.max_binlog_size
+262144
+CREATE TABLE t1 (a INT NOT NULL, b INT NOT NULL, c TEXT, PRIMARY KEY(a, b)) ENGINE=InnoDB;
+INSERT INTO t1 VALUES (1, 0, REPEAT('0', 24000));
+INSERT INTO t1 VALUES (1, 1, REPEAT('1', 24000));
+INSERT INTO t1 VALUES (1, 2, REPEAT('2', 24000));
+INSERT INTO t1 VALUES (1, 3, REPEAT('3', 24000));
+INSERT INTO t1 VALUES (1, 4, REPEAT('4', 24000));
+INSERT INTO t1 VALUES (1, 5, REPEAT('5', 24000));
+INSERT INTO t1 VALUES (1, 6, REPEAT('6', 24000));
+INSERT INTO t1 VALUES (1, 7, REPEAT('7', 24000));
+INSERT INTO t1 VALUES (1, 8, REPEAT('8', 24000));
+INSERT INTO t1 VALUES (1, 9, REPEAT('9', 24000));
+*** Check that we are still in binlog file 0:
+binlog-000000.ibb
+*** Create a group commit that spans into file 1
+connection master;
+connect con_$i,127.0.0.1,root,,test,$SERVER_MYPORT_1,;
+SET debug_sync='commit_after_release_LOCK_prepare_ordered SIGNAL master_queued1 WAIT_FOR master_cont NO_CLEAR_EVENT';
+INSERT INTO t1 VALUES (2, 1, REPEAT(CHR(64 + 1), 400));
+connection master;
+SET debug_sync='now WAIT_FOR master_queued1';
+connect con_$i,127.0.0.1,root,,test,$SERVER_MYPORT_1,;
+SET debug_sync='commit_after_release_LOCK_prepare_ordered SIGNAL master_queued2 WAIT_FOR master_cont NO_CLEAR_EVENT';
+INSERT INTO t1 VALUES (2, 2, REPEAT(CHR(64 + 2), 400));
+connection master;
+SET debug_sync='now WAIT_FOR master_queued2';
+connect con_$i,127.0.0.1,root,,test,$SERVER_MYPORT_1,;
+SET debug_sync='commit_after_release_LOCK_prepare_ordered SIGNAL master_queued3 WAIT_FOR master_cont NO_CLEAR_EVENT';
+INSERT INTO t1 VALUES (2, 3, REPEAT(CHR(64 + 3), 400));
+connection master;
+SET debug_sync='now WAIT_FOR master_queued3';
+connect con_$i,127.0.0.1,root,,test,$SERVER_MYPORT_1,;
+SET debug_sync='commit_after_release_LOCK_prepare_ordered SIGNAL master_queued4 WAIT_FOR master_cont NO_CLEAR_EVENT';
+INSERT INTO t1 VALUES (2, 4, REPEAT(CHR(64 + 4), 400));
+connection master;
+SET debug_sync='now WAIT_FOR master_queued4';
+connect con_$i,127.0.0.1,root,,test,$SERVER_MYPORT_1,;
+SET debug_sync='commit_after_release_LOCK_prepare_ordered SIGNAL master_queued5 WAIT_FOR master_cont NO_CLEAR_EVENT';
+INSERT INTO t1 VALUES (2, 5, REPEAT(CHR(64 + 5), 400));
+connection master;
+SET debug_sync='now WAIT_FOR master_queued5';
+connect con_$i,127.0.0.1,root,,test,$SERVER_MYPORT_1,;
+SET debug_sync='commit_after_release_LOCK_prepare_ordered SIGNAL master_queued6 WAIT_FOR master_cont NO_CLEAR_EVENT';
+INSERT INTO t1 VALUES (2, 6, REPEAT(CHR(64 + 6), 400));
+connection master;
+SET debug_sync='now WAIT_FOR master_queued6';
+connect con_$i,127.0.0.1,root,,test,$SERVER_MYPORT_1,;
+SET debug_sync='commit_after_release_LOCK_prepare_ordered SIGNAL master_queued7 WAIT_FOR master_cont NO_CLEAR_EVENT';
+INSERT INTO t1 VALUES (2, 7, REPEAT(CHR(64 + 7), 400));
+connection master;
+SET debug_sync='now WAIT_FOR master_queued7';
+connect con_$i,127.0.0.1,root,,test,$SERVER_MYPORT_1,;
+SET debug_sync='commit_after_release_LOCK_prepare_ordered SIGNAL master_queued8 WAIT_FOR master_cont NO_CLEAR_EVENT';
+INSERT INTO t1 VALUES (2, 8, REPEAT(CHR(64 + 8), 400));
+connection master;
+SET debug_sync='now WAIT_FOR master_queued8';
+connect con_$i,127.0.0.1,root,,test,$SERVER_MYPORT_1,;
+SET debug_sync='commit_after_release_LOCK_prepare_ordered SIGNAL master_queued9 WAIT_FOR master_cont NO_CLEAR_EVENT';
+INSERT INTO t1 VALUES (2, 9, REPEAT(CHR(64 + 9), 400));
+connection master;
+SET debug_sync='now WAIT_FOR master_queued9';
+connect con_$i,127.0.0.1,root,,test,$SERVER_MYPORT_1,;
+SET debug_sync='commit_after_release_LOCK_prepare_ordered SIGNAL master_queued10 WAIT_FOR master_cont NO_CLEAR_EVENT';
+INSERT INTO t1 VALUES (2, 10, REPEAT(CHR(64 + 10), 400));
+connection master;
+SET debug_sync='now WAIT_FOR master_queued10';
+SET debug_sync= 'now SIGNAL master_cont';
+connection con_1;
+connection master;
+disconnect con_1;
+connection con_2;
+connection master;
+disconnect con_2;
+connection con_3;
+connection master;
+disconnect con_3;
+connection con_4;
+connection master;
+disconnect con_4;
+connection con_5;
+connection master;
+disconnect con_5;
+connection con_6;
+connection master;
+disconnect con_6;
+connection con_7;
+connection master;
+disconnect con_7;
+connection con_8;
+connection master;
+disconnect con_8;
+connection con_9;
+connection master;
+disconnect con_9;
+connection con_10;
+connection master;
+disconnect con_10;
+SET debug_sync= 'RESET';
+*** Check that we are now in binlog file 1:
+binlog-000001.ibb
+SELECT a, b, LENGTH(c) FROM t1 ORDER BY a, b;
+a b LENGTH(c)
+1 0 24000
+1 1 24000
+1 2 24000
+1 3 24000
+1 4 24000
+1 5 24000
+1 6 24000
+1 7 24000
+1 8 24000
+1 9 24000
+2 1 400
+2 2 400
+2 3 400
+2 4 400
+2 5 400
+2 6 400
+2 7 400
+2 8 400
+2 9 400
+2 10 400
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+SELECT a, b, LENGTH(c) FROM t1 ORDER BY a, b;
+a b LENGTH(c)
+1 0 24000
+1 1 24000
+1 2 24000
+1 3 24000
+1 4 24000
+1 5 24000
+1 6 24000
+1 7 24000
+1 8 24000
+1 9 24000
+2 1 400
+2 2 400
+2 3 400
+2 4 400
+2 5 400
+2 6 400
+2 7 400
+2 8 400
+2 9 400
+2 10 400
+*** Stop and restart the slave.
+include/stop_slave.inc
+include/start_slave.inc
+*** Test FLUSH BINARY LOGS in parallel with GTID allocation.
+connection master;
+INSERT INTO t1 VALUES (10, 1, 'before race');
+connect con1,127.0.0.1,root,,test,$SERVER_MYPORT_1,;
+SET SESSION debug_dbug= '+d,flush_binlog_sleep_after_release_lock_log';
+FLUSH BINARY LOGS;
+connection master;
+SET @old_dbug= @@SESSION.debug_dbug;
+SET SESSION debug_dbug= '+d,binlog_sleep_after_alloc_gtid';
+INSERT INTO t1 VALUES (10, 2, 'race?');
+connection con1;
+disconnect con1;
+connection master;
+SET SESSION debug_dbug= @old_dbug;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+*** Stop and restart the slave, 2.
+include/stop_slave.inc
+include/start_slave.inc
+connection master;
+INSERT INTO t1 VALUES (10, 3, 'check?');
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection master;
+DROP TABLE t1;
+include/rpl_end.inc
diff --git a/mysql-test/suite/binlog_in_engine/rpl_group_commit.test b/mysql-test/suite/binlog_in_engine/rpl_group_commit.test
new file mode 100644
index 0000000000000..de30b8a0618e5
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/rpl_group_commit.test
@@ -0,0 +1,120 @@
+--source include/have_debug.inc
+--source include/have_debug_sync.inc
+--source include/have_binlog_format_row.inc
+--source include/master-slave.inc
+--source include/have_innodb_binlog.inc
+
+--echo *** Test that the GTID state record is written correctly when inside a group commit.
+
+# The test needs a group commit to cross the file boundary, so needs the
+# binlog size to be as expected.
+SELECT @@GLOBAL.max_binlog_size;
+
+CREATE TABLE t1 (a INT NOT NULL, b INT NOT NULL, c TEXT, PRIMARY KEY(a, b)) ENGINE=InnoDB;
+# Get close to the end of the file.
+--let $i= 0
+while ($i < 10) {
+ eval INSERT INTO t1 VALUES (1, $i, REPEAT('$i', 24000));
+ inc $i;
+}
+--echo *** Check that we are still in binlog file 0:
+--let $file= query_get_value(SHOW MASTER STATUS, File, 1)
+--echo $file
+
+--echo *** Create a group commit that spans into file 1
+--connection master
+--let $i= 1
+while ($i <= 10) {
+ --connect (con_$i,127.0.0.1,root,,test,$SERVER_MYPORT_1,)
+ eval SET debug_sync='commit_after_release_LOCK_prepare_ordered SIGNAL master_queued$i WAIT_FOR master_cont NO_CLEAR_EVENT';
+ send_eval INSERT INTO t1 VALUES (2, $i, REPEAT(CHR(64 + $i), 400));
+ --connection master
+ eval SET debug_sync='now WAIT_FOR master_queued$i';
+ inc $i;
+}
+
+SET debug_sync= 'now SIGNAL master_cont';
+
+--let $i= 1
+while ($i <= 10) {
+ --connection con_$i
+ reap;
+ --connection master
+ --disconnect con_$i
+ inc $i;
+}
+SET debug_sync= 'RESET';
+
+--echo *** Check that we are now in binlog file 1:
+--let $file= query_get_value(SHOW MASTER STATUS, File, 1)
+--echo $file
+
+SELECT a, b, LENGTH(c) FROM t1 ORDER BY a, b;
+
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
+
+SELECT a, b, LENGTH(c) FROM t1 ORDER BY a, b;
+
+--echo *** Stop and restart the slave.
+# There was a bug that the GTID state record at the start of the file
+# would containt the GTID for the _end_ of the group commit, not of the
+# point in the group commit where it crossed into the next file. And then
+# the slave would replicate from too early and get a duplicate key error.
+--source include/stop_slave.inc
+--source include/start_slave.inc
+
+
+--echo *** Test FLUSH BINARY LOGS in parallel with GTID allocation.
+
+--connection master
+INSERT INTO t1 VALUES (10, 1, 'before race');
+
+# Send a FLUSH BINARY LOGS.
+# Make the FLUSH sleep a little after releasing the LOCK_log but before reading
+# the current binlog GTID state, to make the race likely to hit. Using
+# debug_sync here will not be effective after fixing the bug, as the fix
+# will prevent the race and make the debug_sync time out.
+--connect (con1,127.0.0.1,root,,test,$SERVER_MYPORT_1,)
+SET SESSION debug_dbug= '+d,flush_binlog_sleep_after_release_lock_log';
+send FLUSH BINARY LOGS;
+
+--connection master
+# The FLUSH will sleep 0.2, sleep 0.1 here to have a high chance to hit the race.
+SET @old_dbug= @@SESSION.debug_dbug;
+SET SESSION debug_dbug= '+d,binlog_sleep_after_alloc_gtid';
+--sleep 0.1
+send INSERT INTO t1 VALUES (10, 2, 'race?');
+
+--connection con1
+reap;
+--disconnect con1
+
+--connection master
+reap;
+SET SESSION debug_dbug= @old_dbug;
+--source include/save_master_gtid.inc
+
+--connection slave
+--source include/sync_with_master_gtid.inc
+
+--echo *** Stop and restart the slave, 2.
+# The bug was that the FLUSH could run after the new GTID was allocated, but
+# before it was written into the binlog. This would cause the wrong binlog
+# state at the start of the new binlog file, and a slave that reconnects at
+# that exact GTID will get a duplicate GTID.
+--source include/stop_slave.inc
+--source include/start_slave.inc
+
+--connection master
+INSERT INTO t1 VALUES (10, 3, 'check?');
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
+
+
+# Cleanup.
+--connection master
+DROP TABLE t1;
+--source include/rpl_end.inc
diff --git a/mysql-test/suite/binlog_in_engine/rpl_gtid_index-master.opt b/mysql-test/suite/binlog_in_engine/rpl_gtid_index-master.opt
new file mode 100644
index 0000000000000..74472f80acf69
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/rpl_gtid_index-master.opt
@@ -0,0 +1,3 @@
+--max-binlog-size=128k
+--innodb-binlog-state-interval=64k
+--log-bin
diff --git a/mysql-test/suite/binlog_in_engine/rpl_gtid_index-slave.opt b/mysql-test/suite/binlog_in_engine/rpl_gtid_index-slave.opt
new file mode 100644
index 0000000000000..beae84b386292
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/rpl_gtid_index-slave.opt
@@ -0,0 +1 @@
+--log-bin
diff --git a/mysql-test/suite/binlog_in_engine/rpl_gtid_index.result b/mysql-test/suite/binlog_in_engine/rpl_gtid_index.result
new file mode 100644
index 0000000000000..bfcd314b93200
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/rpl_gtid_index.result
@@ -0,0 +1,150 @@
+include/master-slave.inc
+[connection master]
+connection slave;
+include/stop_slave.inc
+CHANGE MASTER TO master_use_gtid= slave_pos;
+include/start_slave.inc
+connection master;
+CREATE TABLE t1 (a INT PRIMARY KEY, b INT) ENGINE=InnoDB;
+INSERT INTO t1 VALUES (0, 0);
+*** Test looking up a lot of different event positions and GTIDs.
+CREATE FUNCTION gtid_eq(a VARCHAR(255), b VARCHAR(255)) RETURNS BOOLEAN DETERMINISTIC
+BEGIN
+DECLARE g VARCHAR(255);
+IF a IS NULL OR b IS NULL OR LENGTH(a) != LENGTH(b) THEN
+RETURN FALSE;
+END IF;
+SET a= CONCAT(a, ',');
+SET b= CONCAT(',', b, ',');
+WHILE LENGTH(a) > 0 DO
+SET g= REGEXP_SUBSTR(a, '^[^,]+,');
+SET a= SUBSTRING(a, LENGTH(g)+1);
+SET b= REPLACE(b, CONCAT(',', g), ',');
+END WHILE;
+RETURN b = ',';
+END //
+*** A lot of GTIDs with small binlog size and state interval
+*** Testing 1000 GTIDs with 100 test connects
+connection master;
+DELETE FROM t1 WHERE a >= 1000;
+FLUSH NO_WRITE_TO_BINLOG BINARY LOGS;
+CREATE TABLE rand_data(idx INT PRIMARY KEY, domain_id INT, server_id INT)
+ENGINE=InnoDB;
+INSERT INTO rand_data(idx, domain_id, server_id) VALUES (0, 0, 1);
+INSERT INTO rand_data(idx, domain_id, server_id)
+SELECT seq,
+@tmp:=floor(10*POW(rand(150),2)),
+100 + 5*@tmp + floor(5*rand(150))
+FROM seq_1_to_1000;
+SELECT COUNT(*), SUM(domain_id), SUM(server_id) FROM rand_data;
+COUNT(*) SUM(domain_id) SUM(server_id)
+1001 2881 116394
+CREATE TABLE gtid_data(
+idx INT PRIMARY KEY,
+gtid VARCHAR(44),
+gtid_pos VARCHAR(255),
+file VARCHAR(100),
+pos INT,
+row_count INT,
+KEY(file, pos)) ENGINE=InnoDB;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+include/stop_slave.inc
+connection master;
+SET @orig_domain_id= @@gtid_domain_id;
+SET @orig_server_id= @@server_id;
+SET gtid_domain_id= @orig_domain_id;
+SET server_id= @orig_server_id;
+SELECT COUNT(*) FROM gtid_data;
+COUNT(*)
+1000
+connection slave;
+SET @orig_pos= @@GLOBAL.gtid_slave_pos;
+SET @orig_t1_limit= (SELECT MAX(a) FROM t1);
+*** Now connect the slave to each position in turn, and test that
+*** the right amount of data is replicated at each point.
+*** Test slave connecting to some GTID positions where the position in
+*** the master's binlog is different between the different domains.
+*** Revind the slave and test on the same binlog data from the master as before.
+connection slave;
+SET sql_log_bin= 0;
+TRUNCATE gtid_data;
+DELETE FROM t1 WHERE a > @orig_t1_limit;
+SET sql_log_bin= 1;
+SET GLOBAL gtid_slave_pos= @orig_pos;
+connection master;
+connection slave;
+connection master;
+DROP TABLE gtid_data, rand_data;
+include/save_master_gtid.inc
+connection slave;
+include/start_slave.inc
+include/sync_with_master_gtid.inc
+connection master;
+*** Switch to a larger binlog size
+SET @old_binlog_size= @@GLOBAL.max_binlog_size;
+SET GLOBAL max_binlog_size= 16*1024*1024;
+*** Testing 4000 GTIDs with 100 test connects
+connection master;
+DELETE FROM t1 WHERE a >= 1000;
+FLUSH NO_WRITE_TO_BINLOG BINARY LOGS;
+CREATE TABLE rand_data(idx INT PRIMARY KEY, domain_id INT, server_id INT)
+ENGINE=InnoDB;
+INSERT INTO rand_data(idx, domain_id, server_id) VALUES (0, 0, 1);
+INSERT INTO rand_data(idx, domain_id, server_id)
+SELECT seq,
+@tmp:=floor(10*POW(rand(666),2)),
+100 + 5*@tmp + floor(5*rand(666))
+FROM seq_1_to_4000;
+SELECT COUNT(*), SUM(domain_id), SUM(server_id) FROM rand_data;
+COUNT(*) SUM(domain_id) SUM(server_id)
+4001 11733 466721
+CREATE TABLE gtid_data(
+idx INT PRIMARY KEY,
+gtid VARCHAR(44),
+gtid_pos VARCHAR(255),
+file VARCHAR(100),
+pos INT,
+row_count INT,
+KEY(file, pos)) ENGINE=InnoDB;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+include/stop_slave.inc
+connection master;
+SET @orig_domain_id= @@gtid_domain_id;
+SET @orig_server_id= @@server_id;
+SET gtid_domain_id= @orig_domain_id;
+SET server_id= @orig_server_id;
+SELECT COUNT(*) FROM gtid_data;
+COUNT(*)
+4000
+connection slave;
+SET @orig_pos= @@GLOBAL.gtid_slave_pos;
+SET @orig_t1_limit= (SELECT MAX(a) FROM t1);
+*** Now connect the slave to each position in turn, and test that
+*** the right amount of data is replicated at each point.
+*** Test slave connecting to some GTID positions where the position in
+*** the master's binlog is different between the different domains.
+*** Revind the slave and test on the same binlog data from the master as before.
+connection slave;
+SET sql_log_bin= 0;
+TRUNCATE gtid_data;
+DELETE FROM t1 WHERE a > @orig_t1_limit;
+SET sql_log_bin= 1;
+SET GLOBAL gtid_slave_pos= @orig_pos;
+connection master;
+connection slave;
+connection master;
+DROP TABLE gtid_data, rand_data;
+include/save_master_gtid.inc
+connection slave;
+include/start_slave.inc
+include/sync_with_master_gtid.inc
+connection master;
+connection master;
+SET GLOBAL max_binlog_size= @old_binlog_size;
+DROP TABLE t1;
+DROP FUNCTION gtid_eq;
+include/rpl_end.inc
diff --git a/mysql-test/suite/binlog_in_engine/rpl_gtid_index.test b/mysql-test/suite/binlog_in_engine/rpl_gtid_index.test
new file mode 100644
index 0000000000000..c1b9dcbdf0041
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/rpl_gtid_index.test
@@ -0,0 +1,31 @@
+--source include/have_sequence.inc
+--source include/have_innodb.inc
+--source include/master-slave.inc
+--source include/have_binlog_format_mixed.inc
+
+--let $binlog_in_engine_test= 1
+--source suite/rpl/include/rpl_gtid_index_setup.inc
+
+
+--echo *** A lot of GTIDs with small binlog size and state interval
+--let $NUM_POS= 1000
+--let $NUM_DOMAIN= 10
+--let $NUM_SERVER= 5
+--let $NUM_SLAVE_CONNECTS= 100
+--let $RND_SEED= 150
+--source suite/rpl/include/rpl_gtid_index.inc
+
+--echo *** Switch to a larger binlog size
+SET @old_binlog_size= @@GLOBAL.max_binlog_size;
+SET GLOBAL max_binlog_size= 16*1024*1024;
+--let $NUM_POS= 4000
+--let $RND_SEED= 666
+--source suite/rpl/include/rpl_gtid_index.inc
+
+
+# Cleanup.
+--connection master
+SET GLOBAL max_binlog_size= @old_binlog_size;
+--source suite/rpl/include/rpl_gtid_index_cleanup.inc
+
+--source include/rpl_end.inc
diff --git a/mysql-test/suite/binlog_in_engine/rpl_heartbeat.result b/mysql-test/suite/binlog_in_engine/rpl_heartbeat.result
new file mode 100644
index 0000000000000..f8de38f53de95
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/rpl_heartbeat.result
@@ -0,0 +1,80 @@
+include/master-slave.inc
+[connection master]
+connection slave;
+include/stop_slave.inc
+connection master;
+include/kill_binlog_dump_threads.inc
+reset master;
+connection slave;
+set @restore_slave_net_timeout= @@global.slave_net_timeout;
+set @@global.slave_net_timeout= 10;
+change master to master_host='127.0.0.1',master_port=MASTER_PORT, master_user='root';
+show status like 'Slave_heartbeat_period';;
+Variable_name Slave_heartbeat_period
+Value 5.000
+change master to master_host='127.0.0.1',master_port=MASTER_PORT, master_user='root', master_heartbeat_period= 4294968;
+ERROR HY000: The requested value for the heartbeat period is either negative or exceeds the maximum allowed (4294967 seconds)
+show status like 'Slave_heartbeat_period';;
+Variable_name Slave_heartbeat_period
+Value 5.000
+connection slave;
+change master to master_host='127.0.0.1',master_port=MASTER_PORT, master_user='root', master_heartbeat_period= 0.0009999;
+Warnings:
+Warning 1703 The requested value for the heartbeat period is less than 1 millisecond. The value is reset to 0, meaning that heartbeating will effectively be disabled
+show status like 'Slave_heartbeat_period';;
+Variable_name Slave_heartbeat_period
+Value 0.000
+change master to master_host='127.0.0.1',master_port=MASTER_PORT, master_user='root', master_heartbeat_period= 4294967;
+Warnings:
+Warning 1704 The requested value for the heartbeat period exceeds the value of `slave_net_timeout' seconds. A sensible value for the period should be less than the timeout
+show status like 'Slave_heartbeat_period';;
+Variable_name Slave_heartbeat_period
+Value 4294967.000
+change master to master_host='127.0.0.1',master_port=MASTER_PORT, master_user='root', master_heartbeat_period= 0.001;
+show status like 'Slave_heartbeat_period';;
+Variable_name Slave_heartbeat_period
+Value 0.001
+reset slave;
+set @@global.slave_net_timeout= 5;
+change master to master_host='127.0.0.1',master_port=MASTER_PORT, master_user='root', master_heartbeat_period= 5.001;
+Warnings:
+Warning 1704 The requested value for the heartbeat period exceeds the value of `slave_net_timeout' seconds. A sensible value for the period should be less than the timeout
+show status like 'Slave_heartbeat_period';;
+Variable_name Slave_heartbeat_period
+Value 5.001
+reset slave;
+set @@global.slave_net_timeout= 5;
+change master to master_host='127.0.0.1',master_port=MASTER_PORT, master_user='root', master_heartbeat_period= 4;
+show status like 'Slave_heartbeat_period';;
+Variable_name Slave_heartbeat_period
+Value 4.000
+set @@global.slave_net_timeout= 3 /* must be a warning */;
+reset slave;
+connection master;
+drop table if exists t1;
+connection slave;
+set @@global.slave_net_timeout= 10;
+change master to master_host='127.0.0.1',master_port=MASTER_PORT, master_user='root', master_ssl_verify_server_cert=0, master_heartbeat_period= 0.5;
+show status like 'Slave_heartbeat_period';;
+Variable_name Slave_heartbeat_period
+Value 0.500
+start slave;
+connection master;
+create table t1 (f1 int);
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+include/check_slave_param.inc [Relay_Log_File]
+show status like 'Slave_heartbeat_period';;
+Variable_name Slave_heartbeat_period
+Value 0.500
+A heartbeat has been received by the slave
+connection master;
+drop table t1;
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+set @@global.slave_net_timeout= @restore_slave_net_timeout;
+include/rpl_end.inc
diff --git a/mysql-test/suite/binlog_in_engine/rpl_heartbeat.test b/mysql-test/suite/binlog_in_engine/rpl_heartbeat.test
new file mode 100644
index 0000000000000..d662330c3f3d0
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/rpl_heartbeat.test
@@ -0,0 +1,7 @@
+--source include/master-slave.inc
+--source include/have_binlog_format_mixed.inc
+--source include/have_innodb_binlog.inc
+
+--source include/rpl_heartbeat.inc
+
+--source include/rpl_end.inc
diff --git a/mysql-test/suite/binlog_in_engine/rpl_mysqlbinlog_slave_consistency_basic.result b/mysql-test/suite/binlog_in_engine/rpl_mysqlbinlog_slave_consistency_basic.result
new file mode 100644
index 0000000000000..3c165e13f7c4f
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/rpl_mysqlbinlog_slave_consistency_basic.result
@@ -0,0 +1,1364 @@
+include/master-slave.inc
+[connection master]
+connection slave;
+SET sql_log_bin=0;
+call mtr.add_suppression("Slave: An attempt was made.*");
+call mtr.add_suppression("Both DO_DOMAIN_IDS & IGNORE_DOMAIN_IDS lists can't be non-empty at the same time");
+SET sql_log_bin=1;
+include/stop_slave.inc
+connection master;
+include/kill_binlog_dump_threads.inc
+RESET MASTER;
+connection slave;
+include/start_slave.inc
+#
+#
+# Test Case 1: Base case to ensure that mariadb-binlog and replica
+# are consistent without any filtering
+#
+include/mysqlbinlog_slave_consistency.inc
+connection slave;
+set @@global.gtid_strict_mode=0;
+connection slave;
+connection master;
+# Populating master data
+connection slave;
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos;
+START SLAVE ;
+# No slave error expecting - waiting for slave to catch up to master
+# Stop slave so it stops receiving master events.
+connection master;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+RESET MASTER;
+# MYSQL_BINLOG BINLOG_FILE_PARAM 2> MYSQLBINLOG_STDERR | MYSQL
+connection slave;
+# Checking consistency of 'test' database tables between master and slave
+connection master;
+# Both servers have 5 tables
+# Verifying integrity of tables..
+connection master;
+connection slave;
+# t1 is equivalent on master and slave
+connection master;
+connection slave;
+# t2 is equivalent on master and slave
+connection master;
+connection slave;
+# t3 is equivalent on master and slave
+connection master;
+connection slave;
+# t4 is equivalent on master and slave
+connection master;
+connection slave;
+# t5 is equivalent on master and slave
+# All tables are consistent
+# Test finished - resetting master and slave..
+connection slave;
+RESET SLAVE;
+RESET MASTER;
+set global gtid_slave_pos="";
+CHANGE MASTER TO DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=(), IGNORE_SERVER_IDS=();
+connection master;
+RESET MASTER;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+connection slave;
+RESET SLAVE;
+set global gtid_slave_pos="";
+RESET MASTER;
+connection master;
+RESET MASTER;
+connection slave;
+connection master;
+#
+#
+# Test Case 2: Ensure filtering by a single id in --do-domain-ids is
+# consistent between mariadb-binlog and replica
+#
+include/mysqlbinlog_slave_consistency.inc
+connection slave;
+set @@global.gtid_strict_mode=0;
+connection slave;
+connection master;
+# Populating master data
+connection slave;
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos, DO_DOMAIN_IDS=(0);
+START SLAVE ;
+# No slave error expecting - waiting for slave to catch up to master
+# Stop slave so it stops receiving master events.
+connection master;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+RESET MASTER;
+# MYSQL_BINLOG BINLOG_FILE_PARAM --do-domain-ids=0 2> MYSQLBINLOG_STDERR | MYSQL
+connection slave;
+# Checking consistency of 'test' database tables between master and slave
+connection master;
+# Both servers have 2 tables
+# Verifying integrity of tables..
+connection master;
+connection slave;
+# t1 is equivalent on master and slave
+connection master;
+connection slave;
+# t2 is equivalent on master and slave
+# All tables are consistent
+# Test finished - resetting master and slave..
+connection slave;
+RESET SLAVE;
+RESET MASTER;
+set global gtid_slave_pos="";
+CHANGE MASTER TO DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=(), IGNORE_SERVER_IDS=();
+connection master;
+RESET MASTER;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+Warnings:
+Note 1051 Unknown table 'test.t3,test.t4,test.t5'
+connection slave;
+RESET SLAVE;
+set global gtid_slave_pos="";
+RESET MASTER;
+connection master;
+RESET MASTER;
+connection slave;
+connection master;
+#
+#
+# Test Case 3: Ensure filtering by multiple ids in --do-domain-ids is
+# consistent between mariadb-binlog and replica
+#
+include/mysqlbinlog_slave_consistency.inc
+connection slave;
+set @@global.gtid_strict_mode=0;
+connection slave;
+connection master;
+# Populating master data
+connection slave;
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos, DO_DOMAIN_IDS=(0,1);
+START SLAVE ;
+# No slave error expecting - waiting for slave to catch up to master
+# Stop slave so it stops receiving master events.
+connection master;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+RESET MASTER;
+# MYSQL_BINLOG BINLOG_FILE_PARAM --do-domain-ids=0,1 2> MYSQLBINLOG_STDERR | MYSQL
+connection slave;
+# Checking consistency of 'test' database tables between master and slave
+connection master;
+# Both servers have 4 tables
+# Verifying integrity of tables..
+connection master;
+connection slave;
+# t1 is equivalent on master and slave
+connection master;
+connection slave;
+# t2 is equivalent on master and slave
+connection master;
+connection slave;
+# t3 is equivalent on master and slave
+connection master;
+connection slave;
+# t4 is equivalent on master and slave
+# All tables are consistent
+# Test finished - resetting master and slave..
+connection slave;
+RESET SLAVE;
+RESET MASTER;
+set global gtid_slave_pos="";
+CHANGE MASTER TO DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=(), IGNORE_SERVER_IDS=();
+connection master;
+RESET MASTER;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+Warnings:
+Note 1051 Unknown table 'test.t5'
+connection slave;
+RESET SLAVE;
+set global gtid_slave_pos="";
+RESET MASTER;
+connection master;
+RESET MASTER;
+connection slave;
+connection master;
+#
+# Test Case 4: Ensure filtering by a single id in --ignore-domain-ids
+# is consistent between mariadb-binlog and replica
+#
+include/mysqlbinlog_slave_consistency.inc
+connection slave;
+set @@global.gtid_strict_mode=0;
+connection slave;
+connection master;
+# Populating master data
+connection slave;
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos, IGNORE_DOMAIN_IDS=(0);
+START SLAVE ;
+# No slave error expecting - waiting for slave to catch up to master
+# Stop slave so it stops receiving master events.
+connection master;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+RESET MASTER;
+# MYSQL_BINLOG BINLOG_FILE_PARAM --ignore-domain-ids=0 2> MYSQLBINLOG_STDERR | MYSQL
+connection slave;
+# Checking consistency of 'test' database tables between master and slave
+connection master;
+# Both servers have 3 tables
+# Verifying integrity of tables..
+connection master;
+connection slave;
+# t3 is equivalent on master and slave
+connection master;
+connection slave;
+# t4 is equivalent on master and slave
+connection master;
+connection slave;
+# t5 is equivalent on master and slave
+# All tables are consistent
+# Test finished - resetting master and slave..
+connection slave;
+RESET SLAVE;
+RESET MASTER;
+set global gtid_slave_pos="";
+CHANGE MASTER TO DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=(), IGNORE_SERVER_IDS=();
+connection master;
+RESET MASTER;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+Warnings:
+Note 1051 Unknown table 'test.t1,test.t2'
+connection slave;
+RESET SLAVE;
+set global gtid_slave_pos="";
+RESET MASTER;
+connection master;
+RESET MASTER;
+connection slave;
+connection master;
+#
+#
+# Test Case 5: Ensure filtering by multiple ids in --ignore-domain-ids
+# is consistent between mariadb-binlog and replica
+#
+include/mysqlbinlog_slave_consistency.inc
+connection slave;
+set @@global.gtid_strict_mode=0;
+connection slave;
+connection master;
+# Populating master data
+connection slave;
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos, IGNORE_DOMAIN_IDS=(0,1);
+START SLAVE ;
+# No slave error expecting - waiting for slave to catch up to master
+# Stop slave so it stops receiving master events.
+connection master;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+RESET MASTER;
+# MYSQL_BINLOG BINLOG_FILE_PARAM --ignore-domain-ids=0,1 2> MYSQLBINLOG_STDERR | MYSQL
+connection slave;
+# Checking consistency of 'test' database tables between master and slave
+connection master;
+# Both servers have 1 tables
+# Verifying integrity of tables..
+connection master;
+connection slave;
+# t5 is equivalent on master and slave
+# All tables are consistent
+# Test finished - resetting master and slave..
+connection slave;
+RESET SLAVE;
+RESET MASTER;
+set global gtid_slave_pos="";
+CHANGE MASTER TO DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=(), IGNORE_SERVER_IDS=();
+connection master;
+RESET MASTER;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+Warnings:
+Note 1051 Unknown table 'test.t1,test.t2,test.t3,test.t4'
+connection slave;
+RESET SLAVE;
+set global gtid_slave_pos="";
+RESET MASTER;
+connection master;
+RESET MASTER;
+connection slave;
+connection master;
+#
+# Test Case 6: Ensure filtering by a single id in --ignore-server-ids
+# is consistent between mariadb-binlog and replica
+#
+include/mysqlbinlog_slave_consistency.inc
+connection slave;
+set @@global.gtid_strict_mode=0;
+connection slave;
+connection master;
+# Populating master data
+connection slave;
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos, IGNORE_SERVER_IDS=(1);
+START SLAVE ;
+# No slave error expecting - waiting for slave to catch up to master
+# Stop slave so it stops receiving master events.
+connection master;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+RESET MASTER;
+# MYSQL_BINLOG BINLOG_FILE_PARAM --ignore-server-ids=1 2> MYSQLBINLOG_STDERR | MYSQL
+connection slave;
+# Checking consistency of 'test' database tables between master and slave
+connection master;
+# Both servers have 2 tables
+# Verifying integrity of tables..
+connection master;
+connection slave;
+# t2 is equivalent on master and slave
+connection master;
+connection slave;
+# t4 is equivalent on master and slave
+# All tables are consistent
+# Test finished - resetting master and slave..
+connection slave;
+RESET SLAVE;
+RESET MASTER;
+set global gtid_slave_pos="";
+CHANGE MASTER TO DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=(), IGNORE_SERVER_IDS=();
+connection master;
+RESET MASTER;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+Warnings:
+Note 1051 Unknown table 'test.t1,test.t3,test.t5'
+connection slave;
+RESET SLAVE;
+set global gtid_slave_pos="";
+RESET MASTER;
+connection master;
+RESET MASTER;
+connection slave;
+connection master;
+#
+#
+# Test Case 7: Ensure filtering by multiple ids in --ignore-server-ids
+# is consistent between mariadb-binlog and replica
+#
+include/mysqlbinlog_slave_consistency.inc
+connection slave;
+set @@global.gtid_strict_mode=0;
+connection slave;
+connection master;
+# Populating master data
+connection slave;
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos, IGNORE_SERVER_IDS=(0,1);
+START SLAVE ;
+# No slave error expecting - waiting for slave to catch up to master
+# Stop slave so it stops receiving master events.
+connection master;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+RESET MASTER;
+# MYSQL_BINLOG BINLOG_FILE_PARAM --ignore-server-ids=0,1 2> MYSQLBINLOG_STDERR | MYSQL
+connection slave;
+# Checking consistency of 'test' database tables between master and slave
+connection master;
+# Both servers have 2 tables
+# Verifying integrity of tables..
+connection master;
+connection slave;
+# t2 is equivalent on master and slave
+connection master;
+connection slave;
+# t4 is equivalent on master and slave
+# All tables are consistent
+# Test finished - resetting master and slave..
+connection slave;
+RESET SLAVE;
+RESET MASTER;
+set global gtid_slave_pos="";
+CHANGE MASTER TO DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=(), IGNORE_SERVER_IDS=();
+connection master;
+RESET MASTER;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+Warnings:
+Note 1051 Unknown table 'test.t1,test.t3,test.t5'
+connection slave;
+RESET SLAVE;
+set global gtid_slave_pos="";
+RESET MASTER;
+connection master;
+RESET MASTER;
+connection slave;
+connection master;
+#
+#
+# Test Case 8: Ensure stop position consistency
+#
+include/mysqlbinlog_slave_consistency.inc
+connection slave;
+set @@global.gtid_strict_mode=0;
+connection slave;
+connection master;
+# Populating master data
+connection slave;
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos;
+START SLAVE UNTIL master_gtid_pos="1-1-2";
+Warnings:
+Note 1278 It is recommended to use --skip-slave-start when doing step-by-step replication with START SLAVE UNTIL; otherwise, you will get problems if you get an unexpected slave's mariadbd restart
+# No slave error expecting - waiting for slave to catch up to master
+# Because there is a stop position we wait for all events to process
+# and slave to automatically stop
+# Stop slave so it stops receiving master events.
+Warnings:
+Note 1255 Slave already has been stopped
+connection master;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+RESET MASTER;
+# MYSQL_BINLOG BINLOG_FILE_PARAM --stop-position=1-1-2 2> MYSQLBINLOG_STDERR | MYSQL
+connection slave;
+# Checking consistency of 'test' database tables between master and slave
+connection master;
+# Both servers have 1 tables
+# Verifying integrity of tables..
+connection master;
+connection slave;
+# t3 is equivalent on master and slave
+# All tables are consistent
+# Test finished - resetting master and slave..
+connection slave;
+RESET SLAVE;
+RESET MASTER;
+set global gtid_slave_pos="";
+CHANGE MASTER TO DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=(), IGNORE_SERVER_IDS=();
+connection master;
+RESET MASTER;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+Warnings:
+Note 1051 Unknown table 'test.t1,test.t2,test.t4,test.t5'
+connection slave;
+RESET SLAVE;
+set global gtid_slave_pos="";
+RESET MASTER;
+connection master;
+RESET MASTER;
+connection slave;
+connection master;
+#
+#
+# Test Case 9: Ensure start position consistency
+#
+include/mysqlbinlog_slave_consistency.inc
+connection slave;
+set @@global.gtid_strict_mode=0;
+connection slave;
+connection master;
+# Populating master data
+connection slave;
+set global gtid_slave_pos="1-4-2";
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos;
+START SLAVE ;
+# No slave error expecting - waiting for slave to catch up to master
+# Stop slave so it stops receiving master events.
+connection master;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+RESET MASTER;
+# MYSQL_BINLOG BINLOG_FILE_PARAM --start-position=1-4-2 2> MYSQLBINLOG_STDERR | MYSQL
+connection slave;
+# Checking consistency of 'test' database tables between master and slave
+connection master;
+# Both servers have 4 tables
+# Verifying integrity of tables..
+connection master;
+connection slave;
+# t1 is equivalent on master and slave
+connection master;
+connection slave;
+# t2 is equivalent on master and slave
+connection master;
+connection slave;
+# t4 is equivalent on master and slave
+connection master;
+connection slave;
+# t5 is equivalent on master and slave
+# All tables are consistent
+# Test finished - resetting master and slave..
+connection slave;
+RESET SLAVE;
+RESET MASTER;
+set global gtid_slave_pos="";
+CHANGE MASTER TO DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=(), IGNORE_SERVER_IDS=();
+connection master;
+RESET MASTER;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+Warnings:
+Note 1051 Unknown table 'test.t3'
+connection slave;
+RESET SLAVE;
+set global gtid_slave_pos="";
+RESET MASTER;
+connection master;
+RESET MASTER;
+connection slave;
+connection master;
+#
+#
+# Test Case 10: Ensure consistency when filtering by both
+# --do-domain-ids and --ignore-server-ids
+#
+include/mysqlbinlog_slave_consistency.inc
+connection slave;
+set @@global.gtid_strict_mode=0;
+connection slave;
+connection master;
+# Populating master data
+connection slave;
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos, DO_DOMAIN_IDS=(0), IGNORE_SERVER_IDS=(1);
+START SLAVE ;
+# No slave error expecting - waiting for slave to catch up to master
+# Stop slave so it stops receiving master events.
+connection master;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+RESET MASTER;
+# MYSQL_BINLOG BINLOG_FILE_PARAM --do-domain-ids=0 --ignore-server-ids=1 2> MYSQLBINLOG_STDERR | MYSQL
+connection slave;
+# Checking consistency of 'test' database tables between master and slave
+connection master;
+# Both servers have 1 tables
+# Verifying integrity of tables..
+connection master;
+connection slave;
+# t2 is equivalent on master and slave
+# All tables are consistent
+# Test finished - resetting master and slave..
+connection slave;
+RESET SLAVE;
+RESET MASTER;
+set global gtid_slave_pos="";
+CHANGE MASTER TO DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=(), IGNORE_SERVER_IDS=();
+connection master;
+RESET MASTER;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+Warnings:
+Note 1051 Unknown table 'test.t1,test.t3,test.t4,test.t5'
+connection slave;
+RESET SLAVE;
+set global gtid_slave_pos="";
+RESET MASTER;
+connection master;
+RESET MASTER;
+connection slave;
+connection master;
+#
+#
+# Test Case 11: Ensure consistency when filtering by both
+# --ignore-domain-ids and --ignore-server-ids
+#
+include/mysqlbinlog_slave_consistency.inc
+connection slave;
+set @@global.gtid_strict_mode=0;
+connection slave;
+connection master;
+# Populating master data
+connection slave;
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos, IGNORE_DOMAIN_IDS=(1,2), IGNORE_SERVER_IDS=(1);
+START SLAVE ;
+# No slave error expecting - waiting for slave to catch up to master
+# Stop slave so it stops receiving master events.
+connection master;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+RESET MASTER;
+# MYSQL_BINLOG BINLOG_FILE_PARAM --ignore-domain-ids=1,2 --ignore-server-ids=1 2> MYSQLBINLOG_STDERR | MYSQL
+connection slave;
+# Checking consistency of 'test' database tables between master and slave
+connection master;
+# Both servers have 1 tables
+# Verifying integrity of tables..
+connection master;
+connection slave;
+# t2 is equivalent on master and slave
+# All tables are consistent
+# Test finished - resetting master and slave..
+connection slave;
+RESET SLAVE;
+RESET MASTER;
+set global gtid_slave_pos="";
+CHANGE MASTER TO DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=(), IGNORE_SERVER_IDS=();
+connection master;
+RESET MASTER;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+Warnings:
+Note 1051 Unknown table 'test.t1,test.t3,test.t4,test.t5'
+connection slave;
+RESET SLAVE;
+set global gtid_slave_pos="";
+RESET MASTER;
+connection master;
+RESET MASTER;
+connection slave;
+connection master;
+#
+#
+# Test Case 12: Ensure consistency when filtering by
+# --do-domain-ids with a stop position
+#
+include/mysqlbinlog_slave_consistency.inc
+connection slave;
+set @@global.gtid_strict_mode=0;
+connection slave;
+connection master;
+# Populating master data
+connection slave;
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos, DO_DOMAIN_IDS=(0);
+START SLAVE UNTIL master_gtid_pos="0-1-4";
+Warnings:
+Note 1278 It is recommended to use --skip-slave-start when doing step-by-step replication with START SLAVE UNTIL; otherwise, you will get problems if you get an unexpected slave's mariadbd restart
+# No slave error expecting - waiting for slave to catch up to master
+# Because there is a stop position we wait for all events to process
+# and slave to automatically stop
+# Stop slave so it stops receiving master events.
+Warnings:
+Note 1255 Slave already has been stopped
+connection master;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+RESET MASTER;
+# MYSQL_BINLOG BINLOG_FILE_PARAM --do-domain-ids=0 --stop-position=0-1-4 2> MYSQLBINLOG_STDERR | MYSQL
+connection slave;
+# Checking consistency of 'test' database tables between master and slave
+connection master;
+# Both servers have 2 tables
+# Verifying integrity of tables..
+connection master;
+connection slave;
+# t1 is equivalent on master and slave
+connection master;
+connection slave;
+# t2 is equivalent on master and slave
+# All tables are consistent
+# Test finished - resetting master and slave..
+connection slave;
+RESET SLAVE;
+RESET MASTER;
+set global gtid_slave_pos="";
+CHANGE MASTER TO DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=(), IGNORE_SERVER_IDS=();
+connection master;
+RESET MASTER;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+Warnings:
+Note 1051 Unknown table 'test.t3,test.t4,test.t5'
+connection slave;
+RESET SLAVE;
+set global gtid_slave_pos="";
+RESET MASTER;
+connection master;
+RESET MASTER;
+connection slave;
+connection master;
+#
+#
+# Test Case 13: Ensure consistency when filtering by
+# --ignore-domain-ids with a stop position
+#
+include/mysqlbinlog_slave_consistency.inc
+connection slave;
+set @@global.gtid_strict_mode=0;
+connection slave;
+connection master;
+# Populating master data
+connection slave;
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos, IGNORE_DOMAIN_IDS=(0);
+START SLAVE UNTIL master_gtid_pos="0-1-4";
+Warnings:
+Note 1278 It is recommended to use --skip-slave-start when doing step-by-step replication with START SLAVE UNTIL; otherwise, you will get problems if you get an unexpected slave's mariadbd restart
+# No slave error expecting - waiting for slave to catch up to master
+# Because there is a stop position we wait for all events to process
+# and slave to automatically stop
+# Stop slave so it stops receiving master events.
+Warnings:
+Note 1255 Slave already has been stopped
+connection master;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+RESET MASTER;
+# MYSQL_BINLOG BINLOG_FILE_PARAM --ignore-domain-ids=0 --stop-position=0-1-4 2> MYSQLBINLOG_STDERR | MYSQL
+connection slave;
+# Checking consistency of 'test' database tables between master and slave
+connection master;
+# Both servers have 0 tables
+# Verifying integrity of tables..
+# All tables are consistent
+# Test finished - resetting master and slave..
+connection slave;
+RESET SLAVE;
+RESET MASTER;
+set global gtid_slave_pos="";
+CHANGE MASTER TO DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=(), IGNORE_SERVER_IDS=();
+connection master;
+RESET MASTER;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+Warnings:
+Note 1051 Unknown table 'test.t1,test.t2,test.t3,test.t4,test.t5'
+connection slave;
+RESET SLAVE;
+set global gtid_slave_pos="";
+RESET MASTER;
+connection master;
+RESET MASTER;
+connection slave;
+connection master;
+#
+#
+# Test Case 14: Ensure consistency when filtering by
+# --ignore-server-ids with a stop position
+#
+include/mysqlbinlog_slave_consistency.inc
+connection slave;
+set @@global.gtid_strict_mode=0;
+connection slave;
+connection master;
+# Populating master data
+connection slave;
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos, IGNORE_SERVER_IDS=(3);
+START SLAVE UNTIL master_gtid_pos="0-1-4";
+Warnings:
+Note 1278 It is recommended to use --skip-slave-start when doing step-by-step replication with START SLAVE UNTIL; otherwise, you will get problems if you get an unexpected slave's mariadbd restart
+# No slave error expecting - waiting for slave to catch up to master
+# Because there is a stop position we wait for all events to process
+# and slave to automatically stop
+# Stop slave so it stops receiving master events.
+Warnings:
+Note 1255 Slave already has been stopped
+connection master;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+RESET MASTER;
+# MYSQL_BINLOG BINLOG_FILE_PARAM --ignore-server-ids=3 --stop-position=0-1-4 2> MYSQLBINLOG_STDERR | MYSQL
+connection slave;
+# Checking consistency of 'test' database tables between master and slave
+connection master;
+# Both servers have 1 tables
+# Verifying integrity of tables..
+connection master;
+connection slave;
+# t1 is equivalent on master and slave
+# All tables are consistent
+# Test finished - resetting master and slave..
+connection slave;
+RESET SLAVE;
+RESET MASTER;
+set global gtid_slave_pos="";
+CHANGE MASTER TO DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=(), IGNORE_SERVER_IDS=();
+connection master;
+RESET MASTER;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+Warnings:
+Note 1051 Unknown table 'test.t2,test.t3,test.t4,test.t5'
+connection slave;
+RESET SLAVE;
+set global gtid_slave_pos="";
+RESET MASTER;
+connection master;
+RESET MASTER;
+connection slave;
+connection master;
+#
+#
+# Test Case 15: Ensure consistency when filtering by
+# --do-domain-ids with a start position
+#
+include/mysqlbinlog_slave_consistency.inc
+connection slave;
+set @@global.gtid_strict_mode=0;
+connection slave;
+connection master;
+# Populating master data
+connection slave;
+set global gtid_slave_pos="1-1-2";
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos, DO_DOMAIN_IDS=(2);
+START SLAVE ;
+# No slave error expecting - waiting for slave to catch up to master
+# Stop slave so it stops receiving master events.
+connection master;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+RESET MASTER;
+# MYSQL_BINLOG BINLOG_FILE_PARAM --start-position=1-1-2 --do-domain-ids=2 2> MYSQLBINLOG_STDERR | MYSQL
+connection slave;
+# Checking consistency of 'test' database tables between master and slave
+connection master;
+# Both servers have 1 tables
+# Verifying integrity of tables..
+connection master;
+connection slave;
+# t5 is equivalent on master and slave
+# All tables are consistent
+# Test finished - resetting master and slave..
+connection slave;
+RESET SLAVE;
+RESET MASTER;
+set global gtid_slave_pos="";
+CHANGE MASTER TO DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=(), IGNORE_SERVER_IDS=();
+connection master;
+RESET MASTER;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+Warnings:
+Note 1051 Unknown table 'test.t1,test.t2,test.t3,test.t4'
+connection slave;
+RESET SLAVE;
+set global gtid_slave_pos="";
+RESET MASTER;
+connection master;
+RESET MASTER;
+connection slave;
+connection master;
+#
+#
+# Test Case 16: Ensure consistency when filtering by
+# --ignore-domain-ids with a start position
+#
+include/mysqlbinlog_slave_consistency.inc
+connection slave;
+set @@global.gtid_strict_mode=0;
+connection slave;
+connection master;
+# Populating master data
+connection slave;
+set global gtid_slave_pos="0-1-1";
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos, IGNORE_DOMAIN_IDS=(0);
+START SLAVE ;
+# No slave error expecting - waiting for slave to catch up to master
+# Stop slave so it stops receiving master events.
+connection master;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+RESET MASTER;
+# MYSQL_BINLOG BINLOG_FILE_PARAM --start-position=0-1-1 --ignore-domain-ids=0 2> MYSQLBINLOG_STDERR | MYSQL
+connection slave;
+# Checking consistency of 'test' database tables between master and slave
+connection master;
+# Both servers have 3 tables
+# Verifying integrity of tables..
+connection master;
+connection slave;
+# t3 is equivalent on master and slave
+connection master;
+connection slave;
+# t4 is equivalent on master and slave
+connection master;
+connection slave;
+# t5 is equivalent on master and slave
+# All tables are consistent
+# Test finished - resetting master and slave..
+connection slave;
+RESET SLAVE;
+RESET MASTER;
+set global gtid_slave_pos="";
+CHANGE MASTER TO DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=(), IGNORE_SERVER_IDS=();
+connection master;
+RESET MASTER;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+Warnings:
+Note 1051 Unknown table 'test.t1,test.t2'
+connection slave;
+RESET SLAVE;
+set global gtid_slave_pos="";
+RESET MASTER;
+connection master;
+RESET MASTER;
+connection slave;
+connection master;
+#
+#
+# Test Case 17: Ensure consistency when filtering by
+# --ignore-server-ids with a start position
+#
+include/mysqlbinlog_slave_consistency.inc
+connection slave;
+set @@global.gtid_strict_mode=0;
+connection slave;
+connection master;
+# Populating master data
+connection slave;
+set global gtid_slave_pos="0-1-1";
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos, IGNORE_SERVER_IDS=(1);
+START SLAVE ;
+# No slave error expecting - waiting for slave to catch up to master
+# Stop slave so it stops receiving master events.
+connection master;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+RESET MASTER;
+# MYSQL_BINLOG BINLOG_FILE_PARAM --start-position=0-1-1 --ignore-server-ids=1 2> MYSQLBINLOG_STDERR | MYSQL
+connection slave;
+# Checking consistency of 'test' database tables between master and slave
+connection master;
+# Both servers have 2 tables
+# Verifying integrity of tables..
+connection master;
+connection slave;
+# t2 is equivalent on master and slave
+connection master;
+connection slave;
+# t4 is equivalent on master and slave
+# All tables are consistent
+# Test finished - resetting master and slave..
+connection slave;
+RESET SLAVE;
+RESET MASTER;
+set global gtid_slave_pos="";
+CHANGE MASTER TO DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=(), IGNORE_SERVER_IDS=();
+connection master;
+RESET MASTER;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+Warnings:
+Note 1051 Unknown table 'test.t1,test.t3,test.t5'
+connection slave;
+RESET SLAVE;
+set global gtid_slave_pos="";
+RESET MASTER;
+connection master;
+RESET MASTER;
+connection slave;
+connection master;
+#
+#
+# Test Case 18: Ensure consistency when filtering by
+# --do-domain-ids with both a start position and stop position that
+# all have the same domain id
+#
+include/mysqlbinlog_slave_consistency.inc
+connection slave;
+set @@global.gtid_strict_mode=0;
+connection slave;
+connection master;
+# Populating master data
+connection slave;
+set global gtid_slave_pos="0-1-1";
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos, DO_DOMAIN_IDS=(0);
+START SLAVE UNTIL master_gtid_pos="0-3-3";
+Warnings:
+Note 1278 It is recommended to use --skip-slave-start when doing step-by-step replication with START SLAVE UNTIL; otherwise, you will get problems if you get an unexpected slave's mariadbd restart
+# No slave error expecting - waiting for slave to catch up to master
+# Because there is a stop position we wait for all events to process
+# and slave to automatically stop
+# Stop slave so it stops receiving master events.
+Warnings:
+Note 1255 Slave already has been stopped
+connection master;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+RESET MASTER;
+# MYSQL_BINLOG BINLOG_FILE_PARAM --start-position=0-1-1 --do-domain-ids=0 --stop-position=0-3-3 2> MYSQLBINLOG_STDERR | MYSQL
+connection slave;
+# Checking consistency of 'test' database tables between master and slave
+connection master;
+# Both servers have 1 tables
+# Verifying integrity of tables..
+connection master;
+connection slave;
+# t2 is equivalent on master and slave
+# All tables are consistent
+# Test finished - resetting master and slave..
+connection slave;
+RESET SLAVE;
+RESET MASTER;
+set global gtid_slave_pos="";
+CHANGE MASTER TO DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=(), IGNORE_SERVER_IDS=();
+connection master;
+RESET MASTER;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+Warnings:
+Note 1051 Unknown table 'test.t1,test.t3,test.t4,test.t5'
+connection slave;
+RESET SLAVE;
+set global gtid_slave_pos="";
+RESET MASTER;
+connection master;
+RESET MASTER;
+connection slave;
+connection master;
+#
+#
+# Test Case 19: Ensure consistency when filtering by
+# --do-domain-ids with both a start position and stop position that
+# have differing domain ids. Due to the implicit filtering in stop
+# position, the result should be empty (no tables replicated).
+#
+include/mysqlbinlog_slave_consistency.inc
+connection slave;
+set @@global.gtid_strict_mode=0;
+connection slave;
+connection master;
+# Populating master data
+connection slave;
+set global gtid_slave_pos="0-1-1";
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos, DO_DOMAIN_IDS=(1);
+START SLAVE UNTIL master_gtid_pos="0-3-3";
+Warnings:
+Note 1278 It is recommended to use --skip-slave-start when doing step-by-step replication with START SLAVE UNTIL; otherwise, you will get problems if you get an unexpected slave's mariadbd restart
+# No slave error expecting - waiting for slave to catch up to master
+# Because there is a stop position we wait for all events to process
+# and slave to automatically stop
+# Stop slave so it stops receiving master events.
+Warnings:
+Note 1255 Slave already has been stopped
+connection master;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+RESET MASTER;
+# MYSQL_BINLOG BINLOG_FILE_PARAM --start-position=0-1-1 --do-domain-ids=1 --stop-position=0-3-3 2> MYSQLBINLOG_STDERR | MYSQL
+connection slave;
+# Checking consistency of 'test' database tables between master and slave
+connection master;
+# Both servers have 0 tables
+# Verifying integrity of tables..
+# All tables are consistent
+# Test finished - resetting master and slave..
+connection slave;
+RESET SLAVE;
+RESET MASTER;
+set global gtid_slave_pos="";
+CHANGE MASTER TO DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=(), IGNORE_SERVER_IDS=();
+connection master;
+RESET MASTER;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+Warnings:
+Note 1051 Unknown table 'test.t1,test.t2,test.t3,test.t4,test.t5'
+connection slave;
+RESET SLAVE;
+set global gtid_slave_pos="";
+RESET MASTER;
+connection master;
+RESET MASTER;
+connection slave;
+connection master;
+#
+#
+# Test Case 20: Ensure consistency when filtering by
+# --ignore-domain-ids with both a start position and stop position that
+# all have the same domain id. The result set should be empty due to
+# implicit filtering from stop position and ignoring that same domain.
+#
+include/mysqlbinlog_slave_consistency.inc
+connection slave;
+set @@global.gtid_strict_mode=0;
+connection slave;
+connection master;
+# Populating master data
+connection slave;
+set global gtid_slave_pos="0-1-1";
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos, IGNORE_DOMAIN_IDS=(0);
+START SLAVE UNTIL master_gtid_pos="0-3-3";
+Warnings:
+Note 1278 It is recommended to use --skip-slave-start when doing step-by-step replication with START SLAVE UNTIL; otherwise, you will get problems if you get an unexpected slave's mariadbd restart
+# No slave error expecting - waiting for slave to catch up to master
+# Because there is a stop position we wait for all events to process
+# and slave to automatically stop
+# Stop slave so it stops receiving master events.
+Warnings:
+Note 1255 Slave already has been stopped
+connection master;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+RESET MASTER;
+# MYSQL_BINLOG BINLOG_FILE_PARAM --start-position=0-1-1 --ignore-domain-ids=0 --stop-position=0-3-3 2> MYSQLBINLOG_STDERR | MYSQL
+connection slave;
+# Checking consistency of 'test' database tables between master and slave
+connection master;
+# Both servers have 0 tables
+# Verifying integrity of tables..
+# All tables are consistent
+# Test finished - resetting master and slave..
+connection slave;
+RESET SLAVE;
+RESET MASTER;
+set global gtid_slave_pos="";
+CHANGE MASTER TO DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=(), IGNORE_SERVER_IDS=();
+connection master;
+RESET MASTER;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+Warnings:
+Note 1051 Unknown table 'test.t1,test.t2,test.t3,test.t4,test.t5'
+connection slave;
+RESET SLAVE;
+set global gtid_slave_pos="";
+RESET MASTER;
+connection master;
+RESET MASTER;
+connection slave;
+connection master;
+#
+#
+# Test Case 21: Ensure consistency when filtering by
+# --ignore-domain-ids with both a start position and stop position that
+# have differing domain ids. The ignore domain ids should take no
+# effect due to the implicit filtering by stop position, i.e. the
+# specified domain to ignore is already being filtered.
+#
+include/mysqlbinlog_slave_consistency.inc
+connection slave;
+set @@global.gtid_strict_mode=0;
+connection slave;
+connection master;
+# Populating master data
+connection slave;
+set global gtid_slave_pos="0-1-1";
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos, IGNORE_DOMAIN_IDS=(1);
+START SLAVE UNTIL master_gtid_pos="0-3-3";
+Warnings:
+Note 1278 It is recommended to use --skip-slave-start when doing step-by-step replication with START SLAVE UNTIL; otherwise, you will get problems if you get an unexpected slave's mariadbd restart
+# No slave error expecting - waiting for slave to catch up to master
+# Because there is a stop position we wait for all events to process
+# and slave to automatically stop
+# Stop slave so it stops receiving master events.
+Warnings:
+Note 1255 Slave already has been stopped
+connection master;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+RESET MASTER;
+# MYSQL_BINLOG BINLOG_FILE_PARAM --start-position=0-1-1 --ignore-domain-ids=1 --stop-position=0-3-3 2> MYSQLBINLOG_STDERR | MYSQL
+connection slave;
+# Checking consistency of 'test' database tables between master and slave
+connection master;
+# Both servers have 1 tables
+# Verifying integrity of tables..
+connection master;
+connection slave;
+# t2 is equivalent on master and slave
+# All tables are consistent
+# Test finished - resetting master and slave..
+connection slave;
+RESET SLAVE;
+RESET MASTER;
+set global gtid_slave_pos="";
+CHANGE MASTER TO DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=(), IGNORE_SERVER_IDS=();
+connection master;
+RESET MASTER;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+Warnings:
+Note 1051 Unknown table 'test.t1,test.t3,test.t4,test.t5'
+connection slave;
+RESET SLAVE;
+set global gtid_slave_pos="";
+RESET MASTER;
+connection master;
+RESET MASTER;
+connection slave;
+connection master;
+#
+#
+# Test Case 22: Ensure consistency when filtering by
+# --ignore-server-ids with both a start position and stop position.
+#
+include/mysqlbinlog_slave_consistency.inc
+connection slave;
+set @@global.gtid_strict_mode=0;
+connection slave;
+connection master;
+# Populating master data
+connection slave;
+set global gtid_slave_pos="0-1-0";
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos, IGNORE_SERVER_IDS=(3);
+START SLAVE UNTIL master_gtid_pos="0-3-3";
+Warnings:
+Note 1278 It is recommended to use --skip-slave-start when doing step-by-step replication with START SLAVE UNTIL; otherwise, you will get problems if you get an unexpected slave's mariadbd restart
+# No slave error expecting - waiting for slave to catch up to master
+# Because there is a stop position we wait for all events to process
+# and slave to automatically stop
+# Stop slave so it stops receiving master events.
+Warnings:
+Note 1255 Slave already has been stopped
+connection master;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+RESET MASTER;
+# MYSQL_BINLOG BINLOG_FILE_PARAM --start-position=0-1-0 --ignore-server-ids=3 --stop-position=0-3-3 2> MYSQLBINLOG_STDERR | MYSQL
+connection slave;
+# Checking consistency of 'test' database tables between master and slave
+connection master;
+# Both servers have 1 tables
+# Verifying integrity of tables..
+connection master;
+connection slave;
+# t1 is equivalent on master and slave
+# All tables are consistent
+# Test finished - resetting master and slave..
+connection slave;
+RESET SLAVE;
+RESET MASTER;
+set global gtid_slave_pos="";
+CHANGE MASTER TO DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=(), IGNORE_SERVER_IDS=();
+connection master;
+RESET MASTER;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+Warnings:
+Note 1051 Unknown table 'test.t2,test.t3,test.t4,test.t5'
+connection slave;
+RESET SLAVE;
+set global gtid_slave_pos="";
+RESET MASTER;
+connection master;
+RESET MASTER;
+connection slave;
+connection master;
+#
+#
+# Test Case 23: Out of order GTIDs from domains or servers which are
+# filtered out should not error
+#
+include/mysqlbinlog_slave_consistency.inc
+connection slave;
+set @@global.gtid_strict_mode=1;
+connection slave;
+connection master;
+# Populating master data
+connection slave;
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos, DO_DOMAIN_IDS=(0), IGNORE_SERVER_IDS=(3);
+START SLAVE ;
+# No slave error expecting - waiting for slave to catch up to master
+# Stop slave so it stops receiving master events.
+connection master;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+Warnings:
+Note 1051 Unknown table 'test.t4,test.t5'
+RESET MASTER;
+# MYSQL_BINLOG BINLOG_FILE_PARAM --do-domain-ids=0 --ignore-server-ids=3 --gtid-strict-mode 2> MYSQLBINLOG_STDERR | MYSQL
+connection slave;
+# Checking consistency of 'test' database tables between master and slave
+connection master;
+# Both servers have 1 tables
+# Verifying integrity of tables..
+connection master;
+connection slave;
+# t1 is equivalent on master and slave
+# All tables are consistent
+# Strict mode enabled - checking mysqlbinlog error output for out
+# of order GTIDs
+# Not expecting to find out of order GTID error..
+NOT FOUND /Found out of order GTID/ in mysqlbinlog_stderr.out
+# Test finished - resetting master and slave..
+connection slave;
+RESET SLAVE;
+RESET MASTER;
+set global gtid_slave_pos="";
+CHANGE MASTER TO DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=(), IGNORE_SERVER_IDS=();
+connection master;
+RESET MASTER;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+Warnings:
+Note 1051 Unknown table 'test.t2,test.t3,test.t4,test.t5'
+connection slave;
+RESET SLAVE;
+set global gtid_slave_pos="";
+RESET MASTER;
+connection master;
+RESET MASTER;
+connection slave;
+set @@global.gtid_strict_mode=0;
+connection master;
+#
+#
+# Test Case 24: Out of order GTIDs from included domains should error
+#
+include/mysqlbinlog_slave_consistency.inc
+connection slave;
+set @@global.gtid_strict_mode=1;
+connection slave;
+connection master;
+# Populating master data
+connection slave;
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos, DO_DOMAIN_IDS=(1);
+START SLAVE ;
+# slave SQL Thread error expected - waiting for errno 1950
+# Stop slave so it stops receiving master events.
+connection master;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+Warnings:
+Note 1051 Unknown table 'test.t4,test.t5'
+RESET MASTER;
+# MYSQL_BINLOG BINLOG_FILE_PARAM --do-domain-ids=1 --gtid-strict-mode 2> MYSQLBINLOG_STDERR | MYSQL
+connection slave;
+# Checking consistency of 'test' database tables between master and slave
+connection master;
+# Both servers have 1 tables
+# Verifying integrity of tables..
+connection master;
+connection slave;
+# t3 is equivalent on master and slave
+# All tables are consistent
+# Strict mode enabled - checking mysqlbinlog error output for out
+# of order GTIDs
+# Expecting to find out of order GTID error..
+FOUND 1 /Found out of order GTID/ in mysqlbinlog_stderr.out
+# Test finished - resetting master and slave..
+connection slave;
+RESET SLAVE;
+RESET MASTER;
+set global gtid_slave_pos="";
+CHANGE MASTER TO DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=(), IGNORE_SERVER_IDS=();
+connection master;
+RESET MASTER;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+Warnings:
+Note 1051 Unknown table 'test.t1,test.t2,test.t4,test.t5'
+connection slave;
+RESET SLAVE;
+set global gtid_slave_pos="";
+RESET MASTER;
+connection master;
+RESET MASTER;
+connection slave;
+set @@global.gtid_strict_mode=0;
+connection master;
+#
+#
+# Test Case 25: Out of order GTIDs from included servers should error
+#
+include/mysqlbinlog_slave_consistency.inc
+connection slave;
+set @@global.gtid_strict_mode=1;
+connection slave;
+connection master;
+# Populating master data
+connection slave;
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos, IGNORE_SERVER_IDS=(1);
+START SLAVE ;
+# slave SQL Thread error expected - waiting for errno 1950
+# Stop slave so it stops receiving master events.
+connection master;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+Warnings:
+Note 1051 Unknown table 'test.t4,test.t5'
+RESET MASTER;
+# MYSQL_BINLOG BINLOG_FILE_PARAM --ignore-server-ids=1 --gtid-strict-mode 2> MYSQLBINLOG_STDERR | MYSQL
+connection slave;
+# Checking consistency of 'test' database tables between master and slave
+connection master;
+# Both servers have 1 tables
+# Verifying integrity of tables..
+connection master;
+connection slave;
+# t2 is equivalent on master and slave
+# All tables are consistent
+# Strict mode enabled - checking mysqlbinlog error output for out
+# of order GTIDs
+# Expecting to find out of order GTID error..
+FOUND 1 /Found out of order GTID/ in mysqlbinlog_stderr.out
+# Test finished - resetting master and slave..
+connection slave;
+RESET SLAVE;
+RESET MASTER;
+set global gtid_slave_pos="";
+CHANGE MASTER TO DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=(), IGNORE_SERVER_IDS=();
+connection master;
+RESET MASTER;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+Warnings:
+Note 1051 Unknown table 'test.t1,test.t3,test.t4,test.t5'
+connection slave;
+RESET SLAVE;
+set global gtid_slave_pos="";
+RESET MASTER;
+connection master;
+RESET MASTER;
+connection slave;
+set @@global.gtid_strict_mode=0;
+connection master;
+#
+#
+# Test Case 26: Neither mysqlbinlog nor CHANGE MASTER TO should allow
+# both do domain ids and ignore domain ids to be set together
+#
+connection slave;
+include/stop_slave.inc
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos, DO_DOMAIN_IDS=(0), IGNORE_DOMAIN_IDS=(1);
+ERROR HY000: Could not initialize master info structure for ''; more error messages can be found in the MariaDB error log
+RESET MASTER;
+set global gtid_slave_pos="";
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos, DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=();
+connection master;
+include/kill_binlog_dump_threads.inc
+RESET MASTER;
+connection slave;
+include/start_slave.inc
+include/rpl_end.inc
+# End of binlog_in_engine.rpl_mysqlbinlog_slave_consistency_basic.test
diff --git a/mysql-test/suite/binlog_in_engine/rpl_mysqlbinlog_slave_consistency_basic.test b/mysql-test/suite/binlog_in_engine/rpl_mysqlbinlog_slave_consistency_basic.test
new file mode 100644
index 0000000000000..99ef29776c6f0
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/rpl_mysqlbinlog_slave_consistency_basic.test
@@ -0,0 +1,8 @@
+# For test description, see mysql-test/suite/rpl/t/rpl_mysqlbinlog_slave_consistency.inc
+--source include/have_innodb.inc
+--source include/master-slave.inc
+--source include/have_innodb_binlog.inc
+--let $blob_trx_size= 1
+--source suite/rpl/t/rpl_mysqlbinlog_slave_consistency.inc
+--source include/rpl_end.inc
+--echo # End of binlog_in_engine.rpl_mysqlbinlog_slave_consistency_basic.test
diff --git a/mysql-test/suite/binlog_in_engine/rpl_mysqlbinlog_slave_consistency_oob.result b/mysql-test/suite/binlog_in_engine/rpl_mysqlbinlog_slave_consistency_oob.result
new file mode 100644
index 0000000000000..f980b721becc4
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/rpl_mysqlbinlog_slave_consistency_oob.result
@@ -0,0 +1,1364 @@
+include/master-slave.inc
+[connection master]
+connection slave;
+SET sql_log_bin=0;
+call mtr.add_suppression("Slave: An attempt was made.*");
+call mtr.add_suppression("Both DO_DOMAIN_IDS & IGNORE_DOMAIN_IDS lists can't be non-empty at the same time");
+SET sql_log_bin=1;
+include/stop_slave.inc
+connection master;
+include/kill_binlog_dump_threads.inc
+RESET MASTER;
+connection slave;
+include/start_slave.inc
+#
+#
+# Test Case 1: Base case to ensure that mariadb-binlog and replica
+# are consistent without any filtering
+#
+include/mysqlbinlog_slave_consistency.inc
+connection slave;
+set @@global.gtid_strict_mode=0;
+connection slave;
+connection master;
+# Populating master data
+connection slave;
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos;
+START SLAVE ;
+# No slave error expecting - waiting for slave to catch up to master
+# Stop slave so it stops receiving master events.
+connection master;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+RESET MASTER;
+# MYSQL_BINLOG BINLOG_FILE_PARAM 2> MYSQLBINLOG_STDERR | MYSQL
+connection slave;
+# Checking consistency of 'test' database tables between master and slave
+connection master;
+# Both servers have 5 tables
+# Verifying integrity of tables..
+connection master;
+connection slave;
+# t1 is equivalent on master and slave
+connection master;
+connection slave;
+# t2 is equivalent on master and slave
+connection master;
+connection slave;
+# t3 is equivalent on master and slave
+connection master;
+connection slave;
+# t4 is equivalent on master and slave
+connection master;
+connection slave;
+# t5 is equivalent on master and slave
+# All tables are consistent
+# Test finished - resetting master and slave..
+connection slave;
+RESET SLAVE;
+RESET MASTER;
+set global gtid_slave_pos="";
+CHANGE MASTER TO DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=(), IGNORE_SERVER_IDS=();
+connection master;
+RESET MASTER;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+connection slave;
+RESET SLAVE;
+set global gtid_slave_pos="";
+RESET MASTER;
+connection master;
+RESET MASTER;
+connection slave;
+connection master;
+#
+#
+# Test Case 2: Ensure filtering by a single id in --do-domain-ids is
+# consistent between mariadb-binlog and replica
+#
+include/mysqlbinlog_slave_consistency.inc
+connection slave;
+set @@global.gtid_strict_mode=0;
+connection slave;
+connection master;
+# Populating master data
+connection slave;
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos, DO_DOMAIN_IDS=(0);
+START SLAVE ;
+# No slave error expecting - waiting for slave to catch up to master
+# Stop slave so it stops receiving master events.
+connection master;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+RESET MASTER;
+# MYSQL_BINLOG BINLOG_FILE_PARAM --do-domain-ids=0 2> MYSQLBINLOG_STDERR | MYSQL
+connection slave;
+# Checking consistency of 'test' database tables between master and slave
+connection master;
+# Both servers have 2 tables
+# Verifying integrity of tables..
+connection master;
+connection slave;
+# t1 is equivalent on master and slave
+connection master;
+connection slave;
+# t2 is equivalent on master and slave
+# All tables are consistent
+# Test finished - resetting master and slave..
+connection slave;
+RESET SLAVE;
+RESET MASTER;
+set global gtid_slave_pos="";
+CHANGE MASTER TO DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=(), IGNORE_SERVER_IDS=();
+connection master;
+RESET MASTER;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+Warnings:
+Note 1051 Unknown table 'test.t3,test.t4,test.t5'
+connection slave;
+RESET SLAVE;
+set global gtid_slave_pos="";
+RESET MASTER;
+connection master;
+RESET MASTER;
+connection slave;
+connection master;
+#
+#
+# Test Case 3: Ensure filtering by multiple ids in --do-domain-ids is
+# consistent between mariadb-binlog and replica
+#
+include/mysqlbinlog_slave_consistency.inc
+connection slave;
+set @@global.gtid_strict_mode=0;
+connection slave;
+connection master;
+# Populating master data
+connection slave;
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos, DO_DOMAIN_IDS=(0,1);
+START SLAVE ;
+# No slave error expecting - waiting for slave to catch up to master
+# Stop slave so it stops receiving master events.
+connection master;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+RESET MASTER;
+# MYSQL_BINLOG BINLOG_FILE_PARAM --do-domain-ids=0,1 2> MYSQLBINLOG_STDERR | MYSQL
+connection slave;
+# Checking consistency of 'test' database tables between master and slave
+connection master;
+# Both servers have 4 tables
+# Verifying integrity of tables..
+connection master;
+connection slave;
+# t1 is equivalent on master and slave
+connection master;
+connection slave;
+# t2 is equivalent on master and slave
+connection master;
+connection slave;
+# t3 is equivalent on master and slave
+connection master;
+connection slave;
+# t4 is equivalent on master and slave
+# All tables are consistent
+# Test finished - resetting master and slave..
+connection slave;
+RESET SLAVE;
+RESET MASTER;
+set global gtid_slave_pos="";
+CHANGE MASTER TO DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=(), IGNORE_SERVER_IDS=();
+connection master;
+RESET MASTER;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+Warnings:
+Note 1051 Unknown table 'test.t5'
+connection slave;
+RESET SLAVE;
+set global gtid_slave_pos="";
+RESET MASTER;
+connection master;
+RESET MASTER;
+connection slave;
+connection master;
+#
+# Test Case 4: Ensure filtering by a single id in --ignore-domain-ids
+# is consistent between mariadb-binlog and replica
+#
+include/mysqlbinlog_slave_consistency.inc
+connection slave;
+set @@global.gtid_strict_mode=0;
+connection slave;
+connection master;
+# Populating master data
+connection slave;
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos, IGNORE_DOMAIN_IDS=(0);
+START SLAVE ;
+# No slave error expecting - waiting for slave to catch up to master
+# Stop slave so it stops receiving master events.
+connection master;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+RESET MASTER;
+# MYSQL_BINLOG BINLOG_FILE_PARAM --ignore-domain-ids=0 2> MYSQLBINLOG_STDERR | MYSQL
+connection slave;
+# Checking consistency of 'test' database tables between master and slave
+connection master;
+# Both servers have 3 tables
+# Verifying integrity of tables..
+connection master;
+connection slave;
+# t3 is equivalent on master and slave
+connection master;
+connection slave;
+# t4 is equivalent on master and slave
+connection master;
+connection slave;
+# t5 is equivalent on master and slave
+# All tables are consistent
+# Test finished - resetting master and slave..
+connection slave;
+RESET SLAVE;
+RESET MASTER;
+set global gtid_slave_pos="";
+CHANGE MASTER TO DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=(), IGNORE_SERVER_IDS=();
+connection master;
+RESET MASTER;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+Warnings:
+Note 1051 Unknown table 'test.t1,test.t2'
+connection slave;
+RESET SLAVE;
+set global gtid_slave_pos="";
+RESET MASTER;
+connection master;
+RESET MASTER;
+connection slave;
+connection master;
+#
+#
+# Test Case 5: Ensure filtering by multiple ids in --ignore-domain-ids
+# is consistent between mariadb-binlog and replica
+#
+include/mysqlbinlog_slave_consistency.inc
+connection slave;
+set @@global.gtid_strict_mode=0;
+connection slave;
+connection master;
+# Populating master data
+connection slave;
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos, IGNORE_DOMAIN_IDS=(0,1);
+START SLAVE ;
+# No slave error expecting - waiting for slave to catch up to master
+# Stop slave so it stops receiving master events.
+connection master;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+RESET MASTER;
+# MYSQL_BINLOG BINLOG_FILE_PARAM --ignore-domain-ids=0,1 2> MYSQLBINLOG_STDERR | MYSQL
+connection slave;
+# Checking consistency of 'test' database tables between master and slave
+connection master;
+# Both servers have 1 tables
+# Verifying integrity of tables..
+connection master;
+connection slave;
+# t5 is equivalent on master and slave
+# All tables are consistent
+# Test finished - resetting master and slave..
+connection slave;
+RESET SLAVE;
+RESET MASTER;
+set global gtid_slave_pos="";
+CHANGE MASTER TO DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=(), IGNORE_SERVER_IDS=();
+connection master;
+RESET MASTER;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+Warnings:
+Note 1051 Unknown table 'test.t1,test.t2,test.t3,test.t4'
+connection slave;
+RESET SLAVE;
+set global gtid_slave_pos="";
+RESET MASTER;
+connection master;
+RESET MASTER;
+connection slave;
+connection master;
+#
+# Test Case 6: Ensure filtering by a single id in --ignore-server-ids
+# is consistent between mariadb-binlog and replica
+#
+include/mysqlbinlog_slave_consistency.inc
+connection slave;
+set @@global.gtid_strict_mode=0;
+connection slave;
+connection master;
+# Populating master data
+connection slave;
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos, IGNORE_SERVER_IDS=(1);
+START SLAVE ;
+# No slave error expecting - waiting for slave to catch up to master
+# Stop slave so it stops receiving master events.
+connection master;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+RESET MASTER;
+# MYSQL_BINLOG BINLOG_FILE_PARAM --ignore-server-ids=1 2> MYSQLBINLOG_STDERR | MYSQL
+connection slave;
+# Checking consistency of 'test' database tables between master and slave
+connection master;
+# Both servers have 2 tables
+# Verifying integrity of tables..
+connection master;
+connection slave;
+# t2 is equivalent on master and slave
+connection master;
+connection slave;
+# t4 is equivalent on master and slave
+# All tables are consistent
+# Test finished - resetting master and slave..
+connection slave;
+RESET SLAVE;
+RESET MASTER;
+set global gtid_slave_pos="";
+CHANGE MASTER TO DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=(), IGNORE_SERVER_IDS=();
+connection master;
+RESET MASTER;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+Warnings:
+Note 1051 Unknown table 'test.t1,test.t3,test.t5'
+connection slave;
+RESET SLAVE;
+set global gtid_slave_pos="";
+RESET MASTER;
+connection master;
+RESET MASTER;
+connection slave;
+connection master;
+#
+#
+# Test Case 7: Ensure filtering by multiple ids in --ignore-server-ids
+# is consistent between mariadb-binlog and replica
+#
+include/mysqlbinlog_slave_consistency.inc
+connection slave;
+set @@global.gtid_strict_mode=0;
+connection slave;
+connection master;
+# Populating master data
+connection slave;
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos, IGNORE_SERVER_IDS=(0,1);
+START SLAVE ;
+# No slave error expecting - waiting for slave to catch up to master
+# Stop slave so it stops receiving master events.
+connection master;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+RESET MASTER;
+# MYSQL_BINLOG BINLOG_FILE_PARAM --ignore-server-ids=0,1 2> MYSQLBINLOG_STDERR | MYSQL
+connection slave;
+# Checking consistency of 'test' database tables between master and slave
+connection master;
+# Both servers have 2 tables
+# Verifying integrity of tables..
+connection master;
+connection slave;
+# t2 is equivalent on master and slave
+connection master;
+connection slave;
+# t4 is equivalent on master and slave
+# All tables are consistent
+# Test finished - resetting master and slave..
+connection slave;
+RESET SLAVE;
+RESET MASTER;
+set global gtid_slave_pos="";
+CHANGE MASTER TO DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=(), IGNORE_SERVER_IDS=();
+connection master;
+RESET MASTER;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+Warnings:
+Note 1051 Unknown table 'test.t1,test.t3,test.t5'
+connection slave;
+RESET SLAVE;
+set global gtid_slave_pos="";
+RESET MASTER;
+connection master;
+RESET MASTER;
+connection slave;
+connection master;
+#
+#
+# Test Case 8: Ensure stop position consistency
+#
+include/mysqlbinlog_slave_consistency.inc
+connection slave;
+set @@global.gtid_strict_mode=0;
+connection slave;
+connection master;
+# Populating master data
+connection slave;
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos;
+START SLAVE UNTIL master_gtid_pos="1-1-2";
+Warnings:
+Note 1278 It is recommended to use --skip-slave-start when doing step-by-step replication with START SLAVE UNTIL; otherwise, you will get problems if you get an unexpected slave's mariadbd restart
+# No slave error expecting - waiting for slave to catch up to master
+# Because there is a stop position we wait for all events to process
+# and slave to automatically stop
+# Stop slave so it stops receiving master events.
+Warnings:
+Note 1255 Slave already has been stopped
+connection master;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+RESET MASTER;
+# MYSQL_BINLOG BINLOG_FILE_PARAM --stop-position=1-1-2 2> MYSQLBINLOG_STDERR | MYSQL
+connection slave;
+# Checking consistency of 'test' database tables between master and slave
+connection master;
+# Both servers have 1 tables
+# Verifying integrity of tables..
+connection master;
+connection slave;
+# t3 is equivalent on master and slave
+# All tables are consistent
+# Test finished - resetting master and slave..
+connection slave;
+RESET SLAVE;
+RESET MASTER;
+set global gtid_slave_pos="";
+CHANGE MASTER TO DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=(), IGNORE_SERVER_IDS=();
+connection master;
+RESET MASTER;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+Warnings:
+Note 1051 Unknown table 'test.t1,test.t2,test.t4,test.t5'
+connection slave;
+RESET SLAVE;
+set global gtid_slave_pos="";
+RESET MASTER;
+connection master;
+RESET MASTER;
+connection slave;
+connection master;
+#
+#
+# Test Case 9: Ensure start position consistency
+#
+include/mysqlbinlog_slave_consistency.inc
+connection slave;
+set @@global.gtid_strict_mode=0;
+connection slave;
+connection master;
+# Populating master data
+connection slave;
+set global gtid_slave_pos="1-4-2";
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos;
+START SLAVE ;
+# No slave error expecting - waiting for slave to catch up to master
+# Stop slave so it stops receiving master events.
+connection master;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+RESET MASTER;
+# MYSQL_BINLOG BINLOG_FILE_PARAM --start-position=1-4-2 2> MYSQLBINLOG_STDERR | MYSQL
+connection slave;
+# Checking consistency of 'test' database tables between master and slave
+connection master;
+# Both servers have 4 tables
+# Verifying integrity of tables..
+connection master;
+connection slave;
+# t1 is equivalent on master and slave
+connection master;
+connection slave;
+# t2 is equivalent on master and slave
+connection master;
+connection slave;
+# t4 is equivalent on master and slave
+connection master;
+connection slave;
+# t5 is equivalent on master and slave
+# All tables are consistent
+# Test finished - resetting master and slave..
+connection slave;
+RESET SLAVE;
+RESET MASTER;
+set global gtid_slave_pos="";
+CHANGE MASTER TO DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=(), IGNORE_SERVER_IDS=();
+connection master;
+RESET MASTER;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+Warnings:
+Note 1051 Unknown table 'test.t3'
+connection slave;
+RESET SLAVE;
+set global gtid_slave_pos="";
+RESET MASTER;
+connection master;
+RESET MASTER;
+connection slave;
+connection master;
+#
+#
+# Test Case 10: Ensure consistency when filtering by both
+# --do-domain-ids and --ignore-server-ids
+#
+include/mysqlbinlog_slave_consistency.inc
+connection slave;
+set @@global.gtid_strict_mode=0;
+connection slave;
+connection master;
+# Populating master data
+connection slave;
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos, DO_DOMAIN_IDS=(0), IGNORE_SERVER_IDS=(1);
+START SLAVE ;
+# No slave error expecting - waiting for slave to catch up to master
+# Stop slave so it stops receiving master events.
+connection master;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+RESET MASTER;
+# MYSQL_BINLOG BINLOG_FILE_PARAM --do-domain-ids=0 --ignore-server-ids=1 2> MYSQLBINLOG_STDERR | MYSQL
+connection slave;
+# Checking consistency of 'test' database tables between master and slave
+connection master;
+# Both servers have 1 tables
+# Verifying integrity of tables..
+connection master;
+connection slave;
+# t2 is equivalent on master and slave
+# All tables are consistent
+# Test finished - resetting master and slave..
+connection slave;
+RESET SLAVE;
+RESET MASTER;
+set global gtid_slave_pos="";
+CHANGE MASTER TO DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=(), IGNORE_SERVER_IDS=();
+connection master;
+RESET MASTER;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+Warnings:
+Note 1051 Unknown table 'test.t1,test.t3,test.t4,test.t5'
+connection slave;
+RESET SLAVE;
+set global gtid_slave_pos="";
+RESET MASTER;
+connection master;
+RESET MASTER;
+connection slave;
+connection master;
+#
+#
+# Test Case 11: Ensure consistency when filtering by both
+# --ignore-domain-ids and --ignore-server-ids
+#
+include/mysqlbinlog_slave_consistency.inc
+connection slave;
+set @@global.gtid_strict_mode=0;
+connection slave;
+connection master;
+# Populating master data
+connection slave;
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos, IGNORE_DOMAIN_IDS=(1,2), IGNORE_SERVER_IDS=(1);
+START SLAVE ;
+# No slave error expecting - waiting for slave to catch up to master
+# Stop slave so it stops receiving master events.
+connection master;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+RESET MASTER;
+# MYSQL_BINLOG BINLOG_FILE_PARAM --ignore-domain-ids=1,2 --ignore-server-ids=1 2> MYSQLBINLOG_STDERR | MYSQL
+connection slave;
+# Checking consistency of 'test' database tables between master and slave
+connection master;
+# Both servers have 1 tables
+# Verifying integrity of tables..
+connection master;
+connection slave;
+# t2 is equivalent on master and slave
+# All tables are consistent
+# Test finished - resetting master and slave..
+connection slave;
+RESET SLAVE;
+RESET MASTER;
+set global gtid_slave_pos="";
+CHANGE MASTER TO DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=(), IGNORE_SERVER_IDS=();
+connection master;
+RESET MASTER;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+Warnings:
+Note 1051 Unknown table 'test.t1,test.t3,test.t4,test.t5'
+connection slave;
+RESET SLAVE;
+set global gtid_slave_pos="";
+RESET MASTER;
+connection master;
+RESET MASTER;
+connection slave;
+connection master;
+#
+#
+# Test Case 12: Ensure consistency when filtering by
+# --do-domain-ids with a stop position
+#
+include/mysqlbinlog_slave_consistency.inc
+connection slave;
+set @@global.gtid_strict_mode=0;
+connection slave;
+connection master;
+# Populating master data
+connection slave;
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos, DO_DOMAIN_IDS=(0);
+START SLAVE UNTIL master_gtid_pos="0-1-4";
+Warnings:
+Note 1278 It is recommended to use --skip-slave-start when doing step-by-step replication with START SLAVE UNTIL; otherwise, you will get problems if you get an unexpected slave's mariadbd restart
+# No slave error expecting - waiting for slave to catch up to master
+# Because there is a stop position we wait for all events to process
+# and slave to automatically stop
+# Stop slave so it stops receiving master events.
+Warnings:
+Note 1255 Slave already has been stopped
+connection master;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+RESET MASTER;
+# MYSQL_BINLOG BINLOG_FILE_PARAM --do-domain-ids=0 --stop-position=0-1-4 2> MYSQLBINLOG_STDERR | MYSQL
+connection slave;
+# Checking consistency of 'test' database tables between master and slave
+connection master;
+# Both servers have 2 tables
+# Verifying integrity of tables..
+connection master;
+connection slave;
+# t1 is equivalent on master and slave
+connection master;
+connection slave;
+# t2 is equivalent on master and slave
+# All tables are consistent
+# Test finished - resetting master and slave..
+connection slave;
+RESET SLAVE;
+RESET MASTER;
+set global gtid_slave_pos="";
+CHANGE MASTER TO DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=(), IGNORE_SERVER_IDS=();
+connection master;
+RESET MASTER;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+Warnings:
+Note 1051 Unknown table 'test.t3,test.t4,test.t5'
+connection slave;
+RESET SLAVE;
+set global gtid_slave_pos="";
+RESET MASTER;
+connection master;
+RESET MASTER;
+connection slave;
+connection master;
+#
+#
+# Test Case 13: Ensure consistency when filtering by
+# --ignore-domain-ids with a stop position
+#
+include/mysqlbinlog_slave_consistency.inc
+connection slave;
+set @@global.gtid_strict_mode=0;
+connection slave;
+connection master;
+# Populating master data
+connection slave;
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos, IGNORE_DOMAIN_IDS=(0);
+START SLAVE UNTIL master_gtid_pos="0-1-4";
+Warnings:
+Note 1278 It is recommended to use --skip-slave-start when doing step-by-step replication with START SLAVE UNTIL; otherwise, you will get problems if you get an unexpected slave's mariadbd restart
+# No slave error expecting - waiting for slave to catch up to master
+# Because there is a stop position we wait for all events to process
+# and slave to automatically stop
+# Stop slave so it stops receiving master events.
+Warnings:
+Note 1255 Slave already has been stopped
+connection master;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+RESET MASTER;
+# MYSQL_BINLOG BINLOG_FILE_PARAM --ignore-domain-ids=0 --stop-position=0-1-4 2> MYSQLBINLOG_STDERR | MYSQL
+connection slave;
+# Checking consistency of 'test' database tables between master and slave
+connection master;
+# Both servers have 0 tables
+# Verifying integrity of tables..
+# All tables are consistent
+# Test finished - resetting master and slave..
+connection slave;
+RESET SLAVE;
+RESET MASTER;
+set global gtid_slave_pos="";
+CHANGE MASTER TO DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=(), IGNORE_SERVER_IDS=();
+connection master;
+RESET MASTER;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+Warnings:
+Note 1051 Unknown table 'test.t1,test.t2,test.t3,test.t4,test.t5'
+connection slave;
+RESET SLAVE;
+set global gtid_slave_pos="";
+RESET MASTER;
+connection master;
+RESET MASTER;
+connection slave;
+connection master;
+#
+#
+# Test Case 14: Ensure consistency when filtering by
+# --ignore-server-ids with a stop position
+#
+include/mysqlbinlog_slave_consistency.inc
+connection slave;
+set @@global.gtid_strict_mode=0;
+connection slave;
+connection master;
+# Populating master data
+connection slave;
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos, IGNORE_SERVER_IDS=(3);
+START SLAVE UNTIL master_gtid_pos="0-1-4";
+Warnings:
+Note 1278 It is recommended to use --skip-slave-start when doing step-by-step replication with START SLAVE UNTIL; otherwise, you will get problems if you get an unexpected slave's mariadbd restart
+# No slave error expecting - waiting for slave to catch up to master
+# Because there is a stop position we wait for all events to process
+# and slave to automatically stop
+# Stop slave so it stops receiving master events.
+Warnings:
+Note 1255 Slave already has been stopped
+connection master;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+RESET MASTER;
+# MYSQL_BINLOG BINLOG_FILE_PARAM --ignore-server-ids=3 --stop-position=0-1-4 2> MYSQLBINLOG_STDERR | MYSQL
+connection slave;
+# Checking consistency of 'test' database tables between master and slave
+connection master;
+# Both servers have 1 tables
+# Verifying integrity of tables..
+connection master;
+connection slave;
+# t1 is equivalent on master and slave
+# All tables are consistent
+# Test finished - resetting master and slave..
+connection slave;
+RESET SLAVE;
+RESET MASTER;
+set global gtid_slave_pos="";
+CHANGE MASTER TO DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=(), IGNORE_SERVER_IDS=();
+connection master;
+RESET MASTER;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+Warnings:
+Note 1051 Unknown table 'test.t2,test.t3,test.t4,test.t5'
+connection slave;
+RESET SLAVE;
+set global gtid_slave_pos="";
+RESET MASTER;
+connection master;
+RESET MASTER;
+connection slave;
+connection master;
+#
+#
+# Test Case 15: Ensure consistency when filtering by
+# --do-domain-ids with a start position
+#
+include/mysqlbinlog_slave_consistency.inc
+connection slave;
+set @@global.gtid_strict_mode=0;
+connection slave;
+connection master;
+# Populating master data
+connection slave;
+set global gtid_slave_pos="1-1-2";
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos, DO_DOMAIN_IDS=(2);
+START SLAVE ;
+# No slave error expecting - waiting for slave to catch up to master
+# Stop slave so it stops receiving master events.
+connection master;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+RESET MASTER;
+# MYSQL_BINLOG BINLOG_FILE_PARAM --start-position=1-1-2 --do-domain-ids=2 2> MYSQLBINLOG_STDERR | MYSQL
+connection slave;
+# Checking consistency of 'test' database tables between master and slave
+connection master;
+# Both servers have 1 tables
+# Verifying integrity of tables..
+connection master;
+connection slave;
+# t5 is equivalent on master and slave
+# All tables are consistent
+# Test finished - resetting master and slave..
+connection slave;
+RESET SLAVE;
+RESET MASTER;
+set global gtid_slave_pos="";
+CHANGE MASTER TO DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=(), IGNORE_SERVER_IDS=();
+connection master;
+RESET MASTER;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+Warnings:
+Note 1051 Unknown table 'test.t1,test.t2,test.t3,test.t4'
+connection slave;
+RESET SLAVE;
+set global gtid_slave_pos="";
+RESET MASTER;
+connection master;
+RESET MASTER;
+connection slave;
+connection master;
+#
+#
+# Test Case 16: Ensure consistency when filtering by
+# --ignore-domain-ids with a start position
+#
+include/mysqlbinlog_slave_consistency.inc
+connection slave;
+set @@global.gtid_strict_mode=0;
+connection slave;
+connection master;
+# Populating master data
+connection slave;
+set global gtid_slave_pos="0-1-1";
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos, IGNORE_DOMAIN_IDS=(0);
+START SLAVE ;
+# No slave error expecting - waiting for slave to catch up to master
+# Stop slave so it stops receiving master events.
+connection master;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+RESET MASTER;
+# MYSQL_BINLOG BINLOG_FILE_PARAM --start-position=0-1-1 --ignore-domain-ids=0 2> MYSQLBINLOG_STDERR | MYSQL
+connection slave;
+# Checking consistency of 'test' database tables between master and slave
+connection master;
+# Both servers have 3 tables
+# Verifying integrity of tables..
+connection master;
+connection slave;
+# t3 is equivalent on master and slave
+connection master;
+connection slave;
+# t4 is equivalent on master and slave
+connection master;
+connection slave;
+# t5 is equivalent on master and slave
+# All tables are consistent
+# Test finished - resetting master and slave..
+connection slave;
+RESET SLAVE;
+RESET MASTER;
+set global gtid_slave_pos="";
+CHANGE MASTER TO DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=(), IGNORE_SERVER_IDS=();
+connection master;
+RESET MASTER;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+Warnings:
+Note 1051 Unknown table 'test.t1,test.t2'
+connection slave;
+RESET SLAVE;
+set global gtid_slave_pos="";
+RESET MASTER;
+connection master;
+RESET MASTER;
+connection slave;
+connection master;
+#
+#
+# Test Case 17: Ensure consistency when filtering by
+# --ignore-server-ids with a start position
+#
+include/mysqlbinlog_slave_consistency.inc
+connection slave;
+set @@global.gtid_strict_mode=0;
+connection slave;
+connection master;
+# Populating master data
+connection slave;
+set global gtid_slave_pos="0-1-1";
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos, IGNORE_SERVER_IDS=(1);
+START SLAVE ;
+# No slave error expecting - waiting for slave to catch up to master
+# Stop slave so it stops receiving master events.
+connection master;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+RESET MASTER;
+# MYSQL_BINLOG BINLOG_FILE_PARAM --start-position=0-1-1 --ignore-server-ids=1 2> MYSQLBINLOG_STDERR | MYSQL
+connection slave;
+# Checking consistency of 'test' database tables between master and slave
+connection master;
+# Both servers have 2 tables
+# Verifying integrity of tables..
+connection master;
+connection slave;
+# t2 is equivalent on master and slave
+connection master;
+connection slave;
+# t4 is equivalent on master and slave
+# All tables are consistent
+# Test finished - resetting master and slave..
+connection slave;
+RESET SLAVE;
+RESET MASTER;
+set global gtid_slave_pos="";
+CHANGE MASTER TO DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=(), IGNORE_SERVER_IDS=();
+connection master;
+RESET MASTER;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+Warnings:
+Note 1051 Unknown table 'test.t1,test.t3,test.t5'
+connection slave;
+RESET SLAVE;
+set global gtid_slave_pos="";
+RESET MASTER;
+connection master;
+RESET MASTER;
+connection slave;
+connection master;
+#
+#
+# Test Case 18: Ensure consistency when filtering by
+# --do-domain-ids with both a start position and stop position that
+# all have the same domain id
+#
+include/mysqlbinlog_slave_consistency.inc
+connection slave;
+set @@global.gtid_strict_mode=0;
+connection slave;
+connection master;
+# Populating master data
+connection slave;
+set global gtid_slave_pos="0-1-1";
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos, DO_DOMAIN_IDS=(0);
+START SLAVE UNTIL master_gtid_pos="0-3-3";
+Warnings:
+Note 1278 It is recommended to use --skip-slave-start when doing step-by-step replication with START SLAVE UNTIL; otherwise, you will get problems if you get an unexpected slave's mariadbd restart
+# No slave error expecting - waiting for slave to catch up to master
+# Because there is a stop position we wait for all events to process
+# and slave to automatically stop
+# Stop slave so it stops receiving master events.
+Warnings:
+Note 1255 Slave already has been stopped
+connection master;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+RESET MASTER;
+# MYSQL_BINLOG BINLOG_FILE_PARAM --start-position=0-1-1 --do-domain-ids=0 --stop-position=0-3-3 2> MYSQLBINLOG_STDERR | MYSQL
+connection slave;
+# Checking consistency of 'test' database tables between master and slave
+connection master;
+# Both servers have 1 tables
+# Verifying integrity of tables..
+connection master;
+connection slave;
+# t2 is equivalent on master and slave
+# All tables are consistent
+# Test finished - resetting master and slave..
+connection slave;
+RESET SLAVE;
+RESET MASTER;
+set global gtid_slave_pos="";
+CHANGE MASTER TO DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=(), IGNORE_SERVER_IDS=();
+connection master;
+RESET MASTER;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+Warnings:
+Note 1051 Unknown table 'test.t1,test.t3,test.t4,test.t5'
+connection slave;
+RESET SLAVE;
+set global gtid_slave_pos="";
+RESET MASTER;
+connection master;
+RESET MASTER;
+connection slave;
+connection master;
+#
+#
+# Test Case 19: Ensure consistency when filtering by
+# --do-domain-ids with both a start position and stop position that
+# have differing domain ids. Due to the implicit filtering in stop
+# position, the result should be empty (no tables replicated).
+#
+include/mysqlbinlog_slave_consistency.inc
+connection slave;
+set @@global.gtid_strict_mode=0;
+connection slave;
+connection master;
+# Populating master data
+connection slave;
+set global gtid_slave_pos="0-1-1";
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos, DO_DOMAIN_IDS=(1);
+START SLAVE UNTIL master_gtid_pos="0-3-3";
+Warnings:
+Note 1278 It is recommended to use --skip-slave-start when doing step-by-step replication with START SLAVE UNTIL; otherwise, you will get problems if you get an unexpected slave's mariadbd restart
+# No slave error expecting - waiting for slave to catch up to master
+# Because there is a stop position we wait for all events to process
+# and slave to automatically stop
+# Stop slave so it stops receiving master events.
+Warnings:
+Note 1255 Slave already has been stopped
+connection master;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+RESET MASTER;
+# MYSQL_BINLOG BINLOG_FILE_PARAM --start-position=0-1-1 --do-domain-ids=1 --stop-position=0-3-3 2> MYSQLBINLOG_STDERR | MYSQL
+connection slave;
+# Checking consistency of 'test' database tables between master and slave
+connection master;
+# Both servers have 0 tables
+# Verifying integrity of tables..
+# All tables are consistent
+# Test finished - resetting master and slave..
+connection slave;
+RESET SLAVE;
+RESET MASTER;
+set global gtid_slave_pos="";
+CHANGE MASTER TO DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=(), IGNORE_SERVER_IDS=();
+connection master;
+RESET MASTER;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+Warnings:
+Note 1051 Unknown table 'test.t1,test.t2,test.t3,test.t4,test.t5'
+connection slave;
+RESET SLAVE;
+set global gtid_slave_pos="";
+RESET MASTER;
+connection master;
+RESET MASTER;
+connection slave;
+connection master;
+#
+#
+# Test Case 20: Ensure consistency when filtering by
+# --ignore-domain-ids with both a start position and stop position that
+# all have the same domain id. The result set should be empty due to
+# implicit filtering from stop position and ignoring that same domain.
+#
+include/mysqlbinlog_slave_consistency.inc
+connection slave;
+set @@global.gtid_strict_mode=0;
+connection slave;
+connection master;
+# Populating master data
+connection slave;
+set global gtid_slave_pos="0-1-1";
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos, IGNORE_DOMAIN_IDS=(0);
+START SLAVE UNTIL master_gtid_pos="0-3-3";
+Warnings:
+Note 1278 It is recommended to use --skip-slave-start when doing step-by-step replication with START SLAVE UNTIL; otherwise, you will get problems if you get an unexpected slave's mariadbd restart
+# No slave error expecting - waiting for slave to catch up to master
+# Because there is a stop position we wait for all events to process
+# and slave to automatically stop
+# Stop slave so it stops receiving master events.
+Warnings:
+Note 1255 Slave already has been stopped
+connection master;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+RESET MASTER;
+# MYSQL_BINLOG BINLOG_FILE_PARAM --start-position=0-1-1 --ignore-domain-ids=0 --stop-position=0-3-3 2> MYSQLBINLOG_STDERR | MYSQL
+connection slave;
+# Checking consistency of 'test' database tables between master and slave
+connection master;
+# Both servers have 0 tables
+# Verifying integrity of tables..
+# All tables are consistent
+# Test finished - resetting master and slave..
+connection slave;
+RESET SLAVE;
+RESET MASTER;
+set global gtid_slave_pos="";
+CHANGE MASTER TO DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=(), IGNORE_SERVER_IDS=();
+connection master;
+RESET MASTER;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+Warnings:
+Note 1051 Unknown table 'test.t1,test.t2,test.t3,test.t4,test.t5'
+connection slave;
+RESET SLAVE;
+set global gtid_slave_pos="";
+RESET MASTER;
+connection master;
+RESET MASTER;
+connection slave;
+connection master;
+#
+#
+# Test Case 21: Ensure consistency when filtering by
+# --ignore-domain-ids with both a start position and stop position that
+# have differing domain ids. The ignore domain ids should take no
+# effect due to the implicit filtering by stop position, i.e. the
+# specified domain to ignore is already being filtered.
+#
+include/mysqlbinlog_slave_consistency.inc
+connection slave;
+set @@global.gtid_strict_mode=0;
+connection slave;
+connection master;
+# Populating master data
+connection slave;
+set global gtid_slave_pos="0-1-1";
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos, IGNORE_DOMAIN_IDS=(1);
+START SLAVE UNTIL master_gtid_pos="0-3-3";
+Warnings:
+Note 1278 It is recommended to use --skip-slave-start when doing step-by-step replication with START SLAVE UNTIL; otherwise, you will get problems if you get an unexpected slave's mariadbd restart
+# No slave error expecting - waiting for slave to catch up to master
+# Because there is a stop position we wait for all events to process
+# and slave to automatically stop
+# Stop slave so it stops receiving master events.
+Warnings:
+Note 1255 Slave already has been stopped
+connection master;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+RESET MASTER;
+# MYSQL_BINLOG BINLOG_FILE_PARAM --start-position=0-1-1 --ignore-domain-ids=1 --stop-position=0-3-3 2> MYSQLBINLOG_STDERR | MYSQL
+connection slave;
+# Checking consistency of 'test' database tables between master and slave
+connection master;
+# Both servers have 1 tables
+# Verifying integrity of tables..
+connection master;
+connection slave;
+# t2 is equivalent on master and slave
+# All tables are consistent
+# Test finished - resetting master and slave..
+connection slave;
+RESET SLAVE;
+RESET MASTER;
+set global gtid_slave_pos="";
+CHANGE MASTER TO DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=(), IGNORE_SERVER_IDS=();
+connection master;
+RESET MASTER;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+Warnings:
+Note 1051 Unknown table 'test.t1,test.t3,test.t4,test.t5'
+connection slave;
+RESET SLAVE;
+set global gtid_slave_pos="";
+RESET MASTER;
+connection master;
+RESET MASTER;
+connection slave;
+connection master;
+#
+#
+# Test Case 22: Ensure consistency when filtering by
+# --ignore-server-ids with both a start position and stop position.
+#
+include/mysqlbinlog_slave_consistency.inc
+connection slave;
+set @@global.gtid_strict_mode=0;
+connection slave;
+connection master;
+# Populating master data
+connection slave;
+set global gtid_slave_pos="0-1-0";
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos, IGNORE_SERVER_IDS=(3);
+START SLAVE UNTIL master_gtid_pos="0-3-3";
+Warnings:
+Note 1278 It is recommended to use --skip-slave-start when doing step-by-step replication with START SLAVE UNTIL; otherwise, you will get problems if you get an unexpected slave's mariadbd restart
+# No slave error expecting - waiting for slave to catch up to master
+# Because there is a stop position we wait for all events to process
+# and slave to automatically stop
+# Stop slave so it stops receiving master events.
+Warnings:
+Note 1255 Slave already has been stopped
+connection master;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+RESET MASTER;
+# MYSQL_BINLOG BINLOG_FILE_PARAM --start-position=0-1-0 --ignore-server-ids=3 --stop-position=0-3-3 2> MYSQLBINLOG_STDERR | MYSQL
+connection slave;
+# Checking consistency of 'test' database tables between master and slave
+connection master;
+# Both servers have 1 tables
+# Verifying integrity of tables..
+connection master;
+connection slave;
+# t1 is equivalent on master and slave
+# All tables are consistent
+# Test finished - resetting master and slave..
+connection slave;
+RESET SLAVE;
+RESET MASTER;
+set global gtid_slave_pos="";
+CHANGE MASTER TO DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=(), IGNORE_SERVER_IDS=();
+connection master;
+RESET MASTER;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+Warnings:
+Note 1051 Unknown table 'test.t2,test.t3,test.t4,test.t5'
+connection slave;
+RESET SLAVE;
+set global gtid_slave_pos="";
+RESET MASTER;
+connection master;
+RESET MASTER;
+connection slave;
+connection master;
+#
+#
+# Test Case 23: Out of order GTIDs from domains or servers which are
+# filtered out should not error
+#
+include/mysqlbinlog_slave_consistency.inc
+connection slave;
+set @@global.gtid_strict_mode=1;
+connection slave;
+connection master;
+# Populating master data
+connection slave;
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos, DO_DOMAIN_IDS=(0), IGNORE_SERVER_IDS=(3);
+START SLAVE ;
+# No slave error expecting - waiting for slave to catch up to master
+# Stop slave so it stops receiving master events.
+connection master;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+Warnings:
+Note 1051 Unknown table 'test.t4,test.t5'
+RESET MASTER;
+# MYSQL_BINLOG BINLOG_FILE_PARAM --do-domain-ids=0 --ignore-server-ids=3 --gtid-strict-mode 2> MYSQLBINLOG_STDERR | MYSQL
+connection slave;
+# Checking consistency of 'test' database tables between master and slave
+connection master;
+# Both servers have 1 tables
+# Verifying integrity of tables..
+connection master;
+connection slave;
+# t1 is equivalent on master and slave
+# All tables are consistent
+# Strict mode enabled - checking mysqlbinlog error output for out
+# of order GTIDs
+# Not expecting to find out of order GTID error..
+NOT FOUND /Found out of order GTID/ in mysqlbinlog_stderr.out
+# Test finished - resetting master and slave..
+connection slave;
+RESET SLAVE;
+RESET MASTER;
+set global gtid_slave_pos="";
+CHANGE MASTER TO DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=(), IGNORE_SERVER_IDS=();
+connection master;
+RESET MASTER;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+Warnings:
+Note 1051 Unknown table 'test.t2,test.t3,test.t4,test.t5'
+connection slave;
+RESET SLAVE;
+set global gtid_slave_pos="";
+RESET MASTER;
+connection master;
+RESET MASTER;
+connection slave;
+set @@global.gtid_strict_mode=0;
+connection master;
+#
+#
+# Test Case 24: Out of order GTIDs from included domains should error
+#
+include/mysqlbinlog_slave_consistency.inc
+connection slave;
+set @@global.gtid_strict_mode=1;
+connection slave;
+connection master;
+# Populating master data
+connection slave;
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos, DO_DOMAIN_IDS=(1);
+START SLAVE ;
+# slave SQL Thread error expected - waiting for errno 1950
+# Stop slave so it stops receiving master events.
+connection master;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+Warnings:
+Note 1051 Unknown table 'test.t4,test.t5'
+RESET MASTER;
+# MYSQL_BINLOG BINLOG_FILE_PARAM --do-domain-ids=1 --gtid-strict-mode 2> MYSQLBINLOG_STDERR | MYSQL
+connection slave;
+# Checking consistency of 'test' database tables between master and slave
+connection master;
+# Both servers have 1 tables
+# Verifying integrity of tables..
+connection master;
+connection slave;
+# t3 is equivalent on master and slave
+# All tables are consistent
+# Strict mode enabled - checking mysqlbinlog error output for out
+# of order GTIDs
+# Expecting to find out of order GTID error..
+FOUND 1 /Found out of order GTID/ in mysqlbinlog_stderr.out
+# Test finished - resetting master and slave..
+connection slave;
+RESET SLAVE;
+RESET MASTER;
+set global gtid_slave_pos="";
+CHANGE MASTER TO DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=(), IGNORE_SERVER_IDS=();
+connection master;
+RESET MASTER;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+Warnings:
+Note 1051 Unknown table 'test.t1,test.t2,test.t4,test.t5'
+connection slave;
+RESET SLAVE;
+set global gtid_slave_pos="";
+RESET MASTER;
+connection master;
+RESET MASTER;
+connection slave;
+set @@global.gtid_strict_mode=0;
+connection master;
+#
+#
+# Test Case 25: Out of order GTIDs from included servers should error
+#
+include/mysqlbinlog_slave_consistency.inc
+connection slave;
+set @@global.gtid_strict_mode=1;
+connection slave;
+connection master;
+# Populating master data
+connection slave;
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos, IGNORE_SERVER_IDS=(1);
+START SLAVE ;
+# slave SQL Thread error expected - waiting for errno 1950
+# Stop slave so it stops receiving master events.
+connection master;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+Warnings:
+Note 1051 Unknown table 'test.t4,test.t5'
+RESET MASTER;
+# MYSQL_BINLOG BINLOG_FILE_PARAM --ignore-server-ids=1 --gtid-strict-mode 2> MYSQLBINLOG_STDERR | MYSQL
+connection slave;
+# Checking consistency of 'test' database tables between master and slave
+connection master;
+# Both servers have 1 tables
+# Verifying integrity of tables..
+connection master;
+connection slave;
+# t2 is equivalent on master and slave
+# All tables are consistent
+# Strict mode enabled - checking mysqlbinlog error output for out
+# of order GTIDs
+# Expecting to find out of order GTID error..
+FOUND 1 /Found out of order GTID/ in mysqlbinlog_stderr.out
+# Test finished - resetting master and slave..
+connection slave;
+RESET SLAVE;
+RESET MASTER;
+set global gtid_slave_pos="";
+CHANGE MASTER TO DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=(), IGNORE_SERVER_IDS=();
+connection master;
+RESET MASTER;
+DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+Warnings:
+Note 1051 Unknown table 'test.t1,test.t3,test.t4,test.t5'
+connection slave;
+RESET SLAVE;
+set global gtid_slave_pos="";
+RESET MASTER;
+connection master;
+RESET MASTER;
+connection slave;
+set @@global.gtid_strict_mode=0;
+connection master;
+#
+#
+# Test Case 26: Neither mysqlbinlog nor CHANGE MASTER TO should allow
+# both do domain ids and ignore domain ids to be set together
+#
+connection slave;
+include/stop_slave.inc
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos, DO_DOMAIN_IDS=(0), IGNORE_DOMAIN_IDS=(1);
+ERROR HY000: Could not initialize master info structure for ''; more error messages can be found in the MariaDB error log
+RESET MASTER;
+set global gtid_slave_pos="";
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos, DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=();
+connection master;
+include/kill_binlog_dump_threads.inc
+RESET MASTER;
+connection slave;
+include/start_slave.inc
+include/rpl_end.inc
+# End of binlog_in_engine.rpl_mysqlbinlog_slave_consistency_oob.test
diff --git a/mysql-test/suite/binlog_in_engine/rpl_mysqlbinlog_slave_consistency_oob.test b/mysql-test/suite/binlog_in_engine/rpl_mysqlbinlog_slave_consistency_oob.test
new file mode 100644
index 0000000000000..7e27828c66633
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/rpl_mysqlbinlog_slave_consistency_oob.test
@@ -0,0 +1,8 @@
+# For test description, see mysql-test/suite/rpl/t/rpl_mysqlbinlog_slave_consistency.inc
+--source include/have_innodb.inc
+--source include/master-slave.inc
+--source include/have_innodb_binlog.inc
+--let $blob_trx_size= 16384
+--source suite/rpl/t/rpl_mysqlbinlog_slave_consistency.inc
+--source include/rpl_end.inc
+--echo # End of binlog_in_engine.rpl_mysqlbinlog_slave_consistency_oob.test
diff --git a/mysql-test/suite/binlog_in_engine/rpl_oob.result b/mysql-test/suite/binlog_in_engine/rpl_oob.result
new file mode 100644
index 0000000000000..b321db2254731
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/rpl_oob.result
@@ -0,0 +1,63 @@
+include/master-slave.inc
+[connection master]
+CREATE TABLE t1 (a INT NOT NULL, b INT NOT NULL, c TEXT, PRIMARY KEY(a, b)) ENGINE=InnoDB;
+INSERT INTO t1 VALUES (0, 0, 'Start');
+*** Generating 25 large transactions in 5 interleaved connections
+connection master;
+INSERT INTO t1 VALUES (0, 1, 'End');
+SELECT COUNT(*), SUM(a), SUM(b), SUM(LENGTH(c)) FROM t1;
+COUNT(*) SUM(a) SUM(b) SUM(LENGTH(c))
+2552 33150 128776 5000383
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+SELECT COUNT(*), SUM(a), SUM(b), SUM(LENGTH(c)) FROM t1;
+COUNT(*) SUM(a) SUM(b) SUM(LENGTH(c))
+2552 33150 128776 5000383
+connection master;
+*** Test trx cache larger than binlog size is correctly split into multiple pieces when spilled as oob data ***
+connection master;
+CREATE TABLE t2 (a INT PRIMARY KEY, b LONGTEXT) ENGINE=InnoDB;
+SET @old_binlog_size= @@GLOBAL.max_binlog_size;
+SET STATEMENT sql_log_bin=0 FOR
+CALL mtr.add_suppression("Requested max_binlog_size is smaller than the minimum size supported by InnoDB");
+SET GLOBAL max_binlog_size= 4096;
+FLUSH BINARY LOGS;
+FLUSH BINARY LOGS;
+INSERT INTO t2 VALUES (10001, REPEAT('x', 1024*1024));
+SELECT COUNT(*), SUM(a), SUM(LENGTH(b)) FROM t1;
+COUNT(*) SUM(a) SUM(LENGTH(b))
+2602 50033150 4993
+include/save_master_gtid.inc
+SET GLOBAL max_binlog_size= @old_binlog_size;
+connection slave;
+include/sync_with_master_gtid.inc
+SELECT COUNT(*), SUM(a), SUM(LENGTH(b)) FROM t1;
+COUNT(*) SUM(a) SUM(LENGTH(b))
+2602 50033150 4993
+*** Test that triggers re-allocation of the oob stack due to large tree depth
+connection master;
+CREATE TABLE t3 (
+id bigint(20) unsigned NOT NULL AUTO_INCREMENT,
+a text NOT NULL DEFAULT '',
+b text DEFAULT '',
+PRIMARY KEY (id)
+) ENGINE=InnoDB;
+SET @old_cache_size= @@GLOBAL.binlog_cache_size;
+SET GLOBAL binlog_cache_size= 4096;
+INSERT INTO t3 SELECT seq, 'foo', 'bar' FROM seq_1_to_100;
+UPDATE t3 SET a = REPEAT('x', 65535);
+UPDATE t3 SET b = 'qux';
+SET GLOBAL binlog_cache_size= @old_cache_size;
+*** Test oob spilling of DDL
+connection master;
+ALTER TABLE t3
+PARTITION BY LIST(id)
+(PARTITION p1 VALUES IN ($long_list),
+PARTITION p2 VALUES IN (10000));
+CREATE TABLE t4 AS SELECT * FROM t3 LIMIT 10;
+ALTER TABLE t4 ENGINE=MyISAM;
+CREATE TABLE t5 AS SELECT * FROM t4;
+connection master;
+DROP TABLE t1, t2, t3, t4, t5;
+include/rpl_end.inc
diff --git a/mysql-test/suite/binlog_in_engine/rpl_oob.test b/mysql-test/suite/binlog_in_engine/rpl_oob.test
new file mode 100644
index 0000000000000..8b8410e240a8f
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/rpl_oob.test
@@ -0,0 +1,136 @@
+--source include/have_partition.inc
+--source include/have_binlog_format_row.inc
+--source include/have_sequence.inc
+--source include/master-slave.inc
+--source include/have_innodb_binlog.inc
+
+# Test a number of transactions that are large and get interleaved with each
+# other over multiple binlog files.
+--let $NUM_CONNECTIONS= 5
+# $NUM_TRANSACTIONS is total, not per connection.
+--let $NUM_TRANSACTIONS=25
+--let $NUM_PIECES= 100
+--let $PIECE_SIZE= 2000
+
+
+CREATE TABLE t1 (a INT NOT NULL, b INT NOT NULL, c TEXT, PRIMARY KEY(a, b)) ENGINE=InnoDB;
+INSERT INTO t1 VALUES (0, 0, 'Start');
+
+--echo *** Generating $NUM_TRANSACTIONS large transactions in $NUM_CONNECTIONS interleaved connections
+--disable_query_log
+let $t= 0;
+while ($t < $NUM_TRANSACTIONS) {
+ let $b= $t;
+ let $i= 1;
+ while ($i <= $NUM_CONNECTIONS) {
+ --connect(con$i,localhost,root,,)
+ START TRANSACTION;
+ eval INSERT INTO t1 VALUES ($b + $i, 0, 'Initial $i');
+ inc $i;
+ inc $t;
+ }
+
+ let $p= 1;
+ while ($p <= $NUM_PIECES) {
+ let $i= 1;
+ while ($i <= $NUM_CONNECTIONS) {
+ --connection con$i
+ eval INSERT INTO t1 VALUES ($b + $i, $p, REPEAT(CHR(65 + ($p + $i MOD 26)), $PIECE_SIZE));
+ inc $i;
+ }
+ inc $p;
+ }
+
+ let $i= 1;
+ while ($i <= $NUM_CONNECTIONS) {
+ --connection con$i
+ eval INSERT INTO t1 VALUES ($b + $i, $NUM_PIECES+1, 'Last $i');
+ COMMIT;
+ --disconnect con$i
+ inc $i;
+ }
+}
+--enable_query_log
+
+--connection master
+INSERT INTO t1 VALUES (0, 1, 'End');
+
+SELECT COUNT(*), SUM(a), SUM(b), SUM(LENGTH(c)) FROM t1;
+--source include/save_master_gtid.inc
+--exec $MYSQL_BINLOG --read-from-remote-server --user=root --host=127.0.0.1 --port=$MASTER_MYPORT master-bin.000001 --start-position=0-1-1 > $MYSQLTEST_VARDIR/tmp/mysqlbinlog.txt
+
+--connection slave
+--source include/sync_with_master_gtid.inc
+SELECT COUNT(*), SUM(a), SUM(b), SUM(LENGTH(c)) FROM t1;
+
+
+--connection master
+# Test various event group sizes close to the binlog cache size.
+# There was a bug where if the event group fit in the cache without the GTID
+# event, but not with the GTID, then the code would incorrectly attempt to
+# spill part of the GTID event as oob data and the server would hang on
+# incorrect double locking of LOCK_commit_ordered.
+--disable_query_log
+--let $i= 50
+while ($i > 0) {
+ eval INSERT INTO t1 VALUES (1000000, $i, REPEAT('#', @@binlog_cache_size - $i*20));
+ dec $i;
+}
+--enable_query_log
+
+
+--echo *** Test trx cache larger than binlog size is correctly split into multiple pieces when spilled as oob data ***
+--connection master
+CREATE TABLE t2 (a INT PRIMARY KEY, b LONGTEXT) ENGINE=InnoDB;
+SET @old_binlog_size= @@GLOBAL.max_binlog_size;
+SET STATEMENT sql_log_bin=0 FOR
+ CALL mtr.add_suppression("Requested max_binlog_size is smaller than the minimum size supported by InnoDB");
+SET GLOBAL max_binlog_size= 4096;
+FLUSH BINARY LOGS;
+FLUSH BINARY LOGS;
+INSERT INTO t2 VALUES (10001, REPEAT('x', 1024*1024));
+SELECT COUNT(*), SUM(a), SUM(LENGTH(b)) FROM t1;
+--source include/save_master_gtid.inc
+SET GLOBAL max_binlog_size= @old_binlog_size;
+
+--connection slave
+--source include/sync_with_master_gtid.inc
+SELECT COUNT(*), SUM(a), SUM(LENGTH(b)) FROM t1;
+
+
+--echo *** Test that triggers re-allocation of the oob stack due to large tree depth
+--connection master
+CREATE TABLE t3 (
+ id bigint(20) unsigned NOT NULL AUTO_INCREMENT,
+ a text NOT NULL DEFAULT '',
+ b text DEFAULT '',
+ PRIMARY KEY (id)
+) ENGINE=InnoDB;
+
+SET @old_cache_size= @@GLOBAL.binlog_cache_size;
+# Use smaller nodes to get larger tree height.
+SET GLOBAL binlog_cache_size= 4096;
+INSERT INTO t3 SELECT seq, 'foo', 'bar' FROM seq_1_to_100;
+UPDATE t3 SET a = REPEAT('x', 65535);
+UPDATE t3 SET b = 'qux';
+SET GLOBAL binlog_cache_size= @old_cache_size;
+
+
+--echo *** Test oob spilling of DDL
+--connection master
+--let $long_list= `SELECT GROUP_CONCAT(seq SEPARATOR ',') FROM seq_0_to_9999`
+evalp
+ALTER TABLE t3
+ PARTITION BY LIST(id)
+ (PARTITION p1 VALUES IN ($long_list),
+ PARTITION p2 VALUES IN (10000));
+
+CREATE TABLE t4 AS SELECT * FROM t3 LIMIT 10;
+ALTER TABLE t4 ENGINE=MyISAM;
+CREATE TABLE t5 AS SELECT * FROM t4;
+
+# Cleanup.
+--connection master
+DROP TABLE t1, t2, t3, t4, t5;
+--exec $MYSQL_BINLOG --read-from-remote-server --user=root --host=127.0.0.1 --port=$MASTER_MYPORT master-bin.000001 --start-position=0-1-1 > $MYSQLTEST_VARDIR/tmp/mysqlbinlog.txt
+--source include/rpl_end.inc
diff --git a/mysql-test/suite/binlog_in_engine/rpl_out_of_order_gtid.result b/mysql-test/suite/binlog_in_engine/rpl_out_of_order_gtid.result
new file mode 100644
index 0000000000000..ea8163b0cacac
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/rpl_out_of_order_gtid.result
@@ -0,0 +1,46 @@
+include/master-slave.inc
+[connection master]
+connection slave;
+include/stop_slave.inc
+SET @old_parallel= @@GLOBAL.slave_parallel_threads;
+SET GLOBAL slave_parallel_threads= 4;
+SET @old_strict= @@GLOBAL.gtid_strict_mode;
+SET GLOBAL gtid_strict_mode= 1;
+SET @old_dbug= @@GLOBAL.debug_dbug;
+SET GLOBAL debug_dbug= '+d,pause_before_write_gtid_0_x_100';
+include/start_slave.inc
+connection master;
+CREATE TABLE t1 (a INT PRIMARY KEY, b INT) ENGINE=InnoDB;
+INSERT INTO t1 VALUES (1, 7);
+INSERT INTO t1 VALUES (2, 3);
+INSERT INTO t1 VALUES (3, 4);
+SET SESSION gtid_seq_no= 100;
+ALTER TABLE t1 ORDER BY b;
+Warnings:
+Warning 1105 ORDER BY ignored as there is a user-defined clustered index in the table 't1'
+SET SESSION gtid_seq_no= 102;
+INSERT INTO t1 VALUES (5, 0);
+include/save_master_gtid.inc
+connection slave;
+SET debug_sync= 'now wait_for before_write_gtid_event';
+SET SESSION gtid_seq_no= 101;
+CREATE TABLE t2_slave (a INT PRIMARY KEY);
+SET debug_sync= 'now SIGNAL cont';
+include/wait_for_slave_sql_error.inc [errno=1950]
+SET debug_sync= 'RESET';
+SET GLOBAL debug_dbug= @old_dbug;
+SET GLOBAL gtid_strict_mode= 0;
+START SLAVE sql_thread;
+include/sync_with_master_gtid.inc
+connection slave;
+include/stop_slave.inc
+DROP TABLE t2_slave;
+SET GLOBAL gtid_strict_mode= @old_strict;
+CALL mtr.add_suppression('Slave: ORDER BY ignored');
+CALL mtr.add_suppression('Slave: An attempt was made to binlog GTID .* which would create an out-of-order sequence number with existing GTID');
+CALL mtr.add_suppression('Slave: Error writing file .*"Unknown error 1950"');
+SET GLOBAL slave_parallel_threads= @old_parallel;
+include/start_slave.inc
+connection master;
+DROP TABLE t1;
+include/rpl_end.inc
diff --git a/mysql-test/suite/binlog_in_engine/rpl_out_of_order_gtid.test b/mysql-test/suite/binlog_in_engine/rpl_out_of_order_gtid.test
new file mode 100644
index 0000000000000..273c90cfa5425
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/rpl_out_of_order_gtid.test
@@ -0,0 +1,66 @@
+--source include/have_debug.inc
+--source include/have_debug_sync.inc
+--source include/have_binlog_format_row.inc
+--source include/master-slave.inc
+--source include/have_innodb_binlog.inc
+
+--connection slave
+--source include/stop_slave.inc
+SET @old_parallel= @@GLOBAL.slave_parallel_threads;
+SET GLOBAL slave_parallel_threads= 4;
+SET @old_strict= @@GLOBAL.gtid_strict_mode;
+SET GLOBAL gtid_strict_mode= 1;
+SET @old_dbug= @@GLOBAL.debug_dbug;
+SET GLOBAL debug_dbug= '+d,pause_before_write_gtid_0_x_100';
+--source include/start_slave.inc
+
+--connection master
+CREATE TABLE t1 (a INT PRIMARY KEY, b INT) ENGINE=InnoDB;
+
+# Deliberately introduce out-of-order GTID sequence numbers on the slave.
+# There was a bug in the error handling on the slave, it did not release
+# LOCK_log, causing assertion in debug and hang in release build.
+# The ALTER TABLE gets GTID sequence number 100 to trigger the dbug injection
+# above on the slave so we can provoke the error in the right place where
+# the bug was (otherwise, the out-of-order error gets caught in a different
+# place, while applying the GTID event).
+
+INSERT INTO t1 VALUES (1, 7);
+INSERT INTO t1 VALUES (2, 3);
+INSERT INTO t1 VALUES (3, 4);
+SET SESSION gtid_seq_no= 100;
+ALTER TABLE t1 ORDER BY b;
+SET SESSION gtid_seq_no= 102;
+INSERT INTO t1 VALUES (5, 0);
+--source include/save_master_gtid.inc
+
+--connection slave
+
+SET debug_sync= 'now wait_for before_write_gtid_event';
+# Here inject the out-of-order GTID on the slave.
+SET SESSION gtid_seq_no= 101;
+CREATE TABLE t2_slave (a INT PRIMARY KEY);
+SET debug_sync= 'now SIGNAL cont';
+--let $slave_sql_errno= 1950
+--source include/wait_for_slave_sql_error.inc
+SET debug_sync= 'RESET';
+SET GLOBAL debug_dbug= @old_dbug;
+
+SET GLOBAL gtid_strict_mode= 0;
+START SLAVE sql_thread;
+--source include/sync_with_master_gtid.inc
+
+# Clean up.
+--connection slave
+--source include/stop_slave.inc
+DROP TABLE t2_slave;
+SET GLOBAL gtid_strict_mode= @old_strict;
+CALL mtr.add_suppression('Slave: ORDER BY ignored');
+CALL mtr.add_suppression('Slave: An attempt was made to binlog GTID .* which would create an out-of-order sequence number with existing GTID');
+CALL mtr.add_suppression('Slave: Error writing file .*"Unknown error 1950"');
+SET GLOBAL slave_parallel_threads= @old_parallel;
+--source include/start_slave.inc
+
+--connection master
+DROP TABLE t1;
+--source include/rpl_end.inc
diff --git a/mysql-test/suite/binlog_in_engine/rpl_parallel_temptable.result b/mysql-test/suite/binlog_in_engine/rpl_parallel_temptable.result
new file mode 100644
index 0000000000000..9403fa3e2d36e
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/rpl_parallel_temptable.result
@@ -0,0 +1,234 @@
+include/rpl_init.inc [topology=1->2]
+*** MDEV-6321: close_temporary_tables() in format description event not serialised correctly ***
+connection server_2;
+SET @old_parallel_threads=@@GLOBAL.slave_parallel_threads;
+include/stop_slave.inc
+SET GLOBAL slave_parallel_threads=5;
+include/start_slave.inc
+connection server_1;
+CREATE TABLE t1 (a INT PRIMARY KEY, b VARCHAR(100) CHARACTER SET utf8);
+include/save_master_gtid.inc
+connection server_2;
+include/sync_with_master_gtid.inc
+include/stop_slave.inc
+connection server_1;
+SET gtid_domain_id= 1;
+INSERT INTO t1 VALUES (1, 0);
+CREATE TEMPORARY TABLE t2 (a int);
+connection default;
+SET gtid_domain_id= 2;
+CREATE TEMPORARY TABLE t3 (a INT PRIMARY KEY);
+CREATE TEMPORARY TABLE t4 (a int);
+INSERT INTO t3 VALUES (100);
+INSERT INTO t4 SELECT a+1 FROM t3;
+connection server_1;
+INSERT INTO t2 VALUES (2), (4), (6), (8), (10), (12), (14), (16), (18), (20);
+INSERT INTO t2 VALUES (3), (6), (9), (12), (15), (18);
+INSERT INTO t2 VALUES (4), (8), (12), (16), (20);
+connection default;
+INSERT INTO t3 SELECT a+2 FROM t4;
+INSERT INTO t4 SELECT a+4 FROM t3;
+connection server_1;
+INSERT INTO t2 VALUES (5), (10), (15), (20);
+INSERT INTO t2 VALUES (6), (12), (18);
+INSERT INTO t2 VALUES (7), (14);
+INSERT INTO t2 VALUES (8), (16);
+INSERT INTO t2 VALUES (9), (18);
+INSERT INTO t2 VALUES (10), (20);
+connection default;
+INSERT INTO t3 SELECT a+8 FROM t4;
+INSERT INTO t4 SELECT a+16 FROM t3;
+connection server_1;
+INSERT INTO t2 VALUES (11);
+INSERT INTO t2 VALUES (12);
+INSERT INTO t2 VALUES (13);
+connection default;
+INSERT INTO t3 SELECT a+32 FROM t4;
+connection server_1;
+INSERT INTO t2 VALUES (14);
+INSERT INTO t2 VALUES (15);
+INSERT INTO t2 VALUES (16);
+connection default;
+INSERT INTO t4 SELECT a+64 FROM t3;
+connection server_1;
+INSERT INTO t2 VALUES (17);
+INSERT INTO t2 VALUES (18);
+INSERT INTO t2 VALUES (19);
+connection default;
+INSERT INTO t3 SELECT a+128 FROM t4;
+connection server_1;
+INSERT INTO t2 VALUES (20);
+connection default;
+INSERT INTO t1 SELECT a, a MOD 7 FROM t3;
+INSERT INTO t1 SELECT a, a MOD 7 FROM t4;
+connection server_1;
+INSERT INTO t1 SELECT a, COUNT(*) FROM t2 GROUP BY a;
+FLUSH TABLES;
+SET SESSION debug_dbug="+d,crash_dispatch_command_before";
+SELECT 1;
+Got one of the listed errors
+connection default;
+connection default;
+connection server_1;
+INSERT INTO t1 VALUES (0, 1);
+include/save_master_gtid.inc
+connection server_2;
+include/start_slave.inc
+include/sync_with_master_gtid.inc
+SELECT * FROM t1 WHERE a <= 20 ORDER BY a;
+a b
+0 1
+1 0
+2 1
+3 1
+4 2
+5 1
+6 3
+7 1
+8 3
+9 2
+10 3
+11 1
+12 5
+13 1
+14 3
+15 3
+16 4
+17 1
+18 5
+19 1
+20 5
+SELECT COUNT(*) FROM t1 WHERE a BETWEEN 100+0 AND 100+256;
+COUNT(*)
+55
+SHOW STATUS LIKE 'Slave_open_temp_tables';
+Variable_name Value
+Slave_open_temp_tables 0
+*** Test that if master logged partial event group before crash, we finish that group correctly before executing format description event ***
+include/stop_slave.inc
+connection server_1;
+CALL mtr.add_suppression("Statement accesses nontransactional table as well as transactional or temporary table, and writes to any of them");
+SET gtid_domain_id= 1;
+DELETE FROM t1;
+ALTER TABLE t1 ENGINE=InnoDB;
+CREATE TEMPORARY TABLE t2 (a INT PRIMARY KEY);
+INSERT INTO t2 VALUES (1);
+INSERT INTO t2 VALUES (2);
+connection default;
+SET gtid_domain_id= 2;
+CREATE TEMPORARY TABLE t3 (a INT PRIMARY KEY);
+INSERT INTO t3 VALUES (10);
+INSERT INTO t3 VALUES (20);
+connection server_1;
+INSERT INTO t1 SELECT a, 'server_1' FROM t2;
+connection default;
+INSERT INTO t1 SELECT a, 'default' FROM t3;
+connection server_1;
+INSERT INTO t1 SELECT a+2, '+server_1' FROM t2;
+connection default;
+DROP TEMPORARY TABLE t3;
+connection server_1;
+DROP TEMPORARY TABLE t2;
+INSERT INTO t1 VALUES (0, 1);
+include/save_master_gtid.inc
+connection server_2;
+set @@sql_log_bin=0;
+call mtr.add_suppression("Unexpected break of being relay-logged GTID 1-1-32 event group by the current GTID event 0-1-4");
+set @@sql_log_bin=1;
+set @@global.debug_dbug="+d,slave_discard_xid_for_gtid_0_x_1000";
+include/start_slave.inc
+include/sync_with_master_gtid.inc
+set @@global.debug_dbug="";
+SELECT * FROM t1 ORDER BY a;
+a b
+0 1
+1 server_1
+2 server_1
+3 +server_1
+4 +server_1
+10 default
+20 default
+SHOW STATUS LIKE 'Slave_open_temp_tables';
+Variable_name Value
+Slave_open_temp_tables 0
+connection server_1;
+FLUSH LOGS;
+*** MDEV-7936: Assertion `!table || table->in_use == _current_thd()' failed on parallel replication in optimistic mode ***
+connection server_1;
+CREATE TEMPORARY TABLE t4 (a INT PRIMARY KEY) ENGINE=InnoDB;
+SET @old_dbug= @@SESSION.debug_dbug;
+SET SESSION debug_dbug="+d,binlog_force_commit_id";
+SET @commit_id= 10000;
+INSERT INTO t4 VALUES (30);
+INSERT INTO t4 VALUES (31);
+SET SESSION debug_dbug= @old_dbug;
+INSERT INTO t1 SELECT a, "conservative" FROM t4;
+DROP TEMPORARY TABLE t4;
+SELECT * FROM t1 WHERE a >= 30 ORDER BY a;
+a b
+30 conservative
+31 conservative
+include/save_master_gtid.inc
+connection server_2;
+include/sync_with_master_gtid.inc
+SELECT * FROM t1 WHERE a >= 30 ORDER BY a;
+a b
+30 conservative
+31 conservative
+include/stop_slave.inc
+SET @old_mode= @@GLOBAL.slave_parallel_mode;
+SET GLOBAL slave_parallel_mode=optimistic;
+connection server_1;
+CREATE TEMPORARY TABLE t4 (a INT PRIMARY KEY) ENGINE=InnoDB;
+INSERT INTO t4 VALUES (32);
+INSERT INTO t4 VALUES (33);
+INSERT INTO t1 SELECT a, "optimistic" FROM t4;
+DROP TEMPORARY TABLE t4;
+SELECT * FROM t1 WHERE a >= 30 ORDER BY a;
+a b
+30 conservative
+31 conservative
+32 optimistic
+33 optimistic
+include/save_master_gtid.inc
+connection server_2;
+include/start_slave.inc
+include/sync_with_master_gtid.inc
+SELECT * FROM t1 WHERE a >= 30 ORDER BY a;
+a b
+30 conservative
+31 conservative
+32 optimistic
+33 optimistic
+include/stop_slave.inc
+SET GLOBAL slave_parallel_mode=@old_mode;
+include/start_slave.inc
+*** MDEV33426: Memory allocation accounting incorrect for replicated temptable
+connection server_1;
+CREATE TEMPORARY TABLE t5 (a int) ENGINE=Aria;
+CREATE TEMPORARY TABLE t6 (a int) ENGINE=Heap;
+INSERT INTO t5 VALUES (1);
+INSERT INTO t6 VALUES (2);
+include/save_master_gtid.inc
+connection server_2;
+include/sync_with_master_gtid.inc
+include/stop_slave.inc
+connection server_1;
+INSERT INTO t1 SELECT a+40, 5 FROM t5;
+INSERT INTO t1 SELECT a+40, 6 FROM t6;
+DROP TABLE t5, t6;
+include/save_master_gtid.inc
+connection server_2;
+include/start_slave.inc
+include/sync_with_master_gtid.inc
+SELECT * FROM t1 WHERE a>=40 ORDER BY a;
+a b
+41 5
+42 6
+connection server_2;
+include/stop_slave.inc
+SET GLOBAL slave_parallel_threads=@old_parallel_threads;
+include/start_slave.inc
+connection server_1;
+DROP TABLE t1;
+include/rpl_end.inc
diff --git a/mysql-test/suite/binlog_in_engine/rpl_parallel_temptable.test b/mysql-test/suite/binlog_in_engine/rpl_parallel_temptable.test
new file mode 100644
index 0000000000000..a7191f51683d5
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/rpl_parallel_temptable.test
@@ -0,0 +1,12 @@
+--source include/have_debug.inc
+# Valgrind does not work well with test that crashes the server
+--source include/not_valgrind.inc
+--let $rpl_topology=1->2
+--source include/rpl_init.inc
+--source include/have_binlog_format_statement.inc
+--source include/have_innodb_binlog.inc
+
+--let $binlog_in_engine= 1
+--source include/rpl_parallel_temptable.inc
+
+--source include/rpl_end.inc
diff --git a/mysql-test/suite/binlog_in_engine/rpl_rocksdb.result b/mysql-test/suite/binlog_in_engine/rpl_rocksdb.result
new file mode 100644
index 0000000000000..17c1879c9330d
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/rpl_rocksdb.result
@@ -0,0 +1,140 @@
+include/master-slave.inc
+[connection master]
+CREATE TABLE t1(a INT PRIMARY KEY, b INT, c LONGBLOB) ENGINE=InnoDB;
+CREATE TABLE t2(a INT PRIMARY KEY, b INT, c LONGBLOB) ENGINE=RocksDB;
+INSERT INTO t1 SELECT seq, seq*seq, REPEAT('x', 50*seq) FROM seq_1_to_100;
+INSERT INTO t2 SELECT seq, 10000 - seq*seq, REPEAT('y', 50*seq) FROM seq_1_to_100;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+include/stop_slave.inc
+SET @old_threads= @@GLOBAL.slave_parallel_threads;
+SET GLOBAL slave_parallel_threads= 8;
+include/start_slave.inc
+*** Cross-engine transaction, InnoDB and RocksDB.
+connect con1,localhost,root,,;
+BEGIN;
+UPDATE t1 SET b=b+a WHERE a BETWEEN 10 AND 20;
+REPLACE INTO t2 SELECT a, b, c FROM t1 WHERE a BETWEEN 30 and 40;
+connect con2,localhost,root,,;
+BEGIN;
+UPDATE t1, t2
+SET t1.b=t1.b + LENGTH(t2.c), t2.c=CONCAT("|", t2.c, "|")
+WHERE t1.a = t2.a
+AND t1.a BETWEEN 50 AND 60;
+connection con1;
+UPDATE t1 SET b=-b WHERE a=100;
+connection con2;
+UPDATE t2 SET c=CONCAT('-', c) WHERE a BETWEEN 50 AND 90;
+connection con1;
+COMMIT;
+connection con2;
+COMMIT;
+*** RocksDB-only transactions with binlog in InnoDB.
+connection master;
+UPDATE t2 SET c=CONCAT('<', c, '>') WHERE a BETWEEN 20 AND 80;
+UPDATE t2 SET b=b+1 WHERE a=1 OR a=92;
+UPDATE t2 SET b=b*2 WHERE a MOD 7 = 0;
+*** RocksDB transaction that rolls back.
+BEGIN;
+UPDATE t2 SET b=b+1 WHERE a=3;
+UPDATE t2 SET b=b+1 WHERE a=5;
+UPDATE t2 SET b=b+1 WHERE a=8;
+ROLLBACK;
+connection con2;
+BEGIN;
+UPDATE t2 SET b=b+1 WHERE a=4;
+UPDATE t2 SET b=b+1 WHERE a=9;
+UPDATE t2 SET b=b+1 WHERE a=13;
+disconnect con2;
+connection master;
+SELECT COUNT(*), SUM(a), SUM(b), SUM(LENGTH(c)) FROM t1;
+COUNT(*) SUM(a) SUM(b) SUM(LENGTH(c))
+100 5050 348765 252500
+SELECT COUNT(*), SUM(a), SUM(b), SUM(LENGTH(c)) FROM t2;
+COUNT(*) SUM(a) SUM(b) SUM(LENGTH(c))
+100 5050 661537 252685
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+SELECT COUNT(*), SUM(a), SUM(b), SUM(LENGTH(c)) FROM t1;
+COUNT(*) SUM(a) SUM(b) SUM(LENGTH(c))
+100 5050 348765 252500
+SELECT COUNT(*), SUM(a), SUM(b), SUM(LENGTH(c)) FROM t2;
+COUNT(*) SUM(a) SUM(b) SUM(LENGTH(c))
+100 5050 661537 252685
+*** Test a RocksDB transaction that needs to roll back after having binlogged an internal 2pc xid
+connection slave1;
+SET STATEMENT sql_log_bin= 0
+FOR UPDATE t1 SET a= a+1000000 WHERE a=5;
+BEGIN;
+SELECT a FROM t1 WHERE a=2 FOR UPDATE;
+a
+2
+connection master;
+BEGIN;
+UPDATE t1 SET b=b+1 WHERE a=2;
+UPDATE t1 SET b=b+1 WHERE a=5;
+COMMIT;
+UPDATE t2 SET b=b+2 WHERE a=10;
+include/save_master_gtid.inc
+connection slave;
+connection slave1;
+ROLLBACK;
+connection slave;
+include/wait_for_slave_sql_error.inc [errno=1032]
+SET STATEMENT sql_log_bin= 0
+FOR UPDATE t1 SET a= a-1000000 WHERE a=1000000 + 5;
+START SLAVE SQL_THREAD;
+include/sync_with_master_gtid.inc
+*** XA transaction empty because statements roll back on error
+connection master;
+CREATE TRIGGER u AFTER INSERT ON t2 FOR EACH ROW BEGIN RELEASE SAVEPOINT A ; END //
+XA BEGIN 'xid44';
+INSERT IGNORE INTO t2(a) VALUES (0);
+ERROR 42000: SAVEPOINT A does not exist
+XA END 'xid44';
+XA PREPARE 'xid44';
+XA COMMIT 'xid44';
+DROP TRIGGER u;
+XA BEGIN 'xid44';
+INSERT INTO t2(a) VALUES (0);
+XA END 'xid44';
+XA PREPARE 'xid44';
+include/rpl_restart_server.inc [server_number=1]
+XA COMMIT 'xid44';
+SELECT * FROM t2 WHERE a=0;
+a b c
+0 NULL NULL
+DELETE FROM t2 WHERE a=0;
+*** XA transaction XID not freed in RocksDB
+connection master;
+CREATE TRIGGER u AFTER INSERT ON t2 FOR EACH ROW BEGIN RELEASE SAVEPOINT A ; END //
+XA BEGIN 'xid31';
+INSERT IGNORE INTO t2(a) VALUES (0);
+ERROR 42000: SAVEPOINT A does not exist
+XA END 'xid31';
+XA PREPARE 'xid31';
+XA COMMIT 'xid31';
+DROP TRIGGER u;
+XA BEGIN 'xid31';
+INSERT INTO t2(a) VALUES (0);
+XA END 'xid31';
+XA PREPARE 'xid31';
+XA COMMIT 'xid31';
+connection master1;
+XA BEGIN 'xid31';
+UPDATE t2 SET b=b+10 WHERE a=0;
+UPDATE t2 SET b=b+10 WHERE a=1;
+XA END 'xid31';
+XA PREPARE 'xid31';
+XA COMMIT 'xid31';
+connection slave;
+include/stop_slave.inc
+SET GLOBAL slave_parallel_threads= @old_threads;
+include/start_slave.inc
+connection master;
+DROP TABLE t1, t2;
+CALL mtr.add_suppression("Can't find record in 't1'");
+CALL mtr.add_suppression("Commit failed due to failure of an earlier commit on which this one depends");
+include/rpl_end.inc
diff --git a/mysql-test/suite/binlog_in_engine/rpl_rocksdb.test b/mysql-test/suite/binlog_in_engine/rpl_rocksdb.test
new file mode 100644
index 0000000000000..aff871ed7c0f4
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/rpl_rocksdb.test
@@ -0,0 +1,191 @@
+--source include/have_sequence.inc
+--source include/have_rocksdb.inc
+--source include/have_binlog_format_row.inc
+--source include/master-slave.inc
+--source include/have_innodb_binlog.inc
+
+CREATE TABLE t1(a INT PRIMARY KEY, b INT, c LONGBLOB) ENGINE=InnoDB;
+CREATE TABLE t2(a INT PRIMARY KEY, b INT, c LONGBLOB) ENGINE=RocksDB;
+
+INSERT INTO t1 SELECT seq, seq*seq, REPEAT('x', 50*seq) FROM seq_1_to_100;
+INSERT INTO t2 SELECT seq, 10000 - seq*seq, REPEAT('y', 50*seq) FROM seq_1_to_100;
+
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
+--source include/stop_slave.inc
+SET @old_threads= @@GLOBAL.slave_parallel_threads;
+SET GLOBAL slave_parallel_threads= 8;
+--source include/start_slave.inc
+
+--echo *** Cross-engine transaction, InnoDB and RocksDB.
+--connect con1,localhost,root,,
+BEGIN;
+UPDATE t1 SET b=b+a WHERE a BETWEEN 10 AND 20;
+REPLACE INTO t2 SELECT a, b, c FROM t1 WHERE a BETWEEN 30 and 40;
+
+--connect con2,localhost,root,,
+BEGIN;
+UPDATE t1, t2
+ SET t1.b=t1.b + LENGTH(t2.c), t2.c=CONCAT("|", t2.c, "|")
+ WHERE t1.a = t2.a
+ AND t1.a BETWEEN 50 AND 60;
+
+--connection con1
+UPDATE t1 SET b=-b WHERE a=100;
+
+--connection con2
+UPDATE t2 SET c=CONCAT('-', c) WHERE a BETWEEN 50 AND 90;
+
+--connection con1
+COMMIT;
+
+--connection con2
+COMMIT;
+
+
+--echo *** RocksDB-only transactions with binlog in InnoDB.
+--connection master
+UPDATE t2 SET c=CONCAT('<', c, '>') WHERE a BETWEEN 20 AND 80;
+UPDATE t2 SET b=b+1 WHERE a=1 OR a=92;
+UPDATE t2 SET b=b*2 WHERE a MOD 7 = 0;
+
+--echo *** RocksDB transaction that rolls back.
+BEGIN;
+UPDATE t2 SET b=b+1 WHERE a=3;
+UPDATE t2 SET b=b+1 WHERE a=5;
+UPDATE t2 SET b=b+1 WHERE a=8;
+ROLLBACK;
+
+--connection con2
+BEGIN;
+UPDATE t2 SET b=b+1 WHERE a=4;
+UPDATE t2 SET b=b+1 WHERE a=9;
+UPDATE t2 SET b=b+1 WHERE a=13;
+--disconnect con2
+
+--connection master
+SELECT COUNT(*), SUM(a), SUM(b), SUM(LENGTH(c)) FROM t1;
+SELECT COUNT(*), SUM(a), SUM(b), SUM(LENGTH(c)) FROM t2;
+
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
+
+SELECT COUNT(*), SUM(a), SUM(b), SUM(LENGTH(c)) FROM t1;
+SELECT COUNT(*), SUM(a), SUM(b), SUM(LENGTH(c)) FROM t2;
+
+--echo *** Test a RocksDB transaction that needs to roll back after having binlogged an internal 2pc xid
+
+--connection slave1
+# Cause a "row not found" error by removing a row.
+SET STATEMENT sql_log_bin= 0
+FOR UPDATE t1 SET a= a+1000000 WHERE a=5;
+
+# Block a row temporarily to control parallel replication thread scheduling
+BEGIN;
+SELECT a FROM t1 WHERE a=2 FOR UPDATE;
+
+--connection master
+# Create a transaction T1 that will first wait, and then error.
+BEGIN;
+UPDATE t1 SET b=b+1 WHERE a=2;
+UPDATE t1 SET b=b+1 WHERE a=5;
+COMMIT;
+
+# Create a transaction T2 that will queue for group commit and wait for T1
+# to commit (or fail, as it were).
+UPDATE t2 SET b=b+2 WHERE a=10;
+--source include/save_master_gtid.inc
+
+--connection slave
+--let $wait_condition= SELECT COUNT(*)=1 FROM INFORMATION_SCHEMA.PROCESSLIST WHERE Command='Slave_worker' AND State='Waiting for prior transaction to commit'
+--source include/wait_condition.inc
+
+# Release T1 so that it can fail after T2 has queued for group commit.
+--connection slave1
+ROLLBACK;
+
+--connection slave
+--let $slave_sql_errno= 1032
+--source include/wait_for_slave_sql_error.inc
+
+# Now move back the row so the replication can continue and succeed.
+SET STATEMENT sql_log_bin= 0
+FOR UPDATE t1 SET a= a-1000000 WHERE a=1000000 + 5;
+
+START SLAVE SQL_THREAD;
+--source include/sync_with_master_gtid.inc
+
+
+--echo *** XA transaction empty because statements roll back on error
+--connection master
+--delimiter //
+CREATE TRIGGER u AFTER INSERT ON t2 FOR EACH ROW BEGIN RELEASE SAVEPOINT A ; END //
+--delimiter ;
+
+XA BEGIN 'xid44';
+--error ER_SP_DOES_NOT_EXIST
+INSERT IGNORE INTO t2(a) VALUES (0);
+XA END 'xid44';
+XA PREPARE 'xid44';
+XA COMMIT 'xid44';
+
+DROP TRIGGER u;
+XA BEGIN 'xid44';
+INSERT INTO t2(a) VALUES (0);
+XA END 'xid44';
+XA PREPARE 'xid44';
+--let $rpl_server_number= 1
+--source include/rpl_restart_server.inc
+XA COMMIT 'xid44';
+SELECT * FROM t2 WHERE a=0;
+DELETE FROM t2 WHERE a=0;
+
+--echo *** XA transaction XID not freed in RocksDB
+
+# There was another problem with a similar situation as the previous case.
+# The transaction was empty, was not cleaned up in the engine binlog. Then
+# the next transaction failed in the engine binlog, and the transaction XID
+# was not cleaned up in RocksDB. And then a third re-use of the XID failed.
+--connection master
+--delimiter //
+CREATE TRIGGER u AFTER INSERT ON t2 FOR EACH ROW BEGIN RELEASE SAVEPOINT A ; END //
+--delimiter ;
+
+XA BEGIN 'xid31';
+--error ER_SP_DOES_NOT_EXIST
+INSERT IGNORE INTO t2(a) VALUES (0);
+XA END 'xid31';
+XA PREPARE 'xid31';
+XA COMMIT 'xid31';
+
+DROP TRIGGER u;
+XA BEGIN 'xid31';
+INSERT INTO t2(a) VALUES (0);
+XA END 'xid31';
+XA PREPARE 'xid31';
+XA COMMIT 'xid31';
+
+--connection master1
+XA BEGIN 'xid31';
+UPDATE t2 SET b=b+10 WHERE a=0;
+UPDATE t2 SET b=b+10 WHERE a=1;
+XA END 'xid31';
+XA PREPARE 'xid31';
+XA COMMIT 'xid31';
+
+
+# Clean up.
+
+--connection slave
+--source include/stop_slave.inc
+SET GLOBAL slave_parallel_threads= @old_threads;
+--source include/start_slave.inc
+
+--connection master
+DROP TABLE t1, t2;
+CALL mtr.add_suppression("Can't find record in 't1'");
+CALL mtr.add_suppression("Commit failed due to failure of an earlier commit on which this one depends");
+
+--source include/rpl_end.inc
diff --git a/mysql-test/suite/binlog_in_engine/rpl_row_basic_3innodb_mfsi-master.opt b/mysql-test/suite/binlog_in_engine/rpl_row_basic_3innodb_mfsi-master.opt
new file mode 100644
index 0000000000000..e292b98cf2c3b
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/rpl_row_basic_3innodb_mfsi-master.opt
@@ -0,0 +1 @@
+--log-bin=slave-bin --binlog-storage-engine=
diff --git a/mysql-test/suite/binlog_in_engine/rpl_row_basic_3innodb_mfsi-slave.opt b/mysql-test/suite/binlog_in_engine/rpl_row_basic_3innodb_mfsi-slave.opt
new file mode 100644
index 0000000000000..ea0341e94e904
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/rpl_row_basic_3innodb_mfsi-slave.opt
@@ -0,0 +1 @@
+--log-bin --binlog-storage-engine=innodb
diff --git a/mysql-test/suite/binlog_in_engine/rpl_row_basic_3innodb_mfsi.result b/mysql-test/suite/binlog_in_engine/rpl_row_basic_3innodb_mfsi.result
new file mode 100644
index 0000000000000..9ce5f560ed863
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/rpl_row_basic_3innodb_mfsi.result
@@ -0,0 +1,852 @@
+include/master-slave.inc
+[connection master]
+connection slave;
+connection slave;
+connection master;
+CREATE TABLE t1 (C1 CHAR(1), C2 CHAR(1), INDEX (C1)) ENGINE = 'INNODB' ;
+SELECT * FROM t1;
+C1 C2
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+SELECT * FROM t1;
+C1 C2
+connection master;
+INSERT INTO t1 VALUES ('A','B'), ('X','Y'), ('X','X');
+INSERT INTO t1 VALUES ('A','C'), ('X','Z'), ('A','A');
+SELECT * FROM t1 ORDER BY C1,C2;
+C1 C2
+A A
+A B
+A C
+X X
+X Y
+X Z
+SELECT * FROM t1 ORDER BY C1,C2;
+C1 C2
+A A
+A B
+A C
+X X
+X Y
+X Z
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection master;
+DELETE FROM t1 WHERE C1 = C2;
+SELECT * FROM t1 ORDER BY C1,C2;
+C1 C2
+A B
+A C
+X Y
+X Z
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+SELECT * FROM t1 ORDER BY C1,C2;
+C1 C2
+A B
+A C
+X Y
+X Z
+connection master;
+UPDATE t1 SET C2 = 'I' WHERE C1 = 'A' AND C2 = 'C';
+SELECT * FROM t1 ORDER BY C1,C2;
+C1 C2
+A B
+A I
+X Y
+X Z
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+SELECT * FROM t1 ORDER BY C1,C2;
+C1 C2
+A B
+A I
+X Y
+X Z
+connection slave;
+include/assert.inc [Counter for COM_COMMIT is consistent with the number of actual commits]
+include/assert.inc [Counter for COM_INSERT is consistent with the number of actual inserts]
+include/assert.inc [Counter for COM_DELETE is consistent with the number of actual deletes]
+include/assert.inc [Counter for COM_UPDATE is consistent with the number of actual updates]
+connection master;
+UPDATE t1 SET c2 = 'Q' WHERE c1 = 'A' AND c2 = 'N';
+SELECT * FROM t1 ORDER BY c1,c2;
+C1 C2
+A B
+A I
+X Y
+X Z
+SELECT * FROM t1 ORDER BY c1,c2;
+C1 C2
+A B
+A I
+X Y
+X Z
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection master;
+CREATE TABLE t2 (c1 INT, c12 char(1), c2 INT, PRIMARY KEY (c1)) ENGINE = 'INNODB' ;
+INSERT INTO t2
+VALUES (1,'A',2), (2,'A',4), (3,'A',9), (4,'A',15), (5,'A',25),
+(6,'A',35), (7,'A',50), (8,'A',64), (9,'A',81);
+SELECT * FROM t2 ORDER BY c1,c2;
+c1 c12 c2
+1 A 2
+2 A 4
+3 A 9
+4 A 15
+5 A 25
+6 A 35
+7 A 50
+8 A 64
+9 A 81
+SELECT * FROM t2 WHERE c2 = c1 * c1 ORDER BY c1,c2;
+c1 c12 c2
+2 A 4
+3 A 9
+5 A 25
+8 A 64
+9 A 81
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+SELECT * FROM t2 ORDER BY c1,c2;
+c1 c12 c2
+1 A 2
+2 A 4
+3 A 9
+4 A 15
+5 A 25
+6 A 35
+7 A 50
+8 A 64
+9 A 81
+SELECT * FROM t2 WHERE c2 = c1 * c1 ORDER BY c1,c2;
+c1 c12 c2
+2 A 4
+3 A 9
+5 A 25
+8 A 64
+9 A 81
+connection master;
+UPDATE t2 SET c2 = c1*c1 WHERE c2 != c1*c1;
+SELECT * FROM t2 WHERE c2 = c1 * c1 ORDER BY c1,c2;
+c1 c12 c2
+1 A 1
+2 A 4
+3 A 9
+4 A 16
+5 A 25
+6 A 36
+7 A 49
+8 A 64
+9 A 81
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+SELECT * FROM t2 WHERE c2 = c1 * c1 ORDER BY c1,c2;
+c1 c12 c2
+1 A 1
+2 A 4
+3 A 9
+4 A 16
+5 A 25
+6 A 36
+7 A 49
+8 A 64
+9 A 81
+connection master;
+UPDATE t2 SET c12 = 'Q' WHERE c1 = 1 AND c2 = 999;
+SELECT * FROM t2 ORDER BY c1,c2;
+c1 c12 c2
+1 A 1
+2 A 4
+3 A 9
+4 A 16
+5 A 25
+6 A 36
+7 A 49
+8 A 64
+9 A 81
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+SELECT * FROM t2 ORDER BY c1,c2;
+c1 c12 c2
+1 A 1
+2 A 4
+3 A 9
+4 A 16
+5 A 25
+6 A 36
+7 A 49
+8 A 64
+9 A 81
+connection master;
+DELETE FROM t2 WHERE c1 % 4 = 0;
+SELECT * FROM t2 ORDER BY c1,c2;
+c1 c12 c2
+1 A 1
+2 A 4
+3 A 9
+5 A 25
+6 A 36
+7 A 49
+9 A 81
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+SELECT * FROM t2 ORDER BY c1,c2;
+c1 c12 c2
+1 A 1
+2 A 4
+3 A 9
+5 A 25
+6 A 36
+7 A 49
+9 A 81
+connection master;
+UPDATE t2 SET c12='X';
+connection master;
+CREATE TABLE t3 (C1 CHAR(1), C2 CHAR(1), pk1 INT, C3 CHAR(1), pk2 INT, PRIMARY KEY (pk1,pk2)) ENGINE = 'INNODB' ;
+INSERT INTO t3 VALUES ('A','B',1,'B',1), ('X','Y',2,'B',1), ('X','X',3,'B',1);
+INSERT INTO t3 VALUES ('A','C',1,'B',2), ('X','Z',2,'B',2), ('A','A',3,'B',2);
+SELECT * FROM t3 ORDER BY C1,C2;
+C1 C2 pk1 C3 pk2
+A A 3 B 2
+A B 1 B 1
+A C 1 B 2
+X X 3 B 1
+X Y 2 B 1
+X Z 2 B 2
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+SELECT * FROM t3 ORDER BY C1,C2;
+C1 C2 pk1 C3 pk2
+A A 3 B 2
+A B 1 B 1
+A C 1 B 2
+X X 3 B 1
+X Y 2 B 1
+X Z 2 B 2
+connection master;
+DELETE FROM t3 WHERE C1 = C2;
+SELECT * FROM t3 ORDER BY C1,C2;
+C1 C2 pk1 C3 pk2
+A B 1 B 1
+A C 1 B 2
+X Y 2 B 1
+X Z 2 B 2
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+SELECT * FROM t3 ORDER BY C1,C2;
+C1 C2 pk1 C3 pk2
+A B 1 B 1
+A C 1 B 2
+X Y 2 B 1
+X Z 2 B 2
+connection master;
+UPDATE t3 SET C2 = 'I' WHERE C1 = 'A' AND C2 = 'C';
+SELECT * FROM t3 ORDER BY C1,C2;
+C1 C2 pk1 C3 pk2
+A B 1 B 1
+A I 1 B 2
+X Y 2 B 1
+X Z 2 B 2
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+SELECT * FROM t3 ORDER BY C1,C2;
+C1 C2 pk1 C3 pk2
+A B 1 B 1
+A I 1 B 2
+X Y 2 B 1
+X Z 2 B 2
+connection master;
+CREATE TABLE t6 (C1 CHAR(1), C2 CHAR(1), C3 INT) ENGINE = 'INNODB' ;
+INSERT INTO t6 VALUES ('A','B',1), ('X','Y',2), ('X','X',3);
+INSERT INTO t6 VALUES ('A','C',4), ('X','Z',5), ('A','A',6);
+SELECT * FROM t6 ORDER BY C3;
+C1 C2 C3
+A B 1
+X Y 2
+X X 3
+A C 4
+X Z 5
+A A 6
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+SELECT * FROM t6 ORDER BY C3;
+C1 C2 C3
+A B 1
+X Y 2
+X X 3
+A C 4
+X Z 5
+A A 6
+connection master;
+DELETE FROM t6 WHERE C1 = C2;
+SELECT * FROM t6 ORDER BY C3;
+C1 C2 C3
+A B 1
+X Y 2
+A C 4
+X Z 5
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+SELECT * FROM t6 ORDER BY C3;
+C1 C2 C3
+A B 1
+X Y 2
+A C 4
+X Z 5
+connection master;
+UPDATE t6 SET C2 = 'I' WHERE C1 = 'A' AND C2 = 'C';
+SELECT * FROM t6 ORDER BY C3;
+C1 C2 C3
+A B 1
+X Y 2
+A I 4
+X Z 5
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+SELECT * FROM t6 ORDER BY C3;
+C1 C2 C3
+A B 1
+X Y 2
+A I 4
+X Z 5
+connection master;
+CREATE TABLE t5 (C1 CHAR(1), C2 CHAR(1), C3 INT PRIMARY KEY) ENGINE = 'INNODB' ;
+INSERT INTO t5 VALUES ('A','B',1), ('X','Y',2), ('X','X',3);
+INSERT INTO t5 VALUES ('A','C',4), ('X','Z',5), ('A','A',6);
+UPDATE t5,t2,t3 SET t5.C2='Q', t2.c12='R', t3.C3 ='S' WHERE t5.C1 = t2.c12 AND t5.C1 = t3.C1;
+SELECT * FROM t5,t2,t3 WHERE t5.C2='Q' AND t2.c12='R' AND t3.C3 ='S' ORDER BY t5.C3,t2.c1,t3.pk1,t3.pk2;
+C1 C2 C3 c1 c12 c2 C1 C2 pk1 C3 pk2
+X Q 2 1 R 1 X Y 2 S 1
+X Q 2 1 R 1 X Z 2 S 2
+X Q 2 2 R 4 X Y 2 S 1
+X Q 2 2 R 4 X Z 2 S 2
+X Q 2 3 R 9 X Y 2 S 1
+X Q 2 3 R 9 X Z 2 S 2
+X Q 2 5 R 25 X Y 2 S 1
+X Q 2 5 R 25 X Z 2 S 2
+X Q 2 6 R 36 X Y 2 S 1
+X Q 2 6 R 36 X Z 2 S 2
+X Q 2 7 R 49 X Y 2 S 1
+X Q 2 7 R 49 X Z 2 S 2
+X Q 2 9 R 81 X Y 2 S 1
+X Q 2 9 R 81 X Z 2 S 2
+X Q 3 1 R 1 X Y 2 S 1
+X Q 3 1 R 1 X Z 2 S 2
+X Q 3 2 R 4 X Y 2 S 1
+X Q 3 2 R 4 X Z 2 S 2
+X Q 3 3 R 9 X Y 2 S 1
+X Q 3 3 R 9 X Z 2 S 2
+X Q 3 5 R 25 X Y 2 S 1
+X Q 3 5 R 25 X Z 2 S 2
+X Q 3 6 R 36 X Y 2 S 1
+X Q 3 6 R 36 X Z 2 S 2
+X Q 3 7 R 49 X Y 2 S 1
+X Q 3 7 R 49 X Z 2 S 2
+X Q 3 9 R 81 X Y 2 S 1
+X Q 3 9 R 81 X Z 2 S 2
+X Q 5 1 R 1 X Y 2 S 1
+X Q 5 1 R 1 X Z 2 S 2
+X Q 5 2 R 4 X Y 2 S 1
+X Q 5 2 R 4 X Z 2 S 2
+X Q 5 3 R 9 X Y 2 S 1
+X Q 5 3 R 9 X Z 2 S 2
+X Q 5 5 R 25 X Y 2 S 1
+X Q 5 5 R 25 X Z 2 S 2
+X Q 5 6 R 36 X Y 2 S 1
+X Q 5 6 R 36 X Z 2 S 2
+X Q 5 7 R 49 X Y 2 S 1
+X Q 5 7 R 49 X Z 2 S 2
+X Q 5 9 R 81 X Y 2 S 1
+X Q 5 9 R 81 X Z 2 S 2
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+SELECT * FROM t5,t2,t3 WHERE t5.C2='Q' AND t2.c12='R' AND t3.C3 ='S' ORDER BY t5.C3,t2.c1,t3.pk1,t3.pk2;
+C1 C2 C3 c1 c12 c2 C1 C2 pk1 C3 pk2
+X Q 2 1 R 1 X Y 2 S 1
+X Q 2 1 R 1 X Z 2 S 2
+X Q 2 2 R 4 X Y 2 S 1
+X Q 2 2 R 4 X Z 2 S 2
+X Q 2 3 R 9 X Y 2 S 1
+X Q 2 3 R 9 X Z 2 S 2
+X Q 2 5 R 25 X Y 2 S 1
+X Q 2 5 R 25 X Z 2 S 2
+X Q 2 6 R 36 X Y 2 S 1
+X Q 2 6 R 36 X Z 2 S 2
+X Q 2 7 R 49 X Y 2 S 1
+X Q 2 7 R 49 X Z 2 S 2
+X Q 2 9 R 81 X Y 2 S 1
+X Q 2 9 R 81 X Z 2 S 2
+X Q 3 1 R 1 X Y 2 S 1
+X Q 3 1 R 1 X Z 2 S 2
+X Q 3 2 R 4 X Y 2 S 1
+X Q 3 2 R 4 X Z 2 S 2
+X Q 3 3 R 9 X Y 2 S 1
+X Q 3 3 R 9 X Z 2 S 2
+X Q 3 5 R 25 X Y 2 S 1
+X Q 3 5 R 25 X Z 2 S 2
+X Q 3 6 R 36 X Y 2 S 1
+X Q 3 6 R 36 X Z 2 S 2
+X Q 3 7 R 49 X Y 2 S 1
+X Q 3 7 R 49 X Z 2 S 2
+X Q 3 9 R 81 X Y 2 S 1
+X Q 3 9 R 81 X Z 2 S 2
+X Q 5 1 R 1 X Y 2 S 1
+X Q 5 1 R 1 X Z 2 S 2
+X Q 5 2 R 4 X Y 2 S 1
+X Q 5 2 R 4 X Z 2 S 2
+X Q 5 3 R 9 X Y 2 S 1
+X Q 5 3 R 9 X Z 2 S 2
+X Q 5 5 R 25 X Y 2 S 1
+X Q 5 5 R 25 X Z 2 S 2
+X Q 5 6 R 36 X Y 2 S 1
+X Q 5 6 R 36 X Z 2 S 2
+X Q 5 7 R 49 X Y 2 S 1
+X Q 5 7 R 49 X Z 2 S 2
+X Q 5 9 R 81 X Y 2 S 1
+X Q 5 9 R 81 X Z 2 S 2
+connection slave;
+SET @saved_slave_type_conversions = @@SLAVE_TYPE_CONVERSIONS;
+SET GLOBAL SLAVE_TYPE_CONVERSIONS = 'ALL_LOSSY';
+connection master;
+CREATE TABLE t4 (C1 CHAR(1) PRIMARY KEY, B1 BIT(1), B2 BIT(1) NOT NULL DEFAULT 0, C2 CHAR(1) NOT NULL DEFAULT 'A') ENGINE = 'INNODB' ;
+INSERT INTO t4 SET C1 = 1;
+SELECT C1,HEX(B1),HEX(B2) FROM t4 ORDER BY C1;
+C1 HEX(B1) HEX(B2)
+1 NULL 0
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+SELECT C1,HEX(B1),HEX(B2) FROM t4 ORDER BY C1;
+C1 HEX(B1) HEX(B2)
+1 NULL 0
+SET GLOBAL SLAVE_TYPE_CONVERSIONS = @saved_slave_type_conversions;
+connection master;
+CREATE TABLE t7 (C1 INT PRIMARY KEY, C2 INT) ENGINE = 'INNODB' ;
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+--- original values ---
+INSERT INTO t7 VALUES (1,3), (2,6), (3,9);
+SELECT * FROM t7 ORDER BY C1;
+C1 C2
+1 3
+2 6
+3 9
+set @@global.slave_exec_mode= 'IDEMPOTENT';
+connection master;
+--- new values inserted ---
+INSERT INTO t7 VALUES (1,2), (2,4), (3,6);
+SELECT * FROM t7 ORDER BY C1;
+C1 C2
+1 2
+2 4
+3 6
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+set @@global.slave_exec_mode= default;
+--- old values should be overwritten by replicated values ---
+SELECT * FROM t7 ORDER BY C1;
+C1 C2
+1 2
+2 4
+3 6
+connection master;
+CREATE TABLE t8 (a INT PRIMARY KEY, b INT UNIQUE, c INT UNIQUE) ENGINE = 'INNODB' ;
+INSERT INTO t8 VALUES (99,99,99);
+INSERT INTO t8 VALUES (99,22,33);
+ERROR 23000: Duplicate entry '99' for key 'PRIMARY'
+INSERT INTO t8 VALUES (11,99,33);
+ERROR 23000: Duplicate entry '99' for key 'b'
+INSERT INTO t8 VALUES (11,22,99);
+ERROR 23000: Duplicate entry '99' for key 'c'
+SELECT * FROM t8 ORDER BY a;
+a b c
+99 99 99
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+SELECT * FROM t8 ORDER BY a;
+a b c
+99 99 99
+INSERT INTO t8 VALUES (1,2,3), (2,4,6), (3,6,9);
+SELECT * FROM t8 ORDER BY a;
+a b c
+1 2 3
+2 4 6
+3 6 9
+99 99 99
+set @@global.slave_exec_mode= 'IDEMPOTENT';
+connection master;
+INSERT INTO t8 VALUES (2,4,8);
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+set @@global.slave_exec_mode= default;
+SELECT * FROM t8 ORDER BY a;
+a b c
+1 2 3
+2 4 8
+3 6 9
+99 99 99
+**** Test for BUG#31552 ****
+connection master;
+DELETE FROM t1;
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+include/rpl_reset.inc
+connection master;
+INSERT INTO t1 VALUES ('K','K'), ('L','L'), ('M','M');
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+set @@global.slave_exec_mode= 'IDEMPOTENT';
+DELETE FROM t1 WHERE C1 = 'L';
+connection master;
+DELETE FROM t1;
+SELECT COUNT(*) FROM t1 ORDER BY c1,c2;
+COUNT(*) 0
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+set @@global.slave_exec_mode= default;
+include/check_slave_is_running.inc
+SELECT COUNT(*) FROM t1 ORDER BY c1,c2;
+COUNT(*) 0
+**** Test for BUG#37076 ****
+connection master;
+DROP TABLE IF EXISTS t1;
+CREATE TABLE t1 (a TIMESTAMP, b DATETIME, c DATE);
+INSERT INTO t1 VALUES(
+'2005-11-14 01:01:01', '2005-11-14 01:01:02', '2005-11-14');
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+SELECT * FROM t1;
+a b c
+2005-11-14 01:01:01 2005-11-14 01:01:02 2005-11-14
+connection master;
+DROP TABLE IF EXISTS t1,t2,t3,t4,t5,t6,t7,t8;
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection master;
+CREATE TABLE t1 (i INT NOT NULL,
+c CHAR(16) CHARACTER SET utf8 NOT NULL,
+j INT NOT NULL) ENGINE = 'INNODB' ;
+CREATE TABLE t2 (i INT NOT NULL,
+c CHAR(16) CHARACTER SET utf8 NOT NULL,
+j INT NOT NULL) ENGINE = 'INNODB' ;
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+ALTER TABLE t2 MODIFY c CHAR(128) CHARACTER SET utf8 NOT NULL;
+connection master;
+CREATE TABLE t3 (i INT NOT NULL,
+c CHAR(128) CHARACTER SET utf8 NOT NULL,
+j INT NOT NULL) ENGINE = 'INNODB' ;
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+ALTER TABLE t3 MODIFY c CHAR(16) CHARACTER SET utf8 NOT NULL;
+connection master;
+CREATE TABLE t4 (i INT NOT NULL,
+c CHAR(128) CHARACTER SET utf8 NOT NULL,
+j INT NOT NULL) ENGINE = 'INNODB' ;
+CREATE TABLE t5 (i INT NOT NULL,
+c CHAR(255) CHARACTER SET utf8 NOT NULL,
+j INT NOT NULL) ENGINE = 'INNODB' ;
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+ALTER TABLE t5 MODIFY c CHAR(16) CHARACTER SET utf8 NOT NULL;
+connection master;
+CREATE TABLE t6 (i INT NOT NULL,
+c CHAR(255) CHARACTER SET utf8 NOT NULL,
+j INT NOT NULL) ENGINE = 'INNODB' ;
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+ALTER TABLE t6 MODIFY c CHAR(128) CHARACTER SET utf8 NOT NULL;
+connection master;
+CREATE TABLE t7 (i INT NOT NULL,
+c CHAR(255) CHARACTER SET utf8 NOT NULL,
+j INT NOT NULL) ENGINE = 'INNODB' ;
+connection slave;
+SET @saved_slave_type_conversions = @@slave_type_conversions;
+SET GLOBAL SLAVE_TYPE_CONVERSIONS = 'ALL_NON_LOSSY';
+[expecting slave to replicate correctly]
+connection master;
+INSERT INTO t1 VALUES (1, "", 1);
+INSERT INTO t1 VALUES (2, repeat(_utf8'a', 16), 2);
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+include/diff_tables.inc [master:t1, slave:t1]
+[expecting slave to replicate correctly]
+connection master;
+INSERT INTO t2 VALUES (1, "", 1);
+INSERT INTO t2 VALUES (2, repeat(_utf8'a', 16), 2);
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+include/diff_tables.inc [master:t2, slave:t2]
+connection slave;
+SET GLOBAL SLAVE_TYPE_CONVERSIONS = @saved_slave_type_conversions;
+call mtr.add_suppression("Slave SQL.*Table definition on master and slave does not match: Column 1 size mismatch.* error.* 1535");
+call mtr.add_suppression("Slave SQL.*Could not execute Delete_rows event on table test.t1.* error.* 1032");
+call mtr.add_suppression("Slave SQL.*Column 1 of table .test.t.. cannot be converted from type.*, error.* 1677");
+call mtr.add_suppression("Can't find record in 't1'");
+include/rpl_reset.inc
+[expecting slave to replicate correctly]
+connection master;
+INSERT INTO t4 VALUES (1, "", 1);
+INSERT INTO t4 VALUES (2, repeat(_utf8'a', 128), 2);
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+include/diff_tables.inc [master:t4, slave:t4]
+[expecting slave to stop]
+connection master;
+INSERT INTO t5 VALUES (1, "", 1);
+INSERT INTO t5 VALUES (2, repeat(_utf8'a', 255), 2);
+connection slave;
+include/wait_for_slave_sql_error.inc [errno=1677]
+Last_SQL_Error = 'Column 1 of table 'test.t5' cannot be converted from type 'char(765 octets)' to type 'char(48 octets) character set utf8mb3''
+include/rpl_reset.inc
+[expecting slave to stop]
+connection master;
+INSERT INTO t6 VALUES (1, "", 1);
+INSERT INTO t6 VALUES (2, repeat(_utf8'a', 255), 2);
+connection slave;
+include/wait_for_slave_sql_error.inc [errno=1677]
+Last_SQL_Error = 'Column 1 of table 'test.t6' cannot be converted from type 'char(765 octets)' to type 'char(384 octets) character set utf8mb3''
+include/rpl_reset.inc
+[expecting slave to replicate correctly]
+connection master;
+INSERT INTO t7 VALUES (1, "", 1);
+INSERT INTO t7 VALUES (2, repeat(_utf8'a', 255), 2);
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+include/diff_tables.inc [master:t7, slave:t7]
+connection master;
+drop table t1, t2, t3, t4, t5, t6, t7;
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection master;
+CREATE TABLE t1 (a INT PRIMARY KEY) ENGINE='INNODB' ;
+INSERT INTO t1 VALUES (1), (2), (3);
+UPDATE t1 SET a = 10;
+ERROR 23000: Duplicate entry '10' for key 'PRIMARY'
+INSERT INTO t1 VALUES (4);
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+include/diff_tables.inc [master:t1, slave:t1]
+connection master;
+drop table t1;
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection master;
+DROP TABLE IF EXISTS t1, t2;
+CREATE TABLE t1 (
+`pk` int(11) NOT NULL AUTO_INCREMENT,
+`int_nokey` int(11) NOT NULL,
+`int_key` int(11) NOT NULL,
+`date_key` date NOT NULL,
+`date_nokey` date NOT NULL,
+`time_key` time NOT NULL,
+`time_nokey` time NOT NULL,
+`datetime_key` datetime NOT NULL,
+`datetime_nokey` datetime NOT NULL,
+`varchar_key` varchar(1) NOT NULL,
+`varchar_nokey` varchar(1) NOT NULL,
+PRIMARY KEY (`pk`),
+KEY `int_key` (`int_key`),
+KEY `date_key` (`date_key`),
+KEY `time_key` (`time_key`),
+KEY `datetime_key` (`datetime_key`),
+KEY `varchar_key` (`varchar_key`)
+) ENGINE='INNODB' ;
+INSERT INTO t1 VALUES (1,8,5,'0000-00-00','0000-00-00','10:37:38','10:37:38','0000-00-00 00:00:00','0000-00-00 00:00:00','p','p'),(2,0,9,'0000-00-00','0000-00-00','00:00:00','00:00:00','2007-10-14 00:00:00','2007-10-14 00:00:00','d','d');
+CREATE TABLE t2 (
+`pk` int(11) NOT NULL AUTO_INCREMENT,
+`int_nokey` int(11) NOT NULL,
+`int_key` int(11) NOT NULL,
+`date_key` date NOT NULL,
+`date_nokey` date NOT NULL,
+`time_key` time NOT NULL,
+`time_nokey` time NOT NULL,
+`datetime_key` datetime NOT NULL,
+`datetime_nokey` datetime NOT NULL,
+`varchar_key` varchar(1) NOT NULL,
+`varchar_nokey` varchar(1) NOT NULL,
+PRIMARY KEY (`pk`),
+KEY `int_key` (`int_key`),
+KEY `date_key` (`date_key`),
+KEY `time_key` (`time_key`),
+KEY `datetime_key` (`datetime_key`),
+KEY `varchar_key` (`varchar_key`)
+) ENGINE='INNODB' ;
+INSERT INTO t2 VALUES (1,1,6,'2005-12-23','2005-12-23','02:24:28','02:24:28','0000-00-00 00:00:00','0000-00-00 00:00:00','g','g'),(2,0,3,'2009-09-14','2009-09-14','00:00:00','00:00:00','2000-01-30 16:39:40','2000-01-30 16:39:40','q','q'),(3,0,3,'0000-00-00','0000-00-00','00:00:00','00:00:00','0000-00-00 00:00:00','0000-00-00 00:00:00','c','c'),(4,1,6,'2007-03-29','2007-03-29','15:49:00','15:49:00','0000-00-00 00:00:00','0000-00-00 00:00:00','m','m'),(5,4,0,'2002-12-04','2002-12-04','00:00:00','00:00:00','0000-00-00 00:00:00','0000-00-00 00:00:00','o','o'),(6,9,0,'2005-01-28','2005-01-28','00:00:00','00:00:00','2001-05-18 00:00:00','2001-05-18 00:00:00','w','w'),(7,6,0,'0000-00-00','0000-00-00','06:57:25','06:57:25','0000-00-00 00:00:00','0000-00-00 00:00:00','m','m'),(8,0,0,'0000-00-00','0000-00-00','00:00:00','00:00:00','0000-00-00 00:00:00','0000-00-00 00:00:00','z','z'),(9,4,6,'2006-08-15','2006-08-15','00:00:00','00:00:00','2002-04-12 14:44:25','2002-04-12 14:44:25','j','j'),(10,0,5,'2006-12-20','2006-12-20','10:13:53','10:13:53','2008-07-22 00:00:00','2008-07-22 00:00:00','y','y'),(11,9,7,'0000-00-00','0000-00-00','00:00:00','00:00:00','2004-07-05 00:00:00','2004-07-05 00:00:00','{','{'),(12,4,3,'2007-01-26','2007-01-26','23:00:51','23:00:51','2001-05-16 00:00:00','2001-05-16 00:00:00','f','f'),(13,7,0,'2004-03-27','2004-03-27','00:00:00','00:00:00','2005-01-24 03:30:37','2005-01-24 03:30:37','',''),(14,6,0,'2006-07-26','2006-07-26','18:43:57','18:43:57','0000-00-00 00:00:00','0000-00-00 00:00:00','{','{'),(15,0,6,'2000-01-14','2000-01-14','00:00:00','00:00:00','2000-09-21 00:00:00','2000-09-21 00:00:00','o','o'),(16,9,8,'0000-00-00','0000-00-00','21:15:08','21:15:08','0000-00-00 00:00:00','0000-00-00 00:00:00','a','a'),(17,2,0,'2004-10-27','2004-10-27','00:00:00','00:00:00','2004-03-24 22:13:43','2004-03-24 22:13:43','',''),(18,7,4,'0000-00-00','0000-00-00','08:38:27','08:38:27','2002-03-18 19:51:44','2002-03-18 19:51:44','t','t'),(19,5,3,'2008-03-07','2008-03-07','03:29:07','03:29:07','2007-12-01 18:44:44','2007-12-01 18:44:44','t','t'),(20,0,0,'2002-04-09','2002-04-09','16:06:03','16:06:03','2009-04-22 00:00:00','2009-04-22 00:00:00','n','n');
+DELETE FROM t2 WHERE `int_key` < 3 LIMIT 1;
+UPDATE t1 SET `int_key` = 3 ORDER BY `pk` LIMIT 4;
+DELETE FROM t2 WHERE `int_key` < 3 LIMIT 1;
+DELETE FROM t2 WHERE `pk` < 6 LIMIT 1;
+UPDATE t1 SET `int_key` = 6 ORDER BY `pk` LIMIT 3;
+DELETE FROM t2 WHERE `pk` < 6 LIMIT 1;
+UPDATE t1 SET `pk` = 6 ORDER BY `int_key` LIMIT 6;
+ERROR 23000: Duplicate entry '6' for key 'PRIMARY'
+DELETE FROM t2 WHERE `pk` < 7 LIMIT 1;
+UPDATE t1 SET `int_key` = 4 ORDER BY `pk` LIMIT 6;
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+*** results: t2 must be consistent ****
+include/diff_tables.inc [master:t2, slave:t2]
+connection master;
+DROP TABLE t1, t2;
+EOF OF TESTS
+connection master;
+CREATE TABLE t1 (a int) ENGINE='INNODB' ;
+INSERT IGNORE INTO t1 VALUES (NULL);
+INSERT INTO t1 ( a ) VALUES ( 0 );
+INSERT INTO t1 ( a ) VALUES ( 9 );
+INSERT INTO t1 ( a ) VALUES ( 2 );
+INSERT INTO t1 ( a ) VALUES ( 9 );
+INSERT INTO t1 ( a ) VALUES ( 5 );
+UPDATE t1 SET a = 5 WHERE a = 9;
+DELETE FROM t1 WHERE a < 6;
+UPDATE t1 SET a = 9 WHERE a < 3;
+INSERT INTO t1 ( a ) VALUES ( 3 );
+UPDATE t1 SET a = 0 WHERE a < 4;
+UPDATE t1 SET a = 8 WHERE a < 5;
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+include/diff_tables.inc [master:t1, slave:t1]
+connection master;
+drop table t1;
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection master;
+connection slave;
+SET @saved_slave_type_conversions = @@SLAVE_TYPE_CONVERSIONS;
+SET GLOBAL SLAVE_TYPE_CONVERSIONS = 'ALL_LOSSY';
+connection master;
+CREATE TABLE t1 (a bit) ENGINE='INNODB' ;
+INSERT IGNORE INTO t1 VALUES (NULL);
+INSERT INTO t1 ( a ) VALUES ( 0 );
+UPDATE t1 SET a = 0 WHERE a = 1 LIMIT 3;
+INSERT IGNORE INTO t1 ( a ) VALUES ( 5 );
+DELETE FROM t1 WHERE a < 2 LIMIT 4;
+DELETE FROM t1 WHERE a < 9 LIMIT 4;
+INSERT IGNORE INTO t1 ( a ) VALUES ( 9 );
+UPDATE t1 SET a = 8 WHERE a = 0 LIMIT 6;
+INSERT IGNORE INTO t1 ( a ) VALUES ( 8 );
+UPDATE t1 SET a = 0 WHERE a < 6 LIMIT 0;
+INSERT IGNORE INTO t1 ( a ) VALUES ( 4 );
+INSERT IGNORE INTO t1 ( a ) VALUES ( 3 );
+UPDATE t1 SET a = 0 WHERE a = 7 LIMIT 6;
+DELETE FROM t1 WHERE a = 4 LIMIT 7;
+UPDATE IGNORE t1 SET a = 9 WHERE a < 2 LIMIT 9;
+UPDATE t1 SET a = 0 WHERE a < 9 LIMIT 2;
+DELETE FROM t1 WHERE a < 0 LIMIT 5;
+INSERT IGNORE INTO t1 ( a ) VALUES ( 5 );
+UPDATE IGNORE t1 SET a = 4 WHERE a < 6 LIMIT 4;
+INSERT IGNORE INTO t1 ( a ) VALUES ( 5 );
+UPDATE IGNORE t1 SET a = 9 WHERE a < 5 LIMIT 8;
+DELETE FROM t1 WHERE a < 8 LIMIT 8;
+INSERT IGNORE INTO t1 ( a ) VALUES ( 6 );
+DELETE FROM t1 WHERE a < 6 LIMIT 7;
+UPDATE t1 SET a = 7 WHERE a = 3 LIMIT 7;
+UPDATE t1 SET a = 8 WHERE a = 0 LIMIT 6;
+INSERT IGNORE INTO t1 ( a ) VALUES ( 7 );
+DELETE FROM t1 WHERE a < 9 LIMIT 4;
+INSERT IGNORE INTO t1 ( a ) VALUES ( 7 );
+INSERT IGNORE INTO t1 ( a ) VALUES ( 6 );
+UPDATE t1 SET a = 8 WHERE a = 3 LIMIT 4;
+DELETE FROM t1 WHERE a = 2 LIMIT 9;
+DELETE FROM t1 WHERE a = 1 LIMIT 4;
+UPDATE t1 SET a = 4 WHERE a = 2 LIMIT 7;
+INSERT INTO t1 ( a ) VALUES ( 0 );
+DELETE FROM t1 WHERE a < 3 LIMIT 0;
+UPDATE t1 SET a = 8 WHERE a = 5 LIMIT 2;
+INSERT INTO t1 ( a ) VALUES ( 1 );
+UPDATE IGNORE t1 SET a = 9 WHERE a < 5 LIMIT 3;
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+SET GLOBAL SLAVE_TYPE_CONVERSIONS = @saved_slave_type_conversions;
+include/diff_tables.inc [master:t1, slave:t1]
+connection master;
+drop table t1;
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+include/rpl_end.inc
diff --git a/mysql-test/suite/binlog_in_engine/rpl_row_basic_3innodb_mfsi.test b/mysql-test/suite/binlog_in_engine/rpl_row_basic_3innodb_mfsi.test
new file mode 100644
index 0000000000000..c7d2c151a5a22
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/rpl_row_basic_3innodb_mfsi.test
@@ -0,0 +1,11 @@
+--source include/have_innodb.inc
+--source include/have_binlog_format_row.inc
+--source include/master-slave.inc
+
+connection slave;
+let $bit_field_special = ALL_LOSSY;
+let $type= 'INNODB' ;
+let $extra_index= ;
+--source ../rpl/include/rpl_row_basic.test
+
+--source include/rpl_end.inc
diff --git a/mysql-test/suite/binlog_in_engine/rpl_row_basic_3innodb_misf-master.opt b/mysql-test/suite/binlog_in_engine/rpl_row_basic_3innodb_misf-master.opt
new file mode 100644
index 0000000000000..ea0341e94e904
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/rpl_row_basic_3innodb_misf-master.opt
@@ -0,0 +1 @@
+--log-bin --binlog-storage-engine=innodb
diff --git a/mysql-test/suite/binlog_in_engine/rpl_row_basic_3innodb_misf-slave.opt b/mysql-test/suite/binlog_in_engine/rpl_row_basic_3innodb_misf-slave.opt
new file mode 100644
index 0000000000000..e292b98cf2c3b
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/rpl_row_basic_3innodb_misf-slave.opt
@@ -0,0 +1 @@
+--log-bin=slave-bin --binlog-storage-engine=
diff --git a/mysql-test/suite/binlog_in_engine/rpl_row_basic_3innodb_misf.result b/mysql-test/suite/binlog_in_engine/rpl_row_basic_3innodb_misf.result
new file mode 100644
index 0000000000000..9ce5f560ed863
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/rpl_row_basic_3innodb_misf.result
@@ -0,0 +1,852 @@
+include/master-slave.inc
+[connection master]
+connection slave;
+connection slave;
+connection master;
+CREATE TABLE t1 (C1 CHAR(1), C2 CHAR(1), INDEX (C1)) ENGINE = 'INNODB' ;
+SELECT * FROM t1;
+C1 C2
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+SELECT * FROM t1;
+C1 C2
+connection master;
+INSERT INTO t1 VALUES ('A','B'), ('X','Y'), ('X','X');
+INSERT INTO t1 VALUES ('A','C'), ('X','Z'), ('A','A');
+SELECT * FROM t1 ORDER BY C1,C2;
+C1 C2
+A A
+A B
+A C
+X X
+X Y
+X Z
+SELECT * FROM t1 ORDER BY C1,C2;
+C1 C2
+A A
+A B
+A C
+X X
+X Y
+X Z
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection master;
+DELETE FROM t1 WHERE C1 = C2;
+SELECT * FROM t1 ORDER BY C1,C2;
+C1 C2
+A B
+A C
+X Y
+X Z
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+SELECT * FROM t1 ORDER BY C1,C2;
+C1 C2
+A B
+A C
+X Y
+X Z
+connection master;
+UPDATE t1 SET C2 = 'I' WHERE C1 = 'A' AND C2 = 'C';
+SELECT * FROM t1 ORDER BY C1,C2;
+C1 C2
+A B
+A I
+X Y
+X Z
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+SELECT * FROM t1 ORDER BY C1,C2;
+C1 C2
+A B
+A I
+X Y
+X Z
+connection slave;
+include/assert.inc [Counter for COM_COMMIT is consistent with the number of actual commits]
+include/assert.inc [Counter for COM_INSERT is consistent with the number of actual inserts]
+include/assert.inc [Counter for COM_DELETE is consistent with the number of actual deletes]
+include/assert.inc [Counter for COM_UPDATE is consistent with the number of actual updates]
+connection master;
+UPDATE t1 SET c2 = 'Q' WHERE c1 = 'A' AND c2 = 'N';
+SELECT * FROM t1 ORDER BY c1,c2;
+C1 C2
+A B
+A I
+X Y
+X Z
+SELECT * FROM t1 ORDER BY c1,c2;
+C1 C2
+A B
+A I
+X Y
+X Z
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection master;
+CREATE TABLE t2 (c1 INT, c12 char(1), c2 INT, PRIMARY KEY (c1)) ENGINE = 'INNODB' ;
+INSERT INTO t2
+VALUES (1,'A',2), (2,'A',4), (3,'A',9), (4,'A',15), (5,'A',25),
+(6,'A',35), (7,'A',50), (8,'A',64), (9,'A',81);
+SELECT * FROM t2 ORDER BY c1,c2;
+c1 c12 c2
+1 A 2
+2 A 4
+3 A 9
+4 A 15
+5 A 25
+6 A 35
+7 A 50
+8 A 64
+9 A 81
+SELECT * FROM t2 WHERE c2 = c1 * c1 ORDER BY c1,c2;
+c1 c12 c2
+2 A 4
+3 A 9
+5 A 25
+8 A 64
+9 A 81
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+SELECT * FROM t2 ORDER BY c1,c2;
+c1 c12 c2
+1 A 2
+2 A 4
+3 A 9
+4 A 15
+5 A 25
+6 A 35
+7 A 50
+8 A 64
+9 A 81
+SELECT * FROM t2 WHERE c2 = c1 * c1 ORDER BY c1,c2;
+c1 c12 c2
+2 A 4
+3 A 9
+5 A 25
+8 A 64
+9 A 81
+connection master;
+UPDATE t2 SET c2 = c1*c1 WHERE c2 != c1*c1;
+SELECT * FROM t2 WHERE c2 = c1 * c1 ORDER BY c1,c2;
+c1 c12 c2
+1 A 1
+2 A 4
+3 A 9
+4 A 16
+5 A 25
+6 A 36
+7 A 49
+8 A 64
+9 A 81
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+SELECT * FROM t2 WHERE c2 = c1 * c1 ORDER BY c1,c2;
+c1 c12 c2
+1 A 1
+2 A 4
+3 A 9
+4 A 16
+5 A 25
+6 A 36
+7 A 49
+8 A 64
+9 A 81
+connection master;
+UPDATE t2 SET c12 = 'Q' WHERE c1 = 1 AND c2 = 999;
+SELECT * FROM t2 ORDER BY c1,c2;
+c1 c12 c2
+1 A 1
+2 A 4
+3 A 9
+4 A 16
+5 A 25
+6 A 36
+7 A 49
+8 A 64
+9 A 81
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+SELECT * FROM t2 ORDER BY c1,c2;
+c1 c12 c2
+1 A 1
+2 A 4
+3 A 9
+4 A 16
+5 A 25
+6 A 36
+7 A 49
+8 A 64
+9 A 81
+connection master;
+DELETE FROM t2 WHERE c1 % 4 = 0;
+SELECT * FROM t2 ORDER BY c1,c2;
+c1 c12 c2
+1 A 1
+2 A 4
+3 A 9
+5 A 25
+6 A 36
+7 A 49
+9 A 81
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+SELECT * FROM t2 ORDER BY c1,c2;
+c1 c12 c2
+1 A 1
+2 A 4
+3 A 9
+5 A 25
+6 A 36
+7 A 49
+9 A 81
+connection master;
+UPDATE t2 SET c12='X';
+connection master;
+CREATE TABLE t3 (C1 CHAR(1), C2 CHAR(1), pk1 INT, C3 CHAR(1), pk2 INT, PRIMARY KEY (pk1,pk2)) ENGINE = 'INNODB' ;
+INSERT INTO t3 VALUES ('A','B',1,'B',1), ('X','Y',2,'B',1), ('X','X',3,'B',1);
+INSERT INTO t3 VALUES ('A','C',1,'B',2), ('X','Z',2,'B',2), ('A','A',3,'B',2);
+SELECT * FROM t3 ORDER BY C1,C2;
+C1 C2 pk1 C3 pk2
+A A 3 B 2
+A B 1 B 1
+A C 1 B 2
+X X 3 B 1
+X Y 2 B 1
+X Z 2 B 2
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+SELECT * FROM t3 ORDER BY C1,C2;
+C1 C2 pk1 C3 pk2
+A A 3 B 2
+A B 1 B 1
+A C 1 B 2
+X X 3 B 1
+X Y 2 B 1
+X Z 2 B 2
+connection master;
+DELETE FROM t3 WHERE C1 = C2;
+SELECT * FROM t3 ORDER BY C1,C2;
+C1 C2 pk1 C3 pk2
+A B 1 B 1
+A C 1 B 2
+X Y 2 B 1
+X Z 2 B 2
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+SELECT * FROM t3 ORDER BY C1,C2;
+C1 C2 pk1 C3 pk2
+A B 1 B 1
+A C 1 B 2
+X Y 2 B 1
+X Z 2 B 2
+connection master;
+UPDATE t3 SET C2 = 'I' WHERE C1 = 'A' AND C2 = 'C';
+SELECT * FROM t3 ORDER BY C1,C2;
+C1 C2 pk1 C3 pk2
+A B 1 B 1
+A I 1 B 2
+X Y 2 B 1
+X Z 2 B 2
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+SELECT * FROM t3 ORDER BY C1,C2;
+C1 C2 pk1 C3 pk2
+A B 1 B 1
+A I 1 B 2
+X Y 2 B 1
+X Z 2 B 2
+connection master;
+CREATE TABLE t6 (C1 CHAR(1), C2 CHAR(1), C3 INT) ENGINE = 'INNODB' ;
+INSERT INTO t6 VALUES ('A','B',1), ('X','Y',2), ('X','X',3);
+INSERT INTO t6 VALUES ('A','C',4), ('X','Z',5), ('A','A',6);
+SELECT * FROM t6 ORDER BY C3;
+C1 C2 C3
+A B 1
+X Y 2
+X X 3
+A C 4
+X Z 5
+A A 6
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+SELECT * FROM t6 ORDER BY C3;
+C1 C2 C3
+A B 1
+X Y 2
+X X 3
+A C 4
+X Z 5
+A A 6
+connection master;
+DELETE FROM t6 WHERE C1 = C2;
+SELECT * FROM t6 ORDER BY C3;
+C1 C2 C3
+A B 1
+X Y 2
+A C 4
+X Z 5
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+SELECT * FROM t6 ORDER BY C3;
+C1 C2 C3
+A B 1
+X Y 2
+A C 4
+X Z 5
+connection master;
+UPDATE t6 SET C2 = 'I' WHERE C1 = 'A' AND C2 = 'C';
+SELECT * FROM t6 ORDER BY C3;
+C1 C2 C3
+A B 1
+X Y 2
+A I 4
+X Z 5
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+SELECT * FROM t6 ORDER BY C3;
+C1 C2 C3
+A B 1
+X Y 2
+A I 4
+X Z 5
+connection master;
+CREATE TABLE t5 (C1 CHAR(1), C2 CHAR(1), C3 INT PRIMARY KEY) ENGINE = 'INNODB' ;
+INSERT INTO t5 VALUES ('A','B',1), ('X','Y',2), ('X','X',3);
+INSERT INTO t5 VALUES ('A','C',4), ('X','Z',5), ('A','A',6);
+UPDATE t5,t2,t3 SET t5.C2='Q', t2.c12='R', t3.C3 ='S' WHERE t5.C1 = t2.c12 AND t5.C1 = t3.C1;
+SELECT * FROM t5,t2,t3 WHERE t5.C2='Q' AND t2.c12='R' AND t3.C3 ='S' ORDER BY t5.C3,t2.c1,t3.pk1,t3.pk2;
+C1 C2 C3 c1 c12 c2 C1 C2 pk1 C3 pk2
+X Q 2 1 R 1 X Y 2 S 1
+X Q 2 1 R 1 X Z 2 S 2
+X Q 2 2 R 4 X Y 2 S 1
+X Q 2 2 R 4 X Z 2 S 2
+X Q 2 3 R 9 X Y 2 S 1
+X Q 2 3 R 9 X Z 2 S 2
+X Q 2 5 R 25 X Y 2 S 1
+X Q 2 5 R 25 X Z 2 S 2
+X Q 2 6 R 36 X Y 2 S 1
+X Q 2 6 R 36 X Z 2 S 2
+X Q 2 7 R 49 X Y 2 S 1
+X Q 2 7 R 49 X Z 2 S 2
+X Q 2 9 R 81 X Y 2 S 1
+X Q 2 9 R 81 X Z 2 S 2
+X Q 3 1 R 1 X Y 2 S 1
+X Q 3 1 R 1 X Z 2 S 2
+X Q 3 2 R 4 X Y 2 S 1
+X Q 3 2 R 4 X Z 2 S 2
+X Q 3 3 R 9 X Y 2 S 1
+X Q 3 3 R 9 X Z 2 S 2
+X Q 3 5 R 25 X Y 2 S 1
+X Q 3 5 R 25 X Z 2 S 2
+X Q 3 6 R 36 X Y 2 S 1
+X Q 3 6 R 36 X Z 2 S 2
+X Q 3 7 R 49 X Y 2 S 1
+X Q 3 7 R 49 X Z 2 S 2
+X Q 3 9 R 81 X Y 2 S 1
+X Q 3 9 R 81 X Z 2 S 2
+X Q 5 1 R 1 X Y 2 S 1
+X Q 5 1 R 1 X Z 2 S 2
+X Q 5 2 R 4 X Y 2 S 1
+X Q 5 2 R 4 X Z 2 S 2
+X Q 5 3 R 9 X Y 2 S 1
+X Q 5 3 R 9 X Z 2 S 2
+X Q 5 5 R 25 X Y 2 S 1
+X Q 5 5 R 25 X Z 2 S 2
+X Q 5 6 R 36 X Y 2 S 1
+X Q 5 6 R 36 X Z 2 S 2
+X Q 5 7 R 49 X Y 2 S 1
+X Q 5 7 R 49 X Z 2 S 2
+X Q 5 9 R 81 X Y 2 S 1
+X Q 5 9 R 81 X Z 2 S 2
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+SELECT * FROM t5,t2,t3 WHERE t5.C2='Q' AND t2.c12='R' AND t3.C3 ='S' ORDER BY t5.C3,t2.c1,t3.pk1,t3.pk2;
+C1 C2 C3 c1 c12 c2 C1 C2 pk1 C3 pk2
+X Q 2 1 R 1 X Y 2 S 1
+X Q 2 1 R 1 X Z 2 S 2
+X Q 2 2 R 4 X Y 2 S 1
+X Q 2 2 R 4 X Z 2 S 2
+X Q 2 3 R 9 X Y 2 S 1
+X Q 2 3 R 9 X Z 2 S 2
+X Q 2 5 R 25 X Y 2 S 1
+X Q 2 5 R 25 X Z 2 S 2
+X Q 2 6 R 36 X Y 2 S 1
+X Q 2 6 R 36 X Z 2 S 2
+X Q 2 7 R 49 X Y 2 S 1
+X Q 2 7 R 49 X Z 2 S 2
+X Q 2 9 R 81 X Y 2 S 1
+X Q 2 9 R 81 X Z 2 S 2
+X Q 3 1 R 1 X Y 2 S 1
+X Q 3 1 R 1 X Z 2 S 2
+X Q 3 2 R 4 X Y 2 S 1
+X Q 3 2 R 4 X Z 2 S 2
+X Q 3 3 R 9 X Y 2 S 1
+X Q 3 3 R 9 X Z 2 S 2
+X Q 3 5 R 25 X Y 2 S 1
+X Q 3 5 R 25 X Z 2 S 2
+X Q 3 6 R 36 X Y 2 S 1
+X Q 3 6 R 36 X Z 2 S 2
+X Q 3 7 R 49 X Y 2 S 1
+X Q 3 7 R 49 X Z 2 S 2
+X Q 3 9 R 81 X Y 2 S 1
+X Q 3 9 R 81 X Z 2 S 2
+X Q 5 1 R 1 X Y 2 S 1
+X Q 5 1 R 1 X Z 2 S 2
+X Q 5 2 R 4 X Y 2 S 1
+X Q 5 2 R 4 X Z 2 S 2
+X Q 5 3 R 9 X Y 2 S 1
+X Q 5 3 R 9 X Z 2 S 2
+X Q 5 5 R 25 X Y 2 S 1
+X Q 5 5 R 25 X Z 2 S 2
+X Q 5 6 R 36 X Y 2 S 1
+X Q 5 6 R 36 X Z 2 S 2
+X Q 5 7 R 49 X Y 2 S 1
+X Q 5 7 R 49 X Z 2 S 2
+X Q 5 9 R 81 X Y 2 S 1
+X Q 5 9 R 81 X Z 2 S 2
+connection slave;
+SET @saved_slave_type_conversions = @@SLAVE_TYPE_CONVERSIONS;
+SET GLOBAL SLAVE_TYPE_CONVERSIONS = 'ALL_LOSSY';
+connection master;
+CREATE TABLE t4 (C1 CHAR(1) PRIMARY KEY, B1 BIT(1), B2 BIT(1) NOT NULL DEFAULT 0, C2 CHAR(1) NOT NULL DEFAULT 'A') ENGINE = 'INNODB' ;
+INSERT INTO t4 SET C1 = 1;
+SELECT C1,HEX(B1),HEX(B2) FROM t4 ORDER BY C1;
+C1 HEX(B1) HEX(B2)
+1 NULL 0
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+SELECT C1,HEX(B1),HEX(B2) FROM t4 ORDER BY C1;
+C1 HEX(B1) HEX(B2)
+1 NULL 0
+SET GLOBAL SLAVE_TYPE_CONVERSIONS = @saved_slave_type_conversions;
+connection master;
+CREATE TABLE t7 (C1 INT PRIMARY KEY, C2 INT) ENGINE = 'INNODB' ;
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+--- original values ---
+INSERT INTO t7 VALUES (1,3), (2,6), (3,9);
+SELECT * FROM t7 ORDER BY C1;
+C1 C2
+1 3
+2 6
+3 9
+set @@global.slave_exec_mode= 'IDEMPOTENT';
+connection master;
+--- new values inserted ---
+INSERT INTO t7 VALUES (1,2), (2,4), (3,6);
+SELECT * FROM t7 ORDER BY C1;
+C1 C2
+1 2
+2 4
+3 6
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+set @@global.slave_exec_mode= default;
+--- old values should be overwritten by replicated values ---
+SELECT * FROM t7 ORDER BY C1;
+C1 C2
+1 2
+2 4
+3 6
+connection master;
+CREATE TABLE t8 (a INT PRIMARY KEY, b INT UNIQUE, c INT UNIQUE) ENGINE = 'INNODB' ;
+INSERT INTO t8 VALUES (99,99,99);
+INSERT INTO t8 VALUES (99,22,33);
+ERROR 23000: Duplicate entry '99' for key 'PRIMARY'
+INSERT INTO t8 VALUES (11,99,33);
+ERROR 23000: Duplicate entry '99' for key 'b'
+INSERT INTO t8 VALUES (11,22,99);
+ERROR 23000: Duplicate entry '99' for key 'c'
+SELECT * FROM t8 ORDER BY a;
+a b c
+99 99 99
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+SELECT * FROM t8 ORDER BY a;
+a b c
+99 99 99
+INSERT INTO t8 VALUES (1,2,3), (2,4,6), (3,6,9);
+SELECT * FROM t8 ORDER BY a;
+a b c
+1 2 3
+2 4 6
+3 6 9
+99 99 99
+set @@global.slave_exec_mode= 'IDEMPOTENT';
+connection master;
+INSERT INTO t8 VALUES (2,4,8);
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+set @@global.slave_exec_mode= default;
+SELECT * FROM t8 ORDER BY a;
+a b c
+1 2 3
+2 4 8
+3 6 9
+99 99 99
+**** Test for BUG#31552 ****
+connection master;
+DELETE FROM t1;
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+include/rpl_reset.inc
+connection master;
+INSERT INTO t1 VALUES ('K','K'), ('L','L'), ('M','M');
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+set @@global.slave_exec_mode= 'IDEMPOTENT';
+DELETE FROM t1 WHERE C1 = 'L';
+connection master;
+DELETE FROM t1;
+SELECT COUNT(*) FROM t1 ORDER BY c1,c2;
+COUNT(*) 0
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+set @@global.slave_exec_mode= default;
+include/check_slave_is_running.inc
+SELECT COUNT(*) FROM t1 ORDER BY c1,c2;
+COUNT(*) 0
+**** Test for BUG#37076 ****
+connection master;
+DROP TABLE IF EXISTS t1;
+CREATE TABLE t1 (a TIMESTAMP, b DATETIME, c DATE);
+INSERT INTO t1 VALUES(
+'2005-11-14 01:01:01', '2005-11-14 01:01:02', '2005-11-14');
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+SELECT * FROM t1;
+a b c
+2005-11-14 01:01:01 2005-11-14 01:01:02 2005-11-14
+connection master;
+DROP TABLE IF EXISTS t1,t2,t3,t4,t5,t6,t7,t8;
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection master;
+CREATE TABLE t1 (i INT NOT NULL,
+c CHAR(16) CHARACTER SET utf8 NOT NULL,
+j INT NOT NULL) ENGINE = 'INNODB' ;
+CREATE TABLE t2 (i INT NOT NULL,
+c CHAR(16) CHARACTER SET utf8 NOT NULL,
+j INT NOT NULL) ENGINE = 'INNODB' ;
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+ALTER TABLE t2 MODIFY c CHAR(128) CHARACTER SET utf8 NOT NULL;
+connection master;
+CREATE TABLE t3 (i INT NOT NULL,
+c CHAR(128) CHARACTER SET utf8 NOT NULL,
+j INT NOT NULL) ENGINE = 'INNODB' ;
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+ALTER TABLE t3 MODIFY c CHAR(16) CHARACTER SET utf8 NOT NULL;
+connection master;
+CREATE TABLE t4 (i INT NOT NULL,
+c CHAR(128) CHARACTER SET utf8 NOT NULL,
+j INT NOT NULL) ENGINE = 'INNODB' ;
+CREATE TABLE t5 (i INT NOT NULL,
+c CHAR(255) CHARACTER SET utf8 NOT NULL,
+j INT NOT NULL) ENGINE = 'INNODB' ;
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+ALTER TABLE t5 MODIFY c CHAR(16) CHARACTER SET utf8 NOT NULL;
+connection master;
+CREATE TABLE t6 (i INT NOT NULL,
+c CHAR(255) CHARACTER SET utf8 NOT NULL,
+j INT NOT NULL) ENGINE = 'INNODB' ;
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+ALTER TABLE t6 MODIFY c CHAR(128) CHARACTER SET utf8 NOT NULL;
+connection master;
+CREATE TABLE t7 (i INT NOT NULL,
+c CHAR(255) CHARACTER SET utf8 NOT NULL,
+j INT NOT NULL) ENGINE = 'INNODB' ;
+connection slave;
+SET @saved_slave_type_conversions = @@slave_type_conversions;
+SET GLOBAL SLAVE_TYPE_CONVERSIONS = 'ALL_NON_LOSSY';
+[expecting slave to replicate correctly]
+connection master;
+INSERT INTO t1 VALUES (1, "", 1);
+INSERT INTO t1 VALUES (2, repeat(_utf8'a', 16), 2);
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+include/diff_tables.inc [master:t1, slave:t1]
+[expecting slave to replicate correctly]
+connection master;
+INSERT INTO t2 VALUES (1, "", 1);
+INSERT INTO t2 VALUES (2, repeat(_utf8'a', 16), 2);
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+include/diff_tables.inc [master:t2, slave:t2]
+connection slave;
+SET GLOBAL SLAVE_TYPE_CONVERSIONS = @saved_slave_type_conversions;
+call mtr.add_suppression("Slave SQL.*Table definition on master and slave does not match: Column 1 size mismatch.* error.* 1535");
+call mtr.add_suppression("Slave SQL.*Could not execute Delete_rows event on table test.t1.* error.* 1032");
+call mtr.add_suppression("Slave SQL.*Column 1 of table .test.t.. cannot be converted from type.*, error.* 1677");
+call mtr.add_suppression("Can't find record in 't1'");
+include/rpl_reset.inc
+[expecting slave to replicate correctly]
+connection master;
+INSERT INTO t4 VALUES (1, "", 1);
+INSERT INTO t4 VALUES (2, repeat(_utf8'a', 128), 2);
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+include/diff_tables.inc [master:t4, slave:t4]
+[expecting slave to stop]
+connection master;
+INSERT INTO t5 VALUES (1, "", 1);
+INSERT INTO t5 VALUES (2, repeat(_utf8'a', 255), 2);
+connection slave;
+include/wait_for_slave_sql_error.inc [errno=1677]
+Last_SQL_Error = 'Column 1 of table 'test.t5' cannot be converted from type 'char(765 octets)' to type 'char(48 octets) character set utf8mb3''
+include/rpl_reset.inc
+[expecting slave to stop]
+connection master;
+INSERT INTO t6 VALUES (1, "", 1);
+INSERT INTO t6 VALUES (2, repeat(_utf8'a', 255), 2);
+connection slave;
+include/wait_for_slave_sql_error.inc [errno=1677]
+Last_SQL_Error = 'Column 1 of table 'test.t6' cannot be converted from type 'char(765 octets)' to type 'char(384 octets) character set utf8mb3''
+include/rpl_reset.inc
+[expecting slave to replicate correctly]
+connection master;
+INSERT INTO t7 VALUES (1, "", 1);
+INSERT INTO t7 VALUES (2, repeat(_utf8'a', 255), 2);
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+include/diff_tables.inc [master:t7, slave:t7]
+connection master;
+drop table t1, t2, t3, t4, t5, t6, t7;
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection master;
+CREATE TABLE t1 (a INT PRIMARY KEY) ENGINE='INNODB' ;
+INSERT INTO t1 VALUES (1), (2), (3);
+UPDATE t1 SET a = 10;
+ERROR 23000: Duplicate entry '10' for key 'PRIMARY'
+INSERT INTO t1 VALUES (4);
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+include/diff_tables.inc [master:t1, slave:t1]
+connection master;
+drop table t1;
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection master;
+DROP TABLE IF EXISTS t1, t2;
+CREATE TABLE t1 (
+`pk` int(11) NOT NULL AUTO_INCREMENT,
+`int_nokey` int(11) NOT NULL,
+`int_key` int(11) NOT NULL,
+`date_key` date NOT NULL,
+`date_nokey` date NOT NULL,
+`time_key` time NOT NULL,
+`time_nokey` time NOT NULL,
+`datetime_key` datetime NOT NULL,
+`datetime_nokey` datetime NOT NULL,
+`varchar_key` varchar(1) NOT NULL,
+`varchar_nokey` varchar(1) NOT NULL,
+PRIMARY KEY (`pk`),
+KEY `int_key` (`int_key`),
+KEY `date_key` (`date_key`),
+KEY `time_key` (`time_key`),
+KEY `datetime_key` (`datetime_key`),
+KEY `varchar_key` (`varchar_key`)
+) ENGINE='INNODB' ;
+INSERT INTO t1 VALUES (1,8,5,'0000-00-00','0000-00-00','10:37:38','10:37:38','0000-00-00 00:00:00','0000-00-00 00:00:00','p','p'),(2,0,9,'0000-00-00','0000-00-00','00:00:00','00:00:00','2007-10-14 00:00:00','2007-10-14 00:00:00','d','d');
+CREATE TABLE t2 (
+`pk` int(11) NOT NULL AUTO_INCREMENT,
+`int_nokey` int(11) NOT NULL,
+`int_key` int(11) NOT NULL,
+`date_key` date NOT NULL,
+`date_nokey` date NOT NULL,
+`time_key` time NOT NULL,
+`time_nokey` time NOT NULL,
+`datetime_key` datetime NOT NULL,
+`datetime_nokey` datetime NOT NULL,
+`varchar_key` varchar(1) NOT NULL,
+`varchar_nokey` varchar(1) NOT NULL,
+PRIMARY KEY (`pk`),
+KEY `int_key` (`int_key`),
+KEY `date_key` (`date_key`),
+KEY `time_key` (`time_key`),
+KEY `datetime_key` (`datetime_key`),
+KEY `varchar_key` (`varchar_key`)
+) ENGINE='INNODB' ;
+INSERT INTO t2 VALUES (1,1,6,'2005-12-23','2005-12-23','02:24:28','02:24:28','0000-00-00 00:00:00','0000-00-00 00:00:00','g','g'),(2,0,3,'2009-09-14','2009-09-14','00:00:00','00:00:00','2000-01-30 16:39:40','2000-01-30 16:39:40','q','q'),(3,0,3,'0000-00-00','0000-00-00','00:00:00','00:00:00','0000-00-00 00:00:00','0000-00-00 00:00:00','c','c'),(4,1,6,'2007-03-29','2007-03-29','15:49:00','15:49:00','0000-00-00 00:00:00','0000-00-00 00:00:00','m','m'),(5,4,0,'2002-12-04','2002-12-04','00:00:00','00:00:00','0000-00-00 00:00:00','0000-00-00 00:00:00','o','o'),(6,9,0,'2005-01-28','2005-01-28','00:00:00','00:00:00','2001-05-18 00:00:00','2001-05-18 00:00:00','w','w'),(7,6,0,'0000-00-00','0000-00-00','06:57:25','06:57:25','0000-00-00 00:00:00','0000-00-00 00:00:00','m','m'),(8,0,0,'0000-00-00','0000-00-00','00:00:00','00:00:00','0000-00-00 00:00:00','0000-00-00 00:00:00','z','z'),(9,4,6,'2006-08-15','2006-08-15','00:00:00','00:00:00','2002-04-12 14:44:25','2002-04-12 14:44:25','j','j'),(10,0,5,'2006-12-20','2006-12-20','10:13:53','10:13:53','2008-07-22 00:00:00','2008-07-22 00:00:00','y','y'),(11,9,7,'0000-00-00','0000-00-00','00:00:00','00:00:00','2004-07-05 00:00:00','2004-07-05 00:00:00','{','{'),(12,4,3,'2007-01-26','2007-01-26','23:00:51','23:00:51','2001-05-16 00:00:00','2001-05-16 00:00:00','f','f'),(13,7,0,'2004-03-27','2004-03-27','00:00:00','00:00:00','2005-01-24 03:30:37','2005-01-24 03:30:37','',''),(14,6,0,'2006-07-26','2006-07-26','18:43:57','18:43:57','0000-00-00 00:00:00','0000-00-00 00:00:00','{','{'),(15,0,6,'2000-01-14','2000-01-14','00:00:00','00:00:00','2000-09-21 00:00:00','2000-09-21 00:00:00','o','o'),(16,9,8,'0000-00-00','0000-00-00','21:15:08','21:15:08','0000-00-00 00:00:00','0000-00-00 00:00:00','a','a'),(17,2,0,'2004-10-27','2004-10-27','00:00:00','00:00:00','2004-03-24 22:13:43','2004-03-24 22:13:43','',''),(18,7,4,'0000-00-00','0000-00-00','08:38:27','08:38:27','2002-03-18 19:51:44','2002-03-18 19:51:44','t','t'),(19,5,3,'2008-03-07','2008-03-07','03:29:07','03:29:07','2007-12-01 18:44:44','2007-12-01 18:44:44','t','t'),(20,0,0,'2002-04-09','2002-04-09','16:06:03','16:06:03','2009-04-22 00:00:00','2009-04-22 00:00:00','n','n');
+DELETE FROM t2 WHERE `int_key` < 3 LIMIT 1;
+UPDATE t1 SET `int_key` = 3 ORDER BY `pk` LIMIT 4;
+DELETE FROM t2 WHERE `int_key` < 3 LIMIT 1;
+DELETE FROM t2 WHERE `pk` < 6 LIMIT 1;
+UPDATE t1 SET `int_key` = 6 ORDER BY `pk` LIMIT 3;
+DELETE FROM t2 WHERE `pk` < 6 LIMIT 1;
+UPDATE t1 SET `pk` = 6 ORDER BY `int_key` LIMIT 6;
+ERROR 23000: Duplicate entry '6' for key 'PRIMARY'
+DELETE FROM t2 WHERE `pk` < 7 LIMIT 1;
+UPDATE t1 SET `int_key` = 4 ORDER BY `pk` LIMIT 6;
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+*** results: t2 must be consistent ****
+include/diff_tables.inc [master:t2, slave:t2]
+connection master;
+DROP TABLE t1, t2;
+EOF OF TESTS
+connection master;
+CREATE TABLE t1 (a int) ENGINE='INNODB' ;
+INSERT IGNORE INTO t1 VALUES (NULL);
+INSERT INTO t1 ( a ) VALUES ( 0 );
+INSERT INTO t1 ( a ) VALUES ( 9 );
+INSERT INTO t1 ( a ) VALUES ( 2 );
+INSERT INTO t1 ( a ) VALUES ( 9 );
+INSERT INTO t1 ( a ) VALUES ( 5 );
+UPDATE t1 SET a = 5 WHERE a = 9;
+DELETE FROM t1 WHERE a < 6;
+UPDATE t1 SET a = 9 WHERE a < 3;
+INSERT INTO t1 ( a ) VALUES ( 3 );
+UPDATE t1 SET a = 0 WHERE a < 4;
+UPDATE t1 SET a = 8 WHERE a < 5;
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+include/diff_tables.inc [master:t1, slave:t1]
+connection master;
+drop table t1;
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection master;
+connection slave;
+SET @saved_slave_type_conversions = @@SLAVE_TYPE_CONVERSIONS;
+SET GLOBAL SLAVE_TYPE_CONVERSIONS = 'ALL_LOSSY';
+connection master;
+CREATE TABLE t1 (a bit) ENGINE='INNODB' ;
+INSERT IGNORE INTO t1 VALUES (NULL);
+INSERT INTO t1 ( a ) VALUES ( 0 );
+UPDATE t1 SET a = 0 WHERE a = 1 LIMIT 3;
+INSERT IGNORE INTO t1 ( a ) VALUES ( 5 );
+DELETE FROM t1 WHERE a < 2 LIMIT 4;
+DELETE FROM t1 WHERE a < 9 LIMIT 4;
+INSERT IGNORE INTO t1 ( a ) VALUES ( 9 );
+UPDATE t1 SET a = 8 WHERE a = 0 LIMIT 6;
+INSERT IGNORE INTO t1 ( a ) VALUES ( 8 );
+UPDATE t1 SET a = 0 WHERE a < 6 LIMIT 0;
+INSERT IGNORE INTO t1 ( a ) VALUES ( 4 );
+INSERT IGNORE INTO t1 ( a ) VALUES ( 3 );
+UPDATE t1 SET a = 0 WHERE a = 7 LIMIT 6;
+DELETE FROM t1 WHERE a = 4 LIMIT 7;
+UPDATE IGNORE t1 SET a = 9 WHERE a < 2 LIMIT 9;
+UPDATE t1 SET a = 0 WHERE a < 9 LIMIT 2;
+DELETE FROM t1 WHERE a < 0 LIMIT 5;
+INSERT IGNORE INTO t1 ( a ) VALUES ( 5 );
+UPDATE IGNORE t1 SET a = 4 WHERE a < 6 LIMIT 4;
+INSERT IGNORE INTO t1 ( a ) VALUES ( 5 );
+UPDATE IGNORE t1 SET a = 9 WHERE a < 5 LIMIT 8;
+DELETE FROM t1 WHERE a < 8 LIMIT 8;
+INSERT IGNORE INTO t1 ( a ) VALUES ( 6 );
+DELETE FROM t1 WHERE a < 6 LIMIT 7;
+UPDATE t1 SET a = 7 WHERE a = 3 LIMIT 7;
+UPDATE t1 SET a = 8 WHERE a = 0 LIMIT 6;
+INSERT IGNORE INTO t1 ( a ) VALUES ( 7 );
+DELETE FROM t1 WHERE a < 9 LIMIT 4;
+INSERT IGNORE INTO t1 ( a ) VALUES ( 7 );
+INSERT IGNORE INTO t1 ( a ) VALUES ( 6 );
+UPDATE t1 SET a = 8 WHERE a = 3 LIMIT 4;
+DELETE FROM t1 WHERE a = 2 LIMIT 9;
+DELETE FROM t1 WHERE a = 1 LIMIT 4;
+UPDATE t1 SET a = 4 WHERE a = 2 LIMIT 7;
+INSERT INTO t1 ( a ) VALUES ( 0 );
+DELETE FROM t1 WHERE a < 3 LIMIT 0;
+UPDATE t1 SET a = 8 WHERE a = 5 LIMIT 2;
+INSERT INTO t1 ( a ) VALUES ( 1 );
+UPDATE IGNORE t1 SET a = 9 WHERE a < 5 LIMIT 3;
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+SET GLOBAL SLAVE_TYPE_CONVERSIONS = @saved_slave_type_conversions;
+include/diff_tables.inc [master:t1, slave:t1]
+connection master;
+drop table t1;
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+include/rpl_end.inc
diff --git a/mysql-test/suite/binlog_in_engine/rpl_row_basic_3innodb_misf.test b/mysql-test/suite/binlog_in_engine/rpl_row_basic_3innodb_misf.test
new file mode 100644
index 0000000000000..c7d2c151a5a22
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/rpl_row_basic_3innodb_misf.test
@@ -0,0 +1,11 @@
+--source include/have_innodb.inc
+--source include/have_binlog_format_row.inc
+--source include/master-slave.inc
+
+connection slave;
+let $bit_field_special = ALL_LOSSY;
+let $type= 'INNODB' ;
+let $extra_index= ;
+--source ../rpl/include/rpl_row_basic.test
+
+--source include/rpl_end.inc
diff --git a/mysql-test/suite/binlog_in_engine/savepoint.result b/mysql-test/suite/binlog_in_engine/savepoint.result
new file mode 100644
index 0000000000000..2e6bcded665a1
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/savepoint.result
@@ -0,0 +1,518 @@
+include/master-slave.inc
+[connection master]
+CREATE TABLE t1 (i INT, a INT, b TEXT, PRIMARY KEY(i, a)) ENGINE=InnoDB;
+CREATE TABLE t2 (i INT, a INT, b TEXT, PRIMARY KEY(i, a)) ENGINE=MyISAM;
+SET @b= REPEAT('$', 0);
+BEGIN;
+INSERT INTO t1 VALUES (0, 1, @b);
+SAVEPOINT s1;
+INSERT INTO t1 VALUES (0, 2, @b);
+SAVEPOINT s2;
+INSERT INTO t1 VALUES (0, 3, @b);
+SAVEPOINT s3;
+INSERT INTO t1 VALUES (0, 4, @b);
+ROLLBACK TO s2;
+INSERT INTO t1 VALUES (0, 5, @b);
+ROLLBACK TO s2;
+INSERT INTO t1 VALUES (0, 6, @b);
+SAVEPOINT s4;
+INSERT INTO t1 VALUES (0, 7, @b);
+SAVEPOINT s5;
+ROLLBACK TO s5;
+INSERT INTO t1 VALUES (0, 8, @b);
+COMMIT;
+SELECT a, length(b) FROM t1 WHERE i=0 ORDER BY a;
+a length(b)
+1 0
+2 0
+6 0
+7 0
+8 0
+BEGIN;
+INSERT INTO t1 VALUES (0, 10, @b);
+SAVEPOINT s10;
+INSERT INTO t1 VALUES (0, 11, @b);
+INSERT INTO t2 VALUES (0, 12, @b);
+ROLLBACK TO s10;
+Warnings:
+Warning 1196 Some non-transactional changed tables couldn't be rolled back
+COMMIT;
+SELECT a, length(b) FROM t1 WHERE i=0 AND a>=10 ORDER BY a;
+a length(b)
+10 0
+SELECT a, length(b) FROM t2 WHERE i=0 ORDER BY a;
+a length(b)
+12 0
+BEGIN;
+UPDATE t1 SET a=a+1000 WHERE i=0;
+UPDATE t1 SET b='x' WHERE i=0;
+ROLLBACK;
+BEGIN;
+INSERT INTO t1
+VALUES (0, 101, @b), (0, 102, @b), (0, 103, @b), (0, 104, @b), (0, 105, @b);
+UPDATE t1 SET a=a-104 WHERE i=0 AND a > 100;
+ERROR 23000: Duplicate entry '0-1' for key 'PRIMARY'
+UPDATE t1 SET a=a+10 WHERE i=0 AND a > 100;
+COMMIT;
+SELECT a, length(b) FROM t1 WHERE i=0 AND a >= 100 ORDER BY a;
+a length(b)
+111 0
+112 0
+113 0
+114 0
+115 0
+SET @b= REPEAT('$', 10);
+BEGIN;
+INSERT INTO t1 VALUES (1, 1, @b);
+SAVEPOINT s1;
+INSERT INTO t1 VALUES (1, 2, @b);
+SAVEPOINT s2;
+INSERT INTO t1 VALUES (1, 3, @b);
+SAVEPOINT s3;
+INSERT INTO t1 VALUES (1, 4, @b);
+ROLLBACK TO s2;
+INSERT INTO t1 VALUES (1, 5, @b);
+ROLLBACK TO s2;
+INSERT INTO t1 VALUES (1, 6, @b);
+SAVEPOINT s4;
+INSERT INTO t1 VALUES (1, 7, @b);
+SAVEPOINT s5;
+ROLLBACK TO s5;
+INSERT INTO t1 VALUES (1, 8, @b);
+COMMIT;
+SELECT a, length(b) FROM t1 WHERE i=1 ORDER BY a;
+a length(b)
+1 10
+2 10
+6 10
+7 10
+8 10
+BEGIN;
+INSERT INTO t1 VALUES (1, 10, @b);
+SAVEPOINT s10;
+INSERT INTO t1 VALUES (1, 11, @b);
+INSERT INTO t2 VALUES (1, 12, @b);
+ROLLBACK TO s10;
+Warnings:
+Warning 1196 Some non-transactional changed tables couldn't be rolled back
+COMMIT;
+SELECT a, length(b) FROM t1 WHERE i=1 AND a>=10 ORDER BY a;
+a length(b)
+10 10
+SELECT a, length(b) FROM t2 WHERE i=1 ORDER BY a;
+a length(b)
+12 10
+BEGIN;
+UPDATE t1 SET a=a+1000 WHERE i=1;
+UPDATE t1 SET b='x' WHERE i=1;
+ROLLBACK;
+BEGIN;
+INSERT INTO t1
+VALUES (1, 101, @b), (1, 102, @b), (1, 103, @b), (1, 104, @b), (1, 105, @b);
+UPDATE t1 SET a=a-104 WHERE i=1 AND a > 100;
+ERROR 23000: Duplicate entry '1-1' for key 'PRIMARY'
+UPDATE t1 SET a=a+10 WHERE i=1 AND a > 100;
+COMMIT;
+SELECT a, length(b) FROM t1 WHERE i=1 AND a >= 100 ORDER BY a;
+a length(b)
+111 10
+112 10
+113 10
+114 10
+115 10
+SET @b= REPEAT('$', 100);
+BEGIN;
+INSERT INTO t1 VALUES (2, 1, @b);
+SAVEPOINT s1;
+INSERT INTO t1 VALUES (2, 2, @b);
+SAVEPOINT s2;
+INSERT INTO t1 VALUES (2, 3, @b);
+SAVEPOINT s3;
+INSERT INTO t1 VALUES (2, 4, @b);
+ROLLBACK TO s2;
+INSERT INTO t1 VALUES (2, 5, @b);
+ROLLBACK TO s2;
+INSERT INTO t1 VALUES (2, 6, @b);
+SAVEPOINT s4;
+INSERT INTO t1 VALUES (2, 7, @b);
+SAVEPOINT s5;
+ROLLBACK TO s5;
+INSERT INTO t1 VALUES (2, 8, @b);
+COMMIT;
+SELECT a, length(b) FROM t1 WHERE i=2 ORDER BY a;
+a length(b)
+1 100
+2 100
+6 100
+7 100
+8 100
+BEGIN;
+INSERT INTO t1 VALUES (2, 10, @b);
+SAVEPOINT s10;
+INSERT INTO t1 VALUES (2, 11, @b);
+INSERT INTO t2 VALUES (2, 12, @b);
+ROLLBACK TO s10;
+Warnings:
+Warning 1196 Some non-transactional changed tables couldn't be rolled back
+COMMIT;
+SELECT a, length(b) FROM t1 WHERE i=2 AND a>=10 ORDER BY a;
+a length(b)
+10 100
+SELECT a, length(b) FROM t2 WHERE i=2 ORDER BY a;
+a length(b)
+12 100
+BEGIN;
+UPDATE t1 SET a=a+1000 WHERE i=2;
+UPDATE t1 SET b='x' WHERE i=2;
+ROLLBACK;
+BEGIN;
+INSERT INTO t1
+VALUES (2, 101, @b), (2, 102, @b), (2, 103, @b), (2, 104, @b), (2, 105, @b);
+UPDATE t1 SET a=a-104 WHERE i=2 AND a > 100;
+ERROR 23000: Duplicate entry '2-1' for key 'PRIMARY'
+UPDATE t1 SET a=a+10 WHERE i=2 AND a > 100;
+COMMIT;
+SELECT a, length(b) FROM t1 WHERE i=2 AND a >= 100 ORDER BY a;
+a length(b)
+111 100
+112 100
+113 100
+114 100
+115 100
+SET @b= REPEAT('$', 642);
+BEGIN;
+INSERT INTO t1 VALUES (3, 1, @b);
+SAVEPOINT s1;
+INSERT INTO t1 VALUES (3, 2, @b);
+SAVEPOINT s2;
+INSERT INTO t1 VALUES (3, 3, @b);
+SAVEPOINT s3;
+INSERT INTO t1 VALUES (3, 4, @b);
+ROLLBACK TO s2;
+INSERT INTO t1 VALUES (3, 5, @b);
+ROLLBACK TO s2;
+INSERT INTO t1 VALUES (3, 6, @b);
+SAVEPOINT s4;
+INSERT INTO t1 VALUES (3, 7, @b);
+SAVEPOINT s5;
+ROLLBACK TO s5;
+INSERT INTO t1 VALUES (3, 8, @b);
+COMMIT;
+SELECT a, length(b) FROM t1 WHERE i=3 ORDER BY a;
+a length(b)
+1 642
+2 642
+6 642
+7 642
+8 642
+BEGIN;
+INSERT INTO t1 VALUES (3, 10, @b);
+SAVEPOINT s10;
+INSERT INTO t1 VALUES (3, 11, @b);
+INSERT INTO t2 VALUES (3, 12, @b);
+ROLLBACK TO s10;
+Warnings:
+Warning 1196 Some non-transactional changed tables couldn't be rolled back
+COMMIT;
+SELECT a, length(b) FROM t1 WHERE i=3 AND a>=10 ORDER BY a;
+a length(b)
+10 642
+SELECT a, length(b) FROM t2 WHERE i=3 ORDER BY a;
+a length(b)
+12 642
+BEGIN;
+UPDATE t1 SET a=a+1000 WHERE i=3;
+UPDATE t1 SET b='x' WHERE i=3;
+ROLLBACK;
+BEGIN;
+INSERT INTO t1
+VALUES (3, 101, @b), (3, 102, @b), (3, 103, @b), (3, 104, @b), (3, 105, @b);
+UPDATE t1 SET a=a-104 WHERE i=3 AND a > 100;
+ERROR 23000: Duplicate entry '3-1' for key 'PRIMARY'
+UPDATE t1 SET a=a+10 WHERE i=3 AND a > 100;
+COMMIT;
+SELECT a, length(b) FROM t1 WHERE i=3 AND a >= 100 ORDER BY a;
+a length(b)
+111 642
+112 642
+113 642
+114 642
+115 642
+SET @b= REPEAT('$', 3930);
+BEGIN;
+INSERT INTO t1 VALUES (4, 1, @b);
+SAVEPOINT s1;
+INSERT INTO t1 VALUES (4, 2, @b);
+SAVEPOINT s2;
+INSERT INTO t1 VALUES (4, 3, @b);
+SAVEPOINT s3;
+INSERT INTO t1 VALUES (4, 4, @b);
+ROLLBACK TO s2;
+INSERT INTO t1 VALUES (4, 5, @b);
+ROLLBACK TO s2;
+INSERT INTO t1 VALUES (4, 6, @b);
+SAVEPOINT s4;
+INSERT INTO t1 VALUES (4, 7, @b);
+SAVEPOINT s5;
+ROLLBACK TO s5;
+INSERT INTO t1 VALUES (4, 8, @b);
+COMMIT;
+SELECT a, length(b) FROM t1 WHERE i=4 ORDER BY a;
+a length(b)
+1 3930
+2 3930
+6 3930
+7 3930
+8 3930
+BEGIN;
+INSERT INTO t1 VALUES (4, 10, @b);
+SAVEPOINT s10;
+INSERT INTO t1 VALUES (4, 11, @b);
+INSERT INTO t2 VALUES (4, 12, @b);
+ROLLBACK TO s10;
+Warnings:
+Warning 1196 Some non-transactional changed tables couldn't be rolled back
+COMMIT;
+SELECT a, length(b) FROM t1 WHERE i=4 AND a>=10 ORDER BY a;
+a length(b)
+10 3930
+SELECT a, length(b) FROM t2 WHERE i=4 ORDER BY a;
+a length(b)
+12 3930
+BEGIN;
+UPDATE t1 SET a=a+1000 WHERE i=4;
+UPDATE t1 SET b='x' WHERE i=4;
+ROLLBACK;
+BEGIN;
+INSERT INTO t1
+VALUES (4, 101, @b), (4, 102, @b), (4, 103, @b), (4, 104, @b), (4, 105, @b);
+UPDATE t1 SET a=a-104 WHERE i=4 AND a > 100;
+ERROR 23000: Duplicate entry '4-1' for key 'PRIMARY'
+UPDATE t1 SET a=a+10 WHERE i=4 AND a > 100;
+COMMIT;
+SELECT a, length(b) FROM t1 WHERE i=4 AND a >= 100 ORDER BY a;
+a length(b)
+111 3930
+112 3930
+113 3930
+114 3930
+115 3930
+SET @b= REPEAT('$', 16000);
+BEGIN;
+INSERT INTO t1 VALUES (5, 1, @b);
+SAVEPOINT s1;
+INSERT INTO t1 VALUES (5, 2, @b);
+SAVEPOINT s2;
+INSERT INTO t1 VALUES (5, 3, @b);
+SAVEPOINT s3;
+INSERT INTO t1 VALUES (5, 4, @b);
+ROLLBACK TO s2;
+INSERT INTO t1 VALUES (5, 5, @b);
+ROLLBACK TO s2;
+INSERT INTO t1 VALUES (5, 6, @b);
+SAVEPOINT s4;
+INSERT INTO t1 VALUES (5, 7, @b);
+SAVEPOINT s5;
+ROLLBACK TO s5;
+INSERT INTO t1 VALUES (5, 8, @b);
+COMMIT;
+SELECT a, length(b) FROM t1 WHERE i=5 ORDER BY a;
+a length(b)
+1 16000
+2 16000
+6 16000
+7 16000
+8 16000
+BEGIN;
+INSERT INTO t1 VALUES (5, 10, @b);
+SAVEPOINT s10;
+INSERT INTO t1 VALUES (5, 11, @b);
+INSERT INTO t2 VALUES (5, 12, @b);
+ROLLBACK TO s10;
+Warnings:
+Warning 1196 Some non-transactional changed tables couldn't be rolled back
+COMMIT;
+SELECT a, length(b) FROM t1 WHERE i=5 AND a>=10 ORDER BY a;
+a length(b)
+10 16000
+SELECT a, length(b) FROM t2 WHERE i=5 ORDER BY a;
+a length(b)
+12 16000
+BEGIN;
+UPDATE t1 SET a=a+1000 WHERE i=5;
+UPDATE t1 SET b='x' WHERE i=5;
+ROLLBACK;
+BEGIN;
+INSERT INTO t1
+VALUES (5, 101, @b), (5, 102, @b), (5, 103, @b), (5, 104, @b), (5, 105, @b);
+UPDATE t1 SET a=a-104 WHERE i=5 AND a > 100;
+ERROR 23000: Duplicate entry '5-1' for key 'PRIMARY'
+UPDATE t1 SET a=a+10 WHERE i=5 AND a > 100;
+COMMIT;
+SELECT a, length(b) FROM t1 WHERE i=5 AND a >= 100 ORDER BY a;
+a length(b)
+111 16000
+112 16000
+113 16000
+114 16000
+115 16000
+SET @b= REPEAT('$', 40000);
+BEGIN;
+INSERT INTO t1 VALUES (6, 1, @b);
+SAVEPOINT s1;
+INSERT INTO t1 VALUES (6, 2, @b);
+SAVEPOINT s2;
+INSERT INTO t1 VALUES (6, 3, @b);
+SAVEPOINT s3;
+INSERT INTO t1 VALUES (6, 4, @b);
+ROLLBACK TO s2;
+INSERT INTO t1 VALUES (6, 5, @b);
+ROLLBACK TO s2;
+INSERT INTO t1 VALUES (6, 6, @b);
+SAVEPOINT s4;
+INSERT INTO t1 VALUES (6, 7, @b);
+SAVEPOINT s5;
+ROLLBACK TO s5;
+INSERT INTO t1 VALUES (6, 8, @b);
+COMMIT;
+SELECT a, length(b) FROM t1 WHERE i=6 ORDER BY a;
+a length(b)
+1 40000
+2 40000
+6 40000
+7 40000
+8 40000
+BEGIN;
+INSERT INTO t1 VALUES (6, 10, @b);
+SAVEPOINT s10;
+INSERT INTO t1 VALUES (6, 11, @b);
+INSERT INTO t2 VALUES (6, 12, @b);
+ROLLBACK TO s10;
+Warnings:
+Warning 1196 Some non-transactional changed tables couldn't be rolled back
+COMMIT;
+SELECT a, length(b) FROM t1 WHERE i=6 AND a>=10 ORDER BY a;
+a length(b)
+10 40000
+SELECT a, length(b) FROM t2 WHERE i=6 ORDER BY a;
+a length(b)
+12 40000
+BEGIN;
+UPDATE t1 SET a=a+1000 WHERE i=6;
+UPDATE t1 SET b='x' WHERE i=6;
+ROLLBACK;
+BEGIN;
+INSERT INTO t1
+VALUES (6, 101, @b), (6, 102, @b), (6, 103, @b), (6, 104, @b), (6, 105, @b);
+UPDATE t1 SET a=a-104 WHERE i=6 AND a > 100;
+ERROR 23000: Duplicate entry '6-1' for key 'PRIMARY'
+UPDATE t1 SET a=a+10 WHERE i=6 AND a > 100;
+COMMIT;
+SELECT a, length(b) FROM t1 WHERE i=6 AND a >= 100 ORDER BY a;
+a length(b)
+111 40000
+112 40000
+113 40000
+114 40000
+115 40000
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+*** Slave data checksums with master, all ok. ***
+*** Test re-using savepoint names in the same transaction
+TRUNCATE TABLE t1;
+INSERT INTO t1 VALUES (1, 0, 'a');
+BEGIN;
+INSERT INTO t1 VALUES (2, 0, 'b');
+SAVEPOINT A;
+SAVEPOINT B;
+SAVEPOINT A;
+ROLLBACK TO SAVEPOINT B;
+COMMIT;
+BEGIN;
+SAVEPOINT s1;
+INSERT INTO t1 VALUES (3, 1, 's1');
+SAVEPOINT s2;
+INSERT INTO t1 VALUES (4, 1, 's2');
+SAVEPOINT s1;
+SAVEPOINT s3;
+INSERT INTO t1 VALUES (5, 1, 's3');
+ROLLBACK TO SAVEPOINT s1;
+SAVEPOINT s2;
+INSERT INTO t1 VALUES (6, 2, REPEAT('', 10000));
+SAVEPOINT s4;
+INSERT INTO t1 VALUES (7, 2, 's4');
+SAVEPOINT s1;
+INSERT INTO t1 VALUES (8, 2, REPEAT('', 10000));
+SAVEPOINT s3;
+INSERT INTO t1 VALUES (9, 2, REPEAT('', 10000));
+SAVEPOINT s2;
+SAVEPOINT s5;
+INSERT INTO t1 VALUES (10, 2, 's5');
+SAVEPOINT s1;
+INSERT INTO t1 VALUES (11, 2, REPEAT('', 10000));
+SAVEPOINT s5;
+SAVEPOINT s6;
+INSERT INTO t1 VALUES (12, 2, 's6');
+SAVEPOINT s4;
+INSERT INTO t1 VALUES (13, 2, 's4');
+ROLLBACK TO SAVEPOINT s6;
+INSERT INTO t1 VALUES (14, 2, REPEAT('', 10000));
+ROLLBACK TO SAVEPOINT s1;
+COMMIT;
+SELECT i, a, LENGTH(b) FROM t1 ORDER BY i, a;
+i a LENGTH(b)
+1 0 1
+2 0 1
+3 1 2
+4 1 2
+6 2 40000
+7 2 2
+8 2 40000
+9 2 40000
+10 2 2
+*** Test an edge case where the savepoint is just at the cache-size boundary.
+*** Test RELEASE SAVEPOINT.
+BEGIN;
+SAVEPOINT s1;
+INSERT INTO t1(i, a) VALUES (20, 0);
+SAVEPOINT s2;
+INSERT INTO t1(i, a) VALUES (20, 1);
+SAVEPOINT s3;
+INSERT INTO t1(i, a, b) VALUES (20, 2, REPEAT('2', 40000));
+SAVEPOINT s4;
+SAVEPOINT s5;
+INSERT INTO t1(i, a) VALUES (20, 3);
+SAVEPOINT s6;
+INSERT INTO t1(i, a, b) VALUES (20, 4, REPEAT('4', 40000));
+RELEASE SAVEPOINT s5;
+SAVEPOINT s7;
+ROLLBACK TO s7;
+SAVEPOINT s8;
+INSERT INTO t1(i, a) VALUES (20, 5);
+ROLLBACK TO s4;
+INSERT INTO t1(i, a) VALUES (20, 6);
+COMMIT;
+SELECT i, a, LENGTH(b) FROM t1 WHERE i=20 ORDER BY a;
+i a LENGTH(b)
+20 0 NULL
+20 1 NULL
+20 2 40000
+20 6 NULL
+*** Test savepoint combined with non-transactional OOB spill.
+BEGIN;
+INSERT INTO t1 VALUES (30, 0, REPEAT('a', 40000));
+INSERT INTO t1 VALUES (30, 1, REPEAT('b', 20000));
+SAVEPOINT s1;
+INSERT INTO t2 VALUES (30, 2, REPEAT('x', 63000));
+COMMIT;
+SELECT a, LENGTH(b) FROM t1 WHERE i=30 ORDER BY a;
+a LENGTH(b)
+0 40000
+1 20000
+SELECT a, LENGTH(b) FROM t2 WHERE i=30 ORDER BY a;
+a LENGTH(b)
+2 63000
+connection master;
+DROP TABLE t1, t2;
+include/rpl_end.inc
diff --git a/mysql-test/suite/binlog_in_engine/savepoint.test b/mysql-test/suite/binlog_in_engine/savepoint.test
new file mode 100644
index 0000000000000..95a95ca2b56fa
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/savepoint.test
@@ -0,0 +1,238 @@
+--source include/have_binlog_format_row.inc
+--source include/master-slave.inc
+--source include/have_innodb_binlog.inc
+
+CREATE TABLE t1 (i INT, a INT, b TEXT, PRIMARY KEY(i, a)) ENGINE=InnoDB;
+CREATE TABLE t2 (i INT, a INT, b TEXT, PRIMARY KEY(i, a)) ENGINE=MyISAM;
+
+# Add different amounts of data, to test various cases where event
+# groups fit or do not fit in case, are binlogged / not binlogged as
+# oob data.
+--let $i = 0
+while ($i <= 6) {
+ if ($i == 0) {
+ SET @b= REPEAT('$', 0);
+ }
+ if ($i == 1) {
+ SET @b= REPEAT('$', 10);
+ }
+ if ($i == 2) {
+ SET @b= REPEAT('$', 100);
+ }
+ if ($i == 3) {
+ SET @b= REPEAT('$', 642);
+ }
+ if ($i == 4) {
+ SET @b= REPEAT('$', 3930);
+ }
+ if ($i == 5) {
+ SET @b= REPEAT('$', 16000);
+ }
+ if ($i == 6) {
+ SET @b= REPEAT('$', 40000);
+ }
+ BEGIN;
+ eval INSERT INTO t1 VALUES ($i, 1, @b);
+ SAVEPOINT s1;
+ eval INSERT INTO t1 VALUES ($i, 2, @b);
+ SAVEPOINT s2;
+ eval INSERT INTO t1 VALUES ($i, 3, @b);
+ SAVEPOINT s3;
+ eval INSERT INTO t1 VALUES ($i, 4, @b);
+ ROLLBACK TO s2;
+ eval INSERT INTO t1 VALUES ($i, 5, @b);
+ ROLLBACK TO s2;
+ eval INSERT INTO t1 VALUES ($i, 6, @b);
+ SAVEPOINT s4;
+ eval INSERT INTO t1 VALUES ($i, 7, @b);
+ SAVEPOINT s5;
+ ROLLBACK TO s5;
+ eval INSERT INTO t1 VALUES ($i, 8, @b);
+ COMMIT;
+ eval SELECT a, length(b) FROM t1 WHERE i=$i ORDER BY a;
+
+ BEGIN;
+ eval INSERT INTO t1 VALUES ($i, 10, @b);
+ SAVEPOINT s10;
+ eval INSERT INTO t1 VALUES ($i, 11, @b);
+ eval INSERT INTO t2 VALUES ($i, 12, @b);
+ ROLLBACK TO s10;
+ COMMIT;
+
+ eval SELECT a, length(b) FROM t1 WHERE i=$i AND a>=10 ORDER BY a;
+ eval SELECT a, length(b) FROM t2 WHERE i=$i ORDER BY a;
+
+ # Test a full rollback.
+ BEGIN;
+ eval UPDATE t1 SET a=a+1000 WHERE i=$i;
+ eval UPDATE t1 SET b='x' WHERE i=$i;
+ ROLLBACK;
+
+ # Test a statement that fails and is rolled back but the remaining
+ # transaction is committed.
+ BEGIN;
+ eval INSERT INTO t1
+ VALUES ($i, 101, @b), ($i, 102, @b), ($i, 103, @b), ($i, 104, @b), ($i, 105, @b);
+ --error ER_DUP_ENTRY
+ eval UPDATE t1 SET a=a-104 WHERE i=$i AND a > 100;
+ eval UPDATE t1 SET a=a+10 WHERE i=$i AND a > 100;
+ COMMIT;
+ eval SELECT a, length(b) FROM t1 WHERE i=$i AND a >= 100 ORDER BY a;
+
+ inc $i;
+}
+
+# Seeing the events generated useful for debugging, but hard to maintain the
+# .result file over time, better to check slave data vs. master.
+#--let $binlog_file= binlog-000000.ibb
+#--let $binlog_start= 4096
+#--source include/show_binlog_events.inc
+
+--let $master_checksum1= query_get_value(CHECKSUM TABLE t1, Checksum, 1)
+--let $master_checksum2= query_get_value(CHECKSUM TABLE t2, Checksum, 1)
+
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
+
+--let $slave_checksum1= query_get_value(CHECKSUM TABLE t1, Checksum, 1)
+--let slave_checksum2= query_get_value(CHECKSUM TABLE t2, Checksum, 1)
+
+--let $ok= 1
+if ($master_checksum1 != $slave_checksum1) {
+ --let $ok= 0
+}
+if ($master_checksum2 != $slave_checksum2) {
+ --let $ok= 0
+}
+if (!$ok) {
+ --connection master
+ --echo *** Data on master: ***
+ SELECT i, a, length(b) FROM t1 ORDER BY i, a;
+ SELECT i, a, length(b) FROM t2 ORDER BY i, a;
+ --connection slave
+ --echo *** Data on slave: ***
+ SELECT i, a, length(b) FROM t1 ORDER BY i, a;
+ SELECT i, a, length(b) FROM t2 ORDER BY i, a;
+ --die Slave data differs from master. Master checksums $master_checksum1 $master_checksum2, but slave $slave_checksum1 $slave_checksum2
+}
+if ($ok) {
+ --echo *** Slave data checksums with master, all ok. ***
+}
+
+
+--echo *** Test re-using savepoint names in the same transaction
+
+# When a savepoint name is re-used, it overrides and the old savepoint of the
+# same name is effectively deleted.
+TRUNCATE TABLE t1;
+INSERT INTO t1 VALUES (1, 0, 'a');
+BEGIN;
+INSERT INTO t1 VALUES (2, 0, 'b');
+SAVEPOINT A;
+SAVEPOINT B;
+SAVEPOINT A;
+ROLLBACK TO SAVEPOINT B;
+COMMIT;
+
+# A longer test, with multiple savepoint name re-use, and with
+# large data that requires spilling as OOB.
+BEGIN;
+SAVEPOINT s1;
+INSERT INTO t1 VALUES (3, 1, 's1');
+SAVEPOINT s2;
+INSERT INTO t1 VALUES (4, 1, 's2');
+SAVEPOINT s1;
+SAVEPOINT s3;
+INSERT INTO t1 VALUES (5, 1, 's3');
+ROLLBACK TO SAVEPOINT s1;
+SAVEPOINT s2;
+INSERT INTO t1 VALUES (6, 2, REPEAT('', 10000));
+SAVEPOINT s4;
+INSERT INTO t1 VALUES (7, 2, 's4');
+SAVEPOINT s1;
+INSERT INTO t1 VALUES (8, 2, REPEAT('', 10000));
+SAVEPOINT s3;
+INSERT INTO t1 VALUES (9, 2, REPEAT('', 10000));
+SAVEPOINT s2;
+SAVEPOINT s5;
+INSERT INTO t1 VALUES (10, 2, 's5');
+SAVEPOINT s1;
+INSERT INTO t1 VALUES (11, 2, REPEAT('', 10000));
+SAVEPOINT s5;
+SAVEPOINT s6;
+INSERT INTO t1 VALUES (12, 2, 's6');
+SAVEPOINT s4;
+INSERT INTO t1 VALUES (13, 2, 's4');
+ROLLBACK TO SAVEPOINT s6;
+INSERT INTO t1 VALUES (14, 2, REPEAT('', 10000));
+ROLLBACK TO SAVEPOINT s1;
+COMMIT;
+
+SELECT i, a, LENGTH(b) FROM t1 ORDER BY i, a;
+
+
+--echo *** Test an edge case where the savepoint is just at the cache-size boundary.
+
+# Try to hit the condition that the second savepoint is exactly at position
+# 32768, the trx cache size.
+# At the time of writing, this occurs when the REPEAT value of the second INSERT
+# is exactly 32327. But we try all values in an interval around this point, to
+# have a decent chance of still covering the interesting case as couple extra
+# bytes get added or removed by random code changes.
+--let $sz= 32250
+--disable_query_log
+while ($sz <= 32500) {
+ BEGIN;
+ eval INSERT INTO t1 VALUES ($i, 10, '');
+ SAVEPOINT s1;
+ eval INSERT INTO t1 VALUES($i, 11, REPEAT('x', $sz));
+ SAVEPOINT s2;
+ eval INSERT INTO t1 VALUES ($i, 12, '');
+ ROLLBACK TO s2;
+ ROLLBACK TO s1;
+ ROLLBACK;
+ inc $sz;
+}
+--enable_query_log
+
+
+--echo *** Test RELEASE SAVEPOINT.
+BEGIN;
+SAVEPOINT s1;
+INSERT INTO t1(i, a) VALUES (20, 0);
+SAVEPOINT s2;
+INSERT INTO t1(i, a) VALUES (20, 1);
+SAVEPOINT s3;
+INSERT INTO t1(i, a, b) VALUES (20, 2, REPEAT('2', 40000));
+SAVEPOINT s4;
+SAVEPOINT s5;
+INSERT INTO t1(i, a) VALUES (20, 3);
+SAVEPOINT s6;
+INSERT INTO t1(i, a, b) VALUES (20, 4, REPEAT('4', 40000));
+RELEASE SAVEPOINT s5;
+SAVEPOINT s7;
+ROLLBACK TO s7;
+SAVEPOINT s8;
+INSERT INTO t1(i, a) VALUES (20, 5);
+ROLLBACK TO s4;
+INSERT INTO t1(i, a) VALUES (20, 6);
+COMMIT;
+SELECT i, a, LENGTH(b) FROM t1 WHERE i=20 ORDER BY a;
+
+
+--echo *** Test savepoint combined with non-transactional OOB spill.
+BEGIN;
+INSERT INTO t1 VALUES (30, 0, REPEAT('a', 40000));
+INSERT INTO t1 VALUES (30, 1, REPEAT('b', 20000));
+SAVEPOINT s1;
+INSERT INTO t2 VALUES (30, 2, REPEAT('x', 63000));
+COMMIT;
+
+SELECT a, LENGTH(b) FROM t1 WHERE i=30 ORDER BY a;
+SELECT a, LENGTH(b) FROM t2 WHERE i=30 ORDER BY a;
+
+
+--connection master
+DROP TABLE t1, t2;
+--source include/rpl_end.inc
diff --git a/mysql-test/suite/binlog_in_engine/spider.opt b/mysql-test/suite/binlog_in_engine/spider.opt
new file mode 100644
index 0000000000000..e67ef041ae9fd
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/spider.opt
@@ -0,0 +1 @@
+--plugin-load=$HA_SPIDER_SO
diff --git a/mysql-test/suite/binlog_in_engine/spider.result b/mysql-test/suite/binlog_in_engine/spider.result
new file mode 100644
index 0000000000000..7c5925a0638c9
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/spider.result
@@ -0,0 +1,11 @@
+SET spider_same_server_link= on;
+create server s foreign data wrapper mysql options (host "127.0.0.1", database "spider_db", user "root", port $MASTER_MYPORT);;
+CREATE DATABASE spider_db;
+CREATE TABLE spider_db.t (a INT) ENGINE=InnoDB;
+CREATE TABLE t (a INT) ENGINE=SPIDER COMMENT = 'wrapper "mysql", srv "s", table "t"';
+Warnings:
+Warning 138 Spider table params in COMMENT or CONNECTION strings have been deprecated and will be removed in a future release. Please use table options instead.
+INSERT INTO t VALUES (2);
+DROP TABLE t;
+DROP DATABASE spider_db;
+DROP SERVER s;
diff --git a/mysql-test/suite/binlog_in_engine/spider.test b/mysql-test/suite/binlog_in_engine/spider.test
new file mode 100644
index 0000000000000..f0e86f5f8e93a
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/spider.test
@@ -0,0 +1,20 @@
+if (`SELECT COUNT(*) = 0 FROM INFORMATION_SCHEMA.ENGINES WHERE engine = 'spider' AND support IN ('YES', 'DEFAULT', 'ENABLED')`)
+{
+ --skip Test requires Spider engine
+}
+
+--source include/have_binlog_format_mixed.inc
+--source include/have_innodb_binlog.inc
+
+SET spider_same_server_link= on;
+--evalp create server s foreign data wrapper mysql options (host "127.0.0.1", database "spider_db", user "root", port $MASTER_MYPORT);
+
+CREATE DATABASE spider_db;
+
+CREATE TABLE spider_db.t (a INT) ENGINE=InnoDB;
+CREATE TABLE t (a INT) ENGINE=SPIDER COMMENT = 'wrapper "mysql", srv "s", table "t"';
+INSERT INTO t VALUES (2);
+
+DROP TABLE t;
+DROP DATABASE spider_db;
+DROP SERVER s;
diff --git a/mysql-test/suite/binlog_in_engine/xa.result b/mysql-test/suite/binlog_in_engine/xa.result
new file mode 100644
index 0000000000000..e971115f87c91
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/xa.result
@@ -0,0 +1,262 @@
+include/reset_master.inc
+CREATE TABLE t1(a INT PRIMARY KEY, b INT) ENGINE=InnoDB;
+*** Test basic XA COMMIT from same or different client connection.
+connect con1,localhost,root,,;
+XA START 'a';
+INSERT INTO t1 VALUES (1, 0);
+INSERT INTO t1 VALUES (2, 0);
+INSERT INTO t1 VALUES (3, 0);
+XA END 'a';
+XA PREPARE 'a';
+connection default;
+disconnect con1;
+RESET MASTER;
+ERROR HY000: Cannot execute RESET MASTER as the binlog is in use by an active transaction
+connect con2,localhost,root,,;
+XA COMMIT 'a';
+connect con1_roll,localhost,root,,;
+XA START 'a_roll';
+INSERT INTO t1 VALUES (1001, 0);
+INSERT INTO t1 VALUES (1002, 0);
+INSERT INTO t1 VALUES (1003, 0);
+XA END 'a_roll';
+XA PREPARE 'a_roll';
+connection default;
+disconnect con1_roll;
+connection con2;
+XA ROLLBACK 'a_roll';
+connection default;
+XA START 'b';
+UPDATE t1 SET b=2 WHERE a=1;
+XA END 'b';
+XA PREPARE 'b';
+XA COMMIT 'b';
+XA START 'b_roll';
+UPDATE t1 SET b=3 WHERE a=1;
+XA END 'b_roll';
+XA PREPARE 'b_roll';
+XA ROLLBACK 'b_roll';
+XA START 'c';
+UPDATE t1 SET b=3 WHERE a=3;
+XA END 'c';
+XA COMMIT 'c' ONE PHASE;
+SELECT * FROM t1 ORDER BY a;
+a b
+1 2
+2 0
+3 3
+CREATE TABLE t2 (a INT) ENGINE=MyISAM;
+CREATE TABLE t3 (id INT PRIMARY KEY) ENGINE=InnoDB;
+INSERT INTO t2 VALUES (1),(2);
+INSERT INTO t3 VALUES (1),(2);
+XA START '1';
+REPLACE INTO t2 SELECT * FROM t2;
+REPLACE INTO t3 SELECT * FROM t3;
+XA END '1';
+XA PREPARE '1';
+XA RECOVER FORMAT='SQL';
+formatID gtrid_length bqual_length data
+1 1 0 '1'
+XA ROLLBACK '1';
+Warnings:
+Warning 1196 Some non-transactional changed tables couldn't be rolled back
+DROP TABLE t2, t3;
+CREATE TABLE t2 (a INT PRIMARY KEY) ENGINE=InnoDB;
+INSERT INTO t2 VALUES (1);
+CREATE SEQUENCE s ENGINE=InnoDB;
+XA START '2';
+SELECT NEXT VALUE FOR s;
+NEXT VALUE FOR s
+1
+REPLACE INTO t2 SELECT * FROM t2;
+XA END '2';
+XA PREPARE '2';
+XA RECOVER FORMAT='SQL';
+formatID gtrid_length bqual_length data
+1 1 0 '2'
+XA ROLLBACK '2';
+DROP SEQUENCE s;
+DROP TABLE t2;
+include/show_binlog_events.inc
+Log_name Pos Event_type Server_id End_log_pos Info
+binlog-000000.ibb # Gtid # # BEGIN GTID #-#-#
+binlog-000000.ibb # Query # # use `test`; INSERT INTO t1 VALUES (1, 0)
+binlog-000000.ibb # Query # # use `test`; INSERT INTO t1 VALUES (2, 0)
+binlog-000000.ibb # Query # # use `test`; INSERT INTO t1 VALUES (3, 0)
+binlog-000000.ibb # Query # # COMMIT
+binlog-000000.ibb # Gtid # # BEGIN GTID #-#-#
+binlog-000000.ibb # Query # # use `test`; UPDATE t1 SET b=2 WHERE a=1
+binlog-000000.ibb # Query # # COMMIT
+binlog-000000.ibb # Gtid # # BEGIN GTID #-#-#
+binlog-000000.ibb # Query # # use `test`; UPDATE t1 SET b=3 WHERE a=3
+binlog-000000.ibb # Xid # # COMMIT /* XID */
+binlog-000000.ibb # Gtid # # GTID #-#-#
+binlog-000000.ibb # Query # # use `test`; CREATE TABLE t2 (a INT) ENGINE=MyISAM
+binlog-000000.ibb # Gtid # # GTID #-#-#
+binlog-000000.ibb # Query # # use `test`; CREATE TABLE t3 (id INT PRIMARY KEY) ENGINE=InnoDB
+binlog-000000.ibb # Gtid # # BEGIN GTID #-#-#
+binlog-000000.ibb # Query # # use `test`; INSERT INTO t2 VALUES (1),(2)
+binlog-000000.ibb # Query # # COMMIT
+binlog-000000.ibb # Gtid # # BEGIN GTID #-#-#
+binlog-000000.ibb # Query # # use `test`; INSERT INTO t3 VALUES (1),(2)
+binlog-000000.ibb # Xid # # COMMIT /* XID */
+binlog-000000.ibb # Gtid # # BEGIN GTID #-#-#
+binlog-000000.ibb # Annotate_rows # # REPLACE INTO t2 SELECT * FROM t2
+binlog-000000.ibb # Table_map # # table_id: # (test.t2)
+binlog-000000.ibb # Write_rows_v1 # # table_id: # flags: STMT_END_F
+binlog-000000.ibb # Query # # COMMIT
+binlog-000000.ibb # Gtid # # GTID #-#-#
+binlog-000000.ibb # Query # # use `test`; DROP TABLE `t2`,`t3` /* generated by server */
+binlog-000000.ibb # Gtid # # GTID #-#-#
+binlog-000000.ibb # Query # # use `test`; CREATE TABLE t2 (a INT PRIMARY KEY) ENGINE=InnoDB
+binlog-000000.ibb # Gtid # # BEGIN GTID #-#-#
+binlog-000000.ibb # Query # # use `test`; INSERT INTO t2 VALUES (1)
+binlog-000000.ibb # Xid # # COMMIT /* XID */
+binlog-000000.ibb # Gtid # # GTID #-#-#
+binlog-000000.ibb # Query # # use `test`; CREATE SEQUENCE s ENGINE=InnoDB
+binlog-000000.ibb # Gtid # # BEGIN GTID #-#-#
+binlog-000000.ibb # Annotate_rows # # SELECT NEXT VALUE FOR s
+binlog-000000.ibb # Table_map # # table_id: # (test.s)
+binlog-000000.ibb # Write_rows_v1 # # table_id: # flags: STMT_END_F
+binlog-000000.ibb # Query # # COMMIT
+binlog-000000.ibb # Gtid # # GTID #-#-#
+binlog-000000.ibb # Query # # use `test`; DROP SEQUENCE `s` /* generated by server */
+binlog-000000.ibb # Gtid # # GTID #-#-#
+binlog-000000.ibb # Query # # use `test`; DROP TABLE `t2` /* generated by server */
+*** Test that we will not purge a file that is needed by an active XA transaction.
+connection default;
+ALTER TABLE t1 ADD COLUMN c LONGBLOB;
+UPDATE t1 SET b=10 WHERE a=1;
+connection con2;
+SET SESSION binlog_format=ROW;
+XA START 'd';
+INSERT INTO t1 VALUES (10, 2, REPEAT('#', 40000));
+connect con3,localhost,root,,;
+SET SESSION binlog_format=ROW;
+XA START 'e';
+INSERT INTO t1 VALUES (110, 2, REPEAT('#', 40000));
+connection default;
+UPDATE t1 SET b=11 WHERE a=1;
+FLUSH BINARY LOGS;
+UPDATE t1 SET b=12 WHERE a=1;
+connection con2;
+INSERT INTO t1 VALUES (11, 2, REPEAT('*', 40000));
+connection con3;
+INSERT INTO t1 VALUES (111, 2, REPEAT('*', 40000));
+connection default;
+UPDATE t1 SET b=13 WHERE a=1;
+FLUSH BINARY LOGS;
+UPDATE t1 SET b=14 WHERE a=1;
+UPDATE t1 SET b=15 WHERE a=1;
+FLUSH BINARY LOGS;
+UPDATE t1 SET b=16 WHERE a=1;
+FLUSH BINARY LOGS;
+connection default;
+SET @old_min_slaves= @@GLOBAL.slave_connections_needed_for_purge;
+SET GLOBAL slave_connections_needed_for_purge= 0;
+PURGE BINARY LOGS TO 'binlog-000001.ibb';
+ERROR HY000: A purgeable log is in use, will not purge
+connection con2;
+XA END 'd';
+XA PREPARE 'd';
+connection con3;
+XA END 'e';
+XA PREPARE 'e';
+connection default;
+PURGE BINARY LOGS TO 'binlog-000001.ibb';
+ERROR HY000: A purgeable log is in use, will not purge
+connection con2;
+XA COMMIT 'd';
+connection con3;
+XA ROLLBACK 'e';
+connection default;
+PURGE BINARY LOGS TO 'binlog-000001.ibb';
+FLUSH BINARY LOGS;
+UPDATE t1 SET b=17 WHERE a=1;
+FLUSH BINARY LOGS;
+UPDATE t1 SET b=18 WHERE a=1;
+PURGE BINARY LOGS TO 'binlog-000001.ibb';
+SET GLOBAL slave_connections_needed_for_purge= @old_min_slaves;
+*** Test disconnecting in XA transaction before XA PREPARE.
+connection con2;
+SET SESSION binlog_format=MIXED;
+XA BEGIN 'f';
+CREATE TEMPORARY TABLE tmp ENGINE=InnoDB AS SELECT a FROM t1;
+disconnect con2;
+connection con3;
+SET SESSION binlog_format=MIXED;
+XA BEGIN 'g';
+UPDATE t1 SET b=19 WHERE a=1;
+XA END 'g';
+disconnect con3;
+connection default;
+BEGIN;
+SELECT b FROM t1 WHERE a=1 FOR UPDATE;
+b
+18
+ROLLBACK;
+*** Test duplicate SAVEPOINT with XA
+XA BEGIN 'x';
+DELETE FROM t1;
+SAVEPOINT A;
+SAVEPOINT B;
+SAVEPOINT A;
+ROLLBACK TO SAVEPOINT B;
+XA END 'x';
+XA ROLLBACK 'x';
+*** Test server shutdown after XA PREPARE.
+XA BEGIN 'y';
+INSERT INTO t1 VALUES (201, 0, '');
+XA END 'y';
+XA PREPARE 'y';
+# restart
+XA ROLLBACK 'y';
+*** Test binlogging of empty XA transaction.
+SET binlog_format=row;
+DELETE FROM t1 WHERE a=4242;
+INSERT INTO t1 VALUES (4242, 0, 'z1');
+XA BEGIN 'z';
+SELECT a, b, LENGTH(c) FROM t1 ORDER BY a LIMIT 3;
+a b LENGTH(c)
+1 18 NULL
+2 0 NULL
+3 3 NULL
+INSERT IGNORE INTO t1 VALUES (4242, 0, 'z2');
+Warnings:
+Warning 1062 Duplicate entry '4242' for key 'PRIMARY'
+SELECT a, b, LENGTH(c) FROM t1 WHERE a>3 ORDER BY a LIMIT 1;
+a b LENGTH(c)
+10 2 40000
+XA END 'z';
+XA PREPARE 'z';
+XA COMMIT 'z';
+XA BEGIN 'z';
+UPDATE t1 SET b=10 WHERE a=4242;
+XA END 'z';
+XA PREPARE 'z';
+XA COMMIT 'z';
+SELECT a, b, LENGTH(c) FROM t1 WHERE a=4242;
+a b LENGTH(c)
+4242 10 2
+*** Test XA PREPARE when the transaction is empty because statements rolled back on error
+CREATE TRIGGER u AFTER INSERT ON t1 FOR EACH ROW BEGIN RELEASE SAVEPOINT A ; END //
+XA BEGIN 'xid44';
+INSERT IGNORE INTO t1(a) VALUES (10000);
+ERROR 42000: SAVEPOINT A does not exist
+XA END 'xid44';
+XA PREPARE 'xid44';
+XA COMMIT 'xid44';
+DROP TRIGGER u;
+XA BEGIN 'xid44';
+SAVEPOINT A;
+ROLLBACK TO A;
+ROLLBACK TO A;
+INSERT INTO t1(a) VALUES (10001);
+XA END 'xid44';
+XA PREPARE 'xid44';
+# restart
+XA COMMIT 'xid44';
+SELECT * FROM t1 WHERE a >= 10000 ORDER BY a;
+a b c
+10001 NULL NULL
+DROP TABLE t1;
diff --git a/mysql-test/suite/binlog_in_engine/xa.test b/mysql-test/suite/binlog_in_engine/xa.test
new file mode 100644
index 0000000000000..1ecef1742f6c4
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/xa.test
@@ -0,0 +1,287 @@
+--source include/have_binlog_format_mixed.inc
+--source include/have_innodb_binlog.inc
+
+--source include/reset_master.inc
+
+CREATE TABLE t1(a INT PRIMARY KEY, b INT) ENGINE=InnoDB;
+
+--echo *** Test basic XA COMMIT from same or different client connection.
+
+--let $binlog_file= query_get_value(SHOW MASTER STATUS, File, 1)
+--let $binlog_start= query_get_value(SHOW MASTER STATUS, Position, 1)
+
+--connect con1,localhost,root,,
+--let $con1_id= `SELECT connection_id()`
+XA START 'a';
+INSERT INTO t1 VALUES (1, 0);
+INSERT INTO t1 VALUES (2, 0);
+INSERT INTO t1 VALUES (3, 0);
+XA END 'a';
+XA PREPARE 'a';
+
+--connection default
+--disconnect con1
+# Must wait; XA transaction is not available until the creating session has disconnected.
+--let $wait_condition= SELECT NOT EXISTS (SELECT ID FROM INFORMATION_SCHEMA.PROCESSLIST WHERE ID=$con1_id)
+--source include/wait_condition.inc
+
+# Test that we will not allow RESET MASTER while there is an active XA
+# transaction whose commit will eventually reference oob data in the current
+# binlog.
+--error ER_BINLOG_IN_USE_TRX
+RESET MASTER;
+
+--connect con2,localhost,root,,
+XA COMMIT 'a';
+
+# Also test rollback from separate client session.
+--connect con1_roll,localhost,root,,
+--let $con1_roll_id= `SELECT connection_id()`
+XA START 'a_roll';
+INSERT INTO t1 VALUES (1001, 0);
+INSERT INTO t1 VALUES (1002, 0);
+INSERT INTO t1 VALUES (1003, 0);
+XA END 'a_roll';
+XA PREPARE 'a_roll';
+
+--connection default
+--disconnect con1_roll
+--let $wait_condition= SELECT NOT EXISTS (SELECT ID FROM INFORMATION_SCHEMA.PROCESSLIST WHERE ID=$con1_roll_id)
+--source include/wait_condition.inc
+
+--connection con2
+XA ROLLBACK 'a_roll';
+
+--connection default
+
+XA START 'b';
+UPDATE t1 SET b=2 WHERE a=1;
+XA END 'b';
+XA PREPARE 'b';
+XA COMMIT 'b';
+
+XA START 'b_roll';
+UPDATE t1 SET b=3 WHERE a=1;
+XA END 'b_roll';
+XA PREPARE 'b_roll';
+XA ROLLBACK 'b_roll';
+
+XA START 'c';
+UPDATE t1 SET b=3 WHERE a=3;
+XA END 'c';
+XA COMMIT 'c' ONE PHASE;
+
+SELECT * FROM t1 ORDER BY a;
+
+# Some queries that create empty XA transactions.
+CREATE TABLE t2 (a INT) ENGINE=MyISAM;
+CREATE TABLE t3 (id INT PRIMARY KEY) ENGINE=InnoDB;
+INSERT INTO t2 VALUES (1),(2);
+INSERT INTO t3 VALUES (1),(2);
+
+XA START '1';
+REPLACE INTO t2 SELECT * FROM t2;
+REPLACE INTO t3 SELECT * FROM t3;
+XA END '1';
+XA PREPARE '1';
+XA RECOVER FORMAT='SQL';
+
+# Cleanup
+XA ROLLBACK '1';
+DROP TABLE t2, t3;
+
+CREATE TABLE t2 (a INT PRIMARY KEY) ENGINE=InnoDB;
+INSERT INTO t2 VALUES (1);
+CREATE SEQUENCE s ENGINE=InnoDB;
+
+XA START '2';
+--disable_ps2_protocol
+SELECT NEXT VALUE FOR s;
+--enable_ps2_protocol
+REPLACE INTO t2 SELECT * FROM t2;
+XA END '2';
+XA PREPARE '2';
+XA RECOVER FORMAT='SQL';
+
+# Cleanup
+XA ROLLBACK '2';
+DROP SEQUENCE s;
+DROP TABLE t2;
+
+
+--let $binlog_limit= 100
+--source include/show_binlog_events.inc
+
+
+--echo *** Test that we will not purge a file that is needed by an active XA transaction.
+--connection default
+# Add some longer data to force spill to binlog.
+ALTER TABLE t1 ADD COLUMN c LONGBLOB;
+UPDATE t1 SET b=10 WHERE a=1;
+
+--connection con2
+SET SESSION binlog_format=ROW;
+XA START 'd';
+INSERT INTO t1 VALUES (10, 2, REPEAT('#', 40000));
+
+--connect con3,localhost,root,,
+SET SESSION binlog_format=ROW;
+XA START 'e';
+INSERT INTO t1 VALUES (110, 2, REPEAT('#', 40000));
+
+--connection default
+UPDATE t1 SET b=11 WHERE a=1;
+FLUSH BINARY LOGS;
+UPDATE t1 SET b=12 WHERE a=1;
+--let $binlog_file= query_get_value(SHOW MASTER STATUS, File, 1)
+
+--connection con2
+INSERT INTO t1 VALUES (11, 2, REPEAT('*', 40000));
+
+--connection con3
+INSERT INTO t1 VALUES (111, 2, REPEAT('*', 40000));
+
+--connection default
+UPDATE t1 SET b=13 WHERE a=1;
+FLUSH BINARY LOGS;
+UPDATE t1 SET b=14 WHERE a=1;
+UPDATE t1 SET b=15 WHERE a=1;
+FLUSH BINARY LOGS;
+UPDATE t1 SET b=16 WHERE a=1;
+FLUSH BINARY LOGS;
+
+--connection default
+SET @old_min_slaves= @@GLOBAL.slave_connections_needed_for_purge;
+SET GLOBAL slave_connections_needed_for_purge= 0;
+--error ER_LOG_IN_USE
+eval PURGE BINARY LOGS TO '$binlog_file';
+
+--connection con2
+XA END 'd';
+XA PREPARE 'd';
+
+--connection con3
+XA END 'e';
+XA PREPARE 'e';
+
+--connection default
+--error ER_LOG_IN_USE
+eval PURGE BINARY LOGS TO '$binlog_file';
+
+--connection con2
+XA COMMIT 'd';
+
+--connection con3
+XA ROLLBACK 'e';
+
+--connection default
+eval PURGE BINARY LOGS TO '$binlog_file';
+FLUSH BINARY LOGS;
+UPDATE t1 SET b=17 WHERE a=1;
+FLUSH BINARY LOGS;
+UPDATE t1 SET b=18 WHERE a=1;
+eval PURGE BINARY LOGS TO '$binlog_file';
+
+SET GLOBAL slave_connections_needed_for_purge= @old_min_slaves;
+
+
+--echo *** Test disconnecting in XA transaction before XA PREPARE.
+--connection con2
+SET SESSION binlog_format=MIXED;
+XA BEGIN 'f';
+CREATE TEMPORARY TABLE tmp ENGINE=InnoDB AS SELECT a FROM t1;
+--disconnect con2
+
+--connection con3
+SET SESSION binlog_format=MIXED;
+XA BEGIN 'g';
+UPDATE t1 SET b=19 WHERE a=1;
+XA END 'g';
+--disconnect con3
+
+--connection default
+BEGIN;
+SELECT b FROM t1 WHERE a=1 FOR UPDATE;
+ROLLBACK;
+
+
+--echo *** Test duplicate SAVEPOINT with XA
+# This is the original testcase found from RQG runs. See savepoint.test for
+# more savepoint-related tests.
+XA BEGIN 'x';
+DELETE FROM t1;
+SAVEPOINT A;
+SAVEPOINT B;
+SAVEPOINT A;
+ROLLBACK TO SAVEPOINT B;
+XA END 'x';
+XA ROLLBACK 'x';
+
+
+--echo *** Test server shutdown after XA PREPARE.
+# There was a hang during server shutdown when the last record in the active
+# binlog file is an XA prepare record; it was not being marked as durable.
+XA BEGIN 'y';
+INSERT INTO t1 VALUES (201, 0, '');
+XA END 'y';
+XA PREPARE 'y';
+--source include/restart_mysqld.inc
+
+XA ROLLBACK 'y';
+
+
+
+--echo *** Test binlogging of empty XA transaction.
+SET binlog_format=row;
+DELETE FROM t1 WHERE a=4242;
+INSERT INTO t1 VALUES (4242, 0, 'z1');
+XA BEGIN 'z';
+SELECT a, b, LENGTH(c) FROM t1 ORDER BY a LIMIT 3;
+INSERT IGNORE INTO t1 VALUES (4242, 0, 'z2');
+SELECT a, b, LENGTH(c) FROM t1 WHERE a>3 ORDER BY a LIMIT 1;
+XA END 'z';
+XA PREPARE 'z';
+XA COMMIT 'z';
+
+XA BEGIN 'z';
+UPDATE t1 SET b=10 WHERE a=4242;
+XA END 'z';
+XA PREPARE 'z';
+XA COMMIT 'z';
+
+SELECT a, b, LENGTH(c) FROM t1 WHERE a=4242;
+
+--echo *** Test XA PREPARE when the transaction is empty because statements rolled back on error
+
+--delimiter //
+CREATE TRIGGER u AFTER INSERT ON t1 FOR EACH ROW BEGIN RELEASE SAVEPOINT A ; END //
+--delimiter ;
+
+XA BEGIN 'xid44';
+--error ER_SP_DOES_NOT_EXIST
+INSERT IGNORE INTO t1(a) VALUES (10000);
+XA END 'xid44';
+XA PREPARE 'xid44';
+XA COMMIT 'xid44';
+
+DROP TRIGGER u;
+XA BEGIN 'xid44';
+SAVEPOINT A;
+ROLLBACK TO A;
+ROLLBACK TO A;
+INSERT INTO t1(a) VALUES (10001);
+XA END 'xid44';
+XA PREPARE 'xid44';
+
+# Test server restart; this would hang due to a bug where the
+# prepare record written to the binlog was not entered into the
+# pending LSN fifo, and the tablespace close at shutdown then
+# waited forever.
+--source include/restart_mysqld.inc
+
+XA COMMIT 'xid44';
+SELECT * FROM t1 WHERE a >= 10000 ORDER BY a;
+
+
+# Clean up.
+DROP TABLE t1;
diff --git a/mysql-test/suite/binlog_in_engine/xa_innodb_crash.result b/mysql-test/suite/binlog_in_engine/xa_innodb_crash.result
new file mode 100644
index 0000000000000..4a9d11561d9bb
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/xa_innodb_crash.result
@@ -0,0 +1,167 @@
+include/reset_master.inc
+*** Test XA PREPARE, COMMIT, ROLLBACK with crashes at various points.
+CREATE TABLE t1(a INT NOT NULL, b INT NOT NULL, c LONGBLOB, PRIMARY KEY(a, b)) ENGINE=InnoDB;
+INSERT INTO t1 VALUES (0, 1, '');
+INSERT INTO t1 VALUES (0, 2, '');
+INSERT INTO t1 VALUES (0, 3, '');
+*** x1: prepared in binlog but not in engine/tables.
+connect con1,localhost,root,,;
+XA BEGIN 'x1';
+INSERT INTO t1 VALUES (1, 1, '');
+INSERT INTO t1 VALUES (1, 2, REPEAT('', 10000));
+*** x2: prepared in binlog and engine.
+connect con2,localhost,root,,;
+XA BEGIN 'x2';
+INSERT INTO t1 VALUES (2, 1, REPEAT('', 10000));
+*** x3: committed in binlog, not in engine.
+connect con3,localhost,root,,;
+XA BEGIN 'x3';
+INSERT INTO t1 VALUES (3, 1, '');
+connection default;
+FLUSH BINARY LOGS;
+*** x4: committed in binlog and in engine.
+connect con4,localhost,root,,;
+XA BEGIN 'x4';
+INSERT INTO t1 VALUES (4, 1, REPEAT('', 10000));
+*** x5: rolled back in binlog, not in engine.
+connect con5,localhost,root,,;
+XA BEGIN 'x5';
+INSERT INTO t1 VALUES (5, 1, '');
+*** x6: rolled back in binlog and in engine.
+connect con6,localhost,root,,;
+XA BEGIN 'x6';
+INSERT INTO t1 VALUES (6, 1, REPEAT('', 10000));
+connection default;
+FLUSH BINARY LOGS;
+connection con3;
+INSERT INTO t1 VALUES (3, 2, '');
+XA END 'x3';
+XA PREPARE 'x3';
+SET debug_sync= 'ibb_after_commit_redo_log SIGNAL con3_rdy WAIT_FOR crash';
+XA COMMIT 'x3';
+connection default;
+SET debug_sync= 'now WAIT_FOR con3_rdy';
+connection con1;
+INSERT INTO t1 VALUES (1, 3, '');
+XA END 'x1';
+SET debug_sync= 'ibb_after_prepare_redo_log SIGNAL con1_rdy WAIT_FOR crash';
+XA PREPARE 'x1';
+connection default;
+SET debug_sync= 'now WAIT_FOR con1_rdy';
+connection con6;
+INSERT INTO t1 VALUES (6, 2, '');
+XA END 'x6';
+XA PREPARE 'x6';
+SET debug_sync= 'rollback_handlerton_after SIGNAL con6_rdy WAIT_FOR crash';
+XA ROLLBACK 'x6';
+connection default;
+SET debug_sync= 'now WAIT_FOR con6_rdy';
+connection con5;
+INSERT INTO t1 VALUES (5, 2, '');
+XA END 'x5';
+XA PREPARE 'x5';
+SET debug_sync= 'ibb_after_rollback_redo_log SIGNAL con5_rdy WAIT_FOR crash';
+XA ROLLBACK 'x5';
+connection default;
+SET debug_sync= 'now WAIT_FOR con5_rdy';
+FLUSH BINARY LOGS;
+connection con2;
+INSERT INTO t1 VALUES (2, 2, '');
+INSERT INTO t1 VALUES (2, 3, REPEAT('', 10001));
+XA END 'x2';
+SET debug_sync= 'at_unlog_xa_prepare SIGNAL con2_rdy WAIT_FOR crash';
+XA PREPARE 'x2';
+connection default;
+SET debug_sync= 'now WAIT_FOR con2_rdy';
+connection con4;
+INSERT INTO t1 VALUES (4, 2, '');
+XA END 'x4';
+XA PREPARE 'x4';
+SET debug_sync= 'commit_handlerton_after SIGNAL con4_rdy WAIT_FOR crash';
+XA COMMIT 'x4';
+connection default;
+SET debug_sync= 'now WAIT_FOR con4_rdy';
+# restart
+disconnect con1;
+disconnect con2;
+disconnect con3;
+disconnect con4;
+disconnect con5;
+disconnect con6;
+connection default;
+XA RECOVER;
+formatID gtrid_length bqual_length data
+1 2 0 x2
+XA COMMIT 'x2';
+include/show_binlog_events.inc
+Log_name Pos Event_type Server_id End_log_pos Info
+binlog-000000.ibb # Gtid # # GTID #-#-#
+binlog-000000.ibb # Query # # use `test`; CREATE TABLE t1(a INT NOT NULL, b INT NOT NULL, c LONGBLOB, PRIMARY KEY(a, b)) ENGINE=InnoDB
+binlog-000000.ibb # Gtid # # BEGIN GTID #-#-#
+binlog-000000.ibb # Annotate_rows # # INSERT INTO t1 VALUES (0, 1, '')
+binlog-000000.ibb # Table_map # # table_id: # (test.t1)
+binlog-000000.ibb # Write_rows_v1 # # table_id: # flags: STMT_END_F
+binlog-000000.ibb # Xid # # COMMIT /* XID */
+binlog-000000.ibb # Gtid # # BEGIN GTID #-#-#
+binlog-000000.ibb # Annotate_rows # # INSERT INTO t1 VALUES (0, 2, '')
+binlog-000000.ibb # Table_map # # table_id: # (test.t1)
+binlog-000000.ibb # Write_rows_v1 # # table_id: # flags: STMT_END_F
+binlog-000000.ibb # Xid # # COMMIT /* XID */
+binlog-000000.ibb # Gtid # # BEGIN GTID #-#-#
+binlog-000000.ibb # Annotate_rows # # INSERT INTO t1 VALUES (0, 3, '')
+binlog-000000.ibb # Table_map # # table_id: # (test.t1)
+binlog-000000.ibb # Write_rows_v1 # # table_id: # flags: STMT_END_F
+binlog-000000.ibb # Xid # # COMMIT /* XID */
+include/show_binlog_events.inc
+include/show_binlog_events.inc
+Log_name Pos Event_type Server_id End_log_pos Info
+binlog-000002.ibb # Gtid # # BEGIN GTID #-#-#
+binlog-000002.ibb # Annotate_rows # # INSERT INTO t1 VALUES (3, 1, '')
+binlog-000002.ibb # Table_map # # table_id: # (test.t1)
+binlog-000002.ibb # Write_rows_v1 # # table_id: # flags: STMT_END_F
+binlog-000002.ibb # Annotate_rows # # INSERT INTO t1 VALUES (3, 2, '')
+binlog-000002.ibb # Table_map # # table_id: # (test.t1)
+binlog-000002.ibb # Write_rows_v1 # # table_id: # flags: STMT_END_F
+binlog-000002.ibb # Query # # COMMIT
+FLUSH BINARY LOGS;
+include/show_binlog_events.inc
+Log_name Pos Event_type Server_id End_log_pos Info
+binlog-000003.ibb # Gtid # # BEGIN GTID #-#-#
+binlog-000003.ibb # Annotate_rows # # INSERT INTO t1 VALUES (4, 1, REPEAT('', 10000))
+binlog-000003.ibb # Table_map # # table_id: # (test.t1)
+binlog-000003.ibb # Write_rows_v1 # # table_id: # flags: STMT_END_F
+binlog-000003.ibb # Annotate_rows # # INSERT INTO t1 VALUES (4, 2, '')
+binlog-000003.ibb # Table_map # # table_id: # (test.t1)
+binlog-000003.ibb # Write_rows_v1 # # table_id: # flags: STMT_END_F
+binlog-000003.ibb # Query # # COMMIT
+binlog-000003.ibb # Format_desc # # SERVER_VERSION, BINLOG_VERSION
+binlog-000003.ibb # Gtid # # BEGIN GTID #-#-#
+binlog-000003.ibb # Annotate_rows # # INSERT INTO t1 VALUES (2, 1, REPEAT('', 10000))
+binlog-000003.ibb # Table_map # # table_id: # (test.t1)
+binlog-000003.ibb # Write_rows_v1 # # table_id: # flags: STMT_END_F
+binlog-000003.ibb # Annotate_rows # # INSERT INTO t1 VALUES (2, 2, '')
+binlog-000003.ibb # Table_map # # table_id: # (test.t1)
+binlog-000003.ibb # Write_rows_v1 # # table_id: # flags: STMT_END_F
+binlog-000003.ibb # Annotate_rows # # INSERT INTO t1 VALUES (2, 3, REPEAT('', 10001))
+binlog-000003.ibb # Table_map # # table_id: # (test.t1)
+binlog-000003.ibb # Write_rows_v1 # # table_id: # flags: STMT_END_F
+binlog-000003.ibb # Query # # COMMIT
+SELECT a, b, LENGTH(c) FROM t1 ORDER BY a, b;
+a b LENGTH(c)
+0 1 0
+0 2 0
+0 3 0
+2 1 40000
+2 2 0
+2 3 40004
+3 1 0
+3 2 0
+4 1 40000
+4 2 0
+*** Test that we can purge, no leaked XA/OOB refs.
+SET @old_min_slaves= @@GLOBAL.slave_connections_needed_for_purge;
+SET GLOBAL slave_connections_needed_for_purge= 0;
+FLUSH BINARY LOGS;
+PURGE BINARY LOGS TO 'binlog-000004.ibb';
+SET GLOBAL slave_connections_needed_for_purge= @old_min_slaves;
+DROP TABLE t1;
diff --git a/mysql-test/suite/binlog_in_engine/xa_innodb_crash.test b/mysql-test/suite/binlog_in_engine/xa_innodb_crash.test
new file mode 100644
index 0000000000000..f8629107724f0
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/xa_innodb_crash.test
@@ -0,0 +1,6 @@
+--source include/have_debug_sync.inc
+--source include/have_binlog_format_row.inc
+--source include/have_innodb_binlog.inc
+
+--let $engine= InnoDB
+--source include/xa_crash.inc
diff --git a/mysql-test/suite/binlog_in_engine/xa_restart.result b/mysql-test/suite/binlog_in_engine/xa_restart.result
new file mode 100644
index 0000000000000..baec86c2ad34e
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/xa_restart.result
@@ -0,0 +1,52 @@
+include/reset_master.inc
+*** Test that a prepare record at the end of binlog does not cause server shutdown to hang
+CALL mtr.add_suppression('Warning.*Found 1 prepared XA transactions');
+CREATE TABLE t1 (a INT PRIMARY KEY, b LONGBLOB) ENGINE=InnoDB;
+XA START 'a';
+INSERT INTO t1 VALUES (1, REPEAT('<', 50000));
+INSERT INTO t1 VALUES (2, REPEAT('|', 50000));
+INSERT INTO t1 VALUES (3, REPEAT('>', 50000));
+XA END 'a';
+XA PREPARE 'a';
+# restart
+XA ROLLBACK 'a';
+*** Test server restart when some binlog files are missing.
+include/reset_master.inc
+INSERT INTO t1 VALUES (10, REPEAT('a', 100000));
+INSERT INTO t1 VALUES (11, REPEAT('a', 100000));
+INSERT INTO t1 VALUES (12, REPEAT('a', 100000));
+connect con1,localhost,root,,;
+XA START 'b';
+INSERT INTO t1 VALUES (20, REPEAT('b', 10000));
+XA END 'b';
+XA PREPARE 'b';
+connection default;
+FLUSH BINARY LOGS;
+connect con2,localhost,root,,;
+BEGIN;
+INSERT INTO t1 VALUES (21, REPEAT('d', 100000));
+connect con3,localhost,root,,;
+XA START 'c';
+INSERT INTO t1 VALUES (22, REPEAT('c', 10000));
+XA END 'c';
+XA PREPARE 'c';
+connection default;
+FLUSH BINARY LOGS;
+INSERT INTO t1 VALUES (30, '');
+*** Stop the server, then restart it after deleting some still needed binlog files.
+# restart
+disconnect con1;
+disconnect con2;
+disconnect con3;
+*** Should be empty, binlog files needed for XA were deleted.
+XA RECOVER;
+formatID gtrid_length bqual_length data
+SELECT a, LENGTH(b) FROM t1 ORDER BY a;
+a LENGTH(b)
+10 100000
+11 100000
+12 100000
+30 0
+DROP TABLE t1;
+CALL mtr.add_suppression('ERROR.*File .* not found');
+CALL mtr.add_suppression('ERROR.*InnoDB: Error reading binlog while recovering XIDs of possibly prepared transactions. Recovery will be incomplete');
diff --git a/mysql-test/suite/binlog_in_engine/xa_restart.test b/mysql-test/suite/binlog_in_engine/xa_restart.test
new file mode 100644
index 0000000000000..a1f2f56828a08
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/xa_restart.test
@@ -0,0 +1,78 @@
+--source include/have_binlog_format_row.inc
+--source include/have_innodb_binlog.inc
+
+--let $datadir= `SELECT @@datadir`
+--source include/reset_master.inc
+
+--echo *** Test that a prepare record at the end of binlog does not cause server shutdown to hang
+CALL mtr.add_suppression('Warning.*Found 1 prepared XA transactions');
+CREATE TABLE t1 (a INT PRIMARY KEY, b LONGBLOB) ENGINE=InnoDB;
+
+XA START 'a';
+INSERT INTO t1 VALUES (1, REPEAT('<', 50000));
+INSERT INTO t1 VALUES (2, REPEAT('|', 50000));
+INSERT INTO t1 VALUES (3, REPEAT('>', 50000));
+XA END 'a';
+XA PREPARE 'a';
+
+# The bug was that the prepare record written at the current end of
+# the binlog file was not entered into the pending LSN fifo, so it never
+# got marked as having been made durable. Then when the server shuts down
+# and closes the binlog tablespace file, it waits indefinitely for the
+# file to be marked durable.
+--source include/restart_mysqld.inc
+
+XA ROLLBACK 'a';
+
+--echo *** Test server restart when some binlog files are missing.
+
+--source include/reset_master.inc
+
+INSERT INTO t1 VALUES (10, REPEAT('a', 100000));
+INSERT INTO t1 VALUES (11, REPEAT('a', 100000));
+INSERT INTO t1 VALUES (12, REPEAT('a', 100000));
+
+--connect con1,localhost,root,,
+XA START 'b';
+INSERT INTO t1 VALUES (20, REPEAT('b', 10000));
+XA END 'b';
+XA PREPARE 'b';
+
+--connection default
+FLUSH BINARY LOGS;
+
+--connect con2,localhost,root,,
+BEGIN;
+INSERT INTO t1 VALUES (21, REPEAT('d', 100000));
+
+--connect con3,localhost,root,,
+XA START 'c';
+INSERT INTO t1 VALUES (22, REPEAT('c', 10000));
+XA END 'c';
+XA PREPARE 'c';
+
+--connection default
+FLUSH BINARY LOGS;
+INSERT INTO t1 VALUES (30, '');
+
+--echo *** Stop the server, then restart it after deleting some still needed binlog files.
+--source include/shutdown_mysqld.inc
+
+# The server should still start, but with errors in the log, suppressed below.
+--remove_file $datadir/binlog-000000.ibb
+--remove_file $datadir/binlog-000001.ibb
+
+--source include/start_mysqld.inc
+
+--disconnect con1
+--disconnect con2
+--disconnect con3
+
+--echo *** Should be empty, binlog files needed for XA were deleted.
+XA RECOVER;
+
+SELECT a, LENGTH(b) FROM t1 ORDER BY a;
+
+DROP TABLE t1;
+CALL mtr.add_suppression('ERROR.*File .* not found');
+CALL mtr.add_suppression('ERROR.*InnoDB: Error reading binlog while recovering XIDs of possibly prepared transactions. Recovery will be incomplete');
diff --git a/mysql-test/suite/binlog_in_engine/xa_restart2.result b/mysql-test/suite/binlog_in_engine/xa_restart2.result
new file mode 100644
index 0000000000000..f7e2b5b7c7f84
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/xa_restart2.result
@@ -0,0 +1,169 @@
+include/master-slave.inc
+[connection master]
+*** Test XA PREPARE preserved over server restart.
+CREATE TABLE t1(a INT NOT NULL, b INT NOT NULL, c LONGBLOB, PRIMARY KEY(a, b)) ENGINE=InnoDB;
+INSERT INTO t1 VALUES (0, 1, '');
+INSERT INTO t1 VALUES (0, 2, '');
+INSERT INTO t1 VALUES (0, 3, '');
+connect con1,localhost,root,,;
+XA BEGIN 'x1';
+INSERT INTO t1 VALUES (1, 1, '');
+connect con2,localhost,root,,;
+XA BEGIN 'x2';
+INSERT INTO t1 VALUES (2, 1, REPEAT('2', 100000));
+connect con3,localhost,root,,;
+XA BEGIN 'x3';
+INSERT INTO t1 VALUES (3, 1, '');
+connection con1;
+INSERT INTO t1 VALUES (1, 2, '');
+connection con2;
+INSERT INTO t1 VALUES (2, 2, REPEAT('2', 40000));
+connection con3;
+INSERT INTO t1 VALUES (3, 2, REPEAT('3', 50000));
+connection con2;
+INSERT INTO t1 VALUES (2, 3, REPEAT('2', 40000));
+connection con3;
+INSERT INTO t1 VALUES (3, 3, REPEAT('3', 100000));
+INSERT INTO t1 VALUES (3, 4, REPEAT('3', 100000));
+INSERT INTO t1 VALUES (3, 5, REPEAT('3', 100000));
+INSERT INTO t1 VALUES (3, 6, REPEAT('3', 100000));
+connection con2;
+INSERT INTO t1 VALUES (2, 4, REPEAT('2', 40000));
+connection con1;
+INSERT INTO t1 VALUES (1, 3, '');
+XA END 'x1';
+XA PREPARE 'x1';
+connection con2;
+XA END 'x2';
+XA PREPARE 'x2';
+connection con3;
+INSERT INTO t1 VALUES (3, 7, '');
+XA END 'x3';
+XA PREPARE 'x3';
+connection slave;
+include/stop_slave.inc
+connection master;
+include/rpl_restart_server.inc [server_number=1]
+disconnect con1;
+disconnect con2;
+disconnect con3;
+connection slave;
+include/start_slave.inc
+connection master;
+XA RECOVER;
+formatID gtrid_length bqual_length data
+1 2 0 x1
+1 2 0 x2
+1 2 0 x3
+XA COMMIT 'x1';
+XA ROLLBACK 'x2';
+XA COMMIT 'x3';
+SELECT a, b, LENGTH(c) FROM t1 ORDER BY a,b;
+a b LENGTH(c)
+0 1 0
+0 2 0
+0 3 0
+1 1 0
+1 2 0
+1 3 0
+3 1 0
+3 2 50000
+3 3 100000
+3 4 100000
+3 5 100000
+3 6 100000
+3 7 0
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+SELECT a, b, LENGTH(c) FROM t1 ORDER BY a,b;
+a b LENGTH(c)
+0 1 0
+0 2 0
+0 3 0
+1 1 0
+1 2 0
+1 3 0
+3 1 0
+3 2 50000
+3 3 100000
+3 4 100000
+3 5 100000
+3 6 100000
+3 7 0
+*** Test purgability of binlogs containing XA oob after restart.
+connection slave;
+include/stop_slave.inc
+connection master;
+include/reset_master.inc
+connect con1,localhost,root,,;
+XA START 'x4';
+INSERT INTO t1 VALUES (4, 1, REPEAT('4', 40000));
+connection master;
+FLUSH BINARY LOGS;
+connect con2,localhost,root,,;
+XA START 'x5';
+INSERT INTO t1 VALUES (5, 1, REPEAT('5', 40000));
+XA END 'x5';
+XA PREPARE 'x5';
+connection master;
+FLUSH BINARY LOGS;
+connect con3,localhost,root,,;
+XA START 'x6';
+INSERT INTO t1 VALUES (6, 1, 'x6');
+XA END 'x6';
+XA PREPARE 'x6';
+connection con1;
+XA END 'x4';
+XA PREPARE 'x4';
+connection master;
+include/rpl_restart_server.inc [server_number=1]
+disconnect con1;
+disconnect con2;
+disconnect con3;
+connection slave;
+SET GLOBAL gtid_slave_pos= '';
+include/start_slave.inc
+connection master;
+XA RECOVER;
+formatID gtrid_length bqual_length data
+1 2 0 x4
+1 2 0 x5
+1 2 0 x6
+SET @old_min_slaves= @@GLOBAL.slave_connections_needed_for_purge;
+SET GLOBAL slave_connections_needed_for_purge= 1;
+FLUSH BINARY LOGS;
+PURGE BINARY LOGS TO 'binlog-000001.ibb';
+ERROR HY000: A purgeable log is in use, will not purge
+XA COMMIT 'x4';
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection master;
+PURGE BINARY LOGS TO 'binlog-000001.ibb';
+PURGE BINARY LOGS TO 'binlog-000002.ibb';
+ERROR HY000: A purgeable log is in use, will not purge
+XA ROLLBACK 'x5';
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection master;
+PURGE BINARY LOGS TO 'binlog-000002.ibb';
+FLUSH BINARY LOGS;
+PURGE BINARY LOGS TO 'binlog-000003.ibb';
+ERROR HY000: A purgeable log is in use, will not purge
+XA COMMIT 'x6';
+FLUSH BINARY LOGS;
+INSERT INTO t1 VALUES (7, 1, '');
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection master;
+PURGE BINARY LOGS TO 'binlog-000003.ibb';
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection master;
+SET GLOBAL slave_connections_needed_for_purge= @old_min_slaves;
+DROP TABLE t1;
+include/rpl_end.inc
diff --git a/mysql-test/suite/binlog_in_engine/xa_restart2.test b/mysql-test/suite/binlog_in_engine/xa_restart2.test
new file mode 100644
index 0000000000000..27e68d5c66fad
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/xa_restart2.test
@@ -0,0 +1,187 @@
+--source include/have_binlog_format_row.inc
+--source include/master-slave.inc
+--source include/have_innodb_binlog.inc
+
+--echo *** Test XA PREPARE preserved over server restart.
+CREATE TABLE t1(a INT NOT NULL, b INT NOT NULL, c LONGBLOB, PRIMARY KEY(a, b)) ENGINE=InnoDB;
+
+INSERT INTO t1 VALUES (0, 1, '');
+INSERT INTO t1 VALUES (0, 2, '');
+INSERT INTO t1 VALUES (0, 3, '');
+
+--connect con1,localhost,root,,
+XA BEGIN 'x1';
+INSERT INTO t1 VALUES (1, 1, '');
+
+--connect con2,localhost,root,,
+XA BEGIN 'x2';
+INSERT INTO t1 VALUES (2, 1, REPEAT('2', 100000));
+
+--connect con3,localhost,root,,
+XA BEGIN 'x3';
+INSERT INTO t1 VALUES (3, 1, '');
+
+--connection con1
+INSERT INTO t1 VALUES (1, 2, '');
+
+--connection con2
+INSERT INTO t1 VALUES (2, 2, REPEAT('2', 40000));
+
+--connection con3
+INSERT INTO t1 VALUES (3, 2, REPEAT('3', 50000));
+
+--connection con2
+INSERT INTO t1 VALUES (2, 3, REPEAT('2', 40000));
+
+--connection con3
+INSERT INTO t1 VALUES (3, 3, REPEAT('3', 100000));
+INSERT INTO t1 VALUES (3, 4, REPEAT('3', 100000));
+INSERT INTO t1 VALUES (3, 5, REPEAT('3', 100000));
+INSERT INTO t1 VALUES (3, 6, REPEAT('3', 100000));
+
+--connection con2
+INSERT INTO t1 VALUES (2, 4, REPEAT('2', 40000));
+
+--connection con1
+INSERT INTO t1 VALUES (1, 3, '');
+XA END 'x1';
+XA PREPARE 'x1';
+
+--connection con2
+XA END 'x2';
+XA PREPARE 'x2';
+
+--connection con3
+INSERT INTO t1 VALUES (3, 7, '');
+XA END 'x3';
+XA PREPARE 'x3';
+
+--connection slave
+--source include/stop_slave.inc
+
+--connection master
+--let $rpl_server_number= 1
+--source include/rpl_restart_server.inc
+
+--disconnect con1
+--disconnect con2
+--disconnect con3
+
+--connection slave
+--source include/start_slave.inc
+
+--connection master
+--sorted_result
+XA RECOVER;
+XA COMMIT 'x1';
+XA ROLLBACK 'x2';
+XA COMMIT 'x3';
+SELECT a, b, LENGTH(c) FROM t1 ORDER BY a,b;
+--source include/save_master_gtid.inc
+
+--connection slave
+--source include/sync_with_master_gtid.inc
+SELECT a, b, LENGTH(c) FROM t1 ORDER BY a,b;
+
+
+--echo *** Test purgability of binlogs containing XA oob after restart.
+--connection slave
+--source include/stop_slave.inc
+
+--connection master
+--source include/reset_master.inc
+
+--connect con1,localhost,root,,
+# Trx x4 in binlog 0 and 2.
+XA START 'x4';
+INSERT INTO t1 VALUES (4, 1, REPEAT('4', 40000));
+
+--connection master
+FLUSH BINARY LOGS;
+
+--connect con2,localhost,root,,
+# Trx x5 in binlog 1.
+XA START 'x5';
+INSERT INTO t1 VALUES (5, 1, REPEAT('5', 40000));
+XA END 'x5';
+XA PREPARE 'x5';
+
+--connection master
+FLUSH BINARY LOGS;
+
+--connect con3,localhost,root,,
+# Trx x6 in binlog 3.
+XA START 'x6';
+INSERT INTO t1 VALUES (6, 1, 'x6');
+XA END 'x6';
+XA PREPARE 'x6';
+
+--connection con1
+XA END 'x4';
+XA PREPARE 'x4';
+
+--connection master
+--source include/rpl_restart_server.inc
+
+--disconnect con1
+--disconnect con2
+--disconnect con3
+
+--connection slave
+SET GLOBAL gtid_slave_pos= '';
+--source include/start_slave.inc
+
+--connection master
+--sorted_result
+XA RECOVER;
+SET @old_min_slaves= @@GLOBAL.slave_connections_needed_for_purge;
+SET GLOBAL slave_connections_needed_for_purge= 1;
+
+# Purge is a bit tricky.
+# A slave reading file N will prevent purge of N.
+# And then any M', 10000));
+*** x2: prepared in binlog and engine.
+connect con2,localhost,root,,;
+XA BEGIN 'x2';
+INSERT INTO t1 VALUES (2, 1, REPEAT('', 10000));
+*** x3: committed in binlog, not in engine.
+connect con3,localhost,root,,;
+XA BEGIN 'x3';
+INSERT INTO t1 VALUES (3, 1, '');
+connection default;
+FLUSH BINARY LOGS;
+*** x4: committed in binlog and in engine.
+connect con4,localhost,root,,;
+XA BEGIN 'x4';
+INSERT INTO t1 VALUES (4, 1, REPEAT('', 10000));
+*** x5: rolled back in binlog, not in engine.
+connect con5,localhost,root,,;
+XA BEGIN 'x5';
+INSERT INTO t1 VALUES (5, 1, '');
+*** x6: rolled back in binlog and in engine.
+connect con6,localhost,root,,;
+XA BEGIN 'x6';
+INSERT INTO t1 VALUES (6, 1, REPEAT('', 10000));
+connection default;
+FLUSH BINARY LOGS;
+connection con3;
+INSERT INTO t1 VALUES (3, 2, '');
+XA END 'x3';
+XA PREPARE 'x3';
+SET debug_sync= 'ibb_after_commit_redo_log SIGNAL con3_rdy WAIT_FOR crash';
+XA COMMIT 'x3';
+connection default;
+SET debug_sync= 'now WAIT_FOR con3_rdy';
+connection con1;
+INSERT INTO t1 VALUES (1, 3, '');
+XA END 'x1';
+SET debug_sync= 'ibb_after_prepare_redo_log SIGNAL con1_rdy WAIT_FOR crash';
+XA PREPARE 'x1';
+connection default;
+SET debug_sync= 'now WAIT_FOR con1_rdy';
+connection con6;
+INSERT INTO t1 VALUES (6, 2, '');
+XA END 'x6';
+XA PREPARE 'x6';
+SET debug_sync= 'rollback_handlerton_after SIGNAL con6_rdy WAIT_FOR crash';
+XA ROLLBACK 'x6';
+connection default;
+SET debug_sync= 'now WAIT_FOR con6_rdy';
+connection con5;
+INSERT INTO t1 VALUES (5, 2, '');
+XA END 'x5';
+XA PREPARE 'x5';
+SET debug_sync= 'ibb_after_rollback_redo_log SIGNAL con5_rdy WAIT_FOR crash';
+XA ROLLBACK 'x5';
+connection default;
+SET debug_sync= 'now WAIT_FOR con5_rdy';
+FLUSH BINARY LOGS;
+connection con2;
+INSERT INTO t1 VALUES (2, 2, '');
+INSERT INTO t1 VALUES (2, 3, REPEAT('', 10001));
+XA END 'x2';
+SET debug_sync= 'at_unlog_xa_prepare SIGNAL con2_rdy WAIT_FOR crash';
+XA PREPARE 'x2';
+connection default;
+SET debug_sync= 'now WAIT_FOR con2_rdy';
+connection con4;
+INSERT INTO t1 VALUES (4, 2, '');
+XA END 'x4';
+XA PREPARE 'x4';
+SET debug_sync= 'commit_handlerton_after SIGNAL con4_rdy WAIT_FOR crash';
+XA COMMIT 'x4';
+connection default;
+SET debug_sync= 'now WAIT_FOR con4_rdy';
+# restart
+disconnect con1;
+disconnect con2;
+disconnect con3;
+disconnect con4;
+disconnect con5;
+disconnect con6;
+connection default;
+XA RECOVER;
+formatID gtrid_length bqual_length data
+1 2 0 x2
+XA COMMIT 'x2';
+include/show_binlog_events.inc
+Log_name Pos Event_type Server_id End_log_pos Info
+binlog-000000.ibb # Gtid # # GTID #-#-#
+binlog-000000.ibb # Query # # use `test`; CREATE TABLE t1(a INT NOT NULL, b INT NOT NULL, c LONGBLOB, PRIMARY KEY(a, b)) ENGINE=RocksDB
+binlog-000000.ibb # Gtid # # BEGIN GTID #-#-#
+binlog-000000.ibb # Annotate_rows # # INSERT INTO t1 VALUES (0, 1, '')
+binlog-000000.ibb # Table_map # # table_id: # (test.t1)
+binlog-000000.ibb # Write_rows_v1 # # table_id: # flags: STMT_END_F
+binlog-000000.ibb # Xid # # COMMIT /* XID */
+binlog-000000.ibb # Gtid # # BEGIN GTID #-#-#
+binlog-000000.ibb # Annotate_rows # # INSERT INTO t1 VALUES (0, 2, '')
+binlog-000000.ibb # Table_map # # table_id: # (test.t1)
+binlog-000000.ibb # Write_rows_v1 # # table_id: # flags: STMT_END_F
+binlog-000000.ibb # Xid # # COMMIT /* XID */
+binlog-000000.ibb # Gtid # # BEGIN GTID #-#-#
+binlog-000000.ibb # Annotate_rows # # INSERT INTO t1 VALUES (0, 3, '')
+binlog-000000.ibb # Table_map # # table_id: # (test.t1)
+binlog-000000.ibb # Write_rows_v1 # # table_id: # flags: STMT_END_F
+binlog-000000.ibb # Xid # # COMMIT /* XID */
+include/show_binlog_events.inc
+include/show_binlog_events.inc
+Log_name Pos Event_type Server_id End_log_pos Info
+binlog-000002.ibb # Gtid # # BEGIN GTID #-#-#
+binlog-000002.ibb # Annotate_rows # # INSERT INTO t1 VALUES (3, 1, '')
+binlog-000002.ibb # Table_map # # table_id: # (test.t1)
+binlog-000002.ibb # Write_rows_v1 # # table_id: # flags: STMT_END_F
+binlog-000002.ibb # Annotate_rows # # INSERT INTO t1 VALUES (3, 2, '')
+binlog-000002.ibb # Table_map # # table_id: # (test.t1)
+binlog-000002.ibb # Write_rows_v1 # # table_id: # flags: STMT_END_F
+binlog-000002.ibb # Query # # COMMIT
+FLUSH BINARY LOGS;
+include/show_binlog_events.inc
+Log_name Pos Event_type Server_id End_log_pos Info
+binlog-000003.ibb # Gtid # # BEGIN GTID #-#-#
+binlog-000003.ibb # Annotate_rows # # INSERT INTO t1 VALUES (4, 1, REPEAT('', 10000))
+binlog-000003.ibb # Table_map # # table_id: # (test.t1)
+binlog-000003.ibb # Write_rows_v1 # # table_id: # flags: STMT_END_F
+binlog-000003.ibb # Annotate_rows # # INSERT INTO t1 VALUES (4, 2, '')
+binlog-000003.ibb # Table_map # # table_id: # (test.t1)
+binlog-000003.ibb # Write_rows_v1 # # table_id: # flags: STMT_END_F
+binlog-000003.ibb # Query # # COMMIT
+binlog-000003.ibb # Format_desc # # SERVER_VERSION, BINLOG_VERSION
+binlog-000003.ibb # Gtid # # BEGIN GTID #-#-#
+binlog-000003.ibb # Annotate_rows # # INSERT INTO t1 VALUES (2, 1, REPEAT('', 10000))
+binlog-000003.ibb # Table_map # # table_id: # (test.t1)
+binlog-000003.ibb # Write_rows_v1 # # table_id: # flags: STMT_END_F
+binlog-000003.ibb # Annotate_rows # # INSERT INTO t1 VALUES (2, 2, '')
+binlog-000003.ibb # Table_map # # table_id: # (test.t1)
+binlog-000003.ibb # Write_rows_v1 # # table_id: # flags: STMT_END_F
+binlog-000003.ibb # Annotate_rows # # INSERT INTO t1 VALUES (2, 3, REPEAT('', 10001))
+binlog-000003.ibb # Table_map # # table_id: # (test.t1)
+binlog-000003.ibb # Write_rows_v1 # # table_id: # flags: STMT_END_F
+binlog-000003.ibb # Query # # COMMIT
+SELECT a, b, LENGTH(c) FROM t1 ORDER BY a, b;
+a b LENGTH(c)
+0 1 0
+0 2 0
+0 3 0
+2 1 40000
+2 2 0
+2 3 40004
+3 1 0
+3 2 0
+4 1 40000
+4 2 0
+*** Test that we can purge, no leaked XA/OOB refs.
+SET @old_min_slaves= @@GLOBAL.slave_connections_needed_for_purge;
+SET GLOBAL slave_connections_needed_for_purge= 0;
+FLUSH BINARY LOGS;
+PURGE BINARY LOGS TO 'binlog-000004.ibb';
+SET GLOBAL slave_connections_needed_for_purge= @old_min_slaves;
+DROP TABLE t1;
diff --git a/mysql-test/suite/binlog_in_engine/xa_rocksdb_crash.test b/mysql-test/suite/binlog_in_engine/xa_rocksdb_crash.test
new file mode 100644
index 0000000000000..7170755f098f5
--- /dev/null
+++ b/mysql-test/suite/binlog_in_engine/xa_rocksdb_crash.test
@@ -0,0 +1,7 @@
+--source include/have_rocksdb.inc
+--source include/have_debug_sync.inc
+--source include/have_binlog_format_row.inc
+--source include/have_innodb_binlog.inc
+
+--let $engine= RocksDB
+--source include/xa_crash.inc
diff --git a/mysql-test/suite/engines/funcs/r/rpl_000015.result b/mysql-test/suite/engines/funcs/r/rpl_000015.result
index 56fc002ac534d..c948289fba520 100644
--- a/mysql-test/suite/engines/funcs/r/rpl_000015.result
+++ b/mysql-test/suite/engines/funcs/r/rpl_000015.result
@@ -1,11 +1,14 @@
include/master-slave.inc
[connection master]
+connection slave;
+include/stop_slave.inc
+connection master;
+include/kill_binlog_dump_threads.inc
reset master;
show master status;
File Position Binlog_Do_DB Binlog_Ignore_DB
master-bin.000001 329
connection slave;
-include/stop_slave.inc
reset slave;
Slave_IO_Running = 'No'
Slave_SQL_Running = 'No'
diff --git a/mysql-test/suite/engines/funcs/r/rpl_server_id2.result b/mysql-test/suite/engines/funcs/r/rpl_server_id2.result
index 3d8eeb02ca675..033643177fce8 100644
--- a/mysql-test/suite/engines/funcs/r/rpl_server_id2.result
+++ b/mysql-test/suite/engines/funcs/r/rpl_server_id2.result
@@ -20,6 +20,7 @@ stop slave;
include/wait_for_slave_to_stop.inc
drop table t1;
connection master;
+include/kill_binlog_dump_threads.inc
reset master;
create table t1(n int);
create table t2(n int);
diff --git a/mysql-test/suite/engines/funcs/r/rpl_trigger.result b/mysql-test/suite/engines/funcs/r/rpl_trigger.result
index 3913508d032dd..8f2c1f6a7b859 100644
--- a/mysql-test/suite/engines/funcs/r/rpl_trigger.result
+++ b/mysql-test/suite/engines/funcs/r/rpl_trigger.result
@@ -1006,6 +1006,7 @@ SHOW TABLES LIKE 't_';
Tables_in_test (t_)
SHOW TRIGGERS;
Trigger Event Table Statement Timing Created sql_mode Definer character_set_client collation_connection Database Collation
+include/kill_binlog_dump_threads.inc
RESET MASTER;
connection slave;
include/start_slave.inc
diff --git a/mysql-test/suite/engines/funcs/t/rpl_000015.test b/mysql-test/suite/engines/funcs/t/rpl_000015.test
index 8996affe667c5..fa07877329535 100644
--- a/mysql-test/suite/engines/funcs/t/rpl_000015.test
+++ b/mysql-test/suite/engines/funcs/t/rpl_000015.test
@@ -5,11 +5,14 @@
#####################
--source include/master-slave.inc
+connection slave;
+--source include/stop_slave.inc
+connection master;
+--source include/kill_binlog_dump_threads.inc
reset master;
show master status;
save_master_pos;
connection slave;
---source include/stop_slave.inc
reset slave;
--let $status_items= Slave_IO_Running, Slave_SQL_Running, Last_SQL_Errno, Last_SQL_Error, Exec_Master_Log_Pos
--source include/show_slave_status.inc
diff --git a/mysql-test/suite/engines/funcs/t/rpl_trigger.test b/mysql-test/suite/engines/funcs/t/rpl_trigger.test
index 12eef32ef8321..8f58a577818d1 100644
--- a/mysql-test/suite/engines/funcs/t/rpl_trigger.test
+++ b/mysql-test/suite/engines/funcs/t/rpl_trigger.test
@@ -357,6 +357,7 @@ connection master;
SHOW TABLES LIKE 't_';
SHOW TRIGGERS;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
# Restart slave.
diff --git a/mysql-test/suite/galera/r/MDEV-28053.result b/mysql-test/suite/galera/r/MDEV-28053.result
index b3f93688dd050..5aa2ab5cb7ab8 100644
--- a/mysql-test/suite/galera/r/MDEV-28053.result
+++ b/mysql-test/suite/galera/r/MDEV-28053.result
@@ -11,4 +11,5 @@ connection node_2;
STOP SLAVE;
RESET SLAVE ALL;
connection node_3;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
diff --git a/mysql-test/suite/galera/r/galera_2primary_replica.result b/mysql-test/suite/galera/r/galera_2primary_replica.result
index 9901caa9eb685..cd10fc978329b 100644
--- a/mysql-test/suite/galera/r/galera_2primary_replica.result
+++ b/mysql-test/suite/galera/r/galera_2primary_replica.result
@@ -86,8 +86,10 @@ Note 1938 SLAVE 'stream1' stopped
Note 1938 SLAVE 'stream2' stopped
RESET SLAVE ALL;
connection primary1;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
connection primary2;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
connection node_1;
disconnect primary1;
diff --git a/mysql-test/suite/galera/r/galera_as_master.result b/mysql-test/suite/galera/r/galera_as_master.result
index dd3e017379c54..6764199818ba1 100644
--- a/mysql-test/suite/galera/r/galera_as_master.result
+++ b/mysql-test/suite/galera/r/galera_as_master.result
@@ -59,5 +59,6 @@ RESET SLAVE ALL;
CALL mtr.add_suppression('You need to use --log-bin to make --binlog-format work');
connection node_1;
set global wsrep_on=OFF;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
set global wsrep_on=ON;
diff --git a/mysql-test/suite/galera/r/galera_as_slave.result b/mysql-test/suite/galera/r/galera_as_slave.result
index 1035fe1e9be1a..ecd7a2c063df2 100644
--- a/mysql-test/suite/galera/r/galera_as_slave.result
+++ b/mysql-test/suite/galera/r/galera_as_slave.result
@@ -29,4 +29,5 @@ connection node_2;
STOP SLAVE;
RESET SLAVE ALL;
connection node_3;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
diff --git a/mysql-test/suite/galera/r/galera_as_slave_autoinc.result b/mysql-test/suite/galera/r/galera_as_slave_autoinc.result
index ffaa62c0003fd..5e3bd335731b6 100644
--- a/mysql-test/suite/galera/r/galera_as_slave_autoinc.result
+++ b/mysql-test/suite/galera/r/galera_as_slave_autoinc.result
@@ -97,4 +97,5 @@ connection node_2;
STOP SLAVE;
RESET SLAVE ALL;
connection node_3;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
diff --git a/mysql-test/suite/galera/r/galera_as_slave_ctas.result b/mysql-test/suite/galera/r/galera_as_slave_ctas.result
index a3d28ce42aa33..d3c9543fd3695 100644
--- a/mysql-test/suite/galera/r/galera_as_slave_ctas.result
+++ b/mysql-test/suite/galera/r/galera_as_slave_ctas.result
@@ -28,4 +28,5 @@ connection node_1;
STOP SLAVE;
RESET SLAVE ALL;
connection node_3;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
diff --git a/mysql-test/suite/galera/r/galera_as_slave_gtid.result b/mysql-test/suite/galera/r/galera_as_slave_gtid.result
index 41d9085ccacae..665b58bd5d4ce 100644
--- a/mysql-test/suite/galera/r/galera_as_slave_gtid.result
+++ b/mysql-test/suite/galera/r/galera_as_slave_gtid.result
@@ -28,11 +28,14 @@ RESET SLAVE ALL;
#cleanup
connection node_1;
set global wsrep_on=OFF;
+include/kill_binlog_dump_threads.inc
reset master;
set global wsrep_on=ON;
connection node_2;
set global wsrep_on=OFF;
+include/kill_binlog_dump_threads.inc
reset master;
set global wsrep_on=ON;
connection node_3;
+include/kill_binlog_dump_threads.inc
reset master;
diff --git a/mysql-test/suite/galera/r/galera_as_slave_gtid_auto_engine.result b/mysql-test/suite/galera/r/galera_as_slave_gtid_auto_engine.result
index cff5bb0a5d006..91c86901e87b1 100644
--- a/mysql-test/suite/galera/r/galera_as_slave_gtid_auto_engine.result
+++ b/mysql-test/suite/galera/r/galera_as_slave_gtid_auto_engine.result
@@ -28,13 +28,16 @@ RESET SLAVE ALL;
#cleanup
connection node_1;
set global wsrep_on=OFF;
+include/kill_binlog_dump_threads.inc
reset master;
set global wsrep_on=ON;
connection node_2;
set global wsrep_on=OFF;
+include/kill_binlog_dump_threads.inc
reset master;
set global wsrep_on=ON;
connection node_3;
+include/kill_binlog_dump_threads.inc
reset master;
connection node_2;
DROP TABLE mysql.gtid_slave_pos_InnoDB;
diff --git a/mysql-test/suite/galera/r/galera_as_slave_gtid_myisam.result b/mysql-test/suite/galera/r/galera_as_slave_gtid_myisam.result
index 5698ebf9fd39e..ddfebf7db99d8 100644
--- a/mysql-test/suite/galera/r/galera_as_slave_gtid_myisam.result
+++ b/mysql-test/suite/galera/r/galera_as_slave_gtid_myisam.result
@@ -28,16 +28,18 @@ DROP TABLE t1;
connection node_1;
connection node_2;
connection node_3;
-RESET MASTER;
connection node_1;
STOP SLAVE;
RESET SLAVE ALL;
SET GLOBAL WSREP_ON=OFF;
+include/kill_binlog_dump_threads.inc
reset master;
SET GLOBAL WSREP_ON=ON;
connection node_2;
SET GLOBAL WSREP_ON=OFF;
+include/kill_binlog_dump_threads.inc
reset master;
SET GLOBAL WSREP_ON=ON;
connection node_3;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
diff --git a/mysql-test/suite/galera/r/galera_as_slave_nonprim.result b/mysql-test/suite/galera/r/galera_as_slave_nonprim.result
index 3f625989fa12d..b72740eb8d488 100644
--- a/mysql-test/suite/galera/r/galera_as_slave_nonprim.result
+++ b/mysql-test/suite/galera/r/galera_as_slave_nonprim.result
@@ -30,4 +30,5 @@ CALL mtr.add_suppression("(Transport endpoint|Socket) is not connected");
CALL mtr.add_suppression("Slave SQL: Error in Xid_log_event: Commit could not be completed, 'Deadlock found when trying to get lock; try restarting transaction', Error_code: 1213");
CALL mtr.add_suppression("Slave SQL: Node has dropped from cluster, Error_code: 1047");
connection node_4;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
diff --git a/mysql-test/suite/galera/r/galera_circular_replication.result b/mysql-test/suite/galera/r/galera_circular_replication.result
index 0094283d40b0b..e9f59f773854e 100644
--- a/mysql-test/suite/galera/r/galera_circular_replication.result
+++ b/mysql-test/suite/galera/r/galera_circular_replication.result
@@ -128,6 +128,7 @@ RESET SLAVE ALL;
connection replica2;
STOP SLAVE;
RESET SLAVE ALL;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
connection node_1;
disconnect primary1;
diff --git a/mysql-test/suite/galera/r/galera_gtid_slave.result b/mysql-test/suite/galera/r/galera_gtid_slave.result
index 180be320102f9..f9a093eab34a9 100644
--- a/mysql-test/suite/galera/r/galera_gtid_slave.result
+++ b/mysql-test/suite/galera/r/galera_gtid_slave.result
@@ -33,13 +33,16 @@ connection node_2;
STOP SLAVE;
RESET SLAVE ALL;
SET GLOBAL wsrep_on=OFF;
+include/kill_binlog_dump_threads.inc
reset master;
SET GLOBAL wsrep_on=ON;
connection node_1;
SET GLOBAL wsrep_on=OFF;
+include/kill_binlog_dump_threads.inc
reset master;
SET GLOBAL wsrep_on=ON;
connection node_3;
+include/kill_binlog_dump_threads.inc
reset master;
connection node_2;
CALL mtr.add_suppression("Ignoring server id .* for non bootstrap node");
diff --git a/mysql-test/suite/galera/r/galera_gtid_slave_sst_rsync.result b/mysql-test/suite/galera/r/galera_gtid_slave_sst_rsync.result
index e9f11d6f44b5a..a7e69a5cff35b 100644
--- a/mysql-test/suite/galera/r/galera_gtid_slave_sst_rsync.result
+++ b/mysql-test/suite/galera/r/galera_gtid_slave_sst_rsync.result
@@ -155,16 +155,19 @@ connection node_2;
STOP SLAVE;
RESET SLAVE ALL;
set global wsrep_on=OFF;
+include/kill_binlog_dump_threads.inc
reset master;
set global wsrep_on=ON;
set global gtid_slave_pos="";
#Connection 1
connection node_1;
set global wsrep_on=OFF;
+include/kill_binlog_dump_threads.inc
reset master;
set global wsrep_on=ON;
#Connection 3
connection node_3;
+include/kill_binlog_dump_threads.inc
reset master;
connection node_2;
CALL mtr.add_suppression("Ignoring server id .* for non bootstrap node");
diff --git a/mysql-test/suite/galera/r/galera_query_cache_invalidate.result b/mysql-test/suite/galera/r/galera_query_cache_invalidate.result
index d07fe62c2d54b..755f2218923c2 100644
--- a/mysql-test/suite/galera/r/galera_query_cache_invalidate.result
+++ b/mysql-test/suite/galera/r/galera_query_cache_invalidate.result
@@ -113,6 +113,7 @@ Warnings:
Note 4190 RESET SLAVE is implicitly changing the value of 'Using_Gtid' from 'Current_Pos' to 'Slave_Pos'
connection node_1;
SET SESSION WSREP_ON=OFF;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
SET SESSION WSREP_ON=ON;
disconnect node_2;
diff --git a/mysql-test/suite/galera/r/galera_replica_no_gtid.result b/mysql-test/suite/galera/r/galera_replica_no_gtid.result
index f4b1a344400ec..5643ba614c373 100644
--- a/mysql-test/suite/galera/r/galera_replica_no_gtid.result
+++ b/mysql-test/suite/galera/r/galera_replica_no_gtid.result
@@ -59,6 +59,7 @@ connection node_2;
STOP SLAVE;
RESET SLAVE ALL;
connection node_3;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
drop table t1;
connection node_2;
diff --git a/mysql-test/suite/galera/r/galera_restart_replica.result b/mysql-test/suite/galera/r/galera_restart_replica.result
index 1691f9822a8d0..6bd5ba4311d4d 100644
--- a/mysql-test/suite/galera/r/galera_restart_replica.result
+++ b/mysql-test/suite/galera/r/galera_restart_replica.result
@@ -161,6 +161,7 @@ connection replica;
STOP SLAVE;
RESET SLAVE ALL;
connection primary;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
connection node_1;
disconnect primary;
diff --git a/mysql-test/suite/galera/r/rpl_galera_to_mariadb_clone_slave_using_mariadb-backup.result b/mysql-test/suite/galera/r/rpl_galera_to_mariadb_clone_slave_using_mariadb-backup.result
index 71c84d01a8928..05c71ee517712 100644
--- a/mysql-test/suite/galera/r/rpl_galera_to_mariadb_clone_slave_using_mariadb-backup.result
+++ b/mysql-test/suite/galera/r/rpl_galera_to_mariadb_clone_slave_using_mariadb-backup.result
@@ -207,5 +207,6 @@ Warnings:
Note 4190 RESET SLAVE is implicitly changing the value of 'Using_Gtid' from 'No' to 'Slave_Pos'
connection master;
set global wsrep_on=OFF;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
set global wsrep_on=ON;
diff --git a/mysql-test/suite/galera/t/MDEV-28053.test b/mysql-test/suite/galera/t/MDEV-28053.test
index 323d7a5ba0e73..894bdb4b58044 100644
--- a/mysql-test/suite/galera/t/MDEV-28053.test
+++ b/mysql-test/suite/galera/t/MDEV-28053.test
@@ -59,4 +59,5 @@ STOP SLAVE;
RESET SLAVE ALL;
--connection node_3
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
diff --git a/mysql-test/suite/galera/t/galera_2primary_replica.test b/mysql-test/suite/galera/t/galera_2primary_replica.test
index da20026f5c9c2..8636b4d9147ca 100644
--- a/mysql-test/suite/galera/t/galera_2primary_replica.test
+++ b/mysql-test/suite/galera/t/galera_2primary_replica.test
@@ -157,8 +157,10 @@ STOP ALL SLAVES;
RESET SLAVE ALL;
--connection primary1
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
--connection primary2
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
--source include/auto_increment_offset_restore.inc
diff --git a/mysql-test/suite/galera/t/galera_as_master.test b/mysql-test/suite/galera/t/galera_as_master.test
index c19784aaa3403..d288a1ea24bb2 100644
--- a/mysql-test/suite/galera/t/galera_as_master.test
+++ b/mysql-test/suite/galera/t/galera_as_master.test
@@ -71,5 +71,6 @@ CALL mtr.add_suppression('You need to use --log-bin to make --binlog-format work
--connection node_1
set global wsrep_on=OFF;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
set global wsrep_on=ON;
diff --git a/mysql-test/suite/galera/t/galera_as_slave.test b/mysql-test/suite/galera/t/galera_as_slave.test
index e908209dc95d1..609e5bdf914c6 100644
--- a/mysql-test/suite/galera/t/galera_as_slave.test
+++ b/mysql-test/suite/galera/t/galera_as_slave.test
@@ -63,4 +63,5 @@ STOP SLAVE;
RESET SLAVE ALL;
--connection node_3
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
diff --git a/mysql-test/suite/galera/t/galera_as_slave_autoinc.test b/mysql-test/suite/galera/t/galera_as_slave_autoinc.test
index cb028cd816104..b84eb0c28033e 100644
--- a/mysql-test/suite/galera/t/galera_as_slave_autoinc.test
+++ b/mysql-test/suite/galera/t/galera_as_slave_autoinc.test
@@ -82,4 +82,5 @@ STOP SLAVE;
RESET SLAVE ALL;
--connection node_3
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
diff --git a/mysql-test/suite/galera/t/galera_as_slave_ctas.test b/mysql-test/suite/galera/t/galera_as_slave_ctas.test
index d09fa0b78c3ac..8ace18fbdf3f8 100644
--- a/mysql-test/suite/galera/t/galera_as_slave_ctas.test
+++ b/mysql-test/suite/galera/t/galera_as_slave_ctas.test
@@ -72,4 +72,5 @@ STOP SLAVE;
RESET SLAVE ALL;
--connection node_3
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
diff --git a/mysql-test/suite/galera/t/galera_as_slave_gtid.inc b/mysql-test/suite/galera/t/galera_as_slave_gtid.inc
index 983f6ebd706cf..38f5898a1461b 100644
--- a/mysql-test/suite/galera/t/galera_as_slave_gtid.inc
+++ b/mysql-test/suite/galera/t/galera_as_slave_gtid.inc
@@ -82,13 +82,16 @@ RESET SLAVE ALL;
--echo #cleanup
--connection node_1
set global wsrep_on=OFF;
+--source include/kill_binlog_dump_threads.inc
reset master;
set global wsrep_on=ON;
--connection node_2
set global wsrep_on=OFF;
+--source include/kill_binlog_dump_threads.inc
reset master;
set global wsrep_on=ON;
--connection node_3
+--source include/kill_binlog_dump_threads.inc
reset master;
diff --git a/mysql-test/suite/galera/t/galera_as_slave_gtid_myisam.test b/mysql-test/suite/galera/t/galera_as_slave_gtid_myisam.test
index 38f062b1748e7..2876d0629e55d 100644
--- a/mysql-test/suite/galera/t/galera_as_slave_gtid_myisam.test
+++ b/mysql-test/suite/galera/t/galera_as_slave_gtid_myisam.test
@@ -68,19 +68,21 @@ DROP TABLE t1;
--connection node_3
--let $wait_condition = SELECT COUNT(*) = 0 FROM INFORMATION_SCHEMA.TABLES WHERE TABLE_NAME = 't1';
--source include/wait_condition.inc
-RESET MASTER;
--connection node_1
STOP SLAVE;
RESET SLAVE ALL;
SET GLOBAL WSREP_ON=OFF;
+--source include/kill_binlog_dump_threads.inc
reset master;
SET GLOBAL WSREP_ON=ON;
--connection node_2
SET GLOBAL WSREP_ON=OFF;
+--source include/kill_binlog_dump_threads.inc
reset master;
SET GLOBAL WSREP_ON=ON;
--connection node_3
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
diff --git a/mysql-test/suite/galera/t/galera_as_slave_nonprim.test b/mysql-test/suite/galera/t/galera_as_slave_nonprim.test
index b0bbf5e0030a9..2dab9c9b86163 100644
--- a/mysql-test/suite/galera/t/galera_as_slave_nonprim.test
+++ b/mysql-test/suite/galera/t/galera_as_slave_nonprim.test
@@ -88,4 +88,5 @@ CALL mtr.add_suppression("Slave SQL: Error in Xid_log_event: Commit could not be
CALL mtr.add_suppression("Slave SQL: Node has dropped from cluster, Error_code: 1047");
--connection node_4
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
diff --git a/mysql-test/suite/galera/t/galera_circular_replication.test b/mysql-test/suite/galera/t/galera_circular_replication.test
index 11dff973c007f..e1f29c1b56a10 100644
--- a/mysql-test/suite/galera/t/galera_circular_replication.test
+++ b/mysql-test/suite/galera/t/galera_circular_replication.test
@@ -221,6 +221,7 @@ RESET SLAVE ALL;
--connection replica2
STOP SLAVE;
RESET SLAVE ALL;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
--source include/auto_increment_offset_restore.inc
diff --git a/mysql-test/suite/galera/t/galera_gtid_slave.test b/mysql-test/suite/galera/t/galera_gtid_slave.test
index b326ffdffcd9f..726164d1b129b 100644
--- a/mysql-test/suite/galera/t/galera_gtid_slave.test
+++ b/mysql-test/suite/galera/t/galera_gtid_slave.test
@@ -80,15 +80,18 @@ DROP TABLE t1,t2;
STOP SLAVE;
RESET SLAVE ALL;
SET GLOBAL wsrep_on=OFF;
+--source include/kill_binlog_dump_threads.inc
reset master;
SET GLOBAL wsrep_on=ON;
--connection node_1
SET GLOBAL wsrep_on=OFF;
+--source include/kill_binlog_dump_threads.inc
reset master;
SET GLOBAL wsrep_on=ON;
--connection node_3
+--source include/kill_binlog_dump_threads.inc
reset master;
--connection node_2
diff --git a/mysql-test/suite/galera/t/galera_gtid_slave_sst_rsync.test b/mysql-test/suite/galera/t/galera_gtid_slave_sst_rsync.test
index b54c36207cd3a..5c19dae80b47e 100644
--- a/mysql-test/suite/galera/t/galera_gtid_slave_sst_rsync.test
+++ b/mysql-test/suite/galera/t/galera_gtid_slave_sst_rsync.test
@@ -196,6 +196,7 @@ DROP TABLE t2,t1;
STOP SLAVE;
RESET SLAVE ALL;
set global wsrep_on=OFF;
+--source include/kill_binlog_dump_threads.inc
reset master;
set global wsrep_on=ON;
@@ -206,11 +207,13 @@ set global gtid_slave_pos="";
--echo #Connection 1
--connection node_1
set global wsrep_on=OFF;
+--source include/kill_binlog_dump_threads.inc
reset master;
set global wsrep_on=ON;
--echo #Connection 3
--connection node_3
+--source include/kill_binlog_dump_threads.inc
reset master;
--connection node_2
diff --git a/mysql-test/suite/galera/t/galera_query_cache_invalidate.test b/mysql-test/suite/galera/t/galera_query_cache_invalidate.test
index e27eb5fc86ab7..1bd2515cc3915 100644
--- a/mysql-test/suite/galera/t/galera_query_cache_invalidate.test
+++ b/mysql-test/suite/galera/t/galera_query_cache_invalidate.test
@@ -114,6 +114,7 @@ RESET SLAVE ALL;
--connection node_1
SET SESSION WSREP_ON=OFF;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
SET SESSION WSREP_ON=ON;
diff --git a/mysql-test/suite/galera/t/galera_replica_no_gtid.test b/mysql-test/suite/galera/t/galera_replica_no_gtid.test
index de3bbc4b27643..7851bc83ba26e 100644
--- a/mysql-test/suite/galera/t/galera_replica_no_gtid.test
+++ b/mysql-test/suite/galera/t/galera_replica_no_gtid.test
@@ -106,6 +106,7 @@ STOP SLAVE;
RESET SLAVE ALL;
--connection node_3
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
drop table t1;
diff --git a/mysql-test/suite/galera/t/galera_restart_replica.test b/mysql-test/suite/galera/t/galera_restart_replica.test
index d28bd0ed1f716..6706caca9b97d 100644
--- a/mysql-test/suite/galera/t/galera_restart_replica.test
+++ b/mysql-test/suite/galera/t/galera_restart_replica.test
@@ -274,6 +274,7 @@ STOP SLAVE;
RESET SLAVE ALL;
--connection primary
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
--source include/auto_increment_offset_restore.inc
diff --git a/mysql-test/suite/galera_3nodes/r/galera_2_cluster.result b/mysql-test/suite/galera_3nodes/r/galera_2_cluster.result
index c3e233a9fb7be..23412f70e53a8 100644
--- a/mysql-test/suite/galera_3nodes/r/galera_2_cluster.result
+++ b/mysql-test/suite/galera_3nodes/r/galera_2_cluster.result
@@ -75,27 +75,33 @@ RESET SLAVE;
Warnings:
Note 4190 RESET SLAVE is implicitly changing the value of 'Using_Gtid' from 'Current_Pos' to 'Slave_Pos'
SET GLOBAL wsrep_on = OFF;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
SET GLOBAL wsrep_on = ON;
SET GLOBAL GTID_SLAVE_POS="";
connection node_1;
SET GLOBAL wsrep_on = OFF;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
SET GLOBAL wsrep_on = ON;
connection node_2;
SET GLOBAL wsrep_on = OFF;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
SET GLOBAL wsrep_on = ON;
connection node_3;
SET GLOBAL wsrep_on = OFF;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
SET GLOBAL wsrep_on = ON;
connection node_5;
SET GLOBAL wsrep_on = OFF;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
SET GLOBAL wsrep_on = ON;
connection node_6;
SET GLOBAL wsrep_on = OFF;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
SET GLOBAL wsrep_on = ON;
connection node_1;
diff --git a/mysql-test/suite/galera_3nodes/r/galera_gtid_2_cluster.result b/mysql-test/suite/galera_3nodes/r/galera_gtid_2_cluster.result
index 36a63c8771369..b3888cd1f821e 100644
--- a/mysql-test/suite/galera_3nodes/r/galera_gtid_2_cluster.result
+++ b/mysql-test/suite/galera_3nodes/r/galera_gtid_2_cluster.result
@@ -235,6 +235,7 @@ cluster 1 node 1
connection node_1;
change master to master_use_gtid=no, ignore_server_ids=();
set global wsrep_on=OFF;
+include/kill_binlog_dump_threads.inc
reset master;
set global wsrep_on=ON;
set global GTID_SLAVE_POS="";
@@ -242,23 +243,28 @@ cluster 2 node 1
connection node_4;
change master to master_use_gtid=no, ignore_server_ids=();
set global wsrep_on=OFF;
+include/kill_binlog_dump_threads.inc
reset master;
set global wsrep_on=ON;
set global GTID_SLAVE_POS="";
connection node_2;
set global wsrep_on=OFF;
+include/kill_binlog_dump_threads.inc
reset master;
set global wsrep_on=ON;
connection node_3;
set global wsrep_on=OFF;
+include/kill_binlog_dump_threads.inc
reset master;
set global wsrep_on=ON;
connection node_5;
set global wsrep_on=OFF;
+include/kill_binlog_dump_threads.inc
reset master;
set global wsrep_on=ON;
connection node_6;
set global wsrep_on=OFF;
+include/kill_binlog_dump_threads.inc
reset master;
set global wsrep_on=ON;
connection node_1;
@@ -448,6 +454,7 @@ drop table t1;
stop slave;
change master to master_use_gtid=no, ignore_server_ids=();
set global wsrep_on=OFF;
+include/kill_binlog_dump_threads.inc
reset master;
set global wsrep_on=ON;
set global GTID_SLAVE_POS="";
@@ -456,22 +463,27 @@ connection node_4;
stop slave;
change master to master_use_gtid=no, ignore_server_ids=();
set global wsrep_on=OFF;
+include/kill_binlog_dump_threads.inc
reset master;
set global wsrep_on=ON;
set global GTID_SLAVE_POS="";
connection node_2;
set global wsrep_on=OFF;
+include/kill_binlog_dump_threads.inc
reset master;
set global wsrep_on=ON;
connection node_3;
set global wsrep_on=OFF;
+include/kill_binlog_dump_threads.inc
reset master;
set global wsrep_on=ON;
connection node_5;
set global wsrep_on=OFF;
+include/kill_binlog_dump_threads.inc
reset master;
set global wsrep_on=ON;
connection node_6;
set global wsrep_on=OFF;
+include/kill_binlog_dump_threads.inc
reset master;
set global wsrep_on=ON;
diff --git a/mysql-test/suite/galera_3nodes/t/galera_2_cluster.test b/mysql-test/suite/galera_3nodes/t/galera_2_cluster.test
index d20b64e7c81f3..48f44c8817623 100644
--- a/mysql-test/suite/galera_3nodes/t/galera_2_cluster.test
+++ b/mysql-test/suite/galera_3nodes/t/galera_2_cluster.test
@@ -138,6 +138,7 @@ DROP TABLE t1;
STOP SLAVE;
RESET SLAVE;
SET GLOBAL wsrep_on = OFF;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
SET GLOBAL wsrep_on = ON;
--source include/wait_until_ready.inc
@@ -146,6 +147,7 @@ SET GLOBAL GTID_SLAVE_POS="";
--connection node_1
SET GLOBAL wsrep_on = OFF;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
SET GLOBAL wsrep_on = ON;
--source include/wait_until_ready.inc
@@ -153,6 +155,7 @@ SET GLOBAL wsrep_on = ON;
--connection node_2
SET GLOBAL wsrep_on = OFF;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
SET GLOBAL wsrep_on = ON;
--source include/wait_until_ready.inc
@@ -160,6 +163,7 @@ SET GLOBAL wsrep_on = ON;
--connection node_3
SET GLOBAL wsrep_on = OFF;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
SET GLOBAL wsrep_on = ON;
--source include/wait_until_ready.inc
@@ -167,6 +171,7 @@ SET GLOBAL wsrep_on = ON;
--connection node_5
SET GLOBAL wsrep_on = OFF;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
SET GLOBAL wsrep_on = ON;
--source include/wait_until_ready.inc
@@ -174,6 +179,7 @@ SET GLOBAL wsrep_on = ON;
--connection node_6
SET GLOBAL wsrep_on = OFF;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
SET GLOBAL wsrep_on = ON;
--source include/wait_until_ready.inc
diff --git a/mysql-test/suite/galera_3nodes/t/galera_gtid_2_cluster.test b/mysql-test/suite/galera_3nodes/t/galera_gtid_2_cluster.test
index 659b293582b13..f38ab6c7b639f 100644
--- a/mysql-test/suite/galera_3nodes/t/galera_gtid_2_cluster.test
+++ b/mysql-test/suite/galera_3nodes/t/galera_gtid_2_cluster.test
@@ -203,6 +203,7 @@ reset slave;
--connection node_1
change master to master_use_gtid=no, ignore_server_ids=();
set global wsrep_on=OFF;
+--source include/kill_binlog_dump_threads.inc
reset master;
set global wsrep_on=ON;
set global GTID_SLAVE_POS="";
@@ -212,24 +213,29 @@ set global GTID_SLAVE_POS="";
--connection node_4
change master to master_use_gtid=no, ignore_server_ids=();
set global wsrep_on=OFF;
+--source include/kill_binlog_dump_threads.inc
reset master;
set global wsrep_on=ON;
set global GTID_SLAVE_POS="";
--connection node_2
set global wsrep_on=OFF;
+--source include/kill_binlog_dump_threads.inc
reset master;
set global wsrep_on=ON;
--connection node_3
set global wsrep_on=OFF;
+--source include/kill_binlog_dump_threads.inc
reset master;
set global wsrep_on=ON;
--connection node_5
set global wsrep_on=OFF;
+--source include/kill_binlog_dump_threads.inc
reset master;
set global wsrep_on=ON;
--connection node_6
set global wsrep_on=OFF;
+--source include/kill_binlog_dump_threads.inc
reset master;
set global wsrep_on=ON;
#--enable_parsing
@@ -398,6 +404,7 @@ drop table t1;
stop slave;
change master to master_use_gtid=no, ignore_server_ids=();
set global wsrep_on=OFF;
+--source include/kill_binlog_dump_threads.inc
reset master;
set global wsrep_on=ON;
set global GTID_SLAVE_POS="";
@@ -407,23 +414,28 @@ set global GTID_SLAVE_POS="";
stop slave;
change master to master_use_gtid=no, ignore_server_ids=();
set global wsrep_on=OFF;
+--source include/kill_binlog_dump_threads.inc
reset master;
set global wsrep_on=ON;
set global GTID_SLAVE_POS="";
--connection node_2
set global wsrep_on=OFF;
+--source include/kill_binlog_dump_threads.inc
reset master;
set global wsrep_on=ON;
--connection node_3
set global wsrep_on=OFF;
+--source include/kill_binlog_dump_threads.inc
reset master;
set global wsrep_on=ON;
--connection node_5
set global wsrep_on=OFF;
+--source include/kill_binlog_dump_threads.inc
reset master;
set global wsrep_on=ON;
--connection node_6
set global wsrep_on=OFF;
+--source include/kill_binlog_dump_threads.inc
reset master;
set global wsrep_on=ON;
diff --git a/mysql-test/suite/perfschema/r/relaylog.result b/mysql-test/suite/perfschema/r/relaylog.result
index 7cc87530770ca..3f4ac49ac9974 100644
--- a/mysql-test/suite/perfschema/r/relaylog.result
+++ b/mysql-test/suite/perfschema/r/relaylog.result
@@ -65,11 +65,13 @@ where event_name like "%MYSQL_BIN_LOG%"
and event_name not like "%MYSQL_BIN_LOG::COND_xid_list"
order by event_name;
EVENT_NAME COUNT_STAR
+wait/synch/cond/sql/MYSQL_BIN_LOG::COND_binlog_use NONE
wait/synch/cond/sql/MYSQL_BIN_LOG::COND_bin_log_updated MANY
wait/synch/cond/sql/MYSQL_BIN_LOG::COND_queue_busy NONE
wait/synch/cond/sql/MYSQL_BIN_LOG::COND_relay_log_updated NONE
wait/synch/mutex/sql/MYSQL_BIN_LOG::LOCK_binlog_background_thread MANY
wait/synch/mutex/sql/MYSQL_BIN_LOG::LOCK_binlog_end_pos MANY
+wait/synch/mutex/sql/MYSQL_BIN_LOG::LOCK_binlog_use MANY
wait/synch/mutex/sql/MYSQL_BIN_LOG::LOCK_index MANY
wait/synch/mutex/sql/MYSQL_BIN_LOG::LOCK_xid_list MANY
"Expect no slave relay log"
@@ -155,11 +157,13 @@ where event_name like "%MYSQL_BIN_LOG%"
and event_name not like "%MYSQL_BIN_LOG::COND_xid_list"
order by event_name;
EVENT_NAME COUNT_STAR
+wait/synch/cond/sql/MYSQL_BIN_LOG::COND_binlog_use NONE
wait/synch/cond/sql/MYSQL_BIN_LOG::COND_bin_log_updated NONE
wait/synch/cond/sql/MYSQL_BIN_LOG::COND_queue_busy NONE
wait/synch/cond/sql/MYSQL_BIN_LOG::COND_relay_log_updated NONE
wait/synch/mutex/sql/MYSQL_BIN_LOG::LOCK_binlog_background_thread MANY
wait/synch/mutex/sql/MYSQL_BIN_LOG::LOCK_binlog_end_pos MANY
+wait/synch/mutex/sql/MYSQL_BIN_LOG::LOCK_binlog_use MANY
wait/synch/mutex/sql/MYSQL_BIN_LOG::LOCK_index MANY
wait/synch/mutex/sql/MYSQL_BIN_LOG::LOCK_xid_list MANY
"Expect a slave relay log"
diff --git a/mysql-test/suite/perfschema/r/transaction.result b/mysql-test/suite/perfschema/r/transaction.result
index c516a4543c274..35941e6d470fe 100644
--- a/mysql-test/suite/perfschema/r/transaction.result
+++ b/mysql-test/suite/perfschema/r/transaction.result
@@ -520,9 +520,9 @@ ACTUAL EXPECTED
connection con1;
RELEASE SAVEPOINT SVP001;
#
-## Expect 1 active transaction with 3 savepoints, 1 rollback to savepoint, 1 release savepoint
+## Expect 1 active transaction with 3 savepoints, 1 rollback to savepoint, 2 release savepoint
connection default;
-CALL transaction_verifier(0, @con1_thread_id, 'transaction', 'ACTIVE', 0, '', '', '', '', 'READ WRITE', '', 'NO', 3, 1, 1, 1);
+CALL transaction_verifier(0, @con1_thread_id, 'transaction', 'ACTIVE', 0, '', '', '', '', 'READ WRITE', '', 'NO', 3, 1, 2, 1);
ACTUAL EXPECTED
1 1
#
@@ -531,9 +531,9 @@ ACTUAL EXPECTED
connection con1;
COMMIT;
#
-## Expect 1 committed transaction with 3 savepoints, 1 rollback to savepoint, 1 release savepoint
+## Expect 1 committed transaction with 3 savepoints, 1 rollback to savepoint, 2 release savepoint
connection default;
-CALL transaction_verifier(1, @con1_thread_id, 'transaction', 'COMMITTED', 0, '', '', '', '', 'READ WRITE', '', 'NO', 3, 1, 1, 1);
+CALL transaction_verifier(1, @con1_thread_id, 'transaction', 'COMMITTED', 0, '', '', '', '', 'READ WRITE', '', 'NO', 3, 1, 2, 1);
ACTUAL EXPECTED
1 1
CALL clear_transaction_tables();
diff --git a/mysql-test/suite/perfschema/t/transaction.test b/mysql-test/suite/perfschema/t/transaction.test
index d950139f7d6dc..d9b6a1ebb791d 100644
--- a/mysql-test/suite/perfschema/t/transaction.test
+++ b/mysql-test/suite/perfschema/t/transaction.test
@@ -524,12 +524,14 @@ CALL transaction_verifier(0, @con1_thread_id, 'transaction', 'ACTIVE', 0, '', ''
--echo # STEP 9.4 - RELEASE SAVEPOINT 1
--echo #
--connection con1
+# Note that this implicitly releases savepoing SVP002 also, hence we expect
+# *2* savepoint releases below.
RELEASE SAVEPOINT SVP001;
--echo #
---echo ## Expect 1 active transaction with 3 savepoints, 1 rollback to savepoint, 1 release savepoint
+--echo ## Expect 1 active transaction with 3 savepoints, 1 rollback to savepoint, 2 release savepoint
--connection default
-CALL transaction_verifier(0, @con1_thread_id, 'transaction', 'ACTIVE', 0, '', '', '', '', 'READ WRITE', '', 'NO', 3, 1, 1, 1);
+CALL transaction_verifier(0, @con1_thread_id, 'transaction', 'ACTIVE', 0, '', '', '', '', 'READ WRITE', '', 'NO', 3, 1, 2, 1);
--echo #
--echo # STEP 9.5 - COMMIT
@@ -538,9 +540,9 @@ CALL transaction_verifier(0, @con1_thread_id, 'transaction', 'ACTIVE', 0, '', ''
COMMIT;
--echo #
---echo ## Expect 1 committed transaction with 3 savepoints, 1 rollback to savepoint, 1 release savepoint
+--echo ## Expect 1 committed transaction with 3 savepoints, 1 rollback to savepoint, 2 release savepoint
--connection default
-CALL transaction_verifier(1, @con1_thread_id, 'transaction', 'COMMITTED', 0, '', '', '', '', 'READ WRITE', '', 'NO', 3, 1, 1, 1);
+CALL transaction_verifier(1, @con1_thread_id, 'transaction', 'COMMITTED', 0, '', '', '', '', 'READ WRITE', '', 'NO', 3, 1, 2, 1);
CALL clear_transaction_tables();
--echo #
diff --git a/mysql-test/suite/rpl/include/mysqlbinlog_slave_consistency.inc b/mysql-test/suite/rpl/include/mysqlbinlog_slave_consistency.inc
index b571c24e9dc70..60b6104ec3691 100644
--- a/mysql-test/suite/rpl/include/mysqlbinlog_slave_consistency.inc
+++ b/mysql-test/suite/rpl/include/mysqlbinlog_slave_consistency.inc
@@ -38,14 +38,14 @@ if (!$strict_mode)
{
--connection $con2
set @@global.gtid_strict_mode=0;
- --let $sql_input_file=include/sql_multisource.inc
+ --let $sql_input_file=suite/rpl/include/sql_multisource.inc
}
if ($strict_mode)
{
--connection $con2
set @@global.gtid_strict_mode=1;
- --let $sql_input_file=include/sql_out_of_order_gtid.inc
+ --let $sql_input_file=suite/rpl/include/sql_out_of_order_gtid.inc
}
--connection $con2
@@ -133,11 +133,12 @@ if (!$slave_sql_errno)
--connection $con1
DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
--echo # MYSQL_BINLOG BINLOG_FILE_PARAM $msbl_args 2> MYSQLBINLOG_STDERR | MYSQL
--exec $MYSQL_BINLOG $BINLOG_FILE_PARAM $msbl_args 2> $MYSQLBINLOG_STDERR | $MYSQL
---source include/rpl_check_table_consistency.inc
+--source suite/rpl/include/rpl_check_table_consistency.inc
if ($strict_mode)
{
@@ -164,13 +165,13 @@ set global gtid_slave_pos="";
CHANGE MASTER TO DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=(), IGNORE_SERVER_IDS=();
--connection $con1
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
DROP TABLE IF EXISTS t1, t2, t3, t4, t5;
--source include/save_master_gtid.inc
--connection $con2
--source include/start_slave.inc
---source include/wait_for_slave_to_start.inc
--source include/sync_with_master_gtid.inc
--source include/stop_slave.inc
RESET SLAVE;
@@ -178,6 +179,7 @@ set global gtid_slave_pos="";
RESET MASTER;
--connection $con1
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
--connection $con2
diff --git a/mysql-test/suite/rpl/include/rpl_ddl.test b/mysql-test/suite/rpl/include/rpl_ddl.test
index e3083a7f3ae53..661e9e011e70a 100644
--- a/mysql-test/suite/rpl/include/rpl_ddl.test
+++ b/mysql-test/suite/rpl/include/rpl_ddl.test
@@ -130,8 +130,11 @@
###############################################################
set global sql_mode='';
set local sql_mode='';
-# The sync_slave_with_master is needed to make the xids deterministic.
-sync_slave_with_master;
+# Sync the slave with master to make the xids deterministic.
+--connection master
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
connection master;
SET AUTOCOMMIT = 1;
@@ -146,8 +149,10 @@ eval CREATE TABLE mysqltest1.t1 (f1 BIGINT) ENGINE=$engine_type;
# Prevent Bug#26687 rpl_ddl test fails if run with --innodb option
# The testscript (suite/rpl/rpl_ddl.test) + the expected result need that the
# slave uses MyISAM for the table mysqltest.t1.
-sync_slave_with_master;
-connection slave;
+--connection master
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
if (`SELECT COUNT(*) FROM INFORMATION_SCHEMA.TABLES
WHERE TABLE_SCHEMA = 'mysqltest1' AND TABLE_NAME = 't1' AND ENGINE <> 'MyISAM'`)
{
@@ -183,7 +188,10 @@ eval CREATE TEMPORARY TABLE mysqltest1.t23 (f1 BIGINT) ENGINE=$temp_engine_type;
#
SET AUTOCOMMIT = 0;
use mysqltest1;
-sync_slave_with_master;
+--connection master
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
connection slave;
SET AUTOCOMMIT = 1;
use mysqltest1;
@@ -303,7 +311,10 @@ let $my_master_commit= true;
let $my_slave_commit= true;
--source include/rpl_stmt_seq.inc
SELECT * FROM mysqltest1.t7;
-sync_slave_with_master;
+--connection master
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
connection slave;
SELECT * FROM mysqltest1.t7;
connection master;
diff --git a/mysql-test/suite/rpl/include/rpl_extra_col_master.inc b/mysql-test/suite/rpl/include/rpl_extra_col_master.inc
index 3fef3cc1fd037..ac91630f8b339 100644
--- a/mysql-test/suite/rpl/include/rpl_extra_col_master.inc
+++ b/mysql-test/suite/rpl/include/rpl_extra_col_master.inc
@@ -466,6 +466,7 @@ eval CREATE TABLE t10 (a INT PRIMARY KEY, b BLOB, c CHAR(5)
--connection master
eval CREATE TABLE t10 (a INT KEY, b BLOB, f DOUBLE DEFAULT '233',
c CHAR(5), e INT DEFAULT '1')ENGINE=$engine_type;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
--echo
@@ -522,6 +523,7 @@ connection master;
eval CREATE TABLE t11 (a INT KEY, b BLOB, f TEXT,
c CHAR(5) DEFAULT 'test', e INT DEFAULT '1')ENGINE=$engine_type;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
--echo
@@ -577,6 +579,7 @@ connection master;
eval CREATE TABLE t12 (a INT KEY, b BLOB, f TEXT,
c CHAR(5) DEFAULT 'test', e INT DEFAULT '1')ENGINE=$engine_type;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
--echo
@@ -630,6 +633,7 @@ eval CREATE TABLE t14 (c1 INT KEY, c4 BLOB, c5 CHAR(5),
c7 TIMESTAMP NULL DEFAULT CURRENT_TIMESTAMP
)ENGINE=$engine_type;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
--echo
@@ -701,6 +705,7 @@ DROP TABLE t14;
connection master;
DROP TABLE t14;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
connection slave;
@@ -727,6 +732,7 @@ eval CREATE TABLE t15 (c1 INT KEY, c4 BLOB, c5 CHAR(5),
c7 TIMESTAMP NULL DEFAULT CURRENT_TIMESTAMP
)ENGINE=$engine_type;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
--echo
@@ -765,6 +771,7 @@ DROP TABLE t15;
connection master;
DROP TABLE t15;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
connection slave;
@@ -791,6 +798,7 @@ eval CREATE TABLE t16 (c1 INT KEY, c4 BLOB, c5 CHAR(5),
c7 TIMESTAMP NULL DEFAULT CURRENT_TIMESTAMP
)ENGINE=$engine_type;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
--echo
@@ -842,6 +850,7 @@ DROP TABLE t16;
connection master;
DROP TABLE t16;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
connection slave;
@@ -869,6 +878,7 @@ eval CREATE TABLE t17 (c1 INT KEY, c4 BLOB, c5 CHAR(5),
c7 TIMESTAMP NULL DEFAULT CURRENT_TIMESTAMP
)ENGINE=$engine_type;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
--echo
@@ -935,6 +945,7 @@ eval CREATE TABLE t18 (c1 INT KEY, c4 BLOB, c5 CHAR(5),
c7 TIMESTAMP NULL DEFAULT CURRENT_TIMESTAMP
)ENGINE=$engine_type;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
--echo
@@ -998,6 +1009,7 @@ eval CREATE TABLE t5 (c1 INT KEY, c4 BLOB, c5 CHAR(5),
c7 TIMESTAMP NULL DEFAULT CURRENT_TIMESTAMP
)ENGINE=$engine_type;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
--echo
diff --git a/mysql-test/suite/rpl/include/rpl_extra_col_slave.test b/mysql-test/suite/rpl/include/rpl_extra_col_slave.test
index 028c833a6e534..96621ace68e68 100644
--- a/mysql-test/suite/rpl/include/rpl_extra_col_slave.test
+++ b/mysql-test/suite/rpl/include/rpl_extra_col_slave.test
@@ -40,6 +40,7 @@ connection master;
eval CREATE TABLE t1 (a INT PRIMARY KEY, b INT, c CHAR(10)
) ENGINE=$engine_type;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
--echo *** Start Slave ***
@@ -81,6 +82,7 @@ eval CREATE TABLE t2 (a INT, b INT PRIMARY KEY, c CHAR(5),
connection master;
eval CREATE TABLE t2 (a INT PRIMARY KEY, b INT, c CHAR(10)
) ENGINE=$engine_type;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
--echo *** Master Data Insert ***
@@ -100,6 +102,7 @@ STOP SLAVE;
SELECT * FROM t2 ORDER BY a;
connection master;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
connection slave;
@@ -126,6 +129,7 @@ eval CREATE TABLE t3 (a INT, b INT PRIMARY KEY, c CHAR(20),
connection master;
eval CREATE TABLE t3 (a BLOB, b INT PRIMARY KEY, c CHAR(20)
) ENGINE=$engine_type;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
--echo *** Start Slave ***
@@ -171,6 +175,7 @@ eval CREATE TABLE t4 (a INT, b INT PRIMARY KEY, c CHAR(20),
connection master;
eval CREATE TABLE t4 (a DECIMAL(8,2), b INT PRIMARY KEY, c CHAR(20)
) ENGINE=$engine_type;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
--echo *** Start Slave ***
@@ -214,6 +219,7 @@ connection master;
eval CREATE TABLE t5 (a INT PRIMARY KEY, b VARCHAR(6),
c DECIMAL(8,2), d BIT, e BLOB,
f FLOAT) ENGINE=$engine_type;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
--echo *** Start Slave ***
@@ -258,6 +264,7 @@ connection master;
eval CREATE TABLE t6 (a INT PRIMARY KEY, b VARCHAR(6),
c DECIMAL(8,2), d BIT
) ENGINE=$engine_type;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
--echo *** Start Slave ***
@@ -317,6 +324,7 @@ eval CREATE TABLE t7 (a INT KEY, b BLOB, c CHAR(5),
connection master;
eval CREATE TABLE t7 (a INT PRIMARY KEY, b BLOB, c CHAR(5)
) ENGINE=$engine_type;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
--echo *** Start Slave ***
@@ -358,6 +366,7 @@ eval CREATE TABLE t8 (a INT KEY, b BLOB, c CHAR(5),
connection master;
eval CREATE TABLE t8 (a INT PRIMARY KEY, b BLOB, c CHAR(5)
) ENGINE=$engine_type;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
--echo *** Start Slave ***
@@ -408,6 +417,7 @@ sync_slave_with_master;
connection master;
eval CREATE TABLE t9 (a INT PRIMARY KEY, b BLOB, c CHAR(5)
) ENGINE=$engine_type;
+ --source include/kill_binlog_dump_threads.inc
RESET MASTER;
--echo *** Start Slave ***
@@ -456,6 +466,7 @@ eval CREATE TABLE t10 (a INT KEY, b BLOB, f DOUBLE DEFAULT '233',
connection master;
eval CREATE TABLE t10 (a INT PRIMARY KEY, b BLOB, c CHAR(5)
) ENGINE=$engine_type;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
--echo *** Start Slave ***
@@ -498,6 +509,7 @@ eval CREATE TABLE t11 (a INT KEY, b BLOB, f INT,
connection master;
eval CREATE TABLE t11 (a INT PRIMARY KEY, b BLOB, c VARCHAR(254)
) ENGINE=$engine_type;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
--echo *** Start Slave ***
@@ -540,6 +552,7 @@ eval CREATE TABLE t12 (a INT KEY, b BLOB, f TEXT,
connection master;
eval CREATE TABLE t12 (a INT PRIMARY KEY, b BLOB, c BLOB
) ENGINE=$engine_type;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
--echo *** Start Slave ***
@@ -581,6 +594,7 @@ eval CREATE TABLE t13 (a INT KEY, b BLOB, c CHAR(5),
connection master;
eval CREATE TABLE t13 (a INT PRIMARY KEY, b BLOB, c CHAR(5)
) ENGINE=$engine_type;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
--echo *** Start Slave ***
@@ -629,6 +643,8 @@ eval CREATE TABLE t14 (c1 INT KEY, c4 BLOB, c5 CHAR(5),
connection master;
eval CREATE TABLE t14 (c1 INT PRIMARY KEY, c4 BLOB, c5 CHAR(5)
) ENGINE=$engine_type;
+--source include/kill_binlog_dump_threads.inc
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
--echo *** Start Slave ***
@@ -669,6 +685,7 @@ eval CREATE TABLE t14a (c1 INT KEY, c4 BLOB, c5 CHAR(5),
connection master;
eval CREATE TABLE t14a (c1 INT PRIMARY KEY, c4 BLOB, c5 CHAR(5)
) ENGINE=$engine_type;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
--echo *** Start Slave ***
@@ -694,6 +711,7 @@ STOP SLAVE;
--echo *** Master Drop c5 ***
connection master;
ALTER TABLE t14a DROP COLUMN c5;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
--echo *** Start Slave ***
@@ -759,6 +777,7 @@ eval CREATE TABLE t15 (c1 INT KEY, c2 DECIMAL(8,2), c3 TEXT,
connection master;
eval CREATE TABLE t15 (c1 INT PRIMARY KEY, c2 DECIMAL(8,2), c3 TEXT,
c4 BLOB, c5 CHAR(5)) ENGINE=$engine_type;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
--echo *** Start Slave ***
@@ -832,6 +851,7 @@ eval CREATE TABLE t16 (c1 INT KEY, c2 DECIMAL(8,2), c3 TEXT,
connection master;
eval CREATE TABLE t16 (c1 INT PRIMARY KEY, c2 DECIMAL(8,2), c3 TEXT,
c4 BLOB, c5 CHAR(5))ENGINE=$engine_type;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
--echo *** Start Slave ***
@@ -886,6 +906,7 @@ eval CREATE TABLE t17 (a SMALLINT, b INT PRIMARY KEY, c CHAR(5),
connection master;
eval CREATE TABLE t17 (a BIGINT PRIMARY KEY, b INT, c CHAR(10)
) ENGINE=$engine_type;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
--echo *** Start Slave ***
diff --git a/mysql-test/suite/rpl/include/rpl_gtid_index.inc b/mysql-test/suite/rpl/include/rpl_gtid_index.inc
index 262d43d0109c4..ae8f49e70fcdf 100644
--- a/mysql-test/suite/rpl/include/rpl_gtid_index.inc
+++ b/mysql-test/suite/rpl/include/rpl_gtid_index.inc
@@ -81,12 +81,14 @@ SET server_id= @orig_server_id;
SELECT COUNT(*) FROM gtid_data;
-# Test that BINLOG_GTID_POS returns correct positions for every GTID position.
---echo *** The result should be empty, otherwise some result is wrong:
-SELECT idx, gtid_pos, BINLOG_GTID_POS(file, pos)
- FROM gtid_data
- WHERE NOT gtid_eq(CONVERT(gtid_pos USING utf8),BINLOG_GTID_POS(file, pos))
- ORDER BY idx;
+if ($binlog_in_engine_test != '1') {
+ # Test that BINLOG_GTID_POS returns correct positions for every GTID position.
+ --echo *** The result should be empty, otherwise some result is wrong:
+ SELECT idx, gtid_pos, BINLOG_GTID_POS(file, pos)
+ FROM gtid_data
+ WHERE NOT gtid_eq(CONVERT(gtid_pos USING utf8),BINLOG_GTID_POS(file, pos))
+ ORDER BY idx;
+}
# Prepare to rewind the slave to this point to test again on same binlog.
--connection slave
@@ -107,7 +109,7 @@ while ($i < $NUM_POS) {
--disable_result_log
eval START SLAVE UNTIL master_gtid_pos='$gtid_pos';
--enable_result_log
- --let $res= `SELECT MASTER_GTID_WAIT('$gtid_pos')`
+ --let $res= `SELECT MASTER_GTID_WAIT('$gtid_pos', 60)`
if ($res != 0) {
--die "FAIL: MASTER_GTID_WAIT($gtid_pos) returned $res, should have been 0"
}
@@ -152,7 +154,7 @@ while ($i <= $NUM_DOMAIN) {
--disable_result_log
eval START SLAVE UNTIL master_gtid_pos='$until_pos';
--enable_result_log
- --let $res= `SELECT MASTER_GTID_WAIT('$until_pos')`
+ --let $res= `SELECT MASTER_GTID_WAIT('$until_pos', 60)`
if ($res != 0) {
--die "FAIL: MASTER_GTID_WAIT($until_pos) returned $res, should have been 0"
}
diff --git a/mysql-test/suite/rpl/include/rpl_gtid_index_cleanup.inc b/mysql-test/suite/rpl/include/rpl_gtid_index_cleanup.inc
new file mode 100644
index 0000000000000..dbafdfc3dc2a3
--- /dev/null
+++ b/mysql-test/suite/rpl/include/rpl_gtid_index_cleanup.inc
@@ -0,0 +1,2 @@
+DROP TABLE t1;
+DROP FUNCTION gtid_eq;
diff --git a/mysql-test/suite/rpl/include/rpl_gtid_index_setup.inc b/mysql-test/suite/rpl/include/rpl_gtid_index_setup.inc
new file mode 100644
index 0000000000000..f37e391229327
--- /dev/null
+++ b/mysql-test/suite/rpl/include/rpl_gtid_index_setup.inc
@@ -0,0 +1,32 @@
+--connection slave
+--source include/stop_slave.inc
+CHANGE MASTER TO master_use_gtid= slave_pos;
+--source include/start_slave.inc
+
+--connection master
+CREATE TABLE t1 (a INT PRIMARY KEY, b INT) ENGINE=InnoDB;
+INSERT INTO t1 VALUES (0, 0);
+
+--echo *** Test looking up a lot of different event positions and GTIDs.
+
+# A function for comparing GTID positions.
+# Handles that the domain_id order is different in the two strings.
+# Works by repeatedly removing one GTID from each string. If the strings have
+# the same length and nothing is left at the end, then they are identical.
+delimiter //;
+CREATE FUNCTION gtid_eq(a VARCHAR(255), b VARCHAR(255)) RETURNS BOOLEAN DETERMINISTIC
+BEGIN
+ DECLARE g VARCHAR(255);
+ IF a IS NULL OR b IS NULL OR LENGTH(a) != LENGTH(b) THEN
+ RETURN FALSE;
+ END IF;
+ SET a= CONCAT(a, ',');
+ SET b= CONCAT(',', b, ',');
+ WHILE LENGTH(a) > 0 DO
+ SET g= REGEXP_SUBSTR(a, '^[^,]+,');
+ SET a= SUBSTRING(a, LENGTH(g)+1);
+ SET b= REPLACE(b, CONCAT(',', g), ',');
+ END WHILE;
+ RETURN b = ',';
+END //
+delimiter ;//
diff --git a/mysql-test/suite/rpl/include/rpl_gtid_until_before_after_gtids.test b/mysql-test/suite/rpl/include/rpl_gtid_until_before_after_gtids.test
index 4f907d0b012cd..dc917448c5f50 100644
--- a/mysql-test/suite/rpl/include/rpl_gtid_until_before_after_gtids.test
+++ b/mysql-test/suite/rpl/include/rpl_gtid_until_before_after_gtids.test
@@ -33,6 +33,7 @@ if (!$ssu_before_gtids)
--source include/stop_slave.inc
--source include/reset_slave.inc
--connection master
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
set session gtid_domain_id=0;
diff --git a/mysql-test/suite/rpl/include/rpl_loaddata.test b/mysql-test/suite/rpl/include/rpl_loaddata.test
index 0a6d03aa8371c..a841d0f123157 100644
--- a/mysql-test/suite/rpl/include/rpl_loaddata.test
+++ b/mysql-test/suite/rpl/include/rpl_loaddata.test
@@ -115,6 +115,7 @@ stop slave;
# Finally, see if logging is done ok on master for a failing LOAD DATA INFILE
connection master;
+--source include/kill_binlog_dump_threads.inc
reset master;
eval create table t2 (day date,id int(9),category enum('a','b','c'),name varchar(60),
unique(day)) engine=$engine_type; # no transactions
diff --git a/mysql-test/suite/rpl/include/rpl_parallel_29322.inc b/mysql-test/suite/rpl/include/rpl_parallel_29322.inc
index a8b729d0cb512..3ef5fdb5c005a 100644
--- a/mysql-test/suite/rpl/include/rpl_parallel_29322.inc
+++ b/mysql-test/suite/rpl/include/rpl_parallel_29322.inc
@@ -25,6 +25,7 @@
--connection master
--let $datadir= `SELECT @@datadir`
+--source include/kill_binlog_dump_threads.inc
reset master; # Master starts a new serious of binlog files
create table t2 (a timestamp, b int);
diff --git a/mysql-test/suite/rpl/include/rpl_row_basic.test b/mysql-test/suite/rpl/include/rpl_row_basic.test
index 33d5dd328f3c6..4db3efd806f31 100644
--- a/mysql-test/suite/rpl/include/rpl_row_basic.test
+++ b/mysql-test/suite/rpl/include/rpl_row_basic.test
@@ -18,7 +18,10 @@ let $slave_com_update_before= query_get_value(SHOW GLOBAL STATUS LIKE 'com_updat
connection master;
eval CREATE TABLE t1 (C1 CHAR(1), C2 CHAR(1), INDEX (C1)$extra_index_t1) ENGINE = $type ;
SELECT * FROM t1;
-sync_slave_with_master;
+--connection master
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
SELECT * FROM t1;
# Testing insert
@@ -26,8 +29,11 @@ connection master;
INSERT INTO t1 VALUES ('A','B'), ('X','Y'), ('X','X');
INSERT INTO t1 VALUES ('A','C'), ('X','Z'), ('A','A');
SELECT * FROM t1 ORDER BY C1,C2;
-sync_slave_with_master;
SELECT * FROM t1 ORDER BY C1,C2;
+--connection master
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
# Testing delete
# Observe that are several rows having the value for the index but only one
@@ -35,7 +41,10 @@ SELECT * FROM t1 ORDER BY C1,C2;
connection master;
DELETE FROM t1 WHERE C1 = C2;
SELECT * FROM t1 ORDER BY C1,C2;
-sync_slave_with_master;
+--connection master
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
SELECT * FROM t1 ORDER BY C1,C2;
#
@@ -46,7 +55,10 @@ SELECT * FROM t1 ORDER BY C1,C2;
connection master;
UPDATE t1 SET C2 = 'I' WHERE C1 = 'A' AND C2 = 'C';
SELECT * FROM t1 ORDER BY C1,C2;
-sync_slave_with_master;
+--connection master
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
SELECT * FROM t1 ORDER BY C1,C2;
#BUG#12662190 - COM_COMMIT IS NOT INCREMENTED FROM THE BINARY LOGS ON SLAVE, COM_BEGIN IS
@@ -80,8 +92,11 @@ let $slave_com_update_after= query_get_value(SHOW GLOBAL STATUS LIKE 'com_update
connection master;
UPDATE t1 SET c2 = 'Q' WHERE c1 = 'A' AND c2 = 'N';
SELECT * FROM t1 ORDER BY c1,c2;
-sync_slave_with_master;
SELECT * FROM t1 ORDER BY c1,c2;
+--connection master
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
#
# Testing table with primary key
@@ -93,14 +108,20 @@ INSERT INTO t2
(6,'A',35), (7,'A',50), (8,'A',64), (9,'A',81);
SELECT * FROM t2 ORDER BY c1,c2;
SELECT * FROM t2 WHERE c2 = c1 * c1 ORDER BY c1,c2;
-sync_slave_with_master;
+--connection master
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
SELECT * FROM t2 ORDER BY c1,c2;
SELECT * FROM t2 WHERE c2 = c1 * c1 ORDER BY c1,c2;
connection master;
UPDATE t2 SET c2 = c1*c1 WHERE c2 != c1*c1;
SELECT * FROM t2 WHERE c2 = c1 * c1 ORDER BY c1,c2;
-sync_slave_with_master;
+--connection master
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
SELECT * FROM t2 WHERE c2 = c1 * c1 ORDER BY c1,c2;
# Testing update with a condition that does not match any rows, but
@@ -108,13 +129,19 @@ SELECT * FROM t2 WHERE c2 = c1 * c1 ORDER BY c1,c2;
connection master;
UPDATE t2 SET c12 = 'Q' WHERE c1 = 1 AND c2 = 999;
SELECT * FROM t2 ORDER BY c1,c2;
-sync_slave_with_master;
+--connection master
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
SELECT * FROM t2 ORDER BY c1,c2;
connection master;
DELETE FROM t2 WHERE c1 % 4 = 0;
SELECT * FROM t2 ORDER BY c1,c2;
-sync_slave_with_master;
+--connection master
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
SELECT * FROM t2 ORDER BY c1,c2;
connection master;
@@ -129,19 +156,28 @@ eval CREATE TABLE t3 (C1 CHAR(1), C2 CHAR(1), pk1 INT, C3 CHAR(1), pk2 INT, PRIM
INSERT INTO t3 VALUES ('A','B',1,'B',1), ('X','Y',2,'B',1), ('X','X',3,'B',1);
INSERT INTO t3 VALUES ('A','C',1,'B',2), ('X','Z',2,'B',2), ('A','A',3,'B',2);
SELECT * FROM t3 ORDER BY C1,C2;
-sync_slave_with_master;
+--connection master
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
SELECT * FROM t3 ORDER BY C1,C2;
connection master;
DELETE FROM t3 WHERE C1 = C2;
SELECT * FROM t3 ORDER BY C1,C2;
-sync_slave_with_master;
+--connection master
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
SELECT * FROM t3 ORDER BY C1,C2;
connection master;
UPDATE t3 SET C2 = 'I' WHERE C1 = 'A' AND C2 = 'C';
SELECT * FROM t3 ORDER BY C1,C2;
-sync_slave_with_master;
+--connection master
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
SELECT * FROM t3 ORDER BY C1,C2;
#
@@ -154,7 +190,10 @@ eval CREATE TABLE t6 (C1 CHAR(1), C2 CHAR(1), C3 INT$extra_index_t6) ENGINE = $t
INSERT INTO t6 VALUES ('A','B',1), ('X','Y',2), ('X','X',3);
INSERT INTO t6 VALUES ('A','C',4), ('X','Z',5), ('A','A',6);
SELECT * FROM t6 ORDER BY C3;
-sync_slave_with_master;
+--connection master
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
SELECT * FROM t6 ORDER BY C3;
# Testing delete
@@ -163,7 +202,10 @@ SELECT * FROM t6 ORDER BY C3;
connection master;
DELETE FROM t6 WHERE C1 = C2;
SELECT * FROM t6 ORDER BY C3;
-sync_slave_with_master;
+--connection master
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
SELECT * FROM t6 ORDER BY C3;
#
@@ -174,7 +216,10 @@ SELECT * FROM t6 ORDER BY C3;
connection master;
UPDATE t6 SET C2 = 'I' WHERE C1 = 'A' AND C2 = 'C';
SELECT * FROM t6 ORDER BY C3;
-sync_slave_with_master;
+--connection master
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
SELECT * FROM t6 ORDER BY C3;
# now mixing the 3 tables without begin/commit
@@ -185,7 +230,10 @@ INSERT INTO t5 VALUES ('A','C',4), ('X','Z',5), ('A','A',6);
UPDATE t5,t2,t3 SET t5.C2='Q', t2.c12='R', t3.C3 ='S' WHERE t5.C1 = t2.c12 AND t5.C1 = t3.C1;
SELECT * FROM t5,t2,t3 WHERE t5.C2='Q' AND t2.c12='R' AND t3.C3 ='S' ORDER BY t5.C3,t2.c1,t3.pk1,t3.pk2;
-sync_slave_with_master;
+--connection master
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
SELECT * FROM t5,t2,t3 WHERE t5.C2='Q' AND t2.c12='R' AND t3.C3 ='S' ORDER BY t5.C3,t2.c1,t3.pk1,t3.pk2;
#
@@ -203,7 +251,10 @@ eval CREATE TABLE t4 (C1 CHAR(1) PRIMARY KEY, B1 BIT(1), B2 BIT(1) NOT NULL DEFA
INSERT INTO t4 SET C1 = 1;
SELECT C1,HEX(B1),HEX(B2) FROM t4 ORDER BY C1;
-sync_slave_with_master;
+--connection master
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
SELECT C1,HEX(B1),HEX(B2) FROM t4 ORDER BY C1;
if (`select char_length('$bit_field_special') > 0`) {
@@ -215,7 +266,10 @@ if (`select char_length('$bit_field_special') > 0`) {
#
connection master;
eval CREATE TABLE t7 (C1 INT PRIMARY KEY, C2 INT) ENGINE = $type ;
-sync_slave_with_master;
+--connection master
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
--echo --- original values ---
INSERT INTO t7 VALUES (1,3), (2,6), (3,9);
SELECT * FROM t7 ORDER BY C1;
@@ -229,7 +283,10 @@ connection master;
--echo --- new values inserted ---
INSERT INTO t7 VALUES (1,2), (2,4), (3,6);
SELECT * FROM t7 ORDER BY C1;
-sync_slave_with_master;
+--connection master
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
set @@global.slave_exec_mode= default;
--echo --- old values should be overwritten by replicated values ---
@@ -252,7 +309,10 @@ INSERT INTO t8 VALUES (11,99,33);
INSERT INTO t8 VALUES (11,22,99);
SELECT * FROM t8 ORDER BY a;
-sync_slave_with_master;
+--connection master
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
SELECT * FROM t8 ORDER BY a;
INSERT INTO t8 VALUES (1,2,3), (2,4,6), (3,6,9);
SELECT * FROM t8 ORDER BY a;
@@ -266,7 +326,10 @@ connection master;
# We insert a row that will cause conflict on the primary key but not
# on the other keys.
INSERT INTO t8 VALUES (2,4,8);
-sync_slave_with_master;
+--connection master
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
set @@global.slave_exec_mode= default;
SELECT * FROM t8 ORDER BY a;
@@ -279,14 +342,20 @@ SELECT * FROM t8 ORDER BY a;
# Clean up t1 so that we can use it.
connection master;
DELETE FROM t1;
-sync_slave_with_master;
+--connection master
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
# Just to get a clean binary log
--source include/rpl_reset.inc
connection master;
INSERT INTO t1 VALUES ('K','K'), ('L','L'), ('M','M');
-sync_slave_with_master;
+--connection master
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
# since bug#31552/31609 idempotency is not default any longer. In order
# the following test DELETE FROM t1 to pass the mode is switched
# temprorarily
@@ -296,7 +365,10 @@ DELETE FROM t1 WHERE C1 = 'L';
connection master;
DELETE FROM t1;
query_vertical SELECT COUNT(*) FROM t1 ORDER BY c1,c2;
-sync_slave_with_master;
+--connection master
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
set @@global.slave_exec_mode= default;
source include/check_slave_is_running.inc;
query_vertical SELECT COUNT(*) FROM t1 ORDER BY c1,c2;
@@ -312,7 +384,10 @@ CREATE TABLE t1 (a TIMESTAMP, b DATETIME, c DATE);
INSERT INTO t1 VALUES(
'2005-11-14 01:01:01', '2005-11-14 01:01:02', '2005-11-14');
-sync_slave_with_master slave;
+--connection master
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
SELECT * FROM t1;
#
@@ -321,7 +396,10 @@ SELECT * FROM t1;
connection master;
DROP TABLE IF EXISTS t1,t2,t3,t4,t5,t6,t7,t8;
-sync_slave_with_master;
+--connection master
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
#
# BUG#37426: RBR breaks for CHAR() UTF8 fields > 85 chars
@@ -351,14 +429,20 @@ eval CREATE TABLE t2 (i INT NOT NULL,
c CHAR(16) CHARACTER SET utf8 NOT NULL,
j INT NOT NULL) ENGINE = $type ;
-sync_slave_with_master;
+--connection master
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
ALTER TABLE t2 MODIFY c CHAR(128) CHARACTER SET utf8 NOT NULL;
connection master;
eval CREATE TABLE t3 (i INT NOT NULL,
c CHAR(128) CHARACTER SET utf8 NOT NULL,
j INT NOT NULL) ENGINE = $type ;
-sync_slave_with_master;
+--connection master
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
ALTER TABLE t3 MODIFY c CHAR(16) CHARACTER SET utf8 NOT NULL;
connection master;
@@ -369,14 +453,20 @@ eval CREATE TABLE t4 (i INT NOT NULL,
eval CREATE TABLE t5 (i INT NOT NULL,
c CHAR(255) CHARACTER SET utf8 NOT NULL,
j INT NOT NULL) ENGINE = $type ;
-sync_slave_with_master;
+--connection master
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
ALTER TABLE t5 MODIFY c CHAR(16) CHARACTER SET utf8 NOT NULL;
connection master;
eval CREATE TABLE t6 (i INT NOT NULL,
c CHAR(255) CHARACTER SET utf8 NOT NULL,
j INT NOT NULL) ENGINE = $type ;
-sync_slave_with_master;
+--connection master
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
ALTER TABLE t6 MODIFY c CHAR(128) CHARACTER SET utf8 NOT NULL;
connection master;
@@ -392,7 +482,10 @@ SET GLOBAL SLAVE_TYPE_CONVERSIONS = 'ALL_NON_LOSSY';
connection master;
INSERT INTO t1 VALUES (1, "", 1);
INSERT INTO t1 VALUES (2, repeat(_utf8'a', 16), 2);
-sync_slave_with_master;
+--connection master
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
let $diff_tables= master:t1, slave:t1;
source include/diff_tables.inc;
@@ -401,7 +494,10 @@ source include/diff_tables.inc;
connection master;
INSERT INTO t2 VALUES (1, "", 1);
INSERT INTO t2 VALUES (2, repeat(_utf8'a', 16), 2);
-sync_slave_with_master;
+--connection master
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
let $diff_tables= master:t2, slave:t2;
source include/diff_tables.inc;
@@ -420,7 +516,10 @@ call mtr.add_suppression("Can't find record in 't1'");
connection master;
INSERT INTO t4 VALUES (1, "", 1);
INSERT INTO t4 VALUES (2, repeat(_utf8'a', 128), 2);
-sync_slave_with_master;
+--connection master
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
let $diff_tables= master:t4, slave:t4;
source include/diff_tables.inc;
@@ -436,7 +535,9 @@ connection slave;
--let $show_slave_sql_error= 1
--source include/wait_for_slave_sql_error.inc
+--let $rpl_skip_sync=1
--source include/rpl_reset.inc
+--let $rpl_skip_sync=
--echo [expecting slave to stop]
connection master;
@@ -449,20 +550,28 @@ connection slave;
--let $show_slave_sql_error= 1
--source include/wait_for_slave_sql_error.inc
+--let $rpl_skip_sync=1
--source include/rpl_reset.inc
+--let $rpl_skip_sync=
--echo [expecting slave to replicate correctly]
connection master;
INSERT INTO t7 VALUES (1, "", 1);
INSERT INTO t7 VALUES (2, repeat(_utf8'a', 255), 2);
-sync_slave_with_master;
+--connection master
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
let $diff_tables= master:t7, slave:t7;
source include/diff_tables.inc;
connection master;
drop table t1, t2, t3, t4, t5, t6, t7;
-sync_slave_with_master;
+--connection master
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
#
# BUG#32709: Assertion failed: trx_data->empty(), file .\log.cc, line 1293
@@ -475,14 +584,20 @@ INSERT INTO t1 VALUES (1), (2), (3);
--error ER_DUP_ENTRY
UPDATE t1 SET a = 10;
INSERT INTO t1 VALUES (4);
-sync_slave_with_master;
+--connection master
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
let $diff_tables= master:t1, slave:t1;
source include/diff_tables.inc;
connection master;
drop table t1;
-sync_slave_with_master;
+--connection master
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
#
# Bug #38230 Differences between master and slave after
@@ -552,7 +667,10 @@ UPDATE t1 SET `pk` = 6 ORDER BY `int_key` LIMIT 6;
DELETE FROM t2 WHERE `pk` < 7 LIMIT 1;
UPDATE t1 SET `int_key` = 4 ORDER BY `pk` LIMIT 6;
---sync_slave_with_master
+--connection master
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
--echo *** results: t2 must be consistent ****
let $diff_tables= master:t2, slave:t2;
@@ -591,14 +709,20 @@ INSERT INTO t1 ( a ) VALUES ( 3 );
UPDATE t1 SET a = 0 WHERE a < 4;
UPDATE t1 SET a = 8 WHERE a < 5;
-sync_slave_with_master;
+--connection master
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
let $diff_tables= master:t1, slave:t1;
source include/diff_tables.inc;
connection master;
drop table t1;
-sync_slave_with_master;
+--connection master
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
#
# Bug #39752: Replication failure on RBR + MyISAM + no PK
@@ -661,7 +785,10 @@ INSERT INTO t1 ( a ) VALUES ( 1 );
UPDATE IGNORE t1 SET a = 9 WHERE a < 5 LIMIT 3;
--enable_warnings
-sync_slave_with_master;
+--connection master
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
if (`select char_length('$bit_field_special') > 0`) {
SET GLOBAL SLAVE_TYPE_CONVERSIONS = @saved_slave_type_conversions;
@@ -672,4 +799,7 @@ source include/diff_tables.inc;
connection master;
drop table t1;
-sync_slave_with_master;
+--connection master
+--source include/save_master_gtid.inc
+--connection slave
+--source include/sync_with_master_gtid.inc
diff --git a/mysql-test/suite/rpl/include/rpl_start_stop_slave.test b/mysql-test/suite/rpl/include/rpl_start_stop_slave.test
index 56b04494a7c3b..ded7743e313be 100644
--- a/mysql-test/suite/rpl/include/rpl_start_stop_slave.test
+++ b/mysql-test/suite/rpl/include/rpl_start_stop_slave.test
@@ -187,13 +187,15 @@ sync_slave_with_master;
# Testing how out of valid range position value is handled with an error.
#
+--connection slave
+--source include/stop_slave.inc
--connection master
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
let $master_pos= query_get_value(SHOW MASTER STATUS, Position, 1);
let $master_pos= `SELECT $master_pos + 1`;
--connection slave
---source include/stop_slave.inc
--replace_regex /[0-9]+/MASTER_POS/
eval CHANGE MASTER TO master_log_pos=$master_pos, master_use_gtid=no;
@@ -211,6 +213,7 @@ START SLAVE;
RESET SLAVE;
--connection master
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
# Slave is stopped by bug#3593869-64035 tests.
diff --git a/mysql-test/suite/rpl/include/sql_multisource.inc b/mysql-test/suite/rpl/include/sql_multisource.inc
index 761b68421d990..7e754560b5f70 100644
--- a/mysql-test/suite/rpl/include/sql_multisource.inc
+++ b/mysql-test/suite/rpl/include/sql_multisource.inc
@@ -1,41 +1,82 @@
# Populate the active connection server with events that come from varying
-# domain and server ids
+# domain and server ids.
+#
+# Parameters:
+# $blob_trx_size : Size of the blob to insert (used for binlog-in-innodb to
+# force out-of-band binlogging for events)
+#
--disable_query_log
+if (!$blob_trx_size)
+{
+ --let $blob_trx_size= 1
+}
+
+--connection $con1
# Save old state
let $ORIG_GTID_DOMAIN_ID = `select @@session.gtid_domain_id`;
let $ORIG_SERVER_ID = `select @@session.server_id`;
+--connection $con1
SET @@session.gtid_domain_id= 0;
SET @@session.server_id= 1;
-CREATE TABLE t1 (a int);
+CREATE TABLE t1 (a int, b longblob) engine=innodb;
SET @@session.server_id= 3;
-CREATE TABLE t2 (a int);
-INSERT INTO t2 values (3);
+CREATE TABLE t2 (a int, b longblob) engine=innodb;
+--connection server_1
SET @@session.gtid_domain_id= 1;
SET @@session.server_id= 1;
-CREATE TABLE t3 (a int);
-INSERT INTO t3 values (4);
+CREATE TABLE t3 (a int, b longblob) engine=innodb;
-SET @@session.server_id= 4;
-CREATE TABLE t4 (a int);
-INSERT INTO t4 values (5);
+# Concurrent transactions from different domains
+--connection $con1
+--send_eval INSERT INTO t2 values (3, repeat('b', $blob_trx_size))
+
+--connection server_1
+--send_eval INSERT INTO t3 values (4, repeat('c', $blob_trx_size))
+
+--connection server_1
+--reap
+--connection $con1
+--reap
+
+
+# Concurrent transactions from the same domain
+--connection $con1
SET @@session.gtid_domain_id= 0;
SET @@session.server_id= 1;
-INSERT INTO t1 values (1);
+BEGIN;
+--send_eval INSERT INTO t1 values (1, repeat('a1', $blob_trx_size))
-SET @@session.gtid_domain_id= 2;
+--connection server_1
+SET @@session.gtid_domain_id= 0;
SET @@session.server_id= 1;
-CREATE TABLE t5 (a int);
-INSERT INTO t5 values (6);
+BEGIN;
+--send_eval INSERT INTO t1 values (2, repeat('a2', $blob_trx_size))
-SET @@session.gtid_domain_id= 0;
+--connection $con1
+--reap
+--connection server_1
+--reap
+--connection $con1
+COMMIT;
+--connection server_1
+COMMIT;
+
+
+--connection server_1
+SET @@session.gtid_domain_id= 1;
+SET @@session.server_id= 4;
+CREATE TABLE t4 (a int, b longblob) engine=innodb;
+
+SET @@session.gtid_domain_id= 2;
SET @@session.server_id= 1;
-INSERT INTO t1 values (2);
+CREATE TABLE t5 (a int, b longblob) engine=innodb;
+INSERT INTO t5 values (6, 'e');
FLUSH LOGS;
diff --git a/mysql-test/suite/rpl/include/sql_out_of_order_gtid.inc b/mysql-test/suite/rpl/include/sql_out_of_order_gtid.inc
index 82291a887e954..0964209a78d32 100644
--- a/mysql-test/suite/rpl/include/sql_out_of_order_gtid.inc
+++ b/mysql-test/suite/rpl/include/sql_out_of_order_gtid.inc
@@ -1,39 +1,67 @@
# SQL file with out of order GTIDs coming from various domains and servers
+#
+# Parameters:
+# $blob_trx_size : Size of the blob to insert (used for binlog-in-innodb to
+# force out-of-band binlogging for events)
+#
--disable_query_log
+if (!$blob_trx_size)
+{
+ --let $blob_trx_size= 1
+}
+
+--connection $con1
# Save old state
let $ORIG_GTID_DOMAIN_ID = `select @@session.gtid_domain_id`;
let $ORIG_SERVER_ID = `select @@session.server_id`;
+--connection $con1
SET @@session.gtid_domain_id= 0;
SET @@session.server_id= 1;
-CREATE TABLE t1 (a int);
-INSERT INTO t1 values (1);
+CREATE TABLE t1 (a int, b longblob) engine=innodb;
+INSERT INTO t1 values (1, 'a1');
SET @@session.server_id= 3;
-CREATE TABLE t2 (a int);
+CREATE TABLE t2 (a int, b longblob) engine=innodb;
SET @@session.gtid_seq_no= 6;
-INSERT INTO t2 values (2);
-SET @@session.gtid_seq_no= 5;
-INSERT INTO t2 values (1);
-SET @@session.gtid_seq_no= 7;
-INSERT INTO t2 values (3);
+--send_eval INSERT INTO t2 values (2, repeat('b2', $blob_trx_size))
+
+--connection server_1
SET @@session.gtid_domain_id= 1;
SET @@session.server_id= 1;
-CREATE TABLE t3 (a int);
-INSERT INTO t3 values (1);
+CREATE TABLE t3 (a int, b longblob) engine=innodb;
+INSERT INTO t3 values (1, 'c1');
SET @@session.gtid_seq_no= 4;
-INSERT INTO t3 values (3);
+--send_eval INSERT INTO t3 values (3, repeat('c3', $blob_trx_size))
+
+
+--connection $con1
+--reap
+SET @@session.gtid_seq_no= 5;
+--send_eval INSERT INTO t2 values (1, repeat('b1', $blob_trx_size))
+
+
+--connection server_1
+--reap
SET @@session.gtid_seq_no= 3;
-INSERT INTO t3 values (2);
+--eval INSERT INTO t3 values (2, repeat('c2', $blob_trx_size))
+
SET @@session.gtid_seq_no= 5;
-INSERT INTO t3 values (4);
+INSERT INTO t3 values (4,'c4');
+
+
+--connection $con1
+--reap
+SET @@session.gtid_seq_no= 7;
+BEGIN;
+INSERT INTO t2 values (3, 'b3');
-SET @@session.gtid_domain_id= 0;
SET @@session.server_id= 1;
-INSERT INTO t1 values (2);
+BEGIN;
+INSERT INTO t1 values (2, 'a2');
FLUSH LOGS;
diff --git a/mysql-test/suite/rpl/r/rpl_binlog_directory.result b/mysql-test/suite/rpl/r/rpl_binlog_directory.result
new file mode 100644
index 0000000000000..0a52b385b839e
--- /dev/null
+++ b/mysql-test/suite/rpl/r/rpl_binlog_directory.result
@@ -0,0 +1,52 @@
+include/master-slave.inc
+[connection master]
+*** Test the --binlog-directory variable with legacy binlog.
+connection slave;
+include/stop_slave.inc
+CHANGE MASTER TO master_use_gtid=Slave_pos;
+connection master;
+SELECT @@GLOBAL.binlog_storage_engine;
+@@GLOBAL.binlog_storage_engine
+NULL
+CREATE TABLE t1 (a INT PRIMARY KEY, b INT);
+INSERT INTO t1 VALUES (1, 10);
+include/rpl_stop_server.inc [server_number=1]
+include/rpl_start_server.inc [server_number=1 parameters: --binlog-directory=binlog_dir]
+INSERT INTO t1 VALUES (2, 11);
+include/rpl_stop_server.inc [server_number=1]
+include/rpl_start_server.inc [server_number=1]
+INSERT INTO t1 VALUES (3, 12);
+include/save_master_gtid.inc
+show binary logs;
+Log_name File_size
+master-bin.000001 #
+master-bin.000002 #
+master-bin.000002 #
+*** Contents of master-bin.index (including directory path):
+./master-bin.000001
+binlog_dir/master-bin.000002
+./master-bin.000002
+SELECT * FROM t1 ORDER BY a;
+a b
+1 10
+2 11
+3 12
+connection slave;
+include/start_slave.inc
+include/sync_with_master_gtid.inc
+SELECT * FROM t1 ORDER BY a;
+a b
+1 10
+2 11
+3 12
+connection slave;
+include/stop_slave.inc
+SET GLOBAL gtid_slave_pos= '';
+connection master;
+include/kill_binlog_dump_threads.inc
+RESET MASTER;
+connection slave;
+include/start_slave.inc
+connection master;
+DROP TABLE t1;
+include/rpl_end.inc
diff --git a/mysql-test/suite/rpl/r/rpl_binlog_errors.result b/mysql-test/suite/rpl/r/rpl_binlog_errors.result
index 442b2a5da1d55..dddf26aa0b91b 100644
--- a/mysql-test/suite/rpl/r/rpl_binlog_errors.result
+++ b/mysql-test/suite/rpl/r/rpl_binlog_errors.result
@@ -13,6 +13,7 @@ call mtr.add_suppression("Could not use master-bin for logging");
SET @old_debug= @@global.debug_dbug;
SELECT repeat('x',8192) INTO OUTFILE 'MYSQLTEST_VARDIR/tmp/bug_46166.data';
SELECT repeat('x',10) INTO OUTFILE 'MYSQLTEST_VARDIR/tmp/bug_46166-2.data';
+include/kill_binlog_dump_threads.inc
RESET MASTER;
###################### TEST #1
FLUSH LOGS;
@@ -273,5 +274,6 @@ Note 1255 Slave already has been stopped
RESET SLAVE;
Warnings:
Note 4190 RESET SLAVE is implicitly changing the value of 'Using_Gtid' from 'No' to 'Slave_Pos'
+include/kill_binlog_dump_threads.inc
RESET MASTER;
include/rpl_end.inc
diff --git a/mysql-test/suite/rpl/r/rpl_blackhole_row_annotate.result b/mysql-test/suite/rpl/r/rpl_blackhole_row_annotate.result
index 33f479ba2c1d3..5df65903a9bb3 100644
--- a/mysql-test/suite/rpl/r/rpl_blackhole_row_annotate.result
+++ b/mysql-test/suite/rpl/r/rpl_blackhole_row_annotate.result
@@ -1,8 +1,13 @@
include/master-slave.inc
[connection master]
+connection slave;
+include/stop_slave.inc
+connection master;
+include/kill_binlog_dump_threads.inc
SET timestamp=1000000000;
RESET MASTER;
connection slave;
+include/start_slave.inc
SET timestamp=1000000000;
RESET MASTER;
connection master;
diff --git a/mysql-test/suite/rpl/r/rpl_bug41902.result b/mysql-test/suite/rpl/r/rpl_bug41902.result
index 7d676ea73e3ce..81187a39356d8 100644
--- a/mysql-test/suite/rpl/r/rpl_bug41902.result
+++ b/mysql-test/suite/rpl/r/rpl_bug41902.result
@@ -20,6 +20,7 @@ SET @@debug_dbug="";
reset slave;
change master to master_host='dummy';
connection master;
+include/kill_binlog_dump_threads.inc
SET @saved_dbug_m = @@global.debug_dbug;
SET @@debug_dbug="d,simulate_find_log_pos_error";
reset master;
diff --git a/mysql-test/suite/rpl/r/rpl_change_master_demote.result b/mysql-test/suite/rpl/r/rpl_change_master_demote.result
index a1ea32e9b1007..29c71f57bb8eb 100644
--- a/mysql-test/suite/rpl/r/rpl_change_master_demote.result
+++ b/mysql-test/suite/rpl/r/rpl_change_master_demote.result
@@ -65,11 +65,15 @@ connection master;
#
connection slave;
include/stop_slave.inc
+include/kill_binlog_dump_threads.inc
RESET MASTER;
set @@global.gtid_slave_pos="";
-include/start_slave.inc
connection master;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
+connection slave;
+include/start_slave.inc
+connection master;
set @@global.gtid_slave_pos="";
set session gtid_domain_id= 0;
CREATE TABLE t1 (a int);
@@ -689,6 +693,7 @@ ERROR 42000: You have an error in your SQL syntax; check the manual that corresp
# Ensure MASTER_DEMOTE_TO_REPLICA aliases MASTER_DEMOTE_TO_SLAVE
#
connection slave;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
include/reset_slave.inc
CREATE TABLE t_mdev_31768 (a int);
@@ -700,6 +705,7 @@ RESET MASTER;
include/reset_slave.inc
# Clear primary binlog state to match replica
connection master;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
#
# Cleanup
diff --git a/mysql-test/suite/rpl/r/rpl_checksum.result b/mysql-test/suite/rpl/r/rpl_checksum.result
index 39ebb85b9e94a..8ebaea3ddc156 100644
--- a/mysql-test/suite/rpl/r/rpl_checksum.result
+++ b/mysql-test/suite/rpl/r/rpl_checksum.result
@@ -134,6 +134,7 @@ set @@global.binlog_checksum= IF(floor((rand()*1000)%2), "CRC32", "NONE");
flush logs;
connection master;
set @@global.binlog_checksum= CRC32;
+include/kill_binlog_dump_threads.inc
reset master;
flush logs;
create table t3 (a int, b char(5));
@@ -148,6 +149,7 @@ include/stop_slave.inc
change master to master_host='127.0.0.1',master_port=MASTER_PORT, master_user='root';
connection master;
flush logs;
+include/kill_binlog_dump_threads.inc
reset master;
insert into t3 value (1, @@global.binlog_checksum);
connection slave;
diff --git a/mysql-test/suite/rpl/r/rpl_circular_for_4_hosts.result b/mysql-test/suite/rpl/r/rpl_circular_for_4_hosts.result
index 85cc0cc7ebc83..ea1932f41edd0 100644
--- a/mysql-test/suite/rpl/r/rpl_circular_for_4_hosts.result
+++ b/mysql-test/suite/rpl/r/rpl_circular_for_4_hosts.result
@@ -181,6 +181,7 @@ DELETE FROM t1 WHERE a = 6;
include/start_slave.inc
connection server_2;
connection server_3;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
connection server_4;
RESET SLAVE;
diff --git a/mysql-test/suite/rpl/r/rpl_circular_semi_sync.result b/mysql-test/suite/rpl/r/rpl_circular_semi_sync.result
index 9dd4aed2d29ee..8c762a591ecb7 100644
--- a/mysql-test/suite/rpl/r/rpl_circular_semi_sync.result
+++ b/mysql-test/suite/rpl/r/rpl_circular_semi_sync.result
@@ -6,11 +6,13 @@ call mtr.add_suppression("Timeout waiting for reply of binlog");
connection server_2;
include/stop_slave.inc
connection server_1;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
set @@session.gtid_domain_id=10;
set @@global.rpl_semi_sync_master_enabled = 1;
set @@global.rpl_semi_sync_master_wait_point=AFTER_SYNC;
connection server_2;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
ALTER TABLE mysql.gtid_slave_pos ENGINE=InnoDB;
set @@session.gtid_domain_id=20;
diff --git a/mysql-test/suite/rpl/r/rpl_colSize.result b/mysql-test/suite/rpl/r/rpl_colSize.result
index 95a30c6ee4d34..adb64d1350237 100644
--- a/mysql-test/suite/rpl/r/rpl_colSize.result
+++ b/mysql-test/suite/rpl/r/rpl_colSize.result
@@ -53,6 +53,7 @@ r TEXT,
s MEDIUMTEXT,
t LONGTEXT
);
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Start replication ***
connection slave;
diff --git a/mysql-test/suite/rpl/r/rpl_crash_ddl_master.result b/mysql-test/suite/rpl/r/rpl_crash_ddl_master.result
new file mode 100644
index 0000000000000..e340f220a838a
--- /dev/null
+++ b/mysql-test/suite/rpl/r/rpl_crash_ddl_master.result
@@ -0,0 +1,49 @@
+include/master-slave.inc
+[connection master]
+CREATE TABLE t1 (a INT PRIMARY KEY, b INT, c INT) ENGINE=InnoDB;
+CREATE TABLE t2 LIKE t1;
+INSERT INTO t1 VALUES (1, 0, 0), (2, 3, 1), (3, 2, 5), (4, 10, 7), (5, 6, 5);
+SET SESSION gtid_seq_no= 99;
+INSERT INTO t2 SELECT * FROM t1;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+include/stop_slave.inc
+SET @old_parallel= @@GLOBAL.slave_parallel_threads;
+SET GLOBAL slave_parallel_threads=8;
+SET @old_mode= @@GLOBAL.slave_parallel_mode;
+SET GLOBAL slave_parallel_mode= optimistic;
+SET @old_strict= @@GLOBAL.gtid_strict_mode;
+SET GLOBAL gtid_strict_mode= 1;
+connection master;
+SET @debug_crash_counter= 1;
+SET SESSION debug_dbug= '+d,ddl_log_alter_after_log';
+ALTER TABLE t1 ORDER BY b;
+Got one of the listed errors
+connection master1;
+connection server_1;
+connection default;
+connection master;
+include/rpl_reconnect.inc
+INSERT INTO t2 VALUES (10, 10, 10);
+UPDATE t2 SET c=c+1 WHERE a=4;
+UPDATE t2 SET c=20 WHERE a=1;
+ALTER TABLE t2 ORDER BY b;
+Warnings:
+Warning 1105 ORDER BY ignored as there is a user-defined clustered index in the table 't2'
+include/save_master_gtid.inc
+connection slave;
+SET @old_dbug= @@GLOBAL.debug_dbug;
+SET GLOBAL debug_dbug= '+d,rpl_parallel_delay_gtid_0_x_100_write';
+include/start_slave.inc
+include/sync_with_master_gtid.inc
+SET GLOBAL debug_dbug= @old_dbug;
+connection slave;
+include/stop_slave.inc
+SET GLOBAL slave_parallel_threads= @old_parallel;
+SET GLOBAL slave_parallel_mode= @old_mode;
+SET GLOBAL gtid_strict_mode= @old_strict;
+include/start_slave.inc
+connection master;
+DROP TABLE t1, t2;
+include/rpl_end.inc
diff --git a/mysql-test/suite/rpl/r/rpl_ddl.result b/mysql-test/suite/rpl/r/rpl_ddl.result
index 27d84fea1e476..7038ea8250eab 100644
--- a/mysql-test/suite/rpl/r/rpl_ddl.result
+++ b/mysql-test/suite/rpl/r/rpl_ddl.result
@@ -2,14 +2,19 @@ include/master-slave.inc
[connection master]
set global sql_mode='';
set local sql_mode='';
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection master;
SET AUTOCOMMIT = 1;
CREATE DATABASE mysqltest1;
CREATE DATABASE mysqltest2;
CREATE TABLE mysqltest1.t1 (f1 BIGINT) ENGINE=InnoDB;
+connection master;
+include/save_master_gtid.inc
connection slave;
-connection slave;
+include/sync_with_master_gtid.inc
connection master;
INSERT INTO mysqltest1.t1 SET f1= 0;
CREATE TABLE mysqltest1.t2 (f1 BIGINT) ENGINE=InnoDB;
@@ -35,7 +40,10 @@ CREATE TABLE mysqltest1.t19 (f1 BIGINT) ENGINE=InnoDB;
CREATE TEMPORARY TABLE mysqltest1.t23 (f1 BIGINT) ENGINE=MEMORY;
SET AUTOCOMMIT = 0;
use mysqltest1;
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SET AUTOCOMMIT = 1;
use mysqltest1;
@@ -47,7 +55,10 @@ INSERT INTO t1 SET f1= 0 + 1;
SELECT MAX(f1) FROM t1;
MAX(f1)
1
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -59,7 +70,10 @@ SELECT 1;
SELECT MAX(f1) FROM t1;
MAX(f1)
1
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -71,7 +85,10 @@ MAX(f1)
0
TEST-INFO: MASTER: The INSERT is not committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -86,7 +103,10 @@ INSERT INTO t1 SET f1= 0 + 1;
SELECT MAX(f1) FROM t1;
MAX(f1)
1
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -98,7 +118,10 @@ COUNT(*)
SELECT MAX(f1) FROM t1;
MAX(f1)
1
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -110,7 +133,10 @@ MAX(f1)
0
TEST-INFO: MASTER: The INSERT is not committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -125,7 +151,10 @@ INSERT INTO t1 SET f1= 0 + 1;
SELECT MAX(f1) FROM t1;
MAX(f1)
1
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -135,7 +164,10 @@ COMMIT;
SELECT MAX(f1) FROM t1;
MAX(f1)
1
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -147,7 +179,10 @@ MAX(f1)
1
TEST-INFO: MASTER: The INSERT is committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -162,7 +197,10 @@ INSERT INTO t1 SET f1= 1 + 1;
SELECT MAX(f1) FROM t1;
MAX(f1)
2
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -172,7 +210,10 @@ ROLLBACK;
SELECT MAX(f1) FROM t1;
MAX(f1)
1
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -184,7 +225,10 @@ MAX(f1)
1
TEST-INFO: MASTER: The INSERT is not committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -199,7 +243,10 @@ INSERT INTO t1 SET f1= 1 + 1;
SELECT MAX(f1) FROM t1;
MAX(f1)
2
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -209,7 +256,10 @@ SET AUTOCOMMIT=1;
SELECT MAX(f1) FROM t1;
MAX(f1)
2
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -221,7 +271,10 @@ MAX(f1)
2
TEST-INFO: MASTER: The INSERT is committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -237,7 +290,10 @@ INSERT INTO t1 SET f1= 2 + 1;
SELECT MAX(f1) FROM t1;
MAX(f1)
3
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -247,7 +303,10 @@ START TRANSACTION;
SELECT MAX(f1) FROM t1;
MAX(f1)
3
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -259,7 +318,10 @@ MAX(f1)
3
TEST-INFO: MASTER: The INSERT is committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -274,7 +336,10 @@ INSERT INTO t1 SET f1= 3 + 1;
SELECT MAX(f1) FROM t1;
MAX(f1)
4
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -284,7 +349,10 @@ BEGIN;
SELECT MAX(f1) FROM t1;
MAX(f1)
4
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -296,7 +364,10 @@ MAX(f1)
4
TEST-INFO: MASTER: The INSERT is committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -311,7 +382,10 @@ INSERT INTO t1 SET f1= 4 + 1;
SELECT MAX(f1) FROM t1;
MAX(f1)
5
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -321,7 +395,10 @@ DROP TABLE mysqltest1.t2;
SELECT MAX(f1) FROM t1;
MAX(f1)
5
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -333,7 +410,10 @@ MAX(f1)
5
TEST-INFO: MASTER: The INSERT is committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -354,7 +434,10 @@ INSERT INTO t1 SET f1= 5 + 1;
SELECT MAX(f1) FROM t1;
MAX(f1)
6
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -364,7 +447,10 @@ DROP TEMPORARY TABLE mysqltest1.t23;
SELECT MAX(f1) FROM t1;
MAX(f1)
6
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -376,7 +462,10 @@ MAX(f1)
5
TEST-INFO: MASTER: The INSERT is not committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -397,7 +486,10 @@ INSERT INTO t1 SET f1= 5 + 1;
SELECT MAX(f1) FROM t1;
MAX(f1)
6
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -407,7 +499,10 @@ RENAME TABLE mysqltest1.t3 to mysqltest1.t20;
SELECT MAX(f1) FROM t1;
MAX(f1)
6
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -419,7 +514,10 @@ MAX(f1)
6
TEST-INFO: MASTER: The INSERT is committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -442,7 +540,10 @@ INSERT INTO t1 SET f1= 6 + 1;
SELECT MAX(f1) FROM t1;
MAX(f1)
7
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -452,7 +553,10 @@ ALTER TABLE mysqltest1.t4 ADD column f2 BIGINT;
SELECT MAX(f1) FROM t1;
MAX(f1)
7
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -464,7 +568,10 @@ MAX(f1)
7
TEST-INFO: MASTER: The INSERT is committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -489,7 +596,10 @@ INSERT INTO t1 SET f1= 7 + 1;
SELECT MAX(f1) FROM t1;
MAX(f1)
8
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -499,7 +609,10 @@ CREATE TABLE mysqltest1.t21 (f1 BIGINT) ENGINE= InnoDB;
SELECT MAX(f1) FROM t1;
MAX(f1)
8
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -511,7 +624,10 @@ MAX(f1)
8
TEST-INFO: MASTER: The INSERT is committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -526,7 +642,10 @@ INSERT INTO t1 SET f1= 8 + 1;
SELECT MAX(f1) FROM t1;
MAX(f1)
9
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -536,7 +655,10 @@ CREATE TEMPORARY TABLE mysqltest1.t22 (f1 BIGINT) ENGINE=MEMORY;
SELECT MAX(f1) FROM t1;
MAX(f1)
9
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -548,7 +670,10 @@ MAX(f1)
8
TEST-INFO: MASTER: The INSERT is not committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -563,7 +688,10 @@ INSERT INTO t1 SET f1= 8 + 1;
SELECT MAX(f1) FROM t1;
MAX(f1)
9
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -573,7 +701,10 @@ TRUNCATE TABLE mysqltest1.t7;
SELECT MAX(f1) FROM t1;
MAX(f1)
9
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -585,7 +716,10 @@ MAX(f1)
9
TEST-INFO: MASTER: The INSERT is committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -595,7 +729,10 @@ TEST-INFO: SLAVE: The INSERT is committed (Succeeded)
connection master;
SELECT * FROM mysqltest1.t7;
f1
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT * FROM mysqltest1.t7;
f1
@@ -607,7 +744,10 @@ INSERT INTO t1 SET f1= 9 + 1;
SELECT MAX(f1) FROM t1;
MAX(f1)
10
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -617,7 +757,10 @@ LOCK TABLES mysqltest1.t1 WRITE, mysqltest1.t8 READ;
SELECT MAX(f1) FROM t1;
MAX(f1)
10
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -629,7 +772,10 @@ MAX(f1)
10
TEST-INFO: MASTER: The INSERT is committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -645,7 +791,10 @@ INSERT INTO t1 SET f1= 10 + 1;
SELECT MAX(f1) FROM t1;
MAX(f1)
11
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -655,7 +804,10 @@ UNLOCK TABLES;
SELECT MAX(f1) FROM t1;
MAX(f1)
11
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -667,7 +819,10 @@ MAX(f1)
10
TEST-INFO: MASTER: The INSERT is not committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -684,7 +839,10 @@ ERROR HY000: Table 't1' was locked with a READ lock and can't be updated
SELECT MAX(f1) FROM t1;
MAX(f1)
10
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -694,7 +852,10 @@ UNLOCK TABLES;
SELECT MAX(f1) FROM t1;
MAX(f1)
10
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -706,7 +867,10 @@ MAX(f1)
10
TEST-INFO: MASTER: The INSERT is not committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -722,7 +886,10 @@ INSERT INTO t1 SET f1= 10 + 1;
SELECT MAX(f1) FROM t1;
MAX(f1)
11
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -732,7 +899,10 @@ UNLOCK TABLES;
SELECT MAX(f1) FROM t1;
MAX(f1)
11
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -744,7 +914,10 @@ MAX(f1)
11
TEST-INFO: MASTER: The INSERT is committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -759,7 +932,10 @@ INSERT INTO t1 SET f1= 11 + 1;
SELECT MAX(f1) FROM t1;
MAX(f1)
12
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -769,7 +945,10 @@ DROP INDEX my_idx6 ON mysqltest1.t6;
SELECT MAX(f1) FROM t1;
MAX(f1)
12
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -781,7 +960,10 @@ MAX(f1)
12
TEST-INFO: MASTER: The INSERT is committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -802,7 +984,10 @@ INSERT INTO t1 SET f1= 12 + 1;
SELECT MAX(f1) FROM t1;
MAX(f1)
13
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -812,7 +997,10 @@ CREATE INDEX my_idx5 ON mysqltest1.t5(f1);
SELECT MAX(f1) FROM t1;
MAX(f1)
13
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -824,7 +1012,10 @@ MAX(f1)
13
TEST-INFO: MASTER: The INSERT is committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -847,7 +1038,10 @@ INSERT INTO t1 SET f1= 13 + 1;
SELECT MAX(f1) FROM t1;
MAX(f1)
14
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -857,7 +1051,10 @@ DROP DATABASE mysqltest2;
SELECT MAX(f1) FROM t1;
MAX(f1)
14
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -869,7 +1066,10 @@ MAX(f1)
14
TEST-INFO: MASTER: The INSERT is committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -890,7 +1090,10 @@ INSERT INTO t1 SET f1= 14 + 1;
SELECT MAX(f1) FROM t1;
MAX(f1)
15
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -900,7 +1103,10 @@ CREATE DATABASE mysqltest3;
SELECT MAX(f1) FROM t1;
MAX(f1)
15
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -912,7 +1118,10 @@ MAX(f1)
15
TEST-INFO: MASTER: The INSERT is committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -935,7 +1144,10 @@ INSERT INTO t1 SET f1= 15 + 1;
SELECT MAX(f1) FROM t1;
MAX(f1)
16
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -945,7 +1157,10 @@ CREATE PROCEDURE p1() READS SQL DATA SELECT "this is p1";
SELECT MAX(f1) FROM t1;
MAX(f1)
16
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -957,7 +1172,10 @@ MAX(f1)
16
TEST-INFO: MASTER: The INSERT is committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -998,7 +1216,10 @@ INSERT INTO t1 SET f1= 16 + 1;
SELECT MAX(f1) FROM t1;
MAX(f1)
17
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -1008,7 +1229,10 @@ ALTER PROCEDURE p1 COMMENT "I have been altered";
SELECT MAX(f1) FROM t1;
MAX(f1)
17
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -1020,7 +1244,10 @@ MAX(f1)
17
TEST-INFO: MASTER: The INSERT is committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -1061,7 +1288,10 @@ INSERT INTO t1 SET f1= 17 + 1;
SELECT MAX(f1) FROM t1;
MAX(f1)
18
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -1071,7 +1301,10 @@ DROP PROCEDURE p1;
SELECT MAX(f1) FROM t1;
MAX(f1)
18
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -1083,7 +1316,10 @@ MAX(f1)
18
TEST-INFO: MASTER: The INSERT is committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -1102,7 +1338,10 @@ INSERT INTO t1 SET f1= 18 + 1;
SELECT MAX(f1) FROM t1;
MAX(f1)
19
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -1112,7 +1351,10 @@ CREATE OR REPLACE VIEW v1 as select * from t1;
SELECT MAX(f1) FROM t1;
MAX(f1)
19
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -1124,7 +1366,10 @@ MAX(f1)
19
TEST-INFO: MASTER: The INSERT is committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -1147,7 +1392,10 @@ INSERT INTO t1 SET f1= 19 + 1;
SELECT MAX(f1) FROM t1;
MAX(f1)
20
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -1157,7 +1405,10 @@ ALTER VIEW v1 AS select f1 from t1;
SELECT MAX(f1) FROM t1;
MAX(f1)
20
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -1169,7 +1420,10 @@ MAX(f1)
20
TEST-INFO: MASTER: The INSERT is committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -1192,7 +1446,10 @@ INSERT INTO t1 SET f1= 20 + 1;
SELECT MAX(f1) FROM t1;
MAX(f1)
21
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -1202,7 +1459,10 @@ DROP VIEW IF EXISTS v1;
SELECT MAX(f1) FROM t1;
MAX(f1)
21
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -1214,7 +1474,10 @@ MAX(f1)
21
TEST-INFO: MASTER: The INSERT is committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -1235,7 +1498,10 @@ INSERT INTO t1 SET f1= 21 + 1;
SELECT MAX(f1) FROM t1;
MAX(f1)
22
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -1245,7 +1511,10 @@ CREATE TRIGGER trg1 BEFORE INSERT ON t1 FOR EACH ROW SET @a:=1;
SELECT MAX(f1) FROM t1;
MAX(f1)
22
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -1257,7 +1526,10 @@ MAX(f1)
22
TEST-INFO: MASTER: The INSERT is committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -1280,7 +1552,10 @@ INSERT INTO t1 SET f1= 22 + 1;
SELECT MAX(f1) FROM t1;
MAX(f1)
23
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -1290,7 +1565,10 @@ DROP TRIGGER trg1;
SELECT MAX(f1) FROM t1;
MAX(f1)
23
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -1302,7 +1580,10 @@ MAX(f1)
23
TEST-INFO: MASTER: The INSERT is committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -1323,7 +1604,10 @@ INSERT INTO t1 SET f1= 23 + 1;
SELECT MAX(f1) FROM t1;
MAX(f1)
24
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -1333,7 +1617,10 @@ CREATE USER user1@localhost;
SELECT MAX(f1) FROM t1;
MAX(f1)
24
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -1345,7 +1632,10 @@ MAX(f1)
24
TEST-INFO: MASTER: The INSERT is committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -1368,7 +1658,10 @@ INSERT INTO t1 SET f1= 24 + 1;
SELECT MAX(f1) FROM t1;
MAX(f1)
25
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -1378,7 +1671,10 @@ RENAME USER user1@localhost TO rename1@localhost;
SELECT MAX(f1) FROM t1;
MAX(f1)
25
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -1390,7 +1686,10 @@ MAX(f1)
25
TEST-INFO: MASTER: The INSERT is committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -1413,7 +1712,10 @@ INSERT INTO t1 SET f1= 25 + 1;
SELECT MAX(f1) FROM t1;
MAX(f1)
26
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -1423,7 +1725,10 @@ DROP USER rename1@localhost;
SELECT MAX(f1) FROM t1;
MAX(f1)
26
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
@@ -1435,7 +1740,10 @@ MAX(f1)
26
TEST-INFO: MASTER: The INSERT is committed (Succeeded)
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
SELECT MAX(f1) FROM t1;
MAX(f1)
diff --git a/mysql-test/suite/rpl/r/rpl_delayed_slave.result b/mysql-test/suite/rpl/r/rpl_delayed_slave.result
index 7ac68e25ac5c5..35d7f40800b37 100644
--- a/mysql-test/suite/rpl/r/rpl_delayed_slave.result
+++ b/mysql-test/suite/rpl/r/rpl_delayed_slave.result
@@ -164,6 +164,7 @@ Warnings:
Note 4190 RESET SLAVE is implicitly changing the value of 'Using_Gtid' from 'No' to 'Slave_Pos'
[on master]
connection master;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
[on slave]
connection slave;
diff --git a/mysql-test/suite/rpl/r/rpl_extra_col_master_innodb.result b/mysql-test/suite/rpl/r/rpl_extra_col_master_innodb.result
index 145b269aac9d8..f40b2b8540543 100644
--- a/mysql-test/suite/rpl/r/rpl_extra_col_master_innodb.result
+++ b/mysql-test/suite/rpl/r/rpl_extra_col_master_innodb.result
@@ -448,6 +448,7 @@ CREATE TABLE t10 (a INT PRIMARY KEY, b BLOB, c CHAR(5)
connection master;
CREATE TABLE t10 (a INT KEY, b BLOB, f DOUBLE DEFAULT '233',
c CHAR(5), e INT DEFAULT '1')ENGINE='InnoDB';
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Start Slave ***
@@ -490,6 +491,7 @@ CREATE TABLE t11 (a INT PRIMARY KEY, b BLOB, c VARCHAR(254)
connection master;
CREATE TABLE t11 (a INT KEY, b BLOB, f TEXT,
c CHAR(5) DEFAULT 'test', e INT DEFAULT '1')ENGINE='InnoDB';
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Start Slave ***
@@ -532,6 +534,7 @@ CREATE TABLE t12 (a INT PRIMARY KEY, b BLOB, c BLOB
connection master;
CREATE TABLE t12 (a INT KEY, b BLOB, f TEXT,
c CHAR(5) DEFAULT 'test', e INT DEFAULT '1')ENGINE='InnoDB';
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Start Slave ***
@@ -583,6 +586,7 @@ CREATE TABLE t14 (c1 INT KEY, c4 BLOB, c5 CHAR(5),
c6 INT DEFAULT '1',
c7 TIMESTAMP NULL DEFAULT CURRENT_TIMESTAMP
)ENGINE='InnoDB';
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Start Slave ***
@@ -644,6 +648,7 @@ include/reset_slave.inc
DROP TABLE t14;
connection master;
DROP TABLE t14;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
connection slave;
START SLAVE;
@@ -667,6 +672,7 @@ CREATE TABLE t15 (c1 INT KEY, c4 BLOB, c5 CHAR(5),
c6 INT DEFAULT '1',
c7 TIMESTAMP NULL DEFAULT CURRENT_TIMESTAMP
)ENGINE='InnoDB';
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Start Slave ***
@@ -703,6 +709,7 @@ Warning 1948 Specified value for @@gtid_slave_pos contains no value for replicat
DROP TABLE t15;
connection master;
DROP TABLE t15;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
connection slave;
START SLAVE;
@@ -726,6 +733,7 @@ CREATE TABLE t16 (c1 INT KEY, c4 BLOB, c5 CHAR(5),
c6 INT DEFAULT '1',
c7 TIMESTAMP NULL DEFAULT CURRENT_TIMESTAMP
)ENGINE='InnoDB';
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Start Slave ***
@@ -762,6 +770,7 @@ Warning 1948 Specified value for @@gtid_slave_pos contains no value for replicat
DROP TABLE t16;
connection master;
DROP TABLE t16;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
connection slave;
START SLAVE;
@@ -785,6 +794,7 @@ CREATE TABLE t17 (c1 INT KEY, c4 BLOB, c5 CHAR(5),
c6 INT DEFAULT '1',
c7 TIMESTAMP NULL DEFAULT CURRENT_TIMESTAMP
)ENGINE='InnoDB';
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Start Slave ***
@@ -851,6 +861,7 @@ CREATE TABLE t18 (c1 INT KEY, c4 BLOB, c5 CHAR(5),
c6 INT DEFAULT '1',
c7 TIMESTAMP NULL DEFAULT CURRENT_TIMESTAMP
)ENGINE='InnoDB';
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Start Slave ***
@@ -916,6 +927,7 @@ CREATE TABLE t5 (c1 INT KEY, c4 BLOB, c5 CHAR(5),
c6 LONG,
c7 TIMESTAMP NULL DEFAULT CURRENT_TIMESTAMP
)ENGINE='InnoDB';
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Start Slave ***
diff --git a/mysql-test/suite/rpl/r/rpl_extra_col_master_myisam.result b/mysql-test/suite/rpl/r/rpl_extra_col_master_myisam.result
index dae497d5f2923..f9f195af64a10 100644
--- a/mysql-test/suite/rpl/r/rpl_extra_col_master_myisam.result
+++ b/mysql-test/suite/rpl/r/rpl_extra_col_master_myisam.result
@@ -448,6 +448,7 @@ CREATE TABLE t10 (a INT PRIMARY KEY, b BLOB, c CHAR(5)
connection master;
CREATE TABLE t10 (a INT KEY, b BLOB, f DOUBLE DEFAULT '233',
c CHAR(5), e INT DEFAULT '1')ENGINE='MyISAM';
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Start Slave ***
@@ -490,6 +491,7 @@ CREATE TABLE t11 (a INT PRIMARY KEY, b BLOB, c VARCHAR(254)
connection master;
CREATE TABLE t11 (a INT KEY, b BLOB, f TEXT,
c CHAR(5) DEFAULT 'test', e INT DEFAULT '1')ENGINE='MyISAM';
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Start Slave ***
@@ -532,6 +534,7 @@ CREATE TABLE t12 (a INT PRIMARY KEY, b BLOB, c BLOB
connection master;
CREATE TABLE t12 (a INT KEY, b BLOB, f TEXT,
c CHAR(5) DEFAULT 'test', e INT DEFAULT '1')ENGINE='MyISAM';
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Start Slave ***
@@ -583,6 +586,7 @@ CREATE TABLE t14 (c1 INT KEY, c4 BLOB, c5 CHAR(5),
c6 INT DEFAULT '1',
c7 TIMESTAMP NULL DEFAULT CURRENT_TIMESTAMP
)ENGINE='MyISAM';
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Start Slave ***
@@ -644,6 +648,7 @@ include/reset_slave.inc
DROP TABLE t14;
connection master;
DROP TABLE t14;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
connection slave;
START SLAVE;
@@ -667,6 +672,7 @@ CREATE TABLE t15 (c1 INT KEY, c4 BLOB, c5 CHAR(5),
c6 INT DEFAULT '1',
c7 TIMESTAMP NULL DEFAULT CURRENT_TIMESTAMP
)ENGINE='MyISAM';
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Start Slave ***
@@ -703,6 +709,7 @@ Warning 1948 Specified value for @@gtid_slave_pos contains no value for replicat
DROP TABLE t15;
connection master;
DROP TABLE t15;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
connection slave;
START SLAVE;
@@ -726,6 +733,7 @@ CREATE TABLE t16 (c1 INT KEY, c4 BLOB, c5 CHAR(5),
c6 INT DEFAULT '1',
c7 TIMESTAMP NULL DEFAULT CURRENT_TIMESTAMP
)ENGINE='MyISAM';
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Start Slave ***
@@ -762,6 +770,7 @@ Warning 1948 Specified value for @@gtid_slave_pos contains no value for replicat
DROP TABLE t16;
connection master;
DROP TABLE t16;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
connection slave;
START SLAVE;
@@ -785,6 +794,7 @@ CREATE TABLE t17 (c1 INT KEY, c4 BLOB, c5 CHAR(5),
c6 INT DEFAULT '1',
c7 TIMESTAMP NULL DEFAULT CURRENT_TIMESTAMP
)ENGINE='MyISAM';
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Start Slave ***
@@ -851,6 +861,7 @@ CREATE TABLE t18 (c1 INT KEY, c4 BLOB, c5 CHAR(5),
c6 INT DEFAULT '1',
c7 TIMESTAMP NULL DEFAULT CURRENT_TIMESTAMP
)ENGINE='MyISAM';
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Start Slave ***
@@ -916,6 +927,7 @@ CREATE TABLE t5 (c1 INT KEY, c4 BLOB, c5 CHAR(5),
c6 LONG,
c7 TIMESTAMP NULL DEFAULT CURRENT_TIMESTAMP
)ENGINE='MyISAM';
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Start Slave ***
diff --git a/mysql-test/suite/rpl/r/rpl_extra_col_slave_innodb.result b/mysql-test/suite/rpl/r/rpl_extra_col_slave_innodb.result
index 5a9eb5ee1980c..2041625c7c1ee 100644
--- a/mysql-test/suite/rpl/r/rpl_extra_col_slave_innodb.result
+++ b/mysql-test/suite/rpl/r/rpl_extra_col_slave_innodb.result
@@ -16,6 +16,7 @@ ENGINE='InnoDB';
connection master;
CREATE TABLE t1 (a INT PRIMARY KEY, b INT, c CHAR(10)
) ENGINE='InnoDB';
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Start Slave ***
connection slave;
@@ -51,6 +52,7 @@ ENGINE='InnoDB';
connection master;
CREATE TABLE t2 (a INT PRIMARY KEY, b INT, c CHAR(10)
) ENGINE='InnoDB';
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Master Data Insert ***
connection master;
@@ -72,6 +74,7 @@ Warning 1948 Specified value for @@gtid_slave_pos contains no value for replicat
SELECT * FROM t2 ORDER BY a;
a b c d e
connection master;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
connection slave;
START SLAVE;
@@ -90,6 +93,7 @@ ENGINE='InnoDB';
connection master;
CREATE TABLE t3 (a BLOB, b INT PRIMARY KEY, c CHAR(20)
) ENGINE='InnoDB';
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Start Slave ***
connection slave;
@@ -120,6 +124,7 @@ ENGINE='InnoDB';
connection master;
CREATE TABLE t4 (a DECIMAL(8,2), b INT PRIMARY KEY, c CHAR(20)
) ENGINE='InnoDB';
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Start Slave ***
connection slave;
@@ -149,6 +154,7 @@ connection master;
CREATE TABLE t5 (a INT PRIMARY KEY, b VARCHAR(6),
c DECIMAL(8,2), d BIT, e BLOB,
f FLOAT) ENGINE='InnoDB';
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Start Slave ***
connection slave;
@@ -177,6 +183,7 @@ connection master;
CREATE TABLE t6 (a INT PRIMARY KEY, b VARCHAR(6),
c DECIMAL(8,2), d BIT
) ENGINE='InnoDB';
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Start Slave ***
connection slave;
@@ -209,6 +216,7 @@ ENGINE='InnoDB';
connection master;
CREATE TABLE t7 (a INT PRIMARY KEY, b BLOB, c CHAR(5)
) ENGINE='InnoDB';
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Start Slave ***
connection slave;
@@ -244,6 +252,7 @@ e INT)ENGINE='InnoDB';
connection master;
CREATE TABLE t8 (a INT PRIMARY KEY, b BLOB, c CHAR(5)
) ENGINE='InnoDB';
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Start Slave ***
connection slave;
@@ -270,6 +279,7 @@ i blob) ENGINE='InnoDB';
connection master;
CREATE TABLE t9 (a INT PRIMARY KEY, b BLOB, c CHAR(5)
) ENGINE='InnoDB';
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Start Slave ***
connection slave;
@@ -297,6 +307,7 @@ c CHAR(5), e INT DEFAULT '1')ENGINE='InnoDB';
connection master;
CREATE TABLE t10 (a INT PRIMARY KEY, b BLOB, c CHAR(5)
) ENGINE='InnoDB';
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Start Slave ***
connection slave;
@@ -325,6 +336,7 @@ c CHAR(5) DEFAULT 'test', e INT DEFAULT '1')ENGINE='InnoDB';
connection master;
CREATE TABLE t11 (a INT PRIMARY KEY, b BLOB, c VARCHAR(254)
) ENGINE='InnoDB';
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Start Slave ***
connection slave;
@@ -353,6 +365,7 @@ c CHAR(5) DEFAULT 'test', e INT DEFAULT '1')ENGINE='InnoDB';
connection master;
CREATE TABLE t12 (a INT PRIMARY KEY, b BLOB, c BLOB
) ENGINE='InnoDB';
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Start Slave ***
connection slave;
@@ -391,6 +404,7 @@ e TIMESTAMP NULL DEFAULT CURRENT_TIMESTAMP
connection master;
CREATE TABLE t13 (a INT PRIMARY KEY, b BLOB, c CHAR(5)
) ENGINE='InnoDB';
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Start Slave ***
connection slave;
@@ -429,6 +443,8 @@ c7 TIMESTAMP NULL DEFAULT CURRENT_TIMESTAMP
connection master;
CREATE TABLE t14 (c1 INT PRIMARY KEY, c4 BLOB, c5 CHAR(5)
) ENGINE='InnoDB';
+include/kill_binlog_dump_threads.inc
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Start Slave ***
connection slave;
@@ -465,6 +481,7 @@ c7 TIMESTAMP NULL DEFAULT CURRENT_TIMESTAMP
connection master;
CREATE TABLE t14a (c1 INT PRIMARY KEY, c4 BLOB, c5 CHAR(5)
) ENGINE='InnoDB';
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Start Slave ***
connection slave;
@@ -493,6 +510,7 @@ include/reset_slave.inc
*** Master Drop c5 ***
connection master;
ALTER TABLE t14a DROP COLUMN c5;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Start Slave ***
connection slave;
@@ -558,6 +576,7 @@ c7 TIMESTAMP NULL DEFAULT CURRENT_TIMESTAMP
connection master;
CREATE TABLE t15 (c1 INT PRIMARY KEY, c2 DECIMAL(8,2), c3 TEXT,
c4 BLOB, c5 CHAR(5)) ENGINE='InnoDB';
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Start Slave ***
connection slave;
@@ -626,6 +645,7 @@ c7 TIMESTAMP NULL DEFAULT CURRENT_TIMESTAMP
connection master;
CREATE TABLE t16 (c1 INT PRIMARY KEY, c2 DECIMAL(8,2), c3 TEXT,
c4 BLOB, c5 CHAR(5))ENGINE='InnoDB';
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Start Slave ***
connection slave;
@@ -697,6 +717,7 @@ ENGINE='InnoDB';
connection master;
CREATE TABLE t17 (a BIGINT PRIMARY KEY, b INT, c CHAR(10)
) ENGINE='InnoDB';
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Start Slave ***
connection slave;
diff --git a/mysql-test/suite/rpl/r/rpl_extra_col_slave_minimal.result b/mysql-test/suite/rpl/r/rpl_extra_col_slave_minimal.result
index adbee24d852ee..cdbf01e033ad9 100644
--- a/mysql-test/suite/rpl/r/rpl_extra_col_slave_minimal.result
+++ b/mysql-test/suite/rpl/r/rpl_extra_col_slave_minimal.result
@@ -11,6 +11,7 @@ d char(10) as (concat(trim(c), '-U')) unique,
e char(10) as (concat('S', b)) stored) engine=innodb;
connection master;
create table t1 (a int primary key, b blob, c char(5)) engine=innodb;
+include/kill_binlog_dump_threads.inc
reset master;
connection slave;
start slave;
diff --git a/mysql-test/suite/rpl/r/rpl_extra_col_slave_myisam.result b/mysql-test/suite/rpl/r/rpl_extra_col_slave_myisam.result
index d0fb5a305f8a8..199f38743915a 100644
--- a/mysql-test/suite/rpl/r/rpl_extra_col_slave_myisam.result
+++ b/mysql-test/suite/rpl/r/rpl_extra_col_slave_myisam.result
@@ -16,6 +16,7 @@ ENGINE='MyISAM';
connection master;
CREATE TABLE t1 (a INT PRIMARY KEY, b INT, c CHAR(10)
) ENGINE='MyISAM';
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Start Slave ***
connection slave;
@@ -51,6 +52,7 @@ ENGINE='MyISAM';
connection master;
CREATE TABLE t2 (a INT PRIMARY KEY, b INT, c CHAR(10)
) ENGINE='MyISAM';
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Master Data Insert ***
connection master;
@@ -72,6 +74,7 @@ Warning 1948 Specified value for @@gtid_slave_pos contains no value for replicat
SELECT * FROM t2 ORDER BY a;
a b c d e
connection master;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
connection slave;
START SLAVE;
@@ -90,6 +93,7 @@ ENGINE='MyISAM';
connection master;
CREATE TABLE t3 (a BLOB, b INT PRIMARY KEY, c CHAR(20)
) ENGINE='MyISAM';
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Start Slave ***
connection slave;
@@ -120,6 +124,7 @@ ENGINE='MyISAM';
connection master;
CREATE TABLE t4 (a DECIMAL(8,2), b INT PRIMARY KEY, c CHAR(20)
) ENGINE='MyISAM';
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Start Slave ***
connection slave;
@@ -149,6 +154,7 @@ connection master;
CREATE TABLE t5 (a INT PRIMARY KEY, b VARCHAR(6),
c DECIMAL(8,2), d BIT, e BLOB,
f FLOAT) ENGINE='MyISAM';
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Start Slave ***
connection slave;
@@ -177,6 +183,7 @@ connection master;
CREATE TABLE t6 (a INT PRIMARY KEY, b VARCHAR(6),
c DECIMAL(8,2), d BIT
) ENGINE='MyISAM';
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Start Slave ***
connection slave;
@@ -209,6 +216,7 @@ ENGINE='MyISAM';
connection master;
CREATE TABLE t7 (a INT PRIMARY KEY, b BLOB, c CHAR(5)
) ENGINE='MyISAM';
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Start Slave ***
connection slave;
@@ -244,6 +252,7 @@ e INT)ENGINE='MyISAM';
connection master;
CREATE TABLE t8 (a INT PRIMARY KEY, b BLOB, c CHAR(5)
) ENGINE='MyISAM';
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Start Slave ***
connection slave;
@@ -270,6 +279,7 @@ i blob) ENGINE='MyISAM';
connection master;
CREATE TABLE t9 (a INT PRIMARY KEY, b BLOB, c CHAR(5)
) ENGINE='MyISAM';
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Start Slave ***
connection slave;
@@ -297,6 +307,7 @@ c CHAR(5), e INT DEFAULT '1')ENGINE='MyISAM';
connection master;
CREATE TABLE t10 (a INT PRIMARY KEY, b BLOB, c CHAR(5)
) ENGINE='MyISAM';
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Start Slave ***
connection slave;
@@ -325,6 +336,7 @@ c CHAR(5) DEFAULT 'test', e INT DEFAULT '1')ENGINE='MyISAM';
connection master;
CREATE TABLE t11 (a INT PRIMARY KEY, b BLOB, c VARCHAR(254)
) ENGINE='MyISAM';
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Start Slave ***
connection slave;
@@ -353,6 +365,7 @@ c CHAR(5) DEFAULT 'test', e INT DEFAULT '1')ENGINE='MyISAM';
connection master;
CREATE TABLE t12 (a INT PRIMARY KEY, b BLOB, c BLOB
) ENGINE='MyISAM';
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Start Slave ***
connection slave;
@@ -391,6 +404,7 @@ e TIMESTAMP NULL DEFAULT CURRENT_TIMESTAMP
connection master;
CREATE TABLE t13 (a INT PRIMARY KEY, b BLOB, c CHAR(5)
) ENGINE='MyISAM';
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Start Slave ***
connection slave;
@@ -429,6 +443,8 @@ c7 TIMESTAMP NULL DEFAULT CURRENT_TIMESTAMP
connection master;
CREATE TABLE t14 (c1 INT PRIMARY KEY, c4 BLOB, c5 CHAR(5)
) ENGINE='MyISAM';
+include/kill_binlog_dump_threads.inc
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Start Slave ***
connection slave;
@@ -465,6 +481,7 @@ c7 TIMESTAMP NULL DEFAULT CURRENT_TIMESTAMP
connection master;
CREATE TABLE t14a (c1 INT PRIMARY KEY, c4 BLOB, c5 CHAR(5)
) ENGINE='MyISAM';
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Start Slave ***
connection slave;
@@ -493,6 +510,7 @@ include/reset_slave.inc
*** Master Drop c5 ***
connection master;
ALTER TABLE t14a DROP COLUMN c5;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Start Slave ***
connection slave;
@@ -558,6 +576,7 @@ c7 TIMESTAMP NULL DEFAULT CURRENT_TIMESTAMP
connection master;
CREATE TABLE t15 (c1 INT PRIMARY KEY, c2 DECIMAL(8,2), c3 TEXT,
c4 BLOB, c5 CHAR(5)) ENGINE='MyISAM';
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Start Slave ***
connection slave;
@@ -626,6 +645,7 @@ c7 TIMESTAMP NULL DEFAULT CURRENT_TIMESTAMP
connection master;
CREATE TABLE t16 (c1 INT PRIMARY KEY, c2 DECIMAL(8,2), c3 TEXT,
c4 BLOB, c5 CHAR(5))ENGINE='MyISAM';
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Start Slave ***
connection slave;
@@ -697,6 +717,7 @@ ENGINE='MyISAM';
connection master;
CREATE TABLE t17 (a BIGINT PRIMARY KEY, b INT, c CHAR(10)
) ENGINE='MyISAM';
+include/kill_binlog_dump_threads.inc
RESET MASTER;
*** Start Slave ***
connection slave;
diff --git a/mysql-test/suite/rpl/r/rpl_gtid_basic.result b/mysql-test/suite/rpl/r/rpl_gtid_basic.result
index afc700a72c551..42a171d496b2a 100644
--- a/mysql-test/suite/rpl/r/rpl_gtid_basic.result
+++ b/mysql-test/suite/rpl/r/rpl_gtid_basic.result
@@ -197,6 +197,7 @@ connection server_1;
SET @old_state= @@GLOBAL.gtid_binlog_state;
SET GLOBAL gtid_binlog_state = '';
ERROR HY000: This operation is not allowed if any GTID has been logged to the binary log. Run RESET MASTER first to erase the log
+include/kill_binlog_dump_threads.inc
RESET MASTER;
SET GLOBAL gtid_binlog_state = '';
FLUSH LOGS;
diff --git a/mysql-test/suite/rpl/r/rpl_gtid_crash.result b/mysql-test/suite/rpl/r/rpl_gtid_crash.result
index f46c17db3fc43..41cd35b3863b1 100644
--- a/mysql-test/suite/rpl/r/rpl_gtid_crash.result
+++ b/mysql-test/suite/rpl/r/rpl_gtid_crash.result
@@ -43,6 +43,7 @@ include/stop_slave.inc
RESET MASTER;
SET GLOBAL gtid_slave_pos='';
connection server_1;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
SHOW BINLOG EVENTS IN 'master-bin.000001' LIMIT 1,1;
Log_name Pos Event_type Server_id End_log_pos Info
diff --git a/mysql-test/suite/rpl/r/rpl_gtid_errorhandling.result b/mysql-test/suite/rpl/r/rpl_gtid_errorhandling.result
index c62760ac4d0f6..1d6332cb76f59 100644
--- a/mysql-test/suite/rpl/r/rpl_gtid_errorhandling.result
+++ b/mysql-test/suite/rpl/r/rpl_gtid_errorhandling.result
@@ -93,6 +93,7 @@ SET SESSION binlog_format= @old_mode;
connection slave;
include/stop_slave.inc
connection master;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
INSERT INTO t1 VALUES (2);
INSERT INTO t1 VALUES (4);
diff --git a/mysql-test/suite/rpl/r/rpl_gtid_header_valid.result b/mysql-test/suite/rpl/r/rpl_gtid_header_valid.result
index 6731abc27b783..c733c67f88666 100644
--- a/mysql-test/suite/rpl/r/rpl_gtid_header_valid.result
+++ b/mysql-test/suite/rpl/r/rpl_gtid_header_valid.result
@@ -32,6 +32,7 @@ include/wait_for_slave_sql_error.inc [errno=1594]
STOP SLAVE IO_THREAD;
# Reset master binlogs (as there is an invalid event) and slave state
connection master;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
connection slave;
RESET MASTER;
@@ -58,6 +59,7 @@ STOP SLAVE IO_THREAD;
set statement SQL_LOG_BIN=0 for XA COMMIT 'x1';
# Reset master binlogs (as there is an invalid event) and slave state
connection master;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
connection slave;
RESET MASTER;
@@ -84,6 +86,7 @@ STOP SLAVE IO_THREAD;
set statement SQL_LOG_BIN=0 for XA COMMIT 'x1';
# Reset master binlogs (as there is an invalid event) and slave state
connection master;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
connection slave;
RESET MASTER;
@@ -105,6 +108,7 @@ include/wait_for_slave_sql_error.inc [errno=1594]
STOP SLAVE IO_THREAD;
# Reset master binlogs (as there is an invalid event) and slave state
connection master;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
connection slave;
RESET SLAVE;
@@ -128,6 +132,7 @@ include/wait_for_slave_sql_error.inc [errno=1594]
STOP SLAVE IO_THREAD;
# Reset master binlogs (as there is an invalid event) and slave state
connection master;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
connection slave;
SET STATEMENT sql_log_bin=0 FOR alter table t1 add column (nc int);
diff --git a/mysql-test/suite/rpl/r/rpl_gtid_mdev4820.result b/mysql-test/suite/rpl/r/rpl_gtid_mdev4820.result
index 5fba9966a43fb..8227011f6fbd8 100644
--- a/mysql-test/suite/rpl/r/rpl_gtid_mdev4820.result
+++ b/mysql-test/suite/rpl/r/rpl_gtid_mdev4820.result
@@ -18,6 +18,7 @@ include/stop_slave.inc
connection server_1;
INSERT INTO t1 VALUES (2);
INSERT INTO t1 VALUES (3);
+include/kill_binlog_dump_threads.inc
RESET MASTER;
SET GLOBAL gtid_slave_pos= 'OLD_GTID_POS';
connection server_2;
diff --git a/mysql-test/suite/rpl/r/rpl_gtid_reconnect.result b/mysql-test/suite/rpl/r/rpl_gtid_reconnect.result
index f9d6cd3b74352..ce96cb7348a73 100644
--- a/mysql-test/suite/rpl/r/rpl_gtid_reconnect.result
+++ b/mysql-test/suite/rpl/r/rpl_gtid_reconnect.result
@@ -43,6 +43,7 @@ SET GLOBAL gtid_slave_pos= "";
connection server_1;
SET GLOBAL debug_dbug= @old_debug;
TRUNCATE t1;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
SET gtid_domain_id=10;
SET gtid_seq_no=50;
@@ -91,6 +92,7 @@ SET GLOBAL gtid_slave_pos= "";
connection server_1;
SET GLOBAL debug_dbug= @old_debug;
TRUNCATE t1;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
include/kill_binlog_dump_threads.inc
SET gtid_domain_id= 9;
@@ -143,6 +145,7 @@ SET GLOBAL gtid_slave_pos= "";
connection server_1;
SET GLOBAL debug_dbug= @old_debug;
TRUNCATE t1;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
include/kill_binlog_dump_threads.inc
SET GLOBAL debug_dbug="+d,dummy_disable_default_dbug_output";
diff --git a/mysql-test/suite/rpl/r/rpl_gtid_sort.result b/mysql-test/suite/rpl/r/rpl_gtid_sort.result
index 46a65e7d865bb..6630a802248fd 100644
--- a/mysql-test/suite/rpl/r/rpl_gtid_sort.result
+++ b/mysql-test/suite/rpl/r/rpl_gtid_sort.result
@@ -17,6 +17,7 @@ SHOW VARIABLES LIKE 'gtid_binlog_state';
Variable_name Value
gtid_binlog_state
connection server_1;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
FLUSH LOGS;
CREATE TABLE t1 (a INT PRIMARY KEY) ENGINE=InnoDB;
diff --git a/mysql-test/suite/rpl/r/rpl_gtid_startpos.result b/mysql-test/suite/rpl/r/rpl_gtid_startpos.result
index f271e73092156..0f0de0c4f65e9 100644
--- a/mysql-test/suite/rpl/r/rpl_gtid_startpos.result
+++ b/mysql-test/suite/rpl/r/rpl_gtid_startpos.result
@@ -5,6 +5,7 @@ include/stop_slave.inc
RESET MASTER;
RESET SLAVE;
connection server_1;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
FLUSH LOGS;
CREATE TABLE t1 (a INT PRIMARY KEY) ENGINE=InnoDB;
@@ -92,6 +93,7 @@ include/stop_slave.inc
RESET SLAVE ALL;
RESET MASTER;
connection server_1;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
connection server_2;
SET GLOBAL gtid_slave_pos='';
@@ -199,6 +201,7 @@ include/stop_slave.inc
RESET MASTER;
SET GLOBAL gtid_slave_pos="";
connection server_1;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
TRUNCATE TABLE t1;
INSERT INTO t1 VALUES (10);
@@ -244,6 +247,7 @@ SET GLOBAL gtid_slave_pos= "";
CHANGE MASTER TO master_host='127.0.0.1', master_port=MASTER_PORT, master_user='root', master_use_gtid=no, master_log_file="", master_log_pos= 4;
connection server_1;
DROP TABLE t1;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
CREATE TABLE t1 (a INT PRIMARY KEY);
connection server_2;
diff --git a/mysql-test/suite/rpl/r/rpl_gtid_until.result b/mysql-test/suite/rpl/r/rpl_gtid_until.result
index 029d2367e5026..3169bff2622cd 100644
--- a/mysql-test/suite/rpl/r/rpl_gtid_until.result
+++ b/mysql-test/suite/rpl/r/rpl_gtid_until.result
@@ -206,6 +206,7 @@ RESET SLAVE ALL;
RESET MASTER;
SET GLOBAL gtid_slave_pos='';
connection server_1;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
INSERT INTO t1 VALUES (10);
INSERT INTO t1 VALUES (11);
diff --git a/mysql-test/suite/rpl/r/rpl_heartbeat.result b/mysql-test/suite/rpl/r/rpl_heartbeat.result
index 42cf90bc73804..f634eeebb8d0c 100644
--- a/mysql-test/suite/rpl/r/rpl_heartbeat.result
+++ b/mysql-test/suite/rpl/r/rpl_heartbeat.result
@@ -1,9 +1,11 @@
include/master-slave.inc
[connection master]
+connection slave;
+include/stop_slave.inc
connection master;
+include/kill_binlog_dump_threads.inc
reset master;
connection slave;
-include/stop_slave.inc
set @restore_slave_net_timeout= @@global.slave_net_timeout;
set @@global.slave_net_timeout= 10;
change master to master_host='127.0.0.1',master_port=MASTER_PORT, master_user='root';
@@ -59,7 +61,10 @@ Value 0.500
start slave;
connection master;
create table t1 (f1 int);
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
include/check_slave_param.inc [Relay_Log_File]
show status like 'Slave_heartbeat_period';;
Variable_name Slave_heartbeat_period
@@ -67,7 +72,10 @@ Value 0.500
A heartbeat has been received by the slave
connection master;
drop table t1;
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
set @@global.slave_net_timeout= @restore_slave_net_timeout;
include/rpl_end.inc
End of tests
diff --git a/mysql-test/suite/rpl/r/rpl_heartbeat_basic.result b/mysql-test/suite/rpl/r/rpl_heartbeat_basic.result
index a9bd16cc85ad2..3ec4bab25727c 100644
--- a/mysql-test/suite/rpl/r/rpl_heartbeat_basic.result
+++ b/mysql-test/suite/rpl/r/rpl_heartbeat_basic.result
@@ -7,6 +7,7 @@ include/stop_slave.inc
RESET SLAVE;
SET @restore_slave_net_timeout=@@global.slave_net_timeout;
connection master;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
SET @restore_slave_net_timeout=@@global.slave_net_timeout;
@@ -268,6 +269,7 @@ DROP TABLE t1;
set sql_log_bin= 1;
connection master;
DROP TABLE t1;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
connection slave;
CHANGE MASTER TO MASTER_HOST='127.0.0.1', MASTER_PORT=MASTER_PORT, MASTER_USER='root', MASTER_CONNECT_RETRY=20, MASTER_HEARTBEAT_PERIOD=0.5;
@@ -293,10 +295,13 @@ connection slave;
STOP SLAVE;
include/reset_slave.inc
CHANGE MASTER TO MASTER_HOST='127.0.0.1', MASTER_PORT=MASTER_PORT, MASTER_USER='root', MASTER_CONNECT_RETRY=20, MASTER_HEARTBEAT_PERIOD=0.1;
-include/start_slave.inc
connection master;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
connection slave;
+include/start_slave.inc
+connection master;
+connection slave;
Heartbeat events are received after reset of master (1 means 'yes'): 1
*** Reload master ***
diff --git a/mysql-test/suite/rpl/r/rpl_ip_mix.result b/mysql-test/suite/rpl/r/rpl_ip_mix.result
index cc11f48199132..a7fbda34a9539 100644
--- a/mysql-test/suite/rpl/r/rpl_ip_mix.result
+++ b/mysql-test/suite/rpl/r/rpl_ip_mix.result
@@ -2,6 +2,7 @@
connect master,$IPv6,root,,test,$MASTER_MYPORT;
connect slave,127.0.0.1,root,,test,$SLAVE_MYPORT;
connection master;
+include/kill_binlog_dump_threads.inc
reset master;
show master status;
File Position Binlog_Do_DB Binlog_Ignore_DB
@@ -18,6 +19,7 @@ connection default;
connect master,$IPv6,root,,test,$MASTER_MYPORT;
connect slave,127.0.0.1,root,,test,$SLAVE_MYPORT;
connection master;
+include/kill_binlog_dump_threads.inc
reset master;
show master status;
File Position Binlog_Do_DB Binlog_Ignore_DB
@@ -34,6 +36,7 @@ connection default;
connect master,$IPv6,root,,test,$MASTER_MYPORT;
connect slave,127.0.0.1,root,,test,$SLAVE_MYPORT;
connection master;
+include/kill_binlog_dump_threads.inc
reset master;
show master status;
File Position Binlog_Do_DB Binlog_Ignore_DB
diff --git a/mysql-test/suite/rpl/r/rpl_ipv4_as_ipv6.result b/mysql-test/suite/rpl/r/rpl_ipv4_as_ipv6.result
index 5ad4b3eacc522..cdf50dc2f532c 100644
--- a/mysql-test/suite/rpl/r/rpl_ipv4_as_ipv6.result
+++ b/mysql-test/suite/rpl/r/rpl_ipv4_as_ipv6.result
@@ -1,6 +1,7 @@
#################### IP: 127.0.0.1 ###########################
connect master,$IPv6,root,,test,$MASTER_MYPORT;
connect slave,$IPv6,root,,test,$SLAVE_MYPORT;
+include/kill_binlog_dump_threads.inc
connection master;
reset master;
show master status;
@@ -16,6 +17,7 @@ connection default;
#################### IP: 0:0:0:0:0:FFFF:127.0.0.1 ###########################
connect master,$IPv6,root,,test,$MASTER_MYPORT;
connect slave,$IPv6,root,,test,$SLAVE_MYPORT;
+include/kill_binlog_dump_threads.inc
connection master;
reset master;
show master status;
@@ -31,6 +33,7 @@ connection default;
#################### IP: 0000:0000:0000:0000:0000:FFFF:127.0.0.1 ###########################
connect master,$IPv6,root,,test,$MASTER_MYPORT;
connect slave,$IPv6,root,,test,$SLAVE_MYPORT;
+include/kill_binlog_dump_threads.inc
connection master;
reset master;
show master status;
@@ -46,6 +49,7 @@ connection default;
#################### IP: 0:0000:0000:0:0000:FFFF:127.0.0.1 ###########################
connect master,$IPv6,root,,test,$MASTER_MYPORT;
connect slave,$IPv6,root,,test,$SLAVE_MYPORT;
+include/kill_binlog_dump_threads.inc
connection master;
reset master;
show master status;
@@ -61,6 +65,7 @@ connection default;
#################### IP: 0::0000:FFFF:127.0.0.1 ###########################
connect master,$IPv6,root,,test,$MASTER_MYPORT;
connect slave,$IPv6,root,,test,$SLAVE_MYPORT;
+include/kill_binlog_dump_threads.inc
connection master;
reset master;
show master status;
@@ -76,6 +81,7 @@ connection default;
#################### IP: ::FFFF:127.0.0.1 ###########################
connect master,$IPv6,root,,test,$MASTER_MYPORT;
connect slave,$IPv6,root,,test,$SLAVE_MYPORT;
+include/kill_binlog_dump_threads.inc
connection master;
reset master;
show master status;
diff --git a/mysql-test/suite/rpl/r/rpl_ipv6.result b/mysql-test/suite/rpl/r/rpl_ipv6.result
index fc9171d8b37ee..2ffddfa8f2152 100644
--- a/mysql-test/suite/rpl/r/rpl_ipv6.result
+++ b/mysql-test/suite/rpl/r/rpl_ipv6.result
@@ -1,6 +1,7 @@
#################### IP: ::1 ###########################
connect master,$IPv6,root,,test,$MASTER_MYPORT;
connect slave,$IPv6,root,,test,$SLAVE_MYPORT;
+include/kill_binlog_dump_threads.inc
connection master;
reset master;
show master status;
@@ -16,6 +17,7 @@ connection default;
#################### IP: 0000:0000:0000:0000:0000:0000:0000:0001 ###########################
connect master,$IPv6,root,,test,$MASTER_MYPORT;
connect slave,$IPv6,root,,test,$SLAVE_MYPORT;
+include/kill_binlog_dump_threads.inc
connection master;
reset master;
show master status;
@@ -31,6 +33,7 @@ connection default;
#################### IP: 0:0:0:0:0:0:0:1 ###########################
connect master,$IPv6,root,,test,$MASTER_MYPORT;
connect slave,$IPv6,root,,test,$SLAVE_MYPORT;
+include/kill_binlog_dump_threads.inc
connection master;
reset master;
show master status;
diff --git a/mysql-test/suite/rpl/r/rpl_loaddata.result b/mysql-test/suite/rpl/r/rpl_loaddata.result
index 0be197642e9af..d7c0166561561 100644
--- a/mysql-test/suite/rpl/r/rpl_loaddata.result
+++ b/mysql-test/suite/rpl/r/rpl_loaddata.result
@@ -62,6 +62,7 @@ Warnings:
Warning 1948 Specified value for @@gtid_slave_pos contains no value for replication domain 0. This conflicts with the binary log which contains GTID 0-2-14. If MASTER_GTID_POS=CURRENT_POS is used, the binlog position will override the new value of @@gtid_slave_pos
include/check_slave_no_error.inc
connection master;
+include/kill_binlog_dump_threads.inc
reset master;
create table t2 (day date,id int(9),category enum('a','b','c'),name varchar(60),
unique(day)) engine=MyISAM;
diff --git a/mysql-test/suite/rpl/r/rpl_mdev_17614.result b/mysql-test/suite/rpl/r/rpl_mdev_17614.result
index 7e2e8727b4536..779bfac0faadc 100644
--- a/mysql-test/suite/rpl/r/rpl_mdev_17614.result
+++ b/mysql-test/suite/rpl/r/rpl_mdev_17614.result
@@ -32,6 +32,7 @@ a b c
include/stop_slave_io.inc
include/reset_slave.inc
connection master;
+include/kill_binlog_dump_threads.inc
reset master;
drop table t1;
connection slave;
@@ -191,6 +192,7 @@ a b c
include/stop_slave_io.inc
include/reset_slave.inc
connection master;
+include/kill_binlog_dump_threads.inc
reset master;
drop table t1;
connection slave;
diff --git a/mysql-test/suite/rpl/r/rpl_mysqlbinlog_slave_consistency.result b/mysql-test/suite/rpl/r/rpl_mysqlbinlog_slave_consistency.result
index e69fb2fc62d93..6110232d55f6a 100644
--- a/mysql-test/suite/rpl/r/rpl_mysqlbinlog_slave_consistency.result
+++ b/mysql-test/suite/rpl/r/rpl_mysqlbinlog_slave_consistency.result
@@ -7,6 +7,7 @@ call mtr.add_suppression("Both DO_DOMAIN_IDS & IGNORE_DOMAIN_IDS lists can't be
SET sql_log_bin=1;
include/stop_slave.inc
connection master;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
connection slave;
include/start_slave.inc
@@ -1336,8 +1337,10 @@ ERROR HY000: Could not initialize master info structure for ''; more error messa
RESET MASTER;
set global gtid_slave_pos="";
CHANGE MASTER TO MASTER_USE_GTID=slave_pos, DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=();
-include/start_slave.inc
connection master;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
+connection slave;
+include/start_slave.inc
include/rpl_end.inc
-# End of tests (rpl.rpl_mysqlbinlog_slave_consistency)
+# End of test rpl.rpl_mysqlbinlog_slave_consistency.test
diff --git a/mysql-test/suite/rpl/r/rpl_packet.result b/mysql-test/suite/rpl/r/rpl_packet.result
index bb6269607fed0..3f62e585ad54f 100644
--- a/mysql-test/suite/rpl/r/rpl_packet.result
+++ b/mysql-test/suite/rpl/r/rpl_packet.result
@@ -65,6 +65,7 @@ Last_IO_Error = 'Got a packet bigger than 'slave_max_allowed_packet' bytes'
STOP SLAVE;
RESET SLAVE;
connection master;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
SET @max_allowed_packet_0= @@session.max_allowed_packet;
SHOW BINLOG EVENTS;
diff --git a/mysql-test/suite/rpl/r/rpl_parallel_29322.result b/mysql-test/suite/rpl/r/rpl_parallel_29322.result
index 781804100e00b..bdc690765c428 100644
--- a/mysql-test/suite/rpl/r/rpl_parallel_29322.result
+++ b/mysql-test/suite/rpl/r/rpl_parallel_29322.result
@@ -20,6 +20,7 @@ connection slave;
include/stop_slave.inc
include/reset_slave.inc
connection master;
+include/kill_binlog_dump_threads.inc
reset master;
create table t2 (a timestamp, b int);
show create table t2;
@@ -117,6 +118,7 @@ include/reset_slave.inc
Warnings:
Note 4190 RESET SLAVE is implicitly changing the value of 'Using_Gtid' from 'No' to 'Slave_Pos'
connection master;
+include/kill_binlog_dump_threads.inc
reset master;
create table t2 (a timestamp, b int);
show create table t2;
@@ -211,6 +213,7 @@ include/reset_slave.inc
Warnings:
Note 4190 RESET SLAVE is implicitly changing the value of 'Using_Gtid' from 'No' to 'Slave_Pos'
connection master;
+include/kill_binlog_dump_threads.inc
reset master;
create table t2 (a timestamp, b int);
show create table t2;
diff --git a/mysql-test/suite/rpl/r/rpl_parallel_optimistic_until.result b/mysql-test/suite/rpl/r/rpl_parallel_optimistic_until.result
index a83a9b61b9f27..593f32469e9cb 100644
--- a/mysql-test/suite/rpl/r/rpl_parallel_optimistic_until.result
+++ b/mysql-test/suite/rpl/r/rpl_parallel_optimistic_until.result
@@ -5,6 +5,7 @@ include/stop_slave.inc
RESET MASTER;
RESET SLAVE;
connection master;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
CREATE TABLE t1 (a int primary key, b text) ENGINE=InnoDB;
INSERT INTO t1 SET a=25, b='trx0';
diff --git a/mysql-test/suite/rpl/r/rpl_parallel_seq.result b/mysql-test/suite/rpl/r/rpl_parallel_seq.result
index faeb93fcf68f9..6adf7545bf26c 100644
--- a/mysql-test/suite/rpl/r/rpl_parallel_seq.result
+++ b/mysql-test/suite/rpl/r/rpl_parallel_seq.result
@@ -48,6 +48,7 @@ Note 1255 Slave already has been stopped
RESET MASTER;
SET @@global.gtid_slave_pos="";
connection master;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
CREATE TABLE ti (a INT) ENGINE=innodb;
CREATE SEQUENCE s2 ENGINE=innodb;
diff --git a/mysql-test/suite/rpl/r/rpl_parallel_temptable.result b/mysql-test/suite/rpl/r/rpl_parallel_temptable.result
index 0f7af25303e7c..6fe371e30f141 100644
--- a/mysql-test/suite/rpl/r/rpl_parallel_temptable.result
+++ b/mysql-test/suite/rpl/r/rpl_parallel_temptable.result
@@ -7,7 +7,9 @@ SET GLOBAL slave_parallel_threads=5;
include/start_slave.inc
connection server_1;
CREATE TABLE t1 (a INT PRIMARY KEY, b VARCHAR(100) CHARACTER SET utf8);
+include/save_master_gtid.inc
connection server_2;
+include/sync_with_master_gtid.inc
include/stop_slave.inc
connection server_1;
SET gtid_domain_id= 1;
@@ -69,8 +71,10 @@ connection default;
connection default;
connection server_1;
INSERT INTO t1 VALUES (0, 1);
+include/save_master_gtid.inc
connection server_2;
include/start_slave.inc
+include/sync_with_master_gtid.inc
SELECT * FROM t1 WHERE a <= 20 ORDER BY a;
a b
0 1
@@ -208,14 +212,18 @@ CREATE TEMPORARY TABLE t5 (a int) ENGINE=Aria;
CREATE TEMPORARY TABLE t6 (a int) ENGINE=Heap;
INSERT INTO t5 VALUES (1);
INSERT INTO t6 VALUES (2);
+include/save_master_gtid.inc
connection server_2;
+include/sync_with_master_gtid.inc
include/stop_slave.inc
connection server_1;
INSERT INTO t1 SELECT a+40, 5 FROM t5;
INSERT INTO t1 SELECT a+40, 6 FROM t6;
DROP TABLE t5, t6;
+include/save_master_gtid.inc
connection server_2;
include/start_slave.inc
+include/sync_with_master_gtid.inc
SELECT * FROM t1 WHERE a>=40 ORDER BY a;
a b
41 5
diff --git a/mysql-test/suite/rpl/r/rpl_perfschema_applier_status_by_coordinator.result b/mysql-test/suite/rpl/r/rpl_perfschema_applier_status_by_coordinator.result
index ffffb580518cb..2ebfa90456f59 100644
--- a/mysql-test/suite/rpl/r/rpl_perfschema_applier_status_by_coordinator.result
+++ b/mysql-test/suite/rpl/r/rpl_perfschema_applier_status_by_coordinator.result
@@ -50,9 +50,12 @@ include/assert.inc [Value returned by SSS and PS table for Last_Error_Number sho
Value returned by SSS and PS table for Last_Error_Message is same.
connection master;
drop table t;
-reset master;
connection slave;
include/stop_slave.inc
+connection master;
+include/kill_binlog_dump_threads.inc
+reset master;
+connection slave;
reset slave;
reset master;
set @@global.gtid_slave_pos= "";
diff --git a/mysql-test/suite/rpl/r/rpl_perfschema_applier_status_by_worker.result b/mysql-test/suite/rpl/r/rpl_perfschema_applier_status_by_worker.result
index 8180d0c0f44af..6129f1276ea96 100644
--- a/mysql-test/suite/rpl/r/rpl_perfschema_applier_status_by_worker.result
+++ b/mysql-test/suite/rpl/r/rpl_perfschema_applier_status_by_worker.result
@@ -82,6 +82,7 @@ include/stop_slave.inc
RESET SLAVE;
connection master;
DROP TABLE t;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
# Verify that number of rows in 'replication_applier_status_by_worker' table match with
diff --git a/mysql-test/suite/rpl/r/rpl_reset_master.result b/mysql-test/suite/rpl/r/rpl_reset_master.result
new file mode 100644
index 0000000000000..07174433ac3e6
--- /dev/null
+++ b/mysql-test/suite/rpl/r/rpl_reset_master.result
@@ -0,0 +1,36 @@
+include/master-slave.inc
+[connection master]
+connection master;
+CREATE TABLE t1 (a INT PRIMARY KEY);
+INSERT INTO t1 VALUES (1);
+FLUSH BINARY LOGS;
+INSERT INTO t1 VALUES (2);
+*** Test that RESET MASTER fails when a slave is connected.
+RESET MASTER;
+ERROR HY000: Cannot execute RESET MASTER as the binlog is in use by a connected slave or other RESET MASTER or binlog reader. Check SHOW PROCESSLIST for "Binlog Dump" commands and use KILL to stop such readers
+connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+include/stop_slave.inc
+connection master;
+*** Test that RESET MASTER fails on concurrent SHOW BINLOG EVENTS.
+include/kill_binlog_dump_threads.inc
+connection master1;
+SET debug_sync= 'after_show_binlog_events SIGNAL waiting WAIT_FOR go';
+SHOW BINLOG EVENTS in 'master-bin.000001';
+connection master;
+SET debug_sync= 'now WAIT_FOR waiting';
+RESET MASTER;
+SET debug_sync= 'now SIGNAL go';
+connection master1;
+connection master;
+*** RESET MASTER works when no concurrent reader.
+RESET MASTER;
+DROP TABLE t1;
+include/save_master_gtid.inc
+connection slave;
+SET GLOBAL gtid_slave_pos= '';
+include/start_slave.inc
+include/sync_with_master_gtid.inc
+include/rpl_end.inc
diff --git a/mysql-test/suite/rpl/r/rpl_row_001.result b/mysql-test/suite/rpl/r/rpl_row_001.result
index 926404d29be09..f5813ed49d8a7 100644
--- a/mysql-test/suite/rpl/r/rpl_row_001.result
+++ b/mysql-test/suite/rpl/r/rpl_row_001.result
@@ -2,10 +2,11 @@ include/master-slave.inc
[connection master]
CREATE TABLE t1 (n INT);
connection slave;
+include/stop_slave.inc
connection master;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
connection slave;
-STOP SLAVE;
include/reset_slave.inc
connection master;
SELECT COUNT(*) FROM t1;
diff --git a/mysql-test/suite/rpl/r/rpl_row_basic_2myisam.result b/mysql-test/suite/rpl/r/rpl_row_basic_2myisam.result
index 0fc957ec2601b..a164b75a231be 100644
--- a/mysql-test/suite/rpl/r/rpl_row_basic_2myisam.result
+++ b/mysql-test/suite/rpl/r/rpl_row_basic_2myisam.result
@@ -5,7 +5,10 @@ connection master;
CREATE TABLE t1 (C1 CHAR(1), C2 CHAR(1), INDEX (C1)) ENGINE = 'MYISAM' ;
SELECT * FROM t1;
C1 C2
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
SELECT * FROM t1;
C1 C2
connection master;
@@ -19,7 +22,6 @@ A C
X X
X Y
X Z
-connection slave;
SELECT * FROM t1 ORDER BY C1,C2;
C1 C2
A A
@@ -29,6 +31,10 @@ X X
X Y
X Z
connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection master;
DELETE FROM t1 WHERE C1 = C2;
SELECT * FROM t1 ORDER BY C1,C2;
C1 C2
@@ -36,7 +42,10 @@ A B
A C
X Y
X Z
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
SELECT * FROM t1 ORDER BY C1,C2;
C1 C2
A B
@@ -51,7 +60,10 @@ A B
A I
X Y
X Z
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
SELECT * FROM t1 ORDER BY C1,C2;
C1 C2
A B
@@ -71,7 +83,6 @@ A B
A I
X Y
X Z
-connection slave;
SELECT * FROM t1 ORDER BY c1,c2;
C1 C2
A B
@@ -79,6 +90,10 @@ A I
X Y
X Z
connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection master;
CREATE TABLE t2 (c1 INT, c12 char(1), c2 INT, PRIMARY KEY (c1)) ENGINE = 'MYISAM' ;
INSERT INTO t2
VALUES (1,'A',2), (2,'A',4), (3,'A',9), (4,'A',15), (5,'A',25),
@@ -101,7 +116,10 @@ c1 c12 c2
5 A 25
8 A 64
9 A 81
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
SELECT * FROM t2 ORDER BY c1,c2;
c1 c12 c2
1 A 2
@@ -133,7 +151,10 @@ c1 c12 c2
7 A 49
8 A 64
9 A 81
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
SELECT * FROM t2 WHERE c2 = c1 * c1 ORDER BY c1,c2;
c1 c12 c2
1 A 1
@@ -158,7 +179,10 @@ c1 c12 c2
7 A 49
8 A 64
9 A 81
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
SELECT * FROM t2 ORDER BY c1,c2;
c1 c12 c2
1 A 1
@@ -181,7 +205,10 @@ c1 c12 c2
6 A 36
7 A 49
9 A 81
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
SELECT * FROM t2 ORDER BY c1,c2;
c1 c12 c2
1 A 1
@@ -205,7 +232,10 @@ A C 1 B 2
X X 3 B 1
X Y 2 B 1
X Z 2 B 2
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
SELECT * FROM t3 ORDER BY C1,C2;
C1 C2 pk1 C3 pk2
A A 3 B 2
@@ -222,7 +252,10 @@ A B 1 B 1
A C 1 B 2
X Y 2 B 1
X Z 2 B 2
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
SELECT * FROM t3 ORDER BY C1,C2;
C1 C2 pk1 C3 pk2
A B 1 B 1
@@ -237,7 +270,10 @@ A B 1 B 1
A I 1 B 2
X Y 2 B 1
X Z 2 B 2
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
SELECT * FROM t3 ORDER BY C1,C2;
C1 C2 pk1 C3 pk2
A B 1 B 1
@@ -256,7 +292,10 @@ X X 3
A C 4
X Z 5
A A 6
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
SELECT * FROM t6 ORDER BY C3;
C1 C2 C3
A B 1
@@ -273,7 +312,10 @@ A B 1
X Y 2
A C 4
X Z 5
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
SELECT * FROM t6 ORDER BY C3;
C1 C2 C3
A B 1
@@ -288,7 +330,10 @@ A B 1
X Y 2
A I 4
X Z 5
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
SELECT * FROM t6 ORDER BY C3;
C1 C2 C3
A B 1
@@ -344,7 +389,10 @@ X Q 5 7 R 49 X Y 2 S 1
X Q 5 7 R 49 X Z 2 S 2
X Q 5 9 R 81 X Y 2 S 1
X Q 5 9 R 81 X Z 2 S 2
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
SELECT * FROM t5,t2,t3 WHERE t5.C2='Q' AND t2.c12='R' AND t3.C3 ='S' ORDER BY t5.C3,t2.c1,t3.pk1,t3.pk2;
C1 C2 C3 c1 c12 c2 C1 C2 pk1 C3 pk2
X Q 2 1 R 1 X Y 2 S 1
@@ -395,13 +443,19 @@ INSERT INTO t4 SET C1 = 1;
SELECT C1,HEX(B1),HEX(B2) FROM t4 ORDER BY C1;
C1 HEX(B1) HEX(B2)
1 NULL 0
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
SELECT C1,HEX(B1),HEX(B2) FROM t4 ORDER BY C1;
C1 HEX(B1) HEX(B2)
1 NULL 0
connection master;
CREATE TABLE t7 (C1 INT PRIMARY KEY, C2 INT) ENGINE = 'MYISAM' ;
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
--- original values ---
INSERT INTO t7 VALUES (1,3), (2,6), (3,9);
SELECT * FROM t7 ORDER BY C1;
@@ -418,7 +472,10 @@ C1 C2
1 2
2 4
3 6
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
set @@global.slave_exec_mode= default;
--- old values should be overwritten by replicated values ---
SELECT * FROM t7 ORDER BY C1;
@@ -438,7 +495,10 @@ ERROR 23000: Duplicate entry '99' for key 'c'
SELECT * FROM t8 ORDER BY a;
a b c
99 99 99
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
SELECT * FROM t8 ORDER BY a;
a b c
99 99 99
@@ -452,7 +512,10 @@ a b c
set @@global.slave_exec_mode= 'IDEMPOTENT';
connection master;
INSERT INTO t8 VALUES (2,4,8);
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
set @@global.slave_exec_mode= default;
SELECT * FROM t8 ORDER BY a;
a b c
@@ -463,18 +526,27 @@ a b c
**** Test for BUG#31552 ****
connection master;
DELETE FROM t1;
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
include/rpl_reset.inc
connection master;
INSERT INTO t1 VALUES ('K','K'), ('L','L'), ('M','M');
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
set @@global.slave_exec_mode= 'IDEMPOTENT';
DELETE FROM t1 WHERE C1 = 'L';
connection master;
DELETE FROM t1;
SELECT COUNT(*) FROM t1 ORDER BY c1,c2;
COUNT(*) 0
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
set @@global.slave_exec_mode= default;
include/check_slave_is_running.inc
SELECT COUNT(*) FROM t1 ORDER BY c1,c2;
@@ -485,13 +557,19 @@ DROP TABLE IF EXISTS t1;
CREATE TABLE t1 (a TIMESTAMP, b DATETIME, c DATE);
INSERT INTO t1 VALUES(
'2005-11-14 01:01:01', '2005-11-14 01:01:02', '2005-11-14');
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
SELECT * FROM t1;
a b c
2005-11-14 01:01:01 2005-11-14 01:01:02 2005-11-14
connection master;
DROP TABLE IF EXISTS t1,t2,t3,t4,t5,t6,t7,t8;
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection master;
CREATE TABLE t1 (i INT NOT NULL,
c CHAR(16) CHARACTER SET utf8 NOT NULL,
@@ -499,13 +577,19 @@ j INT NOT NULL) ENGINE = 'MYISAM' ;
CREATE TABLE t2 (i INT NOT NULL,
c CHAR(16) CHARACTER SET utf8 NOT NULL,
j INT NOT NULL) ENGINE = 'MYISAM' ;
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
ALTER TABLE t2 MODIFY c CHAR(128) CHARACTER SET utf8 NOT NULL;
connection master;
CREATE TABLE t3 (i INT NOT NULL,
c CHAR(128) CHARACTER SET utf8 NOT NULL,
j INT NOT NULL) ENGINE = 'MYISAM' ;
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
ALTER TABLE t3 MODIFY c CHAR(16) CHARACTER SET utf8 NOT NULL;
connection master;
CREATE TABLE t4 (i INT NOT NULL,
@@ -514,13 +598,19 @@ j INT NOT NULL) ENGINE = 'MYISAM' ;
CREATE TABLE t5 (i INT NOT NULL,
c CHAR(255) CHARACTER SET utf8 NOT NULL,
j INT NOT NULL) ENGINE = 'MYISAM' ;
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
ALTER TABLE t5 MODIFY c CHAR(16) CHARACTER SET utf8 NOT NULL;
connection master;
CREATE TABLE t6 (i INT NOT NULL,
c CHAR(255) CHARACTER SET utf8 NOT NULL,
j INT NOT NULL) ENGINE = 'MYISAM' ;
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
ALTER TABLE t6 MODIFY c CHAR(128) CHARACTER SET utf8 NOT NULL;
connection master;
CREATE TABLE t7 (i INT NOT NULL,
@@ -533,13 +623,19 @@ SET GLOBAL SLAVE_TYPE_CONVERSIONS = 'ALL_NON_LOSSY';
connection master;
INSERT INTO t1 VALUES (1, "", 1);
INSERT INTO t1 VALUES (2, repeat(_utf8'a', 16), 2);
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
include/diff_tables.inc [master:t1, slave:t1]
[expecting slave to replicate correctly]
connection master;
INSERT INTO t2 VALUES (1, "", 1);
INSERT INTO t2 VALUES (2, repeat(_utf8'a', 16), 2);
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
include/diff_tables.inc [master:t2, slave:t2]
connection slave;
SET GLOBAL SLAVE_TYPE_CONVERSIONS = @saved_slave_type_conversions;
@@ -552,7 +648,10 @@ include/rpl_reset.inc
connection master;
INSERT INTO t4 VALUES (1, "", 1);
INSERT INTO t4 VALUES (2, repeat(_utf8'a', 128), 2);
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
include/diff_tables.inc [master:t4, slave:t4]
[expecting slave to stop]
connection master;
@@ -574,22 +673,34 @@ include/rpl_reset.inc
connection master;
INSERT INTO t7 VALUES (1, "", 1);
INSERT INTO t7 VALUES (2, repeat(_utf8'a', 255), 2);
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
include/diff_tables.inc [master:t7, slave:t7]
connection master;
drop table t1, t2, t3, t4, t5, t6, t7;
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection master;
CREATE TABLE t1 (a INT PRIMARY KEY) ENGINE='MYISAM' ;
INSERT INTO t1 VALUES (1), (2), (3);
UPDATE t1 SET a = 10;
ERROR 23000: Duplicate entry '10' for key 'PRIMARY'
INSERT INTO t1 VALUES (4);
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
include/diff_tables.inc [master:t1, slave:t1]
connection master;
drop table t1;
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection master;
DROP TABLE IF EXISTS t1, t2;
CREATE TABLE t1 (
@@ -642,7 +753,10 @@ UPDATE t1 SET `pk` = 6 ORDER BY `int_key` LIMIT 6;
ERROR 23000: Duplicate entry '6' for key 'PRIMARY'
DELETE FROM t2 WHERE `pk` < 7 LIMIT 1;
UPDATE t1 SET `int_key` = 4 ORDER BY `pk` LIMIT 6;
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
*** results: t2 must be consistent ****
include/diff_tables.inc [master:t2, slave:t2]
connection master;
@@ -662,11 +776,17 @@ UPDATE t1 SET a = 9 WHERE a < 3;
INSERT INTO t1 ( a ) VALUES ( 3 );
UPDATE t1 SET a = 0 WHERE a < 4;
UPDATE t1 SET a = 8 WHERE a < 5;
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
include/diff_tables.inc [master:t1, slave:t1]
connection master;
drop table t1;
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection master;
connection master;
CREATE TABLE t1 (a bit) ENGINE='MYISAM' ;
@@ -709,11 +829,17 @@ DELETE FROM t1 WHERE a < 3 LIMIT 0;
UPDATE t1 SET a = 8 WHERE a = 5 LIMIT 2;
INSERT INTO t1 ( a ) VALUES ( 1 );
UPDATE IGNORE t1 SET a = 9 WHERE a < 5 LIMIT 3;
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
include/diff_tables.inc [master:t1, slave:t1]
connection master;
drop table t1;
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection slave;
call mtr.add_suppression("Can't find record in 't1'");
include/rpl_end.inc
diff --git a/mysql-test/suite/rpl/r/rpl_row_basic_3innodb.result b/mysql-test/suite/rpl/r/rpl_row_basic_3innodb.result
index 3cc5a0e2faed8..9ce5f560ed863 100644
--- a/mysql-test/suite/rpl/r/rpl_row_basic_3innodb.result
+++ b/mysql-test/suite/rpl/r/rpl_row_basic_3innodb.result
@@ -6,7 +6,10 @@ connection master;
CREATE TABLE t1 (C1 CHAR(1), C2 CHAR(1), INDEX (C1)) ENGINE = 'INNODB' ;
SELECT * FROM t1;
C1 C2
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
SELECT * FROM t1;
C1 C2
connection master;
@@ -20,7 +23,6 @@ A C
X X
X Y
X Z
-connection slave;
SELECT * FROM t1 ORDER BY C1,C2;
C1 C2
A A
@@ -30,6 +32,10 @@ X X
X Y
X Z
connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection master;
DELETE FROM t1 WHERE C1 = C2;
SELECT * FROM t1 ORDER BY C1,C2;
C1 C2
@@ -37,7 +43,10 @@ A B
A C
X Y
X Z
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
SELECT * FROM t1 ORDER BY C1,C2;
C1 C2
A B
@@ -52,7 +61,10 @@ A B
A I
X Y
X Z
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
SELECT * FROM t1 ORDER BY C1,C2;
C1 C2
A B
@@ -72,7 +84,6 @@ A B
A I
X Y
X Z
-connection slave;
SELECT * FROM t1 ORDER BY c1,c2;
C1 C2
A B
@@ -80,6 +91,10 @@ A I
X Y
X Z
connection master;
+include/save_master_gtid.inc
+connection slave;
+include/sync_with_master_gtid.inc
+connection master;
CREATE TABLE t2 (c1 INT, c12 char(1), c2 INT, PRIMARY KEY (c1)) ENGINE = 'INNODB' ;
INSERT INTO t2
VALUES (1,'A',2), (2,'A',4), (3,'A',9), (4,'A',15), (5,'A',25),
@@ -102,7 +117,10 @@ c1 c12 c2
5 A 25
8 A 64
9 A 81
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
SELECT * FROM t2 ORDER BY c1,c2;
c1 c12 c2
1 A 2
@@ -134,7 +152,10 @@ c1 c12 c2
7 A 49
8 A 64
9 A 81
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
SELECT * FROM t2 WHERE c2 = c1 * c1 ORDER BY c1,c2;
c1 c12 c2
1 A 1
@@ -159,7 +180,10 @@ c1 c12 c2
7 A 49
8 A 64
9 A 81
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
SELECT * FROM t2 ORDER BY c1,c2;
c1 c12 c2
1 A 1
@@ -182,7 +206,10 @@ c1 c12 c2
6 A 36
7 A 49
9 A 81
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
SELECT * FROM t2 ORDER BY c1,c2;
c1 c12 c2
1 A 1
@@ -206,7 +233,10 @@ A C 1 B 2
X X 3 B 1
X Y 2 B 1
X Z 2 B 2
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
SELECT * FROM t3 ORDER BY C1,C2;
C1 C2 pk1 C3 pk2
A A 3 B 2
@@ -223,7 +253,10 @@ A B 1 B 1
A C 1 B 2
X Y 2 B 1
X Z 2 B 2
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
SELECT * FROM t3 ORDER BY C1,C2;
C1 C2 pk1 C3 pk2
A B 1 B 1
@@ -238,7 +271,10 @@ A B 1 B 1
A I 1 B 2
X Y 2 B 1
X Z 2 B 2
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
SELECT * FROM t3 ORDER BY C1,C2;
C1 C2 pk1 C3 pk2
A B 1 B 1
@@ -257,7 +293,10 @@ X X 3
A C 4
X Z 5
A A 6
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
SELECT * FROM t6 ORDER BY C3;
C1 C2 C3
A B 1
@@ -274,7 +313,10 @@ A B 1
X Y 2
A C 4
X Z 5
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
SELECT * FROM t6 ORDER BY C3;
C1 C2 C3
A B 1
@@ -289,7 +331,10 @@ A B 1
X Y 2
A I 4
X Z 5
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
SELECT * FROM t6 ORDER BY C3;
C1 C2 C3
A B 1
@@ -345,7 +390,10 @@ X Q 5 7 R 49 X Y 2 S 1
X Q 5 7 R 49 X Z 2 S 2
X Q 5 9 R 81 X Y 2 S 1
X Q 5 9 R 81 X Z 2 S 2
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
SELECT * FROM t5,t2,t3 WHERE t5.C2='Q' AND t2.c12='R' AND t3.C3 ='S' ORDER BY t5.C3,t2.c1,t3.pk1,t3.pk2;
C1 C2 C3 c1 c12 c2 C1 C2 pk1 C3 pk2
X Q 2 1 R 1 X Y 2 S 1
@@ -399,14 +447,20 @@ INSERT INTO t4 SET C1 = 1;
SELECT C1,HEX(B1),HEX(B2) FROM t4 ORDER BY C1;
C1 HEX(B1) HEX(B2)
1 NULL 0
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
SELECT C1,HEX(B1),HEX(B2) FROM t4 ORDER BY C1;
C1 HEX(B1) HEX(B2)
1 NULL 0
SET GLOBAL SLAVE_TYPE_CONVERSIONS = @saved_slave_type_conversions;
connection master;
CREATE TABLE t7 (C1 INT PRIMARY KEY, C2 INT) ENGINE = 'INNODB' ;
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
--- original values ---
INSERT INTO t7 VALUES (1,3), (2,6), (3,9);
SELECT * FROM t7 ORDER BY C1;
@@ -423,7 +477,10 @@ C1 C2
1 2
2 4
3 6
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
set @@global.slave_exec_mode= default;
--- old values should be overwritten by replicated values ---
SELECT * FROM t7 ORDER BY C1;
@@ -443,7 +500,10 @@ ERROR 23000: Duplicate entry '99' for key 'c'
SELECT * FROM t8 ORDER BY a;
a b c
99 99 99
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
SELECT * FROM t8 ORDER BY a;
a b c
99 99 99
@@ -457,7 +517,10 @@ a b c
set @@global.slave_exec_mode= 'IDEMPOTENT';
connection master;
INSERT INTO t8 VALUES (2,4,8);
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
set @@global.slave_exec_mode= default;
SELECT * FROM t8 ORDER BY a;
a b c
@@ -468,18 +531,27 @@ a b c
**** Test for BUG#31552 ****
connection master;
DELETE FROM t1;
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
include/rpl_reset.inc
connection master;
INSERT INTO t1 VALUES ('K','K'), ('L','L'), ('M','M');
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
set @@global.slave_exec_mode= 'IDEMPOTENT';
DELETE FROM t1 WHERE C1 = 'L';
connection master;
DELETE FROM t1;
SELECT COUNT(*) FROM t1 ORDER BY c1,c2;
COUNT(*) 0
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
set @@global.slave_exec_mode= default;
include/check_slave_is_running.inc
SELECT COUNT(*) FROM t1 ORDER BY c1,c2;
@@ -490,13 +562,19 @@ DROP TABLE IF EXISTS t1;
CREATE TABLE t1 (a TIMESTAMP, b DATETIME, c DATE);
INSERT INTO t1 VALUES(
'2005-11-14 01:01:01', '2005-11-14 01:01:02', '2005-11-14');
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
SELECT * FROM t1;
a b c
2005-11-14 01:01:01 2005-11-14 01:01:02 2005-11-14
connection master;
DROP TABLE IF EXISTS t1,t2,t3,t4,t5,t6,t7,t8;
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection master;
CREATE TABLE t1 (i INT NOT NULL,
c CHAR(16) CHARACTER SET utf8 NOT NULL,
@@ -504,13 +582,19 @@ j INT NOT NULL) ENGINE = 'INNODB' ;
CREATE TABLE t2 (i INT NOT NULL,
c CHAR(16) CHARACTER SET utf8 NOT NULL,
j INT NOT NULL) ENGINE = 'INNODB' ;
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
ALTER TABLE t2 MODIFY c CHAR(128) CHARACTER SET utf8 NOT NULL;
connection master;
CREATE TABLE t3 (i INT NOT NULL,
c CHAR(128) CHARACTER SET utf8 NOT NULL,
j INT NOT NULL) ENGINE = 'INNODB' ;
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
ALTER TABLE t3 MODIFY c CHAR(16) CHARACTER SET utf8 NOT NULL;
connection master;
CREATE TABLE t4 (i INT NOT NULL,
@@ -519,13 +603,19 @@ j INT NOT NULL) ENGINE = 'INNODB' ;
CREATE TABLE t5 (i INT NOT NULL,
c CHAR(255) CHARACTER SET utf8 NOT NULL,
j INT NOT NULL) ENGINE = 'INNODB' ;
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
ALTER TABLE t5 MODIFY c CHAR(16) CHARACTER SET utf8 NOT NULL;
connection master;
CREATE TABLE t6 (i INT NOT NULL,
c CHAR(255) CHARACTER SET utf8 NOT NULL,
j INT NOT NULL) ENGINE = 'INNODB' ;
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
ALTER TABLE t6 MODIFY c CHAR(128) CHARACTER SET utf8 NOT NULL;
connection master;
CREATE TABLE t7 (i INT NOT NULL,
@@ -538,13 +628,19 @@ SET GLOBAL SLAVE_TYPE_CONVERSIONS = 'ALL_NON_LOSSY';
connection master;
INSERT INTO t1 VALUES (1, "", 1);
INSERT INTO t1 VALUES (2, repeat(_utf8'a', 16), 2);
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
include/diff_tables.inc [master:t1, slave:t1]
[expecting slave to replicate correctly]
connection master;
INSERT INTO t2 VALUES (1, "", 1);
INSERT INTO t2 VALUES (2, repeat(_utf8'a', 16), 2);
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
include/diff_tables.inc [master:t2, slave:t2]
connection slave;
SET GLOBAL SLAVE_TYPE_CONVERSIONS = @saved_slave_type_conversions;
@@ -557,7 +653,10 @@ include/rpl_reset.inc
connection master;
INSERT INTO t4 VALUES (1, "", 1);
INSERT INTO t4 VALUES (2, repeat(_utf8'a', 128), 2);
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
include/diff_tables.inc [master:t4, slave:t4]
[expecting slave to stop]
connection master;
@@ -579,22 +678,34 @@ include/rpl_reset.inc
connection master;
INSERT INTO t7 VALUES (1, "", 1);
INSERT INTO t7 VALUES (2, repeat(_utf8'a', 255), 2);
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
include/diff_tables.inc [master:t7, slave:t7]
connection master;
drop table t1, t2, t3, t4, t5, t6, t7;
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection master;
CREATE TABLE t1 (a INT PRIMARY KEY) ENGINE='INNODB' ;
INSERT INTO t1 VALUES (1), (2), (3);
UPDATE t1 SET a = 10;
ERROR 23000: Duplicate entry '10' for key 'PRIMARY'
INSERT INTO t1 VALUES (4);
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
include/diff_tables.inc [master:t1, slave:t1]
connection master;
drop table t1;
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection master;
DROP TABLE IF EXISTS t1, t2;
CREATE TABLE t1 (
@@ -647,7 +758,10 @@ UPDATE t1 SET `pk` = 6 ORDER BY `int_key` LIMIT 6;
ERROR 23000: Duplicate entry '6' for key 'PRIMARY'
DELETE FROM t2 WHERE `pk` < 7 LIMIT 1;
UPDATE t1 SET `int_key` = 4 ORDER BY `pk` LIMIT 6;
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
*** results: t2 must be consistent ****
include/diff_tables.inc [master:t2, slave:t2]
connection master;
@@ -667,11 +781,17 @@ UPDATE t1 SET a = 9 WHERE a < 3;
INSERT INTO t1 ( a ) VALUES ( 3 );
UPDATE t1 SET a = 0 WHERE a < 4;
UPDATE t1 SET a = 8 WHERE a < 5;
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
include/diff_tables.inc [master:t1, slave:t1]
connection master;
drop table t1;
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
connection master;
connection slave;
SET @saved_slave_type_conversions = @@SLAVE_TYPE_CONVERSIONS;
@@ -717,10 +837,16 @@ DELETE FROM t1 WHERE a < 3 LIMIT 0;
UPDATE t1 SET a = 8 WHERE a = 5 LIMIT 2;
INSERT INTO t1 ( a ) VALUES ( 1 );
UPDATE IGNORE t1 SET a = 9 WHERE a < 5 LIMIT 3;
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
SET GLOBAL SLAVE_TYPE_CONVERSIONS = @saved_slave_type_conversions;
include/diff_tables.inc [master:t1, slave:t1]
connection master;
drop table t1;
+connection master;
+include/save_master_gtid.inc
connection slave;
+include/sync_with_master_gtid.inc
include/rpl_end.inc
diff --git a/mysql-test/suite/rpl/r/rpl_row_colSize.result b/mysql-test/suite/rpl/r/rpl_row_colSize.result
index dbfb9325f9c56..3c78409e07705 100644
--- a/mysql-test/suite/rpl/r/rpl_row_colSize.result
+++ b/mysql-test/suite/rpl/r/rpl_row_colSize.result
@@ -15,6 +15,7 @@ include/reset_slave.inc
CREATE TABLE t1 (a DECIMAL(5,2));
connection master;
CREATE TABLE t1 (a DECIMAL(20, 10));
+include/kill_binlog_dump_threads.inc
RESET MASTER;
INSERT INTO t1 VALUES (901251.90125);
connection slave;
@@ -29,6 +30,7 @@ include/reset_slave.inc
Warnings:
Note 4190 RESET SLAVE is implicitly changing the value of 'Using_Gtid' from 'No' to 'Slave_Pos'
connection master;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
connection slave;
START SLAVE;
@@ -42,6 +44,7 @@ Note 4190 RESET SLAVE is implicitly changing the value of 'Using_Gtid' from 'No'
CREATE TABLE t1 (a DECIMAL(27, 9));
connection master;
CREATE TABLE t1 (a DECIMAL(27, 18));
+include/kill_binlog_dump_threads.inc
RESET MASTER;
INSERT INTO t1 VALUES (901251.90125);
connection slave;
@@ -56,6 +59,7 @@ include/reset_slave.inc
Warnings:
Note 4190 RESET SLAVE is implicitly changing the value of 'Using_Gtid' from 'No' to 'Slave_Pos'
connection master;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
connection slave;
START SLAVE;
@@ -69,6 +73,7 @@ Note 4190 RESET SLAVE is implicitly changing the value of 'Using_Gtid' from 'No'
CREATE TABLE t1 (a NUMERIC(5,2));
connection master;
CREATE TABLE t1 (a NUMERIC(20, 10));
+include/kill_binlog_dump_threads.inc
RESET MASTER;
INSERT INTO t1 VALUES (901251.90125);
connection slave;
@@ -83,6 +88,7 @@ include/reset_slave.inc
Warnings:
Note 4190 RESET SLAVE is implicitly changing the value of 'Using_Gtid' from 'No' to 'Slave_Pos'
connection master;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
connection slave;
START SLAVE;
@@ -97,6 +103,7 @@ Note 4190 RESET SLAVE is implicitly changing the value of 'Using_Gtid' from 'No'
CREATE TABLE t1 (a FLOAT(20));
connection master;
CREATE TABLE t1 (a FLOAT(47));
+include/kill_binlog_dump_threads.inc
RESET MASTER;
INSERT INTO t1 VALUES (901251.90125);
connection slave;
@@ -111,6 +118,7 @@ include/reset_slave.inc
Warnings:
Note 4190 RESET SLAVE is implicitly changing the value of 'Using_Gtid' from 'No' to 'Slave_Pos'
connection master;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
connection slave;
START SLAVE;
@@ -125,6 +133,7 @@ Note 4190 RESET SLAVE is implicitly changing the value of 'Using_Gtid' from 'No'
CREATE TABLE t1 (a BIT(5));
connection master;
CREATE TABLE t1 (a BIT(64));
+include/kill_binlog_dump_threads.inc
RESET MASTER;
INSERT INTO t1 VALUES (B'10101');
connection slave;
@@ -139,6 +148,7 @@ include/reset_slave.inc
Warnings:
Note 4190 RESET SLAVE is implicitly changing the value of 'Using_Gtid' from 'No' to 'Slave_Pos'
connection master;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
connection slave;
START SLAVE;
@@ -152,6 +162,7 @@ Note 4190 RESET SLAVE is implicitly changing the value of 'Using_Gtid' from 'No'
CREATE TABLE t1 (a BIT(11));
connection master;
CREATE TABLE t1 (a BIT(12));
+include/kill_binlog_dump_threads.inc
RESET MASTER;
INSERT INTO t1 VALUES (B'10101');
connection slave;
@@ -166,6 +177,7 @@ include/reset_slave.inc
Warnings:
Note 4190 RESET SLAVE is implicitly changing the value of 'Using_Gtid' from 'No' to 'Slave_Pos'
connection master;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
connection slave;
START SLAVE;
@@ -180,6 +192,7 @@ Note 4190 RESET SLAVE is implicitly changing the value of 'Using_Gtid' from 'No'
CREATE TABLE t1 (a SET('4'));
connection master;
CREATE TABLE t1 (a SET('1','2','3','4','5','6','7','8','9'));
+include/kill_binlog_dump_threads.inc
RESET MASTER;
INSERT INTO t1 VALUES ('4');
connection slave;
@@ -194,6 +207,7 @@ include/reset_slave.inc
Warnings:
Note 4190 RESET SLAVE is implicitly changing the value of 'Using_Gtid' from 'No' to 'Slave_Pos'
connection master;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
connection slave;
START SLAVE;
@@ -208,6 +222,7 @@ Note 4190 RESET SLAVE is implicitly changing the value of 'Using_Gtid' from 'No'
CREATE TABLE t1 (a CHAR(10));
connection master;
CREATE TABLE t1 (a CHAR(20));
+include/kill_binlog_dump_threads.inc
RESET MASTER;
INSERT INTO t1 VALUES ('This is a test.');
connection slave;
@@ -222,6 +237,7 @@ include/reset_slave.inc
Warnings:
Note 4190 RESET SLAVE is implicitly changing the value of 'Using_Gtid' from 'No' to 'Slave_Pos'
connection master;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
connection slave;
START SLAVE;
@@ -267,6 +283,7 @@ CREATE TABLE t1 (a ENUM(
'281','282','283','284','285','286','287','288','289',
'291','292','293','294','295','296','297','298','299'
));
+include/kill_binlog_dump_threads.inc
RESET MASTER;
INSERT INTO t1 VALUES ('44');
connection slave;
@@ -281,6 +298,7 @@ include/reset_slave.inc
Warnings:
Note 4190 RESET SLAVE is implicitly changing the value of 'Using_Gtid' from 'No' to 'Slave_Pos'
connection master;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
connection slave;
START SLAVE;
@@ -295,6 +313,7 @@ Note 4190 RESET SLAVE is implicitly changing the value of 'Using_Gtid' from 'No'
CREATE TABLE t1 (a VARCHAR(100));
connection master;
CREATE TABLE t1 (a VARCHAR(2000));
+include/kill_binlog_dump_threads.inc
RESET MASTER;
INSERT INTO t1 VALUES ('This is a test.');
connection slave;
@@ -309,6 +328,7 @@ include/reset_slave.inc
Warnings:
Note 4190 RESET SLAVE is implicitly changing the value of 'Using_Gtid' from 'No' to 'Slave_Pos'
connection master;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
connection slave;
START SLAVE;
@@ -322,6 +342,7 @@ Note 4190 RESET SLAVE is implicitly changing the value of 'Using_Gtid' from 'No'
CREATE TABLE t1 (a VARCHAR(10));
connection master;
CREATE TABLE t1 (a VARCHAR(200));
+include/kill_binlog_dump_threads.inc
RESET MASTER;
INSERT INTO t1 VALUES ('This is a test.');
connection slave;
@@ -336,6 +357,7 @@ include/reset_slave.inc
Warnings:
Note 4190 RESET SLAVE is implicitly changing the value of 'Using_Gtid' from 'No' to 'Slave_Pos'
connection master;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
connection slave;
START SLAVE;
@@ -349,6 +371,7 @@ Note 4190 RESET SLAVE is implicitly changing the value of 'Using_Gtid' from 'No'
CREATE TABLE t1 (a VARCHAR(1000));
connection master;
CREATE TABLE t1 (a VARCHAR(2000));
+include/kill_binlog_dump_threads.inc
RESET MASTER;
INSERT INTO t1 VALUES ('This is a test.');
connection slave;
@@ -363,6 +386,7 @@ include/reset_slave.inc
Warnings:
Note 4190 RESET SLAVE is implicitly changing the value of 'Using_Gtid' from 'No' to 'Slave_Pos'
connection master;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
connection slave;
START SLAVE;
@@ -377,6 +401,7 @@ Note 4190 RESET SLAVE is implicitly changing the value of 'Using_Gtid' from 'No'
CREATE TABLE t1 (a TINYBLOB);
connection master;
CREATE TABLE t1 (a LONGBLOB);
+include/kill_binlog_dump_threads.inc
RESET MASTER;
INSERT INTO t1 VALUES ('This is a test.');
connection slave;
@@ -391,6 +416,7 @@ include/reset_slave.inc
Warnings:
Note 4190 RESET SLAVE is implicitly changing the value of 'Using_Gtid' from 'No' to 'Slave_Pos'
connection master;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
connection slave;
START SLAVE;
diff --git a/mysql-test/suite/rpl/r/rpl_row_loaddata_concurrent.result b/mysql-test/suite/rpl/r/rpl_row_loaddata_concurrent.result
index bf63dc57c4b00..7044e95aefede 100644
--- a/mysql-test/suite/rpl/r/rpl_row_loaddata_concurrent.result
+++ b/mysql-test/suite/rpl/r/rpl_row_loaddata_concurrent.result
@@ -80,6 +80,7 @@ Warnings:
Warning 1948 Specified value for @@gtid_slave_pos contains no value for replication domain 0. This conflicts with the binary log which contains GTID 0-2-11. If MASTER_GTID_POS=CURRENT_POS is used, the binlog position will override the new value of @@gtid_slave_pos
include/check_slave_no_error.inc
connection master;
+include/kill_binlog_dump_threads.inc
reset master;
create table t2 (day date,id int(9),category enum('a','b','c'),name varchar(60),
unique(day)) engine=MyISAM;
diff --git a/mysql-test/suite/rpl/r/rpl_row_mysqlbinlog.result b/mysql-test/suite/rpl/r/rpl_row_mysqlbinlog.result
index 817b9b12c596c..1464dba606a40 100644
--- a/mysql-test/suite/rpl/r/rpl_row_mysqlbinlog.result
+++ b/mysql-test/suite/rpl/r/rpl_row_mysqlbinlog.result
@@ -89,6 +89,7 @@ connection slave;
stop slave;
include/wait_for_slave_to_stop.inc
connection master;
+include/kill_binlog_dump_threads.inc
reset master;
connection slave;
include/reset_slave.inc
@@ -219,6 +220,7 @@ connection slave;
stop slave;
include/wait_for_slave_to_stop.inc
connection master;
+include/kill_binlog_dump_threads.inc
reset master;
connection slave;
include/reset_slave.inc
@@ -354,6 +356,7 @@ connection slave;
stop slave;
include/wait_for_slave_to_stop.inc
connection master;
+include/kill_binlog_dump_threads.inc
reset master;
connection slave;
include/reset_slave.inc
diff --git a/mysql-test/suite/rpl/r/rpl_semi_sync.result b/mysql-test/suite/rpl/r/rpl_semi_sync.result
index 03e3443b31ed4..307d54ee78b66 100644
--- a/mysql-test/suite/rpl/r/rpl_semi_sync.result
+++ b/mysql-test/suite/rpl/r/rpl_semi_sync.result
@@ -20,7 +20,7 @@ include/reset_slave.inc
set global rpl_semi_sync_master_enabled= 0;
set global rpl_semi_sync_slave_enabled= 0;
connection master;
-reset master;
+include/reset_master.inc
set global rpl_semi_sync_master_enabled= 0;
set global rpl_semi_sync_slave_enabled= 0;
#
@@ -61,7 +61,7 @@ Rpl_semi_sync_master_status OFF
show status like 'Rpl_semi_sync_master_yes_tx';
Variable_name Value
Rpl_semi_sync_master_yes_tx 0
-reset master;
+include/reset_master.inc
connection slave;
[ default state of semi-sync on slave should be OFF ]
show variables like 'rpl_semi_sync_slave_enabled';
@@ -301,9 +301,10 @@ connection master;
create table t1 (a int) engine = ENGINE_TYPE;
drop table t1;
connection slave;
+include/stop_slave.inc
[ test reset master ]
connection master;
-reset master;
+include/reset_master.inc
show status like 'Rpl_semi_sync_master_status';
Variable_name Value
Rpl_semi_sync_master_status ON
@@ -314,7 +315,6 @@ show status like 'Rpl_semi_sync_master_yes_tx';
Variable_name Value
Rpl_semi_sync_master_yes_tx 0
connection slave;
-include/stop_slave.inc
include/reset_slave.inc
include/kill_binlog_dump_threads.inc
connection slave;
@@ -349,7 +349,7 @@ connection slave;
include/stop_slave.inc
include/reset_slave.inc
connection master;
-reset master;
+include/reset_master.inc
include/kill_binlog_dump_threads.inc
set sql_log_bin=0;
grant replication slave on *.* to rpl@127.0.0.1 identified by 'rpl_password';
diff --git a/mysql-test/suite/rpl/r/rpl_semi_sync_after_sync.result b/mysql-test/suite/rpl/r/rpl_semi_sync_after_sync.result
index a79b372f93762..f314d27b839b8 100644
--- a/mysql-test/suite/rpl/r/rpl_semi_sync_after_sync.result
+++ b/mysql-test/suite/rpl/r/rpl_semi_sync_after_sync.result
@@ -21,7 +21,7 @@ include/reset_slave.inc
set global rpl_semi_sync_master_enabled= 0;
set global rpl_semi_sync_slave_enabled= 0;
connection master;
-reset master;
+include/reset_master.inc
set global rpl_semi_sync_master_enabled= 0;
set global rpl_semi_sync_slave_enabled= 0;
#
@@ -62,7 +62,7 @@ Rpl_semi_sync_master_status OFF
show status like 'Rpl_semi_sync_master_yes_tx';
Variable_name Value
Rpl_semi_sync_master_yes_tx 0
-reset master;
+include/reset_master.inc
connection slave;
[ default state of semi-sync on slave should be OFF ]
show variables like 'rpl_semi_sync_slave_enabled';
@@ -302,9 +302,10 @@ connection master;
create table t1 (a int) engine = ENGINE_TYPE;
drop table t1;
connection slave;
+include/stop_slave.inc
[ test reset master ]
connection master;
-reset master;
+include/reset_master.inc
show status like 'Rpl_semi_sync_master_status';
Variable_name Value
Rpl_semi_sync_master_status ON
@@ -315,7 +316,6 @@ show status like 'Rpl_semi_sync_master_yes_tx';
Variable_name Value
Rpl_semi_sync_master_yes_tx 0
connection slave;
-include/stop_slave.inc
include/reset_slave.inc
include/kill_binlog_dump_threads.inc
connection slave;
@@ -350,7 +350,7 @@ connection slave;
include/stop_slave.inc
include/reset_slave.inc
connection master;
-reset master;
+include/reset_master.inc
include/kill_binlog_dump_threads.inc
set sql_log_bin=0;
grant replication slave on *.* to rpl@127.0.0.1 identified by 'rpl_password';
diff --git a/mysql-test/suite/rpl/r/rpl_semi_sync_after_sync_row.result b/mysql-test/suite/rpl/r/rpl_semi_sync_after_sync_row.result
index 393c9c55a0dde..c96a294838e97 100644
--- a/mysql-test/suite/rpl/r/rpl_semi_sync_after_sync_row.result
+++ b/mysql-test/suite/rpl/r/rpl_semi_sync_after_sync_row.result
@@ -21,7 +21,7 @@ include/reset_slave.inc
set global rpl_semi_sync_master_enabled= 0;
set global rpl_semi_sync_slave_enabled= 0;
connection master;
-reset master;
+include/reset_master.inc
set global rpl_semi_sync_master_enabled= 0;
set global rpl_semi_sync_slave_enabled= 0;
#
@@ -62,7 +62,7 @@ Rpl_semi_sync_master_status OFF
show status like 'Rpl_semi_sync_master_yes_tx';
Variable_name Value
Rpl_semi_sync_master_yes_tx 0
-reset master;
+include/reset_master.inc
connection slave;
[ default state of semi-sync on slave should be OFF ]
show variables like 'rpl_semi_sync_slave_enabled';
@@ -302,9 +302,10 @@ connection master;
create table t1 (a int) engine = ENGINE_TYPE;
drop table t1;
connection slave;
+include/stop_slave.inc
[ test reset master ]
connection master;
-reset master;
+include/reset_master.inc
show status like 'Rpl_semi_sync_master_status';
Variable_name Value
Rpl_semi_sync_master_status ON
@@ -315,7 +316,6 @@ show status like 'Rpl_semi_sync_master_yes_tx';
Variable_name Value
Rpl_semi_sync_master_yes_tx 0
connection slave;
-include/stop_slave.inc
include/reset_slave.inc
include/kill_binlog_dump_threads.inc
connection slave;
@@ -350,7 +350,7 @@ connection slave;
include/stop_slave.inc
include/reset_slave.inc
connection master;
-reset master;
+include/reset_master.inc
include/kill_binlog_dump_threads.inc
set sql_log_bin=0;
grant replication slave on *.* to rpl@127.0.0.1 identified by 'rpl_password';
diff --git a/mysql-test/suite/rpl/r/rpl_semi_sync_fail_over.result b/mysql-test/suite/rpl/r/rpl_semi_sync_fail_over.result
index b751c0873fa23..0ac50b531f99a 100644
--- a/mysql-test/suite/rpl/r/rpl_semi_sync_fail_over.result
+++ b/mysql-test/suite/rpl/r/rpl_semi_sync_fail_over.result
@@ -3,6 +3,7 @@ include/master-slave.inc
connection server_2;
include/stop_slave.inc
connection server_1;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
SET @@global.max_binlog_size= 4096;
set @@global.rpl_semi_sync_master_enabled = 1;
@@ -276,9 +277,11 @@ include/stop_slave.inc
set global rpl_semi_sync_slave_enabled = 0;
set global rpl_semi_sync_master_enabled = 0;
set global rpl_semi_sync_master_wait_point=default;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
RESET SLAVE;
connection server_2;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
RESET SLAVE;
set @@global.rpl_semi_sync_master_enabled = 0;
diff --git a/mysql-test/suite/rpl/r/rpl_semi_sync_gtid_reconnect.result b/mysql-test/suite/rpl/r/rpl_semi_sync_gtid_reconnect.result
index 4285228f1628a..0a72904fea376 100644
--- a/mysql-test/suite/rpl/r/rpl_semi_sync_gtid_reconnect.result
+++ b/mysql-test/suite/rpl/r/rpl_semi_sync_gtid_reconnect.result
@@ -1,10 +1,12 @@
include/master-slave.inc
[connection master]
+connection slave;
+include/stop_slave.inc
connection master;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
SET @@GLOBAL.rpl_semi_sync_master_enabled = 1;
connection slave;
-include/stop_slave.inc
SET @@GLOBAL. rpl_semi_sync_slave_enabled = 1;
include/start_slave.inc
connection master;
@@ -22,6 +24,7 @@ include/sync_slave_io_with_master.inc
connection slave;
include/stop_slave_io.inc
connection master;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
SET @@global.gtid_binlog_state = '0-1-2';
connection slave;
diff --git a/mysql-test/suite/rpl/r/rpl_server_id2.result b/mysql-test/suite/rpl/r/rpl_server_id2.result
index 3d8eeb02ca675..033643177fce8 100644
--- a/mysql-test/suite/rpl/r/rpl_server_id2.result
+++ b/mysql-test/suite/rpl/r/rpl_server_id2.result
@@ -20,6 +20,7 @@ stop slave;
include/wait_for_slave_to_stop.inc
drop table t1;
connection master;
+include/kill_binlog_dump_threads.inc
reset master;
create table t1(n int);
create table t2(n int);
diff --git a/mysql-test/suite/rpl/r/rpl_slave_alias_replica.result b/mysql-test/suite/rpl/r/rpl_slave_alias_replica.result
index 0ede17132dfbf..8452d175bed93 100644
--- a/mysql-test/suite/rpl/r/rpl_slave_alias_replica.result
+++ b/mysql-test/suite/rpl/r/rpl_slave_alias_replica.result
@@ -32,6 +32,7 @@ include/wait_for_slave_sql_to_stop.inc
RESET REPLICA ALL;
set @@global.gtid_slave_pos= "";
connection master;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
CREATE TABLE t(f INT) ENGINE=INNODB;
INSERT INTO t VALUES (10);
diff --git a/mysql-test/suite/rpl/r/rpl_slave_skip.result b/mysql-test/suite/rpl/r/rpl_slave_skip.result
index 8896199f499e1..417a1a6f95ec3 100644
--- a/mysql-test/suite/rpl/r/rpl_slave_skip.result
+++ b/mysql-test/suite/rpl/r/rpl_slave_skip.result
@@ -65,6 +65,7 @@ STOP SLAVE;
include/wait_for_slave_to_stop.inc
include/reset_slave.inc
connection master;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
SET SESSION BINLOG_FORMAT=STATEMENT;
SET @foo = 12;
diff --git a/mysql-test/suite/rpl/r/rpl_start_alter_7.result b/mysql-test/suite/rpl/r/rpl_start_alter_7.result
index df7664d4aa2c1..93895612c9126 100644
--- a/mysql-test/suite/rpl/r/rpl_start_alter_7.result
+++ b/mysql-test/suite/rpl/r/rpl_start_alter_7.result
@@ -362,10 +362,12 @@ SET GLOBAL gtid_slave_pos= '';
connection server_1;
set global binlog_alter_two_phase=0;;
set global gtid_domain_id= 0;
+include/kill_binlog_dump_threads.inc
reset master;
connection server_2;
set global gtid_domain_id= 0;
set global binlog_alter_two_phase=0;
+include/kill_binlog_dump_threads.inc
reset master;
disconnect server_1;
disconnect server_2;
diff --git a/mysql-test/suite/rpl/r/rpl_start_alter_8.result b/mysql-test/suite/rpl/r/rpl_start_alter_8.result
index 406f2d1f6fc86..635c612fbaa9c 100644
--- a/mysql-test/suite/rpl/r/rpl_start_alter_8.result
+++ b/mysql-test/suite/rpl/r/rpl_start_alter_8.result
@@ -356,10 +356,12 @@ SET GLOBAL gtid_slave_pos= '';
connection server_1;
set global binlog_alter_two_phase=0;;
set global gtid_domain_id= 0;
+include/kill_binlog_dump_threads.inc
reset master;
connection server_2;
set global gtid_domain_id= 0;
set global binlog_alter_two_phase=0;;
+include/kill_binlog_dump_threads.inc
reset master;
disconnect server_1;
disconnect server_2;
diff --git a/mysql-test/suite/rpl/r/rpl_start_alter_mysqlbinlog_1.result b/mysql-test/suite/rpl/r/rpl_start_alter_mysqlbinlog_1.result
index b11804ed1c772..630bd4e325bf3 100644
--- a/mysql-test/suite/rpl/r/rpl_start_alter_mysqlbinlog_1.result
+++ b/mysql-test/suite/rpl/r/rpl_start_alter_mysqlbinlog_1.result
@@ -161,6 +161,7 @@ connection master;
select @@gtid_binlog_state;
@@gtid_binlog_state
0-1-410
+include/kill_binlog_dump_threads.inc
RESET master;
connection slave;
select @@gtid_binlog_state;
diff --git a/mysql-test/suite/rpl/r/rpl_start_alter_mysqlbinlog_2.result b/mysql-test/suite/rpl/r/rpl_start_alter_mysqlbinlog_2.result
index 55bec7d399815..fc9d06e94e6e6 100644
--- a/mysql-test/suite/rpl/r/rpl_start_alter_mysqlbinlog_2.result
+++ b/mysql-test/suite/rpl/r/rpl_start_alter_mysqlbinlog_2.result
@@ -413,10 +413,12 @@ SET GLOBAL gtid_slave_pos= '';
connection server_1;
SET GLOBAL binlog_alter_two_phase=@save_binlog_alter_two_phase;
SET GLOBAL gtid_domain_id= 0;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
connection server_2;
SET GLOBAL gtid_domain_id= 0;
SET GLOBAL binlog_alter_two_phase=@save_binlog_alter_two_phase;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
disconnect server_1;
disconnect server_2;
diff --git a/mysql-test/suite/rpl/r/rpl_stm_000001.result b/mysql-test/suite/rpl/r/rpl_stm_000001.result
index 5d6d61d175b4a..0e2abbfbf2343 100644
--- a/mysql-test/suite/rpl/r/rpl_stm_000001.result
+++ b/mysql-test/suite/rpl/r/rpl_stm_000001.result
@@ -45,10 +45,11 @@ connection slave;
connection master;
create table t1 (n int);
connection slave;
+stop slave;
connection master;
+include/kill_binlog_dump_threads.inc
reset master;
connection slave;
-stop slave;
include/reset_slave.inc
connection master;
connection slave;
diff --git a/mysql-test/suite/rpl/r/rpl_stm_loaddata_concurrent.result b/mysql-test/suite/rpl/r/rpl_stm_loaddata_concurrent.result
index a437b66dd7859..bb66a0d81426d 100644
--- a/mysql-test/suite/rpl/r/rpl_stm_loaddata_concurrent.result
+++ b/mysql-test/suite/rpl/r/rpl_stm_loaddata_concurrent.result
@@ -79,6 +79,7 @@ Warnings:
Warning 1948 Specified value for @@gtid_slave_pos contains no value for replication domain 0. This conflicts with the binary log which contains GTID 0-2-14. If MASTER_GTID_POS=CURRENT_POS is used, the binlog position will override the new value of @@gtid_slave_pos
include/check_slave_no_error.inc
connection master;
+include/kill_binlog_dump_threads.inc
reset master;
create table t2 (day date,id int(9),category enum('a','b','c'),name varchar(60),
unique(day)) engine=MyISAM;
diff --git a/mysql-test/suite/rpl/r/rpl_stm_start_stop_slave.result b/mysql-test/suite/rpl/r/rpl_stm_start_stop_slave.result
index 9575ea8989262..6ede72aa89d6b 100644
--- a/mysql-test/suite/rpl/r/rpl_stm_start_stop_slave.result
+++ b/mysql-test/suite/rpl/r/rpl_stm_start_stop_slave.result
@@ -100,10 +100,12 @@ include/start_slave.inc
connection master;
DROP TABLE t1;
connection slave;
+connection slave;
+include/stop_slave.inc
connection master;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
connection slave;
-include/stop_slave.inc
CHANGE MASTER TO master_log_pos=MASTER_POS, master_use_gtid=no;
START SLAVE;
include/wait_for_slave_param.inc [Last_IO_Errno]
@@ -114,6 +116,7 @@ RESET SLAVE;
Warnings:
Note 4190 RESET SLAVE is implicitly changing the value of 'Using_Gtid' from 'No' to 'Slave_Pos'
connection master;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
connection slave;
SET @@global.innodb_flush_log_at_trx_commit= @old_innodb_flush_log_at_trx_commit;
diff --git a/mysql-test/suite/rpl/r/rpl_stm_until.result b/mysql-test/suite/rpl/r/rpl_stm_until.result
index b51e3d847e3f8..2e717970a87ce 100644
--- a/mysql-test/suite/rpl/r/rpl_stm_until.result
+++ b/mysql-test/suite/rpl/r/rpl_stm_until.result
@@ -75,6 +75,7 @@ Note 4190 RESET SLAVE is implicitly changing the value of 'Using_Gtid' from 'No'
change master to master_host='127.0.0.1',master_port=MASTER_PORT, master_user='root';
connection master;
drop table if exists t1;
+include/kill_binlog_dump_threads.inc
reset master;
create table t1 (a int primary key auto_increment);
connection slave;
diff --git a/mysql-test/suite/rpl/r/rpl_system_versioning_partitions.result b/mysql-test/suite/rpl/r/rpl_system_versioning_partitions.result
index 8edc6996a051e..6718bcaf05cec 100644
--- a/mysql-test/suite/rpl/r/rpl_system_versioning_partitions.result
+++ b/mysql-test/suite/rpl/r/rpl_system_versioning_partitions.result
@@ -2,9 +2,15 @@ include/master-slave.inc
[connection master]
#
# Initialize system-versioned and partitioned table and its data
+connection slave;
+include/stop_slave.inc
connection master;
SET timestamp=UNIX_TIMESTAMP('2025-01-01 01:00:00.000000');
+include/kill_binlog_dump_threads.inc
RESET MASTER;
+connection slave;
+include/start_slave.inc
+connection master;
create table t1 (x int) engine=InnoDB with system versioning partition by system_time limit 3 partitions 5;
insert into t1 values(1);
insert into t1 values(2);
diff --git a/mysql-test/suite/rpl/r/rpl_timezone.result b/mysql-test/suite/rpl/r/rpl_timezone.result
index d1978102f4e4b..0dcefccbaa4fa 100644
--- a/mysql-test/suite/rpl/r/rpl_timezone.result
+++ b/mysql-test/suite/rpl/r/rpl_timezone.result
@@ -141,6 +141,7 @@ SET @@session.time_zone = default;
connection slave;
include/stop_slave.inc
connection master;
+include/kill_binlog_dump_threads.inc
reset master;
CREATE TABLE t1 (date timestamp NOT NULL default CURRENT_TIMESTAMP on update CURRENT_TIMESTAMP, a int(11) default NULL);
SET @@session.time_zone='+01:00';
diff --git a/mysql-test/suite/rpl/r/rpl_trigger.result b/mysql-test/suite/rpl/r/rpl_trigger.result
index f4f700ac47754..9ca7363417d50 100644
--- a/mysql-test/suite/rpl/r/rpl_trigger.result
+++ b/mysql-test/suite/rpl/r/rpl_trigger.result
@@ -1006,6 +1006,7 @@ SHOW TABLES LIKE 't_';
Tables_in_test (t_)
SHOW TRIGGERS;
Trigger Event Table Statement Timing Created sql_mode Definer character_set_client collation_connection Database Collation
+include/kill_binlog_dump_threads.inc
RESET MASTER;
connection slave;
START SLAVE;
diff --git a/mysql-test/suite/rpl/r/semisync_future-7591.result b/mysql-test/suite/rpl/r/semisync_future-7591.result
index 8287b8feab5ad..1de77de60d09d 100644
--- a/mysql-test/suite/rpl/r/semisync_future-7591.result
+++ b/mysql-test/suite/rpl/r/semisync_future-7591.result
@@ -11,6 +11,7 @@ start slave;
include/wait_for_slave_io_error.inc [errno=1236]
connection master;
insert into t1 values (1);
+include/kill_binlog_dump_threads.inc
reset master;
connection slave;
include/stop_slave_sql.inc
diff --git a/mysql-test/suite/rpl/t/rpl_binlog_directory.test b/mysql-test/suite/rpl/t/rpl_binlog_directory.test
new file mode 100644
index 0000000000000..e1b8fd1836cdc
--- /dev/null
+++ b/mysql-test/suite/rpl/t/rpl_binlog_directory.test
@@ -0,0 +1,67 @@
+--source include/have_binlog_format_mixed.inc
+--source include/master-slave.inc
+
+--echo *** Test the --binlog-directory variable with legacy binlog.
+
+--connection slave
+# Stop the slave while restarting master, just to not have to worry about
+# any connect/re-connect tries. We are testing the location of the binlog,
+# not the slave re-connect abilities.
+--source include/stop_slave.inc
+CHANGE MASTER TO master_use_gtid=Slave_pos;
+
+--connection master
+SELECT @@GLOBAL.binlog_storage_engine;
+--let $master_datadir= `SELECT @@datadir`
+CREATE TABLE t1 (a INT PRIMARY KEY, b INT);
+
+INSERT INTO t1 VALUES (1, 10);
+
+# Test that the master can be restarted with binlogs in a separate
+# directory specificed by --binlog-directory.
+--let $rpl_server_number= 1
+--source include/rpl_stop_server.inc
+
+--mkdir $master_datadir/binlog_dir
+--copy_file $master_datadir/master-bin.000001 $master_datadir/binlog_dir/master-bin.000001
+--copy_file $master_datadir/master-bin.000001.idx $master_datadir/binlog_dir/master-bin.000001.idx
+--move_file $master_datadir/master-bin.state $master_datadir/binlog_dir/master-bin.state
+--let $rpl_server_parameters= --binlog-directory=binlog_dir
+--source include/rpl_start_server.inc
+
+INSERT INTO t1 VALUES (2, 11);
+
+# Move master back to using the standard binlog directory.
+--source include/rpl_stop_server.inc
+--move_file $master_datadir/binlog_dir/master-bin.state $master_datadir/master-bin.state
+--let $rpl_server_parameters=
+--source include/rpl_start_server.inc
+
+INSERT INTO t1 VALUES (3, 12);
+--source include/save_master_gtid.inc
+--source include/show_binary_logs.inc
+--echo *** Contents of master-bin.index (including directory path):
+--cat_file $master_datadir/master-bin.index
+SELECT * FROM t1 ORDER BY a;
+
+--connection slave
+--source include/start_slave.inc
+--source include/sync_with_master_gtid.inc
+SELECT * FROM t1 ORDER BY a;
+
+# Clean up.
+--connection slave
+--source include/stop_slave.inc
+SET GLOBAL gtid_slave_pos= '';
+
+--connection master
+--source include/kill_binlog_dump_threads.inc
+RESET MASTER;
+--rmdir $master_datadir/binlog_dir
+
+--connection slave
+--source include/start_slave.inc
+
+--connection master
+DROP TABLE t1;
+--source include/rpl_end.inc
diff --git a/mysql-test/suite/rpl/t/rpl_binlog_errors.test b/mysql-test/suite/rpl/t/rpl_binlog_errors.test
index ab95c9b4ee353..faf665f3b502e 100644
--- a/mysql-test/suite/rpl/t/rpl_binlog_errors.test
+++ b/mysql-test/suite/rpl/t/rpl_binlog_errors.test
@@ -69,6 +69,7 @@ SET @old_debug= @@global.debug_dbug;
-- replace_result $MYSQLTEST_VARDIR MYSQLTEST_VARDIR
-- eval SELECT repeat('x',10) INTO OUTFILE '$load_file2'
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
-- echo ###################### TEST #1
@@ -436,6 +437,7 @@ call mtr.add_suppression("Can't generate a unique log-filename .*");
### clean up
-- source include/stop_slave_sql.inc
RESET SLAVE;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
--remove_file $load_file
--remove_file $load_file2
diff --git a/mysql-test/suite/rpl/t/rpl_blackhole_row_annotate.test b/mysql-test/suite/rpl/t/rpl_blackhole_row_annotate.test
index f7aefd625a1be..38e9a9f92f3ec 100644
--- a/mysql-test/suite/rpl/t/rpl_blackhole_row_annotate.test
+++ b/mysql-test/suite/rpl/t/rpl_blackhole_row_annotate.test
@@ -25,9 +25,14 @@ source include/have_binlog_format_row.inc;
source include/binlog_start_pos.inc;
source include/master-slave.inc;
+connection slave;
+source include/stop_slave.inc;
+connection master;
+source include/kill_binlog_dump_threads.inc;
SET timestamp=1000000000;
RESET MASTER;
connection slave;
+source include/start_slave.inc;
SET timestamp=1000000000;
RESET MASTER;
diff --git a/mysql-test/suite/rpl/t/rpl_bug41902.test b/mysql-test/suite/rpl/t/rpl_bug41902.test
index bb6c572580ff5..9b80d5bfd460b 100644
--- a/mysql-test/suite/rpl/t/rpl_bug41902.test
+++ b/mysql-test/suite/rpl/t/rpl_bug41902.test
@@ -36,6 +36,7 @@ reset slave;
change master to master_host='dummy';
connection master;
+--source include/kill_binlog_dump_threads.inc
SET @saved_dbug_m = @@global.debug_dbug;
SET @@debug_dbug="d,simulate_find_log_pos_error";
--error ER_UNKNOWN_TARGET_BINLOG
diff --git a/mysql-test/suite/rpl/t/rpl_change_master_demote.test b/mysql-test/suite/rpl/t/rpl_change_master_demote.test
index 255e9e34b9c59..0f6adfc88eced 100644
--- a/mysql-test/suite/rpl/t/rpl_change_master_demote.test
+++ b/mysql-test/suite/rpl/t/rpl_change_master_demote.test
@@ -104,11 +104,15 @@ change master to master_use_gtid=slave_pos;
--echo #
--connection slave
--source include/stop_slave.inc
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
set @@global.gtid_slave_pos="";
---source include/start_slave.inc
--connection master
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
+--connection slave
+--source include/start_slave.inc
+--connection master
set @@global.gtid_slave_pos="";
set session gtid_domain_id= 0;
CREATE TABLE t1 (a int);
@@ -482,6 +486,7 @@ EOF
--echo # Ensure MASTER_DEMOTE_TO_REPLICA aliases MASTER_DEMOTE_TO_SLAVE
--echo #
--connection slave
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
--source include/reset_slave.inc
CREATE TABLE t_mdev_31768 (a int);
@@ -505,6 +510,7 @@ RESET MASTER;
--echo # Clear primary binlog state to match replica
--connection master
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
diff --git a/mysql-test/suite/rpl/t/rpl_checksum.test b/mysql-test/suite/rpl/t/rpl_checksum.test
index bea325b251f1a..1ec75d0bb2e92 100644
--- a/mysql-test/suite/rpl/t/rpl_checksum.test
+++ b/mysql-test/suite/rpl/t/rpl_checksum.test
@@ -214,6 +214,7 @@ flush logs;
connection master;
set @@global.binlog_checksum= CRC32;
+--source include/kill_binlog_dump_threads.inc
reset master;
flush logs;
create table t3 (a int, b char(5));
@@ -232,6 +233,7 @@ eval change master to master_host='127.0.0.1',master_port=$MASTER_MYPORT, master
connection master;
flush logs;
+--source include/kill_binlog_dump_threads.inc
reset master;
insert into t3 value (1, @@global.binlog_checksum);
diff --git a/mysql-test/suite/rpl/t/rpl_circular_for_4_hosts.test b/mysql-test/suite/rpl/t/rpl_circular_for_4_hosts.test
index aeb849b893ec7..0aa5bd5758005 100644
--- a/mysql-test/suite/rpl/t/rpl_circular_for_4_hosts.test
+++ b/mysql-test/suite/rpl/t/rpl_circular_for_4_hosts.test
@@ -17,6 +17,9 @@
# Use wait_for_slave_to_(start|stop) for current connections
let $keep_connection= 1;
+# This test does not configure domain_id, don't use GTID for syncing.
+--let $disable_gtid= 1
+
# Set up circular ring and new names for servers
--echo *** Set up circular replication on four servers ***
--let $rpl_topology= 1->2->3->4->1
@@ -197,6 +200,7 @@ DELETE FROM t1 WHERE a = 6;
--source include/start_slave.inc
--connection server_2
--sync_slave_with_master server_3
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
--let $file_d= query_get_value(SHOW MASTER STATUS, File, 1)
--let $pos_d= query_get_value(SHOW MASTER STATUS, Position, 1)
diff --git a/mysql-test/suite/rpl/t/rpl_circular_semi_sync.test b/mysql-test/suite/rpl/t/rpl_circular_semi_sync.test
index 75d04d6545ba8..5304a4b49acb9 100644
--- a/mysql-test/suite/rpl/t/rpl_circular_semi_sync.test
+++ b/mysql-test/suite/rpl/t/rpl_circular_semi_sync.test
@@ -26,12 +26,14 @@ call mtr.add_suppression("Timeout waiting for reply of binlog");
# Initial master
--connection server_1
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
set @@session.gtid_domain_id=10;
set @@global.rpl_semi_sync_master_enabled = 1;
set @@global.rpl_semi_sync_master_wait_point=AFTER_SYNC;
--connection server_2
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
ALTER TABLE mysql.gtid_slave_pos ENGINE=InnoDB;
set @@session.gtid_domain_id=20;
diff --git a/mysql-test/suite/rpl/t/rpl_colSize.test b/mysql-test/suite/rpl/t/rpl_colSize.test
index 23c8bdc3e2df6..cec1fbb3bf467 100644
--- a/mysql-test/suite/rpl/t/rpl_colSize.test
+++ b/mysql-test/suite/rpl/t/rpl_colSize.test
@@ -68,6 +68,7 @@ eval CREATE TABLE t1 (
t LONGTEXT
);
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
--echo *** Start replication ***
diff --git a/mysql-test/suite/rpl/t/rpl_crash_ddl_master.test b/mysql-test/suite/rpl/t/rpl_crash_ddl_master.test
new file mode 100644
index 0000000000000..17de07ef36914
--- /dev/null
+++ b/mysql-test/suite/rpl/t/rpl_crash_ddl_master.test
@@ -0,0 +1,86 @@
+--source include/have_innodb.inc
+--source include/not_embedded.inc
+--source include/not_valgrind.inc
+--source include/have_debug.inc
+--source include/have_binlog_format_row.inc
+--source include/master-slave.inc
+
+
+CREATE TABLE t1 (a INT PRIMARY KEY, b INT, c INT) ENGINE=InnoDB;
+CREATE TABLE t2 LIKE t1;
+INSERT INTO t1 VALUES (1, 0, 0), (2, 3, 1), (3, 2, 5), (4, 10, 7), (5, 6, 5);
+# The GTID sequence number 99 makes the ALTER below get the sequence number
+# 100 that triggers the rpl_parallel_delay_gtid_0_x_100_write
+# dbug injection below.
+SET SESSION gtid_seq_no= 99;
+INSERT INTO t2 SELECT * FROM t1;
+--source include/save_master_gtid.inc
+
+--connection slave
+--source include/sync_with_master_gtid.inc
+--source include/stop_slave.inc
+SET @old_parallel= @@GLOBAL.slave_parallel_threads;
+SET GLOBAL slave_parallel_threads=8;
+SET @old_mode= @@GLOBAL.slave_parallel_mode;
+SET GLOBAL slave_parallel_mode= optimistic;
+SET @old_strict= @@GLOBAL.gtid_strict_mode;
+SET GLOBAL gtid_strict_mode= 1;
+
+--connection master
+--write_file $MYSQLTEST_VARDIR/tmp/mysqld.1.expect
+wait
+EOF
+
+# Crash the master in DDL so that the binlogging happens during DDL recovery.
+# There was a bug that the GTID was binlogged without the 'ddl' marker.
+SET @debug_crash_counter= 1;
+SET SESSION debug_dbug= '+d,ddl_log_alter_after_log';
+--error 2006,2013
+ALTER TABLE t1 ORDER BY b;
+
+--source include/wait_until_disconnected.inc
+--connection master1
+--source include/wait_until_disconnected.inc
+--connection server_1
+--source include/wait_until_disconnected.inc
+--connection default
+--source include/wait_until_disconnected.inc
+
+--connection master
+--append_file $MYSQLTEST_VARDIR/tmp/mysqld.1.expect
+restart
+EOF
+--let $rpl_server_number= 1
+--source include/rpl_reconnect.inc
+
+INSERT INTO t2 VALUES (10, 10, 10);
+UPDATE t2 SET c=c+1 WHERE a=4;
+UPDATE t2 SET c=20 WHERE a=1;
+# This just to terminate the gco on the slave, give an opportunity to
+# trigger an assertion "!tmp_gco->next_gco || tmp_gco->last_sub_id > sub_id".
+ALTER TABLE t2 ORDER BY b;
+--source include/save_master_gtid.inc
+
+--connection slave
+# Delay the binlogging of GTID for the DDL on the slave.
+# There was a bug that, when the GTID was incorrectly not marked as DDL and
+# following DML could run in parallel, that the DDL would
+# wakeup_subsequent_commits too early and allow later DML to commit out-of-order.
+# This would then cause assertion due to out-of-order GCO management.
+SET @old_dbug= @@GLOBAL.debug_dbug;
+SET GLOBAL debug_dbug= '+d,rpl_parallel_delay_gtid_0_x_100_write';
+--source include/start_slave.inc
+--source include/sync_with_master_gtid.inc
+SET GLOBAL debug_dbug= @old_dbug;
+
+# Clean up.
+--connection slave
+--source include/stop_slave.inc
+SET GLOBAL slave_parallel_threads= @old_parallel;
+SET GLOBAL slave_parallel_mode= @old_mode;
+SET GLOBAL gtid_strict_mode= @old_strict;
+--source include/start_slave.inc
+
+--connection master
+DROP TABLE t1, t2;
+--source include/rpl_end.inc
diff --git a/mysql-test/suite/rpl/t/rpl_delayed_slave.test b/mysql-test/suite/rpl/t/rpl_delayed_slave.test
index 32d0b030eed4c..693b912face0c 100644
--- a/mysql-test/suite/rpl/t/rpl_delayed_slave.test
+++ b/mysql-test/suite/rpl/t/rpl_delayed_slave.test
@@ -384,6 +384,7 @@ CHANGE MASTER TO MASTER_DELAY = 71;
--source include/reset_slave.inc
--echo [on master]
--connection master
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
--echo [on slave]
--connection slave
diff --git a/mysql-test/suite/rpl/t/rpl_extra_col_slave_minimal.test b/mysql-test/suite/rpl/t/rpl_extra_col_slave_minimal.test
index 7e2daa9abc9ea..f4b54cfe4393b 100644
--- a/mysql-test/suite/rpl/t/rpl_extra_col_slave_minimal.test
+++ b/mysql-test/suite/rpl/t/rpl_extra_col_slave_minimal.test
@@ -16,6 +16,7 @@ create table t1 (
connection master;
create table t1 (a int primary key, b blob, c char(5)) engine=innodb;
+--source include/kill_binlog_dump_threads.inc
reset master;
connection slave;
diff --git a/mysql-test/suite/rpl/t/rpl_gtid_basic.test b/mysql-test/suite/rpl/t/rpl_gtid_basic.test
index a7af234d47e87..2f2669bda6dab 100644
--- a/mysql-test/suite/rpl/t/rpl_gtid_basic.test
+++ b/mysql-test/suite/rpl/t/rpl_gtid_basic.test
@@ -180,6 +180,7 @@ SET @old_state= @@GLOBAL.gtid_binlog_state;
--error ER_BINLOG_MUST_BE_EMPTY
SET GLOBAL gtid_binlog_state = '';
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
SET GLOBAL gtid_binlog_state = '';
FLUSH LOGS;
diff --git a/mysql-test/suite/rpl/t/rpl_gtid_crash.test b/mysql-test/suite/rpl/t/rpl_gtid_crash.test
index 05283139533e2..da3c7e31a0f8a 100644
--- a/mysql-test/suite/rpl/t/rpl_gtid_crash.test
+++ b/mysql-test/suite/rpl/t/rpl_gtid_crash.test
@@ -89,6 +89,7 @@ RESET MASTER;
SET GLOBAL gtid_slave_pos='';
--connection server_1
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
--replace_column 2 # 4 # 5 #
SHOW BINLOG EVENTS IN 'master-bin.000001' LIMIT 1,1;
diff --git a/mysql-test/suite/rpl/t/rpl_gtid_errorhandling.test b/mysql-test/suite/rpl/t/rpl_gtid_errorhandling.test
index 578f1e3521372..f3008dcc83eb7 100644
--- a/mysql-test/suite/rpl/t/rpl_gtid_errorhandling.test
+++ b/mysql-test/suite/rpl/t/rpl_gtid_errorhandling.test
@@ -115,6 +115,7 @@ SET SESSION binlog_format= @old_mode;
--source include/stop_slave.inc
--connection master
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
# This insert will be GTID 0-1-1
INSERT INTO t1 VALUES (2);
diff --git a/mysql-test/suite/rpl/t/rpl_gtid_header_valid.test b/mysql-test/suite/rpl/t/rpl_gtid_header_valid.test
index d4d75ba89280e..d0aa57f64f891 100644
--- a/mysql-test/suite/rpl/t/rpl_gtid_header_valid.test
+++ b/mysql-test/suite/rpl/t/rpl_gtid_header_valid.test
@@ -63,6 +63,7 @@ STOP SLAVE IO_THREAD;
--echo # Reset master binlogs (as there is an invalid event) and slave state
--connection master
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
--connection slave
RESET MASTER;
@@ -97,6 +98,7 @@ set statement SQL_LOG_BIN=0 for XA COMMIT 'x1';
--echo # Reset master binlogs (as there is an invalid event) and slave state
--connection master
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
--connection slave
RESET MASTER;
@@ -131,6 +133,7 @@ set statement SQL_LOG_BIN=0 for XA COMMIT 'x1';
--echo # Reset master binlogs (as there is an invalid event) and slave state
--connection master
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
--connection slave
RESET MASTER;
@@ -159,6 +162,7 @@ STOP SLAVE IO_THREAD;
--echo # Reset master binlogs (as there is an invalid event) and slave state
--connection master
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
--connection slave
@@ -188,6 +192,7 @@ STOP SLAVE IO_THREAD;
--echo # Reset master binlogs (as there is an invalid event) and slave state
--connection master
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
--connection slave
diff --git a/mysql-test/suite/rpl/t/rpl_gtid_index.test b/mysql-test/suite/rpl/t/rpl_gtid_index.test
index 6001cc6e6009a..ffb0323b6d91c 100644
--- a/mysql-test/suite/rpl/t/rpl_gtid_index.test
+++ b/mysql-test/suite/rpl/t/rpl_gtid_index.test
@@ -3,39 +3,7 @@
--source include/master-slave.inc
--source include/have_binlog_format_mixed.inc
---connection slave
---source include/stop_slave.inc
-CHANGE MASTER TO master_use_gtid= slave_pos;
---source include/start_slave.inc
-
---connection master
-CREATE TABLE t1 (a INT PRIMARY KEY, b INT) ENGINE=InnoDB;
-INSERT INTO t1 VALUES (0, 0);
-
-
---echo *** Test looking up a lot of different event positions and GTIDs.
-
-# A function for comparing GTID positions.
-# Handles that the domain_id order is different in the two strings.
-# Works by repeatedly removing one GTID from each string. If the strings have
-# the same length and nothing is left at the end, then they are identical.
-delimiter //;
-CREATE FUNCTION gtid_eq(a VARCHAR(255), b VARCHAR(255)) RETURNS BOOLEAN DETERMINISTIC
-BEGIN
- DECLARE g VARCHAR(255);
- IF a IS NULL OR b IS NULL OR LENGTH(a) != LENGTH(b) THEN
- RETURN FALSE;
- END IF;
- SET a= CONCAT(a, ',');
- SET b= CONCAT(',', b, ',');
- WHILE LENGTH(a) > 0 DO
- SET g= REGEXP_SUBSTR(a, '^[^,]+,');
- SET a= SUBSTRING(a, LENGTH(g)+1);
- SET b= REPLACE(b, CONCAT(',', g), ',');
- END WHILE;
- RETURN b = ',';
-END //
-delimiter ;//
+--source suite/rpl/include/rpl_gtid_index_setup.inc
SET @old_page_size= @@GLOBAL.binlog_gtid_index_page_size;
SET @old_span_min= @@GLOBAL.binlog_gtid_index_span_min;
@@ -82,8 +50,6 @@ SET GLOBAL binlog_gtid_index_span_min= 1;
--connection master
SET GLOBAL binlog_gtid_index_page_size= @old_page_size;
SET GLOBAL binlog_gtid_index_span_min= @old_span_min;
-
-DROP TABLE t1;
-DROP FUNCTION gtid_eq;
+--source suite/rpl/include/rpl_gtid_index_cleanup.inc
--source include/rpl_end.inc
diff --git a/mysql-test/suite/rpl/t/rpl_gtid_mdev4820.test b/mysql-test/suite/rpl/t/rpl_gtid_mdev4820.test
index 79b3b7d464a32..db0e0d2079319 100644
--- a/mysql-test/suite/rpl/t/rpl_gtid_mdev4820.test
+++ b/mysql-test/suite/rpl/t/rpl_gtid_mdev4820.test
@@ -25,6 +25,7 @@ SELECT * FROM t1 ORDER BY a;
INSERT INTO t1 VALUES (2);
INSERT INTO t1 VALUES (3);
--let $old_gtid_pos= `SELECT @@GLOBAL.gtid_current_pos`
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
--replace_result $old_gtid_pos OLD_GTID_POS
eval SET GLOBAL gtid_slave_pos= '$old_gtid_pos';
diff --git a/mysql-test/suite/rpl/t/rpl_gtid_reconnect.test b/mysql-test/suite/rpl/t/rpl_gtid_reconnect.test
index 1452d6b01f03a..1d4d9c9e715b3 100644
--- a/mysql-test/suite/rpl/t/rpl_gtid_reconnect.test
+++ b/mysql-test/suite/rpl/t/rpl_gtid_reconnect.test
@@ -55,6 +55,7 @@ SET GLOBAL gtid_slave_pos= "";
--connection server_1
SET GLOBAL debug_dbug= @old_debug;
TRUNCATE t1;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
# A1 B1 A2 B2 A3 B3, slave reached A1 and B3 and stopped. Slave starts,
@@ -101,6 +102,7 @@ SET GLOBAL gtid_slave_pos= "";
--connection server_1
SET GLOBAL debug_dbug= @old_debug;
TRUNCATE t1;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
# A1 B1 A2 B2 A3 B3. START SLAVE UNTIL A1,B3, gets reconnect at B2.
@@ -154,6 +156,7 @@ SET GLOBAL gtid_slave_pos= "";
--connection server_1
SET GLOBAL debug_dbug= @old_debug;
TRUNCATE t1;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
--source include/kill_binlog_dump_threads.inc
diff --git a/mysql-test/suite/rpl/t/rpl_gtid_sort.test b/mysql-test/suite/rpl/t/rpl_gtid_sort.test
index c31ba877bbf86..99465e2ce8503 100644
--- a/mysql-test/suite/rpl/t/rpl_gtid_sort.test
+++ b/mysql-test/suite/rpl/t/rpl_gtid_sort.test
@@ -13,6 +13,7 @@ SHOW VARIABLES LIKE 'gtid_slave_pos';
SHOW VARIABLES LIKE 'gtid_binlog_state';
--connection server_1
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
FLUSH LOGS;
CREATE TABLE t1 (a INT PRIMARY KEY) ENGINE=InnoDB;
diff --git a/mysql-test/suite/rpl/t/rpl_gtid_startpos.test b/mysql-test/suite/rpl/t/rpl_gtid_startpos.test
index d0885ab891236..5e1fe67b9f663 100644
--- a/mysql-test/suite/rpl/t/rpl_gtid_startpos.test
+++ b/mysql-test/suite/rpl/t/rpl_gtid_startpos.test
@@ -9,6 +9,7 @@ RESET MASTER;
RESET SLAVE;
--connection server_1
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
# Create an empty binlog file, to check that empty binlog state is handled correctly.
FLUSH LOGS;
@@ -113,6 +114,7 @@ RESET SLAVE ALL;
RESET MASTER;
--connection server_1
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
--connection server_2
@@ -250,6 +252,7 @@ RESET MASTER;
SET GLOBAL gtid_slave_pos="";
--connection server_1
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
TRUNCATE TABLE t1;
INSERT INTO t1 VALUES (10); # Will be GTID 0-1-2
@@ -299,6 +302,7 @@ eval CHANGE MASTER TO master_host='127.0.0.1', master_port=$MASTER_MYPORT, maste
--connection server_1
DROP TABLE t1;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
CREATE TABLE t1 (a INT PRIMARY KEY);
diff --git a/mysql-test/suite/rpl/t/rpl_gtid_until.test b/mysql-test/suite/rpl/t/rpl_gtid_until.test
index c89cea23e933c..1aee5fd529418 100644
--- a/mysql-test/suite/rpl/t/rpl_gtid_until.test
+++ b/mysql-test/suite/rpl/t/rpl_gtid_until.test
@@ -211,6 +211,7 @@ SET GLOBAL gtid_slave_pos='';
--connection server_1
# Do it once to compute the right GTID, then throw it away and do it again
# for the actual test.
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
INSERT INTO t1 VALUES (10);
INSERT INTO t1 VALUES (11);
diff --git a/mysql-test/suite/rpl/t/rpl_heartbeat.test b/mysql-test/suite/rpl/t/rpl_heartbeat.test
index a34cf761c8a11..ae77377f7f5b4 100644
--- a/mysql-test/suite/rpl/t/rpl_heartbeat.test
+++ b/mysql-test/suite/rpl/t/rpl_heartbeat.test
@@ -1,173 +1,7 @@
-# Testing master to slave heartbeat protocol
-#
-# Including:
-# - user interface, grammar, checking the range and warnings about
-# unreasonable values for the heartbeat period;
-# - no rotation of relay log if heartbeat is less that slave_net_timeout
-# - SHOW STATUS like 'Slave_received_heartbeats' action
-# - SHOW STATUS like 'Slave_heartbeat_period' report
-
-- source include/have_log_bin.inc
-- source include/master-slave.inc
-connection master;
-reset master;
-
-connection slave;
--- source include/stop_slave.inc
-set @restore_slave_net_timeout= @@global.slave_net_timeout;
---disable_warnings
-set @@global.slave_net_timeout= 10;
---enable_warnings
-
---enable_prepare_warnings
-###
-### Checking the range
-###
-
-#
-# default period slave_net_timeout/2
-#
---replace_result $MASTER_MYPORT MASTER_PORT
-eval change master to master_host='127.0.0.1',master_port=$MASTER_MYPORT, master_user='root';
---query_vertical show status like 'Slave_heartbeat_period';
-
-#
-# the max for the period is ULONG_MAX/1000; an attempt to exceed it is denied
-#
---replace_result $MASTER_MYPORT MASTER_PORT
---error ER_SLAVE_HEARTBEAT_VALUE_OUT_OF_RANGE
-eval change master to master_host='127.0.0.1',master_port=$MASTER_MYPORT, master_user='root', master_heartbeat_period= 4294968;
---query_vertical show status like 'Slave_heartbeat_period';
-
-#
-# the min value for the period is 1 millisecond an attempt to assign a
-# lesser will be warned with treating the value as zero
-#
-connection slave;
---replace_result $MASTER_MYPORT MASTER_PORT
-### 5.1 mtr does not have --warning ER_SLAVE_HEARTBEAT_VALUE_OUT_OF_RANGE
-eval change master to master_host='127.0.0.1',master_port=$MASTER_MYPORT, master_user='root', master_heartbeat_period= 0.0009999;
---query_vertical show status like 'Slave_heartbeat_period';
-
-#
-# the actual max and min must be accepted
-#
---replace_result $MASTER_MYPORT MASTER_PORT
-eval change master to master_host='127.0.0.1',master_port=$MASTER_MYPORT, master_user='root', master_heartbeat_period= 4294967;
---query_vertical show status like 'Slave_heartbeat_period';
-
---replace_result $MASTER_MYPORT MASTER_PORT
-eval change master to master_host='127.0.0.1',master_port=$MASTER_MYPORT, master_user='root', master_heartbeat_period= 0.001;
---query_vertical show status like 'Slave_heartbeat_period';
-
-reset slave;
-
-#
-# A warning if period greater than slave_net_timeout
-#
-set @@global.slave_net_timeout= 5;
---replace_result $MASTER_MYPORT MASTER_PORT
-eval change master to master_host='127.0.0.1',master_port=$MASTER_MYPORT, master_user='root', master_heartbeat_period= 5.001;
---query_vertical show status like 'Slave_heartbeat_period';
-
-reset slave;
-
-#
-# A warning if slave_net_timeout is set to less than the current HB period
-#
-set @@global.slave_net_timeout= 5;
---replace_result $MASTER_MYPORT MASTER_PORT
-eval change master to master_host='127.0.0.1',master_port=$MASTER_MYPORT, master_user='root', master_heartbeat_period= 4;
---query_vertical show status like 'Slave_heartbeat_period';
-set @@global.slave_net_timeout= 3 /* must be a warning */;
-
-reset slave;
-
-
-###
-### checking no rotation
-###
-
-connection master;
---disable_warnings
-drop table if exists t1;
---enable_warnings
-#
-# Even though master_heartbeat_period= 0.5 is 20 times less than
-# @@global.slave_net_timeout= 10 in some circumstances master will
-# not be able to send any heartbeat during the slave's net timeout
-# and slave's relay log will rotate.
-# The probability for such a scenario is pretty small so the following
-# part is almost deterministic.
-#
-
-connection slave;
-set @@global.slave_net_timeout= 10;
---replace_result $MASTER_MYPORT MASTER_PORT
-# no error this time but rather a warning
-eval change master to master_host='127.0.0.1',master_port=$MASTER_MYPORT, master_user='root', master_ssl_verify_server_cert=0, master_heartbeat_period= 0.5;
---query_vertical show status like 'Slave_heartbeat_period';
-
-start slave;
-
-connection master;
-create table t1 (f1 int);
-
-#connection slave;
-sync_slave_with_master;
-let $slave_param= Relay_Log_File;
-let $slave_param_value= query_get_value(SHOW SLAVE STATUS, Relay_Log_File, 1);
-
-# there is an explicit sleep lasting longer than slave_net_timeout
-# to ensure that nothing will come to slave from master for that period.
-# That would cause reconnecting and relaylog rotation w/o the fix i.e
-# without a heartbeat received.
-
-real_sleep 15;
-
-# check (compare with the previous show's results) that no rotation happened
-source include/check_slave_param.inc;
-
-###
-### SHOW STATUS like 'Slave_heartbeat_period' and 'Slave_received_heartbeats'
-###
-
---query_vertical show status like 'Slave_heartbeat_period';
-
-#
-# proof that there has been received at least one heartbeat;
-# The exact number of received heartbeat is an indeterministic value
-# and therefore it's not recorded into results.
-#
-
-let $slave_wait_param_counter= 300;
-let $slave_value= query_get_value("SHOW STATUS like 'Slave_received_heartbeats'", Value, 1);
-# Checking the fact that at least one heartbeat is received
-while (!$slave_value)
-{
- dec $slave_wait_param_counter;
- if (!$slave_wait_param_counter)
- {
- --echo ERROR: failed while waiting for slave parameter $slave_param: $slave_param_value
- query_vertical show slave status;
- SHOW STATUS like 'Slave_received_heartbeats';
- exit;
- }
- sleep 0.1;
- let $slave_value= query_get_value("SHOW STATUS like 'Slave_received_heartbeats'", Value, 1);
-}
---echo A heartbeat has been received by the slave
-# cleanup
-
-connection master;
-drop table t1;
-
-#connection slave;
-sync_slave_with_master;
-set @@global.slave_net_timeout= @restore_slave_net_timeout;
-
---disable_prepare_warnings
+--source include/rpl_heartbeat.inc
--source include/rpl_end.inc
--echo End of tests
diff --git a/mysql-test/suite/rpl/t/rpl_heartbeat_basic.test b/mysql-test/suite/rpl/t/rpl_heartbeat_basic.test
index fb67ad770d57a..2e0402bb09259 100644
--- a/mysql-test/suite/rpl/t/rpl_heartbeat_basic.test
+++ b/mysql-test/suite/rpl/t/rpl_heartbeat_basic.test
@@ -35,6 +35,7 @@ eval SET @restore_slave_heartbeat_timeout=$slave_heartbeat_timeout;
--enable_query_log
--connection master
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
SET @restore_slave_net_timeout=@@global.slave_net_timeout;
--echo
@@ -415,6 +416,7 @@ DROP TABLE t1;
set sql_log_bin= 1;
--connection master
DROP TABLE t1;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
--connection slave
--replace_result $MASTER_MYPORT MASTER_PORT
@@ -465,10 +467,15 @@ STOP SLAVE;
--source include/reset_slave.inc
--replace_result $MASTER_MYPORT MASTER_PORT
eval CHANGE MASTER TO MASTER_HOST='127.0.0.1', MASTER_PORT=$MASTER_MYPORT, MASTER_USER='root', MASTER_CONNECT_RETRY=$connect_retry, MASTER_HEARTBEAT_PERIOD=0.1;
+
+--connection master
+--source include/kill_binlog_dump_threads.inc
+RESET MASTER;
+
+--connection slave
--source include/start_slave.inc
let $rcvd_heartbeats_before= query_get_value(SHOW STATUS LIKE 'slave_received_heartbeats', Value, 1);
--connection master
-RESET MASTER;
--enable_query_log
--sync_slave_with_master
--sleep 2
diff --git a/mysql-test/suite/rpl/t/rpl_mdev_17614.test b/mysql-test/suite/rpl/t/rpl_mdev_17614.test
index e9a41c95ff064..ca27b8b64b916 100644
--- a/mysql-test/suite/rpl/t/rpl_mdev_17614.test
+++ b/mysql-test/suite/rpl/t/rpl_mdev_17614.test
@@ -43,6 +43,7 @@ SELECT * FROM t1;
--source include/stop_slave_io.inc
--source include/reset_slave.inc
connection master;
+--source include/kill_binlog_dump_threads.inc
reset master;
drop table t1;
connection slave;
@@ -161,6 +162,7 @@ SELECT * FROM t1;
--source include/stop_slave_io.inc
--source include/reset_slave.inc
connection master;
+--source include/kill_binlog_dump_threads.inc
reset master;
drop table t1;
connection slave;
diff --git a/mysql-test/suite/rpl/t/rpl_mysqlbinlog_slave_consistency.inc b/mysql-test/suite/rpl/t/rpl_mysqlbinlog_slave_consistency.inc
new file mode 100644
index 0000000000000..ec89cb390791b
--- /dev/null
+++ b/mysql-test/suite/rpl/t/rpl_mysqlbinlog_slave_consistency.inc
@@ -0,0 +1,399 @@
+#
+# Purpose:
+#
+# This test ensures the mariadb-binlog command line tool filters events
+# by domain id (via --do-domain-ids and --ignore-domain-ids) and server id (via
+# --ignore-server-ids) in the same way that a replica server does.
+#
+#
+# Methodology:
+#
+# This test validates that the domain and server id filtering logic of
+# mariadb-binlog matches that of a replica server. In particular, it validates
+# a mariadb-binlog replay of a master's binary log is consistent with a
+# replica's state which is configured using the same filtering configuration.
+#
+# It uses a repeatable process to allow for multiple test cases that span
+# different filtering configurations. First, a master is seeded with an initial
+# set of SQL statements with varying domain and server ids. Then, a set of
+# filtering parameters supported by both mariadb-binlog and replica
+# capabilities are defined. The replica is configured using these parameters
+# and run it until it has processed all events from the primary server; it is
+# stopped afterward. For mariadb-binlog validation, the binary log of the
+# primary server is copied to a different location for later replay. The
+# primary is then reset to its initial state (i.e. the tables are dropped and
+# the logs are reset). The mariadb-binlog tool is then used to replay the
+# copied binary log file back onto the clean primary server under the same
+# filtering conditions as the replica. At this point, the data on the primary
+# and replica should be exactly the same because the filtering conditions were
+# the same, and all existing tables on both servers are compared using.
+# checksums.
+#
+#
+# References:
+#
+# MDEV-20119: Implement the --do-domain-ids, --ignore-domain-ids, and
+# --ignore-server-ids options for mysqlbinlog
+#
+
+--connection slave
+SET sql_log_bin=0;
+call mtr.add_suppression("Slave: An attempt was made.*");
+call mtr.add_suppression("Both DO_DOMAIN_IDS & IGNORE_DOMAIN_IDS lists can't be non-empty at the same time");
+SET sql_log_bin=1;
+
+--source include/stop_slave.inc
+
+--connection master
+--source include/kill_binlog_dump_threads.inc
+RESET MASTER;
+
+# Save old state
+let $ORIG_GTID_DOMAIN_ID = `select @@session.gtid_domain_id`;
+let $ORIG_SERVER_ID = `select @@session.server_id`;
+
+--connection slave
+--source include/start_slave.inc
+
+# Initial tests do not use strict mode
+--let $strict_mode=0
+--let $slave_sql_errno=0
+
+--echo #
+--echo #
+--echo # Test Case 1: Base case to ensure that mariadb-binlog and replica
+--echo # are consistent without any filtering
+--echo #
+--let $do_domain_ids=
+--let $ignore_domain_ids=
+--let $ignore_server_ids=
+--let $stop_position=
+--let $start_position=
+--source suite/rpl/include/mysqlbinlog_slave_consistency.inc
+
+--echo #
+--echo #
+--echo # Test Case 2: Ensure filtering by a single id in --do-domain-ids is
+--echo # consistent between mariadb-binlog and replica
+--echo #
+--let $do_domain_ids=0
+--let $ignore_domain_ids=
+--let $ignore_server_ids=
+--let $stop_position=
+--let $start_position=
+--source suite/rpl/include/mysqlbinlog_slave_consistency.inc
+
+--echo #
+--echo #
+--echo # Test Case 3: Ensure filtering by multiple ids in --do-domain-ids is
+--echo # consistent between mariadb-binlog and replica
+--echo #
+--let $do_domain_ids=0,1
+--let $ignore_domain_ids=
+--let $ignore_server_ids=
+--let $stop_position=
+--let $start_position=
+--source suite/rpl/include/mysqlbinlog_slave_consistency.inc
+
+--echo #
+--echo # Test Case 4: Ensure filtering by a single id in --ignore-domain-ids
+--echo # is consistent between mariadb-binlog and replica
+--echo #
+--let $do_domain_ids=
+--let $ignore_domain_ids=0
+--let $ignore_server_ids=
+--let $stop_position=
+--let $start_position=
+--source suite/rpl/include/mysqlbinlog_slave_consistency.inc
+
+--echo #
+--echo #
+--echo # Test Case 5: Ensure filtering by multiple ids in --ignore-domain-ids
+--echo # is consistent between mariadb-binlog and replica
+--echo #
+--let $do_domain_ids=
+--let $ignore_domain_ids=0,1
+--let $ignore_server_ids=
+--let $stop_position=
+--let $start_position=
+--source suite/rpl/include/mysqlbinlog_slave_consistency.inc
+
+--echo #
+--echo # Test Case 6: Ensure filtering by a single id in --ignore-server-ids
+--echo # is consistent between mariadb-binlog and replica
+--echo #
+--let $do_domain_ids=
+--let $ignore_domain_ids=
+--let $ignore_server_ids=1
+--let $stop_position=
+--let $start_position=
+--source suite/rpl/include/mysqlbinlog_slave_consistency.inc
+
+--echo #
+--echo #
+--echo # Test Case 7: Ensure filtering by multiple ids in --ignore-server-ids
+--echo # is consistent between mariadb-binlog and replica
+--echo #
+--let $do_domain_ids=
+--let $ignore_domain_ids=
+--let $ignore_server_ids=0,1
+--let $stop_position=
+--let $start_position=
+--source suite/rpl/include/mysqlbinlog_slave_consistency.inc
+
+--echo #
+--echo #
+--echo # Test Case 8: Ensure stop position consistency
+--echo #
+--let $do_domain_ids=
+--let $ignore_domain_ids=
+--let $ignore_server_ids=
+--let $stop_position=1-1-2
+--let $start_position=
+--source suite/rpl/include/mysqlbinlog_slave_consistency.inc
+
+--echo #
+--echo #
+--echo # Test Case 9: Ensure start position consistency
+--echo #
+--let $do_domain_ids=
+--let $ignore_domain_ids=
+--let $ignore_server_ids=
+--let $stop_position=
+--let $start_position=1-4-2
+--source suite/rpl/include/mysqlbinlog_slave_consistency.inc
+
+--echo #
+--echo #
+--echo # Test Case 10: Ensure consistency when filtering by both
+--echo # --do-domain-ids and --ignore-server-ids
+--echo #
+--let $do_domain_ids=0
+--let $ignore_domain_ids=
+--let $ignore_server_ids=1
+--let $stop_position=
+--let $start_position=
+--source suite/rpl/include/mysqlbinlog_slave_consistency.inc
+
+--echo #
+--echo #
+--echo # Test Case 11: Ensure consistency when filtering by both
+--echo # --ignore-domain-ids and --ignore-server-ids
+--echo #
+--let $do_domain_ids=
+--let $ignore_domain_ids=1,2
+--let $ignore_server_ids=1
+--let $stop_position=
+--let $start_position=
+--source suite/rpl/include/mysqlbinlog_slave_consistency.inc
+
+--echo #
+--echo #
+--echo # Test Case 12: Ensure consistency when filtering by
+--echo # --do-domain-ids with a stop position
+--echo #
+--let $do_domain_ids=0
+--let $ignore_domain_ids=
+--let $ignore_server_ids=
+--let $stop_position=0-1-4
+--let $start_position=
+--source suite/rpl/include/mysqlbinlog_slave_consistency.inc
+
+--echo #
+--echo #
+--echo # Test Case 13: Ensure consistency when filtering by
+--echo # --ignore-domain-ids with a stop position
+--echo #
+--let $do_domain_ids=
+--let $ignore_domain_ids=0
+--let $ignore_server_ids=
+--let $stop_position=0-1-4
+--let $start_position=
+--source suite/rpl/include/mysqlbinlog_slave_consistency.inc
+
+--echo #
+--echo #
+--echo # Test Case 14: Ensure consistency when filtering by
+--echo # --ignore-server-ids with a stop position
+--echo #
+--let $do_domain_ids=
+--let $ignore_domain_ids=
+--let $ignore_server_ids=3
+--let $stop_position=0-1-4
+--let $start_position=
+--source suite/rpl/include/mysqlbinlog_slave_consistency.inc
+
+--echo #
+--echo #
+--echo # Test Case 15: Ensure consistency when filtering by
+--echo # --do-domain-ids with a start position
+--echo #
+--let $do_domain_ids=2
+--let $ignore_domain_ids=
+--let $ignore_server_ids=
+--let $stop_position=
+--let $start_position=1-1-2
+--source suite/rpl/include/mysqlbinlog_slave_consistency.inc
+
+--echo #
+--echo #
+--echo # Test Case 16: Ensure consistency when filtering by
+--echo # --ignore-domain-ids with a start position
+--echo #
+--let $do_domain_ids=
+--let $ignore_domain_ids=0
+--let $ignore_server_ids=
+--let $stop_position=
+--let $start_position=0-1-1
+--source suite/rpl/include/mysqlbinlog_slave_consistency.inc
+
+--echo #
+--echo #
+--echo # Test Case 17: Ensure consistency when filtering by
+--echo # --ignore-server-ids with a start position
+--echo #
+--let $do_domain_ids=
+--let $ignore_domain_ids=
+--let $ignore_server_ids=1
+--let $stop_position=
+--let $start_position=0-1-1
+--source suite/rpl/include/mysqlbinlog_slave_consistency.inc
+
+--echo #
+--echo #
+--echo # Test Case 18: Ensure consistency when filtering by
+--echo # --do-domain-ids with both a start position and stop position that
+--echo # all have the same domain id
+--echo #
+--let $do_domain_ids=0
+--let $ignore_domain_ids=
+--let $ignore_server_ids=
+--let $stop_position=0-3-3
+--let $start_position=0-1-1
+--source suite/rpl/include/mysqlbinlog_slave_consistency.inc
+
+--echo #
+--echo #
+--echo # Test Case 19: Ensure consistency when filtering by
+--echo # --do-domain-ids with both a start position and stop position that
+--echo # have differing domain ids. Due to the implicit filtering in stop
+--echo # position, the result should be empty (no tables replicated).
+--echo #
+--let $do_domain_ids=1
+--let $ignore_domain_ids=
+--let $ignore_server_ids=
+--let $stop_position=0-3-3
+--let $start_position=0-1-1
+--source suite/rpl/include/mysqlbinlog_slave_consistency.inc
+
+--echo #
+--echo #
+--echo # Test Case 20: Ensure consistency when filtering by
+--echo # --ignore-domain-ids with both a start position and stop position that
+--echo # all have the same domain id. The result set should be empty due to
+--echo # implicit filtering from stop position and ignoring that same domain.
+--echo #
+--let $do_domain_ids=
+--let $ignore_domain_ids=0
+--let $ignore_server_ids=
+--let $stop_position=0-3-3
+--let $start_position=0-1-1
+--source suite/rpl/include/mysqlbinlog_slave_consistency.inc
+
+--echo #
+--echo #
+--echo # Test Case 21: Ensure consistency when filtering by
+--echo # --ignore-domain-ids with both a start position and stop position that
+--echo # have differing domain ids. The ignore domain ids should take no
+--echo # effect due to the implicit filtering by stop position, i.e. the
+--echo # specified domain to ignore is already being filtered.
+--echo #
+--let $do_domain_ids=
+--let $ignore_domain_ids=1
+--let $ignore_server_ids=
+--let $stop_position=0-3-3
+--let $start_position=0-1-1
+--source suite/rpl/include/mysqlbinlog_slave_consistency.inc
+
+--echo #
+--echo #
+--echo # Test Case 22: Ensure consistency when filtering by
+--echo # --ignore-server-ids with both a start position and stop position.
+--echo #
+--let $do_domain_ids=
+--let $ignore_domain_ids=
+--let $ignore_server_ids=3
+--let $stop_position=0-3-3
+--let $start_position=0-1-0
+--source suite/rpl/include/mysqlbinlog_slave_consistency.inc
+
+--echo #
+--echo #
+--echo # Test Case 23: Out of order GTIDs from domains or servers which are
+--echo # filtered out should not error
+--echo #
+--let $strict_mode=1
+--let $strict_mode_err=0
+--let $slave_sql_errno=0
+--let $do_domain_ids=0
+--let $ignore_domain_ids=
+--let $ignore_server_ids=3
+--let $stop_position=
+--let $start_position=
+--source suite/rpl/include/mysqlbinlog_slave_consistency.inc
+
+--echo #
+--echo #
+--echo # Test Case 24: Out of order GTIDs from included domains should error
+--echo #
+--let $strict_mode=1
+--let $strict_mode_err=1
+--let $slave_sql_errno=1950
+--let $do_domain_ids=1
+--let $ignore_domain_ids=
+--let $ignore_server_ids=
+--let $stop_position=
+--let $start_position=
+--source suite/rpl/include/mysqlbinlog_slave_consistency.inc
+
+--echo #
+--echo #
+--echo # Test Case 25: Out of order GTIDs from included servers should error
+--echo #
+--let $strict_mode=1
+--let $strict_mode_err=1
+--let $slave_sql_errno=1950
+--let $do_domain_ids=
+--let $ignore_domain_ids=
+--let $ignore_server_ids=1
+--let $stop_position=
+--let $start_position=
+--source suite/rpl/include/mysqlbinlog_slave_consistency.inc
+
+--echo #
+--echo #
+--echo # Test Case 26: Neither mysqlbinlog nor CHANGE MASTER TO should allow
+--echo # both do domain ids and ignore domain ids to be set together
+--echo #
+--connection slave
+--source include/stop_slave.inc
+
+--error 1201
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos, DO_DOMAIN_IDS=(0), IGNORE_DOMAIN_IDS=(1);
+
+--let $MYSQLD_DATADIR=`select @@datadir`
+--let BINLOG_FILENAME= query_get_value(SHOW BINARY LOGS, Log_name, 1)
+--let BINLOG_FILE_PARAM= $MYSQLD_DATADIR/$BINLOG_FILENAME.orig
+--error 1
+--exec $MYSQL_BINLOG $MYSQLD_DATADIR/$BINLOG_FILENAME --do-domain-ids=0 --ignore-domain-ids=1
+
+RESET MASTER;
+set global gtid_slave_pos="";
+CHANGE MASTER TO MASTER_USE_GTID=slave_pos, DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=();
+
+# Cleanup
+--connection master
+--source include/kill_binlog_dump_threads.inc
+RESET MASTER;
+--connection slave
+--source include/start_slave.inc
diff --git a/mysql-test/suite/rpl/t/rpl_mysqlbinlog_slave_consistency.test b/mysql-test/suite/rpl/t/rpl_mysqlbinlog_slave_consistency.test
index d2918e3a890f1..1caa71ca00277 100644
--- a/mysql-test/suite/rpl/t/rpl_mysqlbinlog_slave_consistency.test
+++ b/mysql-test/suite/rpl/t/rpl_mysqlbinlog_slave_consistency.test
@@ -1,401 +1,7 @@
-#
-# Purpose:
-#
-# This test ensures the mariadb-binlog command line tool filters events
-# by domain id (via --do-domain-ids and --ignore-domain-ids) and server id (via
-# --ignore-server-ids) in the same way that a replica server does.
-#
-#
-# Methodology:
-#
-# This test validates that the domain and server id filtering logic of
-# mariadb-binlog matches that of a replica server. In particular, it validates
-# a mariadb-binlog replay of a master's binary log is consistent with a
-# replica's state which is configured using the same filtering configuration.
-#
-# It uses a repeatable process to allow for multiple test cases that span
-# different filtering configurations. First, a master is seeded with an initial
-# set of SQL statements with varying domain and server ids. Then, a set of
-# filtering parameters supported by both mariadb-binlog and replica
-# capabilities are defined. The replica is configured using these parameters
-# and run it until it has processed all events from the primary server; it is
-# stopped afterward. For mariadb-binlog validation, the binary log of the
-# primary server is copied to a different location for later replay. The
-# primary is then reset to its initial state (i.e. the tables are dropped and
-# the logs are reset). The mariadb-binlog tool is then used to replay the
-# copied binary log file back onto the clean primary server under the same
-# filtering conditions as the replica. At this point, the data on the primary
-# and replica should be exactly the same because the filtering conditions were
-# the same, and all existing tables on both servers are compared using.
-# checksums.
-#
-#
-# References:
-#
-# MDEV-20119: Implement the --do-domain-ids, --ignore-domain-ids, and
-# --ignore-server-ids options for mysqlbinlog
-#
+# For test description, see mysql-test/suite/rpl/t/rpl_mysqlbinlog_slave_consistency.inc
+--source include/have_innodb.inc
--source include/master-slave.inc
-
---connection slave
-SET sql_log_bin=0;
-call mtr.add_suppression("Slave: An attempt was made.*");
-call mtr.add_suppression("Both DO_DOMAIN_IDS & IGNORE_DOMAIN_IDS lists can't be non-empty at the same time");
-SET sql_log_bin=1;
-
---source include/stop_slave.inc
-
---connection master
-RESET MASTER;
-
-# Save old state
-let $ORIG_GTID_DOMAIN_ID = `select @@session.gtid_domain_id`;
-let $ORIG_SERVER_ID = `select @@session.server_id`;
-
---connection slave
---source include/start_slave.inc
-
-# Initial tests do not use strict mode
---let $strict_mode=0
---let $slave_sql_errno=0
-
---echo #
---echo #
---echo # Test Case 1: Base case to ensure that mariadb-binlog and replica
---echo # are consistent without any filtering
---echo #
---let $do_domain_ids=
---let $ignore_domain_ids=
---let $ignore_server_ids=
---let $stop_position=
---let $start_position=
---source include/mysqlbinlog_slave_consistency.inc
-
---echo #
---echo #
---echo # Test Case 2: Ensure filtering by a single id in --do-domain-ids is
---echo # consistent between mariadb-binlog and replica
---echo #
---let $do_domain_ids=0
---let $ignore_domain_ids=
---let $ignore_server_ids=
---let $stop_position=
---let $start_position=
---source include/mysqlbinlog_slave_consistency.inc
-
---echo #
---echo #
---echo # Test Case 3: Ensure filtering by multiple ids in --do-domain-ids is
---echo # consistent between mariadb-binlog and replica
---echo #
---let $do_domain_ids=0,1
---let $ignore_domain_ids=
---let $ignore_server_ids=
---let $stop_position=
---let $start_position=
---source include/mysqlbinlog_slave_consistency.inc
-
---echo #
---echo # Test Case 4: Ensure filtering by a single id in --ignore-domain-ids
---echo # is consistent between mariadb-binlog and replica
---echo #
---let $do_domain_ids=
---let $ignore_domain_ids=0
---let $ignore_server_ids=
---let $stop_position=
---let $start_position=
---source include/mysqlbinlog_slave_consistency.inc
-
---echo #
---echo #
---echo # Test Case 5: Ensure filtering by multiple ids in --ignore-domain-ids
---echo # is consistent between mariadb-binlog and replica
---echo #
---let $do_domain_ids=
---let $ignore_domain_ids=0,1
---let $ignore_server_ids=
---let $stop_position=
---let $start_position=
---source include/mysqlbinlog_slave_consistency.inc
-
---echo #
---echo # Test Case 6: Ensure filtering by a single id in --ignore-server-ids
---echo # is consistent between mariadb-binlog and replica
---echo #
---let $do_domain_ids=
---let $ignore_domain_ids=
---let $ignore_server_ids=1
---let $stop_position=
---let $start_position=
---source include/mysqlbinlog_slave_consistency.inc
-
---echo #
---echo #
---echo # Test Case 7: Ensure filtering by multiple ids in --ignore-server-ids
---echo # is consistent between mariadb-binlog and replica
---echo #
---let $do_domain_ids=
---let $ignore_domain_ids=
---let $ignore_server_ids=0,1
---let $stop_position=
---let $start_position=
---source include/mysqlbinlog_slave_consistency.inc
-
---echo #
---echo #
---echo # Test Case 8: Ensure stop position consistency
---echo #
---let $do_domain_ids=
---let $ignore_domain_ids=
---let $ignore_server_ids=
---let $stop_position=1-1-2
---let $start_position=
---source include/mysqlbinlog_slave_consistency.inc
-
---echo #
---echo #
---echo # Test Case 9: Ensure start position consistency
---echo #
---let $do_domain_ids=
---let $ignore_domain_ids=
---let $ignore_server_ids=
---let $stop_position=
---let $start_position=1-4-2
---source include/mysqlbinlog_slave_consistency.inc
-
---echo #
---echo #
---echo # Test Case 10: Ensure consistency when filtering by both
---echo # --do-domain-ids and --ignore-server-ids
---echo #
---let $do_domain_ids=0
---let $ignore_domain_ids=
---let $ignore_server_ids=1
---let $stop_position=
---let $start_position=
---source include/mysqlbinlog_slave_consistency.inc
-
---echo #
---echo #
---echo # Test Case 11: Ensure consistency when filtering by both
---echo # --ignore-domain-ids and --ignore-server-ids
---echo #
---let $do_domain_ids=
---let $ignore_domain_ids=1,2
---let $ignore_server_ids=1
---let $stop_position=
---let $start_position=
---source include/mysqlbinlog_slave_consistency.inc
-
---echo #
---echo #
---echo # Test Case 12: Ensure consistency when filtering by
---echo # --do-domain-ids with a stop position
---echo #
---let $do_domain_ids=0
---let $ignore_domain_ids=
---let $ignore_server_ids=
---let $stop_position=0-1-4
---let $start_position=
---source include/mysqlbinlog_slave_consistency.inc
-
---echo #
---echo #
---echo # Test Case 13: Ensure consistency when filtering by
---echo # --ignore-domain-ids with a stop position
---echo #
---let $do_domain_ids=
---let $ignore_domain_ids=0
---let $ignore_server_ids=
---let $stop_position=0-1-4
---let $start_position=
---source include/mysqlbinlog_slave_consistency.inc
-
---echo #
---echo #
---echo # Test Case 14: Ensure consistency when filtering by
---echo # --ignore-server-ids with a stop position
---echo #
---let $do_domain_ids=
---let $ignore_domain_ids=
---let $ignore_server_ids=3
---let $stop_position=0-1-4
---let $start_position=
---source include/mysqlbinlog_slave_consistency.inc
-
---echo #
---echo #
---echo # Test Case 15: Ensure consistency when filtering by
---echo # --do-domain-ids with a start position
---echo #
---let $do_domain_ids=2
---let $ignore_domain_ids=
---let $ignore_server_ids=
---let $stop_position=
---let $start_position=1-1-2
---source include/mysqlbinlog_slave_consistency.inc
-
---echo #
---echo #
---echo # Test Case 16: Ensure consistency when filtering by
---echo # --ignore-domain-ids with a start position
---echo #
---let $do_domain_ids=
---let $ignore_domain_ids=0
---let $ignore_server_ids=
---let $stop_position=
---let $start_position=0-1-1
---source include/mysqlbinlog_slave_consistency.inc
-
---echo #
---echo #
---echo # Test Case 17: Ensure consistency when filtering by
---echo # --ignore-server-ids with a start position
---echo #
---let $do_domain_ids=
---let $ignore_domain_ids=
---let $ignore_server_ids=1
---let $stop_position=
---let $start_position=0-1-1
---source include/mysqlbinlog_slave_consistency.inc
-
---echo #
---echo #
---echo # Test Case 18: Ensure consistency when filtering by
---echo # --do-domain-ids with both a start position and stop position that
---echo # all have the same domain id
---echo #
---let $do_domain_ids=0
---let $ignore_domain_ids=
---let $ignore_server_ids=
---let $stop_position=0-3-3
---let $start_position=0-1-1
---source include/mysqlbinlog_slave_consistency.inc
-
---echo #
---echo #
---echo # Test Case 19: Ensure consistency when filtering by
---echo # --do-domain-ids with both a start position and stop position that
---echo # have differing domain ids. Due to the implicit filtering in stop
---echo # position, the result should be empty (no tables replicated).
---echo #
---let $do_domain_ids=1
---let $ignore_domain_ids=
---let $ignore_server_ids=
---let $stop_position=0-3-3
---let $start_position=0-1-1
---source include/mysqlbinlog_slave_consistency.inc
-
---echo #
---echo #
---echo # Test Case 20: Ensure consistency when filtering by
---echo # --ignore-domain-ids with both a start position and stop position that
---echo # all have the same domain id. The result set should be empty due to
---echo # implicit filtering from stop position and ignoring that same domain.
---echo #
---let $do_domain_ids=
---let $ignore_domain_ids=0
---let $ignore_server_ids=
---let $stop_position=0-3-3
---let $start_position=0-1-1
---source include/mysqlbinlog_slave_consistency.inc
-
---echo #
---echo #
---echo # Test Case 21: Ensure consistency when filtering by
---echo # --ignore-domain-ids with both a start position and stop position that
---echo # have differing domain ids. The ignore domain ids should take no
---echo # effect due to the implicit filtering by stop position, i.e. the
---echo # specified domain to ignore is already being filtered.
---echo #
---let $do_domain_ids=
---let $ignore_domain_ids=1
---let $ignore_server_ids=
---let $stop_position=0-3-3
---let $start_position=0-1-1
---source include/mysqlbinlog_slave_consistency.inc
-
---echo #
---echo #
---echo # Test Case 22: Ensure consistency when filtering by
---echo # --ignore-server-ids with both a start position and stop position.
---echo #
---let $do_domain_ids=
---let $ignore_domain_ids=
---let $ignore_server_ids=3
---let $stop_position=0-3-3
---let $start_position=0-1-0
---source include/mysqlbinlog_slave_consistency.inc
-
---echo #
---echo #
---echo # Test Case 23: Out of order GTIDs from domains or servers which are
---echo # filtered out should not error
---echo #
---let $strict_mode=1
---let $strict_mode_err=0
---let $slave_sql_errno=0
---let $do_domain_ids=0
---let $ignore_domain_ids=
---let $ignore_server_ids=3
---let $stop_position=
---let $start_position=
---source include/mysqlbinlog_slave_consistency.inc
-
---echo #
---echo #
---echo # Test Case 24: Out of order GTIDs from included domains should error
---echo #
---let $strict_mode=1
---let $strict_mode_err=1
---let $slave_sql_errno=1950
---let $do_domain_ids=1
---let $ignore_domain_ids=
---let $ignore_server_ids=
---let $stop_position=
---let $start_position=
---source include/mysqlbinlog_slave_consistency.inc
-
---echo #
---echo #
---echo # Test Case 25: Out of order GTIDs from included servers should error
---echo #
---let $strict_mode=1
---let $strict_mode_err=1
---let $slave_sql_errno=1950
---let $do_domain_ids=
---let $ignore_domain_ids=
---let $ignore_server_ids=1
---let $stop_position=
---let $start_position=
---source include/mysqlbinlog_slave_consistency.inc
-
---echo #
---echo #
---echo # Test Case 26: Neither mysqlbinlog nor CHANGE MASTER TO should allow
---echo # both do domain ids and ignore domain ids to be set together
---echo #
---connection slave
---source include/stop_slave.inc
-
---error 1201
-CHANGE MASTER TO MASTER_USE_GTID=slave_pos, DO_DOMAIN_IDS=(0), IGNORE_DOMAIN_IDS=(1);
-
---let $MYSQLD_DATADIR=`select @@datadir`
---let BINLOG_FILENAME= query_get_value(SHOW BINARY LOGS, Log_name, 1)
---let BINLOG_FILE_PARAM= $MYSQLD_DATADIR/$BINLOG_FILENAME.orig
---error 1
---exec $MYSQL_BINLOG $MYSQLD_DATADIR/$BINLOG_FILENAME --do-domain-ids=0 --ignore-domain-ids=1
-
-RESET MASTER;
-set global gtid_slave_pos="";
-CHANGE MASTER TO MASTER_USE_GTID=slave_pos, DO_DOMAIN_IDS=(), IGNORE_DOMAIN_IDS=();
---source include/start_slave.inc
-
-# Cleanup
---connection master
-RESET MASTER;
-
+--let $blob_trx_size= 1
+--source rpl_mysqlbinlog_slave_consistency.inc
--source include/rpl_end.inc
-
---echo # End of tests (rpl.rpl_mysqlbinlog_slave_consistency)
+--echo # End of test rpl.rpl_mysqlbinlog_slave_consistency.test
diff --git a/mysql-test/suite/rpl/t/rpl_packet.test b/mysql-test/suite/rpl/t/rpl_packet.test
index f1814e61f9b41..2a5d3e4b6886f 100644
--- a/mysql-test/suite/rpl/t/rpl_packet.test
+++ b/mysql-test/suite/rpl/t/rpl_packet.test
@@ -142,6 +142,7 @@ connection slave;
STOP SLAVE;
RESET SLAVE;
--connection master
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
diff --git a/mysql-test/suite/rpl/t/rpl_parallel_optimistic_until.test b/mysql-test/suite/rpl/t/rpl_parallel_optimistic_until.test
index 0797e8bf220ea..23c5f4330d77f 100644
--- a/mysql-test/suite/rpl/t/rpl_parallel_optimistic_until.test
+++ b/mysql-test/suite/rpl/t/rpl_parallel_optimistic_until.test
@@ -15,6 +15,7 @@ RESET MASTER;
RESET SLAVE;
--connection master
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
CREATE TABLE t1 (a int primary key, b text) ENGINE=InnoDB;
--let $a0 = 25
diff --git a/mysql-test/suite/rpl/t/rpl_parallel_seq.test b/mysql-test/suite/rpl/t/rpl_parallel_seq.test
index 17870eb3b86f3..8817336fac5be 100644
--- a/mysql-test/suite/rpl/t/rpl_parallel_seq.test
+++ b/mysql-test/suite/rpl/t/rpl_parallel_seq.test
@@ -80,6 +80,7 @@ SET DEBUG_SYNC = 'now SIGNAL continue_worker';
RESET MASTER;
SET @@global.gtid_slave_pos="";
--connection master
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
# Load from master
diff --git a/mysql-test/suite/rpl/t/rpl_parallel_temptable.test b/mysql-test/suite/rpl/t/rpl_parallel_temptable.test
index 8bab4af2b436c..41da95ba5cd97 100644
--- a/mysql-test/suite/rpl/t/rpl_parallel_temptable.test
+++ b/mysql-test/suite/rpl/t/rpl_parallel_temptable.test
@@ -6,296 +6,6 @@
--let $rpl_topology=1->2
--source include/rpl_init.inc
---echo *** MDEV-6321: close_temporary_tables() in format description event not serialised correctly ***
-
---connection server_2
-SET @old_parallel_threads=@@GLOBAL.slave_parallel_threads;
---source include/stop_slave.inc
-SET GLOBAL slave_parallel_threads=5;
---source include/start_slave.inc
-
---connection server_1
-CREATE TABLE t1 (a INT PRIMARY KEY, b VARCHAR(100) CHARACTER SET utf8);
---save_master_pos
-
---connection server_2
---sync_with_master
---source include/stop_slave.inc
-
-
---connection server_1
-SET gtid_domain_id= 1;
-INSERT INTO t1 VALUES (1, 0);
-
-CREATE TEMPORARY TABLE t2 (a int);
-
---connection default
-SET gtid_domain_id= 2;
-CREATE TEMPORARY TABLE t3 (a INT PRIMARY KEY);
-CREATE TEMPORARY TABLE t4 (a int);
-INSERT INTO t3 VALUES (100);
-INSERT INTO t4 SELECT a+1 FROM t3;
-
---connection server_1
-INSERT INTO t2 VALUES (2), (4), (6), (8), (10), (12), (14), (16), (18), (20);
-INSERT INTO t2 VALUES (3), (6), (9), (12), (15), (18);
-INSERT INTO t2 VALUES (4), (8), (12), (16), (20);
-
---connection default
-INSERT INTO t3 SELECT a+2 FROM t4;
-INSERT INTO t4 SELECT a+4 FROM t3;
-
---connection server_1
-INSERT INTO t2 VALUES (5), (10), (15), (20);
-INSERT INTO t2 VALUES (6), (12), (18);
-INSERT INTO t2 VALUES (7), (14);
-INSERT INTO t2 VALUES (8), (16);
-INSERT INTO t2 VALUES (9), (18);
-INSERT INTO t2 VALUES (10), (20);
-
---connection default
-INSERT INTO t3 SELECT a+8 FROM t4;
-INSERT INTO t4 SELECT a+16 FROM t3;
-
---connection server_1
-INSERT INTO t2 VALUES (11);
-INSERT INTO t2 VALUES (12);
-INSERT INTO t2 VALUES (13);
-
---connection default
-INSERT INTO t3 SELECT a+32 FROM t4;
-
---connection server_1
-INSERT INTO t2 VALUES (14);
-INSERT INTO t2 VALUES (15);
-INSERT INTO t2 VALUES (16);
-
---connection default
-INSERT INTO t4 SELECT a+64 FROM t3;
-
---connection server_1
-INSERT INTO t2 VALUES (17);
-INSERT INTO t2 VALUES (18);
-INSERT INTO t2 VALUES (19);
-
---connection default
-INSERT INTO t3 SELECT a+128 FROM t4;
-
---connection server_1
-INSERT INTO t2 VALUES (20);
-
---connection default
-INSERT INTO t1 SELECT a, a MOD 7 FROM t3;
-INSERT INTO t1 SELECT a, a MOD 7 FROM t4;
-
---connection server_1
-INSERT INTO t1 SELECT a, COUNT(*) FROM t2 GROUP BY a;
-
-# Crash the master server, so that temporary tables are implicitly dropped.
---write_file $MYSQLTEST_VARDIR/tmp/mysqld.1.expect
-wait
-EOF
-
-FLUSH TABLES;
-SET SESSION debug_dbug="+d,crash_dispatch_command_before";
---error 2006,2013
-SELECT 1;
-
---source include/wait_until_disconnected.inc
---connection default
---source include/wait_until_disconnected.inc
-
---append_file $MYSQLTEST_VARDIR/tmp/mysqld.1.expect
-restart
-EOF
-
---connection default
---enable_reconnect
---source include/wait_until_connected_again.inc
-
---connection server_1
---enable_reconnect
---source include/wait_until_connected_again.inc
-
-INSERT INTO t1 VALUES (0, 1);
---save_master_pos
-
---connection server_2
-# Start the slave replicating the events.
-# The bug was that the format description event written after the crash could
-# be fetched ahead of the execution of the temporary table events and executed
-# out-of-band. This would cause drop of all temporary tables and thus failure
-# for execution of remaining events.
-
---source include/start_slave.inc
---sync_with_master
-
-SELECT * FROM t1 WHERE a <= 20 ORDER BY a;
-SELECT COUNT(*) FROM t1 WHERE a BETWEEN 100+0 AND 100+256;
-SHOW STATUS LIKE 'Slave_open_temp_tables';
-
-
---echo *** Test that if master logged partial event group before crash, we finish that group correctly before executing format description event ***
-
---source include/stop_slave.inc
-
---connection server_1
-CALL mtr.add_suppression("Statement accesses nontransactional table as well as transactional or temporary table, and writes to any of them");
-SET gtid_domain_id= 1;
-DELETE FROM t1;
-ALTER TABLE t1 ENGINE=InnoDB;
-CREATE TEMPORARY TABLE t2 (a INT PRIMARY KEY);
-INSERT INTO t2 VALUES (1);
-INSERT INTO t2 VALUES (2);
-
---connection default
-SET gtid_domain_id= 2;
-CREATE TEMPORARY TABLE t3 (a INT PRIMARY KEY);
-INSERT INTO t3 VALUES (10);
-INSERT INTO t3 VALUES (20);
-
---connection server_1
-INSERT INTO t1 SELECT a, 'server_1' FROM t2;
-
---connection default
-INSERT INTO t1 SELECT a, 'default' FROM t3;
-
---connection server_1
-INSERT INTO t1 SELECT a+2, '+server_1' FROM t2;
-
-# Crash the master server in the middle of writing an event group.
---write_file $MYSQLTEST_VARDIR/tmp/mysqld.1.expect
-wait
-EOF
-
-FLUSH TABLES;
-SET SESSION debug_dbug="+d,crash_before_writing_xid";
---error 2006,2013
-INSERT INTO t1 SELECT a+4, '++server_1' FROM t2;
-
---source include/wait_until_disconnected.inc
---connection default
---source include/wait_until_disconnected.inc
-
---append_file $MYSQLTEST_VARDIR/tmp/mysqld.1.expect
-restart
-EOF
-
---connection default
---enable_reconnect
---source include/wait_until_connected_again.inc
-
---connection server_1
---enable_reconnect
---source include/wait_until_connected_again.inc
-
-INSERT INTO t1 VALUES (0, 1);
-#--save_master_pos
---source include/save_master_gtid.inc
-
---connection server_2
-# Start the slave replicating the events.
-# The main thing to test here is that the slave will know that it
-# needs to abort the partially received event group, so that the
-# execution of format_description event will not wait infinitely
-# for a commit of the incomplete group that never happens.
-
-# Apart from the suppression, MDEV-27697 refinement to the original test needs
-# an allowance to one time accept malformed event group.
-set @@sql_log_bin=0;
-call mtr.add_suppression("Unexpected break of being relay-logged GTID 1-1-32 event group by the current GTID event 0-1-4");
-set @@sql_log_bin=1;
-set @@global.debug_dbug="+d,slave_discard_xid_for_gtid_0_x_1000";
---source include/start_slave.inc
-#--sync_with_master
---source include/sync_with_master_gtid.inc
-set @@global.debug_dbug="";
-
-SELECT * FROM t1 ORDER BY a;
-SHOW STATUS LIKE 'Slave_open_temp_tables';
-
---connection server_1
-# This FLUSH can be removed once MDEV-6608 is fixed.
-FLUSH LOGS;
-
-
---echo *** MDEV-7936: Assertion `!table || table->in_use == _current_thd()' failed on parallel replication in optimistic mode ***
-
---connection server_1
-CREATE TEMPORARY TABLE t4 (a INT PRIMARY KEY) ENGINE=InnoDB;
-SET @old_dbug= @@SESSION.debug_dbug;
-SET SESSION debug_dbug="+d,binlog_force_commit_id";
-SET @commit_id= 10000;
-INSERT INTO t4 VALUES (30);
-INSERT INTO t4 VALUES (31);
-SET SESSION debug_dbug= @old_dbug;
-INSERT INTO t1 SELECT a, "conservative" FROM t4;
-DROP TEMPORARY TABLE t4;
-SELECT * FROM t1 WHERE a >= 30 ORDER BY a;
---source include/save_master_gtid.inc
-
---connection server_2
---source include/sync_with_master_gtid.inc
-
-SELECT * FROM t1 WHERE a >= 30 ORDER BY a;
-
---source include/stop_slave.inc
-SET @old_mode= @@GLOBAL.slave_parallel_mode;
-SET GLOBAL slave_parallel_mode=optimistic;
-
---connection server_1
-CREATE TEMPORARY TABLE t4 (a INT PRIMARY KEY) ENGINE=InnoDB;
-INSERT INTO t4 VALUES (32);
-INSERT INTO t4 VALUES (33);
-INSERT INTO t1 SELECT a, "optimistic" FROM t4;
-DROP TEMPORARY TABLE t4;
-
-SELECT * FROM t1 WHERE a >= 30 ORDER BY a;
---source include/save_master_gtid.inc
-
---connection server_2
---source include/start_slave.inc
---source include/sync_with_master_gtid.inc
-
-SELECT * FROM t1 WHERE a >= 30 ORDER BY a;
-
---source include/stop_slave.inc
-SET GLOBAL slave_parallel_mode=@old_mode;
---source include/start_slave.inc
-
-
---echo *** MDEV33426: Memory allocation accounting incorrect for replicated temptable
---connection server_1
-CREATE TEMPORARY TABLE t5 (a int) ENGINE=Aria;
-CREATE TEMPORARY TABLE t6 (a int) ENGINE=Heap;
-INSERT INTO t5 VALUES (1);
-INSERT INTO t6 VALUES (2);
---save_master_pos
-
---connection server_2
---sync_with_master
---source include/stop_slave.inc
-
---connection server_1
-INSERT INTO t1 SELECT a+40, 5 FROM t5;
-INSERT INTO t1 SELECT a+40, 6 FROM t6;
-DROP TABLE t5, t6;
-
---save_master_pos
-
---connection server_2
---source include/start_slave.inc
---sync_with_master
-SELECT * FROM t1 WHERE a>=40 ORDER BY a;
-
-# Clean up.
-
---connection server_2
---source include/stop_slave.inc
-SET GLOBAL slave_parallel_threads=@old_parallel_threads;
---source include/start_slave.inc
-
---connection server_1
-DROP TABLE t1;
+--source include/rpl_parallel_temptable.inc
--source include/rpl_end.inc
diff --git a/mysql-test/suite/rpl/t/rpl_perfschema_applier_status_by_coordinator.test b/mysql-test/suite/rpl/t/rpl_perfschema_applier_status_by_coordinator.test
index 794a0c631d757..a892efa37b9ff 100644
--- a/mysql-test/suite/rpl/t/rpl_perfschema_applier_status_by_coordinator.test
+++ b/mysql-test/suite/rpl/t/rpl_perfschema_applier_status_by_coordinator.test
@@ -206,11 +206,14 @@ EOF
--connection master
drop table t;
-reset master;
-
--connection slave
--let $rpl_only_running_threads= 1
--source include/stop_slave.inc
+--connection master
+--source include/kill_binlog_dump_threads.inc
+reset master;
+
+--connection slave
reset slave;
reset master;
set @@global.gtid_slave_pos= "";
diff --git a/mysql-test/suite/rpl/t/rpl_perfschema_applier_status_by_worker.test b/mysql-test/suite/rpl/t/rpl_perfschema_applier_status_by_worker.test
index 72559fc40905c..de9313a87055d 100644
--- a/mysql-test/suite/rpl/t/rpl_perfschema_applier_status_by_worker.test
+++ b/mysql-test/suite/rpl/t/rpl_perfschema_applier_status_by_worker.test
@@ -232,6 +232,7 @@ select Last_Error_Message from performance_schema.replication_applier_status_by_
RESET SLAVE;
--connection master
DROP TABLE t;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
--echo
diff --git a/mysql-test/suite/rpl/t/rpl_reset_master.test b/mysql-test/suite/rpl/t/rpl_reset_master.test
new file mode 100644
index 0000000000000..05f67cb3b3da1
--- /dev/null
+++ b/mysql-test/suite/rpl/t/rpl_reset_master.test
@@ -0,0 +1,53 @@
+--source include/have_debug_sync.inc
+--source include/have_binlog_format_mixed.inc
+--source include/master-slave.inc
+
+--connection master
+CREATE TABLE t1 (a INT PRIMARY KEY);
+INSERT INTO t1 VALUES (1);
+FLUSH BINARY LOGS;
+INSERT INTO t1 VALUES (2);
+
+--echo *** Test that RESET MASTER fails when a slave is connected.
+--error ER_BINLOG_IN_USE
+RESET MASTER;
+
+--connection master
+--source include/save_master_gtid.inc
+
+--connection slave
+--source include/sync_with_master_gtid.inc
+--source include/stop_slave.inc
+
+--connection master
+--echo *** Test that RESET MASTER fails on concurrent SHOW BINLOG EVENTS.
+--source include/kill_binlog_dump_threads.inc
+
+--connection master1
+SET debug_sync= 'after_show_binlog_events SIGNAL waiting WAIT_FOR go';
+--disable_result_log
+send SHOW BINLOG EVENTS in 'master-bin.000001';
+
+--connection master
+SET debug_sync= 'now WAIT_FOR waiting';
+--error ER_BINLOG_IN_USE
+RESET MASTER;
+SET debug_sync= 'now SIGNAL go';
+
+--connection master1
+reap;
+--enable_result_log
+
+--connection master
+--echo *** RESET MASTER works when no concurrent reader.
+RESET MASTER;
+
+DROP TABLE t1;
+--source include/save_master_gtid.inc
+
+--connection slave
+SET GLOBAL gtid_slave_pos= '';
+--source include/start_slave.inc
+--source include/sync_with_master_gtid.inc
+
+--source include/rpl_end.inc
diff --git a/mysql-test/suite/rpl/t/rpl_row_001.test b/mysql-test/suite/rpl/t/rpl_row_001.test
index 887c0961a63b5..a31570f4273f6 100644
--- a/mysql-test/suite/rpl/t/rpl_row_001.test
+++ b/mysql-test/suite/rpl/t/rpl_row_001.test
@@ -10,10 +10,11 @@
# we'll use table-level locking to delay slave SQL thread
eval CREATE TABLE t1 (n INT);
sync_slave_with_master;
+--source include/stop_slave.inc
connection master;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
connection slave;
-STOP SLAVE;
--source include/reset_slave.inc
connection master;
diff --git a/mysql-test/suite/rpl/t/rpl_row_mysqlbinlog.test b/mysql-test/suite/rpl/t/rpl_row_mysqlbinlog.test
index 9e10aaa621664..cce433a936662 100644
--- a/mysql-test/suite/rpl/t/rpl_row_mysqlbinlog.test
+++ b/mysql-test/suite/rpl/t/rpl_row_mysqlbinlog.test
@@ -98,6 +98,7 @@ sync_slave_with_master;
stop slave;
--source include/wait_for_slave_to_stop.inc
connection master;
+--source include/kill_binlog_dump_threads.inc
reset master;
connection slave;
--source include/reset_slave.inc
@@ -165,6 +166,7 @@ sync_slave_with_master;
stop slave;
--source include/wait_for_slave_to_stop.inc
connection master;
+--source include/kill_binlog_dump_threads.inc
reset master;
connection slave;
--source include/reset_slave.inc
@@ -228,6 +230,7 @@ sync_slave_with_master;
stop slave;
--source include/wait_for_slave_to_stop.inc
connection master;
+--source include/kill_binlog_dump_threads.inc
reset master;
connection slave;
--source include/reset_slave.inc
diff --git a/mysql-test/suite/rpl/t/rpl_semi_sync.test b/mysql-test/suite/rpl/t/rpl_semi_sync.test
index 6f85ef95b79ba..aca5c434e1242 100644
--- a/mysql-test/suite/rpl/t/rpl_semi_sync.test
+++ b/mysql-test/suite/rpl/t/rpl_semi_sync.test
@@ -48,7 +48,7 @@ set global rpl_semi_sync_master_enabled= 0;
set global rpl_semi_sync_slave_enabled= 0;
connection master;
-reset master;
+--source include/reset_master.inc
set global rpl_semi_sync_master_enabled= 0;
set global rpl_semi_sync_slave_enabled= 0;
@@ -103,7 +103,7 @@ show status like 'Rpl_semi_sync_master_status';
show status like 'Rpl_semi_sync_master_yes_tx';
# reset master to make sure the following test will start with a clean environment
-reset master;
+--source include/reset_master.inc
connection slave;
@@ -349,11 +349,12 @@ eval create table t1 (a int) engine = $engine_type;
drop table t1;
sync_slave_with_master;
+source include/stop_slave.inc;
echo [ test reset master ];
connection master;
-reset master;
+--source include/reset_master.inc
show status like 'Rpl_semi_sync_master_status';
show status like 'Rpl_semi_sync_master_no_tx';
@@ -361,7 +362,6 @@ show status like 'Rpl_semi_sync_master_yes_tx';
connection slave;
-source include/stop_slave.inc;
--source include/reset_slave.inc
# Kill the dump thread on master for previous slave connection and
@@ -404,7 +404,7 @@ connection slave;
source include/stop_slave.inc;
--source include/reset_slave.inc
connection master;
-reset master;
+--source include/reset_master.inc
# Kill the dump thread on master for previous slave connection and wait for it to exit
--source include/kill_binlog_dump_threads.inc
diff --git a/mysql-test/suite/rpl/t/rpl_semi_sync_fail_over.test b/mysql-test/suite/rpl/t/rpl_semi_sync_fail_over.test
index e4d0bc0acd123..5add124bd7a92 100644
--- a/mysql-test/suite/rpl/t/rpl_semi_sync_fail_over.test
+++ b/mysql-test/suite/rpl/t/rpl_semi_sync_fail_over.test
@@ -21,6 +21,7 @@ flush tables;
# Initial master
--connection server_1
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
SET @@global.max_binlog_size= 4096;
set @@global.rpl_semi_sync_master_enabled = 1;
@@ -199,10 +200,12 @@ SHOW VARIABLES LIKE 'gtid_binlog_state';
set global rpl_semi_sync_slave_enabled = 0;
set global rpl_semi_sync_master_enabled = 0;
set global rpl_semi_sync_master_wait_point=default;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
RESET SLAVE;
--connection server_2
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
RESET SLAVE;
set @@global.rpl_semi_sync_master_enabled = 0;
diff --git a/mysql-test/suite/rpl/t/rpl_semi_sync_gtid_reconnect.test b/mysql-test/suite/rpl/t/rpl_semi_sync_gtid_reconnect.test
index 96f7e805fc5fb..eed83570aab19 100644
--- a/mysql-test/suite/rpl/t/rpl_semi_sync_gtid_reconnect.test
+++ b/mysql-test/suite/rpl/t/rpl_semi_sync_gtid_reconnect.test
@@ -5,13 +5,15 @@ source include/master-slave.inc;
#
# Semisync initialization
#
+--connection slave
+source include/stop_slave.inc;
--connection master
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
--let $sav_enabled_master=`SELECT @@GLOBAL.rpl_semi_sync_master_enabled`
SET @@GLOBAL.rpl_semi_sync_master_enabled = 1;
--connection slave
-source include/stop_slave.inc;
--let $sav_enabled_slave=`SELECT @@GLOBAL.rpl_semi_sync_slave_enabled`
SET @@GLOBAL. rpl_semi_sync_slave_enabled = 1;
source include/start_slave.inc;
@@ -46,6 +48,7 @@ INSERT INTO t1 SET a = 3;
--source include/stop_slave_io.inc
--connection master
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
--eval SET @@global.gtid_binlog_state = '$resume_gtid'
diff --git a/mysql-test/suite/rpl/t/rpl_server_id2.test b/mysql-test/suite/rpl/t/rpl_server_id2.test
index 6dc2f791954fe..e5df2d3a5b26f 100644
--- a/mysql-test/suite/rpl/t/rpl_server_id2.test
+++ b/mysql-test/suite/rpl/t/rpl_server_id2.test
@@ -36,6 +36,7 @@ drop table t1;
# in presense of --replicate-same-server-id the slave is started with.
connection master;
+--source include/kill_binlog_dump_threads.inc
reset master;
# setting the until position to correspond to the last position of
diff --git a/mysql-test/suite/rpl/t/rpl_slave_alias_replica.test b/mysql-test/suite/rpl/t/rpl_slave_alias_replica.test
index 06cf12ab7a991..f506a980f7d91 100644
--- a/mysql-test/suite/rpl/t/rpl_slave_alias_replica.test
+++ b/mysql-test/suite/rpl/t/rpl_slave_alias_replica.test
@@ -68,6 +68,7 @@ RESET REPLICA ALL;
set @@global.gtid_slave_pos= "";
--connection master
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
CREATE TABLE t(f INT) ENGINE=INNODB;
INSERT INTO t VALUES (10);
diff --git a/mysql-test/suite/rpl/t/rpl_slave_skip.test b/mysql-test/suite/rpl/t/rpl_slave_skip.test
index 933e7d14f9b80..445cd8f8a5dc3 100644
--- a/mysql-test/suite/rpl/t/rpl_slave_skip.test
+++ b/mysql-test/suite/rpl/t/rpl_slave_skip.test
@@ -57,6 +57,7 @@ STOP SLAVE;
--source include/wait_for_slave_to_stop.inc
--source include/reset_slave.inc
connection master;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
SET SESSION BINLOG_FORMAT=STATEMENT;
diff --git a/mysql-test/suite/rpl/t/rpl_start_alter_7.test b/mysql-test/suite/rpl/t/rpl_start_alter_7.test
index 0e72b43e33f7c..8ce54c6bfb9fb 100644
--- a/mysql-test/suite/rpl/t/rpl_start_alter_7.test
+++ b/mysql-test/suite/rpl/t/rpl_start_alter_7.test
@@ -105,10 +105,12 @@ SET GLOBAL gtid_slave_pos= '';
--connection server_1
--eval set global binlog_alter_two_phase=$binlog_alter_two_phase;
set global gtid_domain_id= 0;
+--source include/kill_binlog_dump_threads.inc
reset master;
--connection server_2
set global gtid_domain_id= 0;
--eval set global binlog_alter_two_phase=$binlog_alter_two_phase
+--source include/kill_binlog_dump_threads.inc
reset master;
--disconnect server_1
diff --git a/mysql-test/suite/rpl/t/rpl_start_alter_8.test b/mysql-test/suite/rpl/t/rpl_start_alter_8.test
index ac7abca255e48..17d46ebdb3533 100644
--- a/mysql-test/suite/rpl/t/rpl_start_alter_8.test
+++ b/mysql-test/suite/rpl/t/rpl_start_alter_8.test
@@ -102,10 +102,12 @@ SET GLOBAL gtid_slave_pos= '';
--connection server_1
--eval set global binlog_alter_two_phase=$binlog_alter_two_phase;
set global gtid_domain_id= 0;
+--source include/kill_binlog_dump_threads.inc
reset master;
--connection server_2
set global gtid_domain_id= 0;
--eval set global binlog_alter_two_phase=$binlog_alter_two_phase;
+--source include/kill_binlog_dump_threads.inc
reset master;
--disconnect server_1
diff --git a/mysql-test/suite/rpl/t/rpl_start_alter_mysqlbinlog_1.test b/mysql-test/suite/rpl/t/rpl_start_alter_mysqlbinlog_1.test
index a2f6adcc554d3..52552cdbcb10e 100644
--- a/mysql-test/suite/rpl/t/rpl_start_alter_mysqlbinlog_1.test
+++ b/mysql-test/suite/rpl/t/rpl_start_alter_mysqlbinlog_1.test
@@ -26,6 +26,7 @@ set global gtid_strict_mode=1;
--let $MYSQLD_DATADIR= `select @@datadir;`
--exec $MYSQL_BINLOG $MYSQLD_DATADIR/master-bin.000001 > $MYSQLTEST_VARDIR/tmp/master.sql
select @@gtid_binlog_state;
+--source include/kill_binlog_dump_threads.inc
RESET master;
--connection slave
diff --git a/mysql-test/suite/rpl/t/rpl_start_alter_mysqlbinlog_2.test b/mysql-test/suite/rpl/t/rpl_start_alter_mysqlbinlog_2.test
index 1cbc5b67fb93f..2de8c557e3978 100644
--- a/mysql-test/suite/rpl/t/rpl_start_alter_mysqlbinlog_2.test
+++ b/mysql-test/suite/rpl/t/rpl_start_alter_mysqlbinlog_2.test
@@ -157,10 +157,12 @@ SET GLOBAL gtid_slave_pos= '';
--connection server_1
SET GLOBAL binlog_alter_two_phase=@save_binlog_alter_two_phase;
SET GLOBAL gtid_domain_id= 0;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
--connection server_2
SET GLOBAL gtid_domain_id= 0;
SET GLOBAL binlog_alter_two_phase=@save_binlog_alter_two_phase;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
--disconnect server_1
diff --git a/mysql-test/suite/rpl/t/rpl_stm_000001.test b/mysql-test/suite/rpl/t/rpl_stm_000001.test
index 3851e61868b45..6d1bb047e49d3 100644
--- a/mysql-test/suite/rpl/t/rpl_stm_000001.test
+++ b/mysql-test/suite/rpl/t/rpl_stm_000001.test
@@ -49,10 +49,11 @@ connection master;
# we'll use table-level locking to delay slave SQL thread
eval create table t1 (n int);
sync_slave_with_master;
+stop slave;
connection master;
+--source include/kill_binlog_dump_threads.inc
reset master;
connection slave;
-stop slave;
--source include/reset_slave.inc
connection master;
diff --git a/mysql-test/suite/rpl/t/rpl_stm_until.test b/mysql-test/suite/rpl/t/rpl_stm_until.test
index ebfd355b299e9..bd43d2cab2a52 100644
--- a/mysql-test/suite/rpl/t/rpl_stm_until.test
+++ b/mysql-test/suite/rpl/t/rpl_stm_until.test
@@ -136,6 +136,7 @@ connection master;
--disable_warnings
drop table if exists t1;
--enable_warnings
+--source include/kill_binlog_dump_threads.inc
reset master;
create table t1 (a int primary key auto_increment);
save_master_pos;
diff --git a/mysql-test/suite/rpl/t/rpl_system_versioning_partitions.test b/mysql-test/suite/rpl/t/rpl_system_versioning_partitions.test
index 70103acf86201..dbe0876447823 100644
--- a/mysql-test/suite/rpl/t/rpl_system_versioning_partitions.test
+++ b/mysql-test/suite/rpl/t/rpl_system_versioning_partitions.test
@@ -38,12 +38,18 @@
--echo #
--echo # Initialize system-versioned and partitioned table and its data
+--connection slave
+--source include/stop_slave.inc
--connection master
# Fix the timestamp for the system versioned row_start and row_end fields, so
# the later hard-coded BINLOG base64 data can find the rows.
SET timestamp=UNIX_TIMESTAMP('2025-01-01 01:00:00.000000');
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
+--connection slave
+--source include/start_slave.inc
+--connection master
create table t1 (x int) engine=InnoDB with system versioning partition by system_time limit 3 partitions 5;
insert into t1 values(1);
diff --git a/mysql-test/suite/rpl/t/rpl_timezone.test b/mysql-test/suite/rpl/t/rpl_timezone.test
index 4b5778ca24591..eaa062c595486 100644
--- a/mysql-test/suite/rpl/t/rpl_timezone.test
+++ b/mysql-test/suite/rpl/t/rpl_timezone.test
@@ -179,6 +179,7 @@ SET @@session.time_zone = default;
# Insert 2 values into timestamp col with different time_zone. Check result.
--connection master
+--source include/kill_binlog_dump_threads.inc
reset master;
CREATE TABLE t1 (date timestamp NOT NULL default CURRENT_TIMESTAMP on update CURRENT_TIMESTAMP, a int(11) default NULL);
diff --git a/mysql-test/suite/rpl/t/rpl_trigger.test b/mysql-test/suite/rpl/t/rpl_trigger.test
index e442ed94d5adf..1a14d790b05c7 100644
--- a/mysql-test/suite/rpl/t/rpl_trigger.test
+++ b/mysql-test/suite/rpl/t/rpl_trigger.test
@@ -378,6 +378,7 @@ SHOW TABLES LIKE 't_';
--replace_column 6 #
SHOW TRIGGERS;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
# Restart slave.
diff --git a/mysql-test/suite/rpl/t/semisync_future-7591.test b/mysql-test/suite/rpl/t/semisync_future-7591.test
index ce01e41f8dcc2..17264d0a3d658 100644
--- a/mysql-test/suite/rpl/t/semisync_future-7591.test
+++ b/mysql-test/suite/rpl/t/semisync_future-7591.test
@@ -16,6 +16,7 @@ start slave;
--connection master
insert into t1 values (1);
+--source include/kill_binlog_dump_threads.inc
reset master;
--connection slave
diff --git a/mysql-test/suite/s3/replication_delayed.result b/mysql-test/suite/s3/replication_delayed.result
index 706fcfe6d7e7a..a5714d2ce5ea8 100644
--- a/mysql-test/suite/s3/replication_delayed.result
+++ b/mysql-test/suite/s3/replication_delayed.result
@@ -1,8 +1,13 @@
include/master-slave.inc
[connection master]
+connection slave;
+include/stop_slave.inc
+connection master;
set binlog_format=mixed;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
connection slave;
+include/start_slave.inc
set binlog_format=mixed;
RESET MASTER;
connection master;
diff --git a/mysql-test/suite/s3/replication_delayed.test b/mysql-test/suite/s3/replication_delayed.test
index 0ae93b850c437..431e11fde9cb9 100644
--- a/mysql-test/suite/s3/replication_delayed.test
+++ b/mysql-test/suite/s3/replication_delayed.test
@@ -5,9 +5,14 @@
--source include/have_sequence.inc
# First clear the binlog
+connection slave;
+--source include/stop_slave.inc
+connection master;
set binlog_format=mixed;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
connection slave;
+--source include/start_slave.inc
set binlog_format=mixed;
RESET MASTER;
connection master;
diff --git a/mysql-test/suite/s3/replication_mixed.result b/mysql-test/suite/s3/replication_mixed.result
index 80a32b28d8a13..37fafafb4b98a 100644
--- a/mysql-test/suite/s3/replication_mixed.result
+++ b/mysql-test/suite/s3/replication_mixed.result
@@ -1,8 +1,13 @@
include/master-slave.inc
[connection master]
+connection slave;
+include/stop_slave.inc
+connection master;
set binlog_format=mixed;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
connection slave;
+include/start_slave.inc
set binlog_format=mixed;
RESET MASTER;
connection master;
diff --git a/mysql-test/suite/s3/replication_mixed.test b/mysql-test/suite/s3/replication_mixed.test
index d10d586dab97a..5fbdc21e442a2 100644
--- a/mysql-test/suite/s3/replication_mixed.test
+++ b/mysql-test/suite/s3/replication_mixed.test
@@ -1,9 +1,14 @@
--source include/have_binlog_format_mixed.inc
--source include/master-slave.inc
+connection slave;
+--source include/stop_slave.inc
+connection master;
set binlog_format=mixed;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
connection slave;
+--source include/start_slave.inc
set binlog_format=mixed;
RESET MASTER;
connection master;
diff --git a/mysql-test/suite/s3/replication_stmt.result b/mysql-test/suite/s3/replication_stmt.result
index 4257823bbea46..4f27b7ce47424 100644
--- a/mysql-test/suite/s3/replication_stmt.result
+++ b/mysql-test/suite/s3/replication_stmt.result
@@ -1,8 +1,13 @@
include/master-slave.inc
[connection master]
+connection slave;
+include/stop_slave.inc
+connection master;
set binlog_format=statement;
+include/kill_binlog_dump_threads.inc
RESET MASTER;
connection slave;
+include/start_slave.inc
set binlog_format=statement;
RESET MASTER;
connection master;
diff --git a/mysql-test/suite/s3/replication_stmt.test b/mysql-test/suite/s3/replication_stmt.test
index aba5d155e6c1e..84e097171345a 100644
--- a/mysql-test/suite/s3/replication_stmt.test
+++ b/mysql-test/suite/s3/replication_stmt.test
@@ -1,9 +1,14 @@
--source include/have_binlog_format_statement.inc
--source include/master-slave.inc
+connection slave;
+--source include/stop_slave.inc
+connection master;
set binlog_format=statement;
+--source include/kill_binlog_dump_threads.inc
RESET MASTER;
connection slave;
+--source include/start_slave.inc
set binlog_format=statement;
RESET MASTER;
connection master;
diff --git a/mysql-test/suite/sys_vars/r/default_tmp_storage_engine_rocksdb.result b/mysql-test/suite/sys_vars/r/default_tmp_storage_engine_rocksdb.result
index 19fae14579d97..a3cda0f36acff 100644
--- a/mysql-test/suite/sys_vars/r/default_tmp_storage_engine_rocksdb.result
+++ b/mysql-test/suite/sys_vars/r/default_tmp_storage_engine_rocksdb.result
@@ -55,6 +55,7 @@ DROP TABLE t1;
#
SHOW VARIABLES like '%storage_engine';
Variable_name Value
+binlog_storage_engine
default_storage_engine ROCKSDB
default_tmp_storage_engine Aria
enforce_storage_engine
diff --git a/mysql-test/suite/sys_vars/r/sysvars_innodb.result b/mysql-test/suite/sys_vars/r/sysvars_innodb.result
index 23978fc881825..6966934e08017 100644
--- a/mysql-test/suite/sys_vars/r/sysvars_innodb.result
+++ b/mysql-test/suite/sys_vars/r/sysvars_innodb.result
@@ -92,6 +92,18 @@ NUMERIC_BLOCK_SIZE 0
ENUM_VALUE_LIST NULL
READ_ONLY YES
COMMAND_LINE_ARGUMENT REQUIRED
+VARIABLE_NAME INNODB_BINLOG_STATE_INTERVAL
+SESSION_VALUE NULL
+DEFAULT_VALUE 2097152
+VARIABLE_SCOPE GLOBAL
+VARIABLE_TYPE BIGINT UNSIGNED
+VARIABLE_COMMENT Interval (in bytes) at which to write the GTID binlog state to binlog files to speed up GTID lookups. Must be a power-of-two multiple of the binlog page size (16384 bytes)
+NUMERIC_MIN_VALUE 32768
+NUMERIC_MAX_VALUE 18446744073709551615
+NUMERIC_BLOCK_SIZE 0
+ENUM_VALUE_LIST NULL
+READ_ONLY YES
+COMMAND_LINE_ARGUMENT REQUIRED
VARIABLE_NAME INNODB_BUFFER_POOL_CHUNK_SIZE
SESSION_VALUE NULL
DEFAULT_VALUE 0
diff --git a/mysql-test/suite/sys_vars/r/sysvars_server_embedded.result b/mysql-test/suite/sys_vars/r/sysvars_server_embedded.result
index cf27b7294b40e..91dcb4f497ea5 100644
--- a/mysql-test/suite/sys_vars/r/sysvars_server_embedded.result
+++ b/mysql-test/suite/sys_vars/r/sysvars_server_embedded.result
@@ -402,6 +402,16 @@ NUMERIC_BLOCK_SIZE 1
ENUM_VALUE_LIST NULL
READ_ONLY NO
COMMAND_LINE_ARGUMENT REQUIRED
+VARIABLE_NAME BINLOG_DIRECTORY
+VARIABLE_SCOPE GLOBAL
+VARIABLE_TYPE VARCHAR
+VARIABLE_COMMENT Directory path (absolute or relative to datadir) where binlog files are stored. If this is used, must not specify a directory path for --log-bin
+NUMERIC_MIN_VALUE NULL
+NUMERIC_MAX_VALUE NULL
+NUMERIC_BLOCK_SIZE NULL
+ENUM_VALUE_LIST NULL
+READ_ONLY YES
+COMMAND_LINE_ARGUMENT REQUIRED
VARIABLE_NAME BINLOG_DIRECT_NON_TRANSACTIONAL_UPDATES
VARIABLE_SCOPE SESSION
VARIABLE_TYPE BOOLEAN
diff --git a/mysql-test/suite/sys_vars/r/sysvars_server_notembedded.result b/mysql-test/suite/sys_vars/r/sysvars_server_notembedded.result
index 3a3ea304d71a7..aa3d64ec56a48 100644
--- a/mysql-test/suite/sys_vars/r/sysvars_server_notembedded.result
+++ b/mysql-test/suite/sys_vars/r/sysvars_server_notembedded.result
@@ -412,6 +412,16 @@ NUMERIC_BLOCK_SIZE 1
ENUM_VALUE_LIST NULL
READ_ONLY NO
COMMAND_LINE_ARGUMENT REQUIRED
+VARIABLE_NAME BINLOG_DIRECTORY
+VARIABLE_SCOPE GLOBAL
+VARIABLE_TYPE VARCHAR
+VARIABLE_COMMENT Directory path (absolute or relative to datadir) where binlog files are stored. If this is used, must not specify a directory path for --log-bin
+NUMERIC_MIN_VALUE NULL
+NUMERIC_MAX_VALUE NULL
+NUMERIC_BLOCK_SIZE NULL
+ENUM_VALUE_LIST NULL
+READ_ONLY YES
+COMMAND_LINE_ARGUMENT REQUIRED
VARIABLE_NAME BINLOG_DIRECT_NON_TRANSACTIONAL_UPDATES
VARIABLE_SCOPE SESSION
VARIABLE_TYPE BOOLEAN
@@ -572,6 +582,16 @@ NUMERIC_BLOCK_SIZE 4096
ENUM_VALUE_LIST NULL
READ_ONLY NO
COMMAND_LINE_ARGUMENT REQUIRED
+VARIABLE_NAME BINLOG_STORAGE_ENGINE
+VARIABLE_SCOPE GLOBAL
+VARIABLE_TYPE VARCHAR
+VARIABLE_COMMENT Use a more efficient binlog implementation integrated with the storage engine. Only available for supporting engines
+NUMERIC_MIN_VALUE NULL
+NUMERIC_MAX_VALUE NULL
+NUMERIC_BLOCK_SIZE NULL
+ENUM_VALUE_LIST NULL
+READ_ONLY YES
+COMMAND_LINE_ARGUMENT REQUIRED
VARIABLE_NAME BLOCK_ENCRYPTION_MODE
VARIABLE_SCOPE SESSION
VARIABLE_TYPE ENUM
diff --git a/mysys/CMakeLists.txt b/mysys/CMakeLists.txt
index 8d5467263828c..a3f98d30cd497 100644
--- a/mysys/CMakeLists.txt
+++ b/mysys/CMakeLists.txt
@@ -47,7 +47,7 @@ SET(MYSYS_SOURCES array.c charset-def.c charset.c my_default.c
my_rdtsc.c psi_noop.c
my_atomic_writes.c my_cpu.c my_likely.c my_largepage.c
file_logger.c my_dlerror.c crc32/crc32c.cc
- my_timezone.cc
+ my_timezone.cc my_compr_int.cc
my_virtual_mem.c)
IF (WIN32)
diff --git a/mysys/mf_iocache.c b/mysys/mf_iocache.c
index b22bd4d06ace2..417bda76f9ca1 100644
--- a/mysys/mf_iocache.c
+++ b/mysys/mf_iocache.c
@@ -1710,6 +1710,13 @@ int my_b_flush_io_cache(IO_CACHE *info, int need_append_buffer_lock)
if ((length=(size_t) (info->write_pos - info->write_buffer)))
{
+ /*
+ The write_function() updates info->pos_in_file. So compute the new end
+ position here before calling it, but update the value only after we
+ check for error return.
+ */
+ uchar *new_write_end= (info->write_buffer + info->buffer_length -
+ ((info->pos_in_file + length) & (IO_SIZE - 1)));
if (append_cache)
{
if (mysql_file_write(info->file, info->write_buffer, length,
@@ -1730,8 +1737,7 @@ int my_b_flush_io_cache(IO_CACHE *info, int need_append_buffer_lock)
set_if_bigger(info->end_of_file, info->pos_in_file);
}
- info->write_end= (info->write_buffer + info->buffer_length -
- ((info->pos_in_file + length) & (IO_SIZE - 1)));
+ info->write_end= new_write_end;
info->write_pos= info->write_buffer;
++info->disk_writes;
UNLOCK_APPEND_BUFFER;
diff --git a/mysys/my_compr_int.cc b/mysys/my_compr_int.cc
new file mode 100644
index 0000000000000..9f4bf5854216b
--- /dev/null
+++ b/mysys/my_compr_int.cc
@@ -0,0 +1,257 @@
+/*****************************************************************************
+
+Copyright (c) 2024 Kristian Nielsen.
+
+This program is free software; you can redistribute it and/or modify it under
+the terms of the GNU General Public License as published by the Free Software
+Foundation; version 2 of the License.
+
+This program is distributed in the hope that it will be useful, but WITHOUT
+ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or FITNESS
+FOR A PARTICULAR PURPOSE. See the GNU General Public License for more details.
+
+You should have received a copy of the GNU General Public License along with
+this program; if not, write to the Free Software Foundation, Inc.,
+51 Franklin Street, Fifth Floor, Boston, MA 02110-1335 USA
+
+*****************************************************************************/
+
+/*
+ Reading and writing of compressed integers.
+
+ Created 2024-10-01 Kristian Nielsen
+*/
+
+#include "mysys_priv.h"
+#include "my_compr_int.h"
+
+/* Read and write compressed (up to) 64-bit integers. */
+
+/*
+ Write compressed unsigned integer, efficient version without assuming
+ unaligned writes.
+*/
+unsigned char *compr_int_write(unsigned char *p, uint64_t v) {
+ // Compute bytes needed to store the value v plus 3 bits encoding length.
+ uint32_t needed_bits_minus_1= 66 - my_nlz(v|1);
+ uint32_t needed_bytes= (needed_bits_minus_1 >> 3) + 1;
+
+ // Compute the encoding of the length.
+ // We need 1-9 bytes. Use 9 bytes instead of 8, so we can encode the
+ // length in 3 bits for (1, 2, ..., 7, or 9 bytes).
+ uint32_t bytes= needed_bytes | (needed_bytes >> 3);
+ uint32_t len= needed_bytes - 1;
+ // Encode 1-7 as 0-6, and encode 8,9 both as 8.
+ len-= (len >> 3);
+
+ // Compute the first 64-bit word to write.
+ uintptr_t offset= (uintptr_t)p & (uintptr_t)7;
+ uintptr_t offset_bits= offset << 3;
+ uint64_t v1= (len | (v << 3)) << offset_bits;
+ uint64_t *p1= (uint64_t *)(p - offset);
+ uint64_t mask1= ~(uint64_t)0 << offset_bits;
+
+ // Compute the second word to write (if any).
+ uint64_t v2= v >> ((64 - 3) - offset_bits);
+ uint64_t *p2= p1 + 1;
+
+ // Write the value into next one or two 64-bit words, as needed.
+ // Two words are needed if (offset + bytes) cross into the next word.
+#ifdef WORDS_BIGENDIAN
+ /*
+ Here it might be possible to use a slightly more efficient endian
+ conversion on big-endian, since we know the pointer is 8-byte aligned.
+ */
+ int8store((unsigned char *)p1,
+ (uint8korr((unsigned char *)p1) & ~mask1) | v1);
+#else
+ *p1= (*p1 & ~mask1) | v1;
+#endif
+ if (offset + bytes >= 8) {
+#ifdef WORDS_BIGENDIAN
+ int8store((unsigned char *)p2, v2);
+#else
+ *p2= v2;
+#endif
+ }
+ return p + bytes;
+}
+
+
+/*
+ Read compressed integer, efficient version without assuming unaligned reads.
+ Returns a pair of the value read and the incremented pointer.
+*/
+std::pair
+compr_int_read(const unsigned char *p)
+{
+ uintptr_t offset= (uintptr_t)p & (uintptr_t)7;
+ uintptr_t offset_bits= offset << 3;
+ const uint64_t *p_align= (const uint64_t *)((uintptr_t)p & ~(uintptr_t)7);
+#ifdef WORDS_BIGENDIAN
+ /*
+ Here it might be possible to use a slightly more efficient endian
+ conversion on big-endian, since we know the pointer is 8-byte aligned.
+ */
+ uint64_t v1= uint8korr((unsigned char *)p_align);
+#else
+ uint64_t v1= p_align[0];
+#endif
+ uint32_t len= (v1 >> offset_bits) & 7;
+ uint32_t bytes= len + 1;
+ bytes+= (bytes >> 3);
+ uint64_t v;
+ if (offset + bytes > 8) {
+ uint64_t mask2= (~(uint64_t)0) >> ((16 - (offset + bytes)) << 3);
+#ifdef WORDS_BIGENDIAN
+ v= (v1 >> (3 + offset_bits)) |
+ ((uint8korr((unsigned char *)(p_align + 1)) & mask2) <<
+ (61 - offset_bits));
+#else
+ v= (v1 >> (3 + offset_bits)) | ((p_align[1] & mask2) << (61 - offset_bits));
+#endif
+ } else {
+ uint64_t mask1= (~(uint64_t)0) >> ((7 - (offset + len)) << 3);
+ v= (v1 & mask1) >> (3 + offset_bits);
+ }
+ return std::pair(v, p + bytes);
+}
+
+
+#ifdef TEST_MAIN
+#include
+#include
+#include
+
+
+// Smaller version that assumes unaligned writes of 8-bit values is ok, and
+// that there are up to 8 scratch bytes available after the value written.
+unsigned char *compr_int_write_le_unaligned_buffer(unsigned char *p, uint64_t v) {
+ // Compute bytes needed to store the value v plus 3 bits encoding length.
+ uint32_t needed_bits_minus_1= 66 - my_nlz(v|1);
+ uint32_t needed_bytes= (needed_bits_minus_1 >> 3) + 1;
+
+ // Compute the encoding of the length.
+ // We need 1-9 bytes. Use 9 bytes instead of 8, so we can encode the
+ // length in 3 bits for (1, 2, ..., 7, or 9 bytes).
+ uint32_t bytes= needed_bytes | (needed_bytes >> 3);
+ uint32_t len= needed_bytes - 1;
+ // Encode 1-7 as 0-6, and encode 8,9 both as 8.
+ len-= (len >> 3);
+
+ // Write the (up to) 9 bytes, prefering redundant write to conditional jump.
+ *(uint64_t *)p= len | (v << 3);
+ *(p+8)= v >> 63;
+ return p + bytes;
+}
+
+// Generic version without assumptions.
+unsigned char *compr_int_write_generic(unsigned char *p, uint64_t v) {
+ // Compute bytes needed to store the value v plus 3 bits encoding length.
+ uint32_t needed_bits_minus_1= 66 - my_nlz(v|1);
+ uint32_t needed_bytes= (needed_bits_minus_1 >> 3) + 1;
+
+ // Compute the encoding of the length.
+ // We need 1-9 bytes. Use 9 bytes instead of 8, so we can encode the
+ // length in 3 bits for (1, 2, ..., 7, or 9 bytes).
+ uint32_t bytes= needed_bytes | (needed_bytes >> 3);
+ uint32_t len= needed_bytes - 1;
+ // Encode 1-7 as 0-6, and encode 8,9 both as 8.
+ len-= (len >> 3);
+
+ // Write the necessary bytes out.
+ *p++= len | (v << 3);
+ v >>= 5;
+ while (--bytes > 0) {
+ *p++= v;
+ v>>= 8;
+ }
+ return p;
+
+}
+
+
+// Generic read compressed integers.
+std::pair
+compr_int_read_generic(const unsigned char *p)
+{
+ uint64_t v= *p++;
+ uint32_t bytes= v & 7;
+ v>>= 3;
+ uint32_t shift= 5;
+ bytes+= ((bytes + 1) >> 3); // A 7 means read 8 bytes more (9 total)
+ while (bytes-- > 0) {
+ v|= ((uint64_t)(*p++)) << shift;
+ shift+= 8;
+ }
+ return std::pair(v, p);
+}
+
+
+// Read compressed integers assuming little-endian, efficient unaligned
+// 64-bit reads, and up to 7 bytes of scratch space after.
+std::pair
+compr_int_read_le_unaligned_buf(const unsigned char *p)
+{
+ uint64_t v= *(const uint64_t *)p;
+ uint32_t len= v & 7;
+ uint64_t mask= (~(uint64_t)0) >> ((7 - len) << 3);
+ v= (v & mask) >> 3;
+ // Need for extra read is assumed rare, well-predicted conditional jump
+ // likely faster.
+ uint32_t bytes= len + 1;
+ if (__builtin_expect((len == 7), 0)) {
+ v|= ((uint64_t)p[8]) << 61; // Add last 3 bits
+ bytes+= 1; // 7 means read 9 bytes
+ }
+ return std::pair(v, p + bytes);
+}
+
+
+int
+main(int argc, char *argv[])
+{
+ int N= (argc > 1 ? atoi(argv[1]) : 1000);
+ uint64_t *src= new uint64_t[N];
+ unsigned char *buf1= new unsigned char[N*9];
+ unsigned char *buf2= new unsigned char[N*9];
+ int i;
+
+ // Generate test data.
+ for (i= 0; i < N; ++i)
+ src[i]= ((uint64_t)1 << (rand() % 64)) + (uint64_t)(rand());
+ // Write test data.
+ unsigned char *p1= buf1;
+ unsigned char *p2= buf2;
+ for (i= 0; i < N; ++i) {
+ p1= compr_int_write(p1, src[i]);
+ p2= compr_int_write_generic(p2, src[i]);
+ }
+ if ((p1 - buf1) != (p2 - buf2)) {
+ fprintf(stderr, "Write error! Mismatch lengths of optimised and generic.\n");
+ } else {
+ if (memcmp(buf1, buf2, p1 - buf1))
+ fprintf(stderr, "Write error! Mismatch data of optimised and generic.\n");
+ }
+ // Verify written data.
+ std::pairq1, q2;
+ const unsigned char *c1= buf1;
+ const unsigned char *c2= buf2;
+ for (i= 0; i < N; ++i) {
+ q1= compr_int_read(c1);
+ q2= compr_int_read_generic(c2);
+ uint64_t v1= q1.first;
+ c1= q1.second;
+ uint64_t v2= q2.first;
+ c2= q2.second;
+ if (v1 != v2 || v1 != src[i]) {
+ fprintf(stderr, "Read error! mismatch values @ i=%d 0x%llx 0x%llx 0x%llx.\n",
+ i, (unsigned long long)v1, (unsigned long long)v2,
+ (unsigned long long)(src[i]));
+ break;
+ }
+ }
+ return 0;
+}
+
+#endif /* TEST_MAIN */
diff --git a/sql/ddl_log.cc b/sql/ddl_log.cc
index 95e426bb0d61e..4830aced59b07 100644
--- a/sql/ddl_log.cc
+++ b/sql/ddl_log.cc
@@ -997,6 +997,7 @@ static void ddl_log_to_binary_log(THD *thd, String *query)
query->length(query->length()-1); // Removed end ','
query->append(&end_comment);
mysql_mutex_unlock(&LOCK_gdl);
+ thd->transaction->stmt.mark_trans_did_ddl();
(void) thd->binlog_query(THD::STMT_QUERY_TYPE,
query->ptr(), query->length(),
TRUE, FALSE, FALSE, 0);
@@ -1691,6 +1692,7 @@ static int ddl_log_execute_action(THD *thd, MEM_ROOT *mem_root,
{
mysql_mutex_unlock(&LOCK_gdl);
thd->db= ddl_log_entry->db;
+ thd->transaction->stmt.mark_trans_did_ddl();
(void) thd->binlog_query(THD::STMT_QUERY_TYPE,
recovery_state.drop_table.ptr(),
recovery_state.drop_table.length(), TRUE, FALSE,
@@ -1733,6 +1735,7 @@ static int ddl_log_execute_action(THD *thd, MEM_ROOT *mem_root,
if (mysql_bin_log.is_open())
{
mysql_mutex_unlock(&LOCK_gdl);
+ thd->transaction->stmt.mark_trans_did_ddl();
(void) thd->binlog_query(THD::STMT_QUERY_TYPE,
query->ptr(), query->length(),
TRUE, FALSE, FALSE, 0);
@@ -1787,6 +1790,7 @@ static int ddl_log_execute_action(THD *thd, MEM_ROOT *mem_root,
if (mysql_bin_log.is_open())
{
mysql_mutex_unlock(&LOCK_gdl);
+ thd->transaction->stmt.mark_trans_did_ddl();
(void) thd->binlog_query(THD::STMT_QUERY_TYPE,
query->ptr(), query->length(),
TRUE, FALSE, FALSE, 0);
@@ -2215,6 +2219,7 @@ static int ddl_log_execute_action(THD *thd, MEM_ROOT *mem_root,
save_db= thd->db;
lex_string_set3(&thd->db, recovery_state.db.ptr(),
recovery_state.db.length());
+ thd->transaction->stmt.mark_trans_did_ddl();
(void) thd->binlog_query(THD::STMT_QUERY_TYPE,
recovery_state.query.ptr(),
recovery_state.query.length(),
diff --git a/sql/handler.cc b/sql/handler.cc
index d959ad979fad1..78491fd0aa02c 100644
--- a/sql/handler.cc
+++ b/sql/handler.cc
@@ -1527,6 +1527,12 @@ int ha_prepare(THD *thd)
if (ha_info)
{
+ if (unlikely(tc_log->log_xa_prepare(thd, all)))
+ {
+ ha_rollback_trans(thd, all);
+ error= 1;
+ goto binlog_error;
+ }
for (; ha_info; ha_info= ha_info->next())
{
handlerton *ht= ha_info->ht();
@@ -1549,6 +1555,7 @@ int ha_prepare(THD *thd)
}
}
+binlog_error:
DEBUG_SYNC(thd, "at_unlog_xa_prepare");
if (tc_log->unlog_xa_prepare(thd, all))
@@ -1993,6 +2000,9 @@ int ha_commit_trans(THD *thd, bool all)
*/
if (! hi->is_trx_read_write())
continue;
+ /* We do not need to 2pc the binlog with the engine that implements it. */
+ if (ht == opt_binlog_engine_hton)
+ continue;
/*
Sic: we know that prepare() is not NULL since otherwise
trans->no_2pc would have been set.
@@ -2052,14 +2062,14 @@ int ha_commit_trans(THD *thd, bool all)
if (wsrep_must_abort(thd))
{
mysql_mutex_unlock(&thd->LOCK_thd_data);
- (void)tc_log->unlog(cookie, xid);
+ (void)tc_log->unlog(thd, cookie, xid);
goto wsrep_err;
}
mysql_mutex_unlock(&thd->LOCK_thd_data);
}
#endif /* WITH_WSREP */
DBUG_EXECUTE_IF("crash_commit_before_unlog", DBUG_SUICIDE(););
- if (tc_log->unlog(cookie, xid))
+ if (tc_log->unlog(thd, cookie, xid))
error= 2; /* Error during commit */
done:
@@ -2204,29 +2214,6 @@ inline Ha_trx_info* get_binlog_hton(Ha_trx_info *ha_info)
return ha_info;
}
-static int run_binlog_first(THD *thd, bool all, THD_TRANS *trans,
- bool is_real_trans, bool is_commit)
-{
- int rc= 0;
- Ha_trx_info *ha_info= trans->ha_list;
-
- if ((ha_info= get_binlog_hton(ha_info)))
- {
- int err;
- if ((err= is_commit ? binlog_commit(thd, all,
- is_ro_1pc_trans(thd, ha_info, all,
- is_real_trans))
- : binlog_rollback(ha_info->ht(), thd, all)))
- {
- my_error(is_commit ? ER_ERROR_DURING_COMMIT : ER_ERROR_DURING_ROLLBACK,
- MYF(0), err);
- rc= 1;
- }
- }
-
- return rc;
-}
-
static int
commit_one_phase_2(THD *thd, bool all, THD_TRANS *trans, bool is_real_trans)
{
@@ -2240,19 +2227,19 @@ commit_one_phase_2(THD *thd, bool all, THD_TRANS *trans, bool is_real_trans)
if (ha_info)
{
int err= 0;
- /*
- Binlog hton must be called first regardless of its position
- in trans->ha_list at least to prevent from commiting any engine
- branches when afterward a duplicate GTID error out of binlog_commit()
- is generated.
- */
- for (int binlog_err= error=
- run_binlog_first(thd, all, trans, is_real_trans, true);
- ha_info; ha_info= ha_info_next)
+ Ha_trx_info *binlog_ha_info= get_binlog_hton(ha_info);
+ if (binlog_ha_info &&
+ (err= binlog_commit(thd, all,
+ is_ro_1pc_trans(thd, ha_info, all,
+ is_real_trans))))
{
- if (binlog_err)
- goto err;
+ my_error(ER_ERROR_DURING_COMMIT, MYF(0), err);
+ error= 1;
+ goto err;
+ }
+ for (; ha_info; ha_info= ha_info_next)
+ {
handlerton *ht= ha_info->ht();
if ((err= ht->commit(ht, thd, all)))
{
@@ -2266,6 +2253,9 @@ commit_one_phase_2(THD *thd, bool all, THD_TRANS *trans, bool is_real_trans)
ha_info_next= ha_info->next();
ha_info->reset(); /* keep it conveniently zero-filled */
}
+ DEBUG_SYNC(thd, "commit_handlerton_after");
+ if (binlog_ha_info && is_real_trans)
+ binlog_post_commit(thd, all);
trans->ha_list= 0;
trans->no_2pc=0;
if (all)
@@ -2374,6 +2364,8 @@ int ha_rollback_trans(THD *thd, bool all)
if (ha_info)
{
+ int err;
+
/* Close all cursors that can not survive ROLLBACK */
if (is_real_trans) /* not a statement commit */
thd->stmt_map.close_transient_cursors();
@@ -2384,12 +2376,17 @@ int ha_rollback_trans(THD *thd, bool all)
rollbacker and any transaction that depends on it. This guarantees
the execution time dependency identifies binlog ordering.
*/
- for (error= run_binlog_first(thd, all, trans, is_real_trans, false);
- ha_info; ha_info= ha_info_next)
+ Ha_trx_info *binlog_ha_info= get_binlog_hton(ha_info);
+ if (binlog_ha_info && (err= binlog_rollback(binlog_hton, thd, all)))
+ {
+ my_error(ER_ERROR_DURING_ROLLBACK, MYF(0), err);
+ error= 1;
+ }
+
+ for (; ha_info; ha_info= ha_info_next)
{
- int err;
handlerton *ht= ha_info->ht();
- if (ht != binlog_hton && (err= ht->rollback(ht, thd, all)))
+ if (ha_info != binlog_ha_info && (err= ht->rollback(ht, thd, all)))
{
// cannot happen
my_error(ER_ERROR_DURING_ROLLBACK, MYF(0), err);
@@ -2405,12 +2402,16 @@ int ha_rollback_trans(THD *thd, bool all)
}
#endif /* WITH_WSREP */
}
+ DEBUG_SYNC(thd, "rollback_handlerton_after");
status_var_increment(thd->status_var.ha_rollback_count);
ha_info_next= ha_info->next();
ha_info->reset(); /* keep it conveniently zero-filled */
}
trans->ha_list= 0;
trans->no_2pc=0;
+
+ if (binlog_ha_info)
+ binlog_post_rollback(thd, all);
}
#ifdef WITH_WSREP
@@ -2481,7 +2482,7 @@ struct xahton_st {
int result;
};
-static my_bool xacommit_handlerton(THD *unused1, plugin_ref plugin,
+static my_bool xacommit_handlerton(THD *thd, plugin_ref plugin,
void *arg)
{
handlerton *hton= plugin_hton(plugin);
@@ -2524,6 +2525,15 @@ int ha_commit_or_rollback_by_xid(XID *xid, bool commit)
plugin_foreach(NULL, commit ? xacommit_handlerton : xarollback_handlerton,
MYSQL_STORAGE_ENGINE_PLUGIN, &xaop);
+ if (commit)
+ DEBUG_SYNC(current_thd, "xacommit_handlerton_after");
+ else
+ DEBUG_SYNC(current_thd, "xarollback_handlerton_after");
+
+ if (commit)
+ binlog_post_commit_by_xid(binlog_hton, xid);
+ else
+ binlog_post_rollback_by_xid(binlog_hton, xid);
return xaop.result;
}
@@ -2641,6 +2651,43 @@ struct xarecover_st
bool error;
};
+
+/**
+ Recovery for XID (internal 2pc and user XA) using engine-implemented binlog.
+
+ The binlog provides the state of each XID - prepared, committed, rolled
+ back. For prepared XA, it also provides the count of the number of engines
+ participating in that transaction.
+
+ Each XID found prepared in an engine will be committed, rolled back, or left
+ in prepared state according to the state of the binlog. For an XID in the
+ prepared state, if the number of engines found having that XID is too
+ small, it means the server crashed in the middle of preparing a multi-
+ engine transaction, and that XID will be rolled back both in engines and
+ in the binlog.
+*/
+struct xarecover_engine_binlog
+{
+ static constexpr uint32_t MAX_HTONS= 32;
+
+ /* Buffer for engines to return their prepared XID into. */
+ XID *list;
+ /* Hash (of handler_binlog_xid_info) of binlog state of XIDs. */
+ HASH *xid_hash;
+ /*
+ Engine handlertons involved in XID recovery, used for bits in
+ handler_binlog_xid_info::engine_map.
+ */
+ handlerton *htons[MAX_HTONS];
+ /* Used entries in htons. */
+ uint32_t num_htons;
+ /* Size of the XID *list. */
+ int len;
+ /* Set in case of any error during the processing. */
+ bool error;
+};
+
+
/**
Inserts a new hash member.
@@ -2692,6 +2739,170 @@ static bool xid_member_replace(HASH *hash_arg, my_xid xid_arg,
return member == NULL;
}
+
+static bool
+record_hton_for_xid(xarecover_engine_binlog *info, handler_binlog_xid_info *rec,
+ handlerton *hton)
+{
+ uint32_t idx;
+ for (idx= 0; idx < info->num_htons; ++idx)
+ {
+ if (info->htons[idx] == hton)
+ {
+ rec->engine_map|= 1<num_htons >= xarecover_engine_binlog::MAX_HTONS)
+ {
+ sql_print_error("Too many transactional engines during binlog recovery "
+ "of prepared transactions (max is %u)",
+ (uint)xarecover_engine_binlog::MAX_HTONS);
+ return true;
+ }
+ rec->engine_map|= 1<num_htons;
+ info->htons[info->num_htons++]= hton;
+ return false;
+}
+
+
+static my_bool xarecover_engine_binlog(THD *unused, plugin_ref plugin,
+ void *arg)
+{
+ handlerton *hton= plugin_hton(plugin);
+ struct xarecover_engine_binlog *info=
+ (struct xarecover_engine_binlog *) arg;
+ int got;
+
+ if (hton->recover)
+ {
+ while ((got= hton->recover(hton, info->list, info->len)) > 0 )
+ {
+ sql_print_information("Found %d prepared transaction(s) in %s",
+ got, hton_name(hton)->str);
+
+ for (int i=0; i < got; i ++)
+ {
+ XID *xid= &info->list[i];
+ const uchar *key_ptr= xid->key();
+ size_t key_len= xid->key_length();
+ handler_binlog_xid_info *rec= (handler_binlog_xid_info *)
+ my_hash_search(info->xid_hash, key_ptr, key_len);
+
+ /* If the binlog says to roll back, or says nothing, then roll back. */
+ if (!rec || rec->xid_state == handler_binlog_xid_info::BINLOG_ROLLBACK)
+ {
+ if (hton->rollback_by_xid(hton, info->list+i))
+ info->error= true;
+ continue;
+ }
+
+ /* If the binlog says to commit, or says nothing, then commit. */
+ if (rec->xid_state == handler_binlog_xid_info::BINLOG_COMMIT)
+ {
+ if (hton->commit_by_xid(hton, xid))
+ info->error= true;
+ continue;
+ }
+ DBUG_ASSERT(rec->xid_state == handler_binlog_xid_info::BINLOG_PREPARE);
+
+ /*
+ If the binlog has the transaction in the prepared state, then we
+ must check if all involved engines have it prepared as well. We might
+ have crashed before all engines had time to (durably) prepare, in
+ which case we will roll back the ones that did.
+ So we record in the info->xid_hash that we found the XID in this
+ engine, and at the end we then check whether to commit or roll back.
+ */
+ DBUG_ASSERT(rec->engine_count > 0);
+ if (likely(rec->engine_count > 0))
+ --rec->engine_count;
+ if (record_hton_for_xid(info, rec, hton))
+ info->error= true;
+ }
+ if (got < info->len)
+ break;
+ }
+ }
+ return FALSE;
+}
+
+
+int
+ha_recover_engine_binlog(HASH *xid_hash)
+{
+ DBUG_ENTER("ha_recover_engine_binlog");
+ DBUG_ASSERT(opt_binlog_engine_hton);
+ struct xarecover_engine_binlog info;
+ info.xid_hash= xid_hash;
+ info.num_htons= 0;
+ info.error= false;
+ info.list= nullptr;
+
+ for (info.len= MAX_XID_LIST_SIZE; info.len >= MIN_XID_LIST_SIZE; info.len/=2)
+ {
+ info.list=(XID *)my_malloc(key_memory_XID, info.len*sizeof(XID), MYF(0));
+ if (likely(info.list))
+ break;
+ }
+ if (!info.list)
+ {
+ sql_print_error(ER(ER_OUTOFMEMORY),
+ static_cast(info.len*sizeof(XID)));
+ DBUG_RETURN(1);
+ }
+
+ plugin_foreach(NULL, xarecover_engine_binlog,
+ MYSQL_STORAGE_ENGINE_PLUGIN, &info);
+
+ my_free(info.list);
+
+ if (info.error)
+ DBUG_RETURN(1);
+
+ /*
+ Now handle any XID found in the prepared state in binlog. They will be
+ left prepared if all engines that participated in the transaction managed
+ to prepare them durably before the server restart; otherwise they will be
+ rolled back in binlog and engines (if any).
+ */
+ for (uint32 i= 0; i < xid_hash->records; ++i)
+ {
+ handler_binlog_xid_info *rec= (handler_binlog_xid_info *)
+ my_hash_element(xid_hash, i);
+ if (rec->xid_state != handler_binlog_xid_info::BINLOG_PREPARE)
+ continue;
+ if (rec->engine_count == 0)
+ {
+ /* Recover the XID as a prepared XA transaction. */
+ xid_cache_insert(&rec->xid);
+ }
+ else
+ {
+ /* Not all participating engines prepared, so roll back. */
+ void *engine_data= nullptr;
+ mysql_mutex_lock(&LOCK_commit_ordered);
+ (*opt_binlog_engine_hton->binlog_xa_rollback_ordered)
+ (current_thd, &rec->xid, &engine_data);
+ mysql_mutex_unlock(&LOCK_commit_ordered);
+ (*opt_binlog_engine_hton->binlog_xa_rollback)
+ (current_thd, &rec->xid, &engine_data);
+ for (uint32_t j= 0; j < info.num_htons; ++j)
+ {
+ if (rec->engine_map & (1<rollback_by_xid)(hton, &rec->xid);
+ }
+ }
+ (*opt_binlog_engine_hton->binlog_unlog)(&rec->xid, &engine_data);
+ (*opt_binlog_engine_hton->binlog_oob_free)(engine_data);
+ }
+ }
+
+ DBUG_RETURN(0);
+}
+
+
/*
A "transport" type for recovery completion with ha_recover_complete()
*/
@@ -2942,6 +3153,7 @@ static my_bool xarecover_handlerton(THD *unused, plugin_ref plugin,
return FALSE;
}
+
int ha_recover(HASH *commit_list, MEM_ROOT *arg_mem_root)
{
struct xarecover_st info;
@@ -2953,6 +3165,22 @@ int ha_recover(HASH *commit_list, MEM_ROOT *arg_mem_root)
info.mem_root= arg_mem_root;
info.error= false;
+ if (opt_binlog_engine_hton)
+ {
+ if (tc_heuristic_recover)
+ {
+ sql_print_error("The --tc-heuristic-recover option is not needed with, "
+ "and cannot be used with --binlog-storage-engine");
+ DBUG_RETURN(1);
+ }
+ /*
+ With engine-implemented binlog, recovery is handled during binlog
+ open, calling into ha_recover_engine_binlog().
+ */
+ DBUG_ASSERT(!arg_mem_root);
+ DBUG_RETURN(0);
+ }
+
/* commit_list and tc_heuristic_recover cannot be set both */
DBUG_ASSERT(info.commit_list==0 || tc_heuristic_recover==0);
/* if either is set, total_ha_2pc must be set too */
@@ -2965,11 +3193,12 @@ int ha_recover(HASH *commit_list, MEM_ROOT *arg_mem_root)
if (info.commit_list)
sql_print_information("Starting table crash recovery...");
- for (info.len= MAX_XID_LIST_SIZE ;
- info.list==0 && info.len > MIN_XID_LIST_SIZE; info.len/=2)
+ for (info.len= MAX_XID_LIST_SIZE; info.len >= MIN_XID_LIST_SIZE; info.len/=2)
{
DBUG_EXECUTE_IF("min_xa_len", info.len = 16;);
info.list=(XID *)my_malloc(key_memory_XID, info.len*sizeof(XID), MYF(0));
+ if (likely(info.list))
+ break;
}
if (!info.list)
{
@@ -8425,8 +8654,6 @@ int ha_abort_transaction(THD *bf_thd, THD *victim_thd, my_bool signal)
if (!WSREP(bf_thd) &&
!(bf_thd->variables.wsrep_OSU_method == WSREP_OSU_RSU &&
wsrep_thd_is_toi(bf_thd))) {
- mysql_mutex_unlock(&victim_thd->LOCK_thd_data);
- mysql_mutex_unlock(&victim_thd->LOCK_thd_kill);
DBUG_RETURN(0);
}
@@ -8438,8 +8665,6 @@ int ha_abort_transaction(THD *bf_thd, THD *victim_thd, my_bool signal)
else
{
WSREP_WARN("Cannot abort InnoDB transaction");
- mysql_mutex_unlock(&victim_thd->LOCK_thd_data);
- mysql_mutex_unlock(&victim_thd->LOCK_thd_kill);
}
DBUG_RETURN(0);
diff --git a/sql/handler.h b/sql/handler.h
index db27319513ea5..3cc52c8632a7d 100644
--- a/sql/handler.h
+++ b/sql/handler.h
@@ -38,6 +38,7 @@
#include "vers_string.h"
#include "ha_handler_stats.h"
#include "optimizer_costs.h"
+#include "handler_binlog_reader.h"
#include "sql_analyze_stmt.h" // for Exec_time_tracker
@@ -59,6 +60,12 @@ class Field_varstring;
class Field_blob;
class Column_definition;
class select_result;
+class handler_binlog_reader;
+struct rpl_gtid;
+struct slave_connection_state;
+struct rpl_binlog_state_base;
+struct handler_binlog_event_group_info;
+struct handler_binlog_purge_info;
// the following is for checking tables
@@ -910,7 +917,7 @@ struct xid_t {
{ return !xid->is_null() && eq(xid->gtrid_length, xid->bqual_length, xid->data); }
bool eq(long g, long b, const char *d) const
{ return !is_null() && g == gtrid_length && b == bqual_length && !memcmp(d, data, g+b); }
- void set(struct xid_t *xid)
+ void set(const struct xid_t *xid)
{ memcpy(this, xid, xid->length()); }
void set(long f, const char *g, long gl, const char *b, long bl)
{
@@ -958,7 +965,7 @@ struct xid_t {
memcpy(&trx_server_id, data+MYSQL_XID_PREFIX_LEN, sizeof(trx_server_id));
return trx_server_id;
}
- uint length()
+ uint length() const
{
return static_cast(sizeof(formatID)) + key_length();
}
@@ -1226,6 +1233,17 @@ typedef struct st_ha_create_table_option {
struct st_mysql_sys_var *var;
} ha_create_table_option;
+
+/* Struct used to return binlog file list for SHOW BINARY LOGS from engine. */
+struct binlog_file_entry
+{
+ binlog_file_entry *next;
+ LEX_CSTRING name;
+ /* The size is filled in by server, engine need not return it. */
+ my_off_t size;
+};
+
+
class handler;
class group_by_handler;
class derived_handler;
@@ -1529,6 +1547,142 @@ struct handlerton
void (*update_optimizer_costs)(OPTIMIZER_COSTS *costs);
void *optimizer_costs; /* Costs are stored here */
+ /* Optional implementation of binlog in the engine. */
+ bool (*binlog_init)(size_t binlog_size, const char *directory,
+ HASH *recover_xid_hash);
+ /* Dynamically changing the binlog max size. */
+ void (*set_binlog_max_size)(size_t binlog_size);
+ /* Binlog an event group that doesn't go through commit_ordered. */
+ bool (*binlog_write_direct_ordered)(IO_CACHE *cache,
+ handler_binlog_event_group_info *binlog_info,
+ const rpl_gtid *gtid);
+ bool (*binlog_write_direct)(IO_CACHE *cache,
+ handler_binlog_event_group_info *binlog_info,
+ const rpl_gtid *gtid);
+ /*
+ Called for the last transaction (only) in a binlog group commit, with
+ no locks being held.
+ */
+ void (*binlog_group_commit_ordered)(THD *thd,
+ handler_binlog_event_group_info *binlog_info);
+ /*
+ Binlog parts of large transactions out-of-band, in different chunks in the
+ binlog as the transaction executes. This limits the amount of data that
+ must be binlogged transactionally during COMMIT. The engine_data points to
+ a pointer location that the engine can set to maintain its own context
+ for the out-of-band data.
+
+ Optionally savepoints can be set at the point at the start of the write
+ (ie. before any written data), when stmt_start_data and/or savepoint_data
+ are non-NULL. Such a point can later be rolled back to by calling
+ binlog_savepoint_rollback(). (Only) if stmt_start_data or savepoint_data
+ is non-null can data_len be null (to set savepoint(s) and do nothing else).
+ */
+ bool (*binlog_oob_data_ordered)(THD *thd, const unsigned char *data,
+ size_t data_len, void **engine_data,
+ void **stmt_start_data,
+ void **savepoint_data);
+ bool (*binlog_oob_data)(THD *thd, const unsigned char *data,
+ size_t data_len, void **engine_data);
+ /*
+ Rollback to a prior point in out-of-band binlogged partial transaction
+ data, for savepoint support. The stmt_start_data and/or savepoint_data,
+ if non-NULL, correspond to the point set by an earlier binlog_oob_data()
+ call.
+
+ Exactly one of stmt_start_data or savepoint_data will be non-NULL,
+ corresponding to either rolling back to the start of the current statement,
+ or to an earlier set savepoint.
+ */
+ void (*binlog_savepoint_rollback)(THD *thd, void **engine_data,
+ void **stmt_start_data,
+ void **savepoint_data);
+ /*
+ Call to reset (for new transactions) the engine_data from
+ binlog_oob_data(). Can also change the pointer to point to different data
+ (or set it to NULL).
+ */
+ void (*binlog_oob_reset)(void **engine_data);
+ /* Call to allow engine to release the engine_data from binlog_oob_data(). */
+ void (*binlog_oob_free)(void *engine_data);
+ /*
+ Durably persist the event data for the current user-XA transaction,
+ identified by XID.
+
+ This way, a later XA COMMIT can then be binlogged correctly with the
+ persisted event data, even across server restart.
+
+ The ENGINE_COUNT is the number of storage engines that participate in the
+ XA transaction. This is used to correctly handle crash recovery if the
+ server crashed in the middle of XA PREPARE. If during crash recovery,
+ we find the XID present in less than ENGINE_COUNT engines, then the
+ XA PREPARE did not complete before the crash, and should be rolled back
+ during crash recovery.
+ */
+ /* Binlog an event group that doesn't go through commit_ordered. */
+ bool (*binlog_write_xa_prepare_ordered)(THD *thd,
+ handler_binlog_event_group_info *binlog_info, uchar engine_count);
+ bool (*binlog_write_xa_prepare)(THD *thd,
+ handler_binlog_event_group_info *binlog_info, uchar engine_count);
+ /*
+ Binlog rollback a transaction that was previously made durably prepared
+ with binlog_write_xa_prepare.
+ */
+ bool (*binlog_xa_rollback_ordered)(THD *thd, const XID *xid,
+ void **engine_data);
+ bool (*binlog_xa_rollback)(THD *thd, const XID *xid, void **engine_data);
+ /*
+ The "unlog" method is used after a commit with an XID - either internal
+ 2-phase commit with a separate storage engine, or explicit user
+ XA COMMIT. For user XA, it is also used after XA ROLLBACK.
+
+ The binlog first writes the commit durably, then the engines commit
+ durably, and finally "unlog" is done. The binlog engine must ensure it
+ can recover the committed XID until unlog has been called, after which
+ point resources can be freed, binlog files purged, etc.
+ */
+ void (*binlog_unlog)(const XID *xid, void **engine_data);
+ /*
+ Obtain an object to allow reading from the binlog.
+ The boolean argument wait_durable is set to true to require that
+ transactions be durable before they can be read and returned from the
+ reader. This is used to make replication crash-safe without requiring
+ durability; this way, if the master crashes, when it comes back up the
+ slave will not be ahead and replication will not diverge.
+ */
+ handler_binlog_reader * (*get_binlog_reader)(bool wait_durable);
+ /*
+ Obtain the current position in the binlog.
+ Used to support legacy SHOW MASTER STATUS.
+ */
+ void (*binlog_status)(uint64_t * out_fileno, uint64_t *out_pos);
+ /* Get a binlog name from a file_no. */
+ void (*get_filename)(char name[FN_REFLEN], uint64_t file_no);
+ /* Obtain list of binlog files (SHOW BINARY LOGS). */
+ binlog_file_entry * (*get_binlog_file_list)(MEM_ROOT *mem_root);
+ /*
+ End the current binlog file, and create and switch to a new one.
+ Used to implement FLUSH BINARY LOGS.
+ */
+ bool (*binlog_flush)();
+ /*
+ Read the binlog state at the start of the very first (not purged) binlog
+ file, and return it in *out_state. This is used to check validity of
+ FLUSH BINARY LOGS DELETE_DOMAIN_ID=().
+
+ Returns true on error, false on ok.
+ */
+ bool (*binlog_get_init_state)(rpl_binlog_state_base *out_state);
+ /* Engine implementation of RESET MASTER. */
+ bool (*reset_binlogs)();
+ /*
+ Engine implementation of PURGE BINARY LOGS.
+ Return 0 for ok or one of LOG_INFO_* errors.
+
+ See also ha_binlog_purge_info() for auto-purge.
+ */
+ int (*binlog_purge)(handler_binlog_purge_info *purge_info);
+
/*
Optional clauses in the CREATE/ALTER TABLE
*/
@@ -5672,6 +5826,7 @@ int ha_commit_one_phase(THD *thd, bool all);
int ha_commit_trans(THD *thd, bool all);
int ha_rollback_trans(THD *thd, bool all);
int ha_prepare(THD *thd);
+int ha_recover_engine_binlog(HASH *xid_hash);
int ha_recover(HASH *commit_list, MEM_ROOT *mem_root= NULL);
uint ha_recover_complete(HASH *commit_list, Binlog_offset *coord= NULL);
@@ -5811,4 +5966,132 @@ int get_select_field_pos(Alter_info *alter_info, int select_field_count,
#ifndef DBUG_OFF
String dbug_format_row(TABLE *table, const uchar *rec, bool print_names= true);
#endif /* DBUG_OFF */
+
+/* Struct with info about an event group to be binlogged by a storage engine. */
+struct handler_binlog_event_group_info {
+ /*
+ These are returned by (set by) the binlog_write_direct_ordered hton
+ method to approximate/best-effort position of the start of where the
+ event group was written.
+ */
+ uint64_t out_file_no;
+ uint64_t out_offset;
+ /* Opaque pointer for the engine's use. */
+ void *engine_ptr;
+ /*
+ Secondary engine context ptr.
+ This will be non-null only when both non-transactional (aka statement cache)
+ and transactional (aka transaction cache) updates are binlogged together.
+ Then this secondary pointer is the non-transactional / statement cache
+ part, and it should be considered to go before the transactional /
+ transaction cache part in the commit record.
+ */
+ void *engine_ptr2;
+ /*
+ The XID for XA PREPARE/XA COMMIT; else NULL.
+ When this is set, the IO_CACHE only contains the GTID. All other event data
+ was spilled as OOB and persisted with the binlog_write_xa_prepare hton
+ call; the engine binlog implementation must use the XID to look up or
+ otherwise refer to that OOB data.
+ */
+ const XID *xa_xid;
+ /* End of data that has already been binlogged out-of-band. */
+ my_off_t out_of_band_offset;
+ /*
+ Offset of the GTID event, which comes first in the event group, but is put
+ at the end of the IO_CACHE containing the data to be binlogged.
+ */
+ my_off_t gtid_offset;
+ /*
+ If xa_xid is non-NULL, this is set for an internal 2-phase commit between
+ the engine binlog and one or more additional storage engines participating
+ in the transaction. In this case, there is no call to the
+ binlog_write_xa_prepare() method. The binlog engine must record durably
+ that the xa_xid was committed, and in case of recovery it must pass the
+ xa_xid to the server layer for it to commit in all participating engines.
+
+ If not set, any XID is user external XA, and the xa_xid was previously
+ passed to binlog_write_xa_prepare(). The binlog engine must again record
+ durably that the xa_xid was committed and recover it in case of crash.
+
+ The ability to recover the xa_xid must remain until the binlog_xa_unlog()
+ method is called.
+ */
+ bool internal_xa;
+};
+
+
+/* Structure returned by ha_binlog_purge_info(). */
+struct handler_binlog_purge_info {
+ /* The earliest binlog file that is in use by a dump thread. */
+ uint64_t limit_file_no;
+ /*
+ Set by engine to give a reason why a requested purge could not be done.
+ If set, then nonpurge_filename should be set to the filename.
+
+ Also set by ha_binlog_purge_info() when it returns false, to the reason
+ why no purge is possible. In this case, the nonpurge_filename is set
+ to the empty string.
+ */
+ const char *nonpurge_reason;
+ /* The user-configured maximum total size of the binlog. */
+ ulonglong limit_size;
+ /* Binlog name, for PURGE BINARY LOGS TO. */
+ const char *limit_name;
+ /* The earliest file date (unix timestamp) that should not be purged. */
+ time_t limit_date;
+ /* Whether purge by date and/or by size and/or name is requested. */
+ bool purge_by_date, purge_by_size, purge_by_name;
+ /*
+ The name of the file that could not be purged, when nonpurge_reason
+ is given.
+ */
+ char nonpurge_filename[FN_REFLEN];
+ /* Default constructor to silence compiler warnings -Wuninitialized. */
+ handler_binlog_purge_info()= default;
+};
+
+
+/*
+ Structure holding information about each XID present in binlog engine at
+ server startup.
+
+ Objects of this class (or a class derived from it by the engine binlog
+ implementation) will be inserted into a HASH passed to the binlog_init
+ hton call. The server layer will free these objects using normal delete.
+*/
+class handler_binlog_xid_info {
+public:
+ enum binlog_xid_state {
+ BINLOG_PREPARE, BINLOG_COMMIT, BINLOG_ROLLBACK
+ };
+ XID xid;
+ /*
+ Number of storage engines in which this transaction is prepared. Used when
+ xid_state==BINLOG_PREPARE.
+
+ This is used to correctly recover from a crash in the middle of an XA
+ PREPARE. If the crash happens before all engines had time to durably
+ prepare, then the XID will be rolled back. If all engines got prepared,
+ then the XID will be preserved in "prepared" state.
+ */
+ uint32_t engine_count;
+ /* Bitmap of which engine(s) a prepared transaction was found in. */
+ uint32_t engine_map;
+ enum binlog_xid_state xid_state;
+
+ /* The key function to use for the HASH. */
+ static const uchar *get_key(const void *p, size_t *out_len, my_bool)
+ {
+ const XID *xid=
+ &(reinterpret_cast(p)->xid);
+ *out_len= xid->key_length();
+ return xid->key();
+ }
+ handler_binlog_xid_info(binlog_xid_state typ) :
+ engine_count(0), engine_map(0), xid_state(typ) { }
+ virtual ~handler_binlog_xid_info() { };
+};
+
+
#endif /* HANDLER_INCLUDED */
diff --git a/sql/item_func.cc b/sql/item_func.cc
index eb433c52e55db..e169567ef2694 100644
--- a/sql/item_func.cc
+++ b/sql/item_func.cc
@@ -4028,6 +4028,12 @@ longlong Item_master_pos_wait::val_int()
if (!(mi= get_master_info(&connection_name, Sql_condition::WARN_LEVEL_WARN)))
goto err;
+ if (mi->binlog_storage_engine)
+ {
+ my_error(ER_NOT_AVAILABLE_WITH_ENGINE_BINLOG, MYF(0), "master_pos_wait()");
+ mi->release();
+ goto err;
+ }
if ((event_count = mi->rli.wait_for_pos(thd, log_name, pos, timeout)) == -2)
{
null_value = 1;
diff --git a/sql/item_strfunc.cc b/sql/item_strfunc.cc
index 441c5a0105ea7..8d121002338f1 100644
--- a/sql/item_strfunc.cc
+++ b/sql/item_strfunc.cc
@@ -3635,6 +3635,12 @@ String *Item_func_binlog_gtid_pos::val_str(String *str)
String name_str, *name;
longlong pos;
+ if (opt_binlog_engine_hton)
+ {
+ my_error(ER_NOT_AVAILABLE_WITH_ENGINE_BINLOG, MYF(0), "BINLOG_GTID_POS()");
+ goto err;
+ }
+
name= args[0]->val_str(&name_str);
pos= args[1]->val_int();
diff --git a/sql/log.cc b/sql/log.cc
index 095eee4f8e6fe..b5f718ed2cedf 100644
--- a/sql/log.cc
+++ b/sql/log.cc
@@ -94,6 +94,7 @@ static bool test_if_number(const char *str,
static int binlog_init(void *p);
static int binlog_close_connection(handlerton *hton, THD *thd);
static int binlog_savepoint_set(handlerton *hton, THD *thd, void *sv);
+static int binlog_savepoint_release(handlerton *hton, THD *thd, void *sv);
static int binlog_savepoint_rollback(handlerton *hton, THD *thd, void *sv);
static bool binlog_savepoint_rollback_can_release_mdl(handlerton *hton,
THD *thd);
@@ -102,6 +103,8 @@ static int binlog_start_consistent_snapshot(handlerton *hton, THD *thd);
static int binlog_flush_cache(THD *thd, binlog_cache_mngr *cache_mngr,
Log_event *end_ev, bool all, bool using_stmt,
bool using_trx, bool is_ro_1pc);
+static int binlog_spill_to_engine(struct st_io_cache *cache, const uchar *data,
+ size_t len);
static const LEX_CSTRING write_error_msg=
{ STRING_WITH_LEN("error writing to the binary log") };
@@ -134,6 +137,8 @@ static ulonglong binlog_status_group_commit_trigger_lock_wait;
static ulonglong binlog_status_group_commit_trigger_timeout;
static char binlog_snapshot_file[FN_REFLEN];
static ulonglong binlog_snapshot_position;
+static constexpr size_t BINLOG_SPILL_MAX= 512 * 1024;
+static size_t binlog_max_spill_size;
static const char *fatal_log_error=
"Could not use %s for logging (error %d). "
@@ -161,6 +166,19 @@ static SHOW_VAR binlog_status_vars_detail[]=
{NullS, NullS, SHOW_LONG}
};
+
+/*
+ This struct, for --binlog-storage-engine=ENGINE, keeps track of savepoints
+ set in the current transaction that are still within the in-memory trx
+ cache (not yet spilled as out-of-band data into the binlog).
+*/
+struct binlog_savepoint_info {
+ binlog_savepoint_info *next;
+ void *engine_ptr;
+ my_off_t cache_offset;
+};
+
+
/*
Variables for the binlog background thread.
Protected by the MYSQL_BIN_LOG::LOCK_binlog_background_thread mutex.
@@ -361,15 +379,24 @@ Log_event::select_checksum_alg(const binlog_cache_data *data)
class binlog_cache_mngr {
public:
- binlog_cache_mngr(my_off_t param_max_binlog_stmt_cache_size,
+ binlog_cache_mngr(THD *thd_arg,
+ my_off_t param_max_binlog_stmt_cache_size,
my_off_t param_max_binlog_cache_size,
ulong *param_ptr_binlog_stmt_cache_use,
ulong *param_ptr_binlog_stmt_cache_disk_use,
ulong *param_ptr_binlog_cache_use,
ulong *param_ptr_binlog_cache_disk_use,
bool precompute_checksums)
- : stmt_cache(precompute_checksums), trx_cache(precompute_checksums),
- last_commit_pos_offset(0), using_xa(FALSE), xa_xid(0)
+ : thd(thd_arg),
+ stmt_cache(false, precompute_checksums),
+ trx_cache(true, precompute_checksums),
+ last_commit_pos_offset(0),
+ stmt_start_engine_ptr(nullptr),
+ cache_savepoint_list(nullptr),
+ cache_savepoint_next_ptr(&cache_savepoint_list),
+ using_stmt_cache(FALSE), using_trx_cache(FALSE),
+ using_xa(FALSE), xa_xid(0),
+ engine_binlogged(FALSE), need_write_direct(FALSE)
{
stmt_cache.set_binlog_cache_info(param_max_binlog_stmt_cache_size,
param_ptr_binlog_stmt_cache_use,
@@ -377,20 +404,61 @@ class binlog_cache_mngr {
trx_cache.set_binlog_cache_info(param_max_binlog_cache_size,
param_ptr_binlog_cache_use,
param_ptr_binlog_cache_disk_use);
- last_commit_pos_file[0]= 0;
+ if (opt_binlog_engine_hton)
+ last_commit_pos_file.engine_file_no= ~(uint64_t)0;
+ else
+ last_commit_pos_file.legacy_name[0]= 0;
+ }
+ ~binlog_cache_mngr()
+ {
}
void reset(bool do_stmt, bool do_trx)
{
if (do_stmt)
- stmt_cache.reset();
+ {
+ if (opt_binlog_engine_hton)
+ {
+ stmt_cache.reset_for_engine_binlog();
+ /*
+ Use a custom write_function to spill to the engine-implemented binlog.
+ And re-use the IO_CACHE::append_read_pos as a handle for our
+ write_function; it is unused when the cache is not SEQ_READ_APPEND.
+ */
+ stmt_cache.cache_log.write_function= binlog_spill_to_engine;
+ stmt_cache.cache_log.append_read_pos= (uchar *)this;
+ }
+ else
+ stmt_cache.reset();
+ using_stmt_cache= FALSE;
+ }
if (do_trx)
{
- trx_cache.reset();
- using_xa= FALSE;
- last_commit_pos_file[0]= 0;
+ if (opt_binlog_engine_hton)
+ {
+ trx_cache.reset_for_engine_binlog();
+ trx_cache.cache_log.write_function= binlog_spill_to_engine;
+ trx_cache.cache_log.append_read_pos= (uchar *)this;
+ last_commit_pos_file.engine_file_no= ~(uint64_t)0;
+ stmt_start_engine_ptr= nullptr;
+ cache_savepoint_list= nullptr;
+ cache_savepoint_next_ptr= &cache_savepoint_list;
+ }
+ else
+ {
+ trx_cache.reset();
+ last_commit_pos_file.legacy_name[0]= 0;
+ }
last_commit_pos_offset= 0;
+ using_trx_cache= FALSE;
+ using_xa= FALSE;
}
+ engine_binlogged= FALSE;
+ need_write_direct= FALSE;
+ /*
+ need_engine_2pc is not reset here, as we need it still, at the end of
+ MYSQL_LOG_BIN::log_and_order() where it will be reset.
+ */
}
binlog_cache_data* get_binlog_cache_data(bool is_transactional)
@@ -398,36 +466,109 @@ class binlog_cache_mngr {
return (is_transactional ? &trx_cache : &stmt_cache);
}
+ /*
+ The cache_data to use when binlogging into the --binlog-storage-engine.
+
+ With binlog in storage engine, we're optimizing for transactional
+ event groups, and for simplicity we only pass a single cache into the
+ engine binlog implementation.
+
+ When mixing transactional and non-transactional updates in a single event
+ group, we flush everything as out-of-band-data, and use the transaction
+ cache just for the GTID.
+
+ The special case comes when we are using both the transactional and
+ the statement cache, _but_ the transaction cache happens to be empty.
+ Then we need to put the GTID in the statement cache and pass that to
+ the engine.
+ */
+ binlog_cache_data *engine_cache_data()
+ {
+ return ( unlikely(!using_trx_cache) || ( unlikely(using_stmt_cache) &&
+ !stmt_cache.empty() &&
+ trx_cache.empty() ) ) ?
+ &stmt_cache : &trx_cache;
+ }
+
IO_CACHE* get_binlog_cache_log(bool is_transactional)
{
return (is_transactional ? &trx_cache.cache_log : &stmt_cache.cache_log);
}
+ THD *thd;
+
binlog_cache_data stmt_cache;
binlog_cache_data trx_cache;
+ /* Buffer used to pass internal my_xid into engine as struct xid_t. */
+ XID xid_buf;
+
/*
Binlog position for current transaction.
For START TRANSACTION WITH CONSISTENT SNAPSHOT, this is the binlog
position corresponding to the snapshot taken. During (and after) commit,
this is set to the binlog position corresponding to just after the
commit (so storage engines can store it in their transaction log).
+
+ For the legacy binlog, we have to use the full filename, for binlog
+ implemented in engine we can just keep track of the file number.
*/
- char last_commit_pos_file[FN_REFLEN];
- my_off_t last_commit_pos_offset;
+ union {
+ uint64_t engine_file_no;
+ char legacy_name[FN_REFLEN];
+ } last_commit_pos_file;
+ uint64_t last_commit_pos_offset;
+
+ /* Engine data pointer for start-of-statement savepoint. */
+ void *stmt_start_engine_ptr;
+ /*
+ List of pending savepoints still in the trx cache (for engine-implemented
+ binlogging).
+ */
+ binlog_savepoint_info *cache_savepoint_list;
+ binlog_savepoint_info **cache_savepoint_next_ptr;
+ /*
+ Set from binlog_flush_cache(), to mark if we are flushing the stmt cache
+ or the trx cache (or both).
+ */
+ bool using_stmt_cache;
+ bool using_trx_cache;
/*
Flag set true if this transaction is committed with log_xid() as part of
XA, false if not.
*/
bool using_xa;
my_xid xa_xid;
+ /*
+ Set true when not using --binlog-storage-engine and we need to decrement
+ the xid_list reference count for the transaction at unlog time. (The
+ xid_list refcounting is used to keep binlog files for recovery while
+ transactions may still be in the prepared state).
+ */
bool need_unlog;
+ /*
+ Set true when binlog engine fetches the cache data with binlog_get_cache()
+ and does the binlogging for us.
+ */
+ bool engine_binlogged;
+ /*
+ Set when we called binlog_write_direct_ordered() during binlog group commit
+ (due to engine_binlogged==false) and need to call binlog_write_direct()
+ later after releasing mutex.
+ */
+ bool need_write_direct;
/*
Id of binlog that transaction was written to; only needed if need_unlog is
true.
*/
+ /*
+ Set when using --binlog-storage-engine, but there is another XA-capable
+ engine involved in the transaction, so that we need to do 2-phase commit
+ to ensure consistency in case of crash.
+ */
+ bool need_engine_2pc;
ulong binlog_id;
/* Set if we get an error during commit that must be returned from unlog(). */
bool delayed_error;
@@ -1643,11 +1784,11 @@ int LOGGER::set_handlers(ulonglong slow_log_printer,
*/
static void
-binlog_trans_log_savepos(THD *thd, my_off_t *pos)
+binlog_trans_log_savepos(THD *thd, binlog_cache_mngr *cache_mngr, my_off_t *pos)
{
DBUG_ENTER("binlog_trans_log_savepos");
+// DBUG_ASSERT(!opt_binlog_engine_hton);
DBUG_ASSERT(pos != NULL);
- binlog_cache_mngr *const cache_mngr= thd->binlog_setup_trx_data();
DBUG_ASSERT((WSREP(thd) && wsrep_emulate_bin_log) || mysql_bin_log.is_open());
*pos= cache_mngr->trx_cache.get_byte_position();
DBUG_PRINT("return", ("*pos: %lu", (ulong) *pos));
@@ -1671,9 +1812,10 @@ binlog_trans_log_savepos(THD *thd, my_off_t *pos)
*/
static void
-binlog_trans_log_truncate(THD *thd, my_off_t pos)
+binlog_trans_log_truncate(THD *thd, binlog_savepoint_info *sv)
{
DBUG_ENTER("binlog_trans_log_truncate");
+ my_off_t pos= sv->cache_offset;
DBUG_PRINT("enter", ("pos: %lu", (ulong) pos));
DBUG_ASSERT(thd->binlog_get_cache_mngr() != NULL);
@@ -1681,7 +1823,62 @@ binlog_trans_log_truncate(THD *thd, my_off_t pos)
DBUG_ASSERT(pos != ~(my_off_t) 0);
binlog_cache_mngr *const cache_mngr= thd->binlog_get_cache_mngr();
- cache_mngr->trx_cache.restore_savepoint(pos);
+ binlog_cache_data *trx_cache= &cache_mngr->trx_cache;
+ if (!opt_binlog_engine_hton)
+ {
+ trx_cache->restore_savepoint(pos);
+ DBUG_VOID_RETURN;
+ }
+
+ /*
+ If the savepoint is still in the trx cache, then we can simply truncate
+ the cache.
+ If the savepoint was spilled as oob data, then we need to call into the
+ engine binlog to have it discard the to-be-rolled-back binlog data.
+ */
+ IO_CACHE *cache= &trx_cache->cache_log;
+ if (pos >= cache->pos_in_file)
+ {
+ trx_cache->restore_savepoint(pos);
+ trx_cache->cache_log.write_function= binlog_spill_to_engine;
+ /* Remove any later in-cache savepoints. */
+ binlog_savepoint_info *sp= cache_mngr->cache_savepoint_list;
+ while (sp)
+ {
+ if (sp == sv)
+ {
+ sp->next= nullptr; /* Drop the tail of the list. */
+ cache_mngr->cache_savepoint_next_ptr= &sp->next;
+ break;
+ }
+ sp= sp->next;
+ }
+ /*
+ If the savepoint is at the start of the cache, then it might have been
+ already spilled to the engine binlog, then rolled back to (which would
+ leave the cache truncated to the point of that savepoint).
+
+ But otherwise, the savepoint is pending to be spilled to engine if
+ needed, and should be found in the list.
+ */
+ DBUG_ASSERT(pos == cache->pos_in_file || sp != nullptr);
+
+ DBUG_VOID_RETURN;
+ }
+
+ /*
+ Truncate what's in the cache, then call into the engine to rollback to
+ the prior set savepoint.
+ */
+ trx_cache->restore_savepoint(cache->pos_in_file);
+ trx_cache->reset_cache_for_engine(pos, binlog_spill_to_engine);
+ /* No pending savepoints in-cache anymore. */
+ cache_mngr->cache_savepoint_next_ptr= &cache_mngr->cache_savepoint_list;
+ cache_mngr->cache_savepoint_list= nullptr;
+ cache_mngr->trx_cache.engine_binlog_info.out_of_band_offset= sv->cache_offset;
+ (*opt_binlog_engine_hton->binlog_savepoint_rollback)
+ (thd, &cache_mngr->trx_cache.engine_binlog_info.engine_ptr,
+ nullptr, &sv->engine_ptr);
DBUG_VOID_RETURN;
}
@@ -1695,9 +1892,10 @@ binlog_trans_log_truncate(THD *thd, my_off_t pos)
int binlog_init(void *p)
{
binlog_hton= (handlerton *)p;
- binlog_hton->savepoint_offset= sizeof(my_off_t);
+ binlog_hton->savepoint_offset= sizeof(binlog_savepoint_info);
binlog_hton->close_connection= binlog_close_connection;
binlog_hton->savepoint_set= binlog_savepoint_set;
+ binlog_hton->savepoint_release= binlog_savepoint_release;
binlog_hton->savepoint_rollback= binlog_savepoint_rollback;
binlog_hton->savepoint_rollback_can_release_mdl=
binlog_savepoint_rollback_can_release_mdl;
@@ -1805,9 +2003,9 @@ binlog_flush_cache(THD *thd, binlog_cache_mngr *cache_mngr,
DBUG_ENTER("binlog_flush_cache");
DBUG_PRINT("enter", ("end_ev: %p", end_ev));
- if ((using_stmt && !cache_mngr->stmt_cache.empty()) ||
- (using_trx && !cache_mngr->trx_cache.empty()) ||
- thd->transaction->xid_state.is_explicit_XA())
+ bool doing_stmt= using_stmt && !cache_mngr->stmt_cache.empty();
+ bool doing_trx= using_trx && !cache_mngr->trx_cache.empty();
+ if (doing_stmt || doing_trx || thd->transaction->xid_state.is_explicit_XA())
{
/*
thd->binlog_flush_pending_rows_event() ensures that the pending row event
@@ -1849,6 +2047,46 @@ binlog_flush_cache(THD *thd, binlog_cache_mngr *cache_mngr,
}
#endif /* WITH_WSREP */
+ if (opt_binlog_engine_hton &&
+ likely(!(thd->lex->sql_command == SQLCOM_XA_COMMIT &&
+ thd->lex->xa_opt != XA_ONE_PHASE)))
+ {
+ /*
+ Write the end_event into the cache, in preparation for sending the
+ cache to the engine to be binlogged as a whole.
+
+ Except for user XA COMMIT, where we already wrote the end event into
+ the OOB data that was persisted in the binlog.
+ */
+ binlog_cache_data *cache_data;
+ if (doing_trx || !doing_stmt)
+ {
+ end_ev->cache_type= Log_event::EVENT_TRANSACTIONAL_CACHE;
+ cache_data= &cache_mngr->trx_cache;
+ }
+ else
+ {
+ end_ev->cache_type= Log_event::EVENT_STMT_CACHE;
+ cache_data= &cache_mngr->stmt_cache;
+ }
+ if (mysql_bin_log.write_event(end_ev, cache_data, &cache_data->cache_log))
+ DBUG_RETURN(1);
+
+ if (cache_data->engine_binlog_info.out_of_band_offset)
+ {
+ /*
+ This is a "large" transaction, where parts of the transaction were
+ already binlogged out-of-band to the engine binlog.
+
+ Binlog the remaining bits of event data as well, so all the event
+ group is consecutive out-of-band data and the commit record will
+ only contain the GTID event (depending on engine implementation).
+ */
+ if (my_b_flush_io_cache(&cache_data->cache_log, 0))
+ DBUG_RETURN(1);
+ }
+ }
+
/*
Doing a commit or a rollback including non-transactional tables,
i.e., ending a transaction where we might write the transaction
@@ -1886,6 +2124,69 @@ binlog_flush_cache(THD *thd, binlog_cache_mngr *cache_mngr,
}
+static void
+binlog_setup_engine_commit_data(handler_binlog_event_group_info *context,
+ binlog_cache_mngr *cache_mngr)
+{
+ if (unlikely(context->xa_xid))
+ {
+ /* Mark that we are doing XA and need to unlog. */
+ cache_mngr->need_engine_2pc= true;
+ context->internal_xa= false;
+ }
+ else if (unlikely(cache_mngr->need_engine_2pc))
+ {
+ /* Internal 2-phase with multiple xa-capable engines. */
+ DBUG_ASSERT(cache_mngr->xa_xid != 0);
+ cache_mngr->xid_buf.set(cache_mngr->xa_xid);
+ context->xa_xid= &cache_mngr->xid_buf;
+ context->internal_xa= true;
+ }
+}
+
+
+extern "C"
+void
+binlog_get_cache(THD *thd, uint64_t file_no, uint64_t offset,
+ IO_CACHE **out_cache,
+ handler_binlog_event_group_info **out_context,
+ const rpl_gtid **out_gtid)
+{
+ IO_CACHE *cache= nullptr;
+ handler_binlog_event_group_info *context= nullptr;
+ binlog_cache_mngr *cache_mngr;
+ const rpl_gtid *gtid= nullptr;
+ /* opt_binlog_engine_hton can be unset during bootstrap. */
+ if (likely(opt_binlog_engine_hton) &&
+ (cache_mngr= thd->binlog_get_cache_mngr()))
+ {
+ cache_mngr->engine_binlogged= TRUE;
+ cache_mngr->last_commit_pos_file.engine_file_no= file_no;
+ cache_mngr->last_commit_pos_offset= offset;
+ binlog_cache_data *cache_data= cache_mngr->engine_cache_data();
+ cache= &cache_data->cache_log;
+ context= &cache_data->engine_binlog_info;
+ /*
+ If we are binlogging from both stmt and trx cache in the same event
+ group, pass the engine context for out-of-band stmt data as
+ engine_ptr2. In this case, we have flushed everything in both
+ caches out as out-of-band data already.
+ */
+ if (likely(cache_data->trx_cache()) &&
+ unlikely(!cache_mngr->stmt_cache.empty()))
+ context->engine_ptr2=
+ cache_mngr->stmt_cache.engine_binlog_info.engine_ptr;
+ else
+ context->engine_ptr2= nullptr;
+ binlog_setup_engine_commit_data(context, cache_mngr);
+ gtid= thd->get_last_commit_gtid();
+ }
+ *out_cache= cache;
+ *out_context= context;
+ *out_gtid= gtid;
+}
+
+
/**
This function flushes the stmt-cache upon commit.
@@ -1953,8 +2254,17 @@ binlog_commit_flush_trx_cache(THD *thd, bool all, binlog_cache_mngr *cache_mngr,
DBUG_ASSERT(thd->transaction->xid_state.get_state_code() ==
XA_PREPARED);
- buflen= serialize_with_xid(thd->transaction->xid_state.get_xid(),
- buf, query, q_len);
+ if (opt_binlog_engine_hton)
+ {
+ cache_mngr->trx_cache.engine_binlog_info.xa_xid=
+ thd->transaction->xid_state.get_xid();
+ cache_mngr->trx_cache.engine_binlog_info.internal_xa= false;
+ }
+ else
+ {
+ buflen= serialize_with_xid(thd->transaction->xid_state.get_xid(),
+ buf, query, q_len);
+ }
}
Query_log_event end_evt(thd, buf, buflen, TRUE, TRUE, TRUE, 0);
@@ -1962,6 +2272,33 @@ binlog_commit_flush_trx_cache(THD *thd, bool all, binlog_cache_mngr *cache_mngr,
}
+static int
+binlog_engine_xa_rollback(THD *thd, binlog_cache_mngr *cache_mngr)
+{
+ DBUG_ASSERT(opt_binlog_engine_hton);
+ DBUG_ASSERT(thd->transaction->xid_state.is_explicit_XA());
+
+ int err= 0;
+ binlog_cache_data *cache_data= cache_mngr->get_binlog_cache_data(true);
+ handler_binlog_event_group_info *engine_context=
+ &cache_data->engine_binlog_info;
+ const XID *xid= thd->transaction->xid_state.get_xid();
+ engine_context->xa_xid= xid;
+ engine_context->internal_xa= false;
+ mysql_mutex_lock(&LOCK_commit_ordered);
+ err= (*opt_binlog_engine_hton->binlog_xa_rollback_ordered)
+ (thd, xid, &engine_context->engine_ptr);
+ mysql_mutex_unlock(&LOCK_commit_ordered);
+ if (likely(!err))
+ err= (*opt_binlog_engine_hton->binlog_xa_rollback)
+ (thd, xid, &engine_context->engine_ptr);
+ cache_mngr->reset(false, true);
+ cache_mngr->need_engine_2pc= true;
+
+ return err;
+}
+
+
/**
This function flushes the trx-cache upon rollback.
@@ -1971,7 +2308,7 @@ binlog_commit_flush_trx_cache(THD *thd, bool all, binlog_cache_mngr *cache_mngr,
@return
nonzero if an error pops up when flushing the cache.
*/
-static inline int
+static int
binlog_rollback_flush_trx_cache(THD *thd, bool all,
binlog_cache_mngr *cache_mngr)
{
@@ -1980,12 +2317,21 @@ binlog_rollback_flush_trx_cache(THD *thd, bool all,
char buf[q_len + ser_buf_size]= "ROLLBACK";
size_t buflen= sizeof("ROLLBACK") - 1;
- if (thd->transaction->xid_state.is_explicit_XA())
+ if (unlikely(thd->transaction->xid_state.is_explicit_XA()))
{
/* for not prepared use plain ROLLBACK */
if (thd->transaction->xid_state.get_state_code() == XA_PREPARED)
+ {
+ if (opt_binlog_engine_hton)
+ {
+ if (unlikely(!cache_mngr))
+ return 1;
+ return binlog_engine_xa_rollback(thd, cache_mngr);
+ }
+
buflen= serialize_with_xid(thd->transaction->xid_state.get_xid(),
buf, query, q_len);
+ }
}
Query_log_event end_evt(thd, buf, buflen, TRUE, TRUE, TRUE, 0);
@@ -2060,8 +2406,40 @@ binlog_truncate_trx_cache(THD *thd, binlog_cache_mngr *cache_mngr, bool all)
If rolling back a statement in a transaction, we truncate the
transaction cache to remove the statement.
*/
- else
+ else if (!opt_binlog_engine_hton)
trx_cache.restore_prev_position();
+ else
+ {
+ IO_CACHE *cache= &trx_cache.cache_log;
+ my_off_t stmt_pos= trx_cache.get_prev_position();
+ /* Drop any pending savepoints in the cache beyond statement start. */
+ binlog_savepoint_info **sp_ptr= &cache_mngr->cache_savepoint_list;
+ for (;;)
+ {
+ binlog_savepoint_info *sp= *sp_ptr;
+ if (!sp || sp->cache_offset > stmt_pos)
+ break;
+ sp_ptr= &sp->next;
+ }
+ *sp_ptr= nullptr;
+ cache_mngr->cache_savepoint_next_ptr= sp_ptr;
+ if (stmt_pos >= cache->pos_in_file)
+ {
+ trx_cache.restore_prev_position();
+ cache->write_function= binlog_spill_to_engine;
+ }
+ else
+ {
+ trx_cache.set_prev_position(cache->pos_in_file);
+ trx_cache.restore_prev_position();
+ trx_cache.reset_cache_for_engine(stmt_pos, binlog_spill_to_engine);
+ cache_mngr->trx_cache.engine_binlog_info.out_of_band_offset= stmt_pos;
+ (*opt_binlog_engine_hton->binlog_savepoint_rollback)
+ (thd, &cache_mngr->trx_cache.engine_binlog_info.engine_ptr,
+ &cache_mngr->stmt_start_engine_ptr, nullptr);
+ }
+ DBUG_ASSERT(cache->write_function == binlog_spill_to_engine);
+ }
DBUG_ASSERT(trx_cache.pending() == NULL);
DBUG_RETURN(error);
@@ -2076,10 +2454,70 @@ inline bool is_preparing_xa(THD *thd)
}
-static int binlog_prepare(handlerton *hton, THD *thd, bool all)
+int
+MYSQL_BIN_LOG::log_xa_prepare(THD *thd, bool all)
{
/* Do nothing unless the transaction is a user XA. */
- return is_preparing_xa(thd) ? binlog_commit(thd, all, FALSE) : 0;
+ if (is_preparing_xa(thd) &&
+ thd->ha_data[binlog_hton->slot].ha_info[1].is_started())
+ {
+ if (opt_binlog_engine_hton)
+ {
+ /*
+ Tell the binlog engine to persist the event data for the current
+ transaction, identified by the user-supplied XID.
+ This way, a later XA COMMIT can then be binlogged correctly with the
+ persisted event data, even across server restart.
+ */
+ binlog_cache_mngr *cache_mngr= thd->binlog_setup_trx_data();
+ if (unlikely(!cache_mngr))
+ return 1;
+ binlog_cache_data *cache_data= cache_mngr->get_binlog_cache_data(true);
+ /* Put in the end event. */
+ {
+ Query_log_event end_ev(thd, STRING_WITH_LEN("COMMIT"),
+ TRUE, TRUE, TRUE, 0);
+ end_ev.cache_type= Log_event::EVENT_TRANSACTIONAL_CACHE;
+ if (write_event(&end_ev, BINLOG_CHECKSUM_ALG_OFF, 0,
+ &cache_data->cache_log))
+ return 1;
+ }
+ /* Make sure all event data is flushed as OOB. */
+ if (unlikely(my_b_flush_io_cache(&cache_data->cache_log, 0)))
+ return 1;
+ handler_binlog_event_group_info *engine_context=
+ &cache_data->engine_binlog_info;
+ engine_context->xa_xid= thd->transaction->xid_state.get_xid();
+ uchar engine_count= (uchar)ha_count_rw_2pc(thd, true);
+ mysql_mutex_lock(&LOCK_commit_ordered);
+ bool err= (*opt_binlog_engine_hton->binlog_write_xa_prepare_ordered)
+ (thd, engine_context, engine_count);
+ mysql_mutex_unlock(&LOCK_commit_ordered);
+ if (likely(!err))
+ err= (*opt_binlog_engine_hton->binlog_write_xa_prepare)
+ (thd, engine_context, engine_count);
+ cache_mngr->reset(false, true);
+ return err;
+ }
+ else
+ return binlog_commit(thd, all, FALSE);
+ }
+ return 0;
+}
+
+
+static int binlog_prepare(handlerton *hton, THD *thd, bool all)
+{
+ /*
+ ToDo: We do not really need a prepare() hton method in the binlog, we are
+ the transaction coordinator, should do our work in log_xa_prepare().
+
+ There is currently code that looks at registered htons if they have the
+ "prepare" method and use that to decide how the transaction should be
+ handled; until this is refactored, we need to have a prepare method in the
+ binlog which just does nothing.
+ */
+ return 0;
}
@@ -2286,6 +2724,17 @@ int binlog_commit(THD *thd, bool all, bool ro_1pc)
!(thd->ha_data[binlog_hton->slot].ha_info[1].is_started() &&
thd->ha_data[binlog_hton->slot].ha_info[1].is_trx_read_write())))
{
+ if (unlikely(thd->transaction->xid_state.get_state_code() == XA_PREPARED) &&
+ opt_binlog_engine_hton)
+ {
+ /*
+ The XA transaction is empty, so we just need to inform the binlog
+ engine that it is complete, there is no actual transaction to binlog.
+ Thus, we can just treat this as a rollback.
+ */
+ error= binlog_engine_xa_rollback(thd, cache_mngr);
+ }
+
/*
This is an empty transaction commit (both the regular and xa),
or such transaction xa-prepare or
@@ -2315,7 +2764,8 @@ int binlog_commit(THD *thd, bool all, bool ro_1pc)
if (cache_mngr->need_unlog && !is_xa_prepare)
{
error=
- mysql_bin_log.unlog(BINLOG_COOKIE_MAKE(cache_mngr->binlog_id,
+ mysql_bin_log.unlog(thd,
+ BINLOG_COOKIE_MAKE(cache_mngr->binlog_id,
cache_mngr->delayed_error), 1);
cache_mngr->need_unlog= false;
}
@@ -2330,6 +2780,87 @@ int binlog_commit(THD *thd, bool all, bool ro_1pc)
DBUG_RETURN(error);
}
+
+void
+binlog_post_commit(THD *thd, bool all)
+{
+ if (!opt_binlog_engine_hton)
+ return;
+
+ binlog_cache_mngr *cache_mngr= thd->binlog_get_cache_mngr();
+ if (likely(cache_mngr != nullptr) && unlikely(cache_mngr->need_engine_2pc))
+ {
+ DBUG_ASSERT(!cache_mngr->trx_cache.engine_binlog_info.internal_xa);
+ DBUG_ASSERT(thd->lex->sql_command == SQLCOM_XA_COMMIT &&
+ thd->lex->xa_opt != XA_ONE_PHASE);
+ cache_mngr->need_engine_2pc= false;
+ (*opt_binlog_engine_hton->binlog_unlog)
+ (thd->transaction->xid_state.get_xid(),
+ &cache_mngr->trx_cache.engine_binlog_info.engine_ptr);
+ }
+}
+
+
+void
+binlog_post_commit_by_xid(handlerton *hton, XID *xid)
+{
+ if (!opt_binlog_engine_hton)
+ return;
+
+ THD *thd= current_thd;
+ binlog_cache_mngr *cache_mngr= thd->binlog_get_cache_mngr();
+ if (likely(cache_mngr != nullptr) && unlikely(cache_mngr->need_engine_2pc))
+ {
+ DBUG_ASSERT(!cache_mngr->trx_cache.engine_binlog_info.internal_xa);
+ DBUG_ASSERT(thd->lex->sql_command == SQLCOM_XA_COMMIT &&
+ thd->lex->xa_opt != XA_ONE_PHASE);
+ cache_mngr->need_engine_2pc= false;
+ (*opt_binlog_engine_hton->binlog_unlog)
+ (xid, &cache_mngr->trx_cache.engine_binlog_info.engine_ptr);
+ }
+}
+
+
+void
+binlog_post_rollback(THD *thd, bool all)
+{
+ if (!opt_binlog_engine_hton)
+ return;
+ binlog_cache_mngr *cache_mngr= thd->binlog_get_cache_mngr();
+ if (likely(cache_mngr != nullptr) && unlikely(cache_mngr->need_engine_2pc))
+ {
+ handler_binlog_event_group_info *context=
+ &cache_mngr->trx_cache.engine_binlog_info;
+ DBUG_ASSERT(!context->internal_xa);
+ if (!context->internal_xa)
+ {
+ const XID *xid= thd->transaction->xid_state.get_xid();
+ (*opt_binlog_engine_hton->binlog_unlog)(xid, &context->engine_ptr);
+ }
+ cache_mngr->need_engine_2pc= false;
+ }
+}
+
+
+void
+binlog_post_rollback_by_xid(handlerton *hton, XID *xid)
+{
+ if (!opt_binlog_engine_hton)
+ return;
+
+ THD *thd= current_thd;
+ binlog_cache_mngr *cache_mngr= thd->binlog_get_cache_mngr();
+ if (likely(cache_mngr != nullptr) && unlikely(cache_mngr->need_engine_2pc))
+ {
+ DBUG_ASSERT(!cache_mngr->trx_cache.engine_binlog_info.internal_xa);
+ DBUG_ASSERT(thd->lex->sql_command == SQLCOM_XA_ROLLBACK);
+ cache_mngr->need_engine_2pc= false;
+ (*opt_binlog_engine_hton->binlog_unlog)
+ (xid, &cache_mngr->trx_cache.engine_binlog_info.engine_ptr);
+ }
+}
+
+
/**
This function is called when a transaction or a statement is rolled back.
@@ -2380,6 +2911,17 @@ int binlog_rollback(handlerton *hton, THD *thd, bool all)
!(thd->ha_data[binlog_hton->slot].ha_info[1].is_started() &&
thd->ha_data[binlog_hton->slot].ha_info[1].is_trx_read_write())))
{
+ if (unlikely(thd->transaction->xid_state.get_state_code() == XA_PREPARED) &&
+ opt_binlog_engine_hton)
+ {
+ /*
+ The XA transaction is empty, so we just need to inform the binlog
+ engine that it is complete, there is no actual transaction to binlog.
+ Thus, we can just treat this as a rollback.
+ */
+ error= binlog_engine_xa_rollback(thd, cache_mngr);
+ }
+
/*
The same comments apply as in the binlog commit method's branch.
*/
@@ -2596,12 +3138,103 @@ static int binlog_savepoint_set(handlerton *hton, THD *thd, void *sv)
or "RELEASE S" without the preceding "SAVEPOINT S" in the binary
log.
*/
- if (likely(!(error= mysql_bin_log.write(&qinfo))))
- binlog_trans_log_savepos(thd, (my_off_t*) sv);
+ if (unlikely((error= mysql_bin_log.write(&qinfo)) != 0))
+ DBUG_RETURN(error);
+ binlog_cache_mngr *cache_mngr= thd->binlog_setup_trx_data();
+ binlog_savepoint_info *sp_info= (binlog_savepoint_info*)sv;
+ binlog_trans_log_savepos(thd, cache_mngr, &sp_info->cache_offset);
+ if (opt_binlog_engine_hton)
+ {
+ /*
+ Add the savepoint to the list of pending savepoints in the trx cache.
+ If the savepoint gets spilled to the binlog as oob data, then we need
+ to create an (engine) binlog savepoint from it so that the engine can
+ roll back the oob data if needed.
+ As long as the savepoint is in the cache, we can simply roll it back
+ by truncating the cache.
+
+ Note that re-using the savepoint name is legal in SQL:
+
+ BEGIN
+ SAVEPOINT A;
+ ...
+ SAVEPOINT A;
+ ...
+ ROLLBACK TO A;
+
+ In this case, the second instance replaces the first one, and we get
+ called with the same sv pointer again. So we traverse the list
+ and remove the old instance, if found, before adding the new one.
+ */
+ binlog_savepoint_info *sp= cache_mngr->cache_savepoint_list;
+ binlog_savepoint_info **next_ptr= &cache_mngr->cache_savepoint_list;
+ while (sp)
+ {
+ if (sp == sp_info)
+ {
+ /*
+ The upper layer (in handler.cc) removes the savepoint and calls
+ binlog_savepoint_release() for us, so we do not expect to have to
+ remove anything here. But let's still do so as defensive coding,
+ but assert that it won't be necessary.
+ */
+ DBUG_ASSERT("Should be removed by ha_release_savepoint()" == nullptr);
+ *next_ptr= sp->next;
+ }
+ else
+ next_ptr= &sp->next;
+ sp=sp->next;
+ }
+
+ /*
+ Assert that the existing cache_savepoint_next_ptr matches either
+ the end of the list now, or it was pointing to the savepoint that
+ is now being set, which is being reused and was deleted from the
+ (end of the) list.
+ */
+ DBUG_ASSERT(next_ptr == cache_mngr->cache_savepoint_next_ptr ||
+ cache_mngr->cache_savepoint_next_ptr == &sp_info->next);
+ /* Insert the savepoint at the end of the list. */
+ *next_ptr= sp_info;
+ cache_mngr->cache_savepoint_next_ptr= &sp_info->next;
+ sp_info->next= nullptr;
+ sp_info->engine_ptr= nullptr;
+ }
DBUG_RETURN(error);
}
+
+/*
+ Release a savepoint.
+ We only need to release if the savepoint is still pending in the cache.
+ If the savepoint has been spilled to the engine, it has already been
+ removed from the list, and the engine will just ignore it.
+*/
+static int
+binlog_savepoint_release(handlerton *hton, THD *thd, void *sv)
+{
+ if (!opt_binlog_engine_hton)
+ return 0;
+
+ binlog_savepoint_info *sp_info= (binlog_savepoint_info*)sv;
+ binlog_cache_mngr *cache_mngr= thd->binlog_setup_trx_data();
+ binlog_savepoint_info *sp= cache_mngr->cache_savepoint_list;
+ binlog_savepoint_info **next_ptr= &cache_mngr->cache_savepoint_list;
+ while (sp)
+ {
+ if (sp == sp_info)
+ *next_ptr= sp->next;
+ else
+ next_ptr= &sp->next;
+ sp=sp->next;
+ }
+ /* Make sure to update cache_savepoint_next_ptr if we delete last in list. */
+ cache_mngr->cache_savepoint_next_ptr= next_ptr;
+ return 0;
+}
+
+
static int binlog_savepoint_rollback(handlerton *hton, THD *thd, void *sv)
{
DBUG_ENTER("binlog_savepoint_rollback");
@@ -2629,7 +3262,7 @@ static int binlog_savepoint_rollback(handlerton *hton, THD *thd, void *sv)
DBUG_RETURN(mysql_bin_log.write(&qinfo));
}
- binlog_trans_log_truncate(thd, *(my_off_t*)sv);
+ binlog_trans_log_truncate(thd, (binlog_savepoint_info *)sv);
/*
When a SAVEPOINT is executed inside a stored function/trigger we force the
@@ -3635,7 +4268,13 @@ void MYSQL_BIN_LOG::cleanup()
inited= 0;
mysql_mutex_lock(&LOCK_log);
- close(LOG_CLOSE_INDEX|LOG_CLOSE_STOP_EVENT);
+ if (opt_binlog_engine_hton)
+ {
+ if (!is_relay_log)
+ close_engine();
+ }
+ else
+ close(LOG_CLOSE_INDEX|LOG_CLOSE_STOP_EVENT);
mysql_mutex_unlock(&LOCK_log);
delete description_event_for_queue;
delete description_event_for_exec;
@@ -3655,12 +4294,14 @@ void MYSQL_BIN_LOG::cleanup()
mysql_mutex_destroy(&LOCK_log);
mysql_mutex_destroy(&LOCK_index);
+ mysql_mutex_destroy(&LOCK_binlog_use);
mysql_mutex_destroy(&LOCK_xid_list);
mysql_mutex_destroy(&LOCK_binlog_background_thread);
mysql_mutex_destroy(&LOCK_binlog_end_pos);
mysql_cond_destroy(&COND_relay_log_updated);
mysql_cond_destroy(&COND_bin_log_updated);
mysql_cond_destroy(&COND_queue_busy);
+ mysql_cond_destroy(&COND_binlog_use);
mysql_cond_destroy(&COND_xid_list);
mysql_cond_destroy(&COND_binlog_background_thread);
mysql_cond_destroy(&COND_binlog_background_thread_end);
@@ -3682,11 +4323,14 @@ void MYSQL_BIN_LOG::init_pthread_objects()
Event_log::init_pthread_objects();
mysql_mutex_init(m_key_LOCK_index, &LOCK_index, MY_MUTEX_INIT_SLOW);
mysql_mutex_setflags(&LOCK_index, MYF_NO_DEADLOCK_DETECTION);
+ mysql_mutex_init(key_BINLOG_LOCK_binlog_use, &LOCK_binlog_use,
+ MY_MUTEX_INIT_SLOW);
mysql_mutex_init(key_BINLOG_LOCK_xid_list,
&LOCK_xid_list, MY_MUTEX_INIT_FAST);
mysql_cond_init(m_key_relay_log_update, &COND_relay_log_updated, 0);
mysql_cond_init(m_key_bin_log_update, &COND_bin_log_updated, 0);
mysql_cond_init(m_key_COND_queue_busy, &COND_queue_busy, 0);
+ mysql_cond_init(key_BINLOG_COND_binlog_use, &COND_binlog_use, 0);
mysql_cond_init(key_BINLOG_COND_xid_list, &COND_xid_list, 0);
mysql_mutex_init(key_BINLOG_LOCK_binlog_background_thread,
@@ -3858,6 +4502,7 @@ bool MYSQL_BIN_LOG::open(const char *log_name,
xid_count_per_binlog *new_xid_list_entry= NULL, *b;
DBUG_ENTER("MYSQL_BIN_LOG::open");
+ DBUG_ASSERT(is_relay_log || !opt_binlog_engine_hton);
mysql_mutex_assert_owner(&LOCK_log);
if (!is_relay_log)
@@ -4238,6 +4883,51 @@ bool MYSQL_BIN_LOG::open(const char *log_name,
}
+/*
+ Open the binlog implemented in a storage engine (--binlog-storage-engine). */
+bool
+MYSQL_BIN_LOG::open_engine(handlerton *hton, ulong max_size, const char *dir)
+{
+ binlog_max_spill_size= std::min((size_t)(max_size / 2), BINLOG_SPILL_MAX);
+
+ log_state= LOG_OPENED;
+ {
+ /*
+ Write a format description event to the binlog at server restart.
+ With --binlog-storage-engine, we do not write a format description event
+ at the start of every binlog file (indeed, the "start of binlog file" is
+ mostly a meaningless concept). But we want to inform the slaves about
+ master server restarts, and sending a format description event (with the
+ `created' flag set) is a backwards-compatible way of doing so.
+ */
+ Format_description_log_event s(BINLOG_VERSION, NULL,
+ BINLOG_CHECKSUM_ALG_OFF);
+ s.dont_set_created= false;
+ /* Set stmt cache so end_log_pos gets written as 0. */
+ s.cache_type= Log_event::EVENT_STMT_CACHE;
+
+ IO_CACHE cache;
+ init_io_cache(&cache, (File)-1, binlog_cache_size, WRITE_CACHE, 0, false,
+ MYF(MY_DONT_CHECK_FILESIZE));
+ handler_binlog_event_group_info engine_context=
+ { 0, 0, nullptr, nullptr, nullptr, 0, 0, 0 };
+ write_event(&s, BINLOG_CHECKSUM_ALG_OFF, 0, &cache);
+ mysql_mutex_lock(&LOCK_commit_ordered);
+ (*opt_binlog_engine_hton->binlog_write_direct_ordered) (&cache,
+ &engine_context,
+ nullptr);
+ mysql_mutex_unlock(&LOCK_commit_ordered);
+ (*opt_binlog_engine_hton->binlog_write_direct) (&cache,
+ &engine_context,
+ nullptr);
+ (*opt_binlog_engine_hton->binlog_oob_free)(engine_context.engine_ptr);
+ end_io_cache(&cache);
+ }
+
+ return false;
+}
+
+
int MYSQL_BIN_LOG::get_current_log(LOG_INFO* linfo)
{
mysql_mutex_lock(&LOCK_log);
@@ -4479,6 +5169,67 @@ int MYSQL_BIN_LOG::find_next_log(LOG_INFO* linfo, bool need_lock)
}
+/*
+ Start reading the binlog, eg. a slave dump thread.
+ Wait for any already running RESET MASTER to complete.
+ Then increment the binlog use count.
+ Must be paired with a call to end_use_binlog() when use of the binlog is
+ complete by the reader, unless start_use_binlog() returns true/error.
+
+ Returns:
+ false Successfully marked binlog in use.
+ true Error (wait was terminated by kill).
+*/
+bool
+MYSQL_BIN_LOG::start_use_binlog(THD *thd)
+{
+ PSI_stage_info old_stage;
+ bool killed_err= false;
+
+ if (unlikely(is_relay_log))
+ {
+ DBUG_ASSERT(FALSE);
+ return false;
+ }
+
+ mysql_mutex_lock(&LOCK_binlog_use);
+ thd->ENTER_COND(&COND_binlog_use, &LOCK_binlog_use,
+ &stage_waiting_for_reset_master, &old_stage);
+ while (binlog_use_count < 0 && !thd->check_killed(1))
+ mysql_cond_wait(&COND_binlog_use, &LOCK_binlog_use);
+ if (binlog_use_count < 0)
+ killed_err= true;
+ else
+ ++binlog_use_count;
+ thd->EXIT_COND(&old_stage);
+
+ return killed_err;
+}
+
+
+/*
+ Stop reading the binlog, eg. a slave dump thread.
+ Must be called after a successful start_use_binlog(), once the use of the
+ binlog has completed.
+*/
+void
+MYSQL_BIN_LOG::end_use_binlog(THD *thd)
+{
+ if (unlikely(is_relay_log))
+ {
+ DBUG_ASSERT(FALSE);
+ return;
+ }
+
+ mysql_mutex_lock(&LOCK_binlog_use);
+ if (likely(binlog_use_count > 0))
+ --binlog_use_count;
+ else
+ DBUG_ASSERT(FALSE);
+ mysql_mutex_unlock(&LOCK_binlog_use);
+}
+
+
/**
Delete all logs referred to in the index file.
@@ -4516,6 +5267,36 @@ bool MYSQL_BIN_LOG::reset_logs(THD *thd, bool create_new_log,
DBUG_RETURN(1);
}
+ /*
+ Give an error if any slave dump threads are running, and prevent any
+ new binlog readers (or another RESET MASTER) from running concurrently.
+ */
+ mysql_mutex_lock(&LOCK_binlog_use);
+ if (binlog_use_count)
+ {
+ my_error(ER_BINLOG_IN_USE, MYF(0));
+ mysql_mutex_unlock(&LOCK_binlog_use);
+ DBUG_RETURN(1);
+ }
+ binlog_use_count= -1;
+ mysql_mutex_unlock(&LOCK_binlog_use);
+
+ if (opt_binlog_engine_hton)
+ {
+ if (next_log_number)
+ {
+ my_error(ER_NOT_AVAILABLE_WITH_ENGINE_BINLOG, MYF(0),
+ "RESET MASTER TO");
+ error= true;
+ }
+ else
+ {
+ DBUG_ASSERT(create_new_log);
+ error= reset_engine_binlogs(thd, init_state, init_state_len);
+ }
+ goto exit_engine_binlog;
+ }
+
/*
Mark that a RESET MASTER is in progress.
This ensures that a binlog checkpoint will not try to write binlog
@@ -4774,10 +5555,48 @@ bool MYSQL_BIN_LOG::reset_logs(THD *thd, bool create_new_log,
mysql_mutex_unlock(&LOCK_index);
mysql_mutex_unlock(&LOCK_log);
+
+ if (!is_relay_log)
+ {
+exit_engine_binlog:
+ mysql_mutex_lock(&LOCK_binlog_use);
+ DBUG_ASSERT(binlog_use_count == -1);
+ binlog_use_count= 0;
+ mysql_cond_signal(&COND_binlog_use);
+ mysql_mutex_unlock(&LOCK_binlog_use);
+ }
+
DBUG_RETURN(error);
}
+bool
+MYSQL_BIN_LOG::reset_engine_binlogs(THD *thd, rpl_gtid *init_state,
+ uint32 init_state_len)
+{
+ bool err;
+ DBUG_ASSERT(!is_relay_log);
+
+ mysql_mutex_lock(&LOCK_log);
+ mysql_mutex_lock(&LOCK_index);
+ mysql_mutex_lock(&LOCK_after_binlog_sync);
+ mysql_mutex_unlock(&LOCK_log);
+ mysql_mutex_lock(&LOCK_commit_ordered);
+ mysql_mutex_unlock(&LOCK_after_binlog_sync);
+
+ if (init_state)
+ rpl_global_gtid_binlog_state.load(init_state, init_state_len);
+ else
+ rpl_global_gtid_binlog_state.reset();
+ err= (*opt_binlog_engine_hton->reset_binlogs)();
+
+ mysql_mutex_unlock(&LOCK_commit_ordered);
+ mysql_mutex_unlock(&LOCK_index);
+
+ return err;
+}
+
+
void MYSQL_BIN_LOG::wait_for_last_checkpoint_event()
{
mysql_mutex_lock(&LOCK_xid_list);
@@ -5272,8 +6091,8 @@ int MYSQL_BIN_LOG::purge_index_entry(THD *thd, ulonglong *reclaimed_space,
}
else
{
- if (unlikely((error= find_log_pos(&check_log_info,
- log_info.log_file_name, need_mutex))))
+ if (likely((error= find_log_pos(&check_log_info,
+ log_info.log_file_name, need_mutex))))
{
if (error != LOG_INFO_EOF)
{
@@ -5546,6 +6365,40 @@ int MYSQL_BIN_LOG::real_purge_logs_by_size(ulonglong binlog_pos)
DBUG_RETURN(error);
}
+
+void
+MYSQL_BIN_LOG::engine_purge_logs_by_size(ulonglong max_total_size)
+{
+ DBUG_ASSERT(opt_binlog_engine_hton);
+ if (!is_open())
+ return;
+
+ handler_binlog_purge_info purge_info;
+ auto p= engine_binlog_in_use();
+ purge_info.limit_file_no= p.first;
+ uint num_dump_threads= p.second;
+ if (num_dump_threads < slave_connections_needed_for_purge)
+ {
+ purge_info.limit_file_no= 0;
+ purge_info.nonpurge_reason= "less than "
+ "'slave_connections_needed_for_purge' slaves have processed it";
+ }
+ else
+ purge_info.nonpurge_reason= nullptr;
+ purge_info.nonpurge_filename[0]= '\0';
+ purge_info.purge_by_date= false;
+ purge_info.limit_date= my_time(0);
+ purge_info.purge_by_size= true;
+ purge_info.limit_size= max_total_size;
+ purge_info.purge_by_name= false;
+ purge_info.limit_name= nullptr;
+ int res= (*opt_binlog_engine_hton->binlog_purge)(&purge_info);
+ if (res && purge_info.nonpurge_reason)
+ give_purge_note(purge_info.nonpurge_reason,
+ purge_info.nonpurge_filename, true);
+}
+
+
/*
@param log_file_name_arg Name of log file to check
@param interactive True if called by a PURGE BINLOG command.
@@ -5576,6 +6429,7 @@ MYSQL_BIN_LOG::can_purge_log(const char *log_file_name_arg,
int res;
const char *reason;
+ DBUG_ASSERT(is_relay_log || !opt_binlog_engine_hton);
if (is_active(log_file_name_arg) ||
(!is_relay_log && waiting_for_slave_to_change_binlog &&
purge_sending_new_binlog_file == sending_new_binlog_file &&
@@ -5639,21 +6493,37 @@ MYSQL_BIN_LOG::can_purge_log(const char *log_file_name_arg,
/* purge_warning_given is reset after next sucessful purge */
purge_warning_given= 1;
- if (interactive)
- {
+ give_purge_note(reason, log_file_name_arg, interactive);
+ }
+ return 0;
+}
+#endif /* HAVE_REPLICATION */
+
+
+void
+give_purge_note(const char *reason, const char *file_name, bool interactive)
+{
+ if (interactive)
+ {
+ if (file_name && file_name[0])
my_printf_error(ER_BINLOG_PURGE_PROHIBITED,
"Binary log '%s' is not purged because %s",
- MYF(ME_NOTE), log_file_name_arg, reason);
- }
+ MYF(ME_NOTE), file_name, reason);
else
- {
+ my_printf_error(ER_BINLOG_PURGE_PROHIBITED,
+ "Binary log purge is prevented because %s",
+ MYF(ME_NOTE), reason);
+ }
+ else
+ {
+ if (file_name && file_name[0])
sql_print_information("Binary log '%s' is not purged because %s",
- log_file_name_arg, reason);
- }
+ file_name, reason);
+ else
+ sql_print_information("Binary log purge is prevented because %s",
+ reason);
}
- return 0;
}
-#endif /* HAVE_REPLICATION */
/**
@@ -5673,6 +6543,7 @@ int MYSQL_BIN_LOG::count_binlog_space()
LOG_INFO log_info;
DBUG_ENTER("count_binlog_space");
+ DBUG_ASSERT(!opt_binlog_engine_hton);
binlog_space_total = 0;
if ((error= find_log_pos(&log_info, NullS, false /*need_lock_index=false*/)))
goto done;
@@ -6131,6 +7002,7 @@ bool MYSQL_BIN_LOG::write_event_buffer(uchar* buf, uint len)
bool MYSQL_BIN_LOG::flush_and_sync(bool *synced)
{
+ DBUG_ASSERT(is_relay_log || !opt_binlog_engine_hton);
int err=0, fd=log_file.file;
if (synced)
*synced= 0;
@@ -6299,6 +7171,181 @@ bool stmt_has_updated_non_trans_table(const THD* thd)
return (thd->transaction->stmt.modified_non_trans_table);
}
+
+static int
+binlog_spill_to_engine(struct st_io_cache *cache, const uchar *data, size_t len)
+{
+ /*
+ Tricky: The mysys IO_CACHE write function can be called either from
+ my_b_flush_io_cache(), where it must write everything it was asked to; or
+ from _my_b_write(), where it needs only write as much as is efficient (eg.
+ an integer multiple of some block size), and any remainder (which must be
+ < cache size) will be put in the cache.
+
+ The two cases are distinguished on whether the passed-in data pointer is
+ equal to cache->write_buffer or not.
+
+ We want each oob record to be the full size, so write only integer
+ multiples of the cache size in the latter case.
+ */
+ if (data != cache->write_buffer)
+ {
+ len-= (len % cache->buffer_length);
+ if (!len)
+ return 0;
+ }
+
+ binlog_cache_mngr *mngr= (binlog_cache_mngr *)cache->append_read_pos;
+ binlog_cache_data *cache_data;
+ bool using_trx_cache;
+ if (unlikely(cache==&mngr->stmt_cache.cache_log))
+ {
+ cache_data= &mngr->stmt_cache;
+ using_trx_cache= false;
+ }
+ else
+ {
+ cache_data= &mngr->trx_cache;
+ using_trx_cache= true;
+ }
+ void **engine_ptr= &cache_data->engine_binlog_info.engine_ptr;
+ mysql_mutex_assert_not_owner(&LOCK_commit_ordered);
+
+ size_t max_len= std::min(binlog_max_spill_size, (size_t)binlog_cache_size);
+ my_off_t spill_end= cache->pos_in_file + len;
+ size_t sofar= 0;
+ void **stmt_start_ptr= nullptr;
+ void **savepoint_ptr= nullptr;
+
+ /*
+ If there are any pending savepoints (or a start-of-statement point) in the
+ cache data that we're now spilling to the engine binlog, set an engine
+ savepoint for each of them so that we can roll back such spilled data,
+ if required.
+ */
+ if (data == cache->write_buffer && using_trx_cache)
+ {
+ my_off_t spill_start= cache->pos_in_file;
+ my_off_t stmt_pos= mngr->trx_cache.get_prev_position();
+ bool do_stmt_pos= stmt_pos != MY_OFF_T_UNDEF &&
+ stmt_pos >= spill_start && stmt_pos < spill_end;
+ binlog_savepoint_info *sp= mngr->cache_savepoint_list;
+ for (;;)
+ {
+ /*
+ Find the next spill point.
+ It maybe be the next savepoint in the list, it may be the saved
+ start-of-statement point, or (if they coincide) it may be both.
+ It may also be the next max_len boundary, if len > max_len.
+ */
+ my_off_t spill_pos;
+ void **next_stmt_start_ptr;
+ void **next_savepoint_ptr;
+ binlog_savepoint_info *next_sp;
+ if (do_stmt_pos && sp && stmt_pos == sp->cache_offset)
+ {
+ /* Double savepoint and start-of-statement point. */
+ spill_pos= stmt_pos;
+ next_stmt_start_ptr= &mngr->stmt_start_engine_ptr;
+ next_savepoint_ptr= &sp->engine_ptr;
+ next_sp= sp->next;
+ }
+ else if (do_stmt_pos && (!sp || stmt_pos < sp->cache_offset))
+ {
+ /* Spill the start-of-statement point next. */
+ spill_pos= stmt_pos;
+ next_stmt_start_ptr= &mngr->stmt_start_engine_ptr;
+ next_savepoint_ptr= nullptr;
+ next_sp= sp;
+ }
+ else if (sp)
+ {
+ /* Spill the next savepoint now. */
+ spill_pos= sp->cache_offset;
+ next_stmt_start_ptr= nullptr;
+ next_savepoint_ptr= &sp->engine_ptr;
+ next_sp= sp->next;
+ }
+ else
+ break;
+ DBUG_ASSERT(spill_pos >= spill_start);
+ if (spill_pos >= spill_end)
+ break;
+ DBUG_ASSERT(spill_start + sofar <= spill_pos);
+ size_t part_len= spill_pos - (spill_start + sofar);
+ if (part_len > 0 || stmt_start_ptr || savepoint_ptr)
+ {
+ if (part_len > max_len)
+ {
+ /* Split this spill into smaller pieces. */
+ part_len= max_len;
+ next_stmt_start_ptr= nullptr;
+ next_savepoint_ptr= nullptr;
+ next_sp= sp;
+ }
+
+ mysql_mutex_lock(&LOCK_commit_ordered);
+ int res= (*opt_binlog_engine_hton->binlog_oob_data_ordered)
+ (mngr->thd, data + sofar, part_len,
+ engine_ptr, stmt_start_ptr, savepoint_ptr);
+ mysql_mutex_unlock(&LOCK_commit_ordered);
+ if (likely(!res))
+ res= (*opt_binlog_engine_hton->binlog_oob_data)
+ (mngr->thd, data + sofar, part_len, engine_ptr);
+ if (unlikely(res))
+ return res;
+ sofar+= part_len;
+ }
+
+ stmt_start_ptr= next_stmt_start_ptr;
+ savepoint_ptr= next_savepoint_ptr;
+ sp= next_sp;
+ if (stmt_start_ptr)
+ do_stmt_pos= false; /* Start-of-statement gets done now */
+ }
+ mngr->cache_savepoint_list= sp; /* Remove any points spilled from cache. */
+ if (likely(sp == nullptr))
+ mngr->cache_savepoint_next_ptr= &mngr->cache_savepoint_list;
+ /*
+ We currently always spill the entire cache contents, which should mean
+ that at this point the remaining list of pending savepoints in the cache
+ is always empty - or possibly a savepoint at the current EOF.
+ Let's assert that this is so. However, if we ever want to partially
+ spill the cache and thus have remaining entries at this point, that is
+ fine, it is supported by the code and then this assertion can just be
+ removed.
+ */
+ DBUG_ASSERT(sp == nullptr || sp->cache_offset == my_b_tell(cache));
+ }
+
+ DBUG_ASSERT(sofar < len);
+ do
+ {
+ size_t part_len= len - sofar;
+ if (part_len > max_len)
+ part_len= max_len;
+ mysql_mutex_lock(&LOCK_commit_ordered);
+ int res= (*opt_binlog_engine_hton->binlog_oob_data_ordered)
+ (mngr->thd, data + sofar, part_len, engine_ptr,
+ stmt_start_ptr, savepoint_ptr);
+ mysql_mutex_unlock(&LOCK_commit_ordered);
+ if (likely(!res))
+ res= (*opt_binlog_engine_hton->binlog_oob_data)
+ (mngr->thd, data + sofar, part_len, engine_ptr);
+ if (unlikely(res))
+ return res;
+ stmt_start_ptr= nullptr;
+ savepoint_ptr= nullptr;
+ sofar+= part_len;
+ } while (sofar < len);
+
+ cache_data->engine_binlog_info.out_of_band_offset+= len;
+ cache->pos_in_file= spill_end;
+
+ return false;
+}
+
+
/*
These functions are placed in this file since they need access to
binlog_hton, which has internal linkage.
@@ -6309,11 +7356,50 @@ static binlog_cache_mngr *binlog_setup_cache_mngr(THD *thd)
auto *cache_mngr= (binlog_cache_mngr*) my_malloc(key_memory_binlog_cache_mngr,
sizeof(binlog_cache_mngr),
MYF(MY_ZEROFILL));
- if (!cache_mngr ||
- open_cached_file(&cache_mngr->stmt_cache.cache_log, mysql_tmpdir,
- LOG_PREFIX, (size_t)binlog_stmt_cache_size, MYF(MY_WME)) ||
- open_cached_file(&cache_mngr->trx_cache.cache_log, mysql_tmpdir,
- LOG_PREFIX, (size_t)binlog_cache_size, MYF(MY_WME)))
+ if (!cache_mngr)
+ return NULL;
+ IO_CACHE *stmt_cache= &cache_mngr->stmt_cache.cache_log;
+ my_bool res;
+ if (opt_binlog_engine_hton)
+ {
+ /*
+ With binlog implementation in engine, we do not need to spill large
+ transactions to temporary file, we will binlog data out-of-band spread
+ through the binlog as the transaction runs. Setting the file to INT_MIN
+ makes IO_CACHE not attempt to create the temporary file.
+ */
+ res= init_io_cache(stmt_cache, (File)INT_MIN,
+ (size_t)binlog_stmt_cache_size,
+ WRITE_CACHE, 0L, 0, MYF(MY_WME | MY_NABP));
+ /*
+ Use a custom write_function to spill to the engine-implemented binlog.
+ And re-use the IO_CACHE::append_read_pos as a handle for our
+ write_function; it is unused when the cache is not SEQ_READ_APPEND.
+ */
+ stmt_cache->write_function= binlog_spill_to_engine;
+ stmt_cache->append_read_pos= (uchar *)cache_mngr;
+ }
+ else
+ res= open_cached_file(&cache_mngr->stmt_cache.cache_log, mysql_tmpdir,
+ LOG_PREFIX, (size_t)binlog_stmt_cache_size,
+ MYF(MY_WME));
+ if (unlikely(res))
+ {
+ my_free(cache_mngr);
+ return NULL;
+ }
+ IO_CACHE *trx_cache= &cache_mngr->trx_cache.cache_log;
+ if (opt_binlog_engine_hton)
+ {
+ res= init_io_cache(trx_cache, (File)INT_MIN, (size_t)binlog_cache_size,
+ WRITE_CACHE, 0L, 0, MYF(MY_WME | MY_NABP));
+ trx_cache->write_function= binlog_spill_to_engine;
+ trx_cache->append_read_pos= (uchar *)cache_mngr;
+ }
+ else
+ res= open_cached_file(trx_cache, mysql_tmpdir, LOG_PREFIX,
+ (size_t)binlog_cache_size, MYF(MY_WME));
+ if (unlikely(res))
{
my_free(cache_mngr);
return NULL;
@@ -6328,7 +7414,7 @@ static binlog_cache_mngr *binlog_setup_cache_mngr(THD *thd)
bool precompute_checksums=
!WSREP_NNULL(thd) && !encrypt_binlog && !opt_binlog_legacy_event_pos;
cache_mngr= new (cache_mngr)
- binlog_cache_mngr(max_binlog_stmt_cache_size,
+ binlog_cache_mngr(thd, max_binlog_stmt_cache_size,
max_binlog_cache_size,
&binlog_stmt_cache_use,
&binlog_stmt_cache_disk_use,
@@ -6473,7 +7559,9 @@ THD::binlog_start_trans_and_stmt()
server_id= wsrep_gtid_server.server_id;
}
Gtid_log_event gtid_event(this, seqno, domain_id, true,
- LOG_EVENT_SUPPRESS_USE_F, true, 0);
+ Log_event::EVENT_NO_CACHE,
+ LOG_EVENT_SUPPRESS_USE_F, true, 0,
+ false, false);
gtid_event.server_id= server_id;
writer.write(>id_event);
wsrep_write_cache_buf(&tmp_io_cache, &buf, &len);
@@ -6501,17 +7589,9 @@ THD::binlog_start_trans_and_stmt()
}
void THD::binlog_set_stmt_begin() {
- binlog_cache_mngr *cache_mngr= binlog_get_cache_mngr();
-
- /*
- The call to binlog_trans_log_savepos() might create the cache_mngr
- structure, if it didn't exist before, so we save the position
- into an auto variable and then write it into the transaction
- data for the binary log (i.e., cache_mngr).
- */
my_off_t pos= 0;
- binlog_trans_log_savepos(this, &pos);
- cache_mngr= binlog_get_cache_mngr();
+ binlog_cache_mngr *cache_mngr= binlog_setup_trx_data();
+ binlog_trans_log_savepos(this, cache_mngr, &pos);
cache_mngr->trx_cache.set_prev_position(pos);
}
@@ -6525,8 +7605,18 @@ binlog_start_consistent_snapshot(handlerton *hton, THD *thd)
/* Server layer calls us with LOCK_commit_ordered locked, so this is safe. */
mysql_mutex_assert_owner(&LOCK_commit_ordered);
- strmake_buf(cache_mngr->last_commit_pos_file, mysql_bin_log.last_commit_pos_file);
- cache_mngr->last_commit_pos_offset= mysql_bin_log.last_commit_pos_offset;
+ if (opt_binlog_engine_hton)
+ {
+ (*opt_binlog_engine_hton->binlog_status)
+ (&cache_mngr->last_commit_pos_file.engine_file_no,
+ &cache_mngr->last_commit_pos_offset);
+ }
+ else
+ {
+ strmake_buf(cache_mngr->last_commit_pos_file.legacy_name,
+ mysql_bin_log.last_commit_pos_file);
+ cache_mngr->last_commit_pos_offset= mysql_bin_log.last_commit_pos_offset;
+ }
trans_register_ha(thd, TRUE, binlog_hton, 0);
@@ -6997,9 +8087,9 @@ Event_log::prepare_pending_rows_event(THD *thd, TABLE* table,
/* Generate a new global transaction ID, and write it to the binlog */
bool
-MYSQL_BIN_LOG::write_gtid_event(THD *thd, bool standalone,
- bool is_transactional, uint64 commit_id,
- bool has_xid, bool is_ro_1pc)
+MYSQL_BIN_LOG::write_gtid_event(THD *thd, binlog_cache_data *cache_data,
+ bool standalone, bool is_transactional,
+ uint64 commit_id, bool has_xid, bool is_ro_1pc)
{
rpl_gtid gtid;
uint32 domain_id;
@@ -7052,9 +8142,44 @@ MYSQL_BIN_LOG::write_gtid_event(THD *thd, bool standalone,
if (thd->get_binlog_flags_for_alter() & Gtid_log_event::FL_START_ALTER_E1)
thd->set_binlog_start_alter_seq_no(gtid.seq_no);
- Gtid_log_event gtid_event(thd, seq_no, domain_id, standalone,
+ Log_event::enum_event_cache_type cache_type;
+ IO_CACHE *dest;
+ if (cache_data)
+ {
+ cache_type= cache_data->trx_cache() ?
+ Log_event::EVENT_TRANSACTIONAL_CACHE : Log_event::EVENT_STMT_CACHE;
+ dest= &cache_data->cache_log;
+ }
+ else
+ {
+ cache_type= Log_event::EVENT_NO_CACHE;
+ dest= &log_file;
+ }
+ Gtid_log_event gtid_event(thd, seq_no, domain_id, standalone, cache_type,
LOG_EVENT_SUPPRESS_USE_F, is_transactional,
commit_id, has_xid, is_ro_1pc);
+ /*
+ Check that any binlogging during DDL recovery preserves the FL_DLL flag
+ on the GTID event.
+ */
+ DBUG_ASSERT((gtid_event.flags2 & Gtid_log_event::FL_DDL) ||
+ !is_in_ddl_recovery);
+
+ if (opt_binlog_engine_hton)
+ {
+ DBUG_ASSERT(cache_data != nullptr);
+ uint32_t avail= (uint32_t)(dest->write_end - dest->write_pos);
+ if (unlikely(avail < Gtid_log_event::max_size) &&
+ avail < gtid_event.get_size())
+ {
+ /*
+ The GTID event doesn't fit in the cache, so we have to spill the
+ contents as oob event data.
+ */
+ if (my_b_flush_io_cache(dest, 0))
+ DBUG_RETURN(true);
+ }
+ }
/* Write the event to the binary log. */
DBUG_ASSERT(this == &mysql_bin_log);
@@ -7067,7 +8192,7 @@ MYSQL_BIN_LOG::write_gtid_event(THD *thd, bool standalone,
}
#endif
- if (write_event(>id_event))
+ if (write_event(>id_event, cache_data, dest))
DBUG_RETURN(true);
status_var_add(thd->status_var.binlog_bytes_written, gtid_event.data_written);
@@ -7187,6 +8312,60 @@ MYSQL_BIN_LOG::get_most_recent_gtid_list(rpl_gtid **list, uint32 *size)
}
+bool
+load_global_binlog_state(rpl_binlog_state_base *state)
+{
+ mysql_mutex_lock(&rpl_global_gtid_binlog_state.LOCK_binlog_state);
+ bool err= state->load_nolock(&rpl_global_gtid_binlog_state);
+ mysql_mutex_unlock(&rpl_global_gtid_binlog_state.LOCK_binlog_state);
+ return err;
+}
+
+
+/*
+ Used to initialize the binlog GTID state after restart when using
+ --binlog-storage-engine. The engine passes in the GTID binlog state it has
+ restored, and optionally passes a binlog reader. The intention is that the
+ engine can restore a state corresponding to a slightly earlier point in the
+ binlog file, and then we will use the reader to read any extra GTID events
+ and compute the final restored binlog GTID state from that.
+*/
+bool
+binlog_recover_gtid_state(rpl_binlog_state_base *state,
+ handler_binlog_reader *reader)
+{
+ String packet;
+ Format_description_log_event fd_event(4);
+
+ if (reader)
+ {
+ for (;;)
+ {
+ packet.length(0);
+ int err= reader->read_log_event(&packet, 0, MAX_MAX_ALLOWED_PACKET);
+ if (err == LOG_READ_EOF)
+ break;
+ if (err)
+ return true;
+ Log_event_type event_type=
+ (Log_event_type)((uchar)packet[LOG_EVENT_OFFSET]);
+ if (event_type != GTID_EVENT)
+ continue;
+ rpl_gtid gtid;
+ uchar flags2;
+ if (Gtid_log_event::peek((uchar*) packet.ptr(), packet.length(),
+ BINLOG_CHECKSUM_ALG_OFF, >id.domain_id,
+ >id.server_id, >id.seq_no, &flags2,
+ &fd_event))
+ return true;
+ state->update_nolock(>id);
+ }
+ }
+ rpl_global_gtid_binlog_state.load_nolock(state);
+ return false;
+}
+
+
bool
MYSQL_BIN_LOG::append_state_pos(String *str)
{
@@ -7269,9 +8448,18 @@ bool MYSQL_BIN_LOG::write(Log_event *event_info, my_bool *with_annotate)
bool is_trans_cache= FALSE;
bool using_trans= event_info->use_trans_cache();
bool direct= event_info->use_direct_logging();
+ bool events_direct;
ulong UNINIT_VAR(prev_binlog_id);
+ uint64 UNINIT_VAR(commit_id);
+ const rpl_gtid *commit_gtid;
DBUG_ENTER("MYSQL_BIN_LOG::write(Log_event *)");
+ DBUG_EXECUTE_IF("rpl_parallel_delay_gtid_0_x_100_write", {
+ if (thd->rgi_slave &&
+ thd->rgi_slave->current_gtid.domain_id==0 &&
+ thd->rgi_slave->current_gtid.seq_no == 100)
+ my_sleep(10000);
+ });
/*
When binary logging is not enabled (--log-bin=0), wsrep-patch partially
enables it without opening the binlog file (MYSQL_BIN_LOG::open().
@@ -7356,35 +8544,18 @@ bool MYSQL_BIN_LOG::write(Log_event *event_info, my_bool *with_annotate)
DBUG_RETURN(0);
#endif /* HAVE_REPLICATION */
+ binlog_cache_mngr * cache_mngr= NULL;
IO_CACHE *file= NULL;
+ events_direct= direct;
if (direct)
{
- /* We come here only for incident events */
+ /* Write the event to the binlog immediately. */
int res;
- uint64 commit_id= 0;
- MDL_request mdl_request;
+
DBUG_PRINT("info", ("direct is set"));
DBUG_ASSERT(!thd->backup_commit_lock);
-
- MDL_REQUEST_INIT(&mdl_request, MDL_key::BACKUP, "", "", MDL_BACKUP_COMMIT,
- MDL_EXPLICIT);
- if (thd->mdl_context.acquire_lock(&mdl_request,
- thd->variables.lock_wait_timeout))
- DBUG_RETURN(1);
- thd->backup_commit_lock= &mdl_request;
-
- if ((res= thd->wait_for_prior_commit()))
- {
- if (mdl_request.ticket)
- thd->mdl_context.release_lock(mdl_request.ticket);
- thd->backup_commit_lock= 0;
- DBUG_RETURN(res);
- }
- file= &log_file;
- my_org_b_tell= my_b_tell(file);
- mysql_mutex_lock(&LOCK_log);
- prev_binlog_id= current_binlog_id;
+ commit_id= 0;
DBUG_EXECUTE_IF("binlog_force_commit_id",
{
const LEX_CSTRING commit_name= { STRING_WITH_LEN("commit_id") };
@@ -7395,17 +8566,53 @@ bool MYSQL_BIN_LOG::write(Log_event *event_info, my_bool *with_annotate)
commit_name.length);
commit_id= entry->val_int(&null_value);
});
- res= write_gtid_event(thd, true, using_trans, commit_id);
- if (mdl_request.ticket)
- thd->mdl_context.release_lock(mdl_request.ticket);
- thd->backup_commit_lock= 0;
- if (res)
- goto err;
+
+ if (opt_binlog_engine_hton)
+ {
+ events_direct= false;
+ if (!(cache_mngr= thd->binlog_setup_trx_data()))
+ DBUG_RETURN(1);
+ cache_data= cache_mngr->get_binlog_cache_data(false);
+ DBUG_ASSERT(cache_data->empty());
+ file= &cache_data->cache_log;
+ /* Set cache_type to ensure we don't get checksums for this event */
+ event_info->cache_type= Log_event::EVENT_STMT_CACHE;
+ }
+ else
+ {
+ MDL_request mdl_request;
+
+ MDL_REQUEST_INIT(&mdl_request, MDL_key::BACKUP, "", "", MDL_BACKUP_COMMIT,
+ MDL_EXPLICIT);
+ if (thd->mdl_context.acquire_lock(&mdl_request,
+ thd->variables.lock_wait_timeout))
+ DBUG_RETURN(1);
+ thd->backup_commit_lock= &mdl_request;
+
+ if ((res= thd->wait_for_prior_commit()))
+ {
+ if (mdl_request.ticket)
+ thd->mdl_context.release_lock(mdl_request.ticket);
+ thd->backup_commit_lock= 0;
+ DBUG_RETURN(res);
+ }
+ file= &log_file;
+ my_org_b_tell= my_b_tell(file);
+ mysql_mutex_lock(&LOCK_log);
+ prev_binlog_id= current_binlog_id;
+ res= write_gtid_event(thd, nullptr, true, using_trans, commit_id,
+ false, false);
+ if (mdl_request.ticket)
+ thd->mdl_context.release_lock(mdl_request.ticket);
+ thd->backup_commit_lock= 0;
+ if (res)
+ goto err;
+ }
}
else
{
- binlog_cache_mngr *const cache_mngr= thd->binlog_setup_trx_data();
- if (!cache_mngr)
+ /* Write the event to the stmt or trx cache, and binlog it later. */
+ if (!(cache_mngr= thd->binlog_setup_trx_data()))
goto err;
is_trans_cache= use_trans_cache(thd, using_trans);
@@ -7430,7 +8637,7 @@ bool MYSQL_BIN_LOG::write(Log_event *event_info, my_bool *with_annotate)
if (with_annotate && *with_annotate)
{
DBUG_ASSERT(event_info->get_type_code() == TABLE_MAP_EVENT);
- Annotate_rows_log_event anno(thd, using_trans, direct);
+ Annotate_rows_log_event anno(thd, using_trans, events_direct);
/* Annotate event should be written not more than once */
*with_annotate= 0;
if (write_event(&anno, cache_data, file))
@@ -7444,7 +8651,7 @@ bool MYSQL_BIN_LOG::write(Log_event *event_info, my_bool *with_annotate)
{
Intvar_log_event e(thd,(uchar) LAST_INSERT_ID_EVENT,
thd->first_successful_insert_id_in_prev_stmt_for_binlog,
- using_trans, direct);
+ using_trans, events_direct);
if (write_event(&e, cache_data, file))
goto err;
}
@@ -7455,14 +8662,14 @@ bool MYSQL_BIN_LOG::write(Log_event *event_info, my_bool *with_annotate)
nb_elements()));
Intvar_log_event e(thd, (uchar) INSERT_ID_EVENT,
thd->auto_inc_intervals_in_cur_stmt_for_binlog.
- minimum(), using_trans, direct);
+ minimum(), using_trans, events_direct);
if (write_event(&e, cache_data, file))
goto err;
}
if (thd->used & THD::RAND_USED)
{
Rand_log_event e(thd,thd->rand_saved_seed1,thd->rand_saved_seed2,
- using_trans, direct);
+ using_trans, events_direct);
if (write_event(&e, cache_data, file))
goto err;
}
@@ -7480,7 +8687,7 @@ bool MYSQL_BIN_LOG::write(Log_event *event_info, my_bool *with_annotate)
user_var_event->th->user_var_log_event_data_type(
user_var_event->charset_number),
using_trans,
- direct);
+ events_direct);
if (write_event(&e, cache_data, file))
goto err;
}
@@ -7499,79 +8706,179 @@ bool MYSQL_BIN_LOG::write(Log_event *event_info, my_bool *with_annotate)
err:
if (direct)
{
- my_off_t offset= my_b_tell(file);
- bool check_purge= false;
DBUG_ASSERT(!is_relay_log);
-
- if (likely(!error))
+ if (opt_binlog_engine_hton)
{
- bool synced;
+ handler_binlog_event_group_info *engine_context=
+ &cache_data->engine_binlog_info;
+ engine_context->gtid_offset= my_b_tell(file);
+ my_off_t binlog_total_bytes;
+ MDL_request mdl_request;
+ int res;
+
+ if (engine_context->out_of_band_offset)
+ {
+ /*
+ If we spilled part of the event data as oob, then we have to spill
+ all of it.
+ */
+ if (my_b_flush_io_cache(file, 0))
+ goto engine_fail;
+ }
- update_gtid_index((uint32)offset, thd->get_last_commit_gtid());
+ MDL_REQUEST_INIT(&mdl_request, MDL_key::BACKUP, "", "", MDL_BACKUP_COMMIT,
+ MDL_EXPLICIT);
+ if (thd->mdl_context.acquire_lock(&mdl_request,
+ thd->variables.lock_wait_timeout))
+ goto engine_fail;
+ thd->backup_commit_lock= &mdl_request;
- if ((error= flush_and_sync(&synced)))
+ if (thd->wait_for_prior_commit())
{
+ if (mdl_request.ticket)
+ thd->mdl_context.release_lock(mdl_request.ticket);
+ thd->backup_commit_lock= 0;
+ goto engine_fail;
}
- else
+#ifdef ENABLED_DEBUG_SYNC
+ DBUG_EXECUTE_IF("pause_before_write_gtid_0_x_100", {
+ if (thd->rgi_slave &&
+ thd->rgi_slave->current_gtid.domain_id == 0 &&
+ thd->rgi_slave->current_gtid.seq_no == 100) {
+ debug_sync_set_action(thd,
+ STRING_WITH_LEN("now SIGNAL before_write_gtid_event "
+ "WAIT_FOR cont"));
+ }
+ });
+#endif
+ mysql_mutex_lock(&LOCK_log);
+ res= write_gtid_event(thd, cache_data, true, using_trans, commit_id,
+ false, false);
+ if (mdl_request.ticket)
+ thd->mdl_context.release_lock(mdl_request.ticket);
+ thd->backup_commit_lock= 0;
+ if (res)
{
- mysql_mutex_assert_not_owner(&LOCK_prepare_ordered);
- mysql_mutex_assert_owner(&LOCK_log);
- mysql_mutex_assert_not_owner(&LOCK_after_binlog_sync);
- mysql_mutex_assert_not_owner(&LOCK_commit_ordered);
-#ifdef HAVE_REPLICATION
- if (repl_semisync_master.report_binlog_update(thd, thd,
- log_file_name, offset))
+ mysql_mutex_unlock(&LOCK_log);
+ goto engine_fail;
+ }
+
+ binlog_total_bytes= my_b_bytes_in_cache(file);
+ /*
+ Engine-in-binlog does not support the after-sync method.
+ This is for consistency with the binlogging of transactions in the
+ engine, which commit atomically at the same time in binlog and engine.
+
+ In any case, for non-transactional event group (eg. DDL), the
+ after-sync and after-commit semisync methods are mostly the same; the
+ change has already become visible to other connections on the master
+ when it is binlogged.
+
+ ToDo: If semi-sync is enabled, obtain the binlog coords from the
+ engine to be waited for later at after-commit.
+ */
+ mysql_mutex_lock(&LOCK_after_binlog_sync);
+ mysql_mutex_unlock(&LOCK_log);
+ mysql_mutex_lock(&LOCK_commit_ordered);
+ mysql_mutex_unlock(&LOCK_after_binlog_sync);
+ commit_gtid= thd->get_last_commit_gtid();
+ if (unlikely((*opt_binlog_engine_hton->binlog_write_direct_ordered)
+ (file, engine_context, commit_gtid)))
+ {
+ mysql_mutex_unlock(&LOCK_commit_ordered);
+ goto engine_fail;
+ }
+ mysql_mutex_unlock(&LOCK_commit_ordered);
+ cache_mngr->last_commit_pos_file.engine_file_no=
+ engine_context->out_file_no;
+ cache_mngr->last_commit_pos_offset= engine_context->out_file_no;
+
+ if (unlikely((*opt_binlog_engine_hton->binlog_write_direct)
+ (file, engine_context, commit_gtid)))
+ goto engine_fail;
+ status_var_add(thd->status_var.binlog_bytes_written, binlog_total_bytes);
+
+ goto engine_ok;
+ engine_fail:
+ error= 1;
+ engine_ok:
+ cache_mngr->reset(true, false);
+ }
+ else
+ {
+ my_off_t offset= my_b_tell(file);
+ bool check_purge= false;
+
+ if (likely(!error))
+ {
+ bool synced;
+
+ update_gtid_index((uint32)offset, thd->get_last_commit_gtid());
+
+ if ((error= flush_and_sync(&synced)))
{
- sql_print_error("Failed to run 'after_flush' hooks");
- error= 1;
}
else
-#endif
{
- /*
- update binlog_end_pos so it can be read by dump thread
- note: must be _after_ the RUN_HOOK(after_flush) or else
- semi-sync might not have put the transaction into
- it's list before dump-thread tries to send it
- */
- update_binlog_end_pos(offset);
- if (unlikely((error= rotate(false, &check_purge))))
- check_purge= false;
+ mysql_mutex_assert_not_owner(&LOCK_prepare_ordered);
+ mysql_mutex_assert_owner(&LOCK_log);
+ mysql_mutex_assert_not_owner(&LOCK_after_binlog_sync);
+ mysql_mutex_assert_not_owner(&LOCK_commit_ordered);
+#ifdef HAVE_REPLICATION
+ if (repl_semisync_master.report_binlog_update(thd, thd,
+ log_file_name, offset))
+ {
+ sql_print_error("Failed to run 'after_flush' hooks");
+ error= 1;
+ }
+ else
+#endif
+ {
+ /*
+ update binlog_end_pos so it can be read by dump thread
+ note: must be _after_ the RUN_HOOK(after_flush) or else
+ semi-sync might not have put the transaction into
+ it's list before dump-thread tries to send it
+ */
+ update_binlog_end_pos(offset);
+ if (unlikely((error= rotate(false, &check_purge))))
+ check_purge= false;
+ }
}
}
- }
- status_var_add(thd->status_var.binlog_bytes_written,
- offset - my_org_b_tell);
+ status_var_add(thd->status_var.binlog_bytes_written,
+ offset - my_org_b_tell);
- mysql_mutex_lock(&LOCK_after_binlog_sync);
- mysql_mutex_unlock(&LOCK_log);
+ mysql_mutex_lock(&LOCK_after_binlog_sync);
+ mysql_mutex_unlock(&LOCK_log);
- DEBUG_SYNC(thd, "commit_after_release_LOCK_log");
+ DEBUG_SYNC(thd, "commit_after_release_LOCK_log");
- mysql_mutex_assert_not_owner(&LOCK_prepare_ordered);
- mysql_mutex_assert_not_owner(&LOCK_log);
- mysql_mutex_assert_owner(&LOCK_after_binlog_sync);
- mysql_mutex_assert_not_owner(&LOCK_commit_ordered);
+ mysql_mutex_assert_not_owner(&LOCK_prepare_ordered);
+ mysql_mutex_assert_not_owner(&LOCK_log);
+ mysql_mutex_assert_owner(&LOCK_after_binlog_sync);
+ mysql_mutex_assert_not_owner(&LOCK_commit_ordered);
#ifdef HAVE_REPLICATION
- if (repl_semisync_master.wait_after_sync(log_file_name, offset))
- {
- error=1;
- /* error is already printed inside hook */
- }
+ if (repl_semisync_master.wait_after_sync(log_file_name, offset))
+ {
+ error=1;
+ /* error is already printed inside hook */
+ }
#endif
- /*
- Take mutex to protect against a reader seeing partial writes of 64-bit
- offset on 32-bit CPUs.
- */
- mysql_mutex_lock(&LOCK_commit_ordered);
- mysql_mutex_unlock(&LOCK_after_binlog_sync);
- last_commit_pos_offset= offset;
- mysql_mutex_unlock(&LOCK_commit_ordered);
+ /*
+ Take mutex to protect against a reader seeing partial writes of 64-bit
+ offset on 32-bit CPUs.
+ */
+ mysql_mutex_lock(&LOCK_commit_ordered);
+ mysql_mutex_unlock(&LOCK_after_binlog_sync);
+ last_commit_pos_offset= offset;
+ mysql_mutex_unlock(&LOCK_commit_ordered);
- if (check_purge)
- checkpoint_and_purge(prev_binlog_id);
+ if (check_purge)
+ checkpoint_and_purge(prev_binlog_id);
+ }
}
if (unlikely(error))
@@ -7588,14 +8895,14 @@ bool MYSQL_BIN_LOG::write(Log_event *event_info, my_bool *with_annotate)
void
-MYSQL_BIN_LOG::update_gtid_index(uint32 offset, rpl_gtid gtid)
+MYSQL_BIN_LOG::update_gtid_index(uint32 offset, const rpl_gtid *gtid)
{
if (!unlikely(gtid_index))
return;
rpl_gtid *gtid_list;
uint32 gtid_count;
- int err= gtid_index->process_gtid_check_batch(offset, >id,
+ int err= gtid_index->process_gtid_check_batch(offset, gtid,
>id_list, >id_count);
if (err)
return;
@@ -7761,6 +9068,7 @@ int MYSQL_BIN_LOG::rotate(bool force_rotate, bool* check_purge)
int error= 0;
ulonglong binlog_pos;
DBUG_ENTER("MYSQL_BIN_LOG::rotate");
+ DBUG_ASSERT(!opt_binlog_engine_hton);
#ifdef WITH_WSREP
if (WSREP_ON && wsrep_to_isolation)
@@ -7948,6 +9256,7 @@ static int do_delete_gtid_domain(DYNAMIC_ARRAY *domain_drop_lex)
IO_CACHE cache;
const char* errmsg= NULL;
char errbuf[MYSQL_ERRMSG_SIZE]= {0};
+ rpl_binlog_state_base init_state;
if (!domain_drop_lex)
return 0; // still "effective" having empty domain sequence to delete
@@ -7968,8 +9277,16 @@ static int do_delete_gtid_domain(DYNAMIC_ARRAY *domain_drop_lex)
errmsg= "injected error";);
if (errmsg)
goto end;
+
+ init_state.init();
+ if (init_state.load_nolock(glev->list, glev->count))
+ {
+ my_error(ER_OUT_OF_RESOURCES, MYF(0));
+ rc= -1;
+ goto err;
+ }
errmsg= rpl_global_gtid_binlog_state.drop_domain(domain_drop_lex,
- glev, errbuf);
+ &init_state, errbuf);
end:
if (errmsg)
@@ -7984,6 +9301,7 @@ static int do_delete_gtid_domain(DYNAMIC_ARRAY *domain_drop_lex)
rc= 1;
}
}
+err:
delete glev;
return rc;
@@ -8043,6 +9361,119 @@ int MYSQL_BIN_LOG::rotate_and_purge(bool force_rotate,
DBUG_RETURN(error);
}
+
+/**
+ Remove a list of domains from the in-memory global binlog state, after
+ checking that deletion is safe. "Safe" in this context means that there
+ are no GTID present with the domain in any of the existing binlog files
+ (ie. the binlog files where that domain was used have all been purged).
+ This is checked by comparing the binlog state at the beginning of the
+ earliest current binlog file with the current binlog state.
+
+ @param domain_drop_lex gtid domain id sequence from lex.
+ Passed as a pointer to dynamic array must be not empty
+ unless pointer value NULL.
+ @retval zero on success
+ @retval > 0 ineffective call none from the *non* empty
+ gtid domain sequence is deleted
+ @retval < 0 on error
+*/
+static int
+binlog_engine_delete_gtid_domain(DYNAMIC_ARRAY *domain_drop_lex)
+{
+ int rc= 0;
+ const char* errmsg= NULL;
+ char errbuf[MYSQL_ERRMSG_SIZE]= {0};
+ rpl_binlog_state_base init_state;
+
+ if (!domain_drop_lex)
+ return 0; // still "effective" having empty domain sequence to delete
+
+ DBUG_ASSERT(domain_drop_lex->elements > 0);
+ DBUG_ASSERT(opt_binlog_engine_hton);
+ mysql_mutex_assert_owner(&LOCK_commit_ordered);
+
+ if (!opt_binlog_engine_hton->binlog_get_init_state)
+ {
+ my_error(ER_ENGINE_BINLOG_NO_DELETE_DOMAIN, MYF(0));
+ return -1;
+ }
+
+ init_state.init();
+ if ((*opt_binlog_engine_hton->binlog_get_init_state)(&init_state))
+ {
+ my_error(ER_BINLOG_CANNOT_READ_STATE, MYF(0));
+ return -1;
+ }
+ errmsg= rpl_global_gtid_binlog_state.drop_domain(domain_drop_lex,
+ &init_state, errbuf);
+ if (errmsg)
+ {
+ if (strlen(errmsg) > 0)
+ {
+ my_error(ER_BINLOG_CANT_DELETE_GTID_DOMAIN, MYF(0), errmsg);
+ rc= -1;
+ }
+ else
+ {
+ rc= 1;
+ }
+ }
+ return rc;
+}
+
+
+/* Implementation of FLUSH BINARY LOGS for binlog implemented in engine. */
+int
+MYSQL_BIN_LOG::flush_binlogs_engine(DYNAMIC_ARRAY *domain_drop_lex)
+{
+ int error= 0;
+ DBUG_ENTER("MYSQL_BIN_LOG::flush_binlogs_engine");
+
+ mysql_mutex_lock(&LOCK_log);
+ mysql_mutex_lock(&LOCK_after_binlog_sync);
+ mysql_mutex_lock(&LOCK_commit_ordered);
+ mysql_mutex_unlock(&LOCK_after_binlog_sync);
+ DBUG_EXECUTE_IF("flush_binlog_sleep_after_release_lock_log",
+ my_sleep(200000);
+ );
+
+ if ((error= binlog_engine_delete_gtid_domain(domain_drop_lex)))
+ {
+ if (error < 0)
+ error= 1;
+ else
+ {
+ /*
+ If the domain(s) specified were already deleted, then a warning was
+ sent (by rpl_binlog_state::drop_domain()), but the statement succeeds
+ anyway and the FLUSH to move to a new file is still done.
+
+ (This is inconsistent with the legacy behaviour, which succeeds the
+ statement with a warning but _skips_ the flush/binlog rotation. It
+ seems a more reasonable behaviour that a FLUSH BINARY LOGS statement
+ _either_ performs the FLUSH, _or_ fails with an error, so this is
+ what we do in the engine binlog implementation).
+ */
+ error= 0;
+ }
+ }
+ if (!error && (*opt_binlog_engine_hton->binlog_flush)())
+ error= 1;
+
+ /*
+ Hold on to both LOCK_log and LOCK_commit_ordered across the FLUSH.
+ The former protects allocation of new GTIDs, and the latter protects
+ writing to the engine binlog; this way we ensure a consistent GTID state
+ at the point just after the FLUSH.
+ */
+ mysql_mutex_unlock(&LOCK_log);
+ mysql_mutex_unlock(&LOCK_commit_ordered);
+
+ DBUG_RETURN(error);
+}
+
+
uint MYSQL_BIN_LOG::next_file_id()
{
uint res;
@@ -8265,6 +9696,7 @@ bool MYSQL_BIN_LOG::write_incident_already_locked(THD *thd)
Incident incident= INCIDENT_LOST_EVENTS;
Incident_log_event ev(thd, incident, &write_error_msg);
+ DBUG_ASSERT(!opt_binlog_engine_hton);
if (likely(is_open()))
{
error= write_event(&ev);
@@ -8283,6 +9715,9 @@ bool MYSQL_BIN_LOG::write_incident(THD *thd)
ulong prev_binlog_id;
DBUG_ENTER("MYSQL_BIN_LOG::write_incident");
+ if (opt_binlog_engine_hton)
+ DBUG_RETURN(0);
+
mysql_mutex_lock(&LOCK_log);
if (likely(is_open()))
{
@@ -8345,6 +9780,7 @@ write_binlog_checkpoint_event_already_locked(const char *name_arg, uint len)
bool err;
Binlog_checkpoint_log_event ev(name_arg, len);
+ DBUG_ASSERT(!opt_binlog_engine_hton);
/*
Note that we must sync the binlog checkpoint to disk.
Otherwise a subsequent log purge could delete binlogs that XA recovery
@@ -8432,6 +9868,7 @@ MYSQL_BIN_LOG::write_transaction_to_binlog(THD *thd,
)
{
cache_mngr->need_unlog= false;
+ cache_mngr->need_engine_2pc= false;
DBUG_RETURN(0);
}
@@ -8439,19 +9876,40 @@ MYSQL_BIN_LOG::write_transaction_to_binlog(THD *thd,
entry.cache_mngr= cache_mngr;
entry.error= 0;
entry.all= all;
- entry.using_stmt_cache= using_stmt_cache;
- entry.using_trx_cache= using_trx_cache;
- entry.need_unlog= is_preparing_xa(thd);
+ entry.need_unlog= unlikely(is_preparing_xa(thd)) && !opt_binlog_engine_hton;
ha_info= all ? thd->transaction->all.ha_list : thd->transaction->stmt.ha_list;
entry.ro_1pc= is_ro_1pc;
+ entry.do_binlog_group_commit_ordered= false;
entry.end_event= end_ev;
+ cache_mngr->using_stmt_cache= using_stmt_cache;
+ cache_mngr->using_trx_cache= using_trx_cache;
+ cache_mngr->need_engine_2pc= false;
auto has_xid= entry.end_event->get_type_code() == XID_EVENT;
- for (; has_xid && !entry.need_unlog && ha_info; ha_info= ha_info->next())
+ for (; ha_info; ha_info= ha_info->next())
{
- if (ha_info->is_started() && ha_info->ht() != binlog_hton &&
- !ha_info->ht()->commit_checkpoint_request)
- entry.need_unlog= true;
+ if (likely(has_xid) && likely(ha_info->is_started()))
+ {
+ if (opt_binlog_engine_hton)
+ {
+ if (ha_info->ht() != binlog_hton &&
+ ha_info->ht() != opt_binlog_engine_hton &&
+ ha_info->is_trx_read_write())
+ {
+ cache_mngr->need_engine_2pc= true;
+ cache_mngr->trx_cache.engine_binlog_info.internal_xa= true;
+ }
+ }
+ else
+ {
+ if (ha_info->ht() != binlog_hton &&
+ ha_info->is_trx_read_write() &&
+ !ha_info->ht()->commit_checkpoint_request)
+ entry.need_unlog= true;
+ }
+ }
+ else
+ break;
}
if (cache_mngr->stmt_cache.has_incident() ||
@@ -8538,7 +9996,7 @@ MYSQL_BIN_LOG::queue_for_group_commit(group_commit_entry *orig_entry)
If waitee->commit_started is set, it means that the transaction we need
to wait for has already queued up for group commit. In this case it is
- safe for us to queue up immediately as well, increasing the opprtunities
+ safe for us to queue up immediately as well, increasing the opportunities
for group commit. Because waitee has taken the LOCK_prepare_ordered
before setting the flag, so there is no risk that we can queue ahead of
it.
@@ -8834,6 +10292,7 @@ MYSQL_BIN_LOG::write_transaction_to_binlog_events(group_commit_entry *entry)
{
DEBUG_SYNC(entry->thd, "before_group_commit_queue");
int is_leader= queue_for_group_commit(entry);
+ binlog_cache_mngr *cache_mngr= entry->cache_mngr;
#ifdef WITH_WSREP
/* commit order was released in queue_for_group_commit() call,
here we check if wsrep_commit_ordered() failed or if we are leader */
@@ -8892,9 +10351,32 @@ MYSQL_BIN_LOG::write_transaction_to_binlog_events(group_commit_entry *entry)
DEBUG_SYNC(entry->thd, "commit_loop_entry_commit_ordered");
++num_commits;
- if (entry->cache_mngr->using_xa && !entry->error)
+ if (cache_mngr->using_xa && !entry->error)
run_commit_ordered(entry->thd, entry->all);
+ if (unlikely(!cache_mngr->engine_binlogged) && opt_binlog_engine_hton)
+ {
+ binlog_cache_data *cache_data= cache_mngr->engine_cache_data();
+ IO_CACHE *file= &cache_data->cache_log;
+ handler_binlog_event_group_info *engine_context=
+ &cache_data->engine_binlog_info;
+ binlog_setup_engine_commit_data(engine_context, cache_mngr);
+ if (likely(!entry->error))
+ {
+ entry->error= (*opt_binlog_engine_hton->binlog_write_direct_ordered)
+ (file, engine_context, entry->thd->get_last_commit_gtid());
+ if (likely(!entry->error))
+ {
+ cache_mngr->last_commit_pos_file.engine_file_no=
+ engine_context->out_file_no;
+ cache_mngr->last_commit_pos_offset= engine_context->out_file_no;
+
+ /* Mark to call binlog_write_direct() later. */
+ cache_mngr->need_write_direct= TRUE;
+ }
+ }
+ }
+
group_commit_entry *next= entry->next;
if (!next)
{
@@ -8936,6 +10418,24 @@ MYSQL_BIN_LOG::write_transaction_to_binlog_events(group_commit_entry *entry)
}
+ if (unlikely(cache_mngr->need_write_direct))
+ {
+ binlog_cache_data *cache_data= cache_mngr->engine_cache_data();
+ IO_CACHE *file= &cache_data->cache_log;
+ handler_binlog_event_group_info *engine_context=
+ &cache_data->engine_binlog_info;
+ if (likely(!entry->error))
+ entry->error= (*opt_binlog_engine_hton->binlog_write_direct)
+ (file, engine_context, entry->thd->get_last_commit_gtid());
+ }
+ if (entry->do_binlog_group_commit_ordered)
+ {
+ binlog_cache_data *cache_data=
+ cache_mngr->get_binlog_cache_data(cache_mngr->using_trx_cache);
+ (*opt_binlog_engine_hton->binlog_group_commit_ordered)
+ (entry->thd, &cache_data->engine_binlog_info);
+ }
+
if (likely(!entry->error))
return entry->thd->wait_for_prior_commit();
@@ -8964,9 +10464,9 @@ MYSQL_BIN_LOG::write_transaction_to_binlog_events(group_commit_entry *entry)
we need to mark it as not needed for recovery (unlog() is not called
for a transaction if log_xid() fails).
*/
- if (entry->cache_mngr->using_xa && entry->cache_mngr->xa_xid &&
- entry->cache_mngr->need_unlog)
- mark_xid_done(entry->cache_mngr->binlog_id, true);
+ if (cache_mngr->using_xa && cache_mngr->xa_xid &&
+ cache_mngr->need_unlog)
+ mark_xid_done(cache_mngr->binlog_id, true);
return 1;
}
@@ -9080,11 +10580,19 @@ MYSQL_BIN_LOG::trx_group_commit_leader(group_commit_entry *leader)
commit_id))))
current->commit_errno= errno;
- strmake_buf(cache_mngr->last_commit_pos_file, log_file_name);
- commit_offset= my_b_write_tell(&log_file);
- update_gtid_index((uint32)commit_offset,
- current->thd->get_last_commit_gtid());
- cache_mngr->last_commit_pos_offset= commit_offset;
+ if (!opt_binlog_engine_hton)
+ {
+ strmake_buf(cache_mngr->last_commit_pos_file.legacy_name, log_file_name);
+ commit_offset= my_b_write_tell(&log_file);
+ cache_mngr->last_commit_pos_offset= commit_offset;
+ /*
+ When --binlog-storage-engine, the last_commit_pos is updated in
+ binlog_get_cache().
+ */
+ update_gtid_index((uint32)commit_offset,
+ current->thd->get_last_commit_gtid());
+ }
+
if ((cache_mngr->using_xa && cache_mngr->xa_xid) || current->need_unlog)
{
/*
@@ -9108,7 +10616,7 @@ MYSQL_BIN_LOG::trx_group_commit_leader(group_commit_entry *leader)
set_current_thd(leader->thd);
bool synced= 0;
- if (unlikely(flush_and_sync(&synced)))
+ if (!opt_binlog_engine_hton && unlikely(flush_and_sync(&synced)))
{
for (current= queue; current != NULL; current= current->next)
{
@@ -9122,87 +10630,102 @@ MYSQL_BIN_LOG::trx_group_commit_leader(group_commit_entry *leader)
}
else
{
- DEBUG_SYNC(leader->thd, "commit_before_update_binlog_end_pos");
- bool any_error= false;
+#ifdef HAVE_REPLICATION
+ if (unlikely(repl_semisync_master.get_master_enabled()))
+ {
+ DEBUG_SYNC(leader->thd, "commit_before_update_binlog_end_pos");
+ bool any_error= false;
- mysql_mutex_assert_not_owner(&LOCK_prepare_ordered);
- mysql_mutex_assert_owner(&LOCK_log);
- mysql_mutex_assert_not_owner(&LOCK_after_binlog_sync);
- mysql_mutex_assert_not_owner(&LOCK_commit_ordered);
+ mysql_mutex_assert_not_owner(&LOCK_prepare_ordered);
+ mysql_mutex_assert_owner(&LOCK_log);
+ mysql_mutex_assert_not_owner(&LOCK_after_binlog_sync);
+ mysql_mutex_assert_not_owner(&LOCK_commit_ordered);
- for (current= queue; current != NULL; current= current->next)
- {
-#ifdef HAVE_REPLICATION
- /*
- The thread which will await the ACK from the replica can change
- depending on the wait-point. If AFTER_COMMIT, then the user thread
- will perform the wait. If AFTER_SYNC, the binlog group commit leader
- will perform the wait on behalf of the user thread.
- */
- THD *waiter_thd= (repl_semisync_master.wait_point() ==
- SEMI_SYNC_MASTER_WAIT_POINT_AFTER_STORAGE_COMMIT)
- ? current->thd
- : leader->thd;
- if (likely(!current->error) &&
- unlikely(repl_semisync_master.
- report_binlog_update(current->thd, waiter_thd,
- current->cache_mngr->
- last_commit_pos_file,
- current->cache_mngr->
- last_commit_pos_offset)))
+ for (current= queue; current != NULL; current= current->next)
{
- current->error= ER_ERROR_ON_WRITE;
- current->commit_errno= -1;
- current->error_cache= NULL;
- any_error= true;
+ /*
+ The thread which will await the ACK from the replica can change
+ depending on the wait-point. If AFTER_COMMIT, then the user thread
+ will perform the wait. If AFTER_SYNC, the binlog group commit leader
+ will perform the wait on behalf of the user thread.
+ */
+ THD *waiter_thd= (repl_semisync_master.wait_point() ==
+ SEMI_SYNC_MASTER_WAIT_POINT_AFTER_STORAGE_COMMIT)
+ ? current->thd
+ : leader->thd;
+ char buf[FN_REFLEN];
+ const char *filename= buf;
+ if (opt_binlog_engine_hton)
+ (*opt_binlog_engine_hton->get_filename)
+ (buf, current->cache_mngr->last_commit_pos_file.engine_file_no);
+ else
+ filename= current->cache_mngr->last_commit_pos_file.legacy_name;
+ if (likely(!current->error) &&
+ unlikely(repl_semisync_master.
+ report_binlog_update(current->thd, waiter_thd, filename,
+ (my_off_t)current->cache_mngr->
+ last_commit_pos_offset)))
+ {
+ current->error= ER_ERROR_ON_WRITE;
+ current->commit_errno= -1;
+ current->error_cache= NULL;
+ any_error= true;
+ }
}
-#endif
+
+ if (unlikely(any_error))
+ sql_print_error("Failed to run 'after_flush' hooks");
}
+#endif
/*
update binlog_end_pos so it can be read by dump thread
Note: must be _after_ the RUN_HOOK(after_flush) or else
semi-sync might not have put the transaction into
it's list before dump-thread tries to send it
- */
- update_binlog_end_pos(commit_offset);
- if (unlikely(any_error))
- sql_print_error("Failed to run 'after_flush' hooks");
- }
-
- /*
- If any commit_events are Xid_log_event, increase the number of pending
- XIDs in current binlog (it's decreased in ::unlog()). When the count in
- a (not active) binlog file reaches zero, we know that it is no longer
- needed in XA recovery, and we can log a new binlog checkpoint event.
- */
- if (xid_count > 0)
- {
- mark_xids_active(binlog_id, xid_count);
+ When --binlog-storage-engine, the binlog write happens during
+ commit_ordered(), so postpone the update until then.
+ */
+ if (!opt_binlog_engine_hton)
+ update_binlog_end_pos(commit_offset);
}
- if (rotate(false, &check_purge))
+ if (!opt_binlog_engine_hton)
{
/*
- If we fail to rotate, which thread should get the error?
- We give the error to the leader, as any my_error() thrown inside
- rotate() will have been registered for the leader THD.
+ If any commit_events are Xid_log_event, increase the number of pending
+ XIDs in current binlog (it's decreased in ::unlog()). When the count in
+ a (not active) binlog file reaches zero, we know that it is no longer
+ needed in XA recovery, and we can log a new binlog checkpoint event.
+ */
+ if (xid_count > 0)
+ {
+ mark_xids_active(binlog_id, xid_count);
+ }
- However we must not return error from here - that would cause
- ha_commit_trans() to abort and rollback the transaction, which would
- leave an inconsistent state with the transaction committed in the
- binlog but rolled back in the engine.
+ if (rotate(false, &check_purge))
+ {
+ /*
+ If we fail to rotate, which thread should get the error?
+ We give the error to the leader, as any my_error() thrown inside
+ rotate() will have been registered for the leader THD.
- Instead set a flag so that we can return error later, from unlog(),
- when the transaction has been safely committed in the engine.
- */
- leader->cache_mngr->delayed_error= true;
- my_error(ER_ERROR_ON_WRITE, MYF(ME_ERROR_LOG), name, errno);
- check_purge= false;
+ However we must not return error from here - that would cause
+ ha_commit_trans() to abort and rollback the transaction, which would
+ leave an inconsistent state with the transaction committed in the
+ binlog but rolled back in the engine.
+
+ Instead set a flag so that we can return error later, from unlog(),
+ when the transaction has been safely committed in the engine.
+ */
+ leader->cache_mngr->delayed_error= true;
+ my_error(ER_ERROR_ON_WRITE, MYF(ME_ERROR_LOG), name, errno);
+ check_purge= false;
+ }
+ /* In case of binlog rotate, update the correct current binlog offset. */
+ commit_offset= my_b_write_tell(&log_file);
}
- /* In case of binlog rotate, update the correct current binlog offset. */
- commit_offset= my_b_write_tell(&log_file);
}
DEBUG_SYNC(leader->thd, "commit_before_get_LOCK_after_binlog_sync");
@@ -9217,9 +10740,14 @@ MYSQL_BIN_LOG::trx_group_commit_leader(group_commit_entry *leader)
DEBUG_SYNC(leader->thd, "commit_after_release_LOCK_log");
+#ifdef HAVE_REPLICATION
/*
Loop through threads and run the binlog_sync hook
+ AFTER_SYNC is not available for --binlog-in-engine, as there we avoid the
+ costly two-phase commit between binlog and engine.
*/
+ if (!opt_binlog_engine_hton &&
+ unlikely(repl_semisync_master.get_master_enabled()))
{
mysql_mutex_assert_not_owner(&LOCK_prepare_ordered);
mysql_mutex_assert_not_owner(&LOCK_log);
@@ -9231,17 +10759,16 @@ MYSQL_BIN_LOG::trx_group_commit_leader(group_commit_entry *leader)
for (current= queue; current != NULL; current= current->next)
{
last= current->next == NULL;
-#ifdef HAVE_REPLICATION
if (likely(!current->error))
current->error=
repl_semisync_master.wait_after_sync(current->cache_mngr->
- last_commit_pos_file,
- current->cache_mngr->
+ last_commit_pos_file.legacy_name,
+ (my_off_t)current->cache_mngr->
last_commit_pos_offset);
-#endif
first= false;
}
}
+#endif
DEBUG_SYNC(leader->thd, "commit_before_get_LOCK_commit_ordered");
@@ -9250,7 +10777,10 @@ MYSQL_BIN_LOG::trx_group_commit_leader(group_commit_entry *leader)
{
DBUG_SUICIDE();
});
- last_commit_pos_offset= commit_offset;
+ if (opt_binlog_engine_hton)
+ last_in_queue->do_binlog_group_commit_ordered= true;
+ else
+ last_commit_pos_offset= commit_offset;
/*
Unlock LOCK_after_binlog_sync only *after* LOCK_commit_ordered has been
@@ -9296,13 +10826,50 @@ MYSQL_BIN_LOG::trx_group_commit_leader(group_commit_entry *leader)
while (current != NULL)
{
group_commit_entry *next;
+ binlog_cache_mngr *cache_mngr= current->cache_mngr;
DEBUG_SYNC(leader->thd, "commit_loop_entry_commit_ordered");
+ cache_mngr->engine_binlogged= FALSE;
++num_commits;
set_current_thd(current->thd);
- if (current->cache_mngr->using_xa && likely(!current->error) &&
+ if (cache_mngr->using_xa && likely(!current->error) &&
!DBUG_IF("skip_commit_ordered"))
run_commit_ordered(current->thd, current->all);
+
+ if (unlikely(!cache_mngr->engine_binlogged) && opt_binlog_engine_hton)
+ {
+ /*
+ If the binlog engine did not binlog for us as part of its own internal
+ transaction commit during commit_ordered(), we need to binlog it
+ explicitly here, while still holding LOCK_commit_ordered to ensure the
+ correct commit order.
+
+ The common case is a normal transaction in the binlog engine, and we
+ will not hit this condition. But it can happen for example when mixing
+ transactional and non-transactional DML in the same event group, or when
+ doing CREATE TABLE ... SELECT using row-based binlogging.
+ */
+ binlog_cache_data *cache_data= cache_mngr->engine_cache_data();
+ IO_CACHE *file= &cache_data->cache_log;
+ handler_binlog_event_group_info *engine_context=
+ &cache_data->engine_binlog_info;
+ binlog_setup_engine_commit_data(engine_context, cache_mngr);
+ if (likely(!current->error))
+ {
+ current->error= (*opt_binlog_engine_hton->binlog_write_direct_ordered)
+ (file, engine_context, current->thd->get_last_commit_gtid());
+ if (likely(!current->error))
+ {
+ cache_mngr->last_commit_pos_file.engine_file_no=
+ engine_context->out_file_no;
+ cache_mngr->last_commit_pos_offset= engine_context->out_file_no;
+
+ /* Mark to call binlog_write_direct later. */
+ cache_mngr->need_write_direct= TRUE;
+ }
+ }
+ }
+
current->thd->wakeup_subsequent_commits(current->error);
/*
@@ -9324,7 +10891,7 @@ MYSQL_BIN_LOG::trx_group_commit_leader(group_commit_entry *leader)
mysql_mutex_unlock(&LOCK_commit_ordered);
DEBUG_SYNC(leader->thd, "commit_after_group_release_commit_ordered");
- if (check_purge)
+ if (check_purge && !opt_binlog_engine_hton)
checkpoint_and_purge(binlog_id);
DBUG_VOID_RETURN;
@@ -9349,7 +10916,7 @@ MYSQL_BIN_LOG::write_transaction_or_stmt(group_commit_entry *entry,
#ifndef DBUG_OFF
- if (entry->using_trx_cache)
+ if (mngr->using_trx_cache)
{
IO_CACHE *cache= mngr->get_binlog_cache_log(TRUE);
/*
@@ -9373,22 +10940,24 @@ MYSQL_BIN_LOG::write_transaction_or_stmt(group_commit_entry *entry,
in an incident event being written over a (possibly corrupt) cache content.
Assert that this holds.
*/
- DBUG_ASSERT(!(entry->using_stmt_cache && !mngr->stmt_cache.empty() &&
+ DBUG_ASSERT(!(mngr->using_stmt_cache && !mngr->stmt_cache.empty() &&
mngr->get_binlog_cache_log(FALSE)->error));
- if (write_gtid_event(entry->thd, is_prepared_xa(entry->thd),
- entry->using_trx_cache, commit_id,
+ if (!opt_binlog_engine_hton)
+ {
+ if (write_gtid_event(entry->thd, nullptr, is_prepared_xa(entry->thd),
+ mngr->using_trx_cache, commit_id,
has_xid, entry->ro_1pc))
DBUG_RETURN(ER_ERROR_ON_WRITE);
- if (entry->using_stmt_cache && !mngr->stmt_cache.empty() &&
+ if (mngr->using_stmt_cache && !mngr->stmt_cache.empty() &&
write_cache(entry->thd, mngr->get_binlog_cache_data(FALSE)))
{
entry->error_cache= &mngr->stmt_cache.cache_log;
DBUG_RETURN(ER_ERROR_ON_WRITE);
}
- if (entry->using_trx_cache && !mngr->trx_cache.empty())
+ if (mngr->using_trx_cache && !mngr->trx_cache.empty())
{
DBUG_EXECUTE_IF("crash_before_writing_xid",
{
@@ -9416,6 +10985,12 @@ MYSQL_BIN_LOG::write_transaction_or_stmt(group_commit_entry *entry,
DBUG_RETURN(ER_ERROR_ON_WRITE);
});
+ /*
+ Write the end event (XID_EVENT, commit QUERY_LOG_EVENT) directly to the
+ legacy binary log. This is required to get the correct end position in the
+ event as currently needed by non-GTID slaves (since write_cache() does a
+ direct write of the cache, leaving end positions at zero).
+ */
if (write_event(entry->end_event))
{
entry->error_cache= NULL;
@@ -9423,6 +10998,57 @@ MYSQL_BIN_LOG::write_transaction_or_stmt(group_commit_entry *entry,
}
status_var_add(entry->thd->status_var.binlog_bytes_written,
entry->end_event->data_written);
+ }
+ else
+ {
+ DBUG_ASSERT((mngr->using_stmt_cache && !mngr->stmt_cache.empty()) ||
+ (mngr->using_trx_cache && !mngr->trx_cache.empty()) ||
+ (mngr->using_trx_cache &&
+ mngr->trx_cache.engine_binlog_info.xa_xid != nullptr)
+ /*
+ Assert that empty transaction is handled elsewhere.
+ Except in XA COMMIT, all events are OOB-spilled with the
+ prepare record, the caches are empty.
+ */
+ );
+ if (unlikely((mngr->using_stmt_cache && !mngr->stmt_cache.empty()) &&
+ (mngr->using_trx_cache && !mngr->trx_cache.empty())))
+ {
+ /*
+ We have data in both the statement and the transaction cache.
+ This is usually not the case, but it can occur in autocommit when
+ both transactional and non-transactional tables are changed in a
+ single statement, for example multi-table update or implicit
+ allocation from SEQUENCE as part of transactional INSERT.
+
+ We do not want the complexity of handling two different caches in
+ the engine binlog API for this uncommon case, so we spill both
+ caches as OOB data here, leaving just the transaction cache
+ containing the GTID event.
+
+ (We still need to handle dual OOB data streams, as we might need to
+ roll back the transaction one and binlog the statement one).
+ */
+ if (my_b_flush_io_cache(&mngr->stmt_cache.cache_log, 0) ||
+ my_b_flush_io_cache(&mngr->trx_cache.cache_log, 0))
+ DBUG_RETURN(ER_ERROR_ON_WRITE);
+ }
+
+ binlog_cache_data *cache_data= mngr->engine_cache_data();
+ /*
+ The GTID event cannot go first since we only allocate the GTID at binlog
+ time. So write the GTID at the very end, and record its offset so that the
+ engine can pick it out and binlog it at the start.
+ */
+ cache_data->engine_binlog_info.gtid_offset= my_b_tell(&cache_data->cache_log);
+ if (write_gtid_event(entry->thd, cache_data, false,
+ mngr->using_trx_cache, commit_id,
+ has_xid, entry->ro_1pc))
+ DBUG_RETURN(ER_ERROR_ON_WRITE);
+ DBUG_EXECUTE_IF("binlog_sleep_after_alloc_gtid",
+ my_sleep(200000);
+ );
+ }
if (entry->incident_event)
{
@@ -9626,6 +11252,7 @@ int MYSQL_BIN_LOG::wait_for_update_binlog_end_pos(THD* thd,
int ret= 0;
DBUG_ENTER("wait_for_update_binlog_end_pos");
+ DBUG_ASSERT(!opt_binlog_engine_hton);
thd_wait_begin(thd, THD_WAIT_BINLOG);
mysql_mutex_assert_owner(get_binlog_end_pos_lock());
if (!timeout)
@@ -9770,6 +11397,13 @@ void MYSQL_BIN_LOG::close(uint exiting)
}
+void
+MYSQL_BIN_LOG::close_engine()
+{
+ log_state= LOG_CLOSED;
+}
+
+
/*
Clear the LOG_EVENT_BINLOG_IN_USE_F; this marks the binlog file as cleanly
closed and not needing crash recovery.
@@ -9795,6 +11429,12 @@ void MYSQL_BIN_LOG::set_max_size(ulong max_size_arg)
mysql_mutex_lock(&LOCK_log);
if (is_open())
max_size= max_size_arg;
+ if (opt_binlog_engine_hton)
+ {
+ (*opt_binlog_engine_hton->set_binlog_max_size)((size_t)max_size_arg);
+ binlog_max_spill_size=
+ std::min((size_t)(max_size_arg / 2), BINLOG_SPILL_MAX);
+ }
mysql_mutex_unlock(&LOCK_log);
DBUG_VOID_RETURN;
}
@@ -10682,7 +12322,7 @@ mmap_do_checkpoint_callback(void *data)
++pending->pending_count;
}
-int TC_LOG_MMAP::unlog(ulong cookie, my_xid xid)
+int TC_LOG_MMAP::unlog(THD *thd, ulong cookie, my_xid xid)
{
pending_cookies *full_buffer= NULL;
uint32 ncookies= tc_log_page_size / sizeof(my_xid);
@@ -11048,6 +12688,26 @@ bool MYSQL_BIN_LOG::truncate_and_remove_binlogs(const char *file_name,
#endif
return error > 0;
}
+
+
+static void
+binlog_recover_hash_free(void *p)
+{
+ const handler_binlog_xid_info *info=
+ reinterpret_cast(p);
+ delete info;
+}
+
+
+static const uchar *
+binlog_recover_hash_key(const void *p, size_t *out_len, my_bool)
+{
+ const XID *xid= &(reinterpret_cast(p)->xid);
+ *out_len= xid->key_length();
+ return xid->key();
+}
+
+
int TC_LOG_BINLOG::open(const char *opt_name)
{
int error= 1;
@@ -11057,6 +12717,29 @@ int TC_LOG_BINLOG::open(const char *opt_name)
DBUG_ASSERT(opt_name);
DBUG_ASSERT(opt_name[0]);
+ if (opt_binlog_engine_hton)
+ {
+ HASH recover_hash;
+ my_hash_init(key_memory_binlog_recover_exec, &recover_hash, &my_charset_bin,
+ 128, 0, sizeof(XID), binlog_recover_hash_key,
+ binlog_recover_hash_free, MYF(HASH_UNIQUE));
+ if ((*opt_binlog_engine_hton->binlog_init)
+ ((size_t)max_binlog_size, opt_binlog_directory, &recover_hash))
+ {
+ my_hash_free(&recover_hash);
+ DBUG_RETURN(1);
+ }
+
+ bool err= ha_recover_engine_binlog(&recover_hash);
+ my_hash_free(&recover_hash);
+ if (err)
+ DBUG_RETURN(1);
+
+ /* Engine binlog implementation recovers the GTID state by itself. */
+ binlog_state_recover_done= true;
+ DBUG_RETURN(0);
+ }
+
if (!my_b_inited(&index_file))
{
/* There was a failure to open the index file, can't open the binlog */
@@ -11109,16 +12792,24 @@ TC_LOG_BINLOG::log_and_order(THD *thd, my_xid xid, bool all,
DEBUG_SYNC(thd, "binlog_after_log_and_order");
- if (err)
- DBUG_RETURN(0);
-
bool need_unlog= cache_mngr->need_unlog;
+ bool need_engine_2pc= cache_mngr->need_engine_2pc;
/*
The transaction won't need the flag anymore.
Todo/fixme: consider to move the statement into cache_mngr->reset()
relocated to the current or later point.
*/
cache_mngr->need_unlog= false;
+ cache_mngr->need_engine_2pc= false;
+
+ if (err)
+ DBUG_RETURN(0);
+
+ if (unlikely(need_engine_2pc))
+ {
+ DBUG_ASSERT(!need_unlog);
+ DBUG_RETURN(BINLOG_COOKIE_ENGINE_UNLOG(cache_mngr->delayed_error));
+ }
/*
If using explicit user XA, we will not have XID. We must still return a
non-zero cookie (as zero cookie signals error).
@@ -11279,13 +12970,26 @@ TC_LOG_BINLOG::mark_xid_done(ulong binlog_id, bool write_checkpoint)
DBUG_VOID_RETURN;
}
-int TC_LOG_BINLOG::unlog(ulong cookie, my_xid xid)
+int TC_LOG_BINLOG::unlog(THD *thd, ulong cookie, my_xid xid)
{
DBUG_ENTER("TC_LOG_BINLOG::unlog");
if (!xid)
DBUG_RETURN(0);
- if (!BINLOG_COOKIE_IS_DUMMY(cookie))
+ if (BINLOG_COOKIE_IS_ENGINE_UNLOG(cookie))
+ {
+ DBUG_ASSERT(opt_binlog_engine_hton);
+ binlog_cache_mngr *cache_mngr= thd->binlog_get_cache_mngr();
+ DBUG_ASSERT(cache_mngr != nullptr);
+ if (likely(cache_mngr != nullptr))
+ {
+ cache_mngr->xid_buf.set(xid);
+ (*opt_binlog_engine_hton->binlog_unlog)
+ (&cache_mngr->xid_buf,
+ &cache_mngr->trx_cache.engine_binlog_info.engine_ptr);
+ }
+ }
+ else if (!BINLOG_COOKIE_IS_DUMMY(cookie))
mark_xid_done(BINLOG_COOKIE_GET_ID(cookie), true);
/*
See comment in trx_group_commit_leader() - if rotate() gave a failure,
@@ -11303,6 +13007,9 @@ int TC_LOG_BINLOG::unlog_xa_prepare(THD *thd, bool all)
{
DBUG_ASSERT(is_preparing_xa(thd));
+ if (opt_binlog_engine_hton)
+ return 0;
+
binlog_cache_mngr *cache_mngr= thd->binlog_setup_trx_data();
int cookie= 0;
@@ -11333,7 +13040,7 @@ int TC_LOG_BINLOG::unlog_xa_prepare(THD *thd, bool all)
cookie= BINLOG_COOKIE_MAKE(cache_mngr->binlog_id, cache_mngr->delayed_error);
cache_mngr->need_unlog= false;
- return unlog(cookie, 1);
+ return unlog(thd, cookie, 1);
}
@@ -12749,10 +14456,10 @@ void
mysql_bin_log_commit_pos(THD *thd, ulonglong *out_pos, const char **out_file)
{
binlog_cache_mngr *cache_mngr;
- if (opt_bin_log &&
+ if (likely(opt_bin_log) && likely(!opt_binlog_engine_hton) &&
(cache_mngr= thd->binlog_get_cache_mngr()))
{
- *out_file= cache_mngr->last_commit_pos_file;
+ *out_file= cache_mngr->last_commit_pos_file.legacy_name;
*out_pos= (ulonglong)(cache_mngr->last_commit_pos_offset);
}
else
@@ -12773,6 +14480,13 @@ binlog_checksum_update(MYSQL_THD thd, struct st_mysql_sys_var *var,
ulong UNINIT_VAR(prev_binlog_id);
mysql_mutex_unlock(&LOCK_global_system_variables);
+ if (opt_binlog_engine_hton && value)
+ {
+ sql_print_information("Value of binlog_checksum forced to NONE since "
+ "binlog_storage_engine is enabled, where "
+ "per-event checksumming is not needed");
+ value= 0;
+ }
mysql_mutex_lock(mysql_bin_log.get_log_lock());
if(mysql_bin_log.is_open())
{
@@ -12868,11 +14582,30 @@ TC_LOG_BINLOG::set_status_variables(THD *thd)
{
mysql_mutex_lock(&thd->LOCK_thd_data);
auto cache_mngr= thd->binlog_get_cache_mngr();
- have_snapshot= cache_mngr && cache_mngr->last_commit_pos_file[0];
- if (have_snapshot)
+ if (cache_mngr)
{
- set_binlog_snapshot_file(cache_mngr->last_commit_pos_file);
- binlog_snapshot_position= cache_mngr->last_commit_pos_offset;
+ if (opt_binlog_engine_hton)
+ {
+ have_snapshot=
+ cache_mngr->last_commit_pos_file.engine_file_no != ~(uint64_t)0;
+ if (have_snapshot)
+ {
+ char buf[FN_REFLEN];
+ uint64_t file_no= cache_mngr->last_commit_pos_file.engine_file_no;
+ (*opt_binlog_engine_hton->get_filename)(buf, file_no);
+ set_binlog_snapshot_file(buf);
+ binlog_snapshot_position= cache_mngr->last_commit_pos_offset;
+ }
+ }
+ else
+ {
+ have_snapshot= cache_mngr->last_commit_pos_file.legacy_name[0] != '\0';
+ if (have_snapshot)
+ {
+ set_binlog_snapshot_file(cache_mngr->last_commit_pos_file.legacy_name);
+ binlog_snapshot_position= cache_mngr->last_commit_pos_offset;
+ }
+ }
}
mysql_mutex_unlock(&thd->LOCK_thd_data);
}
@@ -12882,8 +14615,21 @@ TC_LOG_BINLOG::set_status_variables(THD *thd)
binlog_status_var_num_group_commits= this->num_group_commits;
if (!have_snapshot)
{
- set_binlog_snapshot_file(last_commit_pos_file);
- binlog_snapshot_position= last_commit_pos_offset;
+ if (opt_binlog_engine_hton)
+ {
+ char buf[FN_REFLEN];
+ uint64_t file_no;
+ uint64_t offset;
+ (*opt_binlog_engine_hton->binlog_status)(&file_no, &offset);
+ (*opt_binlog_engine_hton->get_filename)(buf, file_no);
+ set_binlog_snapshot_file(buf);
+ binlog_snapshot_position= (ulonglong)offset;
+ }
+ else
+ {
+ set_binlog_snapshot_file(last_commit_pos_file);
+ binlog_snapshot_position= last_commit_pos_offset;
+ }
}
mysql_mutex_unlock(&LOCK_commit_ordered);
mysql_mutex_lock(&LOCK_prepare_ordered);
@@ -13070,7 +14816,7 @@ void wsrep_register_binlog_handler(THD *thd, bool trx)
Set an implicit savepoint in order to be able to truncate a trx-cache.
*/
my_off_t pos= 0;
- binlog_trans_log_savepos(thd, &pos);
+ binlog_trans_log_savepos(thd, cache_mngr, &pos);
cache_mngr->trx_cache.set_prev_position(pos);
/*
diff --git a/sql/log.h b/sql/log.h
index 89d14324980c5..7153f6c3cdfe4 100644
--- a/sql/log.h
+++ b/sql/log.h
@@ -28,6 +28,8 @@ class Gtid_log_event;
bool reopen_fstreams(const char *filename, FILE *outstream, FILE *errstream);
void setup_log_handling();
+void give_purge_note(const char *reason, const char *file_name,
+ bool interactive);
bool trans_has_updated_trans_table(const THD* thd);
bool stmt_has_updated_trans_table(const THD *thd);
bool use_trans_cache(const THD* thd, bool is_transactional);
@@ -61,7 +63,8 @@ class TC_LOG
virtual int log_and_order(THD *thd, my_xid xid, bool all,
bool need_prepare_ordered,
bool need_commit_ordered) = 0;
- virtual int unlog(ulong cookie, my_xid xid)=0;
+ virtual int unlog(THD *thd, ulong cookie, my_xid xid)=0;
+ virtual int log_xa_prepare(THD *thd, bool all)= 0;
virtual int unlog_xa_prepare(THD *thd, bool all)= 0;
virtual void commit_checkpoint_notify(void *cookie)= 0;
@@ -116,7 +119,11 @@ class TC_LOG_DUMMY: public TC_LOG // use it to disable the logging
DBUG_ASSERT(0);
return 1;
}
- int unlog(ulong cookie, my_xid xid) override { return 0; }
+ int unlog(THD *thd, ulong cookie, my_xid xid) override { return 0; }
+ int log_xa_prepare(THD *thd, bool all) override
+ {
+ return 0;
+ }
int unlog_xa_prepare(THD *thd, bool all) override
{
return 0;
@@ -203,7 +210,11 @@ class TC_LOG_MMAP: public TC_LOG
void close() override;
int log_and_order(THD *thd, my_xid xid, bool all,
bool need_prepare_ordered, bool need_commit_ordered) override;
- int unlog(ulong cookie, my_xid xid) override;
+ int unlog(THD *thd, ulong cookie, my_xid xid) override;
+ int log_xa_prepare(THD *thd, bool all) override
+ {
+ return 0;
+ }
int unlog_xa_prepare(THD *thd, bool all) override
{
return 0;
@@ -266,12 +277,14 @@ class Relay_log_info;
*/
typedef struct st_log_info
{
+ /* file_no only used when --binlog-storage-engine set. */
+ std::atomic file_no;
+ /* log_file_name and *_offset only used when --binlog-storage-engine unset. */
char log_file_name[FN_REFLEN];
my_off_t index_file_offset, index_file_start_offset;
my_off_t pos;
- bool fatal; // if the purge happens to give us a negative offset
- st_log_info() : index_file_offset(0), index_file_start_offset(0),
- pos(0), fatal(0)
+ st_log_info() : file_no(~(uint64_t)0), index_file_offset(0),
+ index_file_start_offset(0), pos(0)
{
DBUG_ENTER("LOG_INFO");
log_file_name[0] = '\0';
@@ -577,6 +590,10 @@ class MYSQL_QUERY_LOG: public MYSQL_LOG
case where there is no corresponding binlog id (since nothing was logged).
And we need an error flag to mark that unlog() must return failure.
+ For --binlog-storage-engine, we need unlog if another engine than the
+ binlog engine participated in the transaction, or if we did a user XA
+ commit.
+
We use the following macros to pack all of this information into the single
ulong available with log_and_order() / unlog().
@@ -585,21 +602,27 @@ class MYSQL_QUERY_LOG: public MYSQL_LOG
*/
#define BINLOG_COOKIE_ERROR_RETURN 0
#define BINLOG_COOKIE_DUMMY_ID 1
-#define BINLOG_COOKIE_BASE 2
+#define BINLOG_COOKIE_ENGINE_UNLOG_ID 2
+#define BINLOG_COOKIE_BASE 3
#define BINLOG_COOKIE_DUMMY(error_flag) \
( (BINLOG_COOKIE_DUMMY_ID<<1) | ((error_flag)&1) )
+#define BINLOG_COOKIE_ENGINE_UNLOG(error_flag) \
+ ( (BINLOG_COOKIE_ENGINE_UNLOG_ID<<1) | ((error_flag)&1) )
#define BINLOG_COOKIE_MAKE(id, error_flag) \
( (((id)+BINLOG_COOKIE_BASE)<<1) | ((error_flag)&1) )
#define BINLOG_COOKIE_GET_ERROR_FLAG(c) ((c) & 1)
#define BINLOG_COOKIE_GET_ID(c) ( ((ulong)(c)>>1) - BINLOG_COOKIE_BASE )
#define BINLOG_COOKIE_IS_DUMMY(c) \
( ((ulong)(c)>>1) == BINLOG_COOKIE_DUMMY_ID )
+#define BINLOG_COOKIE_IS_ENGINE_UNLOG(c) \
+ ( ((ulong)(c)>>1) == BINLOG_COOKIE_ENGINE_UNLOG_ID )
class binlog_cache_mngr;
class binlog_cache_data;
struct rpl_gtid;
struct wait_for_commit;
+struct rpl_binlog_state_base;
class MYSQL_BIN_LOG: public TC_LOG, public Event_log
{
@@ -630,8 +653,6 @@ class MYSQL_BIN_LOG: public TC_LOG, public Event_log
struct group_commit_entry *next;
THD *thd;
binlog_cache_mngr *cache_mngr;
- bool using_stmt_cache;
- bool using_trx_cache;
/*
Extra events (COMMIT/ROLLBACK/XID, and possibly INCIDENT) to be
written during group commit. The incident_event is only valid if
@@ -643,6 +664,7 @@ class MYSQL_BIN_LOG: public TC_LOG, public Event_log
int error;
int commit_errno;
IO_CACHE *error_cache;
+ ulong binlog_id;
/* This is the `all' parameter for ha_commit_ordered(). */
bool all;
/*
@@ -659,8 +681,13 @@ class MYSQL_BIN_LOG: public TC_LOG, public Event_log
bool check_purge;
/* Flag used to optimise around wait_for_prior_commit. */
bool queued_by_other;
- ulong binlog_id;
bool ro_1pc; // passes the binlog_cache_mngr::ro_1pc value to Gtid ctor
+ /*
+ Set for the last participant in group commit, it must invoke
+ binlog_group_commit_ordered (in case of --binlog-storage-engine) after
+ LOCK_commit_ordered has been released.
+ */
+ bool do_binlog_group_commit_ordered;
};
/*
@@ -674,6 +701,18 @@ class MYSQL_BIN_LOG: public TC_LOG, public Event_log
uint reset_master_pending;
ulong mark_xid_done_waiting;
+ /*
+ Protect against binlog readers (eg. slave dump threads) running
+ concurrently with RESET MASTER.
+ binlog_use_count counts the number of active readers, or is -1 when a
+ RESET MASTER is running. It is protected by LOCK_binlog_use and
+ COND_binlog_use is signalled when RESET MASTER completes so new
+ readers can wait for that.
+ */
+ int32_t binlog_use_count;
+ mysql_mutex_t LOCK_binlog_use;
+ mysql_cond_t COND_binlog_use;
+
/* LOCK_log and LOCK_index are inited by init_pthread_objects() */
mysql_mutex_t LOCK_index;
mysql_mutex_t LOCK_xid_list;
@@ -763,7 +802,7 @@ class MYSQL_BIN_LOG: public TC_LOG, public Event_log
bool write_transaction_to_binlog_events(group_commit_entry *entry);
void trx_group_commit_leader(group_commit_entry *leader);
bool is_xidlist_idle_nolock();
- void update_gtid_index(uint32 offset, rpl_gtid gtid);
+ void update_gtid_index(uint32 offset, const rpl_gtid *gtid);
public:
void purge(bool all);
@@ -911,7 +950,8 @@ class MYSQL_BIN_LOG: public TC_LOG, public Event_log
ulong next_log_number) override;
int log_and_order(THD *thd, my_xid xid, bool all,
bool need_prepare_ordered, bool need_commit_ordered) override;
- int unlog(ulong cookie, my_xid xid) override;
+ int unlog(THD *thd, ulong cookie, my_xid xid) override;
+ int log_xa_prepare(THD *thd, bool all) override;
int unlog_xa_prepare(THD *thd, bool all) override;
void commit_checkpoint_notify(void *cookie) override;
int recover(LOG_INFO *linfo, const char *last_log_name, IO_CACHE *first_log,
@@ -963,6 +1003,7 @@ class MYSQL_BIN_LOG: public TC_LOG, public Event_log
signal_relay_log_update();
else
{
+ DBUG_ASSERT(!opt_binlog_engine_hton);
lock_binlog_end_pos();
binlog_end_pos= my_b_safe_tell(&log_file);
signal_bin_log_update();
@@ -971,6 +1012,7 @@ class MYSQL_BIN_LOG: public TC_LOG, public Event_log
}
void update_binlog_end_pos(my_off_t pos)
{
+ DBUG_ASSERT(!opt_binlog_engine_hton);
mysql_mutex_assert_owner(&LOCK_log);
mysql_mutex_assert_not_owner(&LOCK_binlog_end_pos);
lock_binlog_end_pos();
@@ -998,6 +1040,7 @@ class MYSQL_BIN_LOG: public TC_LOG, public Event_log
ulong max_size,
bool null_created,
bool need_mutex);
+ bool open_engine(handlerton *hton, ulong max_size, const char *dir);
bool open_index_file(const char *index_file_name_arg,
const char *log_name, bool need_mutex);
/* Use this to start writing a new log file */
@@ -1040,6 +1083,14 @@ class MYSQL_BIN_LOG: public TC_LOG, public Event_log
int rotate(bool force_rotate, bool* check_purge);
void checkpoint_and_purge(ulong binlog_id);
int rotate_and_purge(bool force_rotate, DYNAMIC_ARRAY* drop_gtid_domain= NULL);
+ int flush_binlogs_engine(DYNAMIC_ARRAY *domain_drop_lex);
+ int flush_binlog(DYNAMIC_ARRAY* drop_gtid_domain)
+ {
+ if (opt_binlog_engine_hton)
+ return flush_binlogs_engine(drop_gtid_domain);
+ else
+ return rotate_and_purge(true, drop_gtid_domain);
+ }
/**
Flush binlog cache and synchronize to disk.
@@ -1075,6 +1126,7 @@ class MYSQL_BIN_LOG: public TC_LOG, public Event_log
return 0;
return real_purge_logs_by_size(binlog_pos);
}
+ void engine_purge_logs_by_size(ulonglong max_total_size);
int set_purge_index_file_name(const char *base_file_name);
int open_purge_index_file(bool destroy);
bool truncate_and_remove_binlogs(const char *truncate_file,
@@ -1088,11 +1140,16 @@ class MYSQL_BIN_LOG: public TC_LOG, public Event_log
int register_create_index_entry(const char* entry);
int purge_index_entry(THD *thd, ulonglong *decrease_log_space,
bool need_mutex);
+ bool start_use_binlog(THD *thd);
+ void end_use_binlog(THD *thd);
bool reset_logs(THD* thd, bool create_new_log,
rpl_gtid *init_state, uint32 init_state_len,
ulong next_log_number);
+ bool reset_engine_binlogs(THD *thd, rpl_gtid *init_state,
+ uint32 init_state_len);
void wait_for_last_checkpoint_event();
void close(uint exiting);
+ void close_engine();
void clear_inuse_flag_when_closing(File file);
// iterating through the log index file
@@ -1115,9 +1172,9 @@ class MYSQL_BIN_LOG: public TC_LOG, public Event_log
inline uint32 get_open_count() { return open_count; }
void set_status_variables(THD *thd);
bool is_xidlist_idle();
- bool write_gtid_event(THD *thd, bool standalone, bool is_transactional,
- uint64 commit_id,
- bool has_xid= false, bool ro_1pc= false);
+ bool write_gtid_event(THD *thd, binlog_cache_data *cache_data,
+ bool standalone, bool is_transactional,
+ uint64 commit_id, bool has_xid, bool ro_1pc);
int read_state_from_file();
int write_state_to_file();
int get_most_recent_gtid_list(rpl_gtid **list, uint32 *size);
@@ -1190,6 +1247,11 @@ class MYSQL_BIN_LOG: public TC_LOG, public Event_log
char binlog_end_pos_file[FN_REFLEN];
};
+extern bool load_global_binlog_state(rpl_binlog_state_base *state);
+extern bool binlog_recover_gtid_state(rpl_binlog_state_base *state,
+ handler_binlog_reader *reader);
+
+
class Log_event_handler
{
public:
@@ -1486,8 +1548,12 @@ get_gtid_list_event(IO_CACHE *cache, Gtid_list_log_event **out_gtid_list);
int binlog_commit(THD *thd, bool all, bool is_ro_1pc= false);
int binlog_rollback(handlerton *hton, THD *thd, bool all);
+void binlog_post_commit(THD *thd, bool all);
+void binlog_post_rollback(THD *thd, bool all);
int binlog_commit_by_xid(handlerton *hton, XID *xid);
int binlog_rollback_by_xid(handlerton *hton, XID *xid);
+void binlog_post_commit_by_xid(handlerton *hton, XID *xid);
+void binlog_post_rollback_by_xid(handlerton *hton, XID *xid);
bool write_bin_log_start_alter(THD *thd, bool& partial_alter,
uint64 start_alter_id, bool log_if_exists);
#endif /* LOG_H */
diff --git a/sql/log_cache.h b/sql/log_cache.h
index 1cde55bc94c9f..71cd0b4f1e51b 100644
--- a/sql/log_cache.h
+++ b/sql/log_cache.h
@@ -26,8 +26,10 @@ static constexpr my_off_t CACHE_FILE_TRUNC_SIZE = 65536;
class binlog_cache_data
{
public:
- binlog_cache_data(bool precompute_checksums):
+ binlog_cache_data(bool trx_cache, bool precompute_checksums):
+ engine_binlog_info {0, 0, 0, 0, 0, 0, 0, 0},
before_stmt_pos(MY_OFF_T_UNDEF), m_pending(0), status(0),
+ is_trx_cache(trx_cache),
incident(FALSE), precompute_checksums(precompute_checksums),
saved_max_binlog_cache_size(0), ptr_binlog_cache_use(0),
ptr_binlog_cache_disk_use(0)
@@ -45,9 +47,14 @@ class binlog_cache_data
~binlog_cache_data()
{
DBUG_ASSERT(empty());
+ if (engine_binlog_info.engine_ptr)
+ (*opt_binlog_engine_hton->binlog_oob_free)
+ (engine_binlog_info.engine_ptr);
close_cached_file(&cache_log);
}
+ bool trx_cache() { return is_trx_cache; }
+
/*
Return 1 if there is no relevant entries in the cache
@@ -110,6 +117,44 @@ class binlog_cache_data
DBUG_ASSERT(empty());
}
+ void reset_for_engine_binlog()
+ {
+ bool cache_was_empty= empty();
+
+ if (engine_binlog_info.engine_ptr)
+ (*opt_binlog_engine_hton->binlog_oob_reset)
+ (&engine_binlog_info.engine_ptr);
+ engine_binlog_info.engine_ptr2= nullptr;
+ engine_binlog_info.xa_xid= nullptr;
+ engine_binlog_info.out_of_band_offset= 0;
+ engine_binlog_info.gtid_offset= 0;
+ engine_binlog_info.internal_xa= false;
+ /* Preserve the engine_ptr for the engine to re-use, was reset above. */
+
+ truncate(cache_log.pos_in_file);
+ cache_log.pos_in_file= 0;
+ cache_log.request_pos= cache_log.write_pos= cache_log.buffer;
+ cache_log.write_end= cache_log.buffer + cache_log.buffer_length;
+ checksum_opt= BINLOG_CHECKSUM_ALG_OFF;
+ if (!cache_was_empty)
+ compute_statistics();
+ status= 0;
+ incident= FALSE;
+ before_stmt_pos= MY_OFF_T_UNDEF;
+ DBUG_ASSERT(empty());
+ }
+
+ void reset_cache_for_engine(my_off_t pos,
+ int (*fct)(struct st_io_cache *, const uchar *, size_t))
+ {
+ /* Bit fiddly here as we're abusing the IO_CACHE a bit for oob handling. */
+ cache_log.pos_in_file= pos;
+ cache_log.request_pos= cache_log.write_pos= cache_log.buffer;
+ cache_log.write_end=
+ (cache_log.buffer + cache_log.buffer_length - (pos & (IO_SIZE-1)));
+ cache_log.write_function= fct;
+ }
+
my_off_t get_byte_position() const
{
return my_b_tell(&cache_log);
@@ -176,6 +221,8 @@ class binlog_cache_data
Cache to store data before copying it to the binary log.
*/
IO_CACHE cache_log;
+ /* Context for engine-implemented binlogging. */
+ handler_binlog_event_group_info engine_binlog_info;
protected:
/*
@@ -205,6 +252,8 @@ class binlog_cache_data
enum_binlog_checksum_alg checksum_opt;
private:
+ /* Record whether this is a stmt or trx cache. */
+ bool is_trx_cache;
/*
This indicates that some events did not get into the cache and most likely
it is corrupted.
diff --git a/sql/log_event.cc b/sql/log_event.cc
index 4d521abfad47b..2be8e65a12358 100644
--- a/sql/log_event.cc
+++ b/sql/log_event.cc
@@ -1291,6 +1291,72 @@ Log_event* Log_event::read_log_event(const uchar *buf, uint event_len,
}
+int handler_binlog_reader::read_log_event(String *packet, uint32_t ev_offset,
+ size_t max_allowed)
+{
+ uint32_t sofar= 0;
+ bool header_read= false;
+ uint32_t target_size= EVENT_LEN_OFFSET + 4;
+ int res;
+
+ if (unlikely(!buf))
+ return LOG_READ_MEM;
+
+ /*
+ Loop, first reading the "length" field, and then continuing to read data
+ until a full event has been placed in the packet.
+ */
+ for (;;)
+ {
+ if (buf_data_remain <= 0)
+ {
+ res= read_binlog_data(buf, BUF_SIZE);
+ if (res <= 0)
+ {
+ res= (res < 0 ? LOG_READ_IO : LOG_READ_EOF);
+ goto err;
+ }
+ buf_data_pos= 0;
+ buf_data_remain= res;
+ }
+ uint32_t amount= std::min(target_size - sofar, buf_data_remain);
+ packet->append((char *)buf + buf_data_pos, amount);
+ buf_data_pos+= amount;
+ buf_data_remain-= amount;
+ sofar+= amount;
+ if (target_size == sofar)
+ {
+ if (header_read)
+ break;
+ else
+ {
+ header_read= true;
+ target_size= uint4korr(&((*packet)[EVENT_LEN_OFFSET + ev_offset]));
+ if (target_size < LOG_EVENT_MINIMAL_HEADER_LEN)
+ {
+ res= LOG_READ_BOGUS;
+ goto err;
+ }
+ else if (target_size > max_allowed)
+ {
+ res= LOG_READ_TOO_LARGE;
+ goto err;
+ }
+ /*
+ Note that here we rely on the fact that all valid events have more
+ data after the length. This way we avoid conditional for the
+ (useless) special case where we don't need to read anything more
+ after having read the first part.
+ */
+ DBUG_ASSERT(LOG_EVENT_MINIMAL_HEADER_LEN > EVENT_LEN_OFFSET+4);
+ }
+ }
+ }
+ res= 0; /* Success */
+err:
+ return res;
+}
+
/* 2 utility functions for the next method */
diff --git a/sql/log_event.h b/sql/log_event.h
index 2388ae22155ae..10e53f360c9aa 100644
--- a/sql/log_event.h
+++ b/sql/log_event.h
@@ -582,9 +582,11 @@ class String;
#define MARIA_SLAVE_CAPABILITY_BINLOG_CHECKPOINT 3
/* MariaDB >= 10.0.1, which knows about global transaction id events. */
#define MARIA_SLAVE_CAPABILITY_GTID 4
+/* MariaDB >= 12.2.1, basic engine-implemented binlog capability. */
+#define MARIA_SLAVE_CAPABILITY_ENGINE_BINLOG 5
/* Our capability. */
-#define MARIA_SLAVE_CAPABILITY_MINE MARIA_SLAVE_CAPABILITY_GTID
+#define MARIA_SLAVE_CAPABILITY_MINE MARIA_SLAVE_CAPABILITY_ENGINE_BINLOG
/*
@@ -2574,6 +2576,7 @@ class Format_description_log_event: public Log_event
#ifdef HAVE_REPLICATION
void pack_info(Protocol* protocol) override;
#endif /* HAVE_REPLICATION */
+ bool to_packet(String *packet);
#else
bool print(FILE* file, PRINT_EVENT_INFO* print_event_info) override;
#endif
@@ -3262,6 +3265,40 @@ class Binlog_checkpoint_log_event: public Log_event
};
+#ifdef MYSQL_SERVER
+/*
+ This is used to compute a compile-time constant max for the size (in bytes)
+ of a GTID event (Gtid_log_event::max_size).
+
+ It is carefully written to take boolean parameters corresponding directly
+ to each conditional in Gtid_log_event::write(), so that the calculation here
+ will match the actual length computed by write().
+
+ Please ensure that that any new conditionals added in write() that affect
+ the event length are similarly extended with a boolean parameter for this
+ function so future code changes do not introduce incorrect result of this
+ function.
+*/
+static constexpr uint32_t
+cap_gtid_event_size(uint32_t proposed_size)
+{
+ /* This just because std::min is not constexpr in c++11. */
+ return LOG_EVENT_HEADER_LEN +
+ (proposed_size < GTID_HEADER_LEN ? GTID_HEADER_LEN : proposed_size);
+}
+static constexpr uint32_t
+get_gtid_event_size(bool fl_commit_id, bool fl_xa, bool fl_extra,
+ bool fl_multi_engine, bool fl_alter,
+ int bq_size, int gt_size)
+{
+ return cap_gtid_event_size((fl_commit_id ? GTID_HEADER_LEN + 2 : 13) +
+ (fl_xa ? 6 + bq_size + gt_size : 0) +
+ (fl_extra ? 1 : 0) +
+ (fl_multi_engine ? 1 : 0) +
+ (fl_alter ? 8 : 0));
+}
+#endif
+
/**
@class Gtid_log_event
@@ -3392,15 +3429,23 @@ class Gtid_log_event: public Log_event
involving multiple storage engines. No flag and extra data are added
to the event when the transaction involves only one engine.
*/
- static const uchar FL_EXTRA_MULTI_ENGINE_E1= 1;
- static const uchar FL_START_ALTER_E1= 2;
- static const uchar FL_COMMIT_ALTER_E1= 4;
- static const uchar FL_ROLLBACK_ALTER_E1= 8;
+ static constexpr uchar FL_EXTRA_MULTI_ENGINE_E1= 1;
+ static constexpr uchar FL_START_ALTER_E1= 2;
+ static constexpr uchar FL_COMMIT_ALTER_E1= 4;
+ static constexpr uchar FL_ROLLBACK_ALTER_E1= 8;
#ifdef MYSQL_SERVER
+ static constexpr uint32_t max_size=
+ get_gtid_event_size(FL_GROUP_COMMIT_ID,
+ (bool)(FL_PREPARED_XA|FL_COMPLETED_XA),
+ true, FL_EXTRA_MULTI_ENGINE_E1,
+ (bool)(FL_COMMIT_ALTER_E1|FL_ROLLBACK_ALTER_E1),
+ MAXBQUALSIZE, MAXGTRIDSIZE);
+
Gtid_log_event(THD *thd_arg, uint64 seq_no, uint32 domain_id, bool standalone,
- uint16 flags, bool is_transactional, uint64 commit_id,
- bool has_xid= false, bool is_ro_1pc= false);
+ enum_event_cache_type cache_type_arg, uint16 flags,
+ bool is_transactional, uint64 commit_id,
+ bool has_xid, bool is_ro_1pc);
#ifdef HAVE_REPLICATION
void pack_info(Protocol *protocol) override;
int do_apply_event(rpl_group_info *rgi) override;
@@ -3414,7 +3459,6 @@ class Gtid_log_event: public Log_event
const Format_description_log_event *description_event);
~Gtid_log_event() = default;
Log_event_type get_type_code() override { return GTID_EVENT; }
- enum_logged_status logged_status() override { return LOGGED_NO_DATA; }
int get_data_size() override
{
return GTID_HEADER_LEN + ((flags2 & FL_GROUP_COMMIT_ID) ? 2 : 0);
@@ -3430,6 +3474,7 @@ class Gtid_log_event: public Log_event
}
#ifdef MYSQL_SERVER
+ uint32_t get_size() const noexcept;
bool write(Log_event_writer *writer) override;
static int make_compatible_event(String *packet, bool *need_dummy_event,
ulong ev_offset, enum_binlog_checksum_alg checksum_alg);
diff --git a/sql/log_event_server.cc b/sql/log_event_server.cc
index 18569d9d3901d..4e6437f4720f9 100644
--- a/sql/log_event_server.cc
+++ b/sql/log_event_server.cc
@@ -2410,22 +2410,28 @@ void Format_description_log_event::pack_info(Protocol *protocol)
}
#endif /* defined(HAVE_REPLICATION) */
-bool Format_description_log_event::write(Log_event_writer *writer)
+bool
+Format_description_log_event::to_packet(String *packet)
{
- bool ret;
- /*
- We don't call Start_log_event_v::write() because this would make 2
- my_b_safe_write().
- */
- uchar buff[START_V3_HEADER_LEN+1];
- size_t rec_size= sizeof(buff) + BINLOG_CHECKSUM_ALG_DESC_LEN +
- number_of_event_types;
- int2store(buff + ST_BINLOG_VER_OFFSET,binlog_version);
- memcpy((char*) buff + ST_SERVER_VER_OFFSET,server_version,ST_SERVER_VER_LEN);
+ uchar *p;
+ uint32 needed_length=
+ packet->length() + START_V3_HEADER_LEN + 1 + number_of_event_types + 1;
+ if (packet->reserve(needed_length))
+ return true;
+ p= (uchar *)packet->ptr() + packet->length();;
+ packet->length(needed_length);
+ int2store(p, binlog_version);
+ p += 2;
+ memcpy(p, server_version, ST_SERVER_VER_LEN);
+ p+= ST_SERVER_VER_LEN;
if (!dont_set_created)
created= get_time();
- int4store(buff + ST_CREATED_OFFSET,created);
- buff[ST_COMMON_HEADER_LEN_OFFSET]= common_header_len;
+ int4store(p, created);
+ p+= 4;
+ *p++= common_header_len;
+ memcpy(p, post_header_len, number_of_event_types);
+ p+= number_of_event_types;
+
/*
if checksum is requested
record the checksum-algorithm descriptor next to
@@ -2450,13 +2456,31 @@ bool Format_description_log_event::write(Log_event_writer *writer)
(A), (V) presence in FD of the checksum-aware server makes the event
1 + 4 bytes bigger comparing to the former FD.
*/
+ *p++= checksum_byte;
+
+ return false;
+}
+
+bool Format_description_log_event::write(Log_event_writer *writer)
+{
+ bool ret;
+ /*
+ We don't call Start_log_event_v::write() because this would make 2
+ my_b_safe_write().
+ */
+ constexpr uint32_t needed= START_V3_HEADER_LEN + 1 + LOG_EVENT_TYPES + 1;
+ char buff[needed + 1];
+ String packet(buff, sizeof(buff), system_charset_info);
+ packet.length(0);
+ if (to_packet(&packet))
+ return true;
+ size_t rec_size= packet.length();
+ DBUG_ASSERT(needed == rec_size);
uint orig_checksum_len= writer->checksum_len;
writer->checksum_len= BINLOG_CHECKSUM_LEN;
ret= write_header(writer, rec_size) ||
- write_data(writer, buff, sizeof(buff)) ||
- write_data(writer, post_header_len, number_of_event_types) ||
- write_data(writer, &checksum_byte, sizeof(checksum_byte)) ||
+ write_data(writer, packet.ptr(), packet.length()) ||
write_footer(writer);
writer->checksum_len= orig_checksum_len;
return ret;
@@ -2840,6 +2864,7 @@ bool Binlog_checkpoint_log_event::write(Log_event_writer *writer)
Gtid_log_event::Gtid_log_event(THD *thd_arg, uint64 seq_no_arg,
uint32 domain_id_arg, bool standalone,
+ enum_event_cache_type cache_type_arg,
uint16 flags_arg, bool is_transactional,
uint64 commit_id_arg, bool has_xid,
bool ro_1pc)
@@ -2849,7 +2874,7 @@ Gtid_log_event::Gtid_log_event(THD *thd_arg, uint64 seq_no_arg,
(commit_id_arg ? FL_GROUP_COMMIT_ID : 0)),
flags_extra(0), extra_engines(0)
{
- cache_type= Log_event::EVENT_NO_CACHE;
+ cache_type= cache_type_arg;
bool is_tmp_table= thd_arg->lex->stmt_accessed_temp_table();
if (thd_arg->transaction->stmt.trans_did_wait() ||
thd_arg->transaction->all.trans_did_wait())
@@ -2872,9 +2897,14 @@ Gtid_log_event::Gtid_log_event(THD *thd_arg, uint64 seq_no_arg,
if (thd_arg->rgi_slave)
flags2|= (thd_arg->rgi_slave->gtid_ev_flags2 & (FL_DDL|FL_WAITED));
- XID_STATE &xid_state= thd->transaction->xid_state;
- if (is_transactional)
+ /*
+ When --binlog-storage-engine, we write the GTID event through the trx cache
+ (not directly to the binlog file), and we do not use this XA stuff in the
+ GTID event, that's handled by the engine binlog implementation.
+ */
+ if (cache_type_arg == EVENT_NO_CACHE && is_transactional)
{
+ XID_STATE &xid_state= thd->transaction->xid_state;
if (xid_state.is_explicit_XA() &&
(thd->lex->sql_command == SQLCOM_XA_PREPARE ||
xid_state.get_state_code() == XA_PREPARED))
@@ -2958,6 +2988,18 @@ Gtid_log_event::peek(const uchar *event_start, size_t event_len,
}
+uint32_t
+Gtid_log_event::get_size() const noexcept
+{
+ return get_gtid_event_size(flags2 & FL_GROUP_COMMIT_ID,
+ flags2 & (FL_PREPARED_XA | FL_COMPLETED_XA),
+ flags_extra > 0,
+ flags_extra & FL_EXTRA_MULTI_ENGINE_E1,
+ flags_extra & (FL_COMMIT_ALTER_E1 | FL_ROLLBACK_ALTER_E1),
+ xid.bqual_length, xid.gtrid_length);
+}
+
+
bool
Gtid_log_event::write(Log_event_writer *writer)
{
@@ -3027,6 +3069,17 @@ Gtid_log_event::write(Log_event_writer *writer)
bzero(buf+write_len, GTID_HEADER_LEN-write_len);
write_len= GTID_HEADER_LEN;
}
+ /*
+ Whenever updating this function, make sure that Gtid_log_event::get_size()
+ still computes the same consistent event length! Do not just rely on this
+ assertion, in case test coverage is not 100%.
+ */
+ DBUG_ASSERT(DBUG_IF("negate_xid_from_gtid") ||
+ DBUG_IF("negate_xid_data_from_gtid") ||
+ DBUG_IF("negate_alter_fl_from_gtid") ||
+ DBUG_IF("inject_fl_extra_multi_engine_into_gtid") ||
+ write_len + LOG_EVENT_HEADER_LEN == get_size());
+
return write_header(writer, write_len) ||
write_data(writer, buf, write_len) ||
write_footer(writer);
diff --git a/sql/mysqld.cc b/sql/mysqld.cc
index 116127634dd84..b6e5bd8a2c336 100644
--- a/sql/mysqld.cc
+++ b/sql/mysqld.cc
@@ -365,6 +365,11 @@ char server_uid[SERVER_UID_SIZE+1]; // server uid will be written here
/* Global variables */
bool opt_bin_log, opt_bin_log_used=0, opt_ignore_builtin_innodb= 0;
+static bool opt_bin_log_nonempty, opt_bin_log_path;
+char *opt_binlog_storage_engine= const_cast("");
+static plugin_ref opt_binlog_engine_plugin;
+const char *opt_binlog_directory;
+handlerton *opt_binlog_engine_hton;
bool opt_bin_log_compress;
uint opt_bin_log_compress_min_len;
my_bool opt_log, debug_assert_if_crashed_table= 0, opt_help= 0;
@@ -766,6 +771,7 @@ mysql_rwlock_t LOCK_all_status_vars;
mysql_prlock_t LOCK_system_variables_hash;
mysql_cond_t COND_start_thread;
pthread_t signal_thread;
+bool signal_thread_needs_join= false;
pthread_attr_t connection_attrib;
mysql_mutex_t LOCK_server_started;
mysql_cond_t COND_server_started;
@@ -932,7 +938,8 @@ PSI_mutex_key key_PAGE_lock, key_LOCK_sync, key_LOCK_active, key_LOCK_pool,
key_LOCK_pending_checkpoint;
#endif /* HAVE_MMAP */
-PSI_mutex_key key_BINLOG_LOCK_index, key_BINLOG_LOCK_xid_list,
+PSI_mutex_key key_BINLOG_LOCK_index, key_BINLOG_LOCK_binlog_use,
+ key_BINLOG_LOCK_xid_list,
key_BINLOG_LOCK_binlog_background_thread,
key_LOCK_binlog_end_pos,
key_delayed_insert_mutex, key_hash_filo_lock, key_LOCK_active_mi,
@@ -995,6 +1002,7 @@ static PSI_mutex_info all_server_mutexes[]=
#endif /* HAVE_des */
{ &key_BINLOG_LOCK_index, "MYSQL_BIN_LOG::LOCK_index", 0},
+ { &key_BINLOG_LOCK_binlog_use, "MYSQL_BIN_LOG::LOCK_binlog_use", 0},
{ &key_BINLOG_LOCK_xid_list, "MYSQL_BIN_LOG::LOCK_xid_list", 0},
{ &key_BINLOG_LOCK_binlog_background_thread, "MYSQL_BIN_LOG::LOCK_binlog_background_thread", 0},
{ &key_LOCK_binlog_end_pos, "MYSQL_BIN_LOG::LOCK_binlog_end_pos", 0 },
@@ -1097,7 +1105,7 @@ static PSI_rwlock_info all_server_rwlocks[]=
PSI_cond_key key_PAGE_cond, key_COND_active, key_COND_pool;
#endif /* HAVE_MMAP */
-PSI_cond_key key_BINLOG_COND_xid_list,
+PSI_cond_key key_BINLOG_COND_binlog_use, key_BINLOG_COND_xid_list,
key_BINLOG_COND_bin_log_updated, key_BINLOG_COND_relay_log_updated,
key_BINLOG_COND_binlog_background_thread,
key_BINLOG_COND_binlog_background_thread_end,
@@ -1134,6 +1142,7 @@ static PSI_cond_info all_server_conds[]=
{ &key_TC_LOG_MMAP_COND_queue_busy, "TC_LOG_MMAP::COND_queue_busy", 0},
#endif /* HAVE_MMAP */
{ &key_BINLOG_COND_bin_log_updated, "MYSQL_BIN_LOG::COND_bin_log_updated", 0}, { &key_BINLOG_COND_relay_log_updated, "MYSQL_BIN_LOG::COND_relay_log_updated", 0},
+ { &key_BINLOG_COND_binlog_use, "MYSQL_BIN_LOG::COND_binlog_use", 0},
{ &key_BINLOG_COND_xid_list, "MYSQL_BIN_LOG::COND_xid_list", 0},
{ &key_BINLOG_COND_binlog_background_thread, "MYSQL_BIN_LOG::COND_binlog_background_thread", 0},
{ &key_BINLOG_COND_binlog_background_thread_end, "MYSQL_BIN_LOG::COND_binlog_background_thread_end", 0},
@@ -1492,6 +1501,7 @@ my_bool plugins_are_initialized= FALSE;
#ifndef DBUG_OFF
static const char* default_dbug_option;
+bool is_in_ddl_recovery= false;
#endif
#ifdef HAVE_LIBWRAP
const char *libwrapName= NULL;
@@ -1984,6 +1994,8 @@ static void clean_up(bool print_message)
injector::free_instance();
mysql_bin_log.cleanup();
Gtid_index_writer::gtid_index_cleanup();
+ if (opt_binlog_engine_plugin)
+ plugin_unlock(0, opt_binlog_engine_plugin);
my_tz_free();
my_dboptions_cache_free();
@@ -2116,7 +2128,11 @@ static void wait_for_signal_thread_to_end()
{
sql_print_warning("Signal handler thread did not exit in a timely manner. "
"Continuing to wait for it to stop..");
+ }
+ if (signal_thread_needs_join)
+ {
pthread_join(signal_thread, NULL);
+ signal_thread_needs_join= false;
}
#endif
}
@@ -3215,6 +3231,7 @@ static void start_signal_handler(void)
error,errno);
exit(1);
}
+ signal_thread_needs_join= true;
mysql_cond_wait(&COND_start_thread, &LOCK_start_thread);
mysql_mutex_unlock(&LOCK_start_thread);
@@ -4861,9 +4878,40 @@ static int adjust_optimizer_costs(const LEX_CSTRING *, OPTIMIZER_COSTS *oc, TABL
{ option, OPT_REMOVED_OPTION, \
0, 0, 0, 0, GET_STR, OPT_ARG, 0, 0, 0, 0, 0, 0 }
+
+static int
+create_dir_path_if_needed(const char *dir)
+{
+ MY_STAT stat_buf;
+ char buf[FN_REFLEN];
+ char *end= strmake(buf, dir, FN_REFLEN-1);
+ size_t len= dirname_length(buf);
+ if (len > 0 && end == buf + len)
+ {
+ /* Ends in trailing '/', strip it. */
+ buf[len-1]= '\0';
+ len= dirname_length(buf);
+ }
+ if (my_stat(dir, &stat_buf, MYF(0)))
+ return 0; // Already exists
+ if (len > 1)
+ {
+ /* Create any parent directory as well. */
+ strmake(buf, dir, len);
+ if (create_dir_path_if_needed(buf))
+ return 1;
+ }
+ if(my_mkdir(dir, 0777, MYF(MY_WME)) && my_errno != EEXIST)
+ return 1;
+ return 0;
+}
+
+
static int init_server_components()
{
DBUG_ENTER("init_server_components");
+ bool binlog_engine_used= false;
+
/*
We need to call each of these following functions to ensure that
all things are initialized so that unireg_abort() doesn't fail
@@ -5036,32 +5084,62 @@ static int init_server_components()
if (opt_bin_log)
{
+ if (opt_binlog_storage_engine && *opt_binlog_storage_engine)
+ binlog_engine_used= true;
+
/* Reports an error and aborts, if the --log-bin's path
is a directory.*/
if (opt_bin_logname[0] &&
opt_bin_logname[strlen(opt_bin_logname) - 1] == FN_LIBCHAR)
{
sql_print_error("Path '%s' is a directory name, please specify "
- "a file name for --log-bin option", opt_bin_logname);
+ "a file name for --log-bin option, or use "
+ "--binlog-directory", opt_bin_logname);
unireg_abort(1);
}
- /* Reports an error and aborts, if the --log-bin-index's path
- is a directory.*/
- if (opt_binlog_index_name &&
- opt_binlog_index_name[strlen(opt_binlog_index_name) - 1]
- == FN_LIBCHAR)
+ if (!binlog_engine_used)
{
- sql_print_error("Path '%s' is a directory name, please specify "
- "a file name for --log-bin-index option",
- opt_binlog_index_name);
- unireg_abort(1);
+ /* Reports an error and aborts, if the --log-bin-index's path
+ is a directory.*/
+ if (opt_binlog_index_name &&
+ opt_binlog_index_name[strlen(opt_binlog_index_name) - 1]
+ == FN_LIBCHAR)
+ {
+ sql_print_error("Path '%s' is a directory name, please specify "
+ "a file name for --log-bin-index option",
+ opt_binlog_index_name);
+ unireg_abort(1);
+ }
}
- char buf[FN_REFLEN];
+ char buf[FN_REFLEN], buf2[FN_REFLEN];
const char *ln;
ln= mysql_bin_log.generate_name(opt_bin_logname, "-bin", 1, buf);
- if (!opt_bin_logname[0] && !opt_binlog_index_name)
+ /* Add in opt_binlog_directory, if given. */
+ if (opt_binlog_directory && opt_binlog_directory[0])
+ {
+ if (strlen(opt_binlog_directory) + 1 + strlen(ln) + 1 > FN_REFLEN)
+ {
+ sql_print_error("The combination of --binlog-directory path '%s' with "
+ "filename '%s' from --log-bin results in a too long "
+ "path", opt_binlog_directory, ln);
+ unireg_abort(1);
+ }
+ if (create_dir_path_if_needed(opt_binlog_directory))
+ {
+ sql_print_error("Failed to create the directory '%s' specified in "
+ "--binlog-directory, error code: %d",
+ opt_binlog_directory, my_errno);
+ unireg_abort(1);
+ }
+ const char *end= &buf2[FN_REFLEN-1];
+ char *p= strmake(buf2, opt_binlog_directory, FN_REFLEN - 2);
+ *p++= FN_LIBCHAR;
+ strmake(p, ln, end - p - 1);
+ ln= buf2;
+ }
+ if (!binlog_engine_used && !opt_bin_logname[0] && !opt_binlog_index_name)
{
/*
User didn't give us info to name the binlog index file.
@@ -5080,6 +5158,8 @@ static int init_server_components()
}
if (ln == buf)
opt_bin_logname= my_once_strdup(buf, MYF(MY_WME));
+ else if (ln == buf2)
+ opt_bin_logname= my_once_strdup(buf2, MYF(MY_WME));
}
/*
@@ -5112,6 +5192,12 @@ static int init_server_components()
if (WSREP_ON && !wsrep_recovery && !opt_abort)
{
+ if (binlog_engine_used)
+ {
+ sql_print_error("Galera cannot be used with the "
+ "--binlog-storage-engine option");
+ unireg_abort(1);
+ }
if (opt_bootstrap) // bootsrap option given - disable wsrep functionality
{
wsrep_provider_init(WSREP_NONE);
@@ -5144,7 +5230,7 @@ static int init_server_components()
}
#endif /* WITH_WSREP */
- if (!opt_help && opt_bin_log)
+ if (!opt_help && !binlog_engine_used && opt_bin_log)
{
if (mysql_bin_log.open_index_file(opt_binlog_index_name, opt_bin_logname,
TRUE))
@@ -5464,6 +5550,71 @@ static int init_server_components()
if (init_gtid_pos_auto_engines())
unireg_abort(1);
+ if (opt_binlog_directory && opt_binlog_directory[0] &&
+ opt_bin_log_path)
+ {
+ sql_print_error("Cannot specify a directory path for the binlog in "
+ "--log-bin when --binlog-directory-path is also used");
+ unireg_abort(1);
+ }
+
+ if (binlog_engine_used)
+ {
+ LEX_CSTRING name= { opt_binlog_storage_engine, strlen(opt_binlog_storage_engine) };
+ opt_binlog_engine_plugin= ha_resolve_by_name(0, &name, false);
+ if (!opt_binlog_engine_plugin ||
+ !ha_storage_engine_is_enabled(opt_binlog_engine_hton=
+ plugin_hton(opt_binlog_engine_plugin)))
+ {
+ if (!opt_binlog_engine_plugin)
+ sql_print_error("Unknown/unsupported storage engine: %s",
+ opt_binlog_storage_engine);
+ else
+ sql_print_error("Engine %s is not available for "
+ "--binlog-storage-engine",
+ opt_binlog_storage_engine);
+ unireg_abort(1);
+ }
+ if (!opt_binlog_engine_hton->binlog_write_direct ||
+ !opt_binlog_engine_hton->get_binlog_reader)
+ {
+ sql_print_error("Engine %s does not support --binlog-storage-engine",
+ opt_binlog_storage_engine);
+ unireg_abort(1);
+ }
+
+ if (opt_bin_log_nonempty)
+ {
+ sql_print_error("Binlog name can not be set with --log-bin=NAME when "
+ "--binlog-storage-engine is used. Use --log-bin "
+ "(without argument) to enable the binlog, and use "
+ "--binlog-directory to specify a separate directory "
+ "for binlogs");
+ unireg_abort(1);
+ }
+#ifdef HAVE_REPLICATION
+ if (rpl_semi_sync_master_enabled)
+ {
+ sql_print_error("Semi-synchronous replication is not yet supported "
+ "with --binlog-storage-engine");
+ unireg_abort(1);
+ }
+ if (rpl_status != RPL_AUTH_MASTER)
+ {
+ sql_print_error("The --init-rpl-role option is not available with "
+ "--binlog-storage-engine");
+ unireg_abort(1);
+ }
+ if (encrypt_binlog)
+ {
+ sql_print_error("Binlog encryption is not available with "
+ "--binlog-storage-engine. Using full-disk encryption on "
+ "the operating system level is recommended instead");
+ unireg_abort(1);
+ }
+#endif
+ }
+
#ifdef USE_ARIA_FOR_TMP_TABLES
if (!ha_storage_engine_is_enabled(maria_hton) && !opt_bootstrap)
{
@@ -5496,6 +5647,13 @@ static int init_server_components()
start_handle_manager();
#endif
+ /*
+ When binlog is stored in InnoDB, checksums are done on the page level, so
+ set the default for per-event checksums to OFF.
+ */
+ if (opt_binlog_engine_hton)
+ binlog_checksum_options= 0;
+
tc_log= get_tc_log_implementation();
if (tc_log->open(opt_bin_log ? opt_bin_logname : opt_tc_log_file))
@@ -5509,11 +5667,19 @@ static int init_server_components()
if (opt_bin_log)
{
- int error;
mysql_mutex_t *log_lock= mysql_bin_log.get_log_lock();
+ bool error;
mysql_mutex_lock(log_lock);
- error= mysql_bin_log.open(opt_bin_logname, 0, 0,
- WRITE_CACHE, max_binlog_size, 0, TRUE);
+ if (opt_binlog_engine_hton)
+ {
+ error= mysql_bin_log.open_engine(opt_binlog_engine_hton, max_binlog_size,
+ opt_binlog_directory);
+ }
+ else
+ {
+ error= mysql_bin_log.open(opt_bin_logname, 0, 0,
+ WRITE_CACHE, max_binlog_size, 0, TRUE);
+ }
mysql_mutex_unlock(log_lock);
if (unlikely(error))
unireg_abort(1);
@@ -5526,9 +5692,12 @@ static int init_server_components()
if (opt_bin_log)
{
- if (binlog_space_limit)
- mysql_bin_log.count_binlog_space_with_mutex();
- mysql_bin_log.purge(1);
+ if (!opt_binlog_engine_hton)
+ {
+ if (binlog_space_limit)
+ mysql_bin_log.count_binlog_space_with_mutex();
+ mysql_bin_log.purge(1);
+ }
}
else
{
@@ -5542,8 +5711,14 @@ static int init_server_components()
}
#endif
+#ifndef DBUG_OFF
+ is_in_ddl_recovery= true;
+#endif
if (ddl_log_execute_recovery() > 0)
unireg_abort(1);
+#ifndef DBUG_OFF
+ is_in_ddl_recovery= false;
+#endif
ha_signal_ddl_recovery_done();
if (opt_myisam_log)
@@ -6609,9 +6784,6 @@ struct my_option my_long_options[]=
&debug_assert_on_not_freed_memory, 0, GET_BOOL, NO_ARG, 1, 0, 0, 0, 0,
0},
#endif /* DBUG_OFF */
- /* default-storage-engine should have "MyISAM" as def_value. Instead
- of initializing it here it is done in init_common_variables() due
- to a compiler bug in Sun Studio compiler. */
{"default-storage-engine", 0, "The default storage engine for new tables",
&default_storage_engine, 0, 0, GET_STR, REQUIRED_ARG,
0, 0, 0, 0, 0, 0 },
@@ -8216,6 +8388,9 @@ mysqld_get_one_option(const struct my_option *opt, const char *argument,
case (int) OPT_BIN_LOG:
opt_bin_log= MY_TEST(argument != disabled_my_option);
opt_bin_log_used= 1;
+ opt_bin_log_nonempty= (argument && argument[0]);
+ opt_bin_log_path= argument &&
+ (strchr(argument, FN_LIBCHAR) || strchr(argument, FN_LIBCHAR2));
break;
case (int) OPT_LOG_BASENAME:
{
@@ -9488,6 +9663,7 @@ PSI_stage_info stage_waiting_for_deadlock_kill= { 0, "Waiting for parallel repli
PSI_stage_info stage_starting= { 0, "starting", 0};
PSI_stage_info stage_waiting_for_flush= { 0, "Waiting for non trans tables to be flushed", 0};
PSI_stage_info stage_waiting_for_ddl= { 0, "Waiting for DDLs", 0};
+PSI_stage_info stage_waiting_for_reset_master= { 0, "Waiting for a running RESET MASTER to complete", 0};
#ifdef WITH_WSREP
// Aditional Galera thread states
@@ -9717,7 +9893,8 @@ PSI_stage_info *all_server_stages[]=
& stage_waiting_for_semi_sync_slave,
& stage_reading_semi_sync_ack,
& stage_waiting_for_deadlock_kill,
- & stage_starting
+ & stage_starting,
+ & stage_waiting_for_reset_master
#ifdef WITH_WSREP
,
& stage_waiting_isolation,
diff --git a/sql/mysqld.h b/sql/mysqld.h
index ff3061647163b..b785434d6864e 100644
--- a/sql/mysqld.h
+++ b/sql/mysqld.h
@@ -104,7 +104,10 @@ void temp_pool_clear_bit(uint bit);
uint temp_pool_set_next();
extern bool opt_large_files;
-extern bool opt_update_log, opt_bin_log, opt_error_log, opt_bin_log_compress;
+extern bool opt_update_log, opt_bin_log, opt_error_log, opt_bin_log_compress;
+extern char *opt_binlog_storage_engine;
+extern const char *opt_binlog_directory;
+extern handlerton *opt_binlog_engine_hton;
extern uint opt_bin_log_compress_min_len;
extern my_bool opt_log, opt_bootstrap;
extern my_bool opt_backup_history_log;
@@ -318,6 +321,9 @@ extern const char *encryption_algorithm_names[];
extern long opt_secure_timestamp;
extern uint default_password_lifetime;
extern my_bool disconnect_on_expired_password;
+#ifndef DBUG_OFF
+extern bool is_in_ddl_recovery;
+#endif
enum secure_timestamp { SECTIME_NO, SECTIME_SUPER, SECTIME_REPL, SECTIME_YES };
bool is_set_timestamp_forbidden(THD *thd);
@@ -327,7 +333,8 @@ extern PSI_mutex_key key_PAGE_lock, key_LOCK_sync, key_LOCK_active,
key_LOCK_pool, key_LOCK_pending_checkpoint;
#endif /* HAVE_MMAP */
-extern PSI_mutex_key key_BINLOG_LOCK_index, key_BINLOG_LOCK_xid_list,
+extern PSI_mutex_key key_BINLOG_LOCK_index, key_BINLOG_LOCK_binlog_use,
+ key_BINLOG_LOCK_xid_list,
key_BINLOG_LOCK_binlog_background_thread,
key_LOCK_binlog_end_pos,
key_delayed_insert_mutex, key_hash_filo_lock, key_LOCK_active_mi,
@@ -374,7 +381,8 @@ extern PSI_rwlock_key key_rwlock_LOCK_grant, key_rwlock_LOCK_logger,
extern PSI_cond_key key_PAGE_cond, key_COND_active, key_COND_pool;
#endif /* HAVE_MMAP */
-extern PSI_cond_key key_BINLOG_COND_xid_list, key_BINLOG_update_cond,
+extern PSI_cond_key key_BINLOG_COND_binlog_use,
+ key_BINLOG_COND_xid_list, key_BINLOG_update_cond,
key_BINLOG_COND_binlog_background_thread,
key_BINLOG_COND_binlog_background_thread_end,
key_COND_cache_status_changed, key_COND_manager,
@@ -704,6 +712,7 @@ extern PSI_stage_info stage_slave_background_process_request;
extern PSI_stage_info stage_slave_background_wait_request;
extern PSI_stage_info stage_waiting_for_deadlock_kill;
extern PSI_stage_info stage_starting;
+extern PSI_stage_info stage_waiting_for_reset_master;
#ifdef WITH_WSREP
// Aditional Galera thread states
extern PSI_stage_info stage_waiting_isolation;
diff --git a/sql/online_alter.cc b/sql/online_alter.cc
index 2c5eb4dea6236..8828c92b91258 100644
--- a/sql/online_alter.cc
+++ b/sql/online_alter.cc
@@ -42,7 +42,7 @@ class online_alter_cache_data: public ilist_node<>,
public binlog_cache_data
{
public:
- online_alter_cache_data() : binlog_cache_data(false),
+ online_alter_cache_data() : binlog_cache_data(false, false),
hton(nullptr), sink_log(nullptr) { }
void store_prev_position()
{
diff --git a/sql/rpl_gtid.cc b/sql/rpl_gtid.cc
index 7b210d7c21acf..47e5f382095ed 100644
--- a/sql/rpl_gtid.cc
+++ b/sql/rpl_gtid.cc
@@ -29,6 +29,7 @@
#include "rpl_rli.h"
#include "slave.h"
#include "log_event.h"
+#include "transaction.h"
const LEX_CSTRING rpl_gtid_slave_state_table_name=
{ STRING_WITH_LEN("gtid_slave_pos") };
@@ -457,7 +458,7 @@ rpl_slave_state::truncate_state_table(THD *thd)
{
ha_commit_trans(thd, FALSE);
close_thread_tables(thd);
- ha_commit_trans(thd, TRUE);
+ trans_commit(thd);
}
thd->release_transactional_locks();
}
@@ -1571,28 +1572,11 @@ rpl_binlog_state_base::load_nolock(struct rpl_gtid *list, uint32 count)
bool
rpl_binlog_state_base::load_nolock(rpl_binlog_state_base *orig_state)
{
- ulong i, j;
- HASH *h1= &orig_state->hash;
-
reset_nolock();
- for (i= 0; i < h1->records; ++i)
- {
- element *e= (element *)my_hash_element(h1, i);
- HASH *h2= &e->hash;
- const rpl_gtid *last_gtid= e->last_gtid;
- for (j= 0; j < h2->records; ++j)
- {
- const rpl_gtid *gtid= (const rpl_gtid *)my_hash_element(h2, j);
- if (gtid == last_gtid)
- continue;
- if (update_nolock(gtid))
- return true;
- }
- if (likely(last_gtid) && update_nolock(last_gtid))
- return true;
- }
-
- return false;
+ return orig_state->iterate(
+ [this] (const rpl_gtid *gtid) {
+ return update_nolock(gtid);
+ });
}
@@ -1681,36 +1665,14 @@ rpl_binlog_state_base::count_nolock()
int
rpl_binlog_state_base::get_gtid_list_nolock(rpl_gtid *gtid_list, uint32 list_size)
{
- uint32 i, j, pos;
-
- pos= 0;
- for (i= 0; i < hash.records; ++i)
- {
- element *e= (element *)my_hash_element(&hash, i);
- if (!e->last_gtid)
- {
- DBUG_ASSERT(e->hash.records==0);
- continue;
- }
- for (j= 0; j <= e->hash.records; ++j)
- {
- const rpl_gtid *gtid;
- if (j < e->hash.records)
- {
- gtid= (rpl_gtid *)my_hash_element(&e->hash, j);
- if (gtid == e->last_gtid)
- continue;
- }
- else
- gtid= e->last_gtid;
-
+ uint32_t pos= 0;
+ return iterate(
+ [>id_list, list_size, &pos] (const rpl_gtid *gtid) {
if (pos >= list_size)
- return 1;
+ return true;
memcpy(>id_list[pos++], gtid, sizeof(*gtid));
- }
- }
-
- return 0;
+ return false;
+ });
}
@@ -2061,42 +2023,17 @@ rpl_binlog_state::bump_seq_no_if_needed(uint32 domain_id, uint64 seq_no)
int
rpl_binlog_state::write_to_iocache(IO_CACHE *dest)
{
- ulong i, j;
char buf[21];
- int res= 0;
mysql_mutex_lock(&LOCK_binlog_state);
- for (i= 0; i < hash.records; ++i)
- {
- element *e= (element *)my_hash_element(&hash, i);
- if (!e->last_gtid)
- {
- DBUG_ASSERT(e->hash.records == 0);
- continue;
- }
- for (j= 0; j <= e->hash.records; ++j)
- {
- const rpl_gtid *gtid;
- if (j < e->hash.records)
- {
- gtid= (const rpl_gtid *)my_hash_element(&e->hash, j);
- if (gtid == e->last_gtid)
- continue;
- }
- else
- gtid= e->last_gtid;
- longlong10_to_str(gtid->seq_no, buf, 10);
- if (my_b_printf(dest, "%u-%u-%s\n", gtid->domain_id, gtid->server_id,
- buf))
- {
- res= 1;
- goto end;
- }
- }
- }
+ int res= iterate([&buf, dest] (const rpl_gtid *gtid) {
+ longlong10_to_str(gtid->seq_no, buf, 10);
+ if (my_b_printf(dest, "%u-%u-%s\n", gtid->domain_id, gtid->server_id, buf))
+ return true;
+ return false;
+ });
-end:
mysql_mutex_unlock(&LOCK_binlog_state);
return res;
}
@@ -2248,43 +2185,18 @@ rpl_binlog_state::append_pos(String *str)
bool
rpl_binlog_state::append_state(String *str)
{
- uint32 i, j;
- bool res= false;
-
mysql_mutex_lock(&LOCK_binlog_state);
reset_dynamic(>id_sort_array);
- for (i= 0; i < hash.records; ++i)
- {
- element *e= (element *)my_hash_element(&hash, i);
- if (!e->last_gtid)
- {
- DBUG_ASSERT(e->hash.records==0);
- continue;
- }
- for (j= 0; j <= e->hash.records; ++j)
- {
- const rpl_gtid *gtid;
- if (j < e->hash.records)
- {
- gtid= (rpl_gtid *)my_hash_element(&e->hash, j);
- if (gtid == e->last_gtid)
- continue;
- }
- else
- gtid= e->last_gtid;
-
- if (insert_dynamic(>id_sort_array, (const void *) gtid))
- {
- res= true;
- goto end;
- }
- }
- }
+ bool res= iterate([this] (const rpl_gtid *gtid) {
+ if (insert_dynamic(>id_sort_array, (const void *) gtid))
+ return true;
+ return false;
+ });
- rpl_slave_state_tostring_helper(>id_sort_array, str);
+ if (likely(!res))
+ rpl_slave_state_tostring_helper(>id_sort_array, str);
-end:
mysql_mutex_unlock(&LOCK_binlog_state);
return res;
}
@@ -2292,11 +2204,11 @@ rpl_binlog_state::append_state(String *str)
/**
Remove domains supplied by the first argument from binlog state.
Removal is done for any domain whose last gtids (from all its servers) match
- ones in Gtid list event of the 2nd argument.
+ ones in the binlog state at the start of the current binlog, passed in as the
+ 2nd argument.
@param ids gtid domain id sequence, may contain dups
- @param glev pointer to Gtid list event describing
- the match condition
+ @param init_state Binlog state at the start of the current binlog
@param errbuf [out] pointer to possible error message array
@retval NULL as success when at least one domain is removed
@@ -2306,12 +2218,12 @@ rpl_binlog_state::append_state(String *str)
*/
const char*
rpl_binlog_state::drop_domain(DYNAMIC_ARRAY *ids,
- Gtid_list_log_event *glev,
+ rpl_binlog_state_base *init_state,
char* errbuf)
{
DYNAMIC_ARRAY domain_unique; // sequece (unsorted) of unique element*:s
rpl_binlog_state::element* domain_unique_buffer[16];
- ulong k, l;
+ ulong k;
const char* errmsg= NULL;
DBUG_ENTER("rpl_binlog_state::drop_domain");
@@ -2338,45 +2250,46 @@ rpl_binlog_state::drop_domain(DYNAMIC_ARRAY *ids,
B and C may require the user's attention so any (incl the A's suspected)
inconsistency is diagnosed and *warned*.
*/
- for (l= 0, errbuf[0]= 0; l < glev->count; l++, errbuf[0]= 0)
- {
- rpl_gtid* rb_state_gtid= find_nolock(glev->list[l].domain_id,
- glev->list[l].server_id);
+
+ errbuf[0]= 0;
+ init_state->iterate([this, errbuf](const rpl_gtid *gtid) {
+ rpl_gtid* rb_state_gtid= find_nolock(gtid->domain_id, gtid->server_id);
if (!rb_state_gtid)
sprintf(errbuf,
"missing gtids from the '%u-%u' domain-server pair which is "
"referred to in the gtid list describing an earlier state. Ignore "
"if the domain ('%u') was already explicitly deleted",
- glev->list[l].domain_id, glev->list[l].server_id,
- glev->list[l].domain_id);
- else if (rb_state_gtid->seq_no < glev->list[l].seq_no)
+ gtid->domain_id, gtid->server_id,
+ gtid->domain_id);
+ else if (rb_state_gtid->seq_no < gtid->seq_no)
sprintf(errbuf,
"having a gtid '%u-%u-%llu' which is less than "
"the '%u-%u-%llu' of the gtid list describing an earlier state. "
"The state may have been affected by manually injecting "
"a lower sequence number gtid or via replication",
rb_state_gtid->domain_id, rb_state_gtid->server_id,
- rb_state_gtid->seq_no, glev->list[l].domain_id,
- glev->list[l].server_id, glev->list[l].seq_no);
+ rb_state_gtid->seq_no, gtid->domain_id,
+ gtid->server_id, gtid->seq_no);
if (strlen(errbuf)) // use strlen() as cheap flag
push_warning_printf(current_thd, Sql_condition::WARN_LEVEL_WARN,
ER_BINLOG_CANT_DELETE_GTID_DOMAIN,
"The current gtid binlog state is incompatible with "
"a former one %s.", errbuf);
- }
+ errbuf[0]= 0;
+ return false; // No error
+ });
/*
For each domain_id from ids
If the domain is already absent from the binlog state
Warn && continue
- If any GTID with that domain in binlog state is missing from glev.list
+ If any GTID with that domain in binlog state is missing from init_state
Error out binlog state can't change
*/
for (ulong i= 0; i < ids->elements; i++)
{
rpl_binlog_state::element *elem= NULL;
uint32 *ptr_domain_id;
- bool all_found;
ptr_domain_id= (uint32*) dynamic_array_ptr(ids, i);
elem= (rpl_binlog_state::element *)
@@ -2391,25 +2304,21 @@ rpl_binlog_state::drop_domain(DYNAMIC_ARRAY *ids,
continue;
}
- all_found= true;
- for (k= 0; k < elem->hash.records && all_found; k++)
+ for (k= 0; k < elem->hash.records; k++)
{
rpl_gtid *d_gtid= (rpl_gtid *)my_hash_element(&elem->hash, k);
- bool match_found= false;
- for (ulong l= 0; l < glev->count && !match_found; l++)
- match_found= match_found || (*d_gtid == glev->list[l]);
- if (!match_found)
- all_found= false;
+ rpl_gtid *state_gtid=
+ init_state->find_nolock(d_gtid->domain_id, d_gtid->server_id);
+ if (!state_gtid || state_gtid->seq_no != d_gtid->seq_no)
+ {
+ sprintf(errbuf, "binlog files may contain gtids from the domain ('%u') "
+ "being deleted. Make sure to first purge those files",
+ *ptr_domain_id);
+ errmsg= errbuf;
+ goto end;
+ }
}
- if (!all_found)
- {
- sprintf(errbuf, "binlog files may contain gtids from the domain ('%u') "
- "being deleted. Make sure to first purge those files",
- *ptr_domain_id);
- errmsg= errbuf;
- goto end;
- }
// compose a sequence of unique pointers to domain object
for (k= 0; k < domain_unique.elements; k++)
{
diff --git a/sql/rpl_gtid.h b/sql/rpl_gtid.h
index fe8e3751dbe26..f2e5c7cbc8440 100644
--- a/sql/rpl_gtid.h
+++ b/sql/rpl_gtid.h
@@ -16,9 +16,10 @@
#ifndef RPL_GTID_H
#define RPL_GTID_H
-#include "hash.h"
#include "queues.h"
#include
+#include "rpl_gtid_base.h"
+
/* Definitions for MariaDB global transaction ID (GTID). */
@@ -29,19 +30,9 @@ class String;
#ifdef MYSQL_SERVER
struct TABLE;
#endif
-struct slave_connection_state;
-
-#define PARAM_GTID(G) G.domain_id, G.server_id, G.seq_no
#define GTID_MAX_STR_LENGTH (10+1+10+1+20)
-#define PARAM_GTID(G) G.domain_id, G.server_id, G.seq_no
-
-struct rpl_gtid
-{
- uint32 domain_id;
- uint32 server_id;
- uint64 seq_no;
-};
+#define PARAM_GTID(G) (G).domain_id, (G).server_id, (G).seq_no
inline bool operator==(const rpl_gtid& lhs, const rpl_gtid& rhs)
{
@@ -307,38 +298,6 @@ struct rpl_slave_state
rpl_binlog_state builds server logic on top of that like mutex locking,
gtid_strict_mode handling, etc.
*/
-struct rpl_binlog_state_base
-{
- struct element {
- uint32 domain_id;
- HASH hash; /* Containing all server_id for one domain_id */
- /* The most recent entry in the hash. */
- rpl_gtid *last_gtid;
- /* Counter to allocate next seq_no for this domain. */
- uint64 seq_no_counter;
-
- int update_element(const rpl_gtid *gtid);
- };
-
- /* Mapping from domain_id to collection of elements. */
- HASH hash;
- my_bool initialized;
-
- rpl_binlog_state_base() : initialized(0) {}
- ~rpl_binlog_state_base();
- void init();
- void reset_nolock();
- void free();
- bool load_nolock(struct rpl_gtid *list, uint32 count);
- bool load_nolock(rpl_binlog_state_base *orig_state);
- int update_nolock(const struct rpl_gtid *gtid);
- int alloc_element_nolock(const rpl_gtid *gtid);
- uint32 count_nolock();
- int get_gtid_list_nolock(rpl_gtid *gtid_list, uint32 list_size);
- rpl_gtid *find_nolock(uint32 domain_id, uint32 server_id);
- bool is_before_pos(slave_connection_state *pos);
-};
-
struct rpl_binlog_state : public rpl_binlog_state_base
{
/* Mutex protecting access to the state. */
@@ -370,7 +329,8 @@ struct rpl_binlog_state : public rpl_binlog_state_base
bool append_state(String *str);
rpl_gtid *find(uint32 domain_id, uint32 server_id);
rpl_gtid *find_most_recent(uint32 domain_id);
- const char* drop_domain(DYNAMIC_ARRAY *ids, Gtid_list_log_event *glev, char*);
+ const char* drop_domain(DYNAMIC_ARRAY *ids, rpl_binlog_state_base *init_state,
+ char*);
};
diff --git a/sql/rpl_mi.cc b/sql/rpl_mi.cc
index e0819a07f317f..020c6a804f8ec 100644
--- a/sql/rpl_mi.cc
+++ b/sql/rpl_mi.cc
@@ -36,7 +36,8 @@ Master_info::Master_info(LEX_CSTRING *connection_name_arg,
ssl(1), ssl_verify_server_cert(1), fd(-1), io_thd(0),
rli(is_slave_recovery), port(MYSQL_PORT),
checksum_alg_before_fd(BINLOG_CHECKSUM_ALG_UNDEF),
- connect_retry(DEFAULT_CONNECT_RETRY), inited(0), abort_slave(0),
+ connect_retry(DEFAULT_CONNECT_RETRY), binlog_storage_engine(0),
+ inited(0), abort_slave(0),
slave_running(MYSQL_SLAVE_NOT_RUN), slave_run_id(0),
clock_diff_with_master(0),
sync_counter(0), heartbeat_period(0), received_heartbeats(0),
diff --git a/sql/rpl_mi.h b/sql/rpl_mi.h
index 0b659d81f1152..83fb7c5b2d45d 100644
--- a/sql/rpl_mi.h
+++ b/sql/rpl_mi.h
@@ -262,6 +262,8 @@ class Master_info : public Slave_reporting_capability
bool dbug_do_disconnect;
int dbug_event_counter;
#endif
+ /* Whether the master is using --binlog-storage-engine. */
+ bool binlog_storage_engine;
bool inited;
volatile bool abort_slave;
volatile uint slave_running;
diff --git a/sql/rpl_rli.cc b/sql/rpl_rli.cc
index aab65d5282065..39bc48402b957 100644
--- a/sql/rpl_rli.cc
+++ b/sql/rpl_rli.cc
@@ -1992,7 +1992,7 @@ find_gtid_pos_tables_cb(THD *thd, LEX_CSTRING *table_name, void *arg)
if (table)
{
ha_commit_trans(thd, FALSE);
- ha_commit_trans(thd, TRUE);
+ trans_commit(thd);
close_thread_tables(thd);
thd->release_transactional_locks();
}
diff --git a/sql/share/errmsg-utf8.txt b/sql/share/errmsg-utf8.txt
index c056e3cacbfbe..a156fad80c857 100644
--- a/sql/share/errmsg-utf8.txt
+++ b/sql/share/errmsg-utf8.txt
@@ -12292,3 +12292,23 @@ ER_VECTOR_FORMAT_INVALID
eng "Invalid vector format at offset: %d for '%-.100s'. Must be a valid JSON array of numbers."
ER_PSEUDO_THREAD_ID_OVERWRITE
eng "Pseudo thread id should not be modified by the client as it will be overwritten"
+ER_BINLOG_IN_USE
+ eng "Cannot execute RESET MASTER as the binlog is in use by a connected slave or other RESET MASTER or binlog reader. Check SHOW PROCESSLIST for "Binlog Dump" commands and use KILL to stop such readers"
+ER_BINLOG_IN_USE_TRX
+ eng "Cannot execute RESET MASTER as the binlog is in use by an active transaction"
+ER_CANNOT_INIT_ENGINE_BINLOG_READER
+ eng "Cannot initialize binlog reader from storage engine %s"
+ER_ENGINE_BINLOG_REQUIRES_GTID
+ eng "GTID starting position is required on master with --binlog-storage-engine enabled"
+ER_NOT_AVAILABLE_WITH_ENGINE_BINLOG
+ eng "%s is not available when --binlog-storage-engine is enabled"
+ER_NOT_YET_SUPPORTED_ENGINE_BINLOG
+ eng "%s is not yet supported with --binlog-storage-engine"
+ER_ENGINE_BINLOG_NO_DELETE_DOMAIN
+ eng "The binlog engine does not support DELETE_DOMAIN_ID"
+ER_BINLOG_CANNOT_READ_STATE
+ eng "Error reading GTID state from the binlog"
+ER_BINLOG_POS_INVALID
+ eng "The binlog offset %llu is invalid"
+ER_READING_BINLOG_FILE
+ eng "Error reading from page number %u in binlog file (error code: %d)"
diff --git a/sql/slave.cc b/sql/slave.cc
index e8b7d044fc838..3ea8c638f23cd 100644
--- a/sql/slave.cc
+++ b/sql/slave.cc
@@ -2302,6 +2302,57 @@ when it try to get the value of TIME_ZONE global variable from master.";
}
}
+ /*
+ See if the master is using the new binlog format from
+ --binlog-storage-engine.
+ */
+ if (mysql_real_query(mysql,
+ STRING_WITH_LEN("SELECT @@GLOBAL.binlog_storage_engine")) ||
+ !(master_res= mysql_store_result(mysql)) ||
+ !(master_row= mysql_fetch_row(master_res)))
+ {
+ if (check_io_slave_killed(mi, NULL))
+ goto slave_killed_err;
+
+ err_code= mysql_errno(mysql);
+ if (is_network_error(err_code))
+ {
+ mi->report(ERROR_LEVEL, err_code, NULL,
+ "Checking master binlog format failed with error: %s",
+ mysql_error(mysql));
+ goto network_err;
+ }
+ else if (err_code == ER_UNKNOWN_SYSTEM_VARIABLE)
+ {
+ /*
+ The master is older than the slave and does not support
+ --binlog-storage-engine, so we know it is using the old format.
+ */
+ DBUG_PRINT("info", ("Old master, no --binlog-storage-engine"));
+ mi->binlog_storage_engine= false;
+ }
+ else
+ {
+ /* Fatal error */
+ errmsg= "The slave I/O thread stops because a fatal error is "
+ "encountered when it tries to query the value of "
+ "@@binlog_storage_engine.";
+ sprintf(err_buff, "%s Error: %s", errmsg, mysql_error(mysql));
+ goto err;
+ }
+ }
+ else
+ {
+ mi->binlog_storage_engine= (master_row[0] != NULL);
+ DBUG_PRINT("info", ("Master using --binlog-storage-engine: %d",
+ mi->binlog_storage_engine));
+ }
+ if (master_res)
+ {
+ mysql_free_result(master_res);
+ master_res= NULL;
+ }
+
/* Announce MariaDB slave capabilities. */
DBUG_EXECUTE_IF("simulate_slave_capability_none", goto after_set_capability;);
{
@@ -6463,9 +6514,9 @@ static int queue_event(Master_info* mi, const uchar *buf, ulong event_len)
goto err;
}
mi->received_heartbeats++;
- /*
+ /*
compare local and event's versions of log_file, log_pos.
-
+
Heartbeat is sent only after an event corresponding to the corrdinates
the heartbeat carries.
@@ -6483,13 +6534,19 @@ static int queue_event(Master_info* mi, const uchar *buf, ulong event_len)
Slave can have lower coordinates, if some event from master was omitted.
+ When the master is using new binlog format (--binlog-storage-engine),
+ then binlog coordinates are not meaningful (GTID is used always), the
+ slave does not track the master binlog coordinates, and the heartbeat
+ coordinates should just be ignored.
+
TODO: handling `when' for SHOW SLAVE STATUS' snds behind
TODO: Extend heartbeat events to use GTIDs instead of binlog
coordinates. This would alleviate the strange exceptions during log
rotation.
*/
- if (mi->master_log_pos &&
+ if (!mi->binlog_storage_engine &&
+ mi->master_log_pos &&
!memcmp(mi->master_log_name, hb.get_log_ident(), hb.get_ident_len()) &&
mi->master_log_pos > hb.log_pos)
{
@@ -7062,9 +7119,12 @@ static int queue_event(Master_info* mi, const uchar *buf, ulong event_len)
(uchar)buf[EVENT_TYPE_OFFSET] != STOP_EVENT))
{
mi->master_log_pos+= inc_pos;
- memcpy(rli->ign_master_log_name_end, mi->master_log_name, FN_REFLEN);
- DBUG_ASSERT(rli->ign_master_log_name_end[0]);
- rli->ign_master_log_pos_end= mi->master_log_pos;
+ if (!mi->binlog_storage_engine)
+ {
+ memcpy(rli->ign_master_log_name_end, mi->master_log_name, FN_REFLEN);
+ DBUG_ASSERT(rli->ign_master_log_name_end[0]);
+ rli->ign_master_log_pos_end= mi->master_log_pos;
+ }
if (got_gtid_event)
rli->ign_gtids.update(&event_gtid);
}
diff --git a/sql/sql_binlog.cc b/sql/sql_binlog.cc
index 118b361eb79b8..4b92c9589f0c5 100644
--- a/sql/sql_binlog.cc
+++ b/sql/sql_binlog.cc
@@ -57,21 +57,6 @@ static int check_event_type(int type, Relay_log_info *rli)
{
case START_EVENT_V3:
case FORMAT_DESCRIPTION_EVENT:
- /*
- We need a preliminary FD event in order to parse the FD event,
- if we don't already have one.
- */
- if (!fd_event)
- if (!(rli->relay_log.description_event_for_exec=
- new Format_description_log_event(4)))
- {
- my_error(ER_OUTOFMEMORY, MYF(0), 1);
- return 1;
- }
-
- /* It is always allowed to execute FD events. */
- return 0;
-
case QUERY_EVENT:
case TABLE_MAP_EVENT:
case WRITE_ROWS_EVENT_V1:
@@ -83,19 +68,7 @@ static int check_event_type(int type, Relay_log_info *rli)
case PRE_GA_WRITE_ROWS_EVENT:
case PRE_GA_UPDATE_ROWS_EVENT:
case PRE_GA_DELETE_ROWS_EVENT:
- /*
- Row events are only allowed if a Format_description_event has
- already been seen.
- */
- if (fd_event)
- return 0;
- else
- {
- my_error(ER_NO_FORMAT_DESCRIPTION_EVENT_BEFORE_BINLOG_STATEMENT,
- MYF(0), Log_event::get_type_str((Log_event_type)type));
- return 1;
- }
- break;
+ return 0;
default:
/*
@@ -328,6 +301,19 @@ void mysql_client_binlog_statement(THD* thd)
else if (bytes_decoded == 0)
break; // If no bytes where read, the string contained only whitespace
+ /*
+ Create a default format description event.
+ This is used to read the real Format_description_log_event, or to read
+ all events if there is none (as happens with --binlog-storage-engine).
+ */
+ if (!rli->relay_log.description_event_for_exec &&
+ !(rli->relay_log.description_event_for_exec=
+ new Format_description_log_event(4)))
+ {
+ my_error(ER_OUT_OF_RESOURCES, MYF(0));
+ goto end;
+ }
+
DBUG_ASSERT(bytes_decoded > 0);
DBUG_ASSERT(endptr > strptr);
coded_len-= endptr - strptr;
diff --git a/sql/sql_class.h b/sql/sql_class.h
index f3b18330f4f3d..64005e70a70e1 100644
--- a/sql/sql_class.h
+++ b/sql/sql_class.h
@@ -68,7 +68,7 @@ void set_thd_stage_info(void *thd,
(thd)->enter_stage(&stage, __func__, __FILE__, __LINE__)
#include "my_apc.h"
-#include "rpl_gtid.h"
+#include "rpl_gtid_base.h"
#include "wsrep.h"
#include "wsrep_on.h"
@@ -5539,7 +5539,7 @@ class THD: public THD_count, /* this must be first */
rpl_gtid m_last_commit_gtid;
public:
- rpl_gtid get_last_commit_gtid() { return m_last_commit_gtid; }
+ const rpl_gtid *get_last_commit_gtid() { return &m_last_commit_gtid; }
void set_last_commit_gtid(rpl_gtid >id);
diff --git a/sql/sql_parse.cc b/sql/sql_parse.cc
index b901918e3e29c..0e128473cbc84 100644
--- a/sql/sql_parse.cc
+++ b/sql/sql_parse.cc
@@ -4102,7 +4102,13 @@ mysql_execute_command(THD *thd, bool is_called_from_prepared_stmt)
WSREP_SYNC_WAIT(thd, WSREP_SYNC_WAIT_BEFORE_SHOW);
if (check_global_access(thd, PRIV_STMT_SHOW_BINLOG_EVENTS))
goto error;
+ if (mysql_bin_log.start_use_binlog(thd))
+ {
+ my_error(thd->killed_errno(), MYF(0));
+ goto error;
+ }
res = mysql_show_binlog_events(thd);
+ mysql_bin_log.end_use_binlog(thd);
break;
}
#endif
diff --git a/sql/sql_reload.cc b/sql/sql_reload.cc
index d1ec12f16fa7a..76361bb864495 100644
--- a/sql/sql_reload.cc
+++ b/sql/sql_reload.cc
@@ -174,18 +174,37 @@ bool reload_acl_and_cache(THD *thd, unsigned long long options,
tmp_write_to_binlog= 0;
if (mysql_bin_log.is_open())
{
- DYNAMIC_ARRAY *drop_gtid_domain=
- (thd && (thd->lex->delete_gtid_domain.elements > 0)) ?
- &thd->lex->delete_gtid_domain : NULL;
- if (mysql_bin_log.rotate_and_purge(true, drop_gtid_domain))
- *write_to_binlog= -1;
-
- /* Note that WSREP(thd) might not be true here e.g. during
- SST. */
- if (WSREP_ON)
+ MDL_request mdl_request;
+ MDL_REQUEST_INIT(&mdl_request, MDL_key::BACKUP, "", "", MDL_BACKUP_START,
+ MDL_EXPLICIT);
+ if (thd &&
+ thd->mdl_context.acquire_lock(&mdl_request,
+ thd->variables.lock_wait_timeout))
+ result= 1;
+ else
{
- /* Wait for last binlog checkpoint event to be logged. */
- mysql_bin_log.wait_for_last_checkpoint_event();
+ if (thd)
+ thd->backup_commit_lock= &mdl_request;
+
+ DYNAMIC_ARRAY *drop_gtid_domain=
+ (thd && (thd->lex->delete_gtid_domain.elements > 0)) ?
+ &thd->lex->delete_gtid_domain : NULL;
+ if (mysql_bin_log.flush_binlog(drop_gtid_domain))
+ *write_to_binlog= -1;
+
+ /* Note that WSREP(thd) might not be true here e.g. during
+ SST. */
+ if (WSREP_ON)
+ {
+ /* Wait for last binlog checkpoint event to be logged. */
+ mysql_bin_log.wait_for_last_checkpoint_event();
+ }
+ if (thd)
+ {
+ if (mdl_request.ticket)
+ thd->mdl_context.release_lock(mdl_request.ticket);
+ thd->backup_commit_lock= 0;
+ }
}
}
}
diff --git a/sql/sql_repl.cc b/sql/sql_repl.cc
index 760f7ee2bbe1e..b192dd644c05b 100644
--- a/sql/sql_repl.cc
+++ b/sql/sql_repl.cc
@@ -130,6 +130,12 @@ struct binlog_send_info {
slave_connection_state *until_gtid_state;
slave_connection_state until_gtid_state_obj;
Format_description_log_event *fdev;
+ handler_binlog_reader *engine_binlog_reader;
+ /*
+ Last file_no reported as the current point to slave (using a fake rotate
+ event prior to a GTID event, mainly for debugging purposes).
+ */
+ uint64_t prev_reported_file_no;
int mariadb_slave_capability;
enum_gtid_skip_type gtid_skip_group;
enum_gtid_until_state gtid_until_group;
@@ -169,6 +175,7 @@ struct binlog_send_info {
char *lfn)
: thd(thd_arg), net(&thd_arg->net), packet(packet_arg),
log_file_name(lfn), until_gtid_state(NULL), fdev(NULL),
+ engine_binlog_reader(NULL), prev_reported_file_no(~(uint64_t)0),
gtid_skip_group(GTID_SKIP_NOT), gtid_until_group(GTID_UNTIL_NOT_DONE),
flags(flags_arg), current_checksum_alg(BINLOG_CHECKSUM_ALG_UNDEF),
slave_gtid_strict_mode(false), send_fake_gtid_list(false),
@@ -188,6 +195,7 @@ struct binlog_send_info {
bzero(&error_gtid, sizeof(error_gtid));
until_binlog_state.init();
}
+ ~binlog_send_info() { delete engine_binlog_reader; }
};
// prototype
@@ -301,6 +309,52 @@ static int fake_gtid_list_event(binlog_send_info *info,
}
+static int fake_format_description_event(binlog_send_info *info,
+ Format_description_log_event *fdev,
+ const char **errmsg,
+ uint32 current_pos)
+{
+ my_bool do_checksum;
+ int err;
+ ha_checksum crc;
+ char buf[320];
+ String str(buf, sizeof(buf), system_charset_info);
+ String* packet= info->packet;
+
+ str.length(0);
+ fdev->dont_set_created= true;
+ if (fdev->to_packet(&str))
+ {
+ info->error= ER_UNKNOWN_ERROR;
+ *errmsg= "Failed due to out-of-memory writing Format_description event";
+ return -1;
+ }
+ if ((err= fake_event_header(packet, FORMAT_DESCRIPTION_EVENT,
+ str.length(), &do_checksum, &crc,
+ errmsg, BINLOG_CHECKSUM_ALG_CRC32,
+ current_pos)))
+ {
+ info->error= ER_UNKNOWN_ERROR;
+ return err;
+ }
+
+ packet->append(str);
+ if (do_checksum)
+ {
+ crc= my_checksum(crc, (uchar*)str.ptr(), str.length());
+ }
+
+ if ((err= fake_event_footer(packet, do_checksum, crc, errmsg)) ||
+ (err= fake_event_write(info->net, packet, errmsg)))
+ {
+ info->error= ER_UNKNOWN_ERROR;
+ return err;
+ }
+
+ return 0;
+}
+
+
/*
Reset thread transmit packet buffer for event sending
@@ -563,9 +617,7 @@ static my_bool adjust_callback(THD *thd, my_off_t *purge_offset)
we just started reading the index file. In that case
we have nothing to adjust
*/
- if (linfo->index_file_offset < *purge_offset)
- linfo->fatal= (linfo->index_file_offset != 0);
- else
+ if (linfo->index_file_offset >= *purge_offset)
linfo->index_file_offset-= *purge_offset;
}
mysql_mutex_unlock(&thd->LOCK_thd_data);
@@ -605,7 +657,7 @@ static my_bool log_in_use_callback(THD *thd, st_log_in_use *arg)
/*
- Check if a log is in use.
+ Check if a log is in use (legacy binlog).
@return 0 Not used
@return 1 A slave is reading from the log
@@ -627,6 +679,94 @@ int log_in_use(const char* log_name, uint min_connected)
}
+struct st_engine_binlog_in_use {
+ uint64_t min_file_no;
+ uint count;
+};
+
+
+my_bool
+engine_binlog_in_use_callback(THD *thd, st_engine_binlog_in_use *arg)
+{
+ if (thd->current_linfo)
+ {
+ mysql_mutex_lock(&thd->LOCK_thd_data);
+ if (LOG_INFO *linfo= thd->current_linfo)
+ {
+ uint64_t file_no= linfo->file_no.load(std::memory_order_relaxed);
+ if (file_no < arg->min_file_no)
+ arg->min_file_no= file_no;
+ if (file_no != ~(uint64_t)0)
+ ++arg->count;
+ }
+ mysql_mutex_unlock(&thd->LOCK_thd_data);
+ }
+ return FALSE;
+}
+
+
+/*
+ Find earliest binlog file in use (--binlog-storage-engine).
+
+ Returns a pair of the earliest file_no binlog in use by a dump thread,
+ and the number of actively running dump threads.
+*/
+std::pair
+engine_binlog_in_use()
+{
+ DBUG_ASSERT(opt_binlog_engine_hton);
+ st_engine_binlog_in_use arg{~(uint64_t)0, 0};
+ server_threads.iterate(engine_binlog_in_use_callback, &arg);
+ return {arg.min_file_no, arg.count};
+}
+
+
+/*
+ Inform engine about server state relevant for automatic binlog purge.
+ Used by engines that implement --binlog-storage-engine.
+
+ Returns true if automatic purge should proceed with supplied information,
+ false if automatic purge is disabled due to
+ --slave-connections-needed-for-purge.
+*/
+bool
+ha_binlog_purge_info(handler_binlog_purge_info *out_info)
+{
+ auto p= engine_binlog_in_use();
+ out_info->limit_file_no= p.first;
+ uint num_dump_threads= p.second;
+ out_info->purge_by_name= false;
+ out_info->limit_name= nullptr;
+ if (binlog_expire_logs_seconds)
+ {
+ out_info->purge_by_date= true;
+ out_info->limit_date= my_time(0) - binlog_expire_logs_seconds;
+ }
+ else
+ out_info->purge_by_date= false;
+ if (binlog_space_limit)
+ {
+ out_info->purge_by_size= true;
+ out_info->limit_size= binlog_space_limit;
+ }
+ else
+ out_info->purge_by_size= false;
+
+ out_info->nonpurge_filename[0]= '\0';
+ if (num_dump_threads >= slave_connections_needed_for_purge)
+ {
+ out_info->nonpurge_reason= nullptr;
+ return true;
+ }
+ else
+ {
+ out_info->nonpurge_reason= "less than 'slave_connections_needed_for_purge' "
+ "slaves have processed it";
+ return false;
+ }
+}
+
+
bool purge_error_message(THD* thd, int res)
{
uint errcode;
@@ -654,17 +794,66 @@ bool purge_error_message(THD* thd, int res)
*/
bool purge_master_logs(THD* thd, const char* to_log)
{
- char search_file_name[FN_REFLEN];
if (!mysql_bin_log.is_open())
{
my_ok(thd);
return FALSE;
}
- mysql_bin_log.make_log_name(search_file_name, to_log);
- return purge_error_message(thd,
- mysql_bin_log.purge_logs(thd, search_file_name,
- 0, 1, 1, 1, NULL));
+ MDL_request mdl_request;
+ MDL_REQUEST_INIT(&mdl_request, MDL_key::BACKUP, "", "", MDL_BACKUP_START,
+ MDL_EXPLICIT);
+ if (thd->mdl_context.acquire_lock(&mdl_request,
+ thd->variables.lock_wait_timeout))
+ return TRUE;
+ thd->backup_commit_lock= &mdl_request;
+
+ int res;
+ if (!opt_binlog_engine_hton)
+ {
+ char search_file_name[FN_REFLEN];
+ mysql_bin_log.make_log_name(search_file_name, to_log);
+ res= mysql_bin_log.purge_logs(thd, search_file_name, 0, 1, 1, 1, NULL);
+ }
+ else
+ {
+ handler_binlog_purge_info purge_info;
+ auto p= engine_binlog_in_use();
+ purge_info.limit_file_no= p.first;
+ uint num_dump_threads= p.second;
+ if (num_dump_threads < slave_connections_needed_for_purge)
+ {
+ /*
+ Prevent purging any file.
+ We need to do it this way, since we have to call into the engine to let
+ it check if there are any files to potentially purge. If there are, we
+ want to give an error that purge was not possible. But if there were no
+ files to purge in any case, we do not want to give any error.
+ */
+ purge_info.limit_file_no= 0;
+ purge_info.nonpurge_reason= "less than "
+ "'slave_connections_needed_for_purge' slaves have processed it";
+ }
+ else
+ purge_info.nonpurge_reason= nullptr;
+ purge_info.nonpurge_filename[0]= '\0';
+ purge_info.purge_by_date= false;
+ purge_info.limit_date= (time_t)0;
+ purge_info.purge_by_size= false;
+ purge_info.limit_size= 0;
+ purge_info.purge_by_name= true;
+ purge_info.limit_name= to_log;
+ res= (*opt_binlog_engine_hton->binlog_purge)(&purge_info);
+ if (res && purge_info.nonpurge_reason)
+ give_purge_note(purge_info.nonpurge_reason,
+ purge_info.nonpurge_filename, true);
+ }
+
+ if (mdl_request.ticket)
+ thd->mdl_context.release_lock(mdl_request.ticket);
+ thd->backup_commit_lock= 0;
+
+ return purge_error_message(thd, res);
}
@@ -686,10 +875,36 @@ bool purge_master_logs_before_date(THD* thd, time_t purge_time)
my_ok(thd);
return 0;
}
- return purge_error_message(thd,
- mysql_bin_log.purge_logs_before_date(thd,
- purge_time,
- 1));
+ int res;
+ if (!opt_binlog_engine_hton)
+ res= mysql_bin_log.purge_logs_before_date(thd, purge_time, 1);
+ else
+ {
+ handler_binlog_purge_info purge_info;
+ auto p= engine_binlog_in_use();
+ purge_info.limit_file_no= p.first;
+ uint num_dump_threads= p.second;
+ if (num_dump_threads < slave_connections_needed_for_purge)
+ {
+ purge_info.limit_file_no= 0;
+ purge_info.nonpurge_reason= "less than "
+ "'slave_connections_needed_for_purge' slaves have processed it";
+ }
+ else
+ purge_info.nonpurge_reason= nullptr;
+ purge_info.nonpurge_filename[0]= '\0';
+ purge_info.purge_by_date= true;
+ purge_info.limit_date= purge_time;
+ purge_info.purge_by_size= false;
+ purge_info.limit_size= 0;
+ purge_info.purge_by_name= false;
+ purge_info.limit_name= nullptr;
+ res= (*opt_binlog_engine_hton->binlog_purge)(&purge_info);
+ if (res && purge_info.nonpurge_reason)
+ give_purge_note(purge_info.nonpurge_reason,
+ purge_info.nonpurge_filename, true);
+ }
+ return purge_error_message(thd, res);
}
void set_read_error(binlog_send_info *info, int error)
@@ -941,13 +1156,6 @@ static int send_heartbeat_event(binlog_send_info *info,
}
-struct binlog_file_entry
-{
- binlog_file_entry *next;
- LEX_CSTRING name;
- my_off_t size;
-};
-
/**
Read all binary logs and return as a list
@@ -972,6 +1180,13 @@ get_binlog_list(MEM_ROOT *memroot, bool reverse= true,
binlog_file_entry *current_list= NULL, *current_link= NULL, *e;
DBUG_ENTER("get_binlog_list");
+ if (opt_binlog_engine_hton)
+ {
+ if (already_locked)
+ mysql_bin_log.unlock_index();
+ DBUG_RETURN((*opt_binlog_engine_hton->get_binlog_file_list)(memroot));
+ }
+
if (!mysql_bin_log.is_open())
{
if (already_locked)
@@ -1314,6 +1529,11 @@ check_slave_start_position(binlog_send_info *info, const char **errormsg,
}
+static const char *gtid_too_old_errmsg=
+ "Could not find GTID state requested by slave in any binlog "
+ "files. Probably the slave state is too old and required binlog files "
+ "have been purged.";
+
/*
Helper function for gtid_find_binlog_pos() below.
Check a binlog file against a slave position. Use a GTID index if present.
@@ -1407,6 +1627,52 @@ gtid_check_binlog_file(slave_connection_state *state,
}
+/*
+ Do some checks on each GTID in the starting GTID state found when searching
+ for the starting GTID position in the binlog.
+*/
+static void
+found_pos_check_gtid(const rpl_gtid *found_gtid, slave_connection_state *state,
+ slave_connection_state *until_gtid_state)
+{
+ const rpl_gtid *gtid= state->find(found_gtid->domain_id);
+ if (!gtid)
+ {
+ /*
+ Contains_all_slave_gtid() returns false if there is any domain in
+ Gtid_list_event which is not in the requested slave position.
+
+ We may delete a domain from the slave state inside this loop, but
+ we only do this when it is the very last GTID logged for that
+ domain in earlier binlogs, and then we can not encounter it in any
+ further GTIDs in the Gtid_list.
+ */
+ DBUG_ASSERT(0);
+ } else if (gtid->server_id == found_gtid->server_id &&
+ gtid->seq_no == found_gtid->seq_no)
+ {
+ /*
+ The slave requested to start from the very beginning of this
+ domain in this binlog file. So delete the entry from the state,
+ we do not need to skip anything.
+ */
+ state->remove(gtid);
+ }
+
+ if (until_gtid_state &&
+ (gtid= until_gtid_state->find(found_gtid->domain_id)) &&
+ gtid->server_id == found_gtid->server_id &&
+ gtid->seq_no <= found_gtid->seq_no)
+ {
+ /*
+ We've already reached the stop position in UNTIL for this domain,
+ since it is before the start position.
+ */
+ until_gtid_state->remove(gtid);
+ }
+}
+
+
/*
Find the name of the binlog file to start reading for a slave that connects
using GTID state.
@@ -1503,43 +1769,7 @@ gtid_find_binlog_pos(slave_connection_state *state, char *out_name,
their UNTIL condition.
*/
for (i= 0; i < count; ++i)
- {
- const rpl_gtid *gtid= state->find(gtids[i].domain_id);
- if (!gtid)
- {
- /*
- Contains_all_slave_gtid() returns false if there is any domain in
- Gtid_list_event which is not in the requested slave position.
-
- We may delete a domain from the slave state inside this loop, but
- we only do this when it is the very last GTID logged for that
- domain in earlier binlogs, and then we can not encounter it in any
- further GTIDs in the Gtid_list.
- */
- DBUG_ASSERT(0);
- } else if (gtid->server_id == gtids[i].server_id &&
- gtid->seq_no == gtids[i].seq_no)
- {
- /*
- The slave requested to start from the very beginning of this
- domain in this binlog file. So delete the entry from the state,
- we do not need to skip anything.
- */
- state->remove(gtid);
- }
-
- if (until_gtid_state &&
- (gtid= until_gtid_state->find(gtids[i].domain_id)) &&
- gtid->server_id == gtids[i].server_id &&
- gtid->seq_no <= gtids[i].seq_no)
- {
- /*
- We've already reached the stop position in UNTIL for this domain,
- since it is before the start position.
- */
- until_gtid_state->remove(gtid);
- }
- }
+ found_pos_check_gtid(&(gtids[i]), state, until_gtid_state);
}
goto end;
@@ -1548,9 +1778,7 @@ gtid_find_binlog_pos(slave_connection_state *state, char *out_name,
}
/* We reached the end without finding anything. */
- errormsg= "Could not find GTID state requested by slave in any binlog "
- "files. Probably the slave state is too old and required binlog files "
- "have been purged.";
+ errormsg= gtid_too_old_errmsg;
end:
if (glev)
@@ -1564,6 +1792,28 @@ gtid_find_binlog_pos(slave_connection_state *state, char *out_name,
}
+static const char *
+gtid_find_engine_pos(binlog_send_info *info)
+{
+ handler_binlog_reader *binlog_reader= info->engine_binlog_reader;
+ slave_connection_state *pos= &info->gtid_state;
+ slave_connection_state *until_gtid_pos= info->until_gtid_state;
+ rpl_binlog_state *until_binlog_state= &info->until_binlog_state;
+
+ int res= binlog_reader->init_gtid_pos(info->thd, pos, until_binlog_state);
+ if (res < 0)
+ return "Error while looking up GTID position in engine binlog";
+ if (res == 0)
+ return gtid_too_old_errmsg;
+ until_binlog_state->iterate(
+ [pos, until_gtid_pos] (const rpl_gtid *gtid) -> bool {
+ found_pos_check_gtid(gtid, pos, until_gtid_pos);
+ return false;
+ });
+ return nullptr;
+}
+
+
static bool
gtid_index_lookup_pos(const char *name, uint32 offset, uint32 *out_start_seek,
slave_connection_state *out_gtid_state)
@@ -2245,7 +2495,10 @@ send_event_to_slave(binlog_send_info *info, Log_event_type event_type,
THD_STAGE_INFO(info->thd, stage_sending_binlog_event_to_slave);
- pos= my_b_tell(log);
+ if (opt_binlog_engine_hton)
+ pos= 4; // ToDo: Support for semi-sync in binlog-in-engine
+ else
+ pos= my_b_tell(log);
if (repl_semisync_master.update_sync_header(info->thd,
(uchar*) packet->c_ptr_safe(),
info->log_file_name + info->dirlen,
@@ -2332,6 +2585,17 @@ static int init_binlog_sender(binlog_send_info *info,
String slave_until_gtid_str(str_buf2, sizeof(str_buf2), system_charset_info);
connect_gtid_state.length(0);
+ if (opt_binlog_engine_hton &&
+ !(info->engine_binlog_reader=
+ (*opt_binlog_engine_hton->get_binlog_reader)(true)))
+ {
+ LEX_CSTRING *engine_name= hton_name(opt_binlog_engine_hton);
+ my_error(ER_CANNOT_INIT_ENGINE_BINLOG_READER, MYF(0), engine_name->str);
+ info->errmsg= "Error while initializing engine binlog reader";
+ info->error= ER_CANNOT_INIT_ENGINE_BINLOG_READER;
+ return 1;
+ }
+
/** save start file/pos that was requested by slave */
strmake(info->start_log_file_name, log_ident,
sizeof(info->start_log_file_name));
@@ -2356,6 +2620,14 @@ static int init_binlog_sender(binlog_send_info *info,
info->is_until_before_gtids= get_slave_gtid_until_before_gtids(thd);
}
}
+ else if (opt_binlog_engine_hton)
+ {
+ my_error(ER_ENGINE_BINLOG_REQUIRES_GTID, MYF(0));
+ info->errmsg=
+ "Slave must enable GTID mode when master uses --binlog-storage-engine";
+ info->error= ER_ENGINE_BINLOG_REQUIRES_GTID;
+ return 1;
+ }
DBUG_EXECUTE_IF("binlog_force_reconnect_after_22_events",
{
@@ -2418,14 +2690,28 @@ static int init_binlog_sender(binlog_send_info *info,
info->error= error;
return 1;
}
- if ((info->errmsg= gtid_find_binlog_pos(&info->gtid_state,
- search_file_name,
- info->until_gtid_state,
- &info->until_binlog_state,
- &found_in_index, &start_seek)))
+
+ if (opt_binlog_engine_hton)
{
- info->error= ER_MASTER_FATAL_ERROR_READING_BINLOG;
- return 1;
+ if ((info->errmsg= gtid_find_engine_pos(info)))
+ {
+ info->error= ER_MASTER_FATAL_ERROR_READING_BINLOG;
+ return 1;
+ }
+ found_in_index= true;
+ start_seek= 0; /* Not used when binlog implemented in engine. */
+ }
+ else
+ {
+ if ((info->errmsg= gtid_find_binlog_pos(&info->gtid_state,
+ search_file_name,
+ info->until_gtid_state,
+ &info->until_binlog_state,
+ &found_in_index, &start_seek)))
+ {
+ info->error= ER_MASTER_FATAL_ERROR_READING_BINLOG;
+ return 1;
+ }
}
if (found_in_index)
@@ -2448,7 +2734,8 @@ static int init_binlog_sender(binlog_send_info *info,
}
linfo->index_file_offset= 0;
- if (mysql_bin_log.find_log_pos(linfo, name, 1))
+ if (!opt_binlog_engine_hton &&
+ mysql_bin_log.find_log_pos(linfo, name, 1))
{
info->errmsg= "Could not find first log file name in binary "
"log index file";
@@ -2461,7 +2748,8 @@ static int init_binlog_sender(binlog_send_info *info,
// note: publish that we use file, before we open it
thd->set_current_linfo(linfo);
- if (check_start_offset(info, linfo->log_file_name, *pos))
+ if (!opt_binlog_engine_hton &&
+ check_start_offset(info, linfo->log_file_name, *pos))
return 1;
if (*pos > BIN_LOG_HEADER_SIZE)
@@ -2891,6 +3179,47 @@ static my_off_t get_binlog_end_pos(binlog_send_info *info,
return 0;
}
+
+/*
+ Helper function for send_events() and send_engine_events().
+ After an event has been sent to the client, it handles sending a fake
+ GTID_LIST event if needed; and it handles checking the GTID until stop
+ condition, if any.
+*/
+static bool
+send_event_gtid_list_and_until(binlog_send_info *info, ulong *ev_offset,
+ Log_event_type event_type, my_off_t log_pos)
+{
+ if (unlikely(info->send_fake_gtid_list) &&
+ info->gtid_skip_group == GTID_SKIP_NOT)
+ {
+ Gtid_list_log_event glev(&info->until_binlog_state, 0);
+
+ if (reset_transmit_packet(info, info->flags, ev_offset, &info->errmsg) ||
+ fake_gtid_list_event(info, &glev, &info->errmsg, (uint32)log_pos))
+ {
+ info->error= ER_UNKNOWN_ERROR;
+ return true;
+ }
+ info->send_fake_gtid_list= false;
+ }
+
+ if (info->until_gtid_state &&
+ is_until_reached(info, ev_offset, event_type, &info->errmsg,
+ (uint32)log_pos))
+ {
+ if (info->errmsg)
+ {
+ info->error= ER_UNKNOWN_ERROR;
+ return true;
+ }
+ info->should_stop= true;
+ }
+
+ return false;
+}
+
+
/**
* This function sends events from one binlog file
* but only up until end_pos
@@ -2905,6 +3234,7 @@ static int send_events(binlog_send_info *info, IO_CACHE* log, LOG_INFO* linfo,
ulong ev_offset;
String *packet= info->packet;
+ DBUG_ASSERT(!info->engine_binlog_reader);
linfo->pos= my_b_tell(log);
info->last_pos= my_b_tell(log);
@@ -2976,47 +3306,175 @@ static int send_events(binlog_send_info *info, IO_CACHE* log, LOG_INFO* linfo,
ev_offset, &info->error_gtid))))
return 1;
- if (unlikely(info->send_fake_gtid_list) &&
- info->gtid_skip_group == GTID_SKIP_NOT)
+ if (send_event_gtid_list_and_until(info, &ev_offset, event_type,
+ my_b_tell(log)))
+ return 1;
+
+ /* Abort server before it sends the XID_EVENT */
+ DBUG_EXECUTE_IF("crash_before_send_xid",
+ {
+ if (event_type == XID_EVENT)
+ {
+ my_sleep(2000000);
+ DBUG_SUICIDE();
+ }
+ });
+ }
+
+ return 0;
+}
+
+
+/**
+ * Send events from binlog implemented in storage engine. Will wait for more
+ * data to become available as needed.
+ *
+ * return 0 - OK
+ * else NOK
+ */
+static int send_engine_events(binlog_send_info *info, LOG_INFO* linfo)
+{
+ int error;
+ ulong ev_offset;
+
+ String *packet= info->packet;
+ handler_binlog_reader *reader= info->engine_binlog_reader;
+ DBUG_ASSERT(reader);
+ while (!should_stop(info))
+ {
+ /* reset the transmit packet for the event read from binary log
+ file */
+ if (reset_transmit_packet(info, info->flags, &ev_offset, &info->errmsg))
+ return 1;
+
+ error= reader->read_log_event(packet, packet->length(),
+ info->thd->variables.max_allowed_packet);
+ if (unlikely(error) && error != LOG_READ_EOF)
{
- Gtid_list_log_event glev(&info->until_binlog_state, 0);
+ set_read_error(info, error);
+ return 1;
+ }
- if (reset_transmit_packet(info, info->flags, &ev_offset, &info->errmsg) ||
- fake_gtid_list_event(info, &glev, &info->errmsg, (uint32)my_b_tell(log)))
- {
- info->error= ER_UNKNOWN_ERROR;
- return 1;
- }
- info->send_fake_gtid_list= false;
+ uint64_t prev_file_no= linfo->file_no.load(std::memory_order_relaxed);
+ if (unlikely(prev_file_no == ~(uint64_t)0) ||
+ unlikely(reader->cur_file_no > prev_file_no))
+ {
+ linfo->file_no.store(reader->cur_file_no, std::memory_order_relaxed);
+ (*opt_binlog_engine_hton->get_filename)(info->log_file_name,
+ reader->cur_file_no);
}
+ linfo->pos= (my_off_t) reader->cur_file_pos;
- if (info->until_gtid_state &&
- is_until_reached(info, &ev_offset, event_type, &info->errmsg,
- (uint32)my_b_tell(log)))
+ if (error == LOG_READ_EOF)
{
- if (info->errmsg)
+ /**
+ * check if we should wait for more data
+ */
+ if ((info->flags & BINLOG_DUMP_NON_BLOCK) ||
+ (info->thd->variables.server_id == 0))
+ {
+ info->should_stop= true;
+ return 0;
+ }
+
+ /**
+ * flush data before waiting
+ */
+ if (net_flush(info->net))
{
+ info->errmsg= "failed on net_flush()";
info->error= ER_UNKNOWN_ERROR;
return 1;
}
- info->should_stop= true;
- return 0;
+
+ while (!should_stop(info, true) && !reader->data_available())
+ {
+ struct timespec ts;
+ struct timespec *ts_ptr= nullptr;
+ if (info->heartbeat_period)
+ {
+ set_timespec_nsec(ts, info->heartbeat_period);
+ ts_ptr= &ts;
+ }
+ bool ret= reader->wait_available(info->thd, ts_ptr);
+ if (info->heartbeat_period && ret)
+ {
+ struct event_coordinates coord=
+ { info->log_file_name, reader->cur_file_pos };
+ int err= send_heartbeat_event(info, info->net, info->packet, &coord,
+ BINLOG_CHECKSUM_ALG_OFF);
+ if (err)
+ return 1;
+ info->heartbeat_period= get_heartbeat_period(info->thd);
+ }
+ }
+ continue;
}
- /* Abort server before it sends the XID_EVENT */
- DBUG_EXECUTE_IF("crash_before_send_xid",
+ Log_event_type event_type=
+ (Log_event_type)((uchar)(*packet)[LOG_EVENT_OFFSET+ev_offset]);
+
+ DBUG_ASSERT(event_type != START_ENCRYPTION_EVENT);
+#ifdef ENABLED_DEBUG_SYNC
+ DBUG_EXECUTE_IF("dump_thread_wait_before_send_xid",
{
if (event_type == XID_EVENT)
{
- my_sleep(2000000);
- DBUG_SUICIDE();
+ net_flush(info->net);
+ const char act[]=
+ "now "
+ "wait_for signal.continue";
+ DBUG_ASSERT(debug_sync_service);
+ DBUG_ASSERT(!debug_sync_set_action(
+ info->thd,
+ STRING_WITH_LEN(act)));
+
+ const char act2[]=
+ "now "
+ "signal signal.continued";
+ DBUG_ASSERT(!debug_sync_set_action(
+ info->thd,
+ STRING_WITH_LEN(act2)));
}
});
+#endif
+
+ if (event_type == GTID_EVENT && prev_file_no != info->prev_reported_file_no)
+ {
+ String saved_gtid_packet;
+ saved_gtid_packet.swap(*info->packet);
+ int err=
+ fake_rotate_event(info, 0, &info->errmsg, BINLOG_CHECKSUM_ALG_OFF);
+ info->prev_reported_file_no= prev_file_no;
+ saved_gtid_packet.swap(*info->packet);
+ if (err)
+ return 1;
+ }
+ else if (unlikely(event_type == FORMAT_DESCRIPTION_EVENT) &&
+ info->gtid_state.count() > 0)
+ {
+ /*
+ In the engine-implemented binlog, format description event is (only)
+ written to mark a master server restart; this is used by the slave to
+ know that the master discarded temporary tabls at this point. So don't
+ send such event until we have reached our GTID starting position, so
+ that the slave will not mistakenly discard such temporary tables too
+ early.
+ */
+ continue;
+ }
+ if (((info->errmsg= send_event_to_slave(info, event_type, nullptr,
+ ev_offset, &info->error_gtid))))
+ return 1;
+
+ if (send_event_gtid_list_and_until(info, &ev_offset, event_type, 0))
+ return 1;
}
return 0;
}
+
/**
* This function sends one binlog file to slave
*
@@ -3032,7 +3490,8 @@ static int send_one_binlog_file(binlog_send_info *info,
mysql_mutex_assert_not_owner(mysql_bin_log.get_log_lock());
/* seek to the requested position, to start the requested dump */
- if (start_pos != BIN_LOG_HEADER_SIZE)
+ if (!opt_binlog_engine_hton &&
+ start_pos != BIN_LOG_HEADER_SIZE)
{
my_b_seek(log, start_pos);
linfo->pos= start_pos;
@@ -3042,22 +3501,31 @@ static int send_one_binlog_file(binlog_send_info *info,
sending_new_binlog_file++;
while (!should_stop(info))
{
- /**
- * get end pos of current log file, this function
- * will wait if there is nothing available
- */
- my_off_t end_pos= get_binlog_end_pos(info, log, linfo);
- if (end_pos <= 1)
+ if (opt_binlog_engine_hton)
{
- /** end of file or error */
- return (int)end_pos;
+ info->dirlen= 0;
+ if (send_engine_events(info, linfo))
+ return 1;
+ }
+ else
+ {
+ /**
+ * get end pos of current log file, this function
+ * will wait if there is nothing available
+ */
+ my_off_t end_pos= get_binlog_end_pos(info, log, linfo);
+ if (end_pos <= 1)
+ {
+ /** end of file or error */
+ return (int)end_pos;
+ }
+ info->dirlen= dirname_length(info->log_file_name);
+ /**
+ * send events from current position up to end_pos
+ */
+ if (send_events(info, log, linfo, end_pos))
+ return 1;
}
- info->dirlen= dirname_length(info->log_file_name);
- /**
- * send events from current position up to end_pos
- */
- if (send_events(info, log, linfo, end_pos))
- return 1;
}
return 1;
@@ -3076,6 +3544,7 @@ void mysql_binlog_send(THD* thd, char* log_ident, my_off_t pos,
binlog_send_info infoobj(thd, packet, flags, linfo.log_file_name);
binlog_send_info *info= &infoobj;
bool has_transmit_started= false;
+ bool start_use_binlog= false;
int old_max_allowed_packet= thd->variables.max_allowed_packet;
thd->variables.max_allowed_packet= MAX_MAX_ALLOWED_PACKET;
@@ -3085,6 +3554,14 @@ void mysql_binlog_send(THD* thd, char* log_ident, my_off_t pos,
bzero((char*) &log,sizeof(log));
+ if (mysql_bin_log.start_use_binlog(thd))
+ {
+ info->errmsg= "Binlog dump terminated by user kill";
+ info->error= ER_CONNECTION_KILLED;
+ goto err;
+ }
+ start_use_binlog= true;
+
if (init_binlog_sender(info, &linfo, log_ident, &pos))
goto err;
@@ -3121,56 +3598,77 @@ void mysql_binlog_send(THD* thd, char* log_ident, my_off_t pos,
while (!should_stop(info))
{
- /*
- Tell the client about the log name with a fake Rotate event;
- this is needed even if we also send a Format_description_log_event
- just after, because that event does not contain the binlog's name.
- Note that as this Rotate event is sent before
- Format_description_log_event, the slave cannot have any info to
- understand this event's format, so the header len of
- Rotate_log_event is FROZEN (so in 5.0 it will have a header shorter
- than other events except FORMAT_DESCRIPTION_EVENT).
- Before 4.0.14 we called fake_rotate_event below only if (pos ==
- BIN_LOG_HEADER_SIZE), because if this is false then the slave
- already knows the binlog's name.
- Since, we always call fake_rotate_event; if the slave already knew
- the log's name (ex: CHANGE MASTER TO MASTER_LOG_FILE=...) this is
- useless but does not harm much. It is nice for 3.23 (>=.58) slaves
- which test Rotate events to see if the master is 4.0 (then they
- choose to stop because they can't replicate 4.0); by always calling
- fake_rotate_event we are sure that 3.23.58 and newer will detect the
- problem as soon as replication starts (BUG#198).
- Always calling fake_rotate_event makes sending of normal
- (=from-binlog) Rotate events a priori unneeded, but it is not so
- simple: the 2 Rotate events are not equivalent, the normal one is
- before the Stop event, the fake one is after. If we don't send the
- normal one, then the Stop event will be interpreted (by existing 4.0
- slaves) as "the master stopped", which is wrong. So for safety,
- given that we want minimum modification of 4.0, we send the normal
- and fake Rotates.
- */
- if (fake_rotate_event(info, pos, &info->errmsg, info->current_checksum_alg))
+ if (opt_binlog_engine_hton) {
+ /* Build a legacy Format_description event for slave. */
+ if (!(info->fdev= new Format_description_log_event
+ (4, 0, BINLOG_CHECKSUM_ALG_OFF)))
+ {
+ info->errmsg= "Out of memory initializing format_description event";
+ info->error= ER_MASTER_FATAL_ERROR_READING_BINLOG;
+ goto err;
+ }
+ if (reset_transmit_packet(info, info->flags, &ev_offset, &info->errmsg) ||
+ fake_format_description_event(info, info->fdev, &info->errmsg,
+ (uint32_t)pos))
+ {
+ info->error= ER_MASTER_FATAL_ERROR_READING_BINLOG;
+ goto err;
+ }
+ }
+ else /* !opt_binlog_engine_hton */
{
/*
- This error code is not perfect, as fake_rotate_event() does not
- read anything from the binlog; if it fails it's because of an
- error in my_net_write(), fortunately it will say so in errmsg.
+ Tell the client about the log name with a fake Rotate event;
+ this is needed even if we also send a Format_description_log_event
+ just after, because that event does not contain the binlog's name.
+ Note that as this Rotate event is sent before
+ Format_description_log_event, the slave cannot have any info to
+ understand this event's format, so the header len of
+ Rotate_log_event is FROZEN (so in 5.0 it will have a header shorter
+ than other events except FORMAT_DESCRIPTION_EVENT).
+ Before 4.0.14 we called fake_rotate_event below only if (pos ==
+ BIN_LOG_HEADER_SIZE), because if this is false then the slave
+ already knows the binlog's name.
+ Since, we always call fake_rotate_event; if the slave already knew
+ the log's name (ex: CHANGE MASTER TO MASTER_LOG_FILE=...) this is
+ useless but does not harm much. It is nice for 3.23 (>=.58) slaves
+ which test Rotate events to see if the master is 4.0 (then they
+ choose to stop because they can't replicate 4.0); by always calling
+ fake_rotate_event we are sure that 3.23.58 and newer will detect the
+ problem as soon as replication starts (BUG#198).
+ Always calling fake_rotate_event makes sending of normal
+ (=from-binlog) Rotate events a priori unneeded, but it is not so
+ simple: the 2 Rotate events are not equivalent, the normal one is
+ before the Stop event, the fake one is after. If we don't send the
+ normal one, then the Stop event will be interpreted (by existing 4.0
+ slaves) as "the master stopped", which is wrong. So for safety,
+ given that we want minimum modification of 4.0, we send the normal
+ and fake Rotates.
*/
- info->error= ER_MASTER_FATAL_ERROR_READING_BINLOG;
- goto err;
- }
+ if (fake_rotate_event(info, pos, &info->errmsg, info->current_checksum_alg))
+ {
+ /*
+ This error code is not perfect, as fake_rotate_event() does not
+ read anything from the binlog; if it fails it's because of an
+ error in my_net_write(), fortunately it will say so in errmsg.
+ */
+ info->error= ER_MASTER_FATAL_ERROR_READING_BINLOG;
+ goto err;
+ }
- if ((file=open_binlog(&log, linfo.log_file_name, &info->errmsg)) < 0)
- {
- info->error= ER_MASTER_FATAL_ERROR_READING_BINLOG;
- goto err;
- }
+ if ((file=open_binlog(&log, linfo.log_file_name, &info->errmsg)) < 0)
+ {
+ info->error= ER_MASTER_FATAL_ERROR_READING_BINLOG;
+ goto err;
+ }
- if (send_format_descriptor_event(info, &log, &linfo, pos))
- {
- info->error= ER_MASTER_FATAL_ERROR_READING_BINLOG;
- goto err;
- }
+ if (send_format_descriptor_event(info, &log, &linfo, pos))
+ {
+ info->error= ER_MASTER_FATAL_ERROR_READING_BINLOG;
+ goto err;
+ }
+
+ } /* !opt_binlog_engine_hton */
/*
We want to corrupt the first event that will be sent to the slave.
@@ -3285,6 +3783,9 @@ void mysql_binlog_send(THD* thd, char* log_ident, my_off_t pos,
thd->variables.max_allowed_packet= old_max_allowed_packet;
delete info->fdev;
+ if (start_use_binlog)
+ mysql_bin_log.end_use_binlog(thd);
+
if (likely(info->error == 0))
{
my_eof(thd);
@@ -4392,17 +4893,129 @@ int reset_master(THD* thd, rpl_gtid *init_state, uint32 init_state_len,
}
#endif /* WITH_WSREP */
bool ret= 0;
+
+ MDL_request mdl_request;
+ MDL_REQUEST_INIT(&mdl_request, MDL_key::BACKUP, "", "", MDL_BACKUP_START,
+ MDL_EXPLICIT);
+ if (thd->mdl_context.acquire_lock(&mdl_request,
+ thd->variables.lock_wait_timeout))
+ return 1;
+ thd->backup_commit_lock= &mdl_request;
+
/* Temporarily disable master semisync before resetting master. */
repl_semisync_master.before_reset_master();
ret= mysql_bin_log.reset_logs(thd, 1, init_state, init_state_len,
next_log_number);
repl_semisync_master.after_reset_master();
+
+ if (mdl_request.ticket)
+ thd->mdl_context.release_lock(mdl_request.ticket);
+ thd->backup_commit_lock= 0;
+
DBUG_EXECUTE_IF("crash_after_reset_master", DBUG_SUICIDE(););
return ret;
}
+/* Version of mysql_show_binlog_events() for --binlog-storage-engine. */
+static bool
+show_engine_binlog_events(THD* thd, Protocol *protocol, LEX_MASTER_INFO *lex_mi)
+{
+ bool err= false;
+
+ DBUG_ASSERT(opt_binlog_engine_hton);
+ DBUG_ASSERT(thd->lex->sql_command == SQLCOM_SHOW_BINLOG_EVENTS);
+ handler_binlog_reader *reader=
+ (*opt_binlog_engine_hton->get_binlog_reader)(false);
+ if (!reader)
+ {
+ my_error(ER_OUT_OF_RESOURCES, MYF(0));
+ return true;
+ }
+
+ ulonglong pos= lex_mi->pos;
+ /*
+ The positions "0" and "4" are unfortunately traditionally used
+ interchangeably to mean "the start of the binlog". Thus, we might here
+ easily see a starting position of "4", which is probably not valid in
+ the engine, but which really means "start of the file".
+
+ So here we have this ugly hack where "4" means the same as "0". Well,
+ use of offsets is discourated anyway in the new binlog (in favour of
+ GTID), and "4" is not going to be a valid position most likely, or if
+ it is, "0" will be equivalent (at least it is so for the InnoDB binlog
+ implementation.
+ */
+ if (pos == 4)
+ pos= 0;
+ if (reader->init_legacy_pos(thd, lex_mi->log_file_name, pos))
+ {
+ err= true;
+ goto end;
+ }
+ /* The engine reader will stop at the end of the requested file. */
+ reader->enable_single_file();
+
+ {
+ SELECT_LEX_UNIT *unit= &thd->lex->unit;
+ unit->set_limit(thd->lex->current_select);
+ uint64_t file_no= reader->cur_file_no;
+ ha_rows limit= unit->lim.get_select_limit();
+ String packet;
+ Format_description_log_event fd(4);
+ char name_buf[FN_REFLEN];
+ opt_binlog_engine_hton->get_filename(name_buf, file_no);
+
+ for (ha_rows event_count= 0; event_count < limit; ++event_count)
+ {
+ packet.length(0);
+ int reader_error= reader->read_log_event(&packet, 0,
+ thd->variables.max_allowed_packet + MAX_LOG_EVENT_HEADER);
+ if (reader_error)
+ {
+ if (reader_error != LOG_READ_EOF)
+ {
+ my_error(ER_ERROR_WHEN_EXECUTING_COMMAND, MYF(0),
+ "SHOW BINLOG EVENTS", "error reading event data");
+ err= true;
+ }
+ break;
+ }
+
+ if (unit->lim.check_offset(event_count))
+ continue;
+ const char *errmsg;
+ Log_event *ev= Log_event::read_log_event((const uchar *)packet.ptr(),
+ (uint)packet.length(),
+ &errmsg, &fd, false, false);
+ if (!ev)
+ {
+ my_error(ER_ERROR_WHEN_EXECUTING_COMMAND, MYF(0),
+ "SHOW BINLOG EVENTS", errmsg);
+ err= true;
+ break;
+ }
+ int send_err= ev->net_send(protocol, name_buf, 0);
+ delete ev;
+ if (send_err)
+ {
+ my_error(ER_ERROR_WHEN_EXECUTING_COMMAND, MYF(0),
+ "SHOW BINLOG EVENTS", "Net error");
+ err= true;
+ break;
+ }
+ }
+ }
+
+end:
+ if (!err)
+ my_eof(thd);
+ delete reader;
+ return err;
+}
+
+
/**
Execute a SHOW BINLOG EVENTS statement.
@@ -4442,6 +5055,10 @@ bool mysql_show_binlog_events(THD* thd)
Protocol::SEND_NUM_ROWS | Protocol::SEND_EOF))
DBUG_RETURN(TRUE);
+ if (opt_binlog_engine_hton &&
+ thd->lex->sql_command == SQLCOM_SHOW_BINLOG_EVENTS)
+ DBUG_RETURN(show_engine_binlog_events(thd, protocol, lex_mi));
+
DBUG_ASSERT(thd->lex->sql_command == SQLCOM_SHOW_BINLOG_EVENTS ||
thd->lex->sql_command == SQLCOM_SHOW_RELAYLOG_EVENTS);
@@ -4737,12 +5354,28 @@ bool show_binlog_info(THD* thd)
if (mysql_bin_log.is_open())
{
LOG_INFO li;
- mysql_bin_log.get_current_log(&li);
- size_t dir_len = dirname_length(li.log_file_name);
- const char *base= li.log_file_name + dir_len;
+ char buf[FN_REFLEN];
+ const char *base;
+ uint64_t pos;
+ if (opt_binlog_engine_hton)
+ {
+ uint64_t file_no;
+ mysql_mutex_lock(mysql_bin_log.get_log_lock());
+ (*opt_binlog_engine_hton->binlog_status)(&file_no, &pos);
+ (*opt_binlog_engine_hton->get_filename)(buf, file_no);
+ mysql_mutex_unlock(mysql_bin_log.get_log_lock());
+ base= buf;
+ }
+ else
+ {
+ mysql_bin_log.get_current_log(&li);
+ pos= (uint64_t) li.pos;
+ size_t dir_len = dirname_length(li.log_file_name);
+ base= li.log_file_name + dir_len;
+ }
protocol->store(base, strlen(base), &my_charset_bin);
- protocol->store((ulonglong) li.pos);
+ protocol->store((ulonglong)pos);
protocol->store(binlog_filter->get_do_db());
protocol->store(binlog_filter->get_ignore_db());
if (protocol->write())
@@ -4839,7 +5472,8 @@ bool show_binlogs(THD* thd)
cur_link->name.str+= dir_len;
cur_link->name.length-= dir_len;
- if (mysql_bin_log.get_reset_master_count() > expected_reset_masters)
+ if (!opt_binlog_engine_hton &&
+ mysql_bin_log.get_reset_master_count() > expected_reset_masters)
{
/*
Reset master was called after we cached filenames.
@@ -4849,7 +5483,8 @@ bool show_binlogs(THD* thd)
goto retry;
}
- if (!(strncmp(fname+dir_len, cur.log_file_name+cur_dir_len, length)))
+ if (!opt_binlog_engine_hton &&
+ !(strncmp(fname+dir_len, cur.log_file_name+cur_dir_len, length)))
cur_link->size= cur.pos; /* The active log, use the active position */
else
{
@@ -5170,6 +5805,8 @@ int compare_log_name(const char *log_1, const char *log_2) {
int res= 1;
const char *ext1_str= strrchr(log_1, '.');
const char *ext2_str= strrchr(log_2, '.');
+ if (!ext1_str || !ext2_str)
+ return strcmp(log_1, log_2);
char file_name_1[255], file_name_2[255];
strmake(file_name_1, log_1, (ext1_str - log_1));
strmake(file_name_2, log_2, (ext2_str - log_2));
diff --git a/sql/sql_repl.h b/sql/sql_repl.h
index c03384aa5a3e1..0e5f0f27d0a6f 100644
--- a/sql/sql_repl.h
+++ b/sql/sql_repl.h
@@ -39,6 +39,7 @@ int reset_master(THD* thd, rpl_gtid *init_state, uint32 init_state_len,
bool purge_master_logs(THD* thd, const char* to_log);
bool purge_master_logs_before_date(THD* thd, time_t purge_time);
int log_in_use(const char* log_name, uint min_connections);
+std::pair engine_binlog_in_use();
void adjust_linfo_offsets(my_off_t purge_offset);
void show_binlogs_get_fields(THD *thd, List- *field_list);
bool show_binlogs(THD* thd);
diff --git a/sql/sql_table.cc b/sql/sql_table.cc
index 38810a1b59488..67c507390c20e 100644
--- a/sql/sql_table.cc
+++ b/sql/sql_table.cc
@@ -12413,6 +12413,15 @@ copy_data_between_tables(THD *thd, TABLE *from, TABLE *to,
MYSQL_TIME query_start;
DBUG_ENTER("copy_data_between_tables");
+ /*
+ Various operations as part of the copy may cause call to trans_commit()
+ or otherwise cause wakeup_subsequent_commits() before completion. So
+ suspend those wakeups temporarily.
+ */
+ std::unique_ptr >
+ suspended_wfc(thd->suspend_subsequent_commits(),
+ [thd](wait_for_commit *wfc) { thd->resume_subsequent_commits(wfc); });
+
// Relay_log_info is too big to put on a stack
if (!(rli_buff= thd->alloc(sizeof(Relay_log_info))) ||
!(copy= new (thd->mem_root) Copy_field[to->s->fields]))
diff --git a/sql/sys_vars.cc b/sql/sys_vars.cc
index 9662179356740..c125d66faedf3 100644
--- a/sql/sys_vars.cc
+++ b/sql/sys_vars.cc
@@ -1281,14 +1281,22 @@ static bool update_binlog_space_limit(sys_var *, THD *,
if (opt_bin_log)
{
- if (binlog_space_limit)
- mysql_bin_log.count_binlog_space();
- /* Inform can_purge_log() that it should do a recheck of log_in_use() */
- sending_new_binlog_file++;
- mysql_bin_log.unlock_index();
- mysql_bin_log.purge(1);
- mysql_mutex_lock(&LOCK_global_system_variables);
- return 0;
+ if (opt_binlog_engine_hton)
+ {
+ if (loc_binlog_space_limit)
+ mysql_bin_log.engine_purge_logs_by_size(loc_binlog_space_limit);
+ }
+ else
+ {
+ if (loc_binlog_space_limit)
+ mysql_bin_log.count_binlog_space();
+ /* Inform can_purge_log() that it should do a recheck of log_in_use() */
+ sending_new_binlog_file++;
+ mysql_bin_log.unlock_index();
+ mysql_bin_log.purge(1);
+ mysql_mutex_lock(&LOCK_global_system_variables);
+ return 0;
+ }
}
mysql_bin_log.unlock_index();
mysql_mutex_lock(&LOCK_global_system_variables);
@@ -1828,6 +1836,16 @@ Sys_max_binlog_size(
BLOCK_SIZE(IO_SIZE), NO_MUTEX_GUARD, NOT_IN_BINLOG, ON_CHECK(0),
ON_UPDATE(fix_max_binlog_size));
+
+static Sys_var_charptr_fscs Sys_binlog_directory(
+ "binlog_directory",
+ "Directory path (absolute or relative to datadir) where binlog files "
+ "are stored. If this is used, must not specify a directory path for "
+ "--log-bin",
+ READ_ONLY GLOBAL_VAR(opt_binlog_directory), CMD_LINE(REQUIRED_ARG),
+ DEFAULT(0));
+
+
static bool fix_max_connections(sys_var *self, THD *thd, enum_var_type type)
{
return false;
@@ -2339,7 +2357,7 @@ Sys_var_last_gtid::session_value_ptr(THD *thd, const LEX_CSTRING *base) const
bool first= true;
str.length(0);
- rpl_gtid gtid= thd->get_last_commit_gtid();
+ rpl_gtid gtid= *thd->get_last_commit_gtid();
if ((gtid.seq_no > 0 &&
rpl_slave_state_tostring_helper(&str, >id, &first)) ||
!(p= thd->strmake(str.ptr(), str.length())))
@@ -2631,6 +2649,14 @@ static Sys_var_on_access_global<
GLOBAL_VAR(slave_max_statement_time_double), CMD_LINE(REQUIRED_ARG),
VALID_RANGE(0, LONG_TIMEOUT), DEFAULT(0), NO_MUTEX_GUARD,
NOT_IN_BINLOG, ON_CHECK(0), ON_UPDATE(update_slave_max_statement_time));
+
+
+static Sys_var_charptr_fscs Sys_binlog_storage_engine(
+ "binlog_storage_engine",
+ "Use a more efficient binlog implementation integrated with the "
+ "storage engine. Only available for supporting engines",
+ READ_ONLY GLOBAL_VAR(opt_binlog_storage_engine), CMD_LINE(REQUIRED_ARG),
+ DEFAULT(0));
#endif
@@ -3697,6 +3723,18 @@ Replicate_events_marked_for_skip
/* new options for semisync */
+static bool
+check_rpl_semi_sync_master_enabled(sys_var *self, THD *thd, set_var *var)
+{
+ if (opt_binlog_engine_hton && var->save_result.ulonglong_value)
+ {
+ my_error(ER_NOT_YET_SUPPORTED_ENGINE_BINLOG, MYF(0),
+ "Semi-synchronous replication");
+ return true;
+ }
+ return false;
+}
+
static bool fix_rpl_semi_sync_master_enabled(sys_var *self, THD *thd,
enum_var_type type)
{
@@ -3751,7 +3789,8 @@ Sys_semisync_master_enabled(
"Enable semi-synchronous replication master (disabled by default).",
GLOBAL_VAR(rpl_semi_sync_master_enabled),
CMD_LINE(OPT_ARG), DEFAULT(FALSE),
- NO_MUTEX_GUARD, NOT_IN_BINLOG, ON_CHECK(0),
+ NO_MUTEX_GUARD, NOT_IN_BINLOG,
+ ON_CHECK(check_rpl_semi_sync_master_enabled),
ON_UPDATE(fix_rpl_semi_sync_master_enabled));
static Sys_var_on_access_globaltransaction->xid_state.check_has_uncommitted_xa())
DBUG_RETURN(TRUE);
+ if (unlikely(thd->in_sub_stmt & SUB_STMT_TRIGGER) &&
+ opt_binlog_engine_hton)
+ {
+ /*
+ ToDo: Probably using savepoints in triggers should be disallowed
+ unconditionally. For now, we disallow it in the new binlog
+ implementation, which is not enabled by default, so that backwards
+ compatibility is of smaller concern.
+ */
+ my_error(ER_NOT_AVAILABLE_WITH_ENGINE_BINLOG, MYF(0),
+ "Using savepoints in triggers");
+ DBUG_RETURN(1);
+ }
+
SAVEPOINT *newsv= savepoint_add(thd, name, &thd->transaction->savepoints,
ha_release_savepoint);
@@ -716,10 +730,24 @@ bool trans_savepoint(THD *thd, LEX_CSTRING name)
bool trans_rollback_to_savepoint(THD *thd, LEX_CSTRING name)
{
int res= FALSE;
- SAVEPOINT *sv= *find_savepoint(thd, name);
+ SAVEPOINT *sv;
DBUG_ENTER("trans_rollback_to_savepoint");
- if (sv == NULL)
+ if (unlikely(thd->in_sub_stmt & SUB_STMT_TRIGGER) &&
+ opt_binlog_engine_hton)
+ {
+ /*
+ ToDo: Probably using savepoints in triggers should be disallowed
+ unconditionally. For now, we disallow it in the new binlog
+ implementation, which is not enabled by default, so that backwards
+ compatibility is of smaller concern.
+ */
+ my_error(ER_NOT_AVAILABLE_WITH_ENGINE_BINLOG, MYF(0),
+ "Using savepoints in triggers");
+ DBUG_RETURN(1);
+ }
+
+ if ((sv= *find_savepoint(thd, name)) == NULL)
{
my_error(ER_SP_DOES_NOT_EXIST, MYF(0), "SAVEPOINT", name.str);
DBUG_RETURN(TRUE);
@@ -756,7 +784,8 @@ bool trans_rollback_to_savepoint(THD *thd, LEX_CSTRING name)
/**
Remove the named savepoint from the set of savepoints of
- the current transaction.
+ the current transaction. Note that releasing a savepoint also releases
+ any savepoints set following the specified savepoint.
@note No commit or rollback occurs. It is an error if the
savepoint does not exist.
@@ -780,10 +809,22 @@ bool trans_release_savepoint(THD *thd, LEX_CSTRING name)
DBUG_RETURN(TRUE);
}
- if (ha_release_savepoint(thd, sv))
- res= TRUE;
-
- thd->transaction->savepoints= sv->prev;
+ SAVEPOINT *p= thd->transaction->savepoints;
+ while (p)
+ {
+ if (ha_release_savepoint(thd, p))
+ res= TRUE;
+ if (p == sv)
+ {
+ thd->transaction->savepoints= sv->prev;
+ DBUG_RETURN(MY_TEST(res));
+ }
+ p= p->prev;
+ }
+ DBUG_ASSERT(0 /* Should not get here, would imply that the list of savepoints
+ changed since find_savepoint() called at the start of this
+ function. */);
+ thd->transaction->savepoints= NULL;;
DBUG_RETURN(MY_TEST(res));
}
diff --git a/sql/wsrep_mysqld.cc b/sql/wsrep_mysqld.cc
index 2e671b2af547e..c38d92cbadf4b 100644
--- a/sql/wsrep_mysqld.cc
+++ b/sql/wsrep_mysqld.cc
@@ -2307,7 +2307,9 @@ int wsrep_to_buf_helper(
if (seqno)
{
Gtid_log_event gtid_event(thd, seqno, domain_id, true,
- LOG_EVENT_SUPPRESS_USE_F, true, 0);
+ Log_event::EVENT_NO_CACHE,
+ LOG_EVENT_SUPPRESS_USE_F, true, 0,
+ false, false);
gtid_event.server_id= server_id;
if (!gtid_event.is_valid()) ret= 0;
ret= writer.write(>id_event);
diff --git a/sql/wsrep_thd.cc b/sql/wsrep_thd.cc
index de91c960222d6..65111f9d940dc 100644
--- a/sql/wsrep_thd.cc
+++ b/sql/wsrep_thd.cc
@@ -465,8 +465,6 @@ uint wsrep_kill_thd(THD *thd, THD *victim_thd, killed_state kill_signal)
DEBUG_SYNC(thd, "wsrep_kill_before_awake_no_mutex");
victim_thd->wsrep_abort_by_kill= kill_signal;
victim_thd->awake_no_mutex(kill_signal);
- /* ha_abort_transaction() releases tmp->LOCK_thd_kill, so tmp
- is not safe to access anymore. */
ha_abort_transaction(thd, victim_thd, 1);
DBUG_RETURN(0);
}
diff --git a/storage/innobase/CMakeLists.txt b/storage/innobase/CMakeLists.txt
index 0b7c97c2afb10..0eb5db506d4f9 100644
--- a/storage/innobase/CMakeLists.txt
+++ b/storage/innobase/CMakeLists.txt
@@ -165,6 +165,7 @@ SET(INNOBASE_SOURCES
fsp/fsp0file.cc
fsp/fsp0space.cc
fsp/fsp0sysspace.cc
+ fsp/fsp_binlog.cc
fut/fut0lst.cc
ha/ha0storage.cc
fts/fts0fts.cc
@@ -182,6 +183,7 @@ SET(INNOBASE_SOURCES
fts/fts0plugin.cc
handler/ha_innodb.cc
handler/handler0alter.cc
+ handler/innodb_binlog.cc
handler/i_s.cc
ibuf/ibuf0ibuf.cc
include/btr0btr.h
@@ -238,6 +240,7 @@ SET(INNOBASE_SOURCES
include/fsp0space.h
include/fsp0sysspace.h
include/fsp0types.h
+ include/fsp_binlog.h
include/fts0ast.h
include/fts0blex.h
include/fts0fts.h
@@ -262,6 +265,7 @@ SET(INNOBASE_SOURCES
include/ha0storage.h
include/ha0storage.inl
include/handler0alter.h
+ include/innodb_binlog.h
include/hash0hash.h
include/ibuf0ibuf.h
include/lock0iter.h
diff --git a/storage/innobase/buf/buf0flu.cc b/storage/innobase/buf/buf0flu.cc
index 0fe60203901b8..f48ab270ef781 100644
--- a/storage/innobase/buf/buf0flu.cc
+++ b/storage/innobase/buf/buf0flu.cc
@@ -41,6 +41,7 @@ Created 11/11/1995 Heikki Tuuri
#include "log0crypt.h"
#include "srv0mon.h"
#include "fil0pagecompress.h"
+#include "fsp_binlog.h"
#include "lzo/lzo1x.h"
#include "snappy-c.h"
@@ -2047,6 +2048,27 @@ static bool log_checkpoint_low(lsn_t oldest_lsn, lsn_t end_lsn) noexcept
return true;
}
+void mtr_t::write_binlog(page_id_t page_id, uint16_t offset,
+ const void *buf, size_t size) noexcept
+{
+ ut_ad(!srv_read_only_mode);
+ ut_ad(m_log_mode == MTR_LOG_ALL);
+
+ bool alloc{size < mtr_buf_t::MAX_DATA_SIZE - (1 + 3 + 3 + 5 + 5)};
+ byte *end= log_write(page_id, nullptr, size, alloc, offset);
+ if (alloc)
+ {
+ ::memcpy(end, buf, size);
+ m_log.close(end + size);
+ }
+ else
+ {
+ m_log.close(end);
+ m_log.push(static_cast(buf), uint32_t(size));
+ }
+ m_modifications= true;
+}
+
/** Make a checkpoint. Note that this function does not flush dirty
blocks from the buffer pool: it only checks what is lsn of the oldest
modification in the pool, and writes information about the lsn in
@@ -2067,11 +2089,13 @@ static bool log_checkpoint() noexcept
#endif
fil_flush_file_spaces();
+ binlog_write_up_to_now();
log_sys.latch.wr_lock(SRW_LOCK_CALL);
const lsn_t end_lsn= log_sys.get_lsn();
mysql_mutex_lock(&buf_pool.flush_list_mutex);
const lsn_t oldest_lsn= buf_pool.get_oldest_modification(end_lsn);
+ // FIXME: limit oldest_lsn below binlog split write LSN
mysql_mutex_unlock(&buf_pool.flush_list_mutex);
return log_checkpoint_low(oldest_lsn, end_lsn);
}
@@ -2267,12 +2291,14 @@ static void buf_flush_sync_for_checkpoint(lsn_t lsn) noexcept
os_aio_wait_until_no_pending_writes(false);
fil_flush_file_spaces();
+ binlog_write_up_to_now();
log_sys.latch.wr_lock(SRW_LOCK_CALL);
const lsn_t newest_lsn= log_sys.get_lsn();
mysql_mutex_lock(&buf_pool.flush_list_mutex);
lsn_t measure= buf_pool.get_oldest_modification(0);
const lsn_t checkpoint_lsn= measure ? measure : newest_lsn;
+ // FIXME: limit checkpoint_lsn below binlog split write LSN
if (!recv_recovery_is_on() &&
checkpoint_lsn > log_sys.last_checkpoint_lsn + SIZE_OF_FILE_CHECKPOINT)
diff --git a/storage/innobase/fil/fil0fil.cc b/storage/innobase/fil/fil0fil.cc
index f6f2331f6d258..4d1c5797658ef 100644
--- a/storage/innobase/fil/fil0fil.cc
+++ b/storage/innobase/fil/fil0fil.cc
@@ -186,7 +186,7 @@ it is an absolute path. */
const char* fil_path_to_mysql_datadir;
/** Common InnoDB file extensions */
-const char* dot_ext[] = { "", ".ibd", ".isl", ".cfg" };
+const char* dot_ext[] = { "", ".ibd", ".isl", ".cfg", ".ibb" };
/** Number of pending tablespace flushes */
Atomic_counter fil_n_pending_tablespace_flushes;
@@ -1568,9 +1568,10 @@ inline size_t mtr_t::log_file_op(mfile_type_t type, uint32_t space_id,
ut_ad(!is_predefined_tablespace(space_id));
/* fil_name_parse() requires that there be at least one path
- separator and that the file path end with ".ibd". */
+ separator and that the file path end with ".ibd" or "ibb". */
ut_ad(strchr(path, '/'));
- ut_ad(!strcmp(&path[strlen(path) - strlen(DOT_IBD)], DOT_IBD));
+ ut_ad(!strcmp(&path[strlen(path) - strlen(DOT_IBD)], DOT_IBD) ||
+ !strcmp(&path[strlen(path) - strlen(DOT_IBB)], DOT_IBB));
m_modifications= true;
if (!is_logged())
diff --git a/storage/innobase/fsp/fsp0fsp.cc b/storage/innobase/fsp/fsp0fsp.cc
index 1fa03e2099894..975c6cbb9f4d4 100644
--- a/storage/innobase/fsp/fsp0fsp.cc
+++ b/storage/innobase/fsp/fsp0fsp.cc
@@ -1055,8 +1055,8 @@ fsp_alloc_from_free_frag(buf_block_t *header, buf_block_t *xdes, xdes_t *descr,
@param[in] offset page number of the allocated page
@param[in,out] mtr mini-transaction
@return block, initialized */
-static buf_block_t* fsp_page_create(fil_space_t *space, uint32_t offset,
- mtr_t *mtr) noexcept
+buf_block_t* fsp_page_create(fil_space_t *space, uint32_t offset,
+ mtr_t *mtr) noexcept
{
buf_block_t *free_block= buf_LRU_get_free_block(have_no_mutex),
*block= buf_page_create(space, offset, space->zip_size(), mtr, free_block);
diff --git a/storage/innobase/fsp/fsp_binlog.cc b/storage/innobase/fsp/fsp_binlog.cc
new file mode 100644
index 0000000000000..e5a1d8d2833aa
--- /dev/null
+++ b/storage/innobase/fsp/fsp_binlog.cc
@@ -0,0 +1,2391 @@
+/*****************************************************************************
+
+Copyright (c) 2024, Kristian Nielsen
+
+This program is free software; you can redistribute it and/or modify it under
+the terms of the GNU General Public License as published by the Free Software
+Foundation; version 2 of the License.
+
+This program is distributed in the hope that it will be useful, but WITHOUT
+ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or FITNESS
+FOR A PARTICULAR PURPOSE. See the GNU General Public License for more details.
+
+You should have received a copy of the GNU General Public License along with
+this program; if not, write to the Free Software Foundation, Inc.,
+51 Franklin Street, Fifth Floor, Boston, MA 02110-1335 USA
+
+*****************************************************************************/
+
+/**************************************************//**
+@file fsp/fsp_binlog.cc
+InnoDB implementation of binlog.
+*******************************************************/
+
+#include
+#include "fsp0fsp.h"
+#include "buf0flu.h"
+#include "trx0trx.h"
+#include "fsp_binlog.h"
+#include "innodb_binlog.h"
+
+#include "my_bit.h"
+#include "rpl_gtid_base.h"
+#include "log.h"
+
+
+/**
+ The page size used for binlog pages.
+
+ For now, we just use a 16k page size. It could be changed later to be
+ configurable, changing the page size of the binlog is much easier than for
+ normal InnoDB tablespaces, as we could simply flush out the current file and
+ create the next file with a different page size, just need to put the page
+ size somewhere in the file header.
+
+ On the other hand, the page size does not seem to be very significant for
+ performance or anything. All data can be split across to the next page, and
+ everything is written in sequence through the kernel buffer cache which is
+ then free to flush it to disk in whatever chunk sizes it wants.
+*/
+uint32_t ibb_page_size_shift= 14;
+ulong ibb_page_size= (1 << ibb_page_size_shift);
+
+
+/**
+ How often (in terms of pages written) to dump a (differential) binlog state
+ at the start of the page, to speed up finding the initial GTID position for
+ a connecting slave.
+
+ This value must be used over the setting innodb_binlog_state_interval,
+ because after a restart the latest binlog file will be using the value of the
+ setting prior to the restart; the new value of the setting (if different)
+ will be used for newly created binlog files. The value refers to the file
+ of active_binlog_file_no.
+*/
+uint64_t current_binlog_state_interval;
+
+/**
+ Mutex protecting active_binlog_file_no.
+*/
+mysql_mutex_t active_binlog_mutex;
+pthread_cond_t active_binlog_cond;
+/** Mutex protecting binlog_cur_durable_offset[] and ibb_pending_lsn_fifo. */
+mysql_mutex_t binlog_durable_mutex;
+mysql_cond_t binlog_durable_cond;
+
+/** The currently being written binlog tablespace. */
+std::atomic active_binlog_file_no;
+
+/**
+ The first binlog tablespace that is still open.
+ This can be equal to active_binlog_file_no, if the tablespace prior to the
+ active one has been fully flushed out to disk and closed.
+ Or it can be one less, if the prior tablespace is still being written out and
+ closed.
+*/
+uint64_t first_open_binlog_file_no;
+
+/**
+ The most recent created and open tablespace.
+ This can be equal to active_binlog_file_no+1, if the next tablespace to be
+ used has already been pre-allocated and opened.
+ Or it can be the same as active_binlog_file_no, if the pre-allocation of the
+ next tablespace is still pending.
+*/
+uint64_t last_created_binlog_file_no;
+
+/**
+ Point at which it is guaranteed that all data has been written out to the
+ binlog file (on the OS level; not necessarily fsync()'ed yet).
+
+ Stores the most recent four values, each corresponding to
+ active_binlog_file_no&4. This is so that it can be always valid for both
+ active and active-1 (active-2 is always durable, as we make the entire binlog
+ file N durable before pre-allocating N+2). Just before active moves to the
+ next file_no, we can set the value for active+1, leaving active and active-1
+ still valid. (Only 3 entries are needed, but we use four to be able to use
+ bit-wise and instead of modulo-3).
+*/
+std::atomic binlog_cur_durable_offset[4];
+/**
+ Offset of last valid byte of data in most recent 4 binlog files.
+ A value of ~0 means that file is not opened as a tablespace (and data is
+ valid until the end of the file).
+*/
+std::atomic binlog_cur_end_offset[4];
+
+fsp_binlog_page_fifo *binlog_page_fifo;
+
+/** Object to keep track of outstanding oob references in binlog files. */
+ibb_file_oob_refs ibb_file_hash;
+
+
+fsp_binlog_page_entry *
+fsp_binlog_page_fifo::get_entry(uint64_t file_no, uint64_t page_no,
+ uint32_t latch, bool completed, bool clean)
+{
+ mysql_mutex_assert_owner(&m_mutex);
+ ut_a(file_no == first_file_no || file_no == first_file_no + 1);
+ page_list *pl= &fifos[file_no & 1];
+ ut_ad(pl->first_page_no + pl->used_entries == page_no);
+ if (UNIV_UNLIKELY(pl->used_entries == pl->allocated_entries))
+ {
+ size_t new_allocated_entries= 2*pl->allocated_entries;
+ size_t new_size= new_allocated_entries * sizeof(*pl->entries);
+ fsp_binlog_page_entry **new_entries=
+ (fsp_binlog_page_entry **)ut_realloc(pl->entries, new_size);
+ if (!new_entries)
+ return nullptr;
+ /* Copy any wrapped-around elements into not-wrapped new locations. */
+ if (pl->first_entry + pl->used_entries > pl->allocated_entries)
+ {
+ size_t wrapped_entries=
+ pl->first_entry + pl->used_entries - pl->allocated_entries;
+ ut_ad(new_allocated_entries >= pl->allocated_entries + wrapped_entries);
+ memcpy(new_entries + pl->allocated_entries, new_entries,
+ wrapped_entries * sizeof(*new_entries));
+ }
+ pl->entries= new_entries;
+ pl->allocated_entries= new_allocated_entries;
+ }
+ fsp_binlog_page_entry *&e_loc= pl->entry_at(pl->used_entries);
+ ++pl->used_entries;
+ if (UNIV_LIKELY(freelist != nullptr))
+ {
+ e_loc= (fsp_binlog_page_entry *)freelist;
+ freelist= (byte *)*(uintptr_t *)freelist;
+ --free_buffers;
+ }
+ else
+ {
+ byte *mem=
+ static_cast(ut_malloc(sizeof(*e_loc) + ibb_page_size,
+ mem_key_binlog));
+ if (!mem)
+ return nullptr;
+ e_loc= (fsp_binlog_page_entry *)mem;
+ }
+ e_loc->latched= latch;
+ e_loc->last_page= (page_no + 1 == size_in_pages(file_no));
+ e_loc->complete= completed;
+ e_loc->flushed_clean= clean;
+ e_loc->pending_flush= false;
+ return e_loc;
+}
+
+
+void
+fsp_binlog_page_fifo::release_entry(uint64_t file_no, uint64_t page_no)
+{
+ ut_a(file_no == first_file_no || file_no == first_file_no + 1);
+ page_list *pl= &fifos[file_no & 1];
+ ut_a(page_no == pl->first_page_no);
+ fsp_binlog_page_entry *e= pl->entries[pl->first_entry];
+ ut_ad(pl->used_entries > 0);
+ --pl->used_entries;
+ ++pl->first_entry;
+ ++pl->first_page_no;
+ if (UNIV_UNLIKELY(pl->first_entry == pl->allocated_entries))
+ pl->first_entry= 0;
+ /*
+ Put the page buffer on the freelist. Unless we already have too much on the
+ freelist; then put it on a temporary list so it can be freed later, outside
+ of holding the mutex.
+ */
+ if (UNIV_LIKELY(free_buffers * MAX_FREE_BUFFERS_FRAC <=
+ innodb_binlog_size_in_pages))
+ {
+ *(uintptr_t *)e= (uintptr_t)freelist;
+ freelist= (byte *)e;
+ ++free_buffers;
+ }
+ else
+ {
+ *(uintptr_t *)e= (uintptr_t)to_free_list;
+ to_free_list= (byte *)e;
+ }
+}
+
+
+void
+fsp_binlog_page_fifo::unlock_with_delayed_free()
+{
+ mysql_mutex_assert_owner(&m_mutex);
+ byte *to_free= to_free_list;
+ to_free_list= nullptr;
+ mysql_mutex_unlock(&m_mutex);
+ if (UNIV_UNLIKELY(to_free != nullptr))
+ {
+ do
+ {
+ byte *next= (byte *)*(uintptr_t *)to_free;
+ ut_free(to_free);
+ to_free= next;
+ } while (to_free);
+ }
+}
+
+
+fsp_binlog_page_entry *
+fsp_binlog_page_fifo::create_page(uint64_t file_no, uint32_t page_no)
+{
+ mysql_mutex_lock(&m_mutex);
+ ut_ad(first_file_no != ~(uint64_t)0);
+ ut_a(file_no == first_file_no || file_no == first_file_no + 1);
+ /* Can only allocate pages consecutively. */
+ ut_a(page_no == fifos[file_no & 1].first_page_no +
+ fifos[file_no & 1].used_entries);
+ fsp_binlog_page_entry *e= get_entry(file_no, page_no, 1, false, false);
+ ut_a(e);
+ mysql_mutex_unlock(&m_mutex);
+ memset(e->page_buf(), 0, ibb_page_size);
+
+ return e;
+}
+
+
+fsp_binlog_page_entry *
+fsp_binlog_page_fifo::get_page(uint64_t file_no, uint32_t page_no)
+{
+ fsp_binlog_page_entry *res= nullptr;
+ page_list *pl;
+
+ mysql_mutex_lock(&m_mutex);
+
+ ut_ad(first_file_no != ~(uint64_t)0);
+ ut_a(file_no <= first_file_no + 1);
+ if (file_no < first_file_no)
+ goto end;
+ pl= &fifos[file_no & 1];
+ if (page_no >= pl->first_page_no &&
+ page_no < pl->first_page_no + pl->used_entries)
+ {
+ res= pl->entry_at(page_no - pl->first_page_no);
+ ++res->latched;
+ }
+
+end:
+ mysql_mutex_unlock(&m_mutex);
+ return res;
+}
+
+
+void
+fsp_binlog_page_fifo::release_page(fsp_binlog_page_entry *page)
+{
+ mysql_mutex_lock(&m_mutex);
+ ut_a(page->latched > 0);
+ if (--page->latched == 0 && (page->complete || page->pending_flush))
+ pthread_cond_broadcast(&m_cond); /* Page ready to be flushed to disk */
+ mysql_mutex_unlock(&m_mutex);
+}
+
+
+/**
+ Release a page that is part of an mtr, except that if this is the last page
+ of a binlog tablespace, then delay release until mtr commit.
+
+ This is used to make sure that a tablespace is not closed until any mtr that
+ modified it has been committed and the modification redo logged. This way, a
+ closed tablespace never needs recovery and at most the two most recent binlog
+ tablespaces need to be considered during recovery.
+*/
+void
+fsp_binlog_page_fifo::release_page_mtr(fsp_binlog_page_entry *page, mtr_t *mtr)
+{
+ if (!page->last_page)
+ return release_page(page);
+
+ /*
+ Check against having two pending last-in-binlog-file pages to release.
+ But allow to have the same page released twice in a single mtr (this can
+ happen when 2-phase commit puts an XID/XA complete record just in front
+ of the commit record).
+ */
+ fsp_binlog_page_entry *old_page= mtr->get_binlog_page();
+ ut_ad(!(old_page != nullptr && old_page != page));
+ if (UNIV_UNLIKELY(old_page != nullptr))
+ {
+ if (UNIV_UNLIKELY(old_page != page))
+ sql_print_error("InnoDB: Internal inconsistency with mini-transaction "
+ "that spans more than two binlog files. Recovery may "
+ "be affected until the next checkpoint.");
+ release_page(old_page);
+ }
+ mtr->set_binlog_page(page);
+}
+
+
+/* Check if there are any (complete) non-flushed pages in a tablespace. */
+bool
+fsp_binlog_page_fifo::has_unflushed(uint64_t file_no)
+{
+ mysql_mutex_assert_owner(&m_mutex);
+ if (UNIV_UNLIKELY(file_no < first_file_no))
+ return false;
+ if (UNIV_UNLIKELY(file_no > first_file_no + 1))
+ return false;
+ const page_list *pl= &fifos[file_no & 1];
+ if (pl->used_entries == 0)
+ return false;
+ if (!pl->entries[pl->first_entry]->complete)
+ return false;
+ ut_ad(!pl->entries[pl->first_entry]->flushed_clean
+ /* Clean and complete page should have been freed */);
+ return true;
+}
+
+
+/**
+ Flush (write to disk) the first unflushed page in a file.
+ Returns true when the last page has been flushed.
+
+ Must be called with m_mutex held.
+
+ If called with force=true, will flush even any final, incomplete page.
+ Otherwise such page will not be written out. Any final, incomplete page
+ is left in the FIFO in any case.
+*/
+void
+fsp_binlog_page_fifo::flush_one_page(uint64_t file_no, bool force)
+{
+ page_list *pl;
+ fsp_binlog_page_entry *e;
+
+ mysql_mutex_assert_owner(&m_mutex);
+ /*
+ Wait for the FIFO to be not flushing from another thread, and for the
+ first page to not be latched.
+ */
+ for (;;)
+ {
+ /*
+ Let's make page not present not an error, to allow races where someone else
+ flushed the page ahead of us.
+ */
+ if (file_no < first_file_no)
+ return;
+ /* Guard against simultaneous RESET MASTER. */
+ if (file_no > first_file_no + 1)
+ return;
+
+ if (!flushing)
+ {
+ pl= &fifos[file_no & 1];
+ if (pl->used_entries == 0)
+ return;
+ e= pl->entries[pl->first_entry];
+ if (e->latched == 0)
+ break;
+ if (force)
+ e->pending_flush= true;
+ }
+ my_cond_wait(&m_cond, &m_mutex.m_mutex);
+ }
+ flushing= true;
+ uint32_t page_no= pl->first_page_no;
+ bool is_complete= e->complete;
+ ut_ad(is_complete || pl->used_entries == 1);
+ if (is_complete || (force && !e->flushed_clean))
+ {
+ /*
+ Careful here! We are going to release the mutex while flushing the page
+ to disk. At this point, another thread might come in and add more data
+ to the page in parallel, if e->complete is not set!
+
+ So here we set flushed_clean _before_ releasing the mutex. Then any
+ other thread that in parallel latches the page and tries to update it in
+ parallel will either increment e->latched, or set e->flushed_clean back
+ to false (or both). This allows us to detect a parallel update and retry
+ the write in that case.
+ */
+ retry:
+ if (!is_complete)
+ e->flushed_clean= true;
+ e->pending_flush= false;
+ /* Release the mutex, then free excess page buffers while not holding it. */
+ unlock_with_delayed_free();
+ File fh= get_fh(file_no);
+ ut_a(pl->fh >= (File)0);
+ size_t res= crc32_pwrite_page(fh, e->page_buf(), page_no, MYF(MY_WME));
+ ut_a(res == ibb_page_size);
+ mysql_mutex_lock(&m_mutex);
+ if (UNIV_UNLIKELY(e->latched) ||
+ (!is_complete && UNIV_UNLIKELY(!e->flushed_clean)))
+ {
+ flushing= false;
+ pthread_cond_broadcast(&m_cond);
+ for (;;)
+ {
+ ut_ad(file_no < first_file_no ||
+ pl->first_page_no >= page_no);
+ ut_ad(file_no < first_file_no ||
+ pl->first_page_no > page_no ||
+ pl->entries[pl->first_entry] == e);
+ if (!flushing)
+ {
+ if (file_no < first_file_no ||
+ pl->first_page_no != page_no ||
+ pl->entries[pl->first_entry] != e)
+ {
+ /* Someone else flushed the page for us. */
+ return;
+ }
+ /* Guard against simultaneous RESET MASTER. */
+ if (file_no > first_file_no + 1)
+ return;
+ if (e->latched == 0)
+ break;
+ if (force)
+ e->pending_flush= true;
+ }
+ my_cond_wait(&m_cond, &m_mutex.m_mutex);
+ }
+ flushing= true;
+ if (!is_complete)
+ {
+ /*
+ The page was not complete, a writer may have added data. Need to redo
+ the flush.
+ */
+ is_complete= e->complete;
+ goto retry;
+ }
+ /*
+ The page was complete, but was latched while we were flushing (by a
+ reader). No need to flush again, just needed to wait until the latch
+ was released before we can continue to free the page.
+ */
+ }
+ }
+ /*
+ We marked the FIFO as flushing, page could not have disappeared despite
+ releasing the mutex during the I/O.
+ */
+ ut_ad(flushing);
+ ut_ad(pl->used_entries >= 1);
+ if (is_complete)
+ release_entry(file_no, page_no);
+ flushing= false;
+ pthread_cond_broadcast(&m_cond);
+}
+
+
+void
+fsp_binlog_page_fifo::flush_up_to(uint64_t file_no, uint32_t page_no)
+{
+ mysql_mutex_lock(&m_mutex);
+ for (;;)
+ {
+ const page_list *pl= &fifos[file_no & 1];
+ if (file_no < first_file_no ||
+ (file_no == first_file_no && pl->first_page_no > page_no))
+ break;
+ /* Guard against simultaneous RESET MASTER. */
+ if (file_no > first_file_no + 1)
+ break;
+ /*
+ The flush is complete if there are no pages left, or if there is just
+ one incomplete page left that is fully flushed so far.
+ */
+ if (pl->used_entries == 0 ||
+ (pl->used_entries == 1 && !pl->entries[pl->first_entry]->complete &&
+ pl->entries[pl->first_entry]->flushed_clean))
+ break;
+ uint64_t file_no_to_flush= file_no;
+ /* Flush the prior file to completion first. */
+ if (file_no == first_file_no + 1 && fifos[(file_no - 1) & 1].used_entries)
+ {
+ file_no_to_flush= file_no - 1;
+ pl= &fifos[file_no_to_flush & 1];
+ ut_ad(pl->entries[pl->first_entry]->complete);
+ }
+ flush_one_page(file_no_to_flush, true);
+ }
+ /* Will release the mutex and free any excess page buffers. */
+ unlock_with_delayed_free();
+}
+
+
+void
+fsp_binlog_page_fifo::do_fdatasync(uint64_t file_no)
+{
+ File fh;
+ mysql_mutex_lock(&m_mutex);
+ for (;;)
+ {
+ if (file_no < first_file_no)
+ break; /* Old files are already fully synced. */
+ /* Guard against simultaneous RESET MASTER. */
+ if (file_no > first_file_no + 1)
+ break;
+ fh= fifos[file_no & 1].fh;
+ if (fh <= (File)-1)
+ break;
+ if (flushing)
+ {
+ while (flushing)
+ my_cond_wait(&m_cond, &m_mutex.m_mutex);
+ continue; /* Loop again to recheck state, as we released the mutex */
+ }
+ flushing= true;
+ mysql_mutex_unlock(&m_mutex);
+ int res= my_sync(fh, MYF(MY_WME));
+ ut_a(!res);
+ mysql_mutex_lock(&m_mutex);
+ flushing= false;
+ pthread_cond_broadcast(&m_cond);
+ break;
+ }
+ mysql_mutex_unlock(&m_mutex);
+}
+
+
+File
+fsp_binlog_page_fifo::get_fh(uint64_t file_no)
+{
+ File fh= fifos[file_no & 1].fh;
+ if (fh == (File)-1)
+ {
+ char filename[OS_FILE_MAX_PATH];
+ binlog_name_make(filename, file_no);
+ fifos[file_no & 1].fh= fh= my_open(filename, O_RDWR | O_BINARY, MYF(MY_WME));
+ }
+ return fh;
+}
+
+/**
+ If init_page is not ~(uint32_t)0, then it is the page to continue writing
+ when re-opening existing binlog at server startup.
+
+ If in addition, partial_page is non-NULL, it is an (aligned) page buffer
+ containing the partial data of page init_page.
+
+ If init_page is set but partial_page is not, then init_page is the first,
+ empty page in the tablespace to create and start writing to.
+*/
+void
+fsp_binlog_page_fifo::create_tablespace(uint64_t file_no,
+ uint32_t size_in_pages,
+ uint32_t init_page,
+ byte *partial_page)
+{
+ mysql_mutex_lock(&m_mutex);
+ ut_ad(init_page == ~(uint32_t)0 ||
+ first_file_no == ~(uint64_t)0 ||
+ /* At server startup allow opening N empty and (N-1) partial. */
+ (init_page != ~(uint32_t)0 && file_no + 1 == first_file_no &&
+ fifos[first_file_no & 1].used_entries == 0));
+ ut_a(first_file_no == ~(uint64_t)0 ||
+ file_no == first_file_no + 1 ||
+ file_no == first_file_no + 2 ||
+ (init_page != ~(uint32_t)0 && file_no + 1 == first_file_no &&
+ fifos[first_file_no & 1].used_entries == 0));
+ page_list *pl= &fifos[file_no & 1];
+ if (first_file_no == ~(uint64_t)0)
+ {
+ first_file_no= file_no;
+ }
+ else if (UNIV_UNLIKELY(file_no + 1 == first_file_no))
+ first_file_no= file_no;
+ else if (file_no == first_file_no + 2)
+ {
+ /* All pages in (N-2) must be flushed before doing (N). */
+ ut_a(pl->used_entries == 0);
+ if (UNIV_UNLIKELY(pl->fh != (File)-1))
+ {
+ ut_ad(false /* Should have been done as part of tablespace close. */);
+ my_close(pl->fh, MYF(0));
+ }
+ first_file_no= file_no - 1;
+ }
+
+ pl->fh= (File)-1;
+ pl->size_in_pages= size_in_pages;
+ ut_ad(pl->used_entries == 0);
+ ut_ad(pl->first_entry == 0);
+ if (UNIV_UNLIKELY(init_page != ~(uint32_t)0))
+ {
+ pl->first_page_no= init_page;
+ if (partial_page)
+ {
+ fsp_binlog_page_entry *e= get_entry(file_no, init_page, 0, false, true);
+ ut_a(e);
+ memcpy(e->page_buf(), partial_page, ibb_page_size);
+ }
+ }
+ else
+ pl->first_page_no= 0;
+ pthread_cond_broadcast(&m_cond);
+ mysql_mutex_unlock(&m_mutex);
+}
+
+
+void
+fsp_binlog_page_fifo::release_tablespace(uint64_t file_no)
+{
+ mysql_mutex_lock(&m_mutex);
+ page_list *pl= &fifos[file_no & 1];
+ ut_a(file_no == first_file_no);
+ ut_a(pl->used_entries == 0 ||
+ /* Allow a final, incomplete-but-fully-flushed page in the fifo. */
+ (!pl->entries[pl->first_entry]->complete &&
+ pl->entries[pl->first_entry]->flushed_clean &&
+ pl->used_entries == 1 &&
+ fifos[(file_no + 1) & 1].used_entries == 0));
+ if (pl->fh != (File)-1)
+ {
+ while (flushing)
+ my_cond_wait(&m_cond, &m_mutex.m_mutex);
+ flushing= true;
+ File fh= pl->fh;
+ mysql_mutex_unlock(&m_mutex);
+ int res= my_sync(fh, MYF(MY_WME));
+ ut_a(!res);
+ mysql_mutex_lock(&m_mutex);
+ free_page_list(file_no);
+ flushing= false;
+ pthread_cond_broadcast(&m_cond);
+ }
+ first_file_no= file_no + 1;
+ mysql_mutex_unlock(&m_mutex);
+}
+
+
+fsp_binlog_page_fifo::fsp_binlog_page_fifo()
+ : first_file_no(~(uint64_t)0), free_buffers(0), freelist(nullptr),
+ to_free_list(nullptr), flushing(false),
+ flush_thread_started(false), flush_thread_end(false)
+{
+ for (unsigned i= 0; i < 2; ++i)
+ {
+ fifos[i].allocated_entries= 64;
+ fifos[i].entries=
+ (fsp_binlog_page_entry **)ut_malloc(fifos[i].allocated_entries *
+ sizeof(fsp_binlog_page_entry *),
+ mem_key_binlog);
+ ut_a(fifos[i].entries);
+ fifos[i].used_entries= 0;
+ fifos[i].first_entry= 0;
+ fifos[i].first_page_no= 0;
+ fifos[i].size_in_pages= 0;
+ fifos[i].fh= (File)-1;
+ }
+ mysql_mutex_init(fsp_page_fifo_mutex_key, &m_mutex, nullptr);
+ pthread_cond_init(&m_cond, nullptr);
+}
+
+
+void
+fsp_binlog_page_fifo::free_page_list(uint64_t file_no)
+{
+ page_list *pl= &fifos[file_no & 1];
+ if (pl->fh != (File)-1)
+ my_close(pl->fh, MYF(0));
+ while (pl->used_entries > 0)
+ {
+ memset(pl->entries[pl->first_entry]->page_buf(), 0, ibb_page_size);
+ release_entry(file_no, pl->first_page_no);
+ }
+ /* We hold on to the pl->entries array and reuse for next tablespace. */
+ pl->used_entries= 0;
+ pl->first_entry= 0;
+ pl->first_page_no= 0;
+ pl->size_in_pages= 0;
+ pl->fh= (File)-1;
+}
+
+
+void
+fsp_binlog_page_fifo::reset()
+{
+ ut_ad(!flushing);
+ if (first_file_no != ~(uint64_t)0)
+ {
+ for (uint32_t i= 0; i < 2; ++i)
+ free_page_list(first_file_no + i);
+ }
+ first_file_no= ~(uint64_t)0;
+ /* Release page buffers in the freelist. */
+ while (freelist)
+ {
+ byte *q= (byte *)*(uintptr_t *)freelist;
+ ut_free(freelist);
+ freelist= q;
+ }
+ free_buffers= 0;
+ while (to_free_list)
+ {
+ byte *q= (byte *)*(uintptr_t *)to_free_list;
+ ut_free(to_free_list);
+ to_free_list= q;
+ }
+}
+
+
+fsp_binlog_page_fifo::~fsp_binlog_page_fifo()
+{
+ ut_ad(!flushing);
+ reset();
+ for (uint32_t i= 0; i < 2; ++i)
+ ut_free(fifos[i].entries);
+ mysql_mutex_destroy(&m_mutex);
+ pthread_cond_destroy(&m_cond);
+}
+
+
+void
+fsp_binlog_page_fifo::lock_wait_for_idle()
+{
+ mysql_mutex_lock(&m_mutex);
+ while(flushing)
+ my_cond_wait(&m_cond, &m_mutex.m_mutex);
+}
+
+
+void
+fsp_binlog_page_fifo::start_flush_thread()
+{
+ flush_thread_started= false;
+ flush_thread_end= false;
+ flush_thread_obj= std::thread{ [this] { flush_thread_run(); } };
+ mysql_mutex_lock(&m_mutex);
+ while (!flush_thread_started)
+ my_cond_wait(&m_cond, &m_mutex.m_mutex);
+ mysql_mutex_unlock(&m_mutex);
+}
+
+
+void
+fsp_binlog_page_fifo::stop_flush_thread()
+{
+ if (!flush_thread_started)
+ return;
+ mysql_mutex_lock(&m_mutex);
+ flush_thread_end= true;
+ pthread_cond_broadcast(&m_cond);
+ while (flush_thread_started)
+ my_cond_wait(&m_cond, &m_mutex.m_mutex);
+ mysql_mutex_unlock(&m_mutex);
+ flush_thread_obj.join();
+}
+
+
+void
+fsp_binlog_page_fifo::flush_thread_run()
+{
+ mysql_mutex_lock(&m_mutex);
+ flush_thread_started= true;
+ pthread_cond_broadcast(&m_cond);
+
+ while (!flush_thread_end)
+ {
+ /*
+ Flush pages one by one as long as there are more pages pending.
+ Once all have been flushed, wait for more pages to become pending.
+ Don't try to force flush a final page that is not yet completely
+ filled with data.
+ */
+ uint64_t file_no= first_file_no;
+ if (first_file_no != ~(uint64_t)0)
+ {
+ if (has_unflushed(file_no))
+ {
+ flush_one_page(file_no, false);
+ continue; // Check again for more pages available to flush
+ }
+ else if (has_unflushed(file_no + 1))
+ {
+ flush_one_page(file_no + 1, false);
+ continue;
+ }
+ }
+ if (!flush_thread_end)
+ my_cond_wait(&m_cond, &m_mutex.m_mutex);
+ }
+
+ flush_thread_started= false;
+ pthread_cond_broadcast(&m_cond);
+ mysql_mutex_unlock(&m_mutex);
+}
+
+
+size_t
+crc32_pwrite_page(File fd, byte *buf, uint32_t page_no, myf MyFlags) noexcept
+{
+ const uint32_t payload= (uint32_t)ibb_page_size - BINLOG_PAGE_CHECKSUM;
+ int4store(buf + payload, my_crc32c(0, buf, payload));
+ return my_pwrite(fd, (const uchar *)buf, ibb_page_size,
+ (my_off_t)page_no << ibb_page_size_shift, MyFlags);
+}
+
+
+/**
+ Read a page, with CRC check.
+ Returns:
+
+ -1 error
+ 0 EOF
+ 1 Ok
+*/
+int
+crc32_pread_page(File fd, byte *buf, uint32_t page_no, myf MyFlags) noexcept
+{
+ size_t read= my_pread(fd, buf, ibb_page_size,
+ (my_off_t)page_no << ibb_page_size_shift, MyFlags);
+ int res= 1;
+ if (UNIV_LIKELY(read == ibb_page_size))
+ {
+ const uint32_t payload= (uint32_t)ibb_page_size - BINLOG_PAGE_CHECKSUM;
+ uint32_t crc32= uint4korr(buf + payload);
+ /* Allow a completely zero (empty) page as well. */
+ if (UNIV_UNLIKELY(crc32 != my_crc32c(0, buf, payload)) &&
+ (buf[0] != 0 || 0 != memcmp(buf, buf+1, ibb_page_size - 1)))
+ {
+ res= -1;
+ my_errno= EIO;
+ if (MyFlags & MY_WME)
+ {
+ sql_print_error("InnoDB: Page corruption in binlog tablespace file "
+ "page number %u (invalid crc32 checksum 0x%08X)",
+ page_no, crc32);
+ my_error(ER_BINLOG_READ_EVENT_CHECKSUM_FAILURE, MYF(0));
+ }
+ }
+ }
+ else if (read == (size_t)-1)
+ {
+ if (MyFlags & MY_WME)
+ my_error(ER_READING_BINLOG_FILE, MYF(0), page_no, my_errno);
+ res= -1;
+ }
+ else
+ res= 0;
+
+ return res;
+}
+
+
+int
+crc32_pread_page(pfs_os_file_t fh, byte *buf, uint32_t page_no, myf MyFlags)
+ noexcept
+{
+ const uint32_t page_size= (uint32_t)ibb_page_size;
+ ulint bytes_read= 0;
+ dberr_t err= os_file_read(IORequestRead, fh, buf,
+ (os_offset_t)page_no << ibb_page_size_shift,
+ page_size, &bytes_read);
+ if (UNIV_UNLIKELY(err != DB_SUCCESS))
+ return -1;
+ else if (UNIV_UNLIKELY(bytes_read < page_size))
+ return 0;
+
+ const uint32_t payload= (uint32_t)ibb_page_size - BINLOG_PAGE_CHECKSUM;
+ uint32_t crc32= uint4korr(buf + payload);
+ /* Allow a completely zero (empty) page as well. */
+ if (UNIV_UNLIKELY(crc32 != my_crc32c(0, buf, payload)) &&
+ (buf[0] != 0 || 0 != memcmp(buf, buf+1, ibb_page_size - 1)))
+ {
+ my_errno= EIO;
+ if (MyFlags & MY_WME)
+ sql_print_error("InnoDB: Page corruption in binlog tablespace file "
+ "page number %u (invalid crc32 checksum 0x%08X)",
+ page_no, crc32);
+ return -1;
+ }
+ return 1;
+}
+
+
+/**
+ Need specific constructor/initializer for struct ibb_tblspc_entry stored in
+ the ibb_file_hash. This is a work-around for C++ abstractions that makes it
+ non-standard behaviour to memcpy() std::atomic objects.
+*/
+static void
+ibb_file_hash_constructor(uchar *arg)
+{
+ new(arg + LF_HASH_OVERHEAD) ibb_tblspc_entry();
+}
+
+
+static void
+ibb_file_hash_destructor(uchar *arg)
+{
+ ibb_tblspc_entry *e=
+ reinterpret_cast(arg + LF_HASH_OVERHEAD);
+ e->~ibb_tblspc_entry();
+}
+
+
+static void
+ibb_file_hash_initializer(LF_HASH *hash, void *dst, const void *src)
+{
+ const ibb_tblspc_entry *src_e= static_cast(src);
+ ibb_tblspc_entry *dst_e=
+ const_cast(static_cast(dst));
+ dst_e->file_no= src_e->file_no;
+ dst_e->oob_refs.store(src_e->oob_refs.load(std::memory_order_relaxed),
+ std::memory_order_relaxed);
+ dst_e->xa_refs.store(src_e->xa_refs.load(std::memory_order_relaxed),
+ std::memory_order_relaxed);
+ dst_e->oob_ref_file_no.store(src_e->oob_ref_file_no.load(std::memory_order_relaxed),
+ std::memory_order_relaxed);
+ dst_e->xa_ref_file_no.store(src_e->xa_ref_file_no.load(std::memory_order_relaxed),
+ std::memory_order_relaxed);
+}
+
+
+void
+ibb_file_oob_refs::init() noexcept
+{
+ lf_hash_init(&hash, sizeof(ibb_tblspc_entry), LF_HASH_UNIQUE,
+ offsetof(ibb_tblspc_entry, file_no),
+ sizeof(ibb_tblspc_entry::file_no), nullptr, nullptr);
+ hash.alloc.constructor= ibb_file_hash_constructor;
+ hash.alloc.destructor= ibb_file_hash_destructor;
+ hash.initializer= ibb_file_hash_initializer;
+ earliest_oob_ref= ~(uint64_t)0;
+ earliest_xa_ref= ~(uint64_t)0;
+}
+
+
+void
+ibb_file_oob_refs::destroy() noexcept
+{
+ lf_hash_destroy(&hash);
+}
+
+
+void
+ibb_file_oob_refs::remove(uint64_t file_no, LF_PINS *pins)
+{
+ lf_hash_delete(&hash, pins, &file_no, sizeof(file_no));
+}
+
+
+void
+ibb_file_oob_refs::remove_up_to(uint64_t file_no, LF_PINS *pins)
+{
+ for (;;)
+ {
+ int res= lf_hash_delete(&hash, pins, &file_no, sizeof(file_no));
+ if (res || file_no == 0)
+ break;
+ --file_no;
+ }
+}
+
+
+uint64_t
+ibb_file_oob_refs::oob_ref_inc(uint64_t file_no, LF_PINS *pins, bool do_xa)
+{
+ ibb_tblspc_entry *e= static_cast
+ (lf_hash_search(&hash, pins, &file_no, sizeof(file_no)));
+ if (!e)
+ return ~(uint64_t)0;
+ uint64_t refcnt= e->oob_refs.fetch_add(1, std::memory_order_acquire);
+ if (UNIV_UNLIKELY(do_xa))
+ refcnt= e->xa_refs.fetch_add(1, std::memory_order_acquire);
+ lf_hash_search_unpin(pins);
+ return refcnt + 1;
+}
+
+
+uint64_t
+ibb_file_oob_refs::oob_ref_dec(uint64_t file_no, LF_PINS *pins, bool do_xa)
+{
+ ibb_tblspc_entry *e= static_cast
+ (lf_hash_search(&hash, pins, &file_no, sizeof(file_no)));
+ if (!e)
+ return ~(uint64_t)0;
+ uint64_t oob_refcnt= e->oob_refs.fetch_sub(1, std::memory_order_acquire) - 1;
+ uint64_t ret_refcnt= oob_refcnt;
+ if (UNIV_UNLIKELY(do_xa))
+ {
+ mysql_mutex_assert_owner(&ibb_xa_xid_hash->xid_mutex);
+ ret_refcnt= e->xa_refs.fetch_sub(1, std::memory_order_acquire) - 1;
+ }
+ lf_hash_search_unpin(pins);
+ ut_ad(oob_refcnt != (uint64_t)0 - 1);
+
+ if (oob_refcnt == 0)
+ do_zero_refcnt_action(file_no, pins, false);
+ return ret_refcnt;
+}
+
+
+void
+ibb_file_oob_refs::do_zero_refcnt_action(uint64_t file_no, LF_PINS *pins,
+ bool active_moving)
+{
+ for (;;)
+ {
+ ibb_tblspc_entry *e= static_cast
+ (lf_hash_search(&hash, pins, &file_no, sizeof(file_no)));
+ if (!e)
+ return;
+ uint64_t refcnt= e->oob_refs.load(std::memory_order_acquire);
+ lf_hash_search_unpin(pins);
+ if (refcnt > 0)
+ return;
+ /*
+ Reference count reached zero. Check if this was the earliest_oob_ref
+ that reached zero, and if so move it to the next file. Repeat this
+ for consecutive refcount-is-zero file_no, in case N+1 reaches zero
+ before N does.
+
+ As records are written into the active binlog file, the refcount can
+ reach zero temporarily and then go up again, so do not move the
+ earliest_oob_ref ahead yet.
+
+ As the active is about to move to the next file, we check again, and
+ this time move the earliest_oob_ref if the refcount on the (previously)
+ active binlog file ended up at zero.
+ */
+ uint64_t active= active_binlog_file_no.load(std::memory_order_acquire);
+ ut_ad(file_no <= active + active_moving);
+ if (file_no >= active + active_moving)
+ return;
+ bool ok;
+ do
+ {
+ uint64_t read_file_no= earliest_oob_ref.load(std::memory_order_relaxed);
+ if (read_file_no != file_no)
+ break;
+ ok= earliest_oob_ref.compare_exchange_weak(read_file_no, file_no + 1,
+ std::memory_order_relaxed);
+ } while (!ok);
+ /* Handle any following file_no that may have dropped to zero earlier. */
+ ++file_no;
+ }
+}
+
+
+bool
+ibb_file_oob_refs::update_refs(uint64_t file_no, LF_PINS *pins,
+ uint64_t oob_ref, uint64_t xa_ref)
+{
+ ibb_tblspc_entry *e= static_cast
+ (lf_hash_search(&hash, pins, &file_no, sizeof(file_no)));
+ if (!e)
+ return false;
+ e->oob_ref_file_no.store(oob_ref, std::memory_order_relaxed);
+ e->xa_ref_file_no.store(xa_ref, std::memory_order_relaxed);
+ lf_hash_search_unpin(pins);
+ return true;
+}
+
+
+/*
+ This is called when an xa/xid refcount goes from 1->0 or 0->1, to update
+ the value of ibb_file_hash.earliest_xa_ref if necessary.
+*/
+void
+ibb_file_oob_refs::update_earliest_xa_ref(uint64_t ref_file_no, LF_PINS *pins)
+{
+ mysql_mutex_assert_owner(&ibb_xa_xid_hash->xid_mutex);
+ uint64_t file_no1= earliest_xa_ref.load(std::memory_order_relaxed);
+ if (file_no1 < ref_file_no)
+ {
+ /* Current is before the updated one, no change possible for now. */
+ return;
+ }
+ uint64_t file_no2= active_binlog_file_no.load(std::memory_order_acquire);
+ uint64_t file_no= ref_file_no;
+ for (;;)
+ {
+ if (file_no > file_no2)
+ {
+ /* No active XA anymore. */
+ file_no= ~(uint64_t)0;
+ break;
+ }
+ ibb_tblspc_entry *e= static_cast
+ (lf_hash_search(&hash, pins, &file_no, sizeof(file_no)));
+ if (!e)
+ {
+ ++file_no;
+ continue;
+ }
+ uint64_t refcnt= e->xa_refs.load(std::memory_order_acquire);
+ lf_hash_search_unpin(pins);
+ if (refcnt > 0)
+ break;
+ ++file_no;
+ }
+ earliest_xa_ref.store(file_no, std::memory_order_relaxed);
+}
+
+
+/**
+ Look up the earliest file with OOB references from a given file_no.
+ Insert a new entry into the file hash (reading the file header from disk)
+ if not there already.
+*/
+bool
+ibb_file_oob_refs::get_oob_ref_file_no(uint64_t file_no, LF_PINS *pins,
+ uint64_t *out_oob_ref_file_no)
+{
+ ibb_tblspc_entry *e= static_cast
+ (lf_hash_search(&hash, pins, &file_no, sizeof(file_no)));
+ if (e)
+ {
+ *out_oob_ref_file_no= e->oob_ref_file_no.load(std::memory_order_relaxed);
+ lf_hash_search_unpin(pins);
+ return false;
+ }
+
+ *out_oob_ref_file_no= ~(uint64_t)0;
+ byte *page_buf= static_cast(ut_malloc(ibb_page_size, mem_key_binlog));
+ if (!page_buf)
+ {
+ my_error(ER_OUTOFMEMORY, MYF(0), ibb_page_size);
+ return true;
+ }
+ char filename[OS_FILE_MAX_PATH];
+ binlog_name_make(filename, file_no);
+ File fh= my_open(filename, O_RDONLY | O_BINARY, MYF(0));
+ if (fh < (File)0)
+ {
+ my_error(ER_ERROR_ON_READ, MYF(0), filename, my_errno);
+ ut_free(page_buf);
+ return true;
+ }
+ int res= crc32_pread_page(fh, page_buf, 0, MYF(0));
+ my_close(fh, MYF(0));
+ if (res <= 0)
+ {
+ ut_free(page_buf);
+ my_error(ER_ERROR_ON_READ, MYF(0), filename, my_errno);
+ return true;
+ }
+ binlog_header_data header;
+ fsp_binlog_extract_header_page(page_buf, &header);
+ ut_free(page_buf);
+ if (header.is_invalid || header.is_empty)
+ {
+ my_error(ER_FILE_CORRUPT, MYF(0), filename);
+ return true;
+ }
+ *out_oob_ref_file_no= header.oob_ref_file_no;
+ if (ibb_record_in_file_hash(file_no, header.oob_ref_file_no,
+ header.xa_ref_file_no, pins))
+ return true;
+
+ return false;
+}
+
+
+/*
+ Check if a file_no contains oob data that is needed by an active
+ (ie. not committed) transaction. This is seen simply as having refcount
+ greater than 0.
+*/
+bool
+ibb_file_oob_refs::get_oob_ref_in_use(uint64_t file_no, LF_PINS *pins)
+{
+ ibb_tblspc_entry *e= static_cast
+ (lf_hash_search(&hash, pins, &file_no, sizeof(file_no)));
+ if (!e)
+ return false;
+
+ uint64_t refcnt= e->oob_refs.load(std::memory_order_relaxed);
+ lf_hash_search_unpin(pins);
+ return refcnt > 0;
+}
+
+
+/*
+ Check if there are any of the in-use binlog files that have refcount > 0
+ (meaning any references to oob data from active transactions).
+ Any such references must prevent a RESET MASTER, as otherwise they could
+ be committed with OOB references pointing to garbage data.
+*/
+bool
+ibb_file_oob_refs::check_any_oob_ref_in_use(uint64_t start_file_no,
+ uint64_t end_file_no,
+ LF_PINS *lf_pins)
+{
+ if (unlikely(start_file_no == ~(uint64_t)0)
+ || unlikely(end_file_no == ~(uint64_t)0))
+ return false;
+
+ for (uint64_t file_no= start_file_no; file_no <= end_file_no; ++file_no)
+ {
+ if (get_oob_ref_in_use(file_no, lf_pins))
+ return true;
+ }
+ return false;
+}
+
+
+bool
+ibb_record_in_file_hash(uint64_t file_no, uint64_t oob_ref, uint64_t xa_ref,
+ LF_PINS *in_pins)
+{
+ bool err= false;
+ LF_PINS *pins= in_pins ? in_pins : lf_hash_get_pins(&ibb_file_hash.hash);
+ if (!pins)
+ {
+ my_error(ER_OUT_OF_RESOURCES, MYF(0));
+ return true;
+ }
+ ibb_tblspc_entry entry;
+ entry.file_no= file_no;
+ entry.oob_refs.store(0, std::memory_order_relaxed);
+ entry.xa_refs.store(0, std::memory_order_relaxed);
+ entry.oob_ref_file_no.store(oob_ref, std::memory_order_relaxed);
+ entry.xa_ref_file_no.store(xa_ref, std::memory_order_relaxed);
+ int res= lf_hash_insert(&ibb_file_hash.hash, pins, &entry);
+ if (res)
+ {
+ ut_ad(res < 0 /* Should not get unique violation, never insert twice */);
+ sql_print_error("InnoDB: Could not initialize in-memory structure for "
+ "binlog tablespace file number %" PRIu64 ", %s", file_no,
+ (res < 0 ? "out of memory" : "internal error"));
+ err= true;
+ }
+ if (!in_pins)
+ lf_hash_put_pins(pins);
+ return err;
+}
+
+
+void
+binlog_write_up_to_now() noexcept
+{
+ fsp_binlog_page_fifo *fifo= binlog_page_fifo;
+ if (!fifo)
+ return; /* Startup eg. */
+
+ uint64_t active= active_binlog_file_no.load(std::memory_order_relaxed);
+ uint64_t active2;
+ uint32_t page_no;
+ do
+ {
+ active2= active;
+ page_no= binlog_cur_page_no;
+ active= active_binlog_file_no.load(std::memory_order_relaxed);
+ } while (UNIV_UNLIKELY(active != active2));
+
+ if (active != ~(uint64_t)0)
+ {
+ fifo->flush_up_to(active, page_no);
+ fifo->do_fdatasync(active);
+ }
+}
+
+
+void
+fsp_binlog_extract_header_page(const byte *page_buf,
+ binlog_header_data *out_header_data) noexcept
+{
+ uint32_t magic= uint4korr(page_buf);
+ uint32_t vers_major= uint4korr(page_buf + 8);
+ const uint32_t payload= IBB_HEADER_PAGE_SIZE - BINLOG_PAGE_CHECKSUM;
+ uint32_t crc32= uint4korr(page_buf + payload);
+ out_header_data->is_empty= false;
+ out_header_data->is_invalid= false;
+ if (crc32 != my_crc32c(0, page_buf, payload) ||
+ magic != IBB_MAGIC || vers_major > IBB_FILE_VERS_MAJOR)
+ {
+ if (page_buf[0] == 0 &&
+ 0 == memcmp(page_buf, page_buf+1, IBB_HEADER_PAGE_SIZE - 1))
+ out_header_data->is_empty= true;
+ else
+ out_header_data->is_invalid= true;
+ return;
+ }
+ out_header_data->page_size_shift= uint4korr(page_buf + 4);
+ out_header_data->vers_major= vers_major;
+ out_header_data->vers_minor= uint4korr(page_buf + 12);
+ out_header_data->file_no= uint8korr(page_buf + 16);
+ out_header_data-> page_count= uint8korr(page_buf + 24);
+ out_header_data-> start_lsn= uint8korr(page_buf + 32);
+ out_header_data-> diff_state_interval= uint8korr(page_buf + 40);
+ out_header_data->oob_ref_file_no= uint8korr(page_buf + 48);
+ out_header_data->xa_ref_file_no= uint8korr(page_buf + 56);
+}
+
+
+void
+fsp_log_binlog_write(mtr_t *mtr, fsp_binlog_page_entry *page,
+ uint64_t file_no, uint32_t page_no,
+ uint32_t page_offset, uint32_t len)
+{
+ ut_ad(page->latched);
+ if (page_offset + len >= ibb_page_size - BINLOG_PAGE_DATA_END)
+ page->complete= true;
+ if (page->flushed_clean)
+ {
+ /*
+ If the page with partial data has been written to the file system, then
+ redo log all the data on the page, to be sure we can still recover the
+ entire page reliably even if the latest checkpoint is after that partial
+ write.
+ */
+ len= page_offset + len;
+ page_offset= 0;
+ page->flushed_clean= false;
+ }
+ page_id_t page_id(LOG_BINLOG_ID_0 | static_cast(file_no & 1),
+ page_no);
+ mtr->write_binlog(page_id, (uint16_t)page_offset,
+ page_offset + &page->page_buf()[0], len);
+}
+
+
+void
+fsp_log_header_page(mtr_t *mtr, fsp_binlog_page_entry *page, uint64_t file_no,
+ uint32_t len)
+ noexcept
+{
+ page->complete= true;
+ page_id_t page_id(LOG_BINLOG_ID_0 | static_cast(file_no & 1), 0);
+ mtr->write_binlog(page_id, 0, &page->page_buf()[0], len);
+}
+
+
+/**
+ Initialize the InnoDB implementation of binlog.
+ Note that we do not create or open any binlog tablespaces here.
+ This is only done if InnoDB binlog is enabled on the server level.
+*/
+dberr_t
+fsp_binlog_init()
+{
+ mysql_mutex_init(fsp_active_binlog_mutex_key, &active_binlog_mutex, nullptr);
+ pthread_cond_init(&active_binlog_cond, nullptr);
+ mysql_mutex_init(fsp_binlog_durable_mutex_key, &binlog_durable_mutex, nullptr);
+ mysql_cond_init(fsp_binlog_durable_cond_key, &binlog_durable_cond, nullptr);
+ mysql_mutex_record_order(&binlog_durable_mutex, &active_binlog_mutex);
+
+ ibb_file_hash.init();
+ binlog_page_fifo= new fsp_binlog_page_fifo();
+ if (UNIV_UNLIKELY(!binlog_page_fifo))
+ {
+ sql_print_error("InnoDB: Could not allocate memory for the page fifo, "
+ "cannot proceed");
+ return DB_OUT_OF_MEMORY;
+ }
+ binlog_page_fifo->start_flush_thread();
+ return DB_SUCCESS;
+}
+
+
+void
+fsp_binlog_shutdown()
+{
+ binlog_page_fifo->stop_flush_thread();
+ delete binlog_page_fifo;
+ ibb_file_hash.destroy();
+ mysql_cond_destroy(&binlog_durable_cond);
+ mysql_mutex_destroy(&binlog_durable_mutex);
+ pthread_cond_destroy(&active_binlog_cond);
+ mysql_mutex_destroy(&active_binlog_mutex);
+}
+
+
+/** Write out all pages, flush, and close/detach a binlog tablespace.
+@param[in] file_no Index of the binlog tablespace
+@return DB_SUCCESS or error code */
+dberr_t
+fsp_binlog_tablespace_close(uint64_t file_no)
+{
+ binlog_page_fifo->flush_up_to(file_no, ~(uint32_t)0);
+ uint32_t size=
+ binlog_page_fifo->size_in_pages(file_no) << ibb_page_size_shift;
+ /* release_tablespace() will fdatasync() the file first. */
+ binlog_page_fifo->release_tablespace(file_no);
+ /*
+ Durably sync the redo log. This simplifies things a bit, as then we know
+ that we will not need to discard any data from an old binlog file during
+ recovery, at most from the latest two existing files.
+ */
+ log_buffer_flush_to_disk(true);
+ uint64_t end_offset=
+ binlog_cur_end_offset[file_no & 3].load(std::memory_order_relaxed);
+ binlog_cur_end_offset[file_no & 3].store(size, std::memory_order_relaxed);
+ /*
+ Wait for the last record in the file to be marked durably synced to the
+ (redo) log. We already ensured that the record is durable with the above
+ call to log_buffer_flush_to_disk(); this way, we ensure that the update
+ of binlog_cur_durable_offset[] happens correctly through the
+ ibb_pending_lsn_fifo, so that the current durable position will be
+ consistent with a recorded LSN, and a reader will not see EOF in the
+ middle of a record.
+ */
+ uint64_t dur_offset=
+ binlog_cur_durable_offset[file_no & 3].load(std:: memory_order_relaxed);
+ if (dur_offset < end_offset)
+ ibb_wait_durable_offset(file_no, end_offset);
+
+ return DB_SUCCESS;
+}
+
+
+/**
+ Open an existing tablespace. The filehandle fh is taken over by the tablespace
+ (or closed in case of error).
+*/
+bool
+fsp_binlog_open(const char *file_name, pfs_os_file_t fh,
+ uint64_t file_no, size_t file_size,
+ uint32_t init_page, byte *partial_page)
+{
+ const uint32_t page_size= (uint32_t)ibb_page_size;
+ const uint32_t page_size_shift= ibb_page_size_shift;
+
+ os_offset_t binlog_size= innodb_binlog_size_in_pages << ibb_page_size_shift;
+ if (init_page == ~(uint32_t)0 && file_size < binlog_size) {
+ /*
+ A crash may have left a partially pre-allocated file. If so, extend it
+ to the required size.
+ Note that this may also extend a previously pre-allocated file to the new
+ binlog configured size, if the configuration changed during server
+ restart.
+ */
+ if (!os_file_set_size(file_name, fh, binlog_size, false)) {
+ sql_print_warning("Failed to change the size of InnoDB binlog file '%s' "
+ "from %zu to %zu bytes (error code: %d)", file_name,
+ file_size, (size_t)binlog_size, errno);
+ } else {
+ file_size= (size_t)binlog_size;
+ }
+ }
+ if (file_size < 2*page_size)
+ {
+ sql_print_warning("InnoDB binlog file number %llu is too short (%zu bytes), "
+ "should be at least %u bytes",
+ file_no, file_size, 2*page_size);
+ os_file_close(fh);
+ return true;
+ }
+
+ binlog_page_fifo->create_tablespace(file_no,
+ (uint32_t)(file_size >> page_size_shift),
+ init_page, partial_page);
+ os_file_close(fh);
+ first_open_binlog_file_no= file_no;
+ if (last_created_binlog_file_no == ~(uint64_t)0 ||
+ file_no > last_created_binlog_file_no) {
+ last_created_binlog_file_no= file_no;
+ }
+ return false;
+}
+
+
+/** Create a binlog tablespace file
+@param[in] file_no Index of the binlog tablespace
+@return DB_SUCCESS or error code */
+dberr_t fsp_binlog_tablespace_create(uint64_t file_no, uint32_t size_in_pages,
+ LF_PINS *pins)
+{
+ pfs_os_file_t fh;
+ bool ret;
+
+ if(srv_read_only_mode)
+ return DB_ERROR;
+
+ char name[OS_FILE_MAX_PATH];
+ binlog_name_make(name, file_no);
+
+ os_file_create_subdirs_if_needed(name);
+
+try_again:
+ fh = os_file_create(
+ innodb_data_file_key, name,
+ OS_FILE_CREATE, OS_DATA_FILE, srv_read_only_mode, &ret);
+
+ if (!ret) {
+ os_file_close(fh);
+ return DB_ERROR;
+ }
+
+ /* We created the binlog file and now write it full of zeros */
+ if (!os_file_set_size(name, fh,
+ os_offset_t{size_in_pages} << ibb_page_size_shift)
+ ) {
+ char buf[MYSYS_STRERROR_SIZE];
+ ulong wait_sec= MY_WAIT_FOR_USER_TO_FIX_PANIC;
+ DBUG_EXECUTE_IF("ib_alloc_file_disk_full",
+ wait_sec= 2;);
+ my_strerror(buf, sizeof(buf), errno);
+ sql_print_error("InnoDB: Unable to allocate file %s: \"%s\". "
+ "Waiting %lu seconds before trying again...",
+ name, buf, wait_sec);
+ os_file_close(fh);
+ os_file_delete(innodb_data_file_key, name);
+ my_sleep(wait_sec * 1000000);
+ goto try_again;
+ }
+
+ /*
+ Enter an initial entry in the hash for this binlog tablespace file.
+ It will be later updated with the appropriate values when the file
+ first gets used and the header page is written.
+ */
+ ibb_record_in_file_hash(file_no, ~(uint64_t)0,~(uint64_t)0, pins);
+
+ binlog_page_fifo->create_tablespace(file_no, size_in_pages);
+ os_file_close(fh);
+
+ return DB_SUCCESS;
+}
+
+
+/**
+ Write out a binlog record.
+ Split into chucks that each fit on a page.
+ The data for the record is provided by a class derived from chunk_data_base.
+
+ As a special case, a record write of type FSP_BINLOG_TYPE_FILLER does not
+ write any record, but moves to the next tablespace and writes the initial
+ GTID state record, used for FLUSH BINARY LOGS.
+
+ Returns a pair of {file_no, offset} marking the start of the record written.
+*/
+std::pair
+fsp_binlog_write_rec(chunk_data_base *chunk_data, mtr_t *mtr, byte chunk_type,
+ LF_PINS *pins)
+{
+ uint32_t page_size= (uint32_t)ibb_page_size;
+ uint32_t page_size_shift= ibb_page_size_shift;
+ const uint32_t page_end= page_size - BINLOG_PAGE_DATA_END;
+ uint32_t page_no= binlog_cur_page_no;
+ uint32_t page_offset= binlog_cur_page_offset;
+ fsp_binlog_page_entry *block= nullptr;
+ uint64_t file_no= active_binlog_file_no.load(std::memory_order_relaxed);
+ uint64_t pending_prev_end_offset= 0;
+ uint64_t start_file_no= 0;
+ uint64_t start_offset= 0;
+
+ /*
+ Write out the event data in chunks of whatever size will fit in the current
+ page, until all data has been written.
+ */
+ byte cont_flag= 0;
+ for (;;) {
+ if (page_offset == BINLOG_PAGE_DATA) {
+ ut_ad(!block);
+ uint32_t file_size_in_pages= binlog_page_fifo->size_in_pages(file_no);
+ if (UNIV_UNLIKELY(page_no >= file_size_in_pages)) {
+ /*
+ Signal to the pre-allocation thread that this tablespace has been
+ written full, so that it can be closed and a new one pre-allocated
+ in its place. Then wait for a new tablespace to be pre-allocated that
+ we can use.
+
+ The normal case is that the next tablespace is already pre-allocated
+ and available; binlog tablespace N is active while (N+1) is being
+ pre-allocated. Only under extreme I/O pressure should we need to
+ stall here.
+ */
+ ut_ad(!pending_prev_end_offset);
+ pending_prev_end_offset= page_no << page_size_shift;
+ mysql_mutex_lock(&active_binlog_mutex);
+ while (last_created_binlog_file_no <= file_no) {
+ my_cond_wait(&active_binlog_cond, &active_binlog_mutex.m_mutex);
+ }
+
+ ++file_no;
+ file_size_in_pages= binlog_page_fifo->size_in_pages(file_no);
+ binlog_cur_durable_offset[file_no & 3].store(0, std::memory_order_relaxed);
+ binlog_cur_end_offset[file_no & 3].store(0, std::memory_order_relaxed);
+ pthread_cond_signal(&active_binlog_cond);
+ mysql_mutex_unlock(&active_binlog_mutex);
+ binlog_cur_page_no= page_no= 0;
+ current_binlog_state_interval=
+ (uint64_t)(innodb_binlog_state_interval >> page_size_shift);
+ }
+
+ /* Write the header page at the start of a binlog tablespace file. */
+ if (page_no == 0)
+ {
+ /* Active is moving to next file, so check if oob refcount of previous
+ file is zero.
+ */
+ if (UNIV_LIKELY(file_no > 0))
+ ibb_file_hash.do_zero_refcnt_action(file_no - 1, pins, true);
+
+ lsn_t start_lsn= log_get_lsn();
+ bool err= ibb_write_header_page(mtr, file_no, file_size_in_pages,
+ start_lsn,
+ current_binlog_state_interval, pins);
+ ut_a(!err);
+ page_no= 1;
+ }
+
+ /* Must be a power of two. */
+ ut_ad(current_binlog_state_interval == 0 ||
+ current_binlog_state_interval ==
+ (uint64_t)1 << (63 - my_nlz(current_binlog_state_interval)));
+
+ if (page_no == 1 ||
+ 0 == (page_no & (current_binlog_state_interval - 1))) {
+ if (page_no == 1) {
+ bool err;
+ rpl_binlog_state_base *binlog_state= &binlog_full_state;
+ binlog_diff_state.reset_nolock();
+ if (UNIV_UNLIKELY(file_no == 0 && page_no == 1) &&
+ (binlog_full_state.count_nolock() == 1))
+ {
+ /*
+ The gtid state written here includes the GTID for the event group
+ currently being written. This is precise when the event group
+ data begins before this point. If the event group happens to
+ start exactly on a binlog file boundary, it just means we will
+ have to read slightly more binlog data to find the starting point
+ of that GTID.
+
+ But there is an annoying case if this is the very first binlog
+ file created (no migration from legacy binlog). If we start the
+ binlog with some GTID 0-1-1 and write the state "0-1-1" at the
+ start of the first file, then we will be unable to start
+ replicating from the GTID position "0-1-1", corresponding to the
+ *second* event group in the binlog. Because there will be no
+ slightly earlier point to start reading from!
+
+ So we put a slightly awkward special case here to handle that: If
+ at the start of the first file we have a singleton gtid state
+ with seq_no=1, D-S-1, then it must be the very first GTID in the
+ entire binlog, so we write an *empty* gtid state that will always
+ allow to start replicating from the very start of the binlog.
+
+ (If the user would explicitly set the seq_no of the very first
+ GTID in the binlog greater than 1, then starting from that GTID
+ position will still not be possible).
+ */
+ rpl_gtid singleton_gtid;
+ binlog_full_state.get_gtid_list_nolock(&singleton_gtid, 1);
+ if (singleton_gtid.seq_no == 1)
+ binlog_state= &binlog_diff_state; // Conveniently empty
+ }
+ err= binlog_gtid_state(binlog_state, mtr, block, page_no,
+ page_offset, file_no);
+ ut_a(!err);
+ ut_ad(block);
+ } else {
+ bool err= binlog_gtid_state(&binlog_diff_state, mtr, block, page_no,
+ page_offset, file_no);
+ ut_a(!err);
+ }
+ } else
+ block= binlog_page_fifo->create_page(file_no, page_no);
+ } else {
+ block= binlog_page_fifo->get_page(file_no, page_no);
+ }
+
+ ut_ad(page_offset < page_end);
+ uint32_t page_remain= page_end - page_offset;
+ byte *ptr= page_offset + &block->page_buf()[0];
+ if (page_remain < 4) {
+ /* Pad the remaining few bytes, and move to next page. */
+ if (UNIV_LIKELY(page_remain > 0))
+ {
+ memset(ptr, FSP_BINLOG_TYPE_FILLER, page_remain);
+ fsp_log_binlog_write(mtr, block, file_no, page_no, page_offset,
+ page_remain);
+ }
+ binlog_page_fifo->release_page_mtr(block, mtr);
+ block= nullptr;
+ ++page_no;
+ page_offset= BINLOG_PAGE_DATA;
+ DBUG_EXECUTE_IF("pause_binlog_write_after_release_page",
+ my_sleep(200000););
+ continue;
+ }
+
+ if (UNIV_UNLIKELY(chunk_type == FSP_BINLOG_TYPE_FILLER))
+ {
+ /*
+ Used for FLUSH BINARY LOGS, to move to the next tablespace and write
+ the initial GTID state record without writing any actual event data.
+ */
+ break;
+ }
+
+ if (start_offset == 0)
+ {
+ start_file_no= file_no;
+ start_offset= (page_no << page_size_shift) + page_offset;
+ }
+ page_remain-= 3; /* Type byte and 2-byte length. */
+ std::pair size_last=
+ chunk_data->copy_data(ptr+3, page_remain);
+ uint32_t size= size_last.first;
+ ut_ad(size_last.second || size == page_remain);
+ ut_ad(size <= page_remain);
+ page_remain-= size;
+ byte last_flag= size_last.second ? FSP_BINLOG_FLAG_LAST : 0;
+ ptr[0]= chunk_type | cont_flag | last_flag;
+ ptr[1]= size & 0xff;
+ ptr[2]= (byte)(size >> 8);
+ ut_ad(size <= 0xffff);
+
+ fsp_log_binlog_write(mtr, block, file_no, page_no, page_offset, size + 3);
+ cont_flag= FSP_BINLOG_FLAG_CONT;
+ if (page_remain == 0) {
+ binlog_page_fifo->release_page_mtr(block, mtr);
+ block= nullptr;
+ page_offset= BINLOG_PAGE_DATA;
+ ++page_no;
+ DBUG_EXECUTE_IF("pause_binlog_write_after_release_page",
+ if (!size_last.second)
+ my_sleep(200000);
+ );
+ } else {
+ page_offset+= size+3;
+ }
+ if (size_last.second)
+ break;
+ ut_ad(!block);
+ if (UNIV_UNLIKELY(block != nullptr))
+ {
+ /*
+ Defensive coding, just to not leave a page latch which would hang the
+ entire server hard. This code should not be reachable.
+ */
+ binlog_page_fifo->release_page_mtr(block, mtr);
+ block= nullptr;
+ }
+ }
+ if (block)
+ binlog_page_fifo->release_page_mtr(block, mtr);
+ binlog_cur_page_no= page_no;
+ binlog_cur_page_offset= page_offset;
+ binlog_cur_end_offset[file_no & 3].store
+ (((uint64_t)page_no << page_size_shift) + page_offset,
+ std::memory_order_relaxed);
+ if (UNIV_UNLIKELY(pending_prev_end_offset != 0))
+ {
+ mysql_mutex_lock(&binlog_durable_mutex);
+ mysql_mutex_lock(&active_binlog_mutex);
+ binlog_cur_end_offset[(file_no-1) & 3].store(pending_prev_end_offset,
+ std::memory_order_relaxed);
+ active_binlog_file_no.store(file_no, std::memory_order_release);
+ pthread_cond_signal(&active_binlog_cond);
+ mysql_mutex_unlock(&active_binlog_mutex);
+ mysql_mutex_unlock(&binlog_durable_mutex);
+ }
+ return {start_file_no, start_offset};
+}
+
+
+/**
+ Implementation of FLUSH BINARY LOGS.
+ Truncate the current binlog tablespace, fill up the last page with dummy data
+ (if needed), write the current GTID state to the first page in the next
+ tablespace file (for DELETE_DOMAIN_ID).
+
+ Relies on the server layer to prevent other binlog writes in parallel during
+ the operation.
+*/
+bool
+fsp_binlog_flush()
+{
+ uint64_t file_no= active_binlog_file_no.load(std::memory_order_relaxed);
+ uint32_t page_no= binlog_cur_page_no;
+ chunk_data_flush dummy_data;
+ mtr_t mtr{nullptr};
+
+ mysql_mutex_lock(&purge_binlog_mutex);
+
+ binlog_page_fifo->lock_wait_for_idle();
+ File fh= binlog_page_fifo->get_fh(file_no);
+ if (fh == (File)-1)
+ {
+ binlog_page_fifo->unlock();
+ mysql_mutex_unlock(&purge_binlog_mutex);
+ return true;
+ }
+
+ if (my_chsize(fh, ((uint64_t)page_no + 1) << ibb_page_size_shift, 0,
+ MYF(MY_WME)))
+ {
+ binlog_page_fifo->unlock();
+ mysql_mutex_unlock(&purge_binlog_mutex);
+ return true;
+ }
+ /*
+ Sync the truncate to disk. This way, if we crash after this we are sure the
+ truncate has been effected so we do not put the filler record in what is
+ then the middle of the file. If we crash before the truncate is durable, we
+ just come up as if the flush has never happened. If we crash with the
+ truncate durable but without the filler record, that is not a problem, the
+ binlog file will just be shorter.
+ */
+ my_sync(fh, MYF(0));
+ binlog_page_fifo->unlock();
+
+ LF_PINS *lf_pins= lf_hash_get_pins(&ibb_file_hash.hash);
+ ut_a(lf_pins);
+ uint32_t page_offset= binlog_cur_page_offset;
+ if (page_offset > BINLOG_PAGE_DATA ||
+ page_offset < ibb_page_size - BINLOG_PAGE_DATA_END)
+ {
+ /*
+ If we are not precisely the end of a page, fill up that page with a dummy
+ record. Otherwise the zeros at the end of the page would be detected as
+ end-of-file of the entire binlog.
+ */
+ mtr.start();
+ fsp_binlog_write_rec(&dummy_data, &mtr, FSP_BINLOG_TYPE_DUMMY, lf_pins);
+ mtr.commit();
+ }
+
+ if (page_no + 1 < binlog_page_fifo->size_in_pages(file_no))
+ {
+ binlog_page_fifo->truncate_file_size(file_no, page_no + 1);
+ size_t reclaimed= (binlog_page_fifo->size_in_pages(file_no) - (page_no + 1))
+ << ibb_page_size_shift;
+ if (UNIV_LIKELY(total_binlog_used_size >= reclaimed))
+ total_binlog_used_size-= reclaimed;
+ else
+ ut_ad(0);
+ }
+
+ /* Flush out all pages in the (now filled-up) tablespace. */
+ binlog_page_fifo->flush_up_to(file_no, page_no);
+
+ /*
+ Load the binlog GTID state from the server layer (in case it changed
+ due to FLUSH BINARY LOGS DELETE_DOMAIN_ID).
+ */
+ load_global_binlog_state(&binlog_full_state);
+
+ mysql_mutex_unlock(&purge_binlog_mutex);
+
+ /*
+ Now get a new GTID state record written to the next binlog tablespace.
+ This ensures that the new state (in case of DELETE_DOMAIN_ID) will be
+ persisted across a server restart.
+ */
+ mtr.start();
+ fsp_binlog_write_rec(&dummy_data, &mtr, FSP_BINLOG_TYPE_FILLER, lf_pins);
+ mtr.commit();
+ lf_hash_put_pins(lf_pins);
+ log_buffer_flush_to_disk(srv_flush_log_at_trx_commit & 1);
+ ibb_pending_lsn_fifo.add_to_fifo(mtr.commit_lsn(), file_no+1,
+ binlog_cur_end_offset[(file_no + 1) & 3].load(std::memory_order_relaxed));
+
+ return false;
+}
+
+
+binlog_chunk_reader::binlog_chunk_reader(std::atomic *limit_offset_)
+ : s { 0, ~(uint64_t)0, 0, 0, 0, 0, FSP_BINLOG_TYPE_FILLER, false, false },
+ stop_file_no(~(uint64_t)0), page_ptr(0), cur_block(0), page_buffer(nullptr),
+ limit_offset(limit_offset_), cur_file_handle((File)-1),
+ skipping_partial(false)
+{
+ /* Nothing else. */
+}
+
+
+binlog_chunk_reader::~binlog_chunk_reader()
+{
+ release();
+ if (cur_file_handle >= (File)0)
+ my_close(cur_file_handle, MYF(0));
+}
+
+
+int
+binlog_chunk_reader::read_error_corruption(uint64_t file_no, uint64_t page_no,
+ const char *msg)
+{
+ sql_print_error("InnoDB: Corrupt binlog found on page %llu in binlog number "
+ "%llu: %s", page_no, file_no, msg);
+ return -1;
+}
+
+
+/**
+ Obtain the data on the page currently pointed to by the chunk reader. The
+ page is either latched in the page fifo, or read from the file into the page
+ buffer.
+
+ The code does a dirty read of active_binlog_file_no to determine if the page
+ is known to be available to read from the file, or if it should be looked up
+ in the buffer pool. After making the decision, another dirty read is done to
+ protect against the race where the active tablespace changes in the middle,
+ and if so the operation is re-tried. This is necessary since the binlog files
+ N and N-2 use the same tablespace id, so we must ensure we do not mistake a
+ page from N as belonging to N-2.
+*/
+enum binlog_chunk_reader::chunk_reader_status
+binlog_chunk_reader::fetch_current_page()
+{
+ ut_ad(!cur_block /* Must have no active page latch */);
+ uint64_t active2= active_binlog_file_no.load(std::memory_order_acquire);
+ for (;;) {
+ fsp_binlog_page_entry *block= nullptr;
+ uint64_t offset= (s.page_no << ibb_page_size_shift) | s.in_page_offset;
+ uint64_t active= active2;
+ uint64_t end_offset=
+ limit_offset[s.file_no & 3].load(std::memory_order_acquire);
+ /*
+ Can be different from end_offset if limit_offset is the
+ binlog_cur_durable_offset.
+ */
+ uint64_t real_end_offset=
+ binlog_cur_end_offset[s.file_no & 3].load(std::memory_order_acquire);
+ if (s.file_no > active || UNIV_UNLIKELY(active == ~(uint64_t)0)
+ || UNIV_UNLIKELY(s.file_no > stop_file_no))
+ {
+ ut_ad(s.page_no == 1 || s.file_no > stop_file_no);
+ ut_ad(s.in_page_offset == 0 || s.file_no > stop_file_no);
+ /*
+ Allow a reader that reached the very end of the active binlog file to
+ have moved ahead early to the start of the coming binlog file.
+ */
+ return CHUNK_READER_EOF;
+ }
+
+ if (s.file_no + 1 >= active) {
+ /* Check if we should read from the buffer pool or from the file. */
+ if (end_offset != ~(uint64_t)0 && offset < end_offset)
+ block= binlog_page_fifo->get_page(s.file_no, s.page_no);
+ active2= active_binlog_file_no.load(std::memory_order_acquire);
+ if (UNIV_UNLIKELY(active2 != active)) {
+ /*
+ The active binlog file changed while we were processing; we might
+ have gotten invalid end_offset or a buffer pool page from a wrong
+ tablespace. So just try again.
+ */
+ if (block)
+ binlog_page_fifo->release_page(block);
+ continue;
+ }
+ cur_end_offset= end_offset;
+ if (offset >= end_offset) {
+ ut_ad(!block);
+ if (s.file_no == active) {
+ /* Reached end of the currently active binlog file -> EOF. */
+ return CHUNK_READER_EOF;
+ }
+ ut_ad(s.file_no + 1 == active);
+ if (offset < real_end_offset)
+ {
+ /*
+ Reached durable limit of active-1 _and_ not at the end of the
+ file where we should move to the next one.
+ */
+ return CHUNK_READER_EOF;
+ }
+ }
+ if (block) {
+ cur_block= block;
+ page_ptr= block->page_buf();
+ return CHUNK_READER_FOUND;
+ } else {
+ /* Not in buffer pool, just read it from the file. */
+ /* Fall through to read from file. */
+ }
+ }
+
+ /* Tablespace is not open, just read from the file. */
+ if (cur_file_handle < (File)0)
+ {
+ char filename[OS_FILE_MAX_PATH];
+ MY_STAT stat_buf;
+
+ binlog_name_make(filename, s.file_no);
+ cur_file_handle= my_open(filename, O_RDONLY | O_BINARY, MYF(MY_WME));
+ if (UNIV_UNLIKELY(cur_file_handle < (File)0)) {
+ cur_file_handle= (File)-1;
+ cur_file_length= ~(uint64_t)0;
+ return CHUNK_READER_ERROR;
+ }
+ if (my_fstat(cur_file_handle, &stat_buf, MYF(0))) {
+ my_error(ER_CANT_GET_STAT, MYF(0), filename, errno);
+ my_close(cur_file_handle, MYF(0));
+ cur_file_handle= (File)-1;
+ cur_file_length= ~(uint64_t)0;
+ return CHUNK_READER_ERROR;
+ }
+ cur_file_length= stat_buf.st_size;
+ }
+ if (s.file_no + 1 >= active)
+ cur_end_offset= end_offset;
+ else
+ cur_end_offset= cur_file_length;
+
+ if (offset >= cur_file_length) {
+ /* End of this file, move to the next one. */
+ goto_next_file:
+ if (UNIV_UNLIKELY(s.file_no >= stop_file_no))
+ return CHUNK_READER_EOF;
+ if (cur_file_handle >= (File)0)
+ {
+ my_close(cur_file_handle, MYF(0));
+ cur_file_handle= (File)-1;
+ cur_file_length= ~(uint64_t)0;
+ }
+ ++s.file_no;
+ s.page_no= 1; /* Skip the header page. */
+ continue;
+ }
+
+ int res= crc32_pread_page(cur_file_handle, page_buffer, s.page_no,
+ MYF(MY_WME));
+ if (res < 0)
+ return CHUNK_READER_ERROR;
+ if (res == 0)
+ goto goto_next_file;
+ page_ptr= page_buffer;
+ return CHUNK_READER_FOUND;
+ }
+ /* NOTREACHED */
+}
+
+
+int
+binlog_chunk_reader::read_data(byte *buffer, int max_len, bool multipage)
+{
+ uint32_t size;
+ int sofar= 0;
+
+read_more_data:
+ if (max_len == 0)
+ return sofar;
+
+ if (!page_ptr)
+ {
+ enum chunk_reader_status res= fetch_current_page();
+ if (res == CHUNK_READER_EOF)
+ {
+ if (s.in_record && s.file_no <= stop_file_no)
+ return read_error_corruption(s.file_no, s.page_no, "binlog tablespace "
+ "truncated in the middle of record");
+ else
+ return 0;
+ }
+ else if (res == CHUNK_READER_ERROR)
+ return -1;
+ }
+
+ if (s.chunk_len == 0)
+ {
+ byte type;
+ /*
+ This code gives warning "comparison of unsigned expression in ‘< 0’ is
+ always false" when BINLOG_PAGE_DATA is 0.
+
+ So use a static assert for now; if it ever triggers, replace it with this
+ code:
+
+ if (s.in_page_offset < BINLOG_PAGE_DATA)
+ s.in_page_offset= BINLOG_PAGE_DATA;
+ */
+ if (0)
+ static_assert(BINLOG_PAGE_DATA == 0,
+ "Replace static_assert with code from above comment");
+
+ /* Check for end-of-file. */
+ if ((s.page_no << ibb_page_size_shift) + s.in_page_offset >= cur_end_offset)
+ return sofar;
+
+ if (s.in_page_offset >= ibb_page_size - (BINLOG_PAGE_DATA_END + 3) ||
+ page_ptr[s.in_page_offset] == FSP_BINLOG_TYPE_FILLER)
+ {
+ ut_ad(s.in_page_offset >= ibb_page_size - BINLOG_PAGE_DATA_END ||
+ page_ptr[s.in_page_offset] == FSP_BINLOG_TYPE_FILLER);
+ goto go_next_page;
+ }
+
+ type= page_ptr[s.in_page_offset];
+ if (type == 0)
+ {
+ ut_ad(0 /* Should have detected end-of-file on cur_end_offset. */);
+ return 0;
+ }
+
+ /*
+ Consistency check on the chunks. A record must consist in a sequence of
+ chunks of the same type, all but the first must have the
+ FSP_BINLOG_FLAG_BIT_CONT bit set, and the final one must have the
+ FSP_BINLOG_FLAG_BIT_LAST bit set.
+ */
+ if (!s.in_record)
+ {
+ if (UNIV_UNLIKELY(type & FSP_BINLOG_FLAG_CONT) && !s.skip_current)
+ {
+ if (skipping_partial)
+ {
+ s.chunk_len= page_ptr[s.in_page_offset + 1] |
+ ((uint32_t)page_ptr[s.in_page_offset + 2] << 8);
+ s.skip_current= true;
+ goto skip_chunk;
+ }
+ else
+ return read_error_corruption(s.file_no, s.page_no, "Binlog record "
+ "starts with continuation chunk");
+ }
+ }
+ else
+ {
+ if ((type ^ s.chunk_type) & FSP_BINLOG_TYPE_MASK)
+ {
+ /*
+ As a special case, we must allow a GTID state to appear in the
+ middle of a record.
+ */
+ if (((uint64_t)1 << (type & FSP_BINLOG_TYPE_MASK)) &
+ ALLOWED_NESTED_RECORDS)
+ {
+ s.chunk_len= page_ptr[s.in_page_offset + 1] |
+ ((uint32_t)page_ptr[s.in_page_offset + 2] << 8);
+ goto skip_chunk;
+ }
+ /* Chunk type changed in the middle. */
+ return read_error_corruption(s.file_no, s.page_no, "Binlog record missing "
+ "end chunk");
+ }
+ if (!(type & FSP_BINLOG_FLAG_CONT))
+ {
+ /* START chunk without END chunk. */
+ return read_error_corruption(s.file_no, s.page_no, "Binlog record missing "
+ "end chunk");
+ }
+ }
+
+ s.skip_current= false;
+ s.chunk_type= type;
+ s.in_record= true;
+ s.rec_start_file_no= s.file_no;
+ s.chunk_len= page_ptr[s.in_page_offset + 1] |
+ ((uint32_t)page_ptr[s.in_page_offset + 2] << 8);
+ s.chunk_read_offset= 0;
+ }
+
+ /* Now we have a chunk available to read data from. */
+ ut_ad(s.chunk_read_offset < s.chunk_len);
+ if (s.skip_current &&
+ (s.chunk_read_offset > 0 || (s.chunk_type & FSP_BINLOG_FLAG_CONT)))
+ {
+ /*
+ Skip initial continuation chunks.
+ Used to be able to start reading potentially in the middle of a record,
+ ie. at a GTID state point.
+ */
+ s.chunk_read_offset= s.chunk_len;
+ }
+ else
+ {
+ size= std::min((uint32_t)max_len, s.chunk_len - s.chunk_read_offset);
+ memcpy(buffer, page_ptr + s.in_page_offset + 3 + s.chunk_read_offset, size);
+ buffer+= size;
+ s.chunk_read_offset+= size;
+ max_len-= size;
+ sofar+= size;
+ }
+
+ if (s.chunk_len > s.chunk_read_offset)
+ {
+ ut_ad(max_len == 0 /* otherwise would have read more */);
+ return sofar;
+ }
+
+ /* We have read all of the chunk. Move to next chunk or end of the record. */
+skip_chunk:
+ s.in_page_offset+= 3 + s.chunk_len;
+ s.chunk_len= 0;
+ s.chunk_read_offset= 0;
+
+ if (s.chunk_type & FSP_BINLOG_FLAG_LAST)
+ {
+ s.in_record= false; /* End of record. */
+ s.skip_current= false;
+ }
+
+ if (s.in_page_offset >= ibb_page_size - (BINLOG_PAGE_DATA_END + 3) &&
+ (s.page_no << ibb_page_size_shift) + s.in_page_offset < cur_end_offset)
+ {
+go_next_page:
+ /* End of page reached, move to the next page. */
+ ++s.page_no;
+ page_ptr= nullptr;
+ if (cur_block)
+ {
+ binlog_page_fifo->release_page(cur_block);
+ cur_block= nullptr;
+ }
+ s.in_page_offset= 0;
+
+ if (cur_file_handle >= (File)0 &&
+ (s.page_no << ibb_page_size_shift) >= cur_file_length)
+ {
+ /* Move to the next file. */
+ my_close(cur_file_handle, MYF(0));
+ cur_file_handle= (File)-1;
+ cur_file_length= ~(uint64_t)0;
+ ++s.file_no;
+ s.page_no= 1; /* Skip the header page. */
+ }
+ }
+
+ if (sofar > 0 && (!multipage || !s.in_record))
+ return sofar;
+
+ goto read_more_data;
+}
+
+
+int
+binlog_chunk_reader::find_offset_in_page(uint32_t off)
+{
+ if (!page_ptr)
+ {
+ enum chunk_reader_status res= fetch_current_page();
+ if (res == CHUNK_READER_EOF)
+ return 0;
+ else if (res == CHUNK_READER_ERROR)
+ return -1;
+ }
+
+ /*
+ Skip ahead in the page until we come to the first chunk boundary that
+ is at or later than the requested offset.
+ */
+ s.in_page_offset= 0;
+ s.chunk_len= 0;
+ s.chunk_read_offset= 0;
+ s.chunk_type= FSP_BINLOG_TYPE_FILLER;
+ s.skip_current= 0;
+ s.in_record= 0;
+ while (s.in_page_offset < off &&
+ s.in_page_offset < cur_end_offset &&
+ s.in_page_offset < ibb_page_size)
+ {
+ byte type= page_ptr[s.in_page_offset];
+ if (type == 0 || type == FSP_BINLOG_TYPE_FILLER)
+ break;
+ uint32_t chunk_len= page_ptr[s.in_page_offset + 1] |
+ ((uint32_t)page_ptr[s.in_page_offset + 2] << 8);
+ s.in_page_offset+= std::min(3 + chunk_len, (uint32_t)ibb_page_size);
+ }
+ return 0;
+}
+
+
+/**
+ Read the header page of the current binlog file_no.
+ Returns:
+ 1 Header page found and returned.
+ 0 EOF, no header page found (ie. file is empty / nothing is durable yet).
+ -1 Error.
+*/
+int
+binlog_chunk_reader::get_file_header(binlog_header_data *out_header)
+{
+ seek(current_file_no(), 0);
+ enum chunk_reader_status res= fetch_current_page();
+ if (UNIV_UNLIKELY(res != CHUNK_READER_FOUND))
+ return res == CHUNK_READER_EOF ? 0 : -1;
+ fsp_binlog_extract_header_page(page_ptr, out_header);
+ if (out_header->is_invalid || out_header->is_empty)
+ return -1;
+ return 1;
+}
+
+
+void
+binlog_chunk_reader::restore_pos(binlog_chunk_reader::saved_position *pos)
+{
+ if (page_ptr &&
+ !(pos->file_no == s.file_no && pos->page_no == s.page_no))
+ {
+ /* Seek to a different page, release any current page. */
+ if (cur_block)
+ {
+ binlog_page_fifo->release_page(cur_block);
+ cur_block= nullptr;
+ }
+ page_ptr= nullptr;
+ }
+ if (cur_file_handle != (File)-1 && pos->file_no != s.file_no)
+ {
+ /* Seek to a different file than currently open, close it. */
+ my_close(cur_file_handle, MYF(0));
+ cur_file_handle= (File)-1;
+ cur_file_length= ~(uint64_t)0;
+ }
+ s= *pos;
+}
+
+
+void
+binlog_chunk_reader::seek(uint64_t file_no, uint64_t offset)
+{
+ saved_position pos {
+ file_no, ~(uint64_t)0, (uint32_t)(offset >> ibb_page_size_shift),
+ (uint32_t)(offset & (ibb_page_size - 1)),
+ 0, 0, FSP_BINLOG_TYPE_FILLER, false, false };
+ restore_pos(&pos);
+}
+
+
+void binlog_chunk_reader::release(bool release_file_page)
+{
+ if (cur_block)
+ {
+ binlog_page_fifo->release_page(cur_block);
+ cur_block= nullptr;
+ page_ptr= nullptr;
+ }
+ else if (release_file_page)
+ {
+ /*
+ For when we reach EOF while reading from the file. We need to re-read
+ the page from the file in this case on next read, as data might be added
+ to the page.
+ */
+ page_ptr= nullptr;
+ }
+}
+
+
+bool binlog_chunk_reader::data_available()
+{
+ if (!end_of_record())
+ return true;
+ uint64_t active= active_binlog_file_no.load(std::memory_order_acquire);
+ if (UNIV_UNLIKELY(active == ~(uint64_t)0))
+ return false;
+ uint64_t end_offset;
+ for (;;)
+ {
+ if (active > s.file_no + 1)
+ return true;
+ end_offset= limit_offset[s.file_no & 3].load(std::memory_order_acquire);
+ uint64_t active2= active_binlog_file_no.load(std::memory_order_acquire);
+ if (active2 == active)
+ break;
+ /* Active moved while we were checking, try again. */
+ active= active2;
+ }
+ uint64_t offset= (s.page_no << ibb_page_size_shift) | s.in_page_offset;
+ if (offset < end_offset)
+ return true;
+
+ ut_ad(s.file_no + 1 == active || s.file_no == active);
+ ut_ad(offset == end_offset || (offset == ibb_page_size && end_offset == 0));
+ return false;
+}
+
+
+bool
+binlog_chunk_reader::is_before_pos(uint64_t file_no, uint64_t offset)
+{
+ if (s.file_no < file_no)
+ return true;
+ if (s.file_no > file_no)
+ return false;
+ uint64_t own_offset= (s.page_no << ibb_page_size_shift) | s.in_page_offset;
+ if (own_offset < offset)
+ return true;
+ return false;
+}
diff --git a/storage/innobase/handler/ha_innodb.cc b/storage/innobase/handler/ha_innodb.cc
index f5c0132a39f44..48c84429e39bb 100644
--- a/storage/innobase/handler/ha_innodb.cc
+++ b/storage/innobase/handler/ha_innodb.cc
@@ -30,6 +30,7 @@ this program; if not, write to the Free Software Foundation, Inc.,
#define MYSQL_SERVER
#include "univ.i"
+#include "my_bit.h"
/* Include necessary SQL headers */
#include "ha_prototypes.h"
@@ -108,6 +109,7 @@ extern my_bool opt_readonly;
#include "fil0pagecompress.h"
#include "ut0mem.h"
#include "row0ext.h"
+#include "innodb_binlog.h"
#include "lz4.h"
#include "lzo/lzo1x.h"
@@ -520,6 +522,12 @@ mysql_pfs_key_t lock_wait_mutex_key;
mysql_pfs_key_t trx_sys_mutex_key;
mysql_pfs_key_t srv_threads_mutex_key;
mysql_pfs_key_t tpool_cache_mutex_key;
+mysql_pfs_key_t fsp_active_binlog_mutex_key;
+mysql_pfs_key_t fsp_binlog_durable_mutex_key;
+mysql_pfs_key_t fsp_binlog_durable_cond_key;
+mysql_pfs_key_t fsp_purge_binlog_mutex_key;
+mysql_pfs_key_t fsp_page_fifo_mutex_key;
+mysql_pfs_key_t ibb_xid_hash_mutex_key;
/* all_innodb_mutexes array contains mutexes that are
performance schema instrumented if "UNIV_PFS_MUTEX"
@@ -2430,72 +2438,6 @@ innobase_raw_format(
return(ut_str_sql_format(buf_tmp, buf_tmp_used, buf, buf_size));
}
-/*
-The helper function nlz(x) calculates the number of leading zeros
-in the binary representation of the number "x", either using a
-built-in compiler function or a substitute trick based on the use
-of the multiplication operation and a table indexed by the prefix
-of the multiplication result:
-*/
-#ifdef __GNUC__
-#define nlz(x) __builtin_clzll(x)
-#elif defined(_MSC_VER) && !defined(_M_CEE_PURE) && \
- (defined(_M_IX86) || defined(_M_X64) || defined(_M_ARM64))
-#ifndef __INTRIN_H_
-#pragma warning(push, 4)
-#pragma warning(disable: 4255 4668)
-#include
-#pragma warning(pop)
-#endif
-__forceinline unsigned int nlz (ulonglong x)
-{
-#if defined(_M_IX86) || defined(_M_X64)
- unsigned long n;
-#ifdef _M_X64
- _BitScanReverse64(&n, x);
- return (unsigned int) n ^ 63;
-#else
- unsigned long y = (unsigned long) (x >> 32);
- unsigned int m = 31;
- if (y == 0)
- {
- y = (unsigned long) x;
- m = 63;
- }
- _BitScanReverse(&n, y);
- return (unsigned int) n ^ m;
-#endif
-#elif defined(_M_ARM64)
- return _CountLeadingZeros64(x);
-#endif
-}
-#else
-inline unsigned int nlz (ulonglong x)
-{
- static unsigned char table [48] = {
- 32, 6, 5, 0, 4, 12, 0, 20,
- 15, 3, 11, 0, 0, 18, 25, 31,
- 8, 14, 2, 0, 10, 0, 0, 0,
- 0, 0, 0, 21, 0, 0, 19, 26,
- 7, 0, 13, 0, 16, 1, 22, 27,
- 9, 0, 17, 23, 28, 24, 29, 30
- };
- unsigned int y= (unsigned int) (x >> 32);
- unsigned int n= 0;
- if (y == 0) {
- y= (unsigned int) x;
- n= 32;
- }
- y = y | (y >> 1); // Propagate leftmost 1-bit to the right.
- y = y | (y >> 2);
- y = y | (y >> 4);
- y = y | (y >> 8);
- y = y & ~(y >> 16);
- y = y * 0x3EF5D037;
- return n + table[y >> 26];
-}
-#endif
-
/*********************************************************************//**
Compute the next autoinc value.
@@ -2538,8 +2480,8 @@ innobase_next_autoinc(
operation. The snippet below calculates the product of two numbers
and detects an unsigned integer overflow:
*/
- unsigned int m= nlz(need);
- unsigned int n= nlz(step);
+ unsigned int m= my_nlz(need);
+ unsigned int n= my_nlz(step);
if (m + n <= 8 * sizeof(ulonglong) - 2) {
// The bit width of the original values is too large,
// therefore we are guaranteed to get an overflow.
@@ -2725,6 +2667,7 @@ trx_deregister_from_2pc(
{
trx->is_registered= false;
trx->active_commit_ordered= false;
+ trx->active_prepare= false;
}
/**
@@ -3760,6 +3703,16 @@ static int innodb_init_params()
DBUG_RETURN(HA_ERR_INITIALIZATION);
}
+ if (innodb_binlog_state_interval == 0 ||
+ innodb_binlog_state_interval !=
+ (ulonglong)1 << (63 - my_nlz(innodb_binlog_state_interval)) ||
+ innodb_binlog_state_interval % (ulonglong)ibb_page_size) {
+ sql_print_error("InnoDB: innodb_binlog_state_interval must be "
+ "a power-of-two multiple of the innodb binlog "
+ "page size=%lu KiB", ibb_page_size >> 10);
+ DBUG_RETURN(HA_ERR_INITIALIZATION);
+ }
+
#ifdef _WIN32
if (!is_filename_allowed(srv_buf_dump_filename,
strlen(srv_buf_dump_filename), false))
@@ -4008,6 +3961,41 @@ static void innobase_update_optimizer_costs(OPTIMIZER_COSTS *costs)
}
+static binlog_file_entry *innodb_get_binlog_file_list(MEM_ROOT *mem_root)
+{
+ uint64_t first, last;
+ if (innodb_find_binlogs(&first, &last))
+ return nullptr;
+ binlog_file_entry *list;
+ binlog_file_entry **next_ptr= &list;
+ for (uint64_t i= first; i <= last; ++i)
+ {
+ binlog_file_entry *e= (binlog_file_entry *)alloc_root(mem_root, sizeof(*e));
+ if (!e)
+ return nullptr;
+ char name_buf[OS_FILE_MAX_PATH];
+ binlog_name_make(name_buf, i);
+ e->name.length= strlen(name_buf);
+ char *str= static_cast(alloc_root(mem_root, e->name.length + 1));
+ if (!str)
+ return nullptr;
+ strcpy(str, name_buf);
+ e->name.str= str;
+ *next_ptr= e;
+ next_ptr= &(e->next);
+ }
+ *next_ptr= nullptr;
+ return list;
+}
+
+
+static bool
+innodb_binlog_flush()
+{
+ return fsp_binlog_flush();
+}
+
+
/** Initialize the InnoDB storage engine plugin.
@param[in,out] p InnoDB handlerton
@return error code
@@ -4076,6 +4064,32 @@ static int innodb_init(void* p)
= innodb_prepare_commit_versioned;
innobase_hton->update_optimizer_costs= innobase_update_optimizer_costs;
+ innobase_hton->binlog_init= innodb_binlog_init;
+ innobase_hton->set_binlog_max_size= ibb_set_max_size;
+ innobase_hton->binlog_write_direct_ordered=
+ innobase_binlog_write_direct_ordered;
+ innobase_hton->binlog_write_direct= innobase_binlog_write_direct;
+ innobase_hton->binlog_group_commit_ordered= ibb_group_commit;
+ innobase_hton->binlog_oob_data_ordered= innodb_binlog_oob_ordered;
+ innobase_hton->binlog_oob_data= innodb_binlog_oob;
+ innobase_hton->binlog_savepoint_rollback= ibb_savepoint_rollback;
+ innobase_hton->binlog_oob_reset= innodb_reset_oob;
+ innobase_hton->binlog_oob_free= innodb_free_oob;
+ innobase_hton->binlog_write_xa_prepare_ordered=
+ ibb_write_xa_prepare_ordered;
+ innobase_hton->binlog_write_xa_prepare= ibb_write_xa_prepare;
+ innobase_hton->binlog_xa_rollback_ordered=
+ ibb_xa_rollback_ordered;
+ innobase_hton->binlog_xa_rollback= ibb_xa_rollback;
+ innobase_hton->binlog_unlog= ibb_binlog_unlog;
+ innobase_hton->get_binlog_reader= innodb_get_binlog_reader;
+ innobase_hton->get_binlog_file_list= innodb_get_binlog_file_list;
+ innobase_hton->get_filename= ibb_get_filename;
+ innobase_hton->binlog_status= innodb_binlog_status;
+ innobase_hton->binlog_flush= innodb_binlog_flush;
+ innobase_hton->binlog_get_init_state= innodb_binlog_get_init_state;
+ innobase_hton->reset_binlogs= innodb_reset_binlogs;
+ innobase_hton->binlog_purge= innodb_binlog_purge;
innodb_remember_check_sysvar_funcs();
@@ -4377,8 +4391,8 @@ innobase_commit_ordered(
DBUG_ASSERT(all ||
(!thd_test_options(thd, OPTION_NOT_AUTOCOMMIT | OPTION_BEGIN)));
- innobase_commit_ordered_2(trx, thd);
trx->active_commit_ordered = true;
+ innobase_commit_ordered_2(trx, thd);
DBUG_VOID_RETURN;
}
@@ -15884,7 +15898,8 @@ ha_innobase::extra(
alter_stats_rebuild(m_prebuilt->table, trx);
break;
case HA_EXTRA_ABORT_ALTER_COPY:
- if (m_prebuilt->table->skip_alter_undo) {
+ if (m_prebuilt->table->skip_alter_undo &&
+ !m_prebuilt->table->is_temporary()) {
trx = check_trx_exists(ha_thd());
m_prebuilt->table->skip_alter_undo = 0;
trx->rollback();
@@ -17165,7 +17180,10 @@ innobase_xa_prepare(
case TRX_STATE_ACTIVE:
trx->xid= *thd->get_xid();
if (prepare_trx)
+ {
trx_prepare_for_mysql(trx);
+ trx->active_prepare= true;
+ }
else
{
lock_unlock_table_autoinc(trx);
@@ -19814,6 +19832,15 @@ static MYSQL_SYSVAR_BOOL(truncate_temporary_tablespace_now,
"Shrink the temporary tablespace",
NULL, innodb_trunc_temp_space_update, false);
+static MYSQL_SYSVAR_ULONGLONG(binlog_state_interval,
+ innodb_binlog_state_interval,
+ PLUGIN_VAR_RQCMDARG | PLUGIN_VAR_READONLY,
+ "Interval (in bytes) at which to write the GTID binlog state to binlog "
+ "files to speed up GTID lookups. Must be a power-of-two multiple of the "
+ "binlog page size (16384 bytes)",
+ NULL, NULL, 2*1024*1024,
+ 32768, ULONGLONG_MAX, 0);
+
static struct st_mysql_sys_var* innobase_system_variables[]= {
MYSQL_SYSVAR(autoextend_increment),
MYSQL_SYSVAR(buffer_pool_size),
@@ -19989,6 +20016,7 @@ static struct st_mysql_sys_var* innobase_system_variables[]= {
MYSQL_SYSVAR(background_thread),
MYSQL_SYSVAR(encrypt_temporary_tables),
MYSQL_SYSVAR(truncate_temporary_tablespace_now),
+ MYSQL_SYSVAR(binlog_state_interval),
NULL
};
diff --git a/storage/innobase/handler/innodb_binlog.cc b/storage/innobase/handler/innodb_binlog.cc
new file mode 100644
index 0000000000000..bf7f425e0c578
--- /dev/null
+++ b/storage/innobase/handler/innodb_binlog.cc
@@ -0,0 +1,5095 @@
+/*****************************************************************************
+
+Copyright (c) 2024, Kristian Nielsen
+
+This program is free software; you can redistribute it and/or modify it under
+the terms of the GNU General Public License as published by the Free Software
+Foundation; version 2 of the License.
+
+This program is distributed in the hope that it will be useful, but WITHOUT
+ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or FITNESS
+FOR A PARTICULAR PURPOSE. See the GNU General Public License for more details.
+
+You should have received a copy of the GNU General Public License along with
+this program; if not, write to the Free Software Foundation, Inc.,
+51 Franklin Street, Fifth Floor, Boston, MA 02110-1335 USA
+
+*****************************************************************************/
+
+/**************************************************//**
+@file handler/innodb_binlog.cc
+InnoDB implementation of binlog.
+*******************************************************/
+
+/*
+ Need MYSQL_SERVER defined to be able to use THD_ENTER_COND from sql_class.h
+ to make my_cond_wait() killable.
+*/
+#define MYSQL_SERVER 1
+#include
+#include "sql_class.h"
+
+#include "innodb_binlog.h"
+#include "mtr0log.h"
+#include "fsp0fsp.h"
+#include "trx0trx.h"
+#include "log0log.h"
+#include "small_vector.h"
+
+#include "mysys_err.h"
+#include "my_compr_int.h"
+#include "rpl_gtid_base.h"
+#include "handler_binlog_reader.h"
+#include "log.h"
+
+
+class ibb_xid_hash;
+
+
+static int innodb_binlog_inited= 0;
+
+pending_lsn_fifo ibb_pending_lsn_fifo;
+uint32_t innodb_binlog_size_in_pages;
+const char *innodb_binlog_directory;
+
+/** Current write position in active binlog file. */
+uint32_t binlog_cur_page_no;
+uint32_t binlog_cur_page_offset;
+
+/**
+ Server setting for how often to dump a (differential) binlog state at the
+ start of the page, to speed up finding the initial GTID position, read-only.
+*/
+ulonglong innodb_binlog_state_interval;
+
+/** Binlog state of the engine binlog. */
+rpl_binlog_state_base binlog_full_state;
+/**
+ Differential binlog state in the currently active binlog tablespace, relative
+ to the state at the start.
+*/
+rpl_binlog_state_base binlog_diff_state;
+
+static std::thread binlog_prealloc_thr_obj;
+static bool prealloc_thread_end= false;
+
+/**
+ Mutex around purge operations, including earliest_binlog_file_no and
+ total_binlog_used_size.
+*/
+mysql_mutex_t purge_binlog_mutex;
+
+/** The earliest binlog tablespace file. Used in binlog purge. */
+static uint64_t earliest_binlog_file_no;
+
+/**
+ The total space in use by binlog tablespace files. Maintained in-memory to
+ not have to stat(2) every file for every new binlog tablespace allocated in
+ case of --max-binlog-total-size.
+
+ Initialized at server startup (and in RESET MASTER), and updated as binlog
+ files are pre-allocated and purged.
+*/
+size_t total_binlog_used_size;
+
+static bool purge_warning_given= false;
+
+/** References to pending XA PREPARED transactions in the binlog. */
+ibb_xid_hash *ibb_xa_xid_hash;
+
+#ifdef UNIV_PFS_THREAD
+mysql_pfs_key_t binlog_prealloc_thread_key;
+#endif
+
+
+/**
+ Structure holding context for out-of-band chunks of binlogged event group.
+*/
+struct binlog_oob_context {
+ struct savepoint;
+ /*
+ Structure used to encapsulate the data to be binlogged in an out-of-band
+ chunk, for use by fsp_binlog_write_rec().
+ */
+ struct chunk_data_oob : public chunk_data_base {
+ /*
+ Need room for 5 numbers:
+ node index
+ left child file_no
+ left child offset
+ right child file_no
+ right child offset
+ */
+ static constexpr uint32_t max_buffer= 5*COMPR_INT_MAX64;
+ uint64_t sofar;
+ uint64_t main_len;
+ const byte *main_data;
+ uint32_t header_len;
+ byte header_buf[max_buffer];
+
+ chunk_data_oob(uint64_t idx,
+ uint64_t left_file_no, uint64_t left_offset,
+ uint64_t right_file_no, uint64_t right_offset,
+ const byte *data, size_t data_len);
+ virtual ~chunk_data_oob() {};
+ virtual std::pair copy_data(byte *p, uint32_t max_len) final;
+ };
+
+ bool binlog_node(uint32_t node, uint64_t new_idx,
+ uint32_t left_node, uint32_t right_node,
+ chunk_data_oob *oob_data, LF_PINS *pins, mtr_t *mtr);
+ bool create_stmt_start_point();
+ savepoint *create_savepoint();
+ void rollback_to_savepoint(savepoint *savepoint);
+ void rollback_to_stmt_start();
+
+ /*
+ Pending binlog write for the ibb_pending_lsn_fifo.
+ pending_file_no is ~0 when no write is pending.
+ */
+ uint64_t pending_file_no;
+ uint64_t pending_offset;
+ lsn_t pending_lsn;
+
+ uint64_t first_node_file_no;
+ uint64_t first_node_offset;
+ LF_PINS *lf_pins;
+ savepoint *stmt_start_point;
+ savepoint *savepoint_stack;
+ /*
+ The secondary pointer is for when server layer binlogs both a
+ non-transactional and a transactional oob stream.
+ */
+ binlog_oob_context *secondary_ctx;
+ uint32_t node_list_len;
+ uint32_t node_list_alloc_len;
+ /*
+ Set if we incremented refcount in first_node_file_no, so we need to
+ decrement again at commit record write or reset/rollback.
+ */
+ bool pending_refcount;
+ /* Set when the transaction is sealed after writing an XA PREPARE record. */
+ bool is_xa_prepared;
+ /*
+ The node_list contains the root of each tree in the forest of perfect
+ binary trees.
+ */
+#ifdef _MSC_VER
+/* Flexible array member is not standard C++, disable compiler warning. */
+#pragma warning(disable : 4200)
+#endif
+ struct node_info {
+ uint64_t file_no;
+ uint64_t offset;
+ uint64_t node_index;
+ uint32_t height;
+ } node_list [];
+
+ /* Saved oob state for implementing ROLLBACK TO SAVEPOINT. */
+ struct savepoint {
+ /* Maintain a stack of pending savepoints. */
+ savepoint *next;
+ uint32_t node_list_len;
+ uint32_t alloc_len;
+ struct node_info node_list[];
+ };
+};
+
+
+/**
+ A class for doing the post-order traversal of the forest of perfect binary
+ trees that make up the out-of-band data for a commit record.
+*/
+class innodb_binlog_oob_reader {
+ enum oob_states {
+ /* The initial state, about to visit the node for the first time. */
+ ST_initial,
+ /* State of leaf node while traversing the prior trees in the forest. */
+ ST_traversing_prior_trees,
+ /* State of non-leaf node while traversing its left sub-tree. */
+ ST_traversing_left_child,
+ /* State of non-leaf node while traversing its right sub-tree. */
+ ST_traversing_right_child,
+ /* State of node while reading out its data. */
+ ST_self
+ };
+
+ /*
+ Stack entry for one node currently taking part in post-order traversal.
+ We maintain a stack of pending nodes during the traversal, as the traversal
+ happens in a state machine rather than by recursion.
+ */
+ struct stack_entry {
+ /* Saved position after reading header. */
+ binlog_chunk_reader::saved_position saved_pos;
+ /* The location of this node's OOB record. */
+ uint64_t file_no;
+ uint64_t offset;
+ /* Right child, to be traversed after left child. */
+ uint64_t right_file_no;
+ uint64_t right_offset;
+ /* Offset of real data in this node, after header. */
+ uint32_t header_len;
+ /* Amount of data read into rd_buf, and amount used to parse header. */
+ uint32_t rd_buf_len;
+ uint32_t rd_buf_sofar;
+ /* Current state in post-order traversal state machine. */
+ enum oob_states state;
+ /* Buffer for reading header. */
+ byte rd_buf[5*COMPR_INT_MAX64];
+ /*
+ True when the node is reached using only left child pointers, false
+ otherwise. Used to identify the left-most leaf in a tree which points to
+ a prior tree that must be traversed first.
+ */
+ bool is_leftmost;
+ };
+ small_vectorstack;
+
+ /* State machine current state. */
+ enum oob_states state;
+
+public:
+ innodb_binlog_oob_reader();
+ ~innodb_binlog_oob_reader();
+
+ void start_traversal(uint64_t file_no, uint64_t offset);
+ bool oob_traversal_done() { return stack.empty(); }
+ int read_data(binlog_chunk_reader *chunk_rd, uchar *buf, int max_len);
+
+private:
+ void push_state(enum oob_states state, uint64_t file_no, uint64_t offset,
+ bool is_leftmost);
+};
+
+
+class ha_innodb_binlog_reader : public handler_binlog_reader {
+ enum reader_states {
+ ST_read_next_event_group, ST_read_oob_data, ST_read_commit_record
+ };
+
+ binlog_chunk_reader chunk_rd;
+ innodb_binlog_oob_reader oob_reader;
+ binlog_chunk_reader::saved_position saved_commit_pos;
+
+ /* Out-of-band data to read after commit record, if any. */
+ uint64_t oob_count;
+ uint64_t oob_last_file_no;
+ uint64_t oob_last_offset;
+ /* Any secondary out-of-band data to be also read. */
+ uint64_t oob_count2;
+ uint64_t oob_last_file_no2;
+ uint64_t oob_last_offset2;
+ /*
+ Originally requested starting file_no, from init_gtid_pos() or
+ init_legacy_pos(). Or ~0 if none.
+ */
+ uint64_t requested_file_no;
+ /* Buffer to hold a page read directly from the binlog file. */
+ uchar *page_buf;
+ /* Keep track of pending bytes in the rd_buf. */
+ uint32_t rd_buf_len;
+ uint32_t rd_buf_sofar;
+ /* State for state machine reading chunks one by one. */
+ enum reader_states state;
+
+ /* Used to read the header of the commit record. */
+ byte rd_buf[5*COMPR_INT_MAX64];
+private:
+ int read_data(uchar *buf, uint32_t len);
+
+public:
+ ha_innodb_binlog_reader(bool wait_durable, uint64_t file_no= 0,
+ uint64_t offset= 0);
+ ~ha_innodb_binlog_reader();
+ virtual int read_binlog_data(uchar *buf, uint32_t len) final;
+ virtual bool data_available() final;
+ virtual bool wait_available(THD *thd, const struct timespec *abstime) final;
+ virtual int init_gtid_pos(THD *thd, slave_connection_state *pos,
+ rpl_binlog_state_base *state) final;
+ virtual int init_legacy_pos(THD *thd, const char *filename,
+ ulonglong offset) final;
+ virtual void enable_single_file() final;
+ void seek_internal(uint64_t file_no, uint64_t offset);
+};
+
+
+struct chunk_data_cache : public chunk_data_base {
+ IO_CACHE *cache;
+ /* The GTID to update the binlog state with upon writing the data, or NULL. */
+ const rpl_gtid *gtid_to_update;
+ binlog_oob_context *oob_ctx;
+ my_off_t main_start;
+ size_t main_remain;
+ size_t gtid_remain;
+ uint32_t header_remain;
+ uint32_t header_sofar;
+ byte header_buf[10*COMPR_INT_MAX64];
+
+ chunk_data_cache(IO_CACHE *cache_arg, const rpl_gtid *gtid,
+ handler_binlog_event_group_info *binlog_info)
+ : cache(cache_arg), gtid_to_update(gtid),
+ main_start(binlog_info->out_of_band_offset),
+ main_remain((size_t)(binlog_info->gtid_offset -
+ binlog_info->out_of_band_offset)),
+ header_sofar(0)
+ {
+ size_t end_offset= (size_t)my_b_tell(cache);
+ ut_ad(end_offset > binlog_info->out_of_band_offset);
+ ut_ad(binlog_info->gtid_offset >= binlog_info->out_of_band_offset);
+ ut_ad(end_offset >= binlog_info->gtid_offset);
+ gtid_remain= end_offset - (size_t)binlog_info->gtid_offset;
+
+ binlog_oob_context *c=
+ static_cast(binlog_info->engine_ptr);
+ unsigned char *p= header_buf;
+ ut_ad(c);
+ oob_ctx= c;
+ if (UNIV_UNLIKELY(!c))
+ ;
+ else if (UNIV_UNLIKELY(binlog_info->xa_xid != nullptr) &&
+ !binlog_info->internal_xa)
+ {
+ /*
+ For explicit user XA COMMIT, the commit record must point to the
+ OOB data previously saved in XA PREPARE.
+ */
+ bool err= ibb_xa_xid_hash->run_on_xid(binlog_info->xa_xid,
+ [&p](const ibb_xid_hash::xid_elem *elem) -> bool {
+ if (UNIV_LIKELY(elem->oob_num_nodes > 0))
+ {
+ p= compr_int_write(p, elem->oob_num_nodes);
+ p= compr_int_write(p, elem->oob_first_file_no);
+ p= compr_int_write(p, elem->oob_first_offset);
+ p= compr_int_write(p, elem->oob_last_file_no);
+ p= compr_int_write(p, elem->oob_last_offset);
+ p= compr_int_write(p, 0);
+ }
+ else
+ p= compr_int_write(p, 0);
+ return false;
+ });
+ /*
+ The XID must always be found, else we have a serious
+ inconsistency between the server layer and binlog state.
+ In case of inconsistency, better crash than leave a corrupt
+ binlog.
+ */
+ ut_a(!err);
+ ut_ad(binlog_info->engine_ptr2 == nullptr);
+ }
+ else if (c->node_list_len)
+ {
+ /*
+ Link to the out-of-band data. First store the number of nodes; then
+ store 2 x 2 numbers of file_no/offset for the first and last node.
+
+ There's a special case when we have to link two times out-of-band data,
+ due to mixing non-transactional and transactional stuff. In that case,
+ the non-transactional goes first. In the common case where there is no
+ dual oob references, we just store a single 0 count.
+ */
+ binlog_oob_context *c2= c->secondary_ctx=
+ static_cast(binlog_info->engine_ptr2);
+ if (UNIV_UNLIKELY(c2 != nullptr) && c2->node_list_len)
+ {
+ uint32_t last2= c2->node_list_len-1;
+ uint64_t num_nodes2= c2->node_list[last2].node_index + 1;
+ p= compr_int_write(p, num_nodes2);
+ p= compr_int_write(p, c2->first_node_file_no);
+ p= compr_int_write(p, c2->first_node_offset);
+ p= compr_int_write(p, c2->node_list[last2].file_no);
+ p= compr_int_write(p, c2->node_list[last2].offset);
+ }
+
+ uint32_t last= c->node_list_len-1;
+ uint64_t num_nodes= c->node_list[last].node_index + 1;
+ p= compr_int_write(p, num_nodes);
+ p= compr_int_write(p, c->first_node_file_no);
+ p= compr_int_write(p, c->first_node_offset);
+ p= compr_int_write(p, c->node_list[last].file_no);
+ p= compr_int_write(p, c->node_list[last].offset);
+ if (UNIV_LIKELY(c2 == nullptr) || c2->node_list_len == 0)
+ p= compr_int_write(p, 0);
+ }
+ else
+ {
+ /*
+ No out-of-band data, marked with a single 0 count for nodes and no
+ first/last links.
+ */
+ p= compr_int_write(p, 0);
+ }
+ header_remain= (uint32_t)(p - header_buf);
+ ut_ad((size_t)(p - header_buf) <= sizeof(header_buf));
+
+ ut_ad (cache->pos_in_file <= binlog_info->out_of_band_offset);
+
+ if (UNIV_UNLIKELY(binlog_info->internal_xa))
+ {
+ /*
+ Insert the XID for the internal 2-phase commit in the xid_hash,
+ incrementing the reference count. This will ensure we hold on to
+ the commit record until ibb_binlog_unlog() is called, at which point
+ the other participating storage engine(s) have durably committed.
+ */
+ bool err= ibb_xa_xid_hash->add_xid(binlog_info->xa_xid, c);
+ ut_a(!err);
+ }
+
+ /* Start with the GTID event, which is put at the end of the IO_CACHE. */
+ my_bool res= reinit_io_cache(cache, READ_CACHE, binlog_info->gtid_offset, 0, 0);
+ ut_a(!res);
+ }
+ ~chunk_data_cache() { }
+
+ virtual std::pair copy_data(byte *p, uint32_t max_len) final
+ {
+ uint32_t size= 0;
+
+ /*
+ Update the current GTID state of the binlog. This update must be done
+ exactly here, upon writing the first byte of data into a binlog page.
+
+ Because if updated before calling fsp_binlog_write_rec(), then a GTID
+ state record binlogged right there, at the start of the commit record,
+ would incorrecly mark that the GTID is contained before that point,
+ when in fact it's after.
+
+ And on the other hand, if updated after, then a GTID state record
+ binlogged in the middle of a commit record could incorrectly mark that
+ the GTID is fully contained before that point, where in fact only part
+ of the GTID is available before; this way starting reading at that GTID
+ record would wrongly skip the GTID.
+ */
+ if (gtid_to_update)
+ {
+ binlog_full_state.update_nolock(gtid_to_update);
+ binlog_diff_state.update_nolock(gtid_to_update);
+ gtid_to_update= nullptr;
+ }
+
+ /* Write header data, if any still available. */
+ if (header_remain > 0)
+ {
+ size= header_remain > max_len ? max_len : (uint32_t)header_remain;
+ memcpy(p, header_buf + header_sofar, size);
+ header_remain-= size;
+ header_sofar+= size;
+ max_len-= size;
+ if (UNIV_UNLIKELY(max_len == 0))
+ {
+ ut_ad(gtid_remain + main_remain > 0);
+ return {size, false};
+ }
+ }
+
+ /* Write GTID data, if any still available. */
+ ut_ad(header_remain == 0);
+ if (gtid_remain > 0)
+ {
+ uint32_t size2= gtid_remain > max_len ? max_len : (uint32_t)gtid_remain;
+ int res2= my_b_read(cache, p + size, size2);
+ ut_a(!res2 /* Reading from in-memory cache data cannot fail. */);
+ gtid_remain-= size2;
+ if (gtid_remain == 0)
+ my_b_seek(cache, main_start); /* Move to read the rest of the events. */
+ max_len-= size2;
+ size+= size2;
+ if (max_len == 0)
+ return {size, gtid_remain + main_remain == 0};
+ }
+
+ /* Write remaining data. */
+ ut_ad(gtid_remain == 0);
+ if (main_remain == 0)
+ {
+ /*
+ This means that only GTID data is present, eg. when the main data was
+ already binlogged out-of-band.
+ */
+ ut_ad(size > 0);
+ return {size, true};
+ }
+ uint32_t size2= main_remain > max_len ? max_len : (uint32_t)main_remain;
+ int res2= my_b_read(cache, p + size, size2);
+ ut_a(!res2);
+ ut_ad(main_remain >= size2);
+ main_remain-= size2;
+ return {size + size2, main_remain == 0};
+ }
+
+ /*
+ To be called after binlogging is done, to decrement refcounts to any
+ OOB nodes.
+ */
+ void after_copy_data()
+ {
+ if (UNIV_LIKELY(oob_ctx != nullptr) && oob_ctx->pending_refcount)
+ {
+ ibb_file_hash.oob_ref_dec(oob_ctx->first_node_file_no, oob_ctx->lf_pins);
+ oob_ctx->pending_refcount= false;
+ if (UNIV_UNLIKELY(oob_ctx->secondary_ctx != nullptr) &&
+ oob_ctx->secondary_ctx->pending_refcount)
+ {
+ ibb_file_hash.oob_ref_dec(oob_ctx->secondary_ctx->first_node_file_no,
+ oob_ctx->secondary_ctx->lf_pins);
+ oob_ctx->secondary_ctx->pending_refcount= false;
+ }
+ }
+
+ }
+};
+
+
+template
+struct chunk_data_from_buf : public chunk_data_base {
+ static constexpr uint32_t bufsize= bufsize_;
+
+ uint32_t data_remain;
+ uint32_t data_sofar;
+ byte buffer[bufsize];
+
+ chunk_data_from_buf() : data_sofar(0)
+ {
+ /* data_remain must be initialized in derived class constructor. */
+ }
+
+ virtual std::pair copy_data(byte *p, uint32_t max_len) final
+ {
+ if (UNIV_UNLIKELY(data_remain <= 0))
+ return {0, true};
+ uint32_t size= data_remain > max_len ? max_len : data_remain;
+ memcpy(p, buffer + data_sofar, size);
+ data_remain-= size;
+ data_sofar+= size;
+ return {size, data_remain == 0};
+ }
+ ~chunk_data_from_buf() { }
+};
+
+
+/**
+ Record data for the XA prepare record.
+
+ Size needed for the record data:
+ 1 byte engine count.
+ 4 bytes formatID
+ 1 byte gtrid length
+ 1 byte bqual length
+ 128 bytes (max) gtrid and bqual strings.
+ And last 5 compressed integers at most 5*COMPR_INT_MAX64:
+ num_oob_nodes
+ first_oob_file_no
+ first_oob_offset
+ last_oob_file_no
+ last_oob_offset
+*/
+static constexpr size_t ibb_prepare_record_max_size=
+ 1 + 4 + 1 + 1 + 128 + 5*COMPR_INT_MAX64;
+struct chunk_data_xa_prepare :
+ public chunk_data_from_buf {
+
+ chunk_data_xa_prepare(const XID *xid, uchar engine_count,
+ binlog_oob_context *c)
+ {
+ buffer[0]= engine_count;
+ int4store(&buffer[1], xid->formatID);
+ ut_a(xid->gtrid_length >= 0 && xid->gtrid_length <= 64);
+ buffer[5]= (uchar)xid->gtrid_length;
+ ut_a(xid->bqual_length >= 0 && xid->bqual_length <= 64);
+ buffer[6]= (uchar)xid->bqual_length;
+ memcpy(&buffer[7], &xid->data[0], xid->gtrid_length + xid->bqual_length);
+ byte *p= &buffer[7] + xid->gtrid_length + xid->bqual_length;
+ uint32_t last= c->node_list_len-1;
+ p= compr_int_write(p, c->node_list[last].node_index + 1);
+ p= compr_int_write(p, c->first_node_file_no);
+ p= compr_int_write(p, c->first_node_offset);
+ p= compr_int_write(p, c->node_list[last].file_no);
+ p= compr_int_write(p, c->node_list[last].offset);
+ data_remain= static_cast(p - buffer);
+ }
+ ~chunk_data_xa_prepare() { }
+};
+
+
+/**
+ Record data for the XA COMMIT or XA ROLLBACK record.
+
+ Size needed for the record data:
+ 1 byte type/flag.
+ 4 bytes formatID
+ 1 byte gtrid length
+ 1 byte bqual length
+ 128 bytes (max) gtrid and bqual strings.
+*/
+struct chunk_data_xa_complete :
+ public chunk_data_from_buf<1 + 4 + 1 + 1 + 128> {
+
+ chunk_data_xa_complete(const XID *xid, bool is_commit)
+ {
+ buffer[0]= (is_commit ? IBB_FL_XA_TYPE_COMMIT : IBB_FL_XA_TYPE_ROLLBACK);
+ int4store(&buffer[1], xid->formatID);
+ ut_a(xid->gtrid_length >= 0 && xid->gtrid_length <= 64);
+ buffer[5]= (uchar)xid->gtrid_length;
+ ut_a(xid->bqual_length >= 0 && xid->bqual_length <= 64);
+ buffer[6]= (uchar)xid->bqual_length;
+ memcpy(&buffer[7], &xid->data[0], xid->gtrid_length + xid->bqual_length);
+ data_remain=
+ static_cast(7 + xid->gtrid_length + xid->bqual_length);
+ }
+ ~chunk_data_xa_complete() { }
+};
+
+
+class gtid_search {
+public:
+ gtid_search();
+ ~gtid_search();
+ int find_gtid_pos(slave_connection_state *pos,
+ rpl_binlog_state_base *out_state, uint64_t *out_file_no,
+ uint64_t *out_offset);
+private:
+ uint64_t cur_open_file_no;
+ uint64_t cur_open_file_length;
+ File cur_open_file;
+};
+
+
+struct found_binlogs {
+ uint64_t last_file_no, prev_file_no, earliest_file_no;
+ size_t last_size, prev_size, total_size;
+ int num_found;
+ /* Default constructor to silence compiler warnings -Wuninitialized. */
+ found_binlogs()= default;
+};
+
+
+/**
+ Class used during startup to recover any pending prepared XID for internal
+ 2pc or user XA. Filled-in from prepare and commit/rollback records found
+ during scan of the binlog file, and used by the server to decide whether
+ to keep, commit, or roll back any prepared transactions/XID found in engines.
+*/
+class ibb_binlog_xid_info : public handler_binlog_xid_info
+{
+public:
+ /*
+ In addition to the information needed by the server layer, we need
+ references to the binlogged OOB data of prepared transactions to
+ populate entries in our internal ibb_xa_xid_hash.
+ */
+ uint64_t num_oob_nodes;
+ uint64_t first_oob_file_no;
+ uint64_t first_oob_offset;
+ uint64_t last_oob_file_no;
+ uint64_t last_oob_offset;
+ /* This is the file_no of the prepare/commit/rollback record itself. */
+ uint64_t rec_file_no;
+
+ ibb_binlog_xid_info(binlog_xid_state typ, uint64_t rec_file_no_) :
+ handler_binlog_xid_info(typ), rec_file_no(rec_file_no_) { }
+ virtual ~ibb_binlog_xid_info() override { };
+};
+
+
+/**
+ This structure holds the state needed during InnoDB recovery for recovering
+ binlog tablespace files.
+*/
+class binlog_recovery {
+public:
+ struct found_binlogs scan_result;
+ byte *page_buf;
+ const char *binlog_dir;
+ /*
+ The current file number being recovered.
+ This starts out as the most recent existing non-empty binlog that has a
+ starting LSN no bigger than the recovery starting LSN. This should always be
+ one of the two most recent binlog files found at startup.
+ */
+ uint64_t cur_file_no;
+ /* The physical length of cur_file_no file. */
+ uint64_t cur_phys_size;
+ /*
+ The starting LSN (as stored in the header of the binlog tablespace file).
+ No redo prior to this LSN should be applied to this file.
+ */
+ lsn_t start_file_lsn;
+ /*
+ The LSN of the previously applied redo record. Used to ignore duplicate
+ redo records passed from the InnoDB recovery layer, eg. in multi-batch
+ recovery. Also prev_size, prev_page_no, prev_offset, prev_space_id.
+ */
+ lsn_t prev_lsn;
+ size_t prev_size;
+
+ /* Open file for cur_file_no, or -1 if not open. */
+ File cur_file_fh;
+ /* The sofar position of redo in cur_file_no (end point of previous redo). */
+ uint32_t cur_page_no;
+ uint32_t cur_page_offset;
+
+ uint32_t prev_page_no;
+ uint16_t prev_offset;
+ bool prev_space_id;
+
+ /* The path to cur_file_no. */
+ char full_path[OS_FILE_MAX_PATH];
+
+ bool inited;
+ /*
+ Flag set in case of severe error and --innodb-force_recovery to completely
+ skip any binlog recovery.
+ */
+ bool skip_recovery;
+ /*
+ Special case, if we start from completely empty (no non-empty binlog files).
+ This should recover into an empty binlog state.
+ */
+ bool start_empty;
+ /*
+ Special case: The last two files are empty. Then we ignore the last empty
+ file and use the 2 previous files instead. The ignored file is deleted only
+ after successful recovery, to try to avoid destroying data in case of
+ recovery problems.
+ */
+ bool ignore_last;
+ /*
+ Mark the case where the first binlog tablespace file we need to consider for
+ recovery has file LSN that is later than the first redo record; in this case
+ we need to skip records until the first one that applies to this file.
+ */
+ bool skipping_early_lsn;
+ /*
+ Skip any initial records until the start of a page. We are guaranteed that
+ any page that needs to be recovered will have recovery data for the whole
+ page, and this way we never need to read-modify-write pages during recovery.
+ */
+ bool skipping_partial_page;
+
+ bool init_recovery(bool space_id, uint32_t page_no, uint16_t offset,
+ lsn_t start_lsn, lsn_t lsn,
+ const byte *buf, size_t size) noexcept;
+ bool apply_redo(bool space_id, uint32_t page_no, uint16_t offset,
+ lsn_t start_lsn, lsn_t lsn,
+ const byte *buf, size_t size) noexcept;
+ int get_header(uint64_t file_no, lsn_t &out_lsn, bool &out_empty) noexcept;
+ bool init_recovery_from(uint64_t file_no, lsn_t file_lsn, uint32_t page_no,
+ uint16_t offset, lsn_t lsn,
+ const byte *buf, size_t size) noexcept;
+ void init_recovery_empty() noexcept;
+ void init_recovery_skip_all() noexcept;
+ void end_actions(bool recovery_successful) noexcept;
+ void release() noexcept;
+ bool open_cur_file() noexcept;
+ bool flush_page() noexcept;
+ void zero_out_cur_file();
+ bool close_file() noexcept;
+ bool next_file() noexcept;
+ bool next_page() noexcept;
+ bool update_page_from_record(uint16_t offset,
+ const byte *buf, size_t size) noexcept;
+};
+
+
+static binlog_recovery recover_obj;
+
+
+static void innodb_binlog_prealloc_thread();
+static int scan_for_binlogs(const char *binlog_dir, found_binlogs *binlog_files,
+ bool error_if_missing) noexcept;
+static int innodb_binlog_discover();
+static bool binlog_state_recover(uint64_t *out_xa_file_no,
+ uint64_t *out_xa_offset);
+static void innodb_binlog_autopurge(uint64_t first_open_file_no, LF_PINS *pins);
+static bool binlog_scan_for_xid(uint64_t start_file_no, uint64_t start_offset,
+ HASH *hash);
+static bool ibb_init_xid_hash(HASH *hash, LF_PINS *pins);
+
+/**
+ Read the header of a binlog tablespace file identified by file_no.
+ Sets the out_empty false if the file is empty or has checksum error (or
+ is missing).
+ Else sets out_empty true and sets out_lsn from the header.
+
+ Returns:
+ -1 error
+ 0 File is missing (ENOENT) or has bad checksum on first page.
+ 1 File found (but may be empty according to out_empty).
+*/
+int
+get_binlog_header(const char *binlog_path, byte *page_buf,
+ lsn_t &out_lsn, bool &out_empty) noexcept
+{
+ binlog_header_data header;
+
+ out_empty= true;
+ out_lsn= 0;
+
+ File fh= my_open(binlog_path, O_RDONLY | O_BINARY, MYF(0));
+ if (fh < (File)0)
+ return (my_errno == ENOENT ? 0 : -1);
+ size_t read= my_pread(fh, page_buf, ibb_page_size, 0, MYF(0));
+ my_close(fh, MYF(0));
+ if (UNIV_UNLIKELY(read == (size_t)-1))
+ return -1;
+ if (read == 0)
+ return 0;
+ /*
+ If the crc32 does not match, the page was not written properly, so treat
+ it as an empty file.
+ */
+ const uint32_t payload= (uint32_t)ibb_page_size - BINLOG_PAGE_CHECKSUM;
+ uint32_t crc32= uint4korr(page_buf + payload);
+ if (UNIV_UNLIKELY(crc32 != my_crc32c(0, page_buf, payload)))
+ return 0;
+
+ fsp_binlog_extract_header_page(page_buf, &header);
+ if (header.is_invalid)
+ return 0;
+ if (!header.is_empty)
+ {
+ out_empty= false;
+ out_lsn= header.start_lsn;
+ }
+ return 1;
+}
+
+
+int
+binlog_recovery::get_header(uint64_t file_no, lsn_t &out_lsn, bool &out_empty)
+ noexcept
+{
+ char full_path[OS_FILE_MAX_PATH];
+ binlog_name_make(full_path, file_no, binlog_dir);
+ return get_binlog_header(full_path, page_buf, out_lsn, out_empty);
+}
+
+
+bool binlog_recovery::init_recovery(bool space_id, uint32_t page_no,
+ uint16_t offset,
+ lsn_t start_lsn, lsn_t end_lsn,
+ const byte *buf, size_t size) noexcept
+{
+ /* Start by initializing resource pointers so we are safe to releaes(). */
+ cur_file_fh= (File)-1;
+ if (!(page_buf= static_cast
+ (ut_malloc(ibb_page_size, mem_key_binlog))))
+ {
+ my_error(ER_OUTOFMEMORY, MYF(MY_WME), ibb_page_size);
+ return true;
+ }
+ memset(page_buf, 0, ibb_page_size);
+ inited= true;
+ /*
+ ToDo: It would be good to find a way to not duplicate this logic for
+ where the binlog tablespace filess are stored with the code in
+ innodb_binlog_init(). But it's a bit awkward, because InnoDB recovery
+ runs during plugin init, so not even available for the server to call
+ into until after recovery is done.
+ */
+ binlog_dir= opt_binlog_directory;
+ if (!binlog_dir || !binlog_dir[0])
+ binlog_dir= ".";
+ if (scan_for_binlogs(binlog_dir, &scan_result, true) <= 0)
+ return true;
+
+ /*
+ Here we find the two most recent, non-empty binlogs to do recovery on.
+ Before we allocate binlog tablespace file N+2, we flush and fsync file N
+ to disk. This ensures that we only ever need to apply redo records to the
+ two most recent files during recovery.
+
+ A special case however arises if the two most recent binlog files are
+ both completely empty. Then we do not have any LSN to match against to
+ know if a redo record applies to one of these two files, or to an earlier
+ file with same value of bit 0 of the file_no. In this case, we ignore the
+ most recent file (deleting it later after successful recovery), and
+ consider instead the two prior files, the first of which is guaranteed to
+ have durably saved a starting LSN to use.
+
+ Hence the loop, which can only ever have one or two iterations.
+
+ A further special case is if there are fewer than two (or three if last
+ two are empty) files. If there are no files, or only empty files, then the
+ server must have stopped just after RESET MASTER (or just after
+ initializing the binlogs at first startup), and we should just start the
+ binlogs from scratch.
+ */
+ ignore_last= false;
+ uint64_t file_no2= scan_result.last_file_no;
+ uint64_t file_no1= scan_result.prev_file_no;
+ int num_binlogs= scan_result.num_found;
+ for (;;)
+ {
+ lsn_t lsn1= 0, lsn2= 0;
+ bool is_empty1= true, is_empty2= true;
+ int res2= get_header(file_no2, lsn2, is_empty2);
+
+ if (num_binlogs == 0 ||
+ (num_binlogs == 1 && is_empty2))
+ {
+ init_recovery_empty();
+ return false;
+ }
+ if (num_binlogs == 1)
+ {
+ uint64_t start_file_no= file_no2;
+ /*
+ Only one binlog file found.
+
+ This first recovery record may apply to the previous file (which has
+ then presumably been purged since the last checkpoint). Or it may
+ apply to this file, or only to the following file. The case where it
+ is not this file needs a bit of care.
+
+ If the recovery record lsn is less than the lsn in this file, we know
+ that it must apply to the previous file, and we can start from this
+ file.
+
+ If the recovery record lsn is equal or greater, then it can apply to
+ the previous file if it is part of a mini-transaction that spans into
+ this file. Or it can apply to the following file. If it applies to the
+ following file it must have page_no=0 and offset=0, since that file is
+ missing and will be recovered from scratch. Conversely, if the record
+ has page_no=0 and offset=0, it cannot apply to the previous file, as
+ we keep mini-transactions smaller than one binlog file.
+ */
+ if (space_id != (file_no2 & 1) && start_lsn >= lsn2 &&
+ page_no == 0 && offset == 0)
+ ++start_file_no;
+ return init_recovery_from(start_file_no, lsn2, page_no, offset,
+ start_lsn, buf, size);
+ }
+
+ int res1= get_header(file_no1, lsn1, is_empty1);
+
+ if (res2 < 0 && !srv_force_recovery)
+ {
+ sql_print_error("InnoDB: I/O error reading binlog file number %" PRIu64,
+ file_no2);
+ return true;
+ }
+ if (res1 < 0 && !srv_force_recovery)
+ {
+ sql_print_error("InnoDB: I/O error reading binlog file number %" PRIu64,
+ file_no1);
+ return true;
+ }
+ if (is_empty1 && is_empty2)
+ {
+ if (!ignore_last)
+ {
+ ignore_last= true;
+ if (file_no2 > scan_result.earliest_file_no)
+ {
+ --file_no2;
+ if (file_no1 > scan_result.earliest_file_no)
+ --file_no1;
+ else
+ --num_binlogs;
+ }
+ else
+ --num_binlogs;
+ continue;
+ }
+ if (srv_force_recovery)
+ {
+ /*
+ If the last 3 files are empty, we cannot get an LSN to know which
+ records apply to each file. This should not happen unless there is
+ damage to the file system. If force recovery is requested, we must
+ simply do no recovery at all on the binlog files.
+ */
+ sql_print_warning("InnoDB: Binlog tablespace file recovery is not "
+ "possible. Recovery is skipped due to "
+ "--innodb-force-recovery");
+ init_recovery_skip_all();
+ return false;
+ }
+ sql_print_error("InnoDB: Last 3 binlog tablespace files are all empty. "
+ "Recovery is not possible");
+ return true;
+ }
+ if (is_empty2)
+ {
+ /*
+ As above for the case where only one file is found, we need to
+ carefully distinguish the case where the recovery record applies to
+ file_no1-1 or file_no1+1; when start_lsn >= lsn1, the record can
+ apply to file_no1+1 only if it is for page_no==0 and offset==0.
+ */
+ if (space_id != (file_no1 & 1) && start_lsn >= lsn1 &&
+ page_no == 0 && offset == 0)
+ return init_recovery_from(file_no2, lsn1, page_no, offset,
+ start_lsn, buf, size);
+ else
+ return init_recovery_from(file_no1, lsn1, page_no, offset,
+ start_lsn, buf, size);
+ }
+ else if (space_id == (file_no2 & 1) && start_lsn >= lsn2)
+ {
+ /* The record must apply to file_no2. */
+ return init_recovery_from(file_no2, lsn2,
+ page_no, offset, start_lsn, buf, size);
+ }
+ else
+ {
+ /*
+ The record cannot apply to file_no2, as either the space_id differs
+ or the lsn is too early. Start from file_no1.
+ */
+ return init_recovery_from(file_no1, lsn1,
+ page_no, offset, start_lsn, buf, size);
+ }
+ /* NotReached. */
+ }
+}
+
+
+bool
+binlog_recovery::init_recovery_from(uint64_t file_no, lsn_t file_lsn,
+ uint32_t page_no, uint16_t offset,
+ lsn_t lsn, const byte *buf, size_t size)
+ noexcept
+{
+ cur_file_no= file_no;
+ cur_phys_size= 0;
+ start_file_lsn= file_lsn;
+ prev_lsn= lsn;
+ prev_space_id= file_no & 1;
+ prev_page_no= page_no;
+ prev_offset= offset;
+ prev_size= size;
+ cur_page_no= page_no;
+ cur_page_offset= 0;
+ skip_recovery= false;
+ start_empty= false;
+ skipping_partial_page= true;
+ if (lsn < start_file_lsn)
+ skipping_early_lsn= true;
+ else
+ {
+ skipping_early_lsn= false;
+ if (offset <= BINLOG_PAGE_DATA)
+ {
+ skipping_partial_page= false;
+ return update_page_from_record(offset, buf, size);
+ }
+ }
+ return false;
+}
+
+
+/**
+ Initialize recovery from the state where there are no binlog files, or only
+ completely empty binlog files. In this case we have no file LSN to compare
+ redo records against.
+
+ This can only happen if we crash immediately after RESET MASTER (or fresh
+ server installation) as an initial file header is durably written to disk
+ before binlogging new data. Therefore we should skip _all_ redo records and
+ recover into a completely empty state.
+*/
+void
+binlog_recovery::init_recovery_empty() noexcept
+{
+ cur_file_no= 0;
+ cur_phys_size= 0;
+ start_file_lsn= (lsn_t)0;
+ prev_lsn= (lsn_t)0;
+ prev_space_id= 0;
+ prev_page_no= 0;
+ prev_offset= 0;
+ prev_size= 0;
+ cur_page_no= 0;
+ cur_page_offset= 0;
+ skip_recovery= false;
+ start_empty= true;
+ ignore_last= false;
+ skipping_early_lsn= false;
+ skipping_partial_page= true;
+}
+
+
+void
+binlog_recovery::init_recovery_skip_all() noexcept
+{
+ skip_recovery= true;
+}
+
+
+void
+binlog_recovery::end_actions(bool recovery_successful) noexcept
+{
+ char full_path[OS_FILE_MAX_PATH];
+ if (recovery_successful && !skip_recovery)
+ {
+ if (!start_empty)
+ {
+ if (cur_page_offset)
+ flush_page();
+ if (cur_file_fh > (File)-1)
+ zero_out_cur_file();
+ close_file();
+ ++cur_file_no;
+ }
+
+ /*
+ Delete any binlog tablespace files following the last recovered file.
+ These files could be pre-allocated but never used files, or they could be
+ files that were written with data that was eventually not recovered due
+ to --innodb-flush-log-at-trx-commit=0|2.
+ */
+ for (uint64_t i= cur_file_no;
+ scan_result.num_found >= 1 && i <= scan_result.last_file_no;
+ ++i)
+ {
+ binlog_name_make(full_path, i, binlog_dir);
+ if (my_delete(full_path, MYF(MY_WME)))
+ sql_print_warning("InnoDB: Could not delete empty file '%s' ("
+ "error: %d)", full_path, my_errno);
+ }
+ }
+ release();
+}
+
+
+void
+binlog_recovery::release() noexcept
+{
+ if (cur_file_fh >= (File)0)
+ {
+ my_close(cur_file_fh, MYF(0));
+ cur_file_fh= (File)-1;
+ }
+ ut_free(page_buf);
+ page_buf= nullptr;
+ inited= false;
+}
+
+
+bool
+binlog_recovery::open_cur_file() noexcept
+{
+ if (cur_file_fh >= (File)0)
+ my_close(cur_file_fh, MYF(0));
+ binlog_name_make(full_path, cur_file_no, binlog_dir);
+ cur_file_fh= my_open(full_path, O_RDWR | O_BINARY, MYF(0));
+ if (cur_file_fh < (File)0)
+ {
+ /*
+ If we are on page 0 and the binlog file does not exist, then we should
+ create it (and recover its content).
+ Otherwise, it is an error, we cannot recover it as we are missing the
+ start of it.
+ */
+ if (my_errno != ENOENT ||
+ cur_page_no != 0 ||
+ (cur_file_fh= my_open(full_path, O_RDWR | O_CREAT | O_TRUNC |
+ O_BINARY, MYF(0))) < (File)0)
+ {
+ my_error(EE_FILENOTFOUND, MYF(MY_WME), full_path, my_errno);
+ return true;
+ }
+ }
+ cur_phys_size= (uint64_t)my_seek(cur_file_fh, 0, MY_SEEK_END, MYF(0));
+ return false;
+}
+
+
+bool
+binlog_recovery::flush_page() noexcept
+{
+ if (cur_file_fh < (File)0 &&
+ open_cur_file())
+ return true;
+ size_t res=
+ crc32_pwrite_page(cur_file_fh, page_buf, cur_page_no, MYF(MY_WME));
+ if (res != ibb_page_size)
+ return true;
+ cur_page_offset= 0;
+ memset(page_buf, 0, ibb_page_size);
+ return false;
+}
+
+
+void
+binlog_recovery::zero_out_cur_file()
+{
+ if (cur_file_fh < (File)0)
+ return;
+
+ /* Recover the original size from the current file. */
+ int res= crc32_pread_page(cur_file_fh, page_buf, 0, MYF(0));
+ if (res <= 0)
+ {
+ sql_print_warning("InnoDB: Could not read last binlog file during recovery");
+ return;
+ }
+ binlog_header_data header;
+ fsp_binlog_extract_header_page(page_buf, &header);
+
+ if (header.is_invalid)
+ {
+ sql_print_warning("InnoDB: Invalid header page in last binlog file "
+ "during recovery");
+ return;
+ }
+ if (header.is_empty)
+ {
+ sql_print_warning("InnoDB: Empty binlog file header found during recovery");
+ ut_ad(0);
+ return;
+ }
+
+ /* Fill up or truncate the file to its original size. */
+ if (my_chsize(cur_file_fh, (my_off_t)header.page_count << ibb_page_size_shift,
+ 0, MYF(0)))
+ sql_print_warning("InnoDB: Could not change the size of last binlog file "
+ "during recovery (error: %d)", my_errno);
+ for (uint32_t i= cur_page_no + 1; i < header.page_count; ++i)
+ {
+ if (my_pread(cur_file_fh, page_buf, ibb_page_size,
+ (my_off_t)i << ibb_page_size_shift, MYF(0)) <
+ (size_t)ibb_page_size)
+ break;
+ /* Check if page already zeroed out. */
+ if (page_buf[0] == 0 && !memcmp(page_buf, page_buf+1, ibb_page_size - 1))
+ continue;
+ memset(page_buf, 0, ibb_page_size);
+ if (my_pwrite(cur_file_fh, page_buf, ibb_page_size,
+ (uint64_t)i << ibb_page_size_shift, MYF(MY_WME)) <
+ (size_t)ibb_page_size)
+ {
+ sql_print_warning("InnoDB: Error writing to last binlog file during "
+ "recovery (error code: %d)", my_errno);
+ break;
+ }
+ }
+}
+
+
+bool
+binlog_recovery::close_file() noexcept
+{
+ if (cur_file_fh >= (File)0)
+ {
+ if (my_sync(cur_file_fh, MYF(MY_WME)))
+ return true;
+ my_close(cur_file_fh, (File)0);
+ cur_file_fh= (File)-1;
+ cur_phys_size= 0;
+ }
+ return false;
+}
+
+
+bool
+binlog_recovery::next_file() noexcept
+{
+ if (cur_page_offset && flush_page())
+ return true;
+ if (close_file())
+ return true;
+ ++cur_file_no;
+ cur_page_no= 0;
+ return false;
+}
+
+
+bool
+binlog_recovery::next_page() noexcept
+{
+ if (cur_page_offset && flush_page())
+ return true;
+ ++cur_page_no;
+ return false;
+}
+
+
+bool
+binlog_recovery::apply_redo(bool space_id, uint32_t page_no, uint16_t offset,
+ lsn_t start_lsn, lsn_t end_lsn,
+ const byte *buf, size_t size) noexcept
+{
+ if (UNIV_UNLIKELY(skip_recovery) || start_empty)
+ return false;
+
+ /*
+ In a multi-batch recovery, InnoDB recovery redo parser will sometimes
+ pass the same record(s) twice to the binlog recovery.
+
+ The binlog recovery code wants to do consistency checks that records are
+ processed in strict order, so we handle this special case by detecting
+ and ignoring duplicate records.
+
+ A duplicate record is determined by being in the same mtr (identified by
+ end_lsn); and having page_no/offset either earlier in the same space_id,
+ or later in a different space_id. Using the property that an mtr is always
+ smaller than the binlog maximum file size.
+ */
+ if (end_lsn == prev_lsn &&
+ ( ( space_id == prev_space_id &&
+ ( ((uint64_t)page_no << 32 | offset) <=
+ ((uint64_t)prev_page_no << 32 | prev_offset) ) ) ||
+ ( space_id != prev_space_id &&
+ ( ((uint64_t)page_no << 32 | offset) >
+ ((uint64_t)prev_page_no << 32 | prev_offset) ) ) ) )
+ return false;
+ prev_lsn= end_lsn;
+ prev_space_id= space_id;
+ prev_page_no= page_no;
+ prev_offset= offset;
+ prev_size= size;
+
+ if (skipping_partial_page)
+ {
+ if (offset > BINLOG_PAGE_DATA)
+ return false;
+ skipping_partial_page= false;
+ }
+
+ if (skipping_early_lsn)
+ {
+ if (start_lsn < start_file_lsn || space_id != (cur_file_no & 1))
+ return false; /* Skip record for earlier file that's already durable. */
+ /* Now reset the current page to match the real starting point. */
+ cur_page_no= page_no;
+ }
+
+ if (UNIV_UNLIKELY(start_lsn < start_file_lsn))
+ {
+ ut_a(!skipping_early_lsn /* Was handled in condition above */);
+ if (!srv_force_recovery)
+ {
+ sql_print_error("InnoDB: Unexpected LSN " LSN_PF " during recovery, "
+ "expected at least " LSN_PF, start_lsn, start_file_lsn);
+ return true;
+ }
+ sql_print_warning("InnoDB: Ignoring unexpected LSN " LSN_PF " during "
+ "recovery, ", start_lsn);
+ return false;
+ }
+ skipping_early_lsn= false;
+
+ /* Test for moving to the next file. */
+ if (space_id != (cur_file_no & 1))
+ {
+ /* Check that we recovered all of this file. */
+ if ( ( (cur_page_offset > BINLOG_PAGE_DATA &&
+ cur_page_offset < ibb_page_size - BINLOG_PAGE_DATA_END) ||
+ cur_page_no + (cur_page_offset > BINLOG_PAGE_DATA) <
+ cur_phys_size >> ibb_page_size_shift) &&
+ !srv_force_recovery)
+ {
+ sql_print_error("InnoDB: Missing recovery record at end of file_no=%"
+ PRIu64 ", LSN " LSN_PF, cur_file_no, start_lsn);
+ return true;
+ }
+
+ /* Check that we recover from the start of the next file. */
+ if ((page_no > 0 || offset > BINLOG_PAGE_DATA) && !srv_force_recovery)
+ {
+ sql_print_error("InnoDB: Missing recovery record at start of file_no=%"
+ PRIu64 ", LSN " LSN_PF, cur_file_no+1, start_lsn);
+ return true;
+ }
+
+ if (next_file())
+ return true;
+ }
+ /* Test for moving to the next page. */
+ else if (page_no != cur_page_no)
+ {
+ if (cur_page_offset > BINLOG_PAGE_DATA &&
+ cur_page_offset < ibb_page_size - BINLOG_PAGE_DATA_END &&
+ !srv_force_recovery)
+ {
+ sql_print_error("InnoDB: Missing recovery record in file_no=%"
+ PRIu64 ", page_no=%u, LSN " LSN_PF,
+ cur_file_no, cur_page_no, start_lsn);
+ return true;
+ }
+
+ if ((page_no != cur_page_no + 1 || offset > BINLOG_PAGE_DATA) &&
+ !srv_force_recovery)
+ {
+ sql_print_error("InnoDB: Missing recovery record in file_no=%"
+ PRIu64 ", page_no=%u, LSN " LSN_PF,
+ cur_file_no, cur_page_no + 1, start_lsn);
+ return true;
+ }
+
+ if (next_page())
+ return true;
+ }
+ /* Test no gaps in offset. */
+ else if (offset != cur_page_offset &&
+ offset > BINLOG_PAGE_DATA &&
+ !srv_force_recovery)
+ {
+ sql_print_error("InnoDB: Missing recovery record in file_no=%"
+ PRIu64 ", page_no=%u, LSN " LSN_PF,
+ cur_file_no, cur_page_no, start_lsn);
+ return true;
+ }
+
+ if (offset + size >= ibb_page_size)
+ return !srv_force_recovery;
+
+ return update_page_from_record(offset, buf, size);
+}
+
+
+bool
+binlog_recovery::update_page_from_record(uint16_t offset,
+ const byte *buf, size_t size) noexcept
+{
+ memcpy(page_buf + offset, buf, size);
+ if (cur_page_no == 0 && offset == 0)
+ {
+ binlog_header_data header;
+ /*
+ This recovery record is for the file header page.
+ This record is special, it covers only the used part of the header page.
+ The reaminder of the page must be set to zeroes.
+ Additionally, there is an extra CRC corresponding to a minimum
+ page size of IBB_PAGE_SIZE_MIN, in anticipation for future configurable
+ page size.
+ */
+ memset(page_buf + size, 0, ibb_page_size - (size + BINLOG_PAGE_DATA_END));
+ cur_page_offset= (uint32_t)ibb_page_size - BINLOG_PAGE_DATA_END;
+ uint32_t payload= IBB_HEADER_PAGE_SIZE - BINLOG_PAGE_CHECKSUM;
+ int4store(page_buf + payload, my_crc32c(0, page_buf, payload));
+ fsp_binlog_extract_header_page(page_buf, &header);
+ if (header.is_invalid)
+ {
+ sql_print_error("InnoDB: Corrupt or invalid file header found during "
+ "recovery of file number %" PRIu64, cur_file_no);
+ return !srv_force_recovery;
+ }
+ if (header.is_empty)
+ {
+ sql_print_error("InnoDB: Empty file header found during "
+ "recovery of file number %" PRIu64, cur_file_no);
+ return !srv_force_recovery;
+ }
+ if (header.file_no != cur_file_no)
+ {
+ sql_print_error("InnoDB: Inconsistency in file header during recovery. "
+ "The header in file number %" PRIu64 " is for file "
+ "number %" PRIu64, cur_file_no, header.file_no);
+ return !srv_force_recovery;
+ }
+
+ return false;
+ }
+
+ cur_page_offset= offset + (uint32_t)size;
+ return false;
+}
+
+
+/**
+ Check if this is an InnoDB binlog file name.
+ Return the index/file_no if so.
+*/
+bool
+is_binlog_name(const char *name, uint64_t *out_idx)
+{
+ const size_t base_len= sizeof(BINLOG_NAME_BASE) - 1; // Length without '\0' terminator
+ const size_t ext_len= sizeof(BINLOG_NAME_EXT) - 1;
+
+ if (0 != strncmp(name, BINLOG_NAME_BASE, base_len))
+ return false;
+ size_t name_len= strlen(name);
+ if (name_len < base_len + 1 + ext_len)
+ return false;
+ const char *ext_start= name + (name_len - ext_len);
+ if (0 != strcmp(ext_start, BINLOG_NAME_EXT))
+ return false;
+ if (!std::isdigit((unsigned char)(name[base_len])))
+ return false;
+ char *conv_end= nullptr;
+ unsigned long long idx= std::strtoull(name + base_len, &conv_end, 10);
+ if (idx == ULLONG_MAX || conv_end != ext_start)
+ return false;
+
+ *out_idx= (uint64_t)idx;
+ return true;
+}
+
+
+dberr_t
+innodb_binlog_startup_init()
+{
+ dberr_t err= fsp_binlog_init();
+ if (err != DB_SUCCESS)
+ return err;
+ mysql_mutex_init(fsp_purge_binlog_mutex_key, &purge_binlog_mutex, nullptr);
+ binlog_full_state.init();
+ binlog_diff_state.init();
+ ibb_xa_xid_hash= new ibb_xid_hash();
+ if (UNIV_UNLIKELY(!ibb_xa_xid_hash))
+ {
+ sql_print_error("InnoDB: Could not allocate memory for the internal "
+ "XID hash, cannot proceed");
+ return DB_OUT_OF_MEMORY;
+ }
+
+ innodb_binlog_inited= 1;
+ return DB_SUCCESS;
+}
+
+
+static void
+innodb_binlog_init_state()
+{
+ first_open_binlog_file_no= ~(uint64_t)0;
+ for (uint32_t i= 0; i < 4; ++i)
+ {
+ binlog_cur_end_offset[i].store(~(uint64_t)0, std::memory_order_relaxed);
+ binlog_cur_durable_offset[i].store(~(uint64_t)0, std::memory_order_relaxed);
+ }
+ last_created_binlog_file_no= ~(uint64_t)0;
+ earliest_binlog_file_no= ~(uint64_t)0;
+ total_binlog_used_size= 0;
+ active_binlog_file_no.store(~(uint64_t)0, std::memory_order_release);
+ ibb_file_hash.earliest_oob_ref.store(0, std::memory_order_relaxed);
+ binlog_cur_page_no= 0;
+ binlog_cur_page_offset= BINLOG_PAGE_DATA;
+ current_binlog_state_interval=
+ (uint64_t)(innodb_binlog_state_interval >> ibb_page_size_shift);
+ ut_a(innodb_binlog_state_interval ==
+ (current_binlog_state_interval << ibb_page_size_shift));
+ binlog_full_state.reset_nolock();
+ binlog_diff_state.reset_nolock();
+}
+
+
+/** Start the thread that pre-allocates new binlog files. */
+static void
+start_binlog_prealloc_thread()
+{
+ prealloc_thread_end= false;
+ binlog_prealloc_thr_obj= std::thread{innodb_binlog_prealloc_thread};
+
+ mysql_mutex_lock(&active_binlog_mutex);
+ while (last_created_binlog_file_no == ~(uint64_t)0) {
+ /* Wait for the first binlog file to be available. */
+ my_cond_wait(&active_binlog_cond, &active_binlog_mutex.m_mutex);
+ }
+ mysql_mutex_unlock(&active_binlog_mutex);
+}
+
+
+/**
+ Write the initial header record to the file and durably sync it to disk in
+ the binlog tablespace file and in the redo log.
+
+ This is to ensure recovery can work correctly. This way, recovery will
+ always find a non-empty file with an initial lsn to start recovery from.
+ Except in the case where we crash right here; in this case recovery will
+ find no binlog files at all and will know to recover to the empty state
+ with no binlog files present.
+*/
+static void
+binlog_sync_initial()
+{
+ chunk_data_flush dummy_data;
+ mtr_t mtr{nullptr};
+ LF_PINS *lf_pins= lf_hash_get_pins(&ibb_file_hash.hash);
+ ut_a(lf_pins);
+ mtr.start();
+ fsp_binlog_write_rec(&dummy_data, &mtr, FSP_BINLOG_TYPE_FILLER, lf_pins);
+ uint64_t file_no= active_binlog_file_no.load(std::memory_order_relaxed);
+ mtr.commit();
+ lf_hash_put_pins(lf_pins);
+ log_buffer_flush_to_disk(true);
+ binlog_page_fifo->flush_up_to(0, 0);
+ binlog_page_fifo->do_fdatasync(0);
+ ibb_pending_lsn_fifo.add_to_fifo(mtr.commit_lsn(), file_no,
+ binlog_cur_end_offset[file_no & 3].load(std::memory_order_relaxed));
+}
+
+
+void
+ibb_set_max_size(size_t binlog_size)
+{
+ uint64_t pages= binlog_size >> ibb_page_size_shift;
+ if (UNIV_LIKELY(pages > (uint64_t)UINT32_MAX)) {
+ pages= UINT32_MAX;
+ sql_print_warning("Requested max_binlog_size is larger than the maximum "
+ "InnoDB tablespace size, truncated to %llu",
+ (pages << ibb_page_size_shift));
+ } else if (pages < 4) {
+ pages= 4;
+ sql_print_warning("Requested max_binlog_size is smaller than the minimum "
+ "size supported by InnoDB, truncated to %llu",
+ (pages << ibb_page_size_shift));
+ }
+ innodb_binlog_size_in_pages= (uint32_t)pages;
+}
+
+
+/**
+ Open the InnoDB binlog implementation.
+ This is called from server binlog layer if the user configured the binlog to
+ use the innodb implementation (with --binlog-storage-engine=innodb).
+*/
+bool
+innodb_binlog_init(size_t binlog_size, const char *directory,
+ HASH *recovery_hash)
+{
+ /**
+ The file_no from which we should start scanning to recover any prepare and
+ committed XID.
+ */
+ uint64_t recover_start_file_no= ~(uint64_t)0;
+ uint64_t recover_start_offset= 0;
+
+ ibb_set_max_size(binlog_size);
+ if (!directory || !directory[0])
+ directory= ".";
+ else if (strlen(directory) + BINLOG_NAME_MAX_LEN > OS_FILE_MAX_PATH)
+ {
+ sql_print_error("Specified binlog directory path '%s' is too long",
+ directory);
+ return true;
+ }
+ innodb_binlog_directory= directory;
+
+ innodb_binlog_init_state();
+ innodb_binlog_inited= 2;
+
+ /* Find any existing binlog files and continue writing in them. */
+ int res= innodb_binlog_discover();
+ if (res < 0)
+ return true;
+ if (res > 0)
+ {
+ /* We are continuing from existing binlogs. Recover the binlog state. */
+ if (binlog_state_recover(&recover_start_file_no,
+ &recover_start_offset))
+ return true;
+ }
+ else
+ {
+ /* Starting new binlogs, no XA to recover. */
+ recover_start_file_no= ~(uint64_t)0;
+ recover_start_offset= 0;
+ }
+
+ start_binlog_prealloc_thread();
+
+ if (res <= 0)
+ {
+ /*
+ We are creating binlogs anew from scratch.
+ Write and fsync the initial file-header, so that recovery will know where
+ to start in case of a crash.
+ */
+ binlog_sync_initial();
+ }
+ else
+ {
+ /*
+ Recover XIDs for pending 2pc/XA transactions (if any) by scanning
+ required part of binlog.
+ */
+ if (binlog_scan_for_xid(recover_start_file_no, recover_start_offset,
+ recovery_hash))
+ return true;
+ LF_PINS *lf_pins= lf_hash_get_pins(&ibb_file_hash.hash);
+ if (UNIV_UNLIKELY(!lf_pins))
+ {
+ sql_print_error("InnoDB: Out of memory while recovering pending XID");
+ return true;
+ }
+ bool err= ibb_init_xid_hash(recovery_hash, lf_pins);
+ lf_hash_put_pins(lf_pins);
+ return err;
+ }
+
+ return false;
+}
+
+
+/** Compute the (so far) last and last-but-one binlog files found. */
+static void
+process_binlog_name(found_binlogs *bls, uint64_t idx, size_t size)
+{
+ if (bls->num_found == 0)
+ {
+ bls->earliest_file_no= idx;
+ bls->total_size= size;
+ }
+ else
+ {
+ if (idx < bls->earliest_file_no)
+ bls->earliest_file_no= idx;
+ bls->total_size+= size;
+ }
+
+ if (bls->num_found == 0 ||
+ idx > bls->last_file_no) {
+ if (bls->num_found >= 1 && idx == bls->last_file_no + 1) {
+ bls->prev_file_no= bls->last_file_no;
+ bls->prev_size= bls->last_size;
+ bls->num_found= 2;
+ } else {
+ bls->num_found= 1;
+ }
+ bls->last_file_no= idx;
+ bls->last_size= size;
+ } else if (bls->num_found == 1 && idx + 1 == bls->last_file_no) {
+ bls->num_found= 2;
+ bls->prev_file_no= idx;
+ bls->prev_size= size;
+ }
+}
+
+
+/**
+ Scan the binlog directory for binlog files.
+ Returns:
+ 1 Success
+ 0 Binlog directory not found
+ -1 Other error
+*/
+static int
+scan_for_binlogs(const char *binlog_dir, found_binlogs *binlog_files,
+ bool error_if_missing) noexcept
+{
+ MY_DIR *dir= my_dir(binlog_dir, MYF(MY_WANT_STAT));
+ if (!dir)
+ {
+ if (my_errno != ENOENT || error_if_missing)
+ sql_print_error("Could not read the binlog directory '%s', error code %d",
+ binlog_dir, my_errno);
+ return (my_errno == ENOENT ? 0 : -1);
+ }
+
+ binlog_files->num_found= 0;
+ size_t num_entries= dir->number_of_files;
+ fileinfo *entries= dir->dir_entry;
+ for (size_t i= 0; i < num_entries; ++i) {
+ const char *name= entries[i].name;
+ uint64_t idx;
+ if (!is_binlog_name(name, &idx))
+ continue;
+ process_binlog_name(binlog_files, idx, (size_t)entries[i].mystat->st_size);
+ }
+ my_dirend(dir);
+
+ return 1; /* Success */
+}
+
+
+static bool
+binlog_page_empty(const byte *page)
+{
+ return page[BINLOG_PAGE_DATA] == 0;
+}
+
+
+/**
+ Find the last written position in the binlog file.
+ Do a binary search through the pages to find the last non-empty page, then
+ scan the page to find the place to start writing new binlog data.
+
+ Returns:
+ 1 position found, output in *out_space, *out_page_no and *out_pos_in_page.
+ 0 binlog file is empty.
+ -1 error.
+*/
+
+static int
+find_pos_in_binlog(uint64_t file_no, size_t file_size, byte *page_buf,
+ uint32_t *out_page_no, uint32_t *out_pos_in_page,
+ binlog_header_data *out_header_data)
+{
+ const uint32_t page_size= (uint32_t)ibb_page_size;
+ const uint32_t page_size_shift= (uint32_t)ibb_page_size_shift;
+ const uint32_t idx= file_no & 3;
+ char file_name[OS_FILE_MAX_PATH];
+ uint32_t p_0, p_1, p_2, last_nonempty;
+ byte *p, *page_end;
+ bool ret;
+
+ *out_page_no= 0;
+ *out_pos_in_page= BINLOG_PAGE_DATA;
+ out_header_data->diff_state_interval= 0;
+ out_header_data->is_invalid= true;
+
+ binlog_name_make(file_name, file_no);
+ pfs_os_file_t fh= os_file_create(innodb_data_file_key, file_name,
+ OS_FILE_OPEN, OS_DATA_FILE,
+ srv_read_only_mode, &ret);
+ if (!ret) {
+ sql_print_warning("InnoDB: Unable to open file '%s'", file_name);
+ return -1;
+ }
+
+ int res= crc32_pread_page(fh, page_buf, 0, MYF(MY_WME));
+ if (res <= 0) {
+ os_file_close(fh);
+ return -1;
+ }
+ fsp_binlog_extract_header_page(page_buf, out_header_data);
+ if (out_header_data->is_invalid)
+ {
+ sql_print_error("InnoDB: Invalid or corrupt file header in file "
+ "'%s'", file_name);
+ return -1;
+ }
+ if (out_header_data->is_empty) {
+ ret=
+ fsp_binlog_open(file_name, fh, file_no, file_size, ~(uint32_t)0, nullptr);
+ binlog_cur_durable_offset[idx].store(0, std::memory_order_relaxed);
+ binlog_cur_end_offset[idx].store(0, std::memory_order_relaxed);
+ return (ret ? -1 : 0);
+ }
+ if (out_header_data->file_no != file_no)
+ {
+ sql_print_error("InnoDB: Inconsistent file header in file '%s', "
+ "wrong file_no %" PRIu64, file_name,
+ out_header_data->file_no);
+ return -1;
+ }
+ last_nonempty= 0;
+
+ /*
+ During the binary search, p_0-1 is the largest page number that is know to
+ be non-empty. And p_2 is the first page that is known to be empty.
+ */
+ p_0= 1;
+ p_2= (uint32_t)(file_size / page_size);
+ for (;;) {
+ if (p_0 == p_2)
+ break;
+ ut_ad(p_0 < p_2);
+ p_1= (p_0 + p_2) / 2;
+ res= crc32_pread_page(fh, page_buf, p_1, MYF(MY_WME));
+ if (res <= 0) {
+ os_file_close(fh);
+ return -1;
+ }
+ if (binlog_page_empty(page_buf)) {
+ p_2= p_1;
+ } else {
+ p_0= p_1 + 1;
+ last_nonempty= p_1;
+ }
+ }
+ /* At this point, p_0 == p_2 is the first empty page. */
+ ut_ad(p_0 >= 1);
+
+ /*
+ This sometimes does an extra read, but as this is only during startup it
+ does not matter.
+ */
+ res= crc32_pread_page(fh, page_buf, last_nonempty, MYF(MY_WME));
+ if (res <= 0) {
+ os_file_close(fh);
+ return -1;
+ }
+
+ /* Now scan the last page to find the position in it to continue. */
+ p= &page_buf[BINLOG_PAGE_DATA];
+ page_end= &page_buf[page_size - BINLOG_PAGE_DATA_END];
+ while (*p && p < page_end) {
+ if (*p == FSP_BINLOG_TYPE_FILLER) {
+ p= page_end;
+ break;
+ }
+ p += 3 + (((uint32_t)p[2] << 8) | ((uint32_t)p[1] & 0xff));
+ if(UNIV_UNLIKELY(p > page_end))
+ {
+ sql_print_error("InnoDB: Invalid record in file_no=%" PRIu64
+ " page_no=%u (invalid chunk length)",
+ file_no, last_nonempty);
+ return -1;
+ }
+ }
+
+ /*
+ Normalize the position, so that we store (page_no+1, BINLOG_PAGE_DATA)
+ and not (page_no, page_size - BINLOG_PAGE_DATA_END).
+ */
+ byte *partial_page;
+ if (p == page_end)
+ {
+ *out_page_no= p_0;
+ *out_pos_in_page= BINLOG_PAGE_DATA;
+ partial_page= nullptr;
+ }
+ else
+ {
+ *out_page_no= p_0 - 1;
+ *out_pos_in_page= (uint32_t)(p - page_buf);
+ partial_page= page_buf;
+ }
+
+ ret= fsp_binlog_open(file_name, fh, file_no, file_size,
+ *out_page_no, partial_page);
+ uint64_t pos= (*out_page_no << page_size_shift) | *out_pos_in_page;
+ binlog_cur_durable_offset[idx].store(pos, std::memory_order_relaxed);
+ binlog_cur_end_offset[idx].store(pos, std::memory_order_relaxed);
+ return ret ? -1 : 1;
+}
+
+
+static void
+binlog_discover_init(uint64_t file_no, uint64_t interval)
+{
+ active_binlog_file_no.store(file_no, std::memory_order_release);
+ ibb_file_hash.earliest_oob_ref.store(file_no, std::memory_order_relaxed);
+ current_binlog_state_interval= interval;
+ ibb_pending_lsn_fifo.init(file_no);
+}
+
+
+/**
+ Returns:
+ -1 error
+ 0 No binlogs found
+ 1 Just one binlog file found
+ 2 Found two (or more) existing binlog files
+*/
+static int
+innodb_binlog_discover()
+{
+ uint64_t file_no;
+ const uint32_t page_size= (uint32_t)ibb_page_size;
+ const uint32_t page_size_shift= (uint32_t)ibb_page_size_shift;
+ struct found_binlogs binlog_files;
+ binlog_header_data header;
+
+ int res= scan_for_binlogs(innodb_binlog_directory, &binlog_files, false);
+ if (res <= 0)
+ {
+ if (res == 0)
+ ibb_pending_lsn_fifo.init(0);
+ return res;
+ }
+
+ /*
+ Now, if we found any binlog files, locate the point in one of them where
+ binlogging stopped, and where we should continue writing new binlog data.
+ */
+ uint32_t page_no, prev_page_no, pos_in_page, prev_pos_in_page;
+ std::unique_ptr
+ page_buf(static_cast(aligned_malloc(page_size, page_size)),
+ &aligned_free);
+ if (!page_buf)
+ return -1;
+ if (binlog_files.num_found >= 1) {
+ earliest_binlog_file_no= binlog_files.earliest_file_no;
+ total_binlog_used_size= binlog_files.total_size;
+
+ res= find_pos_in_binlog(binlog_files.last_file_no,
+ binlog_files.last_size,
+ page_buf.get(), &page_no, &pos_in_page,
+ &header);
+ if (res < 0) {
+ file_no= binlog_files.last_file_no;
+ if (ibb_record_in_file_hash(file_no, ~(uint64_t)0, ~(uint64_t)0))
+ return -1;
+ binlog_discover_init(file_no, innodb_binlog_state_interval);
+ sql_print_warning("Binlog number %llu could no be opened. Starting a new "
+ "binlog file from number %llu",
+ binlog_files.last_file_no, (file_no + 1));
+ return 0;
+ }
+
+ if (res > 0) {
+ /* Found start position in the last binlog file. */
+ file_no= binlog_files.last_file_no;
+ if (ibb_record_in_file_hash(file_no, header.oob_ref_file_no,
+ header.xa_ref_file_no))
+ return -1;
+ binlog_discover_init(file_no, header.diff_state_interval);
+ binlog_cur_page_no= page_no;
+ binlog_cur_page_offset= pos_in_page;
+ sql_print_information("InnoDB: Continuing binlog number %" PRIu64
+ " from position %" PRIu64 ".", file_no,
+ (((uint64_t)page_no << page_size_shift)
+ | pos_in_page));
+ return binlog_files.num_found;
+ }
+
+ /* res == 0, the last binlog is empty. */
+ if (ibb_record_in_file_hash(binlog_files.last_file_no,
+ ~(uint64_t)0, ~(uint64_t)0))
+ return -1;
+ if (binlog_files.num_found >= 2) {
+ /* The last binlog is empty, try the previous one. */
+ res= find_pos_in_binlog(binlog_files.prev_file_no,
+ binlog_files.prev_size,
+ page_buf.get(),
+ &prev_page_no, &prev_pos_in_page,
+ &header);
+ if (res < 0) {
+ file_no= binlog_files.last_file_no;
+ if (ibb_record_in_file_hash(file_no, ~(uint64_t)0, ~(uint64_t)0))
+ return -1;
+ binlog_discover_init(file_no, innodb_binlog_state_interval);
+ binlog_cur_page_no= page_no;
+ binlog_cur_page_offset= pos_in_page;
+ sql_print_warning("Binlog number %llu could not be opened, starting "
+ "from binlog number %llu instead",
+ binlog_files.prev_file_no, file_no);
+ return 1;
+ }
+ file_no= binlog_files.prev_file_no;
+ if (ibb_record_in_file_hash(file_no, header.oob_ref_file_no,
+ header.xa_ref_file_no))
+ return -1;
+ binlog_discover_init(file_no, header.diff_state_interval);
+ binlog_cur_page_no= prev_page_no;
+ binlog_cur_page_offset= prev_pos_in_page;
+ sql_print_information("InnoDB: Continuing binlog number %" PRIu64
+ " from position %" PRIu64 ".", file_no,
+ (((uint64_t)prev_page_no << page_size_shift) |
+ prev_pos_in_page));
+ return binlog_files.num_found;
+ }
+
+ /* Just one empty binlog file found. */
+ file_no= binlog_files.last_file_no;
+ if (ibb_record_in_file_hash(file_no, ~(uint64_t)0, ~(uint64_t)0))
+ return -1;
+ binlog_discover_init(file_no, innodb_binlog_state_interval);
+ binlog_cur_page_no= page_no;
+ binlog_cur_page_offset= pos_in_page;
+ sql_print_information("InnoDB: Continuing binlog number %" PRIu64 " from "
+ "position %u.", file_no, BINLOG_PAGE_DATA);
+ return binlog_files.num_found;
+ }
+
+ /* No binlog files found, start from scratch. */
+ file_no= 0;
+ earliest_binlog_file_no= 0;
+ ibb_file_hash.earliest_oob_ref.store(0, std::memory_order_relaxed);
+ total_binlog_used_size= 0;
+ ibb_pending_lsn_fifo.init(0);
+ current_binlog_state_interval= innodb_binlog_state_interval;
+ sql_print_information("InnoDB: Starting a new binlog from file number %"
+ PRIu64 ".", file_no);
+ return 0;
+}
+
+
+void innodb_binlog_close(bool shutdown)
+{
+ if (innodb_binlog_inited >= 2)
+ {
+ if (binlog_prealloc_thr_obj.joinable()) {
+ mysql_mutex_lock(&active_binlog_mutex);
+ prealloc_thread_end= true;
+ pthread_cond_signal(&active_binlog_cond);
+ mysql_mutex_unlock(&active_binlog_mutex);
+ binlog_prealloc_thr_obj.join();
+ }
+
+ uint64_t file_no= first_open_binlog_file_no;
+ if (file_no != ~(uint64_t)0) {
+ if (file_no <= last_created_binlog_file_no) {
+ fsp_binlog_tablespace_close(file_no);
+ if (file_no + 1 <= last_created_binlog_file_no) {
+ fsp_binlog_tablespace_close(file_no + 1);
+ }
+ }
+ }
+ }
+
+ if (shutdown && innodb_binlog_inited >= 1)
+ {
+ delete ibb_xa_xid_hash;
+ binlog_diff_state.free();
+ binlog_full_state.free();
+ fsp_binlog_shutdown();
+ mysql_mutex_destroy(&purge_binlog_mutex);
+ }
+}
+
+
+/**
+ Background thread to close old binlog tablespaces and pre-allocate new ones.
+*/
+static void
+innodb_binlog_prealloc_thread()
+{
+ my_thread_init();
+#ifdef UNIV_PFS_THREAD
+ pfs_register_thread(binlog_prealloc_thread_key);
+#endif
+ LF_PINS *lf_pins= lf_hash_get_pins(&ibb_file_hash.hash);
+ ut_a(lf_pins);
+
+ mysql_mutex_lock(&active_binlog_mutex);
+ while (1)
+ {
+ uint64_t active= active_binlog_file_no.load(std::memory_order_relaxed);
+ uint64_t first_open= first_open_binlog_file_no;
+
+ /* Pre-allocate the next tablespace (if not done already). */
+ uint64_t last_created= last_created_binlog_file_no;
+ if (last_created <= active && last_created <= first_open) {
+ ut_ad(last_created == active);
+ ut_ad(last_created == first_open || first_open == ~(uint64_t)0);
+ /*
+ Note: `last_created` is initialized to ~0, so incrementing it here
+ makes us start from binlog file 0.
+ */
+ ++last_created;
+ mysql_mutex_unlock(&active_binlog_mutex);
+
+ mysql_mutex_lock(&purge_binlog_mutex);
+ uint32_t size_in_pages= innodb_binlog_size_in_pages;
+ dberr_t res2= fsp_binlog_tablespace_create(last_created, size_in_pages,
+ lf_pins);
+ if (earliest_binlog_file_no == ~(uint64_t)0)
+ earliest_binlog_file_no= last_created;
+ total_binlog_used_size+= (size_in_pages << ibb_page_size_shift);
+
+ innodb_binlog_autopurge(first_open, lf_pins);
+ mysql_mutex_unlock(&purge_binlog_mutex);
+
+ mysql_mutex_lock(&active_binlog_mutex);
+ ut_a(res2 == DB_SUCCESS);
+ last_created_binlog_file_no= last_created;
+
+ /* If we created the initial tablespace file, make it the active one. */
+ ut_ad(active < ~(uint64_t)0 || last_created == 0);
+ if (active == ~(uint64_t)0) {
+ binlog_cur_end_offset[last_created & 3].
+ store(0, std::memory_order_release);
+ binlog_cur_durable_offset[last_created & 3]
+ .store(0, std::memory_order_release);
+ active_binlog_file_no.store(last_created, std::memory_order_relaxed);
+ ibb_file_hash.earliest_oob_ref.store(last_created,
+ std::memory_order_relaxed);
+ }
+ if (first_open == ~(uint64_t)0)
+ first_open_binlog_file_no= first_open= last_created;
+
+ pthread_cond_signal(&active_binlog_cond);
+ continue; /* Re-start loop after releasing/reacquiring mutex. */
+ }
+
+ /*
+ Flush out to disk and close any binlog tablespace that has been
+ completely written.
+ */
+ if (first_open < active) {
+ ut_ad(first_open == active - 1);
+ mysql_mutex_unlock(&active_binlog_mutex);
+ fsp_binlog_tablespace_close(active - 1);
+ mysql_mutex_lock(&active_binlog_mutex);
+ first_open_binlog_file_no= first_open + 1;
+ continue; /* Re-start loop after releasing/reacquiring mutex. */
+ }
+
+ /* Exit thread at server shutdown. */
+ if (prealloc_thread_end)
+ break;
+ my_cond_wait(&active_binlog_cond, &active_binlog_mutex.m_mutex);
+
+ }
+ mysql_mutex_unlock(&active_binlog_mutex);
+
+ lf_hash_put_pins(lf_pins);
+ my_thread_end();
+
+#ifdef UNIV_PFS_THREAD
+ pfs_delete_thread();
+#endif
+}
+
+
+bool
+ibb_write_header_page(mtr_t *mtr, uint64_t file_no, uint64_t file_size_in_pages,
+ lsn_t start_lsn, uint64_t gtid_state_interval_in_pages,
+ LF_PINS *pins)
+{
+ fsp_binlog_page_entry *block;
+ uint32_t used_bytes;
+
+ block= binlog_page_fifo->create_page(file_no, 0);
+ if (UNIV_UNLIKELY(!block))
+ return true;
+ byte *ptr= &block->page_buf()[0];
+ uint64_t oob_ref_file_no=
+ ibb_file_hash.earliest_oob_ref.load(std::memory_order_relaxed);
+ uint64_t xa_ref_file_no=
+ ibb_file_hash.earliest_xa_ref.load(std::memory_order_relaxed);
+ ibb_file_hash.update_refs(file_no, pins, oob_ref_file_no, xa_ref_file_no);
+
+ int4store(ptr, IBB_MAGIC);
+ int4store(ptr + 4, ibb_page_size_shift);
+ int4store(ptr + 8, IBB_FILE_VERS_MAJOR);
+ int4store(ptr + 12, IBB_FILE_VERS_MINOR);
+ int8store(ptr + 16, file_no);
+ int8store(ptr + 24, file_size_in_pages);
+ int8store(ptr + 32, start_lsn);
+ int8store(ptr + 40, gtid_state_interval_in_pages);
+ int8store(ptr + 48, oob_ref_file_no);
+ int8store(ptr + 56, xa_ref_file_no);
+ used_bytes= IBB_BINLOG_HEADER_SIZE;
+ ut_ad(ibb_page_size >= IBB_HEADER_PAGE_SIZE);
+ memset(ptr + used_bytes, 0, ibb_page_size - (used_bytes + BINLOG_PAGE_CHECKSUM));
+ /*
+ For future expansion with configurable page size:
+ Write a CRC32 at the end of the minimal page size. This way, the header
+ page can be read and checksummed without knowing the page size used in
+ the file, and then the actual page size can be obtained from the header
+ page.
+ */
+ const uint32_t payload= IBB_HEADER_PAGE_SIZE - BINLOG_PAGE_CHECKSUM;
+ int4store(ptr + payload, my_crc32c(0, ptr, payload));
+
+ fsp_log_header_page(mtr, block, file_no, used_bytes);
+ binlog_page_fifo->release_page_mtr(block, mtr);
+
+ return false; // No error
+}
+
+
+__attribute__((noinline))
+static ssize_t
+serialize_gtid_state(rpl_binlog_state_base *state, byte *buf, size_t buf_size)
+ noexcept
+{
+ unsigned char *p= (unsigned char *)buf;
+ /*
+ 1 uint64_t for the number of entries in the state stored.
+ 1 uint64_t for the XA references file_no.
+ 2 uint32_t + 1 uint64_t for at least one GTID.
+ */
+ ut_ad(buf_size >= 2*COMPR_INT_MAX32 + 3*COMPR_INT_MAX64);
+ p= compr_int_write(p, state->count_nolock());
+ uint64_t xa_ref_file_no=
+ ibb_file_hash.earliest_xa_ref.load(std::memory_order_relaxed);
+ /* Write 1 +file_no, so that 0 (1 + ~0) means "no reference". */
+ p= compr_int_write(p, xa_ref_file_no + 1);
+ unsigned char * const pmax=
+ p + (buf_size - (2*COMPR_INT_MAX32 + COMPR_INT_MAX64));
+
+ if (state->iterate(
+ [pmax, &p] (const rpl_gtid *gtid) {
+ if (UNIV_UNLIKELY(p > pmax))
+ return true;
+ p= compr_int_write(p, gtid->domain_id);
+ p= compr_int_write(p, gtid->server_id);
+ p= compr_int_write(p, gtid->seq_no);
+ return false;
+ }))
+ return -1;
+ else
+ return p - (unsigned char *)buf;
+}
+
+
+bool
+binlog_gtid_state(rpl_binlog_state_base *state, mtr_t *mtr,
+ fsp_binlog_page_entry * &block, uint32_t &page_no,
+ uint32_t &page_offset, uint64_t file_no)
+{
+ /*
+ Use a small, efficient stack-allocated buffer by default, falling back to
+ malloc() if needed for large GTID state.
+ */
+ byte small_buf[192];
+ byte *buf, *alloced_buf;
+ uint32_t block_page_no= ~(uint32_t)0;
+ block= nullptr;
+
+ ssize_t used_bytes= serialize_gtid_state(state, small_buf, sizeof(small_buf));
+ if (used_bytes >= 0)
+ {
+ buf= small_buf;
+ alloced_buf= nullptr;
+ }
+ else
+ {
+ size_t buf_size= 2*COMPR_INT_MAX64 +
+ state->count_nolock() * (2*COMPR_INT_MAX32 + COMPR_INT_MAX64);
+ alloced_buf= static_cast(ut_malloc(buf_size, mem_key_binlog));
+ if (UNIV_UNLIKELY(!alloced_buf))
+ return true;
+ buf= alloced_buf;
+ used_bytes= serialize_gtid_state(state, buf, buf_size);
+ if (UNIV_UNLIKELY(used_bytes < 0))
+ {
+ ut_ad(0 /* Shouldn't happen, as we allocated maximum needed size. */);
+ ut_free(alloced_buf);
+ return true;
+ }
+ }
+
+ const uint32_t page_size= (uint32_t)ibb_page_size;
+ const uint32_t page_room= page_size - (BINLOG_PAGE_DATA + BINLOG_PAGE_DATA_END);
+ uint32_t needed_pages= (uint32_t)((used_bytes + page_room - 1) / page_room);
+
+ /* For now, GTID state always at the start of a page. */
+ ut_ad(page_offset == BINLOG_PAGE_DATA);
+ /* Page 0 is reserved for the header page. */
+ ut_ad(page_no != 0);
+
+ /*
+ Only write the GTID state record if there is room for actual event data
+ afterwards. There is no point in using space to allow fast search to a
+ point if there is no data to search for after that point.
+ */
+ if (page_no + needed_pages < binlog_page_fifo->size_in_pages(file_no))
+ {
+ byte cont_flag= 0;
+ while (used_bytes > 0)
+ {
+ ut_ad(page_no < binlog_page_fifo->size_in_pages(file_no));
+ if (block)
+ binlog_page_fifo->release_page_mtr(block, mtr);
+ block_page_no= page_no;
+ block= binlog_page_fifo->create_page(file_no, block_page_no);
+ if (UNIV_UNLIKELY(!block))
+ return true;
+ page_offset= BINLOG_PAGE_DATA;
+ byte *ptr= page_offset + &block->page_buf()[0];
+ uint32_t chunk= (uint32_t)used_bytes;
+ byte last_flag= FSP_BINLOG_FLAG_LAST;
+ if (chunk > page_room - 3) {
+ last_flag= 0;
+ chunk= page_room - 3;
+ ++page_no;
+ }
+ ptr[0]= FSP_BINLOG_TYPE_GTID_STATE | cont_flag | last_flag;
+ ptr[1] = (byte)chunk & 0xff;
+ ptr[2] = (byte)(chunk >> 8);
+ ut_ad(chunk <= 0xffff);
+ memcpy(ptr+3, buf, chunk);
+ fsp_log_binlog_write(mtr, block, file_no, block_page_no, page_offset,
+ (uint32)(chunk+3));
+ page_offset+= chunk + 3;
+ buf+= chunk;
+ used_bytes-= chunk;
+ cont_flag= FSP_BINLOG_FLAG_CONT;
+ }
+
+ if (page_offset == page_size - BINLOG_PAGE_DATA_END) {
+ if (block)
+ binlog_page_fifo->release_page_mtr(block, mtr);
+ block= nullptr;
+ block_page_no= ~(uint32_t)0;
+ page_offset= BINLOG_PAGE_DATA;
+ ++page_no;
+ }
+ }
+ ut_free(alloced_buf);
+
+ /* Make sure we return a page for caller to write the main event data into. */
+ if (UNIV_UNLIKELY(!block)) {
+ block= binlog_page_fifo->create_page(file_no, page_no);
+ if (UNIV_UNLIKELY(!block))
+ return true;
+ }
+
+ return false; // No error
+}
+
+
+/**
+ Read a binlog state record. The passed in STATE object is updated with the
+ state read.
+
+ Returns:
+ 1 State record found
+ 0 No state record found
+ -1 Error
+*/
+static int
+read_gtid_state(binlog_chunk_reader *chunk_reader,
+ rpl_binlog_state_base *state,
+ uint64_t *out_xa_ref_file_no) noexcept
+{
+ byte buf[256];
+ static_assert(sizeof(buf) >= 2*COMPR_INT_MAX64 + 6*COMPR_INT_MAX64,
+ "buf must hold at least 2 GTIDs");
+ int res= chunk_reader->read_data(buf, sizeof(buf), true);
+ if (UNIV_UNLIKELY(res < 0))
+ return -1;
+ if (res == 0 || chunk_reader->cur_type() != FSP_BINLOG_TYPE_GTID_STATE)
+ return 0;
+ const byte *p= buf;
+ const byte *p_end= buf + res;
+
+ /* Read the number of GTIDs in the gtid state record. */
+ std::pair v_and_p= compr_int_read(buf);
+ p= v_and_p.second;
+ if (UNIV_UNLIKELY(p > p_end))
+ return -1;
+ uint64_t num_gtid= v_and_p.first;
+ /*
+ Read the earliest file_no containing pending XA if any.
+ Note that unsigned underflow means 0 - 1 becomes ~0, as required.
+ */
+ v_and_p= compr_int_read(p);
+ p= v_and_p.second;
+ if (UNIV_UNLIKELY(p > p_end))
+ return -1;
+ *out_xa_ref_file_no= v_and_p.first - 1;
+
+ /* Read each GTID one by one and add into the state. */
+ for (uint64_t count= num_gtid; count > 0; --count)
+ {
+ ptrdiff_t remain= p_end - p;
+ /* Read more data as needed to ensure we have read a full GTID. */
+ if (UNIV_UNLIKELY(!chunk_reader->end_of_record()) &&
+ UNIV_UNLIKELY(remain < 3*COMPR_INT_MAX64))
+ {
+ memmove(buf, p, remain);
+ res= chunk_reader->read_data(buf + remain, (int)(sizeof(buf) - remain),
+ true);
+ if (UNIV_UNLIKELY(res < 0))
+ return -1;
+ p= buf;
+ p_end= p + remain + res;
+ remain+= res;
+ }
+ rpl_gtid gtid;
+ if (UNIV_UNLIKELY(p >= p_end))
+ return -1;
+ v_and_p= compr_int_read(p);
+ if (UNIV_UNLIKELY(v_and_p.first > UINT32_MAX))
+ return -1;
+ gtid.domain_id= (uint32_t)v_and_p.first;
+ p= v_and_p.second;
+ if (UNIV_UNLIKELY(p >= p_end))
+ return -1;
+ v_and_p= compr_int_read(p);
+ if (UNIV_UNLIKELY(v_and_p.first > UINT32_MAX))
+ return -1;
+ gtid.server_id= (uint32_t)v_and_p.first;
+ p= v_and_p.second;
+ if (UNIV_UNLIKELY(p >= p_end))
+ return -1;
+ v_and_p= compr_int_read(p);
+ gtid.seq_no= v_and_p.first;
+ p= v_and_p.second;
+ if (UNIV_UNLIKELY(p > p_end))
+ return -1;
+ if (state->update_nolock(>id))
+ return -1;
+ }
+
+ /*
+ For now, we expect no more data.
+ Later it could be extended, as we store (and read) the count of GTIDs.
+ */
+ ut_ad(p == p_end);
+
+ return 1;
+}
+
+
+/**
+ Recover the GTID binlog state at startup.
+ Read the full binlog state at the start of the current binlog file, as well
+ as the last differential binlog state on top, if any. Then scan from there to
+ the end to obtain the exact current GTID binlog state.
+
+ Return false if ok, true if error.
+*/
+static bool
+binlog_state_recover(uint64_t *out_xa_file_no, uint64_t *out_xa_offset)
+{
+ uint64_t active= active_binlog_file_no.load(std::memory_order_relaxed);
+ uint64_t diff_state_interval= current_binlog_state_interval;
+ uint32_t page_no= 1;
+
+ *out_xa_file_no= earliest_binlog_file_no;
+ *out_xa_offset= (uint64_t)1 << ibb_page_size_shift;
+ binlog_chunk_reader chunk_reader(binlog_cur_end_offset);
+ byte *page_buf=
+ static_cast(ut_malloc(ibb_page_size, mem_key_binlog));
+ if (!page_buf)
+ return true;
+ chunk_reader.set_page_buf(page_buf);
+ *out_xa_offset= page_no << ibb_page_size_shift;
+ chunk_reader.seek(active, *out_xa_offset);
+ int res= read_gtid_state(&chunk_reader, &binlog_full_state, out_xa_file_no);
+ if (res < 0)
+ {
+ ut_free(page_buf);
+ return true;
+ }
+ if (diff_state_interval == 0)
+ {
+ sql_print_warning("Invalid differential binlog state interval %llu found "
+ "in binlog file, ignoring", diff_state_interval);
+ }
+ else
+ {
+ page_no= (uint32_t)(binlog_cur_page_no -
+ (binlog_cur_page_no % diff_state_interval));
+ while (page_no > 1)
+ {
+ *out_xa_offset= page_no << ibb_page_size_shift;
+ chunk_reader.seek(active, *out_xa_offset);
+ chunk_reader.skip_partial(true);
+ res= read_gtid_state(&chunk_reader, &binlog_full_state, out_xa_file_no);
+ if (res > 0)
+ break;
+ page_no-= (uint32_t)diff_state_interval;
+ }
+ }
+ ut_free(page_buf);
+ if (UNIV_LIKELY(*out_xa_file_no == ~(uint64_t)0))
+ {
+ /*
+ If there were no XID references active at the last state record written,
+ then recovery only needs to scan from that point on.
+ */
+ *out_xa_file_no= active;
+ }
+
+ ha_innodb_binlog_reader reader(false, active,
+ page_no << ibb_page_size_shift);
+ return binlog_recover_gtid_state(&binlog_full_state, &reader);
+}
+
+
+static bool
+ibb_recv_record_update(HASH *hash, ibb_binlog_xid_info *info, uint64_t file_no)
+{
+ /* Delete any existing entry for this XID before inserting the newer one. */
+ size_t key_len= 0;
+ const uchar *key_ptr= info->get_key(info, &key_len, 1);
+ uchar *rec= my_hash_search(hash, key_ptr, key_len);
+ if (rec != nullptr)
+ my_hash_delete(hash, rec);
+ if (my_hash_insert(hash, (const uchar *)info))
+ {
+ sql_print_error("InnoDB: Out of memory while scanning file_no=%" PRIu64,
+ file_no);
+ delete info;
+ return true;
+ }
+ return false;
+}
+
+
+static bool
+ibb_recv_record_prepare(HASH *hash, uint64_t file_no,
+ const byte *rec_data, int data_len)
+{
+ const byte *p= rec_data;
+
+ uchar engine_count= *p++;
+ long formatID= uint4korr(p);
+ p+= 4;
+ byte gtrid_length= *p++;
+ byte bqual_length= *p++;
+ if (UNIV_UNLIKELY(gtrid_length > 64) ||
+ UNIV_UNLIKELY(bqual_length > 64))
+ {
+ sql_print_error("InnoDB: Corrupt prepare record found in file_no=%" PRIu64
+ ", invalid XID lengths %u/%u", file_no,
+ (uint)gtrid_length, (uint)bqual_length);
+ return true;
+ }
+ const char *xid_data= reinterpret_cast(p);
+ p+= gtrid_length + bqual_length;
+ std::pair v_and_p;
+ v_and_p= compr_int_read(p);
+ uint64_t num_oob_nodes= v_and_p.first;
+ p= v_and_p.second;
+ v_and_p= compr_int_read(p);
+ uint64_t first_oob_file_no= v_and_p.first;
+ p= v_and_p.second;
+ v_and_p= compr_int_read(p);
+ uint64_t first_oob_offset= v_and_p.first;
+ p= v_and_p.second;
+ v_and_p= compr_int_read(p);
+ uint64_t last_oob_file_no= v_and_p.first;
+ p= v_and_p.second;
+ v_and_p= compr_int_read(p);
+ uint64_t last_oob_offset= v_and_p.first;
+ p= v_and_p.second;
+ if ((int)(p - rec_data) > data_len)
+ {
+ sql_print_error("InnoDB: Corrupt prepare record found in file_no=%" PRIu64
+ ", only %d bytes but expected %u", file_no,
+ data_len, (uint)(p - rec_data));
+ return true;
+ }
+ ibb_binlog_xid_info *xid_info=
+ new ibb_binlog_xid_info(handler_binlog_xid_info::BINLOG_PREPARE, file_no);
+ if (!xid_info)
+ {
+ sql_print_error("InnoDB: Out of memory while scanning file_no=%" PRIu64,
+ file_no);
+ return true;
+ }
+ xid_info->xid.set(formatID, xid_data, gtrid_length,
+ xid_data + bqual_length, bqual_length);
+ xid_info->engine_count= engine_count;
+ xid_info->num_oob_nodes= num_oob_nodes;
+ xid_info->first_oob_file_no= first_oob_file_no;
+ xid_info->first_oob_offset= first_oob_offset;
+ xid_info->last_oob_file_no= last_oob_file_no;
+ xid_info->last_oob_offset= last_oob_offset;
+ if (ibb_recv_record_update(hash, xid_info, file_no))
+ return true;
+
+ return false;
+}
+
+
+static bool
+ibb_recv_record_complete(HASH *hash, uint64_t file_no,
+ const byte *rec_data, int data_len)
+{
+ const byte *p= rec_data;
+
+ byte subtype= *p++;
+ bool is_commit= (subtype & IBB_FL_XA_TYPE_MASK) == IBB_FL_XA_TYPE_COMMIT;
+ long formatID= uint4korr(p);
+ p+= 4;
+ byte gtrid_length= *p++;
+ byte bqual_length= *p++;
+ if (UNIV_UNLIKELY(gtrid_length > 64) ||
+ UNIV_UNLIKELY(bqual_length > 64))
+ {
+ sql_print_error("InnoDB: Corrupt %s record found in file_no=%" PRIu64
+ ", invalid XID lengths %u/%u",
+ (is_commit ? "commit" : "rollback"), file_no,
+ (uint)gtrid_length, (uint)bqual_length);
+ return true;
+ }
+ const char *xid_data= reinterpret_cast(p);
+ p+= gtrid_length + bqual_length;
+ if ((int)(p - rec_data) > data_len)
+ {
+ sql_print_error("InnoDB: Corrupt prepare record found in file_no=%" PRIu64
+ ", only %d bytes but expected %u", file_no,
+ data_len, (uint)(p - rec_data));
+ return true;
+ }
+ handler_binlog_xid_info::binlog_xid_state xid_state= is_commit ?
+ handler_binlog_xid_info::BINLOG_COMMIT :
+ handler_binlog_xid_info::BINLOG_ROLLBACK;
+ ibb_binlog_xid_info *xid_info= new ibb_binlog_xid_info(xid_state, file_no);
+ if (!xid_info)
+ {
+ sql_print_error("InnoDB: Out of memory while scanning file_no=%" PRIu64,
+ file_no);
+ return true;
+ }
+ xid_info->xid.set(formatID, xid_data, gtrid_length,
+ xid_data + bqual_length, bqual_length);
+ if (ibb_recv_record_update(hash, xid_info, file_no))
+ return true;
+ return false;
+}
+
+
+static bool
+binlog_scan_for_xid(uint64_t start_file_no, uint64_t start_offset,
+ HASH *hash)
+{
+ if (start_file_no == ~(uint64_t)0)
+ return false; // No active XID, no scan needed.
+ binlog_chunk_reader chunk_reader(binlog_cur_end_offset);
+ std::unique_ptr
+ page_buf(static_cast(ut_malloc(ibb_page_size, mem_key_binlog)),
+ [](byte *p) {ut_free(p);});
+ if (page_buf == nullptr)
+ return true;
+ chunk_reader.set_page_buf(page_buf.get());
+ chunk_reader.seek(start_file_no, start_offset);
+ chunk_reader.skip_partial(true);
+
+ byte buf[1024];
+ static_assert(sizeof(buf) >= ibb_prepare_record_max_size,
+ "Need space for max size prepare record");
+ for (;;)
+ {
+ int res= chunk_reader.read_data(buf, sizeof(buf), true);
+ if (res < 0)
+ {
+ sql_print_error("InnoDB: Error reading binlog while recovering XIDs of "
+ "possibly prepared transactions. Recovery will be "
+ "incomplete.");
+ break;
+ }
+ if (res == 0)
+ {
+ /* EOF, so scan is done. */
+ return false;
+ }
+ if (chunk_reader.cur_type() == FSP_BINLOG_TYPE_XA_PREPARE)
+ {
+ if (ibb_recv_record_prepare(hash, chunk_reader.s.rec_start_file_no,
+ buf, res))
+ return true;
+ }
+ else if (chunk_reader.cur_type() == FSP_BINLOG_TYPE_XA_COMPLETE)
+ {
+ if (ibb_recv_record_complete(hash, chunk_reader.s.rec_start_file_no,
+ buf, res))
+ return true;
+ }
+ else
+ {
+ /* Skip any other record type. */
+ chunk_reader.skip_current();
+ }
+ }
+
+ return false;
+}
+
+
+static bool
+ibb_init_xid_hash(HASH *hash, LF_PINS *pins)
+{
+ /*
+ Populate our internal XID hash from any prepare records found
+ while scanning the binlogs.
+ */
+ for (uint32 i= 0; i < hash->records; ++i)
+ {
+ const ibb_binlog_xid_info *info= (const ibb_binlog_xid_info *)
+ my_hash_element(hash, i);
+ if (info->xid_state != handler_binlog_xid_info::BINLOG_PREPARE)
+ continue;
+
+ uint64_t oob_file_no= info->num_oob_nodes > 0 ?
+ info->first_oob_file_no : info->rec_file_no;
+ /*
+ This is just to ensure that we load the file header page into the
+ ibb_file_hash if not there already.
+ */
+ uint64_t dummy;
+ if (ibb_file_hash.get_oob_ref_file_no(oob_file_no, pins, &dummy))
+ {
+ sql_print_error("InnoDB: Could not process file number %" PRIu64
+ " while recovering pending XID from existing binlogs, "
+ "out of memory or unable to read file", oob_file_no);
+ return true;
+ }
+
+ if (ibb_xa_xid_hash->add_xid(&info->xid, oob_file_no, pins,
+ info->num_oob_nodes,
+ info->first_oob_file_no,
+ info->first_oob_offset,
+ info->last_oob_file_no,
+ info->last_oob_offset))
+ {
+ fprintf(stderr, "InnoDB: Out of memory while recovering pending "
+ "XID from existing binlogs");
+ return true;
+ }
+ }
+ return false;
+}
+
+
+/** Allocate a context for out-of-band binlogging. */
+static binlog_oob_context *
+alloc_oob_context(uint32 list_length= 10)
+{
+ size_t needed= sizeof(binlog_oob_context) +
+ list_length * sizeof(binlog_oob_context::node_info);
+ binlog_oob_context *c=
+ static_cast(ut_malloc(needed, mem_key_binlog));
+ if (c)
+ {
+ if (!(c->lf_pins= lf_hash_get_pins(&ibb_file_hash.hash)))
+ {
+ my_error(ER_OUT_OF_RESOURCES, MYF(0));
+ ut_free(c);
+ return nullptr;
+ }
+ c->stmt_start_point= nullptr;
+ c->savepoint_stack= nullptr;
+ c->pending_file_no= ~(uint64_t)0;
+ c->node_list_alloc_len= list_length;
+ c->node_list_len= 0;
+ c->secondary_ctx= nullptr;
+ c->pending_refcount= false;
+ c->is_xa_prepared= false;
+ }
+ else
+ my_error(ER_OUTOFMEMORY, MYF(0), needed);
+
+ return c;
+}
+
+
+static void
+innodb_binlog_write_cache(IO_CACHE *cache, const rpl_gtid *gtid,
+ handler_binlog_event_group_info *binlog_info, mtr_t *mtr)
+{
+ binlog_oob_context *c=
+ static_cast(binlog_info->engine_ptr);
+ if (!c)
+ binlog_info->engine_ptr= c= alloc_oob_context();
+ ut_a(c);
+
+ if (unlikely(binlog_info->xa_xid))
+ {
+ /*
+ Write an XID commit record just before the main commit record.
+ The XID commit record just contains the XID, and is used by binlog XA
+ crash recovery to ensure than the other storage engine(s) that are part
+ of the transaciton commit or rollback consistently with the binlog
+ engine.
+ */
+ chunk_data_xa_complete chunk_data2(binlog_info->xa_xid, true);
+ fsp_binlog_write_rec(&chunk_data2, mtr, FSP_BINLOG_TYPE_XA_COMPLETE,
+ c->lf_pins);
+ }
+
+ chunk_data_cache chunk_data(cache, gtid, binlog_info);
+
+ fsp_binlog_write_rec(&chunk_data, mtr, FSP_BINLOG_TYPE_COMMIT, c->lf_pins);
+ chunk_data.after_copy_data();
+
+ uint64_t file_no= active_binlog_file_no.load(std::memory_order_relaxed);
+ c->pending_file_no= file_no;
+ c->pending_offset=
+ binlog_cur_end_offset[file_no & 3].load(std::memory_order_relaxed);
+}
+
+
+static inline void
+reset_oob_context(binlog_oob_context *c)
+{
+ if (c->stmt_start_point)
+ c->stmt_start_point->node_list_len= 0;
+ while (c->savepoint_stack != nullptr)
+ {
+ binlog_oob_context::savepoint *next_savepoint= c->savepoint_stack->next;
+ ut_free(c->savepoint_stack);
+ c->savepoint_stack= next_savepoint;
+ }
+ c->pending_file_no= ~(uint64_t)0;
+ if (c->pending_refcount)
+ {
+ ibb_file_hash.oob_ref_dec(c->first_node_file_no, c->lf_pins);
+ c->pending_refcount= false;
+ }
+ c->node_list_len= 0;
+ c->secondary_ctx= nullptr;
+ c->is_xa_prepared= false;
+}
+
+
+static inline void
+free_oob_context(binlog_oob_context *c)
+{
+ ut_ad(!c->pending_refcount /* Should not have pending until free */);
+ reset_oob_context(c); /* Defensive programming, should be redundant */
+ ut_free(c->stmt_start_point);
+ lf_hash_put_pins(c->lf_pins);
+ ut_free(c);
+}
+
+
+static binlog_oob_context *
+ensure_oob_context(void **engine_data, uint32_t needed_len)
+{
+ binlog_oob_context *c= static_cast(*engine_data);
+ if (c->node_list_alloc_len >= needed_len)
+ return c;
+ if (needed_len < c->node_list_alloc_len + 10)
+ needed_len= c->node_list_alloc_len + 10;
+ binlog_oob_context *new_c= alloc_oob_context(needed_len);
+ if (UNIV_UNLIKELY(!new_c))
+ return nullptr;
+ ut_ad(c->node_list_len <= c->node_list_alloc_len);
+ memcpy(new_c, c, sizeof(binlog_oob_context) +
+ c->node_list_len*sizeof(binlog_oob_context::node_info));
+ new_c->node_list_alloc_len= needed_len;
+ *engine_data= new_c;
+ ut_free(c);
+ return new_c;
+}
+
+
+/**
+ Binlog an out-of-band piece of event group data.
+
+ For large transactions, we binlog the data in pieces spread out over the
+ binlog file(s), to avoid a large stall to write large amounts of data during
+ transaction commit, and to avoid having to keep all of the transaction in
+ memory or spill it to temporary file.
+
+ The chunks of data are written out in a binary tree structure, to allow
+ efficiently reading the transaction back in order from start to end. Note
+ that the binlog is written append-only, so we cannot simply link each chunk
+ to the following chunk, as the following chunk is unknown when binlogging the
+ prior chunk. With a binary tree structure, the reader can do a post-order
+ traversal and only need to keep log_2(N) node pointers in-memory at any time.
+
+ A perfect binary tree of height h has 2**h - 1 nodes. At any time during a
+ transaction, the out-of-band data in the binary log for that transaction
+ consists of a forest (eg. a list) of perfect binary trees of strictly
+ decreasing height, except that the last two trees may have the same height.
+ For example, here is how it looks for a transaction where 13 nodes (0-12)
+ have been binlogged out-of-band so far:
+
+ 6
+ _ / \_
+ 2 5 9 12
+ / \ / \ / \ / \
+ 0 1 3 4 7 8 10 11
+
+ In addition to the shown binary tree parent->child pointers, each leaf has a
+ (single) link to the root node of the prior (at the time the leaf was added)
+ tree. In the example this means the following links:
+ 11->10, 10->9, 8->7, 7->6, 4->3, 3->2, 1->0
+ This allows to fully traverse the forest of perfect binary trees starting
+ from the last node (12 in the example). In the example, only 10->9 and 7->6
+ will be needed, but the other links would be needed if the tree had been
+ completed at earlier stages.
+
+ As a new node is added, there are two different cases on how to maintain
+ the binary tree forest structure:
+
+ 1. If the last two trees in the forest have the same height h, then those
+ two trees are replaced by a single tree of height (h+1) with the new
+ node as root and the two trees as left and right child. The number of
+ trees in the forest thus decrease by one.
+
+ 2. Otherwise the new node is added at the end of the forest as a tree of
+ height 1; in this case the forest increases by one tree.
+
+ In both cases, we maintain the invariants that the forest consist of a list
+ of perfect binary trees, and that the heights of the trees are strictly
+ decreasing except that the last two trees can have the same height.
+
+ When a transaction is committed, the commit record contains a pointer to
+ the root node of the last tree in the forest. If the transaction is never
+ committed (explicitly rolled back or lost due to disconnect or server
+ restart or crash), then the out-of-band data is simply left in place; it
+ will be ignored by readers and eventually discarded as the old binlog files
+ are purged.
+*/
+bool
+innodb_binlog_oob_ordered(THD *thd, const unsigned char *data, size_t data_len,
+ void **engine_data, void **stm_start_data,
+ void **savepoint_data)
+{
+ binlog_oob_context *c= static_cast(*engine_data);
+ if (!c)
+ *engine_data= c= alloc_oob_context();
+ if (UNIV_UNLIKELY(!c))
+ return true;
+ if (UNIV_UNLIKELY(c->is_xa_prepared))
+ {
+ my_error(ER_XAER_RMFAIL, MYF(0), "IDLE");
+ return true;
+ }
+
+ if (stm_start_data)
+ {
+ if (c->create_stmt_start_point())
+ return true;
+ *stm_start_data= nullptr; /* We do not need to store any data there. */
+ if (data_len == 0 && !savepoint_data)
+ return false;
+ }
+ if (savepoint_data)
+ {
+ binlog_oob_context::savepoint *sv= c->create_savepoint();
+ if (!sv)
+ return true;
+ *((binlog_oob_context::savepoint **)savepoint_data)= sv;
+ if (data_len == 0)
+ return false;
+ }
+ ut_ad(data_len > 0);
+
+ mtr_t mtr{thd_to_trx(thd)};
+ uint32_t i= c->node_list_len;
+ uint64_t new_idx= i==0 ? 0 : c->node_list[i-1].node_index + 1;
+ if (i >= 2 && c->node_list[i-2].height == c->node_list[i-1].height)
+ {
+ /* Case 1: Replace two trees with a tree rooted in a new node. */
+ binlog_oob_context::chunk_data_oob oob_data
+ (new_idx,
+ c->node_list[i-2].file_no, c->node_list[i-2].offset,
+ c->node_list[i-1].file_no, c->node_list[i-1].offset,
+ static_cast(data), data_len);
+ if (c->binlog_node(i-2, new_idx, i-2, i-1, &oob_data, c->lf_pins, &mtr))
+ return true;
+ c->node_list_len= i - 1;
+ }
+ else if (i > 0)
+ {
+ /* Case 2: Add the new node as a singleton tree. */
+ c= ensure_oob_context(engine_data, i+1);
+ if (!c)
+ return true;
+ binlog_oob_context::chunk_data_oob oob_data
+ (new_idx,
+ 0, 0, /* NULL left child signifies a leaf */
+ c->node_list[i-1].file_no, c->node_list[i-1].offset,
+ static_cast(data), data_len);
+ if (c->binlog_node(i, new_idx, i-1, i-1, &oob_data, c->lf_pins, &mtr))
+ return true;
+ c->node_list_len= i + 1;
+ }
+ else
+ {
+ /* Special case i==0, like case 2 but no prior node to link to. */
+ binlog_oob_context::chunk_data_oob oob_data
+ (new_idx, 0, 0, 0, 0, static_cast(data), data_len);
+ /*
+ Note that we must increment the refcount _before_ binlogging the
+ record. Because if the record ends up spanning two binlog files, the
+ new binlog file must have oob reference back to the start of the OOB
+ record, not to the end of it!
+
+ We do not need any locking around getting the active file_no here; even
+ if active would move we would just have a slightly conservative oob
+ reference in the file header. (Though at this point the server layer
+ is holding a lock anyway that prevents other binlogging to happen
+ concurrently).
+ */
+ uint64_t active= active_binlog_file_no.load(std::memory_order_relaxed);
+ c->pending_refcount=
+ ibb_file_hash.oob_ref_inc(active, c->lf_pins) != ~(uint64_t)0;
+
+ if (c->binlog_node(i, new_idx, ~(uint32_t)0, ~(uint32_t)0, &oob_data,
+ c->lf_pins, &mtr))
+ return true;
+
+ /*
+ Here we could check c->node_list[i].file_no and see if it differs from
+ the active before we did the binlogging; and if so increment the right
+ one and decrement the incorrect one. But it does not seem worthwhile, as
+ this is unlikely/impossible, and it just causes a slightly more
+ conservative OOB reference protection from purge anyway.
+ */
+ c->first_node_file_no= c->node_list[i].file_no;
+ c->first_node_offset= c->node_list[i].offset;
+ c->node_list_len= 1;
+ }
+
+ uint64_t file_no= active_binlog_file_no.load(std::memory_order_relaxed);
+ c->pending_file_no= file_no;
+ c->pending_offset=
+ binlog_cur_end_offset[file_no & 3].load(std::memory_order_relaxed);
+ innodb_binlog_post_commit(&mtr, c);
+ return false;
+}
+
+
+bool
+innodb_binlog_oob(THD *thd, const unsigned char *data, size_t data_len,
+ void **engine_data)
+{
+ binlog_oob_context *c= static_cast(*engine_data);
+ if (UNIV_LIKELY(c != nullptr))
+ ibb_pending_lsn_fifo.record_commit(c);
+ /*
+ Throttle the binlog writing of this transaction, if we are getting close
+ to the capacity of the cyclic InnoDB write-ahead log, so that we do not
+ end up overwriting the head of the log.
+ */
+ log_free_check();
+ return false;
+}
+
+
+/**
+ Binlog a new out-of-band tree node and put it at position `node` in the list
+ of trees. A leaf node is denoted by left and right child being identical (and
+ in this case they point to the root of the prior tree).
+*/
+bool
+binlog_oob_context::binlog_node(uint32_t node, uint64_t new_idx,
+ uint32_t left_node, uint32_t right_node,
+ chunk_data_oob *oob_data, LF_PINS *pins,
+ mtr_t *mtr)
+{
+ uint32_t new_height=
+ left_node == right_node ? 1 : 1 + node_list[left_node].height;
+ mtr->start();
+ std::pair new_file_no_offset=
+ fsp_binlog_write_rec(oob_data, mtr, FSP_BINLOG_TYPE_OOB_DATA, pins);
+ mtr->commit();
+ node_list[node].file_no= new_file_no_offset.first;
+ node_list[node].offset= new_file_no_offset.second;
+ node_list[node].node_index= new_idx;
+ node_list[node].height= new_height;
+ return false;
+}
+
+
+binlog_oob_context::chunk_data_oob::chunk_data_oob(uint64_t idx,
+ uint64_t left_file_no, uint64_t left_offset,
+ uint64_t right_file_no, uint64_t right_offset,
+ const byte *data, size_t data_len)
+ : sofar(0), main_len(data_len), main_data(data)
+{
+ ut_ad(data_len > 0);
+ byte *p= &header_buf[0];
+ p= compr_int_write(p, idx);
+ p= compr_int_write(p, left_file_no);
+ p= compr_int_write(p, left_offset);
+ p= compr_int_write(p, right_file_no);
+ p= compr_int_write(p, right_offset);
+ ut_ad((uint32_t)(p - &header_buf[0]) <= max_buffer);
+ header_len= (uint32_t)(p - &header_buf[0]);
+}
+
+
+std::pair
+binlog_oob_context::chunk_data_oob::copy_data(byte *p, uint32_t max_len)
+{
+ uint32_t size= 0;
+ /* First write header data, if any left. */
+ if (sofar < header_len)
+ {
+ size= std::min(header_len - (uint32_t)sofar, max_len);
+ memcpy(p, header_buf + sofar, size);
+ p+= size;
+ sofar+= size;
+ if (UNIV_UNLIKELY(max_len == size))
+ return {size, sofar == header_len + main_len};
+ max_len-= size;
+ }
+
+ /* Then write the main chunk data. */
+ ut_ad(sofar >= header_len);
+ ut_ad(main_len > 0);
+ uint32_t size2=
+ (uint32_t)std::min(header_len + main_len - sofar, (uint64_t)max_len);
+ memcpy(p, main_data + (sofar - header_len), size2);
+ sofar+= size2;
+ return {size + size2, sofar == header_len + main_len};
+}
+
+
+bool
+binlog_oob_context::create_stmt_start_point()
+{
+ if (!stmt_start_point || node_list_len > stmt_start_point->alloc_len)
+ {
+ ut_free(stmt_start_point);
+ size_t size= sizeof(savepoint) + node_list_len * sizeof(node_info);
+ stmt_start_point=
+ static_cast(ut_malloc(size, mem_key_binlog));
+ if (!stmt_start_point)
+ {
+ my_error(ER_OUTOFMEMORY, MYF(0), size);
+ return true;
+ }
+ stmt_start_point->alloc_len= node_list_len;
+ }
+ stmt_start_point->node_list_len= node_list_len;
+ memcpy(stmt_start_point->node_list, node_list,
+ node_list_len * sizeof(node_info));
+ return false;
+}
+
+
+binlog_oob_context::savepoint *
+binlog_oob_context::create_savepoint()
+{
+ size_t size= sizeof(savepoint) + node_list_len * sizeof(node_info);
+ savepoint *s= static_cast(ut_malloc(size, mem_key_binlog));
+ if (!s)
+ {
+ my_error(ER_OUTOFMEMORY, MYF(0), size);
+ return nullptr;
+ }
+ s->next= savepoint_stack;
+ s->node_list_len= node_list_len;
+ memcpy(s->node_list, node_list, node_list_len * sizeof(node_info));
+ savepoint_stack= s;
+ return s;
+}
+
+
+void
+binlog_oob_context::rollback_to_savepoint(savepoint *savepoint)
+{
+ ut_a(node_list_alloc_len >= savepoint->node_list_len);
+ node_list_len= savepoint->node_list_len;
+ memcpy(node_list, savepoint->node_list,
+ savepoint->node_list_len * sizeof(node_info));
+
+ /* Remove any later savepoints from the stack. */
+ for (;;)
+ {
+ struct savepoint *s= savepoint_stack;
+ ut_ad(s != nullptr /* Should always find the savepoint on the stack. */);
+ if (UNIV_UNLIKELY(!s))
+ break;
+ if (s == savepoint)
+ break;
+ savepoint_stack= s->next;
+ ut_free(s);
+ }
+}
+
+
+void
+binlog_oob_context::rollback_to_stmt_start()
+{
+ ut_a(node_list_alloc_len >= stmt_start_point->node_list_len);
+ node_list_len= stmt_start_point->node_list_len;
+ memcpy(node_list, stmt_start_point->node_list,
+ stmt_start_point->node_list_len * sizeof(node_info));
+}
+
+
+void
+ibb_savepoint_rollback(THD *thd, void **engine_data,
+ void **stmt_start_data, void **savepoint_data)
+{
+ binlog_oob_context *c= static_cast(*engine_data);
+ ut_a(c != nullptr);
+
+ if (stmt_start_data)
+ {
+ ut_ad(savepoint_data == nullptr);
+ c->rollback_to_stmt_start();
+ }
+
+ if (savepoint_data)
+ {
+ ut_ad(stmt_start_data == nullptr);
+ binlog_oob_context::savepoint *savepoint=
+ (binlog_oob_context::savepoint *)*savepoint_data;
+ c->rollback_to_savepoint(savepoint);
+ }
+}
+
+
+void
+innodb_reset_oob(void **engine_data)
+{
+ binlog_oob_context *c= static_cast(*engine_data);
+ if (c)
+ reset_oob_context(c);
+}
+
+
+void
+innodb_free_oob(void *engine_data)
+{
+ free_oob_context(static_cast(engine_data));
+}
+
+
+innodb_binlog_oob_reader::innodb_binlog_oob_reader()
+{
+ /* Nothing. */
+}
+
+
+innodb_binlog_oob_reader::~innodb_binlog_oob_reader()
+{
+ /* Nothing. */
+}
+
+
+void
+innodb_binlog_oob_reader::push_state(enum oob_states state, uint64_t file_no,
+ uint64_t offset, bool is_leftmost)
+{
+ stack_entry new_entry;
+ new_entry.state= state;
+ new_entry.file_no= file_no;
+ new_entry.offset= offset;
+ new_entry.is_leftmost= is_leftmost;
+ stack.emplace_back(std::move(new_entry));
+}
+
+
+void
+innodb_binlog_oob_reader::start_traversal(uint64_t file_no, uint64_t offset)
+{
+ stack.clear();
+ push_state(ST_initial, file_no, offset, true);
+}
+
+
+/**
+ Read from out-of-band event group data.
+
+ Does a state-machine incremental traversal of the forest of perfect binary
+ trees of oob records in the event group. May read just the data available
+ on one page, thus returning less than the requested number of bytes (this
+ is to prefer to inspect each page only once, returning data page-by-page as
+ long as reader asks for at least a full page worth of data).
+*/
+int
+innodb_binlog_oob_reader::read_data(binlog_chunk_reader *chunk_rd,
+ uchar *buf, int len)
+{
+ stack_entry *e;
+ uint64_t chunk_idx;
+ uint64_t left_file_no;
+ uint64_t left_offset;
+ int res;
+ const uchar *p_end;
+ const uchar *p;
+ std::pair v_and_p;
+ int size;
+
+ if (stack.empty())
+ {
+ ut_ad(0 /* Should not call when no more oob data to read. */);
+ return 0;
+ }
+
+again:
+ e= &(stack[stack.size() - 1]);
+ switch (e->state)
+ {
+ case ST_initial:
+ chunk_rd->seek(e->file_no, e->offset);
+ static_assert(sizeof(e->rd_buf) == 5*COMPR_INT_MAX64,
+ "rd_buf size must match code using it");
+ res= chunk_rd->read_data(e->rd_buf, 5*COMPR_INT_MAX64, true);
+ if (res < 0)
+ return -1;
+ if (chunk_rd->cur_type() != FSP_BINLOG_TYPE_OOB_DATA)
+ return chunk_rd->read_error_corruption("Wrong chunk type");
+ if (res == 0)
+ return chunk_rd->read_error_corruption("Unexpected EOF, expected "
+ "oob chunk");
+ e->rd_buf_len= res;
+ p_end= e->rd_buf + res;
+ v_and_p= compr_int_read(e->rd_buf);
+ p= v_and_p.second;
+ if (p > p_end)
+ return chunk_rd->read_error_corruption("Short chunk");
+ chunk_idx= v_and_p.first;
+ (void)chunk_idx;
+
+ v_and_p= compr_int_read(p);
+ p= v_and_p.second;
+ if (p > p_end)
+ return chunk_rd->read_error_corruption("Short chunk");
+ left_file_no= v_and_p.first;
+ v_and_p= compr_int_read(p);
+ p= v_and_p.second;
+ if (p > p_end)
+ return chunk_rd->read_error_corruption("Short chunk");
+ left_offset= v_and_p.first;
+
+ v_and_p= compr_int_read(p);
+ p= v_and_p.second;
+ if (p > p_end)
+ return chunk_rd->read_error_corruption("Short chunk");
+ e->right_file_no= v_and_p.first;
+ v_and_p= compr_int_read(p);
+ p= v_and_p.second;
+ if (p > p_end)
+ return chunk_rd->read_error_corruption("Short chunk");
+ e->right_offset= v_and_p.first;
+ e->rd_buf_sofar= (uint32_t)(p - e->rd_buf);
+ if (left_file_no == 0 && left_offset == 0)
+ {
+ /* Leaf node. */
+ if (e->is_leftmost && !(e->right_file_no == 0 && e->right_offset == 0))
+ {
+ /* Traverse the prior tree(s) in the forst. */
+ e->state= ST_traversing_prior_trees;
+ chunk_rd->save_pos(&e->saved_pos);
+ push_state(ST_initial, e->right_file_no, e->right_offset, true);
+ }
+ else
+ e->state= ST_self;
+ }
+ else
+ {
+ e->state= ST_traversing_left_child;
+ chunk_rd->save_pos(&e->saved_pos);
+ push_state(ST_initial, left_file_no, left_offset, e->is_leftmost);
+ }
+ goto again;
+
+ case ST_traversing_prior_trees:
+ chunk_rd->restore_pos(&e->saved_pos);
+ e->state= ST_self;
+ goto again;
+
+ case ST_traversing_left_child:
+ e->state= ST_traversing_right_child;
+ push_state(ST_initial, e->right_file_no, e->right_offset, false);
+ goto again;
+
+ case ST_traversing_right_child:
+ chunk_rd->restore_pos(&e->saved_pos);
+ e->state= ST_self;
+ goto again;
+
+ case ST_self:
+ size= 0;
+ if (e->rd_buf_len > e->rd_buf_sofar)
+ {
+ /* Use any excess data from when the header was read. */
+ size= std::min((int)(e->rd_buf_len - e->rd_buf_sofar), len);
+ memcpy(buf, e->rd_buf + e->rd_buf_sofar, size);
+ e->rd_buf_sofar+= size;
+ len-= size;
+ buf+= size;
+ }
+
+ if (UNIV_LIKELY(len > 0) && UNIV_LIKELY(!chunk_rd->end_of_record()))
+ {
+ res= chunk_rd->read_data(buf, len, false);
+ if (res < 0)
+ return -1;
+ size+= res;
+ }
+
+ if (chunk_rd->end_of_record())
+ {
+ /* This oob record done, pop the state. */
+ ut_ad(!stack.empty());
+ stack.erase(stack.end() - 1, stack.end());
+ }
+ return size;
+
+ default:
+ ut_ad(0);
+ return -1;
+ }
+}
+
+
+ha_innodb_binlog_reader::ha_innodb_binlog_reader(bool wait_durable,
+ uint64_t file_no,
+ uint64_t offset)
+ : chunk_rd(wait_durable ?
+ binlog_cur_durable_offset : binlog_cur_end_offset),
+ requested_file_no(~(uint64_t)0),
+ rd_buf_len(0), rd_buf_sofar(0), state(ST_read_next_event_group)
+{
+ page_buf= static_cast(ut_malloc(ibb_page_size, mem_key_binlog));
+ chunk_rd.set_page_buf(page_buf);
+ if (offset < ibb_page_size)
+ offset= ibb_page_size;
+ chunk_rd.seek(file_no, offset);
+ chunk_rd.skip_partial(true);
+}
+
+
+ha_innodb_binlog_reader::~ha_innodb_binlog_reader()
+{
+ ut_free(page_buf);
+}
+
+
+/**
+ Read data from current position in binlog.
+
+ If the data is written to disk (visible at the OS level, even if not
+ necessarily fsync()'ed to disk), we can read directly from the file.
+ Otherwise, the data must still be available in the buffer pool and
+ we can read it from there.
+
+ First try a dirty read of current state; if this says the data is available
+ to read from the file, this is safe to do (data cannot become un-written).
+
+ If not, then check if the page is in the buffer pool; if not, then likewise
+ we know it's safe to read from the file directly.
+
+ Finally, do another check of the current state. This will catch the case
+ where we looked for a page in binlog file N, but its tablespace id has been
+ recycled, so we got a page from (N+2) instead. In this case also, we can
+ then read from the real file.
+*/
+int ha_innodb_binlog_reader::read_binlog_data(uchar *buf, uint32_t len)
+{
+ int res= read_data(buf, len);
+ chunk_rd.release(res == 0);
+ cur_file_no= chunk_rd.current_file_no();
+ cur_file_pos= chunk_rd.current_pos();
+ return res;
+}
+
+
+int ha_innodb_binlog_reader::read_data(uchar *buf, uint32_t len)
+{
+ int res;
+ const uchar *p_end;
+ const uchar *p;
+ std::pair v_and_p;
+ int sofar= 0;
+
+again:
+ switch (state)
+ {
+ case ST_read_next_event_group:
+ static_assert(sizeof(rd_buf) == 5*COMPR_INT_MAX64,
+ "rd_buf size must match code using it");
+ res= chunk_rd.read_data(rd_buf, 5*COMPR_INT_MAX64, true);
+ if (res < 0)
+ return res;
+ if (res == 0)
+ return sofar;
+ if (chunk_rd.cur_type() != FSP_BINLOG_TYPE_COMMIT)
+ {
+ chunk_rd.skip_current();
+ goto again;
+ }
+ /* Found the start of a commit record. */
+ chunk_rd.skip_partial(false);
+
+ /* Read the header of the commit record to see if there's any oob data. */
+ rd_buf_len= res;
+ p_end= rd_buf + res;
+ v_and_p= compr_int_read(rd_buf);
+ p= v_and_p.second;
+ if (p > p_end)
+ return chunk_rd.read_error_corruption("Short chunk");
+ oob_count= v_and_p.first;
+ oob_count2= 0;
+
+ if (oob_count > 0)
+ {
+ /* Skip the pointer to first chunk. */
+ v_and_p= compr_int_read(p);
+ p= v_and_p.second;
+ if (p > p_end)
+ return chunk_rd.read_error_corruption("Short chunk");
+ v_and_p= compr_int_read(p);
+ p= v_and_p.second;
+ if (p > p_end)
+ return chunk_rd.read_error_corruption("Short chunk");
+
+ v_and_p= compr_int_read(p);
+ p= v_and_p.second;
+ if (p > p_end)
+ return chunk_rd.read_error_corruption("Short chunk");
+ oob_last_file_no= v_and_p.first;
+ v_and_p= compr_int_read(p);
+ p= v_and_p.second;
+ if (p > p_end)
+ return chunk_rd.read_error_corruption("Short chunk");
+ oob_last_offset= v_and_p.first;
+
+ /* Check for any secondary oob data. */
+ v_and_p= compr_int_read(p);
+ p= v_and_p.second;
+ if (p > p_end)
+ return chunk_rd.read_error_corruption("Short chunk");
+ oob_count2= v_and_p.first;
+
+ if (oob_count2 > 0)
+ {
+ /* Skip the pointer to first chunk. */
+ v_and_p= compr_int_read(p);
+ p= v_and_p.second;
+ if (p > p_end)
+ return chunk_rd.read_error_corruption("Short chunk");
+ v_and_p= compr_int_read(p);
+ p= v_and_p.second;
+ if (p > p_end)
+ return chunk_rd.read_error_corruption("Short chunk");
+
+ v_and_p= compr_int_read(p);
+ p= v_and_p.second;
+ if (p > p_end)
+ return chunk_rd.read_error_corruption("Short chunk");
+ oob_last_file_no2= v_and_p.first;
+ v_and_p= compr_int_read(p);
+ p= v_and_p.second;
+ if (p > p_end)
+ return chunk_rd.read_error_corruption("Short chunk");
+ oob_last_offset2= v_and_p.first;
+ }
+ }
+
+ rd_buf_sofar= (uint32_t)(p - rd_buf);
+ state= ST_read_commit_record;
+ goto again;
+
+ case ST_read_commit_record:
+ if (rd_buf_len > rd_buf_sofar)
+ {
+ /* Use any excess data from when the header was read. */
+ int size= std::min((int)(rd_buf_len - rd_buf_sofar), (int)len);
+ memcpy(buf, rd_buf + rd_buf_sofar, size);
+ rd_buf_sofar+= size;
+ len-= size;
+ buf+= size;
+ sofar+= size;
+ }
+
+ if (UNIV_LIKELY(len > 0) && UNIV_LIKELY(!chunk_rd.end_of_record()))
+ {
+ res= chunk_rd.read_data(buf, len, false);
+ if (res < 0)
+ return -1;
+ len-= res;
+ buf+= res;
+ sofar+= res;
+ }
+
+ if (UNIV_LIKELY(rd_buf_sofar == rd_buf_len) && chunk_rd.end_of_record())
+ {
+ if (oob_count == 0)
+ {
+ state= ST_read_next_event_group;
+ if (len > 0 && !chunk_rd.is_end_of_page())
+ {
+ /*
+ Let us try to read more data from this page. The goal is to read
+ from each page only once, as long as caller passes in a buffer at
+ least as big as our page size. Though commit record header that
+ spans a page boundary or oob records can break this property.
+ */
+ goto again;
+ }
+ }
+ else
+ {
+ oob_reader.start_traversal(oob_last_file_no, oob_last_offset);
+ chunk_rd.save_pos(&saved_commit_pos);
+ state= ST_read_oob_data;
+ }
+ if (sofar == 0)
+ goto again;
+ }
+
+ return sofar;
+
+ case ST_read_oob_data:
+ res= oob_reader.read_data(&chunk_rd, buf, len);
+ if (res < 0)
+ return -1;
+ if (oob_reader.oob_traversal_done())
+ {
+ if (UNIV_UNLIKELY(oob_count2 > 0))
+ {
+ /* Switch over to secondary oob data. */
+ oob_count= oob_count2;
+ oob_count2= 0;
+ oob_last_file_no= oob_last_file_no2;
+ oob_last_offset= oob_last_offset2;
+ oob_reader.start_traversal(oob_last_file_no, oob_last_offset);
+ state= ST_read_oob_data;
+ }
+ else
+ {
+ chunk_rd.restore_pos(&saved_commit_pos);
+ state= ST_read_next_event_group;
+ }
+ }
+ if (UNIV_UNLIKELY(res == 0))
+ {
+ ut_ad(0 /* Should have had oob_traversal_done() last time then. */);
+ if (sofar == 0)
+ goto again;
+ }
+ return sofar + res;
+
+ default:
+ ut_ad(0);
+ return -1;
+ }
+}
+
+
+bool
+ha_innodb_binlog_reader::data_available()
+{
+ if (state != ST_read_next_event_group)
+ return true;
+ return chunk_rd.data_available();
+}
+
+
+bool
+ha_innodb_binlog_reader::wait_available(THD *thd,
+ const struct timespec *abstime)
+{
+ bool is_timeout= false;
+ lsn_t pending_sync_lsn= 0;
+ bool did_enter_cond= false;
+ PSI_stage_info old_stage;
+
+ if (data_available())
+ return false;
+
+ mysql_mutex_lock(&binlog_durable_mutex);
+ for (;;)
+ {
+ /* Process anything that has become durable since we last looked. */
+ lsn_t durable_lsn= log_sys.get_flushed_lsn(std::memory_order_relaxed);
+ ibb_pending_lsn_fifo.process_durable_lsn(durable_lsn);
+
+ /* Check if there is anything more pending to be made durable. */
+ if (!ibb_pending_lsn_fifo.is_empty())
+ {
+ pending_lsn_fifo::entry &e= ibb_pending_lsn_fifo.cur_head();
+ if (durable_lsn < e.lsn)
+ pending_sync_lsn= e.lsn;
+ }
+
+ /*
+ Check if there is data available for us now.
+ As we are holding binlog_durable_mutex, active_binlog_file_no cannot
+ move during this check.
+ */
+ uint64_t cur= active_binlog_file_no.load(std::memory_order_relaxed);
+ uint64_t durable_offset=
+ binlog_cur_durable_offset[cur & 3].load(std::memory_order_relaxed);
+ if (durable_offset == 0 && chunk_rd.s.file_no + 1 == cur)
+ {
+ /*
+ If active has durable position=0, it means the current durable
+ position is somewhere in active-1.
+ */
+ cur= chunk_rd.s.file_no;
+ durable_offset=
+ binlog_cur_durable_offset[cur & 3].load(std::memory_order_relaxed);
+ }
+ if (chunk_rd.is_before_pos(cur, durable_offset))
+ break;
+
+ if (pending_sync_lsn != 0 && ibb_pending_lsn_fifo.flushing_lsn == 0)
+ {
+ /*
+ There is no data available for us now, but there is data that will be
+ available when the InnoDB redo log has been durably flushed to disk.
+ So now we will do such a sync (unless another thread is already doing
+ it), so we can proceed getting more data out.
+ */
+ ibb_pending_lsn_fifo.flushing_lsn= pending_sync_lsn;
+ mysql_mutex_unlock(&binlog_durable_mutex);
+ log_write_up_to(pending_sync_lsn, true);
+ mysql_mutex_lock(&binlog_durable_mutex);
+ ibb_pending_lsn_fifo.flushing_lsn= pending_sync_lsn= 0;
+ /* Need to loop back to repeat all checks, after releasing the mutex. */
+ continue;
+ }
+
+ if (thd && thd_kill_level(thd))
+ break;
+
+ if (thd && !did_enter_cond)
+ {
+ THD_ENTER_COND(thd, &binlog_durable_cond, &binlog_durable_mutex,
+ &stage_master_has_sent_all_binlog_to_slave, &old_stage);
+ did_enter_cond= true;
+ }
+ if (abstime)
+ {
+ int res= mysql_cond_timedwait(&binlog_durable_cond,
+ &binlog_durable_mutex,
+ abstime);
+ if (res == ETIMEDOUT)
+ {
+ is_timeout= true;
+ break;
+ }
+ }
+ else
+ mysql_cond_wait(&binlog_durable_cond, &binlog_durable_mutex);
+ }
+ /*
+ If there is pending binlog data to durably sync to the redo log, but we
+ did not do this sync ourselves, then signal another thread (if any) to
+ wakeup and sync. This is necessary to not lose the sync wakeup signal.
+
+ (We use wake-one rather than wake-all for signalling a pending redo log
+ sync to avoid wakeup-storm).
+ */
+ if (pending_sync_lsn != 0)
+ mysql_cond_signal(&binlog_durable_cond);
+
+ if (did_enter_cond)
+ THD_EXIT_COND(thd, &old_stage);
+ else
+ mysql_mutex_unlock(&binlog_durable_mutex);
+
+ return is_timeout;
+}
+
+
+handler_binlog_reader *
+innodb_get_binlog_reader(bool wait_durable)
+{
+ return new ha_innodb_binlog_reader(wait_durable);
+}
+
+
+gtid_search::gtid_search()
+ : cur_open_file_no(~(uint64_t)0), cur_open_file_length(0),
+ cur_open_file((File)-1)
+{
+ /* Nothing else. */
+}
+
+
+gtid_search::~gtid_search()
+{
+ if (cur_open_file >= (File)0)
+ my_close(cur_open_file, MYF(0));
+}
+
+
+/**
+ Search for a GTID position in the binlog.
+ Find a binlog file_no and an offset into the file that is guaranteed to
+ be before the target position. It can be a bit earlier, that only means a
+ bit more of the binlog needs to be scanned to find the real position.
+
+ Returns:
+ -1 error
+ 0 Position not found (has been purged)
+ 1 Position found
+*/
+int
+gtid_search::find_gtid_pos(slave_connection_state *pos,
+ rpl_binlog_state_base *out_state,
+ uint64_t *out_file_no, uint64_t *out_offset)
+{
+ uint64_t dummy_xa_ref;
+ /*
+ Dirty read, but getting a slightly stale value is no problem, we will just
+ be starting to scan the binlog file at a slightly earlier position than
+ necessary.
+ */
+ uint64_t file_no= active_binlog_file_no.load(std::memory_order_relaxed);
+
+ std::unique_ptr
+ page_buf(static_cast(ut_malloc(ibb_page_size, mem_key_binlog)),
+ [](byte *p) {ut_free(p);});
+ if (page_buf == nullptr)
+ {
+ my_error(ER_OUTOFMEMORY, MYF(0), ibb_page_size);
+ return -1;
+ }
+ binlog_chunk_reader chunk_reader(binlog_cur_durable_offset);
+ chunk_reader.set_page_buf(page_buf.get());
+
+ /* First search backwards for the right file to start from. */
+ uint64_t diff_state_page_interval= 0;
+ rpl_binlog_state_base base_state, page0_diff_state, tmp_diff_state;
+ base_state.init();
+ for (;;)
+ {
+ /* Read the header page, needed to get the binlog diff state interval. */
+ binlog_header_data header;
+ chunk_reader.seek(file_no, 0);
+ int res= chunk_reader.get_file_header(&header);
+ if (UNIV_UNLIKELY(res < 0))
+ return -1;
+ if (UNIV_UNLIKELY(res == 0))
+ goto not_found_in_file;
+ diff_state_page_interval= header.diff_state_interval;
+
+ chunk_reader.seek(file_no, ibb_page_size);
+ res= read_gtid_state(&chunk_reader, &base_state, &dummy_xa_ref);
+ if (UNIV_UNLIKELY(res < 0))
+ return -1;
+ if (res == 0)
+ {
+ not_found_in_file:
+ if (file_no == 0)
+ {
+ /* Handle the special case of a completely empty binlog file. */
+ out_state->reset_nolock();
+ *out_file_no= file_no;
+ *out_offset= ibb_page_size;
+ return 1;
+ }
+ /* If GTID state is not (durably) available, try the previous file. */
+ }
+ else if (base_state.is_before_pos(pos))
+ break;
+ base_state.reset_nolock();
+ if (file_no <= earliest_binlog_file_no)
+ return 0;
+ --file_no;
+ }
+
+ /*
+ Then binary search for the last differential state record that is still
+ before the searched position.
+
+ The invariant is that page2 is known to be after the target page, and page0
+ is known to be a valid position to start (but possibly earlier than needed).
+ */
+ uint32_t page0= 0;
+ uint32_t page2= (uint32_t) (diff_state_page_interval +
+ ((chunk_reader.cur_end_offset - 1) >> ibb_page_size_shift));
+ /* Round to the next diff_state_page_interval after file end. */
+ page2-= page2 % (uint32_t)diff_state_page_interval;
+ uint32_t page1= page0 +
+ ((page2 - page0) /
+ (2*(uint32_t)diff_state_page_interval) *
+ (uint32_t)diff_state_page_interval);
+ page0_diff_state.init();
+ page0_diff_state.load_nolock(&base_state);
+ tmp_diff_state.init();
+ while (page1 >= page0 + diff_state_page_interval && page1 > 1)
+ {
+ ut_ad((page1 - page0) % diff_state_page_interval == 0);
+ tmp_diff_state.reset_nolock();
+ tmp_diff_state.load_nolock(&base_state);
+ chunk_reader.seek(file_no, page1 << ibb_page_size_shift);
+ chunk_reader.skip_partial(true);
+ int res= read_gtid_state(&chunk_reader, &tmp_diff_state, &dummy_xa_ref);
+ if (UNIV_UNLIKELY(res < 0))
+ return -1;
+ if (res == 0)
+ {
+ /*
+ If the diff state record was not written here for some reason, just
+ try the one just before. It will be safe, even if not always optimal,
+ and this is an abnormal situation anyway.
+ */
+ page1= page1 - (uint32_t)diff_state_page_interval;
+ continue;
+ }
+ if (tmp_diff_state.is_before_pos(pos))
+ {
+ page0= page1;
+ page0_diff_state.reset_nolock();
+ page0_diff_state.load_nolock(&tmp_diff_state);
+ }
+ else
+ page2= page1;
+ page1= page0 +
+ ((page2 - page0) /
+ (2*(uint32_t)diff_state_page_interval) *
+ (uint32_t)diff_state_page_interval);
+ }
+ ut_ad(page1 >= page0);
+ out_state->load_nolock(&page0_diff_state);
+ *out_file_no= file_no;
+ if (page0 == 0)
+ page0= 1; /* Skip the initial file header page. */
+ *out_offset= (uint64_t)page0 << ibb_page_size_shift;
+ return 1;
+}
+
+
+int
+ha_innodb_binlog_reader::init_gtid_pos(THD *thd, slave_connection_state *pos,
+ rpl_binlog_state_base *state)
+{
+ gtid_search search_obj;
+ uint64_t file_no;
+ uint64_t offset;
+
+ /*
+ Wait for at least the initial GTID state record to become durable before
+ looking for the starting GTID position.
+ This is unlikely to need to wait, as it would imply that _no_ part of the
+ binlog is durable at this point. But it might theoretically occur perhaps
+ after a PURGE of all binlog files but the active; and failing to do the
+ wait if needed might wrongly return an error that the GTID position is
+ too old.
+ */
+ chunk_rd.seek(earliest_binlog_file_no, ibb_page_size);
+ if (UNIV_UNLIKELY(wait_available(thd, nullptr)))
+ return -1;
+
+ int res= search_obj.find_gtid_pos(pos, state, &file_no, &offset);
+ if (res < 0)
+ return -1;
+ if (res > 0)
+ {
+ requested_file_no= file_no;
+ chunk_rd.seek(file_no, offset);
+ chunk_rd.skip_partial(true);
+ cur_file_no= chunk_rd.current_file_no();
+ cur_file_pos= chunk_rd.current_pos();
+ }
+ return res;
+}
+
+
+int
+ha_innodb_binlog_reader::init_legacy_pos(THD *thd, const char *filename,
+ ulonglong offset)
+{
+ uint64_t file_no;
+ if (!filename)
+ {
+ mysql_mutex_lock(&purge_binlog_mutex);
+ file_no= earliest_binlog_file_no;
+ mysql_mutex_unlock(&purge_binlog_mutex);
+ }
+ else if (!is_binlog_name(filename, &file_no))
+ {
+ my_error(ER_UNKNOWN_TARGET_BINLOG, MYF(0));
+ return -1;
+ }
+ if (file_no > active_binlog_file_no.load(std::memory_order_acquire))
+ {
+ my_error(ER_ERROR_WHEN_EXECUTING_COMMAND, MYF(0), "SHOW BINLOG EVENTS",
+ "Could not find target log");
+ return -1;
+ }
+ requested_file_no= file_no;
+ if ((uint64_t)offset >= (uint64_t)(UINT32_MAX) << ibb_page_size_shift)
+ {
+ my_error(ER_BINLOG_POS_INVALID, MYF(0), offset);
+ return -1;
+ }
+
+ if (offset < ibb_page_size)
+ offset= ibb_page_size;
+
+ /*
+ Start at the beginning of the page containing the requested position. Then
+ read forwards until the requested position is reached. This way we avoid
+ reading garbaga data for invalid request offset.
+ */
+
+ chunk_rd.seek(file_no,
+ (uint64_t)offset & ((uint64_t)~0 << ibb_page_size_shift));
+ int err=
+ chunk_rd.find_offset_in_page((uint32_t)(offset & (ibb_page_size - 1)));
+ chunk_rd.release(true);
+ chunk_rd.skip_partial(true);
+
+ cur_file_no= chunk_rd.current_file_no();
+ cur_file_pos= chunk_rd.current_pos();
+ return err;
+}
+
+
+void
+ha_innodb_binlog_reader::enable_single_file()
+{
+ chunk_rd.stop_file_no= requested_file_no != ~(uint64_t)0 ?
+ requested_file_no : chunk_rd.s.file_no;
+}
+
+
+void
+ha_innodb_binlog_reader::seek_internal(uint64_t file_no, uint64_t offset)
+{
+ chunk_rd.seek(file_no, offset);
+ chunk_rd.skip_partial(true);
+ cur_file_no= chunk_rd.current_file_no();
+ cur_file_pos= chunk_rd.current_pos();
+}
+
+
+void
+ibb_wait_durable_offset(uint64_t file_no, uint64_t wait_offset)
+{
+ uint64_t dur_offset=
+ binlog_cur_durable_offset[file_no & 3].load(std:: memory_order_relaxed);
+ if (dur_offset >= wait_offset)
+ return;
+
+ ha_innodb_binlog_reader reader(true, file_no, dur_offset);
+ for (;;)
+ {
+ reader.wait_available(nullptr, nullptr);
+ dur_offset=
+ binlog_cur_durable_offset[file_no & 3].load(std:: memory_order_relaxed);
+ if (dur_offset >= wait_offset)
+ break;
+ reader.seek_internal(file_no, dur_offset);
+ }
+}
+
+
+pending_lsn_fifo::pending_lsn_fifo()
+ : flushing_lsn(0), last_lsn_added(0), cur_file_no(~(uint64_t)0),
+ head(0), tail(0)
+{
+}
+
+
+void
+pending_lsn_fifo::init(uint64_t start_file_no)
+{
+ mysql_mutex_lock(&binlog_durable_mutex);
+ ut_ad(cur_file_no == ~(uint64_t)0);
+ cur_file_no= start_file_no;
+ mysql_mutex_unlock(&binlog_durable_mutex);
+}
+
+
+void
+pending_lsn_fifo::reset()
+{
+ mysql_mutex_lock(&binlog_durable_mutex);
+ cur_file_no= ~(uint64_t)0;
+ mysql_mutex_unlock(&binlog_durable_mutex);
+}
+
+
+bool
+pending_lsn_fifo::process_durable_lsn(lsn_t lsn)
+{
+ mysql_mutex_assert_owner(&binlog_durable_mutex);
+ ut_ad(cur_file_no != ~(uint64_t)0);
+
+ entry *got= nullptr;
+ for (;;)
+ {
+ if (is_empty())
+ break;
+ entry &e= cur_tail();
+ if (lsn < e.lsn)
+ break;
+ got= &e;
+ drop_tail();
+ }
+ if (got)
+ {
+ uint64_t active= active_binlog_file_no.load(std::memory_order_relaxed);
+ DBUG_EXECUTE_IF("block_binlog_durable", active= got->file_no + 2;);
+ if (got->file_no + 1 >= active)
+ {
+ /*
+ We must never set the durable offset back to a prior value.
+ This should be assured by never adding a smaller lsn into the fifo than
+ any prior lsn added, and checked by this assertion.
+ */
+ ut_ad(binlog_cur_durable_offset[got->file_no & 3].
+ load(std::memory_order_relaxed) <= got->offset);
+ binlog_cur_durable_offset[got->file_no & 3].store
+ (got->offset, std::memory_order_relaxed);
+ }
+ /*
+ If we moved the durable point to the next file_no, mark the prior
+ file_no as now fully durable.
+ Since we only ever have at most two binlog tablespaces open, and since
+ we make file_no=N fully durable (by calling into this function) before
+ pre-allocating N+2, we can only ever move ahead one file_no at a time
+ here.
+ */
+ if (cur_file_no != got->file_no)
+ {
+ ut_ad(got->file_no == cur_file_no + 1);
+ binlog_cur_durable_offset[cur_file_no & 3].store(
+ binlog_cur_end_offset[cur_file_no & 3].load(std::memory_order_relaxed),
+ std::memory_order_relaxed);
+ cur_file_no= got->file_no;
+ }
+ mysql_cond_broadcast(&binlog_durable_cond);
+ return true;
+ }
+ return false;
+}
+
+
+/**
+ After a binlog commit, put the LSN and the corresponding binlog position
+ into the ibb_pending_lsn_fifo. We do this here (rather than immediately in
+ innodb_binlog_post_commit()), so that we can delay it until we are no longer
+ holding more critical locks that could block other writers. As we will be
+ contending with readers here on binlog_durable_mutex.
+*/
+void
+pending_lsn_fifo::record_commit(binlog_oob_context *c)
+{
+ uint64_t pending_file_no= c->pending_file_no;
+ if (pending_file_no == ~(uint64_t)0)
+ return;
+ c->pending_file_no= ~(uint64_t)0;
+ lsn_t pending_lsn= c->pending_lsn;
+ uint64_t pending_offset= c->pending_offset;
+ add_to_fifo(pending_lsn, pending_file_no, pending_offset);
+}
+
+
+void
+pending_lsn_fifo::add_to_fifo(uint64_t lsn, uint64_t file_no, uint64_t offset)
+{
+ mysql_mutex_lock(&binlog_durable_mutex);
+ /*
+ The record_commit() operation is done outside of critical locks for
+ scalabitily, so can occur out-of-order. So only insert the new entry if
+ it is newer than any previously inserted.
+ */
+ ut_ad(is_empty() || cur_head().lsn == last_lsn_added);
+ if (lsn > last_lsn_added)
+ {
+ if (is_full())
+ {
+ /*
+ When the fifo is full, we just overwrite the head with a newer LSN.
+ This way, whenever _some_ LSN gets synced durably to disk, we will
+ always be able to make some progress and clear some fifo entries. And
+ when this latest LSN gets eventually synced, any overwritten entry
+ will progress as well.
+ */
+ }
+ else
+ {
+ /*
+ Insert a new head.
+ Note that we make the fifo size a power-of-two (1 <(p)->xid);
+ *out_len= xid->key_length();
+ return xid->key();
+}
+
+
+ibb_xid_hash::ibb_xid_hash()
+{
+ mysql_mutex_init(ibb_xid_hash_mutex_key, &xid_mutex, nullptr);
+ my_hash_init(mem_key_binlog, &xid_hash, &my_charset_bin, 32, 0,
+ sizeof(XID), get_xid_hash_key, nullptr, MYF(HASH_UNIQUE));
+}
+
+
+ibb_xid_hash::~ibb_xid_hash()
+{
+ for (uint32 i= 0; i < xid_hash.records; ++i)
+ my_free(my_hash_element(&xid_hash, i));
+ my_hash_free(&xid_hash);
+ mysql_mutex_destroy(&xid_mutex);
+}
+
+
+bool
+ibb_xid_hash::add_xid(const XID *xid, const binlog_oob_context *c)
+{
+ if (UNIV_LIKELY(c->node_list_len > 0))
+ {
+ uint32_t last= c->node_list_len-1;
+ return add_xid(xid, c->first_node_file_no, c->lf_pins,
+ c->node_list[last].node_index + 1,
+ c->first_node_file_no, c->first_node_offset,
+ c->node_list[last].file_no, c->node_list[last].offset);
+ }
+ else
+ {
+ /*
+ Empty XA transaction, but we still need to ensure the prepare record
+ is kept until the (empty) transactions gets XA COMMMIT'ted.
+ */
+ uint64_t refcnt_file_no=
+ active_binlog_file_no.load(std::memory_order_acquire);
+ return add_xid(xid, refcnt_file_no, c->lf_pins, 0, 0, 0, 0, 0);
+ }
+}
+
+
+bool
+ibb_xid_hash::add_xid(const XID *xid, uint64_t refcnt_file_no, LF_PINS *pins,
+ uint64_t num_nodes,
+ uint64_t first_file_no, uint64_t first_offset,
+ uint64_t last_file_no, uint64_t last_offset)
+{
+ xid_elem *e=
+ (xid_elem *)my_malloc(mem_key_binlog, sizeof(xid_elem), MYF(MY_WME));
+ if (UNIV_UNLIKELY(!e))
+ {
+ my_error(ER_OUTOFMEMORY, MYF(0), (int)sizeof(xid_elem));
+ return true;
+ }
+ e->xid.set(xid);
+ e->oob_num_nodes= num_nodes;
+ e->oob_first_file_no= first_file_no;
+ e->oob_first_offset= first_offset;
+ e->oob_last_file_no= last_file_no;
+ e->oob_last_offset= last_offset;
+ e->refcnt_file_no= refcnt_file_no;
+ mysql_mutex_lock(&xid_mutex);
+ if (my_hash_insert(&xid_hash, (uchar *)e))
+ {
+ mysql_mutex_unlock(&xid_mutex);
+ my_free(e);
+ return true;
+ }
+ uint64_t refcnt=
+ ibb_file_hash.oob_ref_inc(refcnt_file_no, pins, true);
+ if (refcnt == 1)
+ ibb_file_hash.update_earliest_xa_ref(refcnt_file_no, pins);
+ mysql_mutex_unlock(&xid_mutex);
+ return false;
+}
+
+
+template bool
+ibb_xid_hash::run_on_xid(const XID *xid, F callback)
+{
+ size_t key_len= 0;
+ const uchar *key_ptr= get_xid_hash_key(xid, &key_len, 1);
+ bool err;
+
+ mysql_mutex_lock(&xid_mutex);
+ uchar *rec= my_hash_search(&xid_hash, key_ptr, key_len);
+ if (UNIV_LIKELY(rec != nullptr))
+ {
+ err= callback(reinterpret_cast(rec));
+ }
+ else
+ err= true;
+ mysql_mutex_unlock(&xid_mutex);
+ return err;
+}
+
+
+/*
+ Look up an XID in the internal XID hash.
+ Remove the entry found (if any) and return it.
+*/
+ibb_xid_hash::xid_elem *
+ibb_xid_hash::grab_xid(const XID *xid)
+{
+ xid_elem *e= nullptr;
+ size_t key_len= 0;
+ const uchar *key_ptr= get_xid_hash_key(xid, &key_len, 1);
+ mysql_mutex_lock(&xid_mutex);
+ uchar *rec= my_hash_search(&xid_hash, key_ptr, key_len);
+ if (UNIV_LIKELY(rec != nullptr))
+ {
+ e= reinterpret_cast(rec);
+ my_hash_delete(&xid_hash, rec);
+ }
+ mysql_mutex_unlock(&xid_mutex);
+ return e;
+}
+
+
+void
+ibb_get_filename(char name[FN_REFLEN], uint64_t file_no)
+{
+ static_assert(BINLOG_NAME_MAX_LEN <= FN_REFLEN,
+ "FN_REFLEN too shot to hold InnoDB binlog name");
+ binlog_name_make_short(name, file_no);
+}
+
+
+extern "C" void binlog_get_cache(THD *, uint64_t, uint64_t, IO_CACHE **,
+ handler_binlog_event_group_info **,
+ const rpl_gtid **);
+
+binlog_oob_context *
+innodb_binlog_trx(trx_t *trx, mtr_t *mtr)
+{
+ IO_CACHE *cache;
+ handler_binlog_event_group_info *binlog_info;
+ const rpl_gtid *gtid;
+ uint64_t file_no, pos;
+
+ if (!trx->mysql_thd)
+ return nullptr;
+ innodb_binlog_status(&file_no, &pos);
+ binlog_get_cache(trx->mysql_thd, file_no, pos, &cache, &binlog_info, >id);
+ if (UNIV_LIKELY(binlog_info != nullptr) &&
+ UNIV_LIKELY(binlog_info->gtid_offset > 0)) {
+ innodb_binlog_write_cache(cache, gtid, binlog_info, mtr);
+ return static_cast(binlog_info->engine_ptr);
+ }
+ return nullptr;
+}
+
+
+void
+innodb_binlog_post_commit(mtr_t *mtr, binlog_oob_context *c)
+{
+ if (c)
+ {
+ c->pending_lsn= mtr->commit_lsn();
+ ut_ad(c->pending_lsn != 0);
+ }
+}
+
+
+/*
+ Function to record the write of a record to the binlog, when done outside
+ of a normal binlog commit, eg. XA PREPARE or XA ROLLBACK.
+*/
+static void
+innodb_binlog_post_write_rec(mtr_t *mtr, binlog_oob_context *c)
+{
+ uint64_t file_no= active_binlog_file_no.load(std::memory_order_relaxed);
+ c->pending_file_no= file_no;
+ c->pending_offset=
+ binlog_cur_end_offset[file_no & 3].load(std::memory_order_relaxed);
+ innodb_binlog_post_commit(mtr, c);
+}
+
+
+bool
+innobase_binlog_write_direct_ordered(IO_CACHE *cache,
+ handler_binlog_event_group_info *binlog_info,
+ const rpl_gtid *gtid)
+{
+ mtr_t mtr{nullptr};
+ ut_ad(binlog_info->engine_ptr2 == nullptr);
+ innodb_binlog_status(&binlog_info->out_file_no, &binlog_info->out_offset);
+ mtr.start();
+ innodb_binlog_write_cache(cache, gtid, binlog_info, &mtr);
+ mtr.commit();
+ innodb_binlog_post_commit(&mtr, static_cast
+ (binlog_info->engine_ptr));
+ return false;
+}
+
+
+bool
+innobase_binlog_write_direct(IO_CACHE *cache,
+ handler_binlog_event_group_info *binlog_info,
+ const rpl_gtid *gtid)
+{
+ ut_ad(binlog_info->engine_ptr2 == nullptr);
+ binlog_oob_context *c=
+ static_cast(binlog_info->engine_ptr);
+ if (UNIV_LIKELY(c != nullptr))
+ {
+ /*
+ This is a binlog write of updates that happen outside of InnoDB, eg.
+ MyISAM. Let's at least write the redo log to the operating system,
+ so a crash of the server process can still recover the binlog data.
+ But only fsync() to disk if requested by srv_flush_log_at_trx_commit=1.
+ This matches the traditional behavior of the binlog and the
+ sync_binlog=0|1 configuration.
+ */
+ log_write_up_to(c->pending_lsn, (srv_flush_log_at_trx_commit & 1) != 0);
+ DEBUG_SYNC(current_thd, "ibb_after_commit_redo_log");
+ ibb_pending_lsn_fifo.record_commit(c);
+ }
+ return false;
+}
+
+
+void
+ibb_group_commit(THD *thd, handler_binlog_event_group_info *binlog_info)
+{
+ binlog_oob_context *c=
+ static_cast(binlog_info->engine_ptr);
+ if (UNIV_LIKELY(c != nullptr))
+ {
+ if (srv_flush_log_at_trx_commit > 0 && c->pending_lsn)
+ {
+ /*
+ Flush the InnoDB redo log to disk here for the entire group commit, so
+ that it will be available for all binlog readers. Durably or
+ non-durably as configured in --innodb-flush-log-at-trx-commit.
+ */
+ log_write_up_to(c->pending_lsn, (srv_flush_log_at_trx_commit & 1) != 0);
+ }
+ DEBUG_SYNC(current_thd, "ibb_after_group_commit_redo_log");
+ ibb_pending_lsn_fifo.record_commit(c);
+ }
+}
+
+
+bool
+ibb_write_xa_prepare_ordered(THD *thd,
+ handler_binlog_event_group_info *binlog_info,
+ uchar engine_count)
+{
+ mtr_t mtr{nullptr};
+ binlog_oob_context *c=
+ static_cast(binlog_info->engine_ptr);
+ chunk_data_xa_prepare chunk_data(binlog_info->xa_xid, engine_count, c);
+ mtr.start();
+ fsp_binlog_write_rec(&chunk_data, &mtr, FSP_BINLOG_TYPE_XA_PREPARE,
+ c->lf_pins);
+ mtr.commit();
+ innodb_binlog_post_write_rec(&mtr, c);
+
+ return false;
+}
+
+
+bool
+ibb_write_xa_prepare(THD *thd,
+ handler_binlog_event_group_info *binlog_info,
+ uchar engine_count)
+{
+ bool err= false;
+
+ binlog_oob_context *c=
+ static_cast(binlog_info->engine_ptr);
+ ut_ad(binlog_info->xa_xid != nullptr);
+ if (ibb_xa_xid_hash->add_xid(binlog_info->xa_xid, c))
+ err= true;
+
+ /*
+ Sync the redo log to ensure that the prepare record is durably written to
+ disk. This is necessary before returning OK to the client, to be sure we
+ can recover the binlog part of the XA transaction in case of crash.
+ */
+ if (srv_flush_log_at_trx_commit > 0)
+ log_write_up_to(c->pending_lsn, (srv_flush_log_at_trx_commit & 1));
+ DEBUG_SYNC(thd, "ibb_after_prepare_redo_log");
+ ibb_pending_lsn_fifo.record_commit(c);
+
+ return err;
+}
+
+
+bool
+ibb_xa_rollback_ordered(THD *thd, const XID *xid, void **engine_data)
+{
+ binlog_oob_context *c=
+ static_cast(*engine_data);
+ if (UNIV_UNLIKELY(c == nullptr))
+ *engine_data= c= alloc_oob_context();
+
+ /*
+ Write ROLLBACK record to the binlog.
+ This will be used during recovery to know that the XID is no longer active,
+ allowing purge of the associated binlogs.
+ */
+ chunk_data_xa_complete chunk_data(xid, false);
+ mtr_t mtr{nullptr};
+ mtr.start();
+ fsp_binlog_write_rec(&chunk_data, &mtr, FSP_BINLOG_TYPE_XA_COMPLETE,
+ c->lf_pins);
+ mtr.commit();
+ innodb_binlog_post_write_rec(&mtr, c);
+
+ return false;
+}
+
+
+bool
+ibb_xa_rollback(THD *thd, const XID *xid, void **engine_data)
+{
+ binlog_oob_context *c=
+ static_cast(*engine_data);
+
+ /*
+ Keep the reference count here, as we need the rollback record to be
+ available for recovery until all engines have durably rolled back.
+ Decrement will happen after that, in ibb_binlog_unlog().
+ */
+
+ /*
+ Durably write the rollback record to disk. This way, when we return the
+ "ok" packet to the client, we are sure that crash recovery will make the
+ XID rollback in engines if needed.
+ */
+ ut_ad(c->pending_lsn > 0);
+ if (srv_flush_log_at_trx_commit > 0)
+ log_write_up_to(c->pending_lsn, (srv_flush_log_at_trx_commit & 1));
+ DEBUG_SYNC(thd, "ibb_after_rollback_redo_log");
+
+ ibb_pending_lsn_fifo.record_commit(c);
+ c->pending_lsn= 0;
+ return false;
+}
+
+
+void
+ibb_binlog_unlog(const XID *xid, void **engine_data)
+{
+ binlog_oob_context *c=
+ static_cast(*engine_data);
+ if (UNIV_UNLIKELY(c == nullptr))
+ *engine_data= c= alloc_oob_context();
+ ibb_xid_hash::xid_elem *elem= ibb_xa_xid_hash->grab_xid(xid);
+ if (elem)
+ {
+ mysql_mutex_lock(&ibb_xa_xid_hash->xid_mutex);
+ uint64_t new_refcnt=
+ ibb_file_hash.oob_ref_dec(elem->refcnt_file_no, c->lf_pins, true);
+ if (new_refcnt == 0)
+ ibb_file_hash.update_earliest_xa_ref(elem->refcnt_file_no, c->lf_pins);
+ mysql_mutex_unlock(&ibb_xa_xid_hash->xid_mutex);
+ my_free(elem);
+ }
+}
+
+
+bool
+innodb_find_binlogs(uint64_t *out_first, uint64_t *out_last)
+{
+ mysql_mutex_lock(&active_binlog_mutex);
+ *out_last= last_created_binlog_file_no;
+ mysql_mutex_unlock(&active_binlog_mutex);
+ mysql_mutex_lock(&purge_binlog_mutex);
+ *out_first= earliest_binlog_file_no;
+ mysql_mutex_unlock(&purge_binlog_mutex);
+ if (*out_first == ~(uint64_t)0 || *out_last == ~(uint64_t)0)
+ {
+ ut_ad(0 /* Impossible, we wait at startup for binlog to be created. */);
+ return true;
+ }
+ return false;
+}
+
+
+void
+innodb_binlog_status(uint64_t *out_file_no, uint64_t *out_pos)
+{
+ static_assert(BINLOG_NAME_MAX_LEN <= FN_REFLEN,
+ "FN_REFLEN too shot to hold InnoDB binlog name");
+ uint64_t file_no= active_binlog_file_no.load(std::memory_order_relaxed);
+ uint32_t page_no= binlog_cur_page_no;
+ uint32_t in_page_offset= binlog_cur_page_offset;
+ *out_file_no= file_no;
+ *out_pos= ((uint64_t)page_no << ibb_page_size_shift) | in_page_offset;
+}
+
+
+bool
+innodb_binlog_get_init_state(rpl_binlog_state_base *out_state)
+{
+ binlog_chunk_reader chunk_reader(binlog_cur_end_offset);
+ bool err= false;
+ uint64_t dummy_xa_ref;
+
+ byte *page_buf= static_cast(ut_malloc(ibb_page_size, mem_key_binlog));
+ if (!page_buf)
+ {
+ my_error(ER_OUTOFMEMORY, MYF(0), ibb_page_size);
+ return true;
+ }
+ chunk_reader.set_page_buf(page_buf);
+
+ mysql_mutex_lock(&purge_binlog_mutex);
+ chunk_reader.seek(earliest_binlog_file_no, ibb_page_size);
+ int res= read_gtid_state(&chunk_reader, out_state, &dummy_xa_ref);
+ mysql_mutex_unlock(&purge_binlog_mutex);
+ if (res != 1)
+ err= true;
+ ut_free(page_buf);
+ return err;
+
+}
+
+
+bool
+innodb_reset_binlogs()
+{
+ bool err= false;
+ LF_PINS *lf_pins= lf_hash_get_pins(&ibb_file_hash.hash);
+ ut_a(lf_pins);
+ ut_a(innodb_binlog_inited >= 2);
+
+ uint64_t active= active_binlog_file_no.load(std::memory_order_relaxed);
+ if (ibb_file_hash.check_any_oob_ref_in_use(earliest_binlog_file_no,
+ active, lf_pins))
+ {
+ my_error(ER_BINLOG_IN_USE_TRX, MYF(0));
+ return true;
+ }
+
+ /* Close existing binlog tablespaces and stop the pre-alloc thread. */
+ innodb_binlog_close(false);
+
+ /*
+ Durably flush the redo log to disk. This is mostly to simplify
+ conceptually (RESET MASTER is not performance critical). This way, we will
+ never see a state where recovery stops at an LSN prior to the RESET
+ MASTER, so we do not have any question around truncating the binlog to a
+ point before the RESET MASTER.
+ */
+ log_buffer_flush_to_disk(true);
+
+ /* Prevent any flushing activity while resetting. */
+ binlog_page_fifo->lock_wait_for_idle();
+ binlog_page_fifo->reset();
+ ibb_pending_lsn_fifo.reset();
+
+ ibb_file_hash.remove_up_to(last_created_binlog_file_no, lf_pins);
+
+ /* Delete all binlog files in the directory. */
+ MY_DIR *dir= my_dir(innodb_binlog_directory, MYF(MY_WME));
+ if (!dir)
+ {
+ sql_print_error("Could not read the binlog directory '%s', error code %d",
+ innodb_binlog_directory, my_errno);
+ err= true;
+ }
+ else
+ {
+ size_t num_entries= dir->number_of_files;
+ fileinfo *entries= dir->dir_entry;
+ for (size_t i= 0; i < num_entries; ++i) {
+ const char *name= entries[i].name;
+ uint64_t file_no;
+ if (!is_binlog_name(name, &file_no))
+ continue;
+ char full_path[OS_FILE_MAX_PATH];
+ binlog_name_make(full_path, file_no);
+ if (my_delete(full_path, MYF(MY_WME)))
+ err= true;
+ /*
+ Just as defensive coding, also remove any entry from the file hash
+ with this file_no. We would expect to have already deleted everything
+ in remove_up_to() above.
+ */
+ ibb_file_hash.remove(file_no, lf_pins);
+ }
+ my_dirend(dir);
+ }
+ /*
+ If we get an error deleting any of the existing files, we report the error
+ back up. But we still try to initialize an empty binlog state, better than
+ leaving a non-functional binlog with corrupt internal state.
+ */
+
+ /* Re-initialize empty binlog state and start the pre-alloc thread. */
+ innodb_binlog_init_state();
+ load_global_binlog_state(&binlog_full_state);
+ ibb_pending_lsn_fifo.init(0);
+ binlog_page_fifo->unlock_with_delayed_free();
+ start_binlog_prealloc_thread();
+ binlog_sync_initial();
+
+ lf_hash_put_pins(lf_pins);
+ return err;
+}
+
+
+/*
+ Given a limit_file_no that is still needed by a slave (dump thread).
+ The dump thread will need to read any oob records references from event
+ groups in that file_no, so it will then also need to read from any earlier
+ file_no referenced from limit_file_no.
+
+ This function handles this dependency, by reading the header page (or
+ getting from the ibb_file_hash if available) to get any earlier file_no
+ containing such references.
+*/
+static bool
+purge_adjust_limit_file_no(handler_binlog_purge_info *purge_info, LF_PINS *pins)
+{
+ uint64_t limit_file_no= purge_info->limit_file_no;
+ if (limit_file_no == ~(uint64_t)0)
+ return false;
+
+ uint64_t referenced_file_no;
+ if (ibb_file_hash.get_oob_ref_file_no(limit_file_no, pins,
+ &referenced_file_no))
+ return true;
+
+ if (referenced_file_no < limit_file_no)
+ purge_info->limit_file_no= referenced_file_no;
+ else
+ ut_ad(referenced_file_no == limit_file_no ||
+ referenced_file_no == ~(uint64_t)0);
+
+ return false;
+}
+
+
+/**
+ The low-level function handling binlog purge.
+
+ How much to purge is determined by:
+
+ 1. Lowest file_no that should not be purged. This is determined as the
+ minimum of:
+ 1a. active_binlog_file_no
+ 1b. first_open_binlog_file_no
+ 1c. Any file_no in use by an active dump thread
+ 1d. Any file_no containing oob data referenced by file_no from (1c)
+ 1e. Any file_no containing oob data referenced by an active transaction.
+ 1f. User specified file_no (from PURGE BINARY LOGS TO, if any).
+
+ 2. Unix timestamp specifying the minimal value that should not be purged,
+ optional (used by PURGE BINARY LOGS BEFORE and --binlog-expire-log-seconds).
+
+ 3. Maximum total size of binlogs, optional (from --max-binlog-total-size).
+
+ Sets out_file_no to the earliest binlog file not purged.
+ Additionally returns:
+
+ 0 Purged all files as requested.
+ 1 Some files were not purged due to being currently in-use (by binlog
+ writing or active dump threads).
+*/
+static int
+innodb_binlog_purge_low(handler_binlog_purge_info *purge_info,
+ uint64_t limit_name_file_no, LF_PINS *lf_pins,
+ uint64_t *out_file_no)
+ noexcept
+{
+ uint64_t limit_file_no= purge_info->limit_file_no;
+ bool by_date= purge_info->purge_by_date;
+ bool by_size= purge_info->purge_by_size;
+ bool by_name= purge_info->purge_by_name;
+ uint64_t active= active_binlog_file_no.load(std::memory_order_relaxed);
+ bool need_active_flush= (active <= limit_file_no + 2);
+ ut_ad(by_date || by_size || by_name);
+ ut_a(limit_file_no <= active);
+ ut_a(limit_file_no <= first_open_binlog_file_no);
+
+ mysql_mutex_assert_owner(&purge_binlog_mutex);
+ size_t loc_total_size= total_binlog_used_size;
+ uint64_t file_no;
+ bool want_purge;
+
+ for (file_no= earliest_binlog_file_no; ; ++file_no)
+ {
+ want_purge= false;
+
+ char filename[OS_FILE_MAX_PATH];
+ binlog_name_make(filename, file_no);
+ MY_STAT stat_buf;
+ if (!my_stat(filename, &stat_buf, MYF(0)))
+ {
+ if (my_errno == ENOENT)
+ sql_print_information("InnoDB: File already gone when purging binlog "
+ "file '%s'", filename);
+ else
+ sql_print_warning("InnoDB: Failed to stat() when trying to purge "
+ "binlog file '%s' (errno: %d)", filename, my_errno);
+ continue;
+ }
+
+ if (by_date && stat_buf.st_mtime < purge_info->limit_date)
+ want_purge= true;
+ if (by_size && loc_total_size > purge_info->limit_size)
+ want_purge= true;
+ if (by_name && file_no < limit_name_file_no)
+ want_purge= true;
+ if (!want_purge ||
+ file_no >= limit_file_no ||
+ ibb_file_hash.get_oob_ref_in_use(file_no, lf_pins))
+ break;
+
+ earliest_binlog_file_no= file_no + 1;
+ if (loc_total_size < (size_t)stat_buf.st_size)
+ {
+ /*
+ Somehow we miscounted size, files changed from outside server or
+ possibly bug. We will handle not underflowing the total. If this
+ assertion becomes a problem for testing, it can just be removed.
+ */
+ ut_ad(0);
+ }
+ else
+ loc_total_size-= (size_t)stat_buf.st_size;
+
+ /*
+ Make sure that we always leave at least one binlog file durably non-empty,
+ by fsync()'ing the first page of the active file before deleting file
+ (active-2). This way, recovery will always have at least one file header
+ from which to determine the LSN at which to start applying redo records.
+ */
+ if (file_no + 2 >= active && need_active_flush)
+ {
+ binlog_page_fifo->flush_up_to(active, 0);
+ need_active_flush= false;
+ }
+
+ ibb_file_hash.remove(file_no, lf_pins);
+ if (my_delete(filename, MYF(0)))
+ {
+ if (my_errno == ENOENT)
+ {
+ /*
+ File already gone, just ignore the error.
+ (This should be somewhat unusual to happen as stat() succeeded).
+ */
+ }
+ else
+ {
+ sql_print_warning("InnoDB: Delete failed while trying to purge binlog "
+ "file '%s' (errno: %d)", filename, my_errno);
+ continue;
+ }
+ }
+ }
+ total_binlog_used_size= loc_total_size;
+ *out_file_no= file_no;
+ return (want_purge ? 1 : 0);
+}
+
+
+static void
+innodb_binlog_autopurge(uint64_t first_open_file_no, LF_PINS *pins)
+{
+ handler_binlog_purge_info purge_info;
+#ifdef HAVE_REPLICATION
+ extern bool ha_binlog_purge_info(handler_binlog_purge_info *out_info);
+ bool can_purge= ha_binlog_purge_info(&purge_info);
+#else
+ bool can_purge= false;
+ memset(&purge_info, 0, sizeof(purge_info)); /* Silence compiler warnings. */
+#endif
+ if (!can_purge ||
+ !(purge_info.purge_by_size || purge_info.purge_by_date))
+ return;
+
+ /*
+ Do not purge the active file_no, nor any oob references out of the active
+ (the latter might be needed to recover the GTID state after server
+ restart).
+ */
+ uint64_t active= active_binlog_file_no.load(std::memory_order_relaxed);
+ if (purge_info.limit_file_no > active)
+ purge_info.limit_file_no= active;
+
+ if (purge_adjust_limit_file_no(&purge_info, pins))
+ return;
+
+ /* Don't purge any actively open tablespace files. */
+ uint64_t orig_limit_file_no= purge_info.limit_file_no;
+ if (purge_info.limit_file_no == ~(uint64_t)0 ||
+ purge_info.limit_file_no > first_open_file_no)
+ purge_info.limit_file_no= first_open_file_no;
+ purge_info.purge_by_name= false;
+
+ uint64_t file_no;
+ int res= innodb_binlog_purge_low(&purge_info, 0, pins, &file_no);
+ if (res)
+ {
+ if (!purge_warning_given)
+ {
+ char filename[BINLOG_NAME_MAX_LEN];
+ binlog_name_make_short(filename, file_no);
+ if (purge_info.nonpurge_reason)
+ sql_print_information("InnoDB: Binlog file %s could not be purged "
+ "because %s",
+ filename, purge_info.nonpurge_reason);
+ else if (orig_limit_file_no == file_no)
+ sql_print_information("InnoDB: Binlog file %s could not be purged "
+ "because it is in use by a binlog dump thread "
+ "(connected slave)", filename);
+ else if (purge_info.limit_file_no == file_no)
+ sql_print_information("InnoDB: Binlog file %s could not be purged "
+ "because it is in active use", filename);
+ else
+ sql_print_information("InnoDB: Binlog file %s could not be purged "
+ "because it might still be needed", filename);
+ purge_warning_given= true;
+ }
+ }
+ else
+ purge_warning_given= false;
+}
+
+
+int
+innodb_binlog_purge(handler_binlog_purge_info *purge_info)
+{
+ /*
+ Let us check that we do not get an attempt to purge by file, date, and/or
+ size at the same time.
+ (If we do, it is not necesarily a problem, but this cannot happen in
+ current server code).
+ */
+ ut_ad(1 == (!!purge_info->purge_by_name +
+ !!purge_info->purge_by_date +
+ !!purge_info->purge_by_size));
+
+ if (!purge_info->purge_by_name && !purge_info->purge_by_date &&
+ !purge_info->purge_by_size)
+ return 0;
+
+ mysql_mutex_lock(&active_binlog_mutex);
+ uint64_t limit_file_no=
+ std::min(active_binlog_file_no.load(std::memory_order_relaxed),
+ first_open_binlog_file_no);
+ uint64_t last_created= last_created_binlog_file_no;
+ mysql_mutex_unlock(&active_binlog_mutex);
+
+ uint64_t to_file_no= ~(uint64_t)0;
+ if (purge_info->purge_by_name)
+ {
+ if (!is_binlog_name(purge_info->limit_name, &to_file_no) ||
+ to_file_no > last_created)
+ return LOG_INFO_EOF;
+ }
+
+ LF_PINS *lf_pins= lf_hash_get_pins(&ibb_file_hash.hash);
+ ut_a(lf_pins);
+ if (purge_adjust_limit_file_no(purge_info, lf_pins))
+ {
+ lf_hash_put_pins(lf_pins);
+ return LOG_INFO_IO;
+ }
+
+ uint64_t orig_limit_file_no= purge_info->limit_file_no;
+ purge_info->limit_file_no= std::min(orig_limit_file_no, limit_file_no);
+
+ mysql_mutex_lock(&purge_binlog_mutex);
+ uint64_t file_no;
+ int res= innodb_binlog_purge_low(purge_info, to_file_no, lf_pins, &file_no);
+ mysql_mutex_unlock(&purge_binlog_mutex);
+ lf_hash_put_pins(lf_pins);
+
+ if (res == 1)
+ {
+ static_assert(sizeof(purge_info->nonpurge_filename) >= BINLOG_NAME_MAX_LEN,
+ "No room to return filename");
+ binlog_name_make_short(purge_info->nonpurge_filename, file_no);
+ if (!purge_info->nonpurge_reason)
+ {
+ if (limit_file_no == file_no)
+ purge_info->nonpurge_reason= "the binlog file is in active use";
+ else if (orig_limit_file_no == file_no)
+ purge_info->nonpurge_reason= "it is in use by a binlog dump thread "
+ "(connected slave)";
+ }
+ res= LOG_INFO_IN_USE;
+ }
+ else
+ purge_warning_given= false;
+
+ return res;
+}
+
+
+bool
+binlog_recover_write_data(bool space_id, uint32_t page_no,
+ uint16_t offset,
+ lsn_t start_lsn, lsn_t lsn,
+ const byte *buf, size_t size) noexcept
+{
+ if (!recover_obj.inited)
+ return recover_obj.init_recovery(space_id, page_no, offset, start_lsn, lsn,
+ buf, size);
+ return recover_obj.apply_redo(space_id, page_no, offset, start_lsn, lsn,
+ buf, size);
+}
+
+
+void
+binlog_recover_end(lsn_t lsn) noexcept
+{
+ if (recover_obj.inited)
+ recover_obj.end_actions(true);
+}
diff --git a/storage/innobase/include/buf0buf.h b/storage/innobase/include/buf0buf.h
index fd36370d7f44d..18f63c6d21ea4 100644
--- a/storage/innobase/include/buf0buf.h
+++ b/storage/innobase/include/buf0buf.h
@@ -1557,7 +1557,7 @@ class buf_pool_t
size_t flush_list_requests;
TPOOL_SUPPRESS_TSAN void add_flush_list_requests(size_t size)
- { ut_ad(size); flush_list_requests+= size; }
+ { flush_list_requests+= size; }
private:
static constexpr unsigned PAGE_CLEANER_IDLE= 1;
static constexpr unsigned FLUSH_LIST_ACTIVE= 2;
diff --git a/storage/innobase/include/fil0fil.h b/storage/innobase/include/fil0fil.h
index 73bbc79f665c3..39149742ce087 100644
--- a/storage/innobase/include/fil0fil.h
+++ b/storage/innobase/include/fil0fil.h
@@ -1156,10 +1156,12 @@ enum ib_extention {
NO_EXT = 0,
IBD = 1,
ISL = 2,
- CFG = 3
+ CFG = 3,
+ IBB = 4
};
extern const char* dot_ext[];
#define DOT_IBD dot_ext[IBD]
+#define DOT_IBB dot_ext[IBB]
#define DOT_ISL dot_ext[ISL]
#define DOT_CFG dot_ext[CFG]
diff --git a/storage/innobase/include/fsp0fsp.h b/storage/innobase/include/fsp0fsp.h
index 14bd89adac71b..2d51083d46945 100644
--- a/storage/innobase/include/fsp0fsp.h
+++ b/storage/innobase/include/fsp0fsp.h
@@ -333,6 +333,9 @@ fsp_header_check_encryption_key(
dberr_t fsp_header_init(fil_space_t *space, uint32_t size, mtr_t *mtr)
MY_ATTRIBUTE((nonnull, warn_unused_result));
+buf_block_t* fsp_page_create(fil_space_t *space, uint32_t offset,
+ mtr_t *mtr) noexcept;
+
/** Create a new segment.
@param space tablespace
@param byte_offset byte offset of the created segment header
diff --git a/storage/innobase/include/fsp0types.h b/storage/innobase/include/fsp0types.h
index b808941411fe0..21121c5f72bf3 100644
--- a/storage/innobase/include/fsp0types.h
+++ b/storage/innobase/include/fsp0types.h
@@ -27,7 +27,8 @@ Created May 26, 2009 Vasil Dimov
#pragma once
#include "ut0byte.h"
-/** All persistent tablespaces have a smaller fil_space_t::id than this. */
+/** All persistent tablespaces (except binlog tablespaces) have a smaller
+fil_space_t::id than this. */
constexpr uint32_t SRV_SPACE_ID_UPPER_BOUND= 0xFFFFFFF0U;
/** The fil_space_t::id of the innodb_temporary tablespace. */
constexpr uint32_t SRV_TMP_SPACE_ID= 0xFFFFFFFEU;
diff --git a/storage/innobase/include/fsp_binlog.h b/storage/innobase/include/fsp_binlog.h
new file mode 100644
index 0000000000000..7972fa77b4696
--- /dev/null
+++ b/storage/innobase/include/fsp_binlog.h
@@ -0,0 +1,557 @@
+/*****************************************************************************
+
+Copyright (c) 2024, Kristian Nielsen
+
+This program is free software; you can redistribute it and/or modify it under
+the terms of the GNU General Public License as published by the Free Software
+Foundation; version 2 of the License.
+
+This program is distributed in the hope that it will be useful, but WITHOUT
+ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or FITNESS
+FOR A PARTICULAR PURPOSE. See the GNU General Public License for more details.
+
+You should have received a copy of the GNU General Public License along with
+this program; if not, write to the Free Software Foundation, Inc.,
+51 Franklin Street, Fifth Floor, Boston, MA 02110-1335 USA
+
+*****************************************************************************/
+
+/**************************************************//**
+@file include/fsp_binlog.h
+InnoDB implementation of binlog.
+*******************************************************/
+
+#ifndef fsp_binlog_h
+#define fsp_binlog_h
+
+#include
+#include
+
+#include "lf.h"
+
+#include "univ.i"
+#include "mtr0mtr.h"
+
+
+struct chunk_data_base;
+struct binlog_header_data;
+
+/* 4-byte "magic" identifying InnoDB binlog file (little endian). */
+static constexpr uint32_t IBB_MAGIC= 0x010dfefe;
+static constexpr uint32_t IBB_FILE_VERS_MAJOR= 1;
+static constexpr uint32_t IBB_FILE_VERS_MINOR= 0;
+
+/**
+ The size of the header page that is stored in the first page of a file.
+ This is the smallest page size that can be used in a backwards compatible
+ way. Having a fixed-size small header page means we can get the real page
+ size of the file from the header page, but still be able to checksum the
+ header page without relying on unchecked page size field to compute the
+ checksum.
+
+ (The remainder of the header page is just unused or could potentially
+ later be used for other data as needed).
+*/
+static constexpr uint32_t IBB_HEADER_PAGE_SIZE= 512;
+static constexpr uint32_t IBB_PAGE_SIZE_MIN= IBB_HEADER_PAGE_SIZE;
+static constexpr uint32_t IBB_PAGE_SIZE_MAX= 65536;
+
+/** Store crc32 checksum at the end of the page */
+#define BINLOG_PAGE_CHECKSUM 4
+
+#define BINLOG_PAGE_DATA 0
+#define BINLOG_PAGE_DATA_END BINLOG_PAGE_CHECKSUM
+
+
+enum fsp_binlog_chunk_types {
+ /* Zero means no data, effectively EOF. */
+ FSP_BINLOG_TYPE_EMPTY= 0,
+ /* A binlogged committed event group. */
+ FSP_BINLOG_TYPE_COMMIT= 1,
+ /* A binlog GTID state record. */
+ FSP_BINLOG_TYPE_GTID_STATE= 2,
+ /* Out-of-band event group data. */
+ FSP_BINLOG_TYPE_OOB_DATA= 3,
+ /* Dummy record, use to fill remainder of page (eg. FLUSH BINARY LOGS). */
+ FSP_BINLOG_TYPE_DUMMY= 4,
+ /* User XA record containing XID and OOB reference for XA PREPARE. */
+ FSP_BINLOG_TYPE_XA_PREPARE= 5,
+ /* User XA record containing XID for XA COMMIT/ROLLBACK. */
+ FSP_BINLOG_TYPE_XA_COMPLETE= 6,
+ /* Must be one more than the last type. */
+ FSP_BINLOG_TYPE_END,
+
+ /* Padding data at end of page. */
+ FSP_BINLOG_TYPE_FILLER= 0xff
+};
+
+/**
+ Bit set on the chunk type for a continuation chunk, when data needs to be
+ split across pages.
+*/
+static constexpr uint32_t FSP_BINLOG_FLAG_BIT_CONT= 7;
+static constexpr uint32_t FSP_BINLOG_FLAG_CONT= (1 << FSP_BINLOG_FLAG_BIT_CONT);
+/**
+ Bit set on the chunk type for the last chunk (no continuation chunks
+ follow)
+*/
+static constexpr uint32_t FSP_BINLOG_FLAG_BIT_LAST= 6;
+static constexpr uint32_t FSP_BINLOG_FLAG_LAST= (1 << FSP_BINLOG_FLAG_BIT_LAST);
+static constexpr uint32_t FSP_BINLOG_TYPE_MASK=
+ ~(FSP_BINLOG_FLAG_CONT | FSP_BINLOG_FLAG_LAST);
+
+/* Flag bits for FSP_BINLOG_TYPE_XA_COMPLETE. */
+static constexpr uint32_t IBB_FL_XA_TYPE_MASK= 0x1;
+static constexpr uint32_t IBB_FL_XA_TYPE_COMMIT= 0x0;
+static constexpr uint32_t IBB_FL_XA_TYPE_ROLLBACK= 0x1;
+
+/**
+ These are the chunk types that are allowed to occur in the middle of
+ another record.
+*/
+static constexpr uint64_t ALLOWED_NESTED_RECORDS=
+ /* GTID STATE at start of page can occur in the middle of other record. */
+ ((uint64_t)1 << FSP_BINLOG_TYPE_GTID_STATE) |
+ /* DUMMY data at tablespace end can occur in the middle of other record. */
+ ((uint64_t)1 << FSP_BINLOG_TYPE_DUMMY)
+ ;
+/* Ensure that all types fit in the ALLOWED_NESTED_RECORDS bitmask. */
+static_assert(FSP_BINLOG_TYPE_END <= 8*sizeof(ALLOWED_NESTED_RECORDS),
+ "Binlog types must be <64 to fit "
+ "in ALLOWED_NESTED_RECORDS bitmask");
+
+
+extern uint32_t ibb_page_size_shift;
+extern ulong ibb_page_size;
+
+
+/**
+ The object representing a binlog page that is not yet flushed to disk.
+ At the end of the object is an additionally allocated byte buffer of
+ size ibb_page_size, ie. the page buffer containing the data in the page.
+
+ The LATCHED count is the number of current writers and readers of the page
+ (the page cannot be flushed and freed until this drops to zero).
+
+ The flag LAST_PAGE is set for the very last page in a tablespace file,
+ used to hold this page latched until the end of a mini-transaction.
+
+ The flag COMPLETE is set when the writer has written the last byte of the
+ page (a page cannot be freed until it is complete, and will normally not be
+ flushed unless required for an InnoDB log checkpoint).
+
+ The flag FLUSHED_CLEAN is set if a (partial) page has been flushed to disk,
+ and cleared again by a writer when more data is added to the page.
+*/
+struct fsp_binlog_page_entry {
+ uint32_t latched;
+ /* Flag set for the last page in a file. */
+ bool last_page;
+ /*
+ Flag set when the page has been filled, no more data will be added and
+ it is safe to write out to disk and remove from the FIFO.
+ */
+ bool complete;
+ /*
+ Flag set when the page is not yet complete, but all data added so far
+ have been written out to the file. So the page should not be written
+ again (until more data is added), but nor can it be removed from the
+ FIFO yet.
+ */
+ bool flushed_clean;
+ /*
+ Flag set when the page is not yet complete, but nevertheless waiting to be
+ flushed to disk (eg. due to InnoDB checkpointing). Used to avoid waking up
+ the flush thread on every release of a last partial page in the file
+ when it is not needed.
+ */
+ bool pending_flush;
+
+ byte *page_buf() { return (byte *)this + sizeof(fsp_binlog_page_entry); }
+};
+
+
+/**
+ A page FIFO, as a lower-level alternative to the buffer pool used for full
+ tablespaces.
+
+ Since binlog files are written strictly append-only, we can simply add new
+ pages at the end and flush them from the beginning.
+
+ Some attempt is made to get reasonable scalability of the page fifo (even
+ though it is still protected by a global mutex that could potentially be
+ contended between writers and readers). The mutex is only held shortly;
+ a "latch" count in each page marks when there are active readers or writers
+ preventing page flush and free. Thus readers and writers can access a page
+ concurrently. File write operations/syscalls are done outside of holding the
+ mutex, and a freelist is used to likewise avoid most malloc/free.
+*/
+class fsp_binlog_page_fifo {
+public:
+ /*
+ Allow at most 1/N of the pages in one binlog file will be kept in-memory
+ on the free list of page buffers.
+ */
+ static constexpr uint64_t MAX_FREE_BUFFERS_FRAC= 4;
+
+ struct page_list {
+ fsp_binlog_page_entry **entries;
+ size_t allocated_entries;
+ size_t used_entries;
+ size_t first_entry;
+ uint32_t first_page_no;
+ uint32_t size_in_pages;
+ File fh;
+
+ fsp_binlog_page_entry *&entry_at(size_t idx)
+ {
+ idx+= first_entry;
+ if (idx >= allocated_entries)
+ idx-= allocated_entries;
+ ut_ad(idx < allocated_entries);
+ return entries[idx];
+ }
+
+ };
+private:
+ mysql_mutex_t m_mutex;
+ pthread_cond_t m_cond;
+ std::thread flush_thread_obj;
+
+ /*
+ The first_file_no is the first valid file in the fifo. The other entry in
+ the fifo holds (first_file_no+1) if it is not empty.
+ If first_file_no==~0, then there are no files in the fifo (initial state
+ just after construction).
+ */
+ uint64_t first_file_no;
+ page_list fifos[2];
+ /*
+ Free list for page objects, to avoid repeated aligned_alloc().
+ Each object is allocated as a byte array of size
+ sizeof(fsp_binlog_page_entry) + ibb_page_size, holding the
+ fsp_binlog_page_entry object and the page buffer just after it.
+ When on the freelist, instead just the first sizeof(byte *) bytes store
+ a simple `next' pointer.
+ */
+ size_t free_buffers;
+ byte *freelist;
+ /* Temporary overflow of freelist, to be freed after mutex is unlocked. */
+ byte *to_free_list;
+ bool flushing;
+ bool flush_thread_started;
+ bool flush_thread_end;
+
+private:
+ fsp_binlog_page_entry *get_entry(uint64_t file_no, uint64_t page_no,
+ uint32_t latch, bool completed, bool clean);
+ void release_entry(uint64_t file_no, uint64_t page_no);
+
+public:
+ fsp_binlog_page_fifo();
+ ~fsp_binlog_page_fifo();
+ void reset();
+ void start_flush_thread();
+ void stop_flush_thread();
+ void flush_thread_run();
+ void lock_wait_for_idle();
+ void unlock() { mysql_mutex_unlock(&m_mutex); }
+ void unlock_with_delayed_free();
+ void create_tablespace(uint64_t file_no, uint32_t size_in_pages,
+ uint32_t init_page= ~(uint32_t)0,
+ byte *partial_page= nullptr);
+ void release_tablespace(uint64_t file_no);
+ void free_page_list(uint64_t file_no);
+ fsp_binlog_page_entry *create_page(uint64_t file_no, uint32_t page_no);
+ fsp_binlog_page_entry *get_page(uint64_t file_no, uint32_t page_no);
+ void release_page(fsp_binlog_page_entry *page);
+ void release_page_mtr(fsp_binlog_page_entry *page, mtr_t *mtr);
+ bool has_unflushed(uint64_t file_no);
+ void flush_one_page(uint64_t file_no, bool force);
+ void flush_up_to(uint64_t file_no, uint32_t page_no);
+ void do_fdatasync(uint64_t file_no);
+ File get_fh(uint64_t file_no);
+ uint32_t size_in_pages(uint64_t file_no) {
+ return fifos[file_no & 1].size_in_pages;
+ }
+ void truncate_file_size(uint64_t file_no, uint32_t size_in_pages)
+ {
+ fifos[file_no & 1].size_in_pages= size_in_pages;
+ }
+};
+
+
+/** Structure of an entry in the hash of binlog tablespace files. */
+struct ibb_tblspc_entry {
+ uint64_t file_no;
+ /*
+ Active transactions/oob-event-groups that start in this binlog tablespace
+ file (including any user XA).
+ */
+ std::atomicoob_refs;
+ /* Active XA transactions whose oob start in this binlog tablespace file. */
+ std::atomicxa_refs;
+ /*
+ The earliest file number that this binlog tablespace file has oob
+ references into.
+ (This is a conservative estimate, references may not actually exist in
+ case their commit record went into a later file, or they ended up rolling
+ back).
+ Includes any XA oob records.
+ */
+ std::atomicoob_ref_file_no;
+ /* Earliest file number that we have XA references into. */
+ std::atomicxa_ref_file_no;
+
+ ibb_tblspc_entry()= default;
+ ~ibb_tblspc_entry()= default;
+};
+
+
+/**
+ Class keeping reference counts of oob records starting in different binlog
+ tablespace files.
+ Used to keep track of which files should not be purged because they contain
+ oob (start) records that are still referenced by needed binlog tablespace
+ files or by active transactions.
+*/
+class ibb_file_oob_refs {
+public:
+ /* Hash contains struct ibb_tblspc_entry keyed on file_no. */
+ LF_HASH hash;
+ /*
+ Earliest file_no with start oob records that are still referenced by active
+ transactions / event groups.
+ */
+ std::atomic earliest_oob_ref;
+ /*
+ Same, but restricted to those oob that constitute XA transactions.
+ Thus, this may be larger than earliest_oob_ref or even ~(uint64_t)0 in
+ case there are no active XA.
+ */
+ std::atomic earliest_xa_ref;
+
+public:
+ /* Init the hash empty. */
+ void init() noexcept;
+ void destroy() noexcept;
+ /* Delete an entry from the hash. */
+ void remove(uint64_t file_no, LF_PINS *pins);
+ /* Delete all (consecutive) entries from file_no down. */
+ void remove_up_to(uint64_t file_no, LF_PINS *pins);
+ /*
+ Update an entry when an OOB record is started/completed.
+ Returns the resulting refcount, or ~0 if entry not found.
+ The return is the xa refcnt if do_xa==true, else the oob refcnt.
+ */
+ uint64_t oob_ref_inc(uint64_t file_no, LF_PINS *pins, bool do_xa= false);
+ uint64_t oob_ref_dec(uint64_t file_no, LF_PINS *pins, bool do_xa= false);
+ /* Update earliest_oob_ref when refcount drops to zero. */
+ void do_zero_refcnt_action(uint64_t file_no, LF_PINS *pins,
+ bool active_moving);
+ /* Update the oob and xa file_no's active at start of this file_no. */
+ bool update_refs(uint64_t file_no, LF_PINS *pins,
+ uint64_t oob_ref, uint64_t xa_ref);
+ /* Update earliest_xa_ref when xa_refs changes 0->1 or 1->0. */
+ void update_earliest_xa_ref(uint64_t ref_file_no, LF_PINS *pins);
+ /* Lookup the oob-referenced file_no from a file_no. */
+ bool get_oob_ref_file_no(uint64_t file_no, LF_PINS *pins,
+ uint64_t *out_oob_ref_file_no);
+ /* Check if file_no needed by active, not committed transaction. */
+ bool get_oob_ref_in_use(uint64_t file_no, LF_PINS *pins);
+ /* Check if _any_ file_no is needed by active, not committed transactions. */
+ bool check_any_oob_ref_in_use(uint64_t start_file_no, uint64_t end_file_no,
+ LF_PINS *lf_pins);
+};
+
+
+class binlog_chunk_reader {
+public:
+ enum chunk_reader_status {
+ CHUNK_READER_ERROR= -1,
+ CHUNK_READER_EOF= 0,
+ CHUNK_READER_FOUND= 1
+ };
+
+ /*
+ Current state, can be obtained from save_pos() and later passed to
+ restore_pos().
+ */
+ struct saved_position {
+ /* Current position file. */
+ uint64_t file_no;
+ /* The file_no of the start of a record, if in_record is true. */
+ uint64_t rec_start_file_no;
+ /* Current position page. */
+ uint32_t page_no;
+ /* Start of current chunk inside page. */
+ uint32_t in_page_offset;
+ /*
+ The length of the current chunk, once the chunk type has been read.
+ If 0, it means the chunk type (and length) has not yet been read.
+ */
+ uint32_t chunk_len;
+ /* The read position inside the current chunk. */
+ uint32_t chunk_read_offset;
+ byte chunk_type;
+ /* When set, read will skip the current chunk, if any. */
+ bool skip_current;
+ /* Set while we are in the middle of reading a record. */
+ bool in_record;
+ } s;
+
+ /* Amount of data in file, valid after fetch_current_page(). */
+ uint64_t cur_end_offset;
+ /* Length of the currently open file, valid if cur_file_handle != -1. */
+ uint64_t cur_file_length;
+ /*
+ If different from ~0, stop (return EOF) when reaching the end of this file.
+ This is used for SHOW BINLOG EVENTS, which has an old file-based interface,
+ and wants to show the events in a single file.
+ */
+ uint64_t stop_file_no;
+ /*
+ After fetch_current_page(), this points into either cur_block or
+ page_buffer as appropriate.
+ */
+ byte *page_ptr;
+ /* Valid after fetch_current_page(), if page found in buffer pool. */
+ fsp_binlog_page_entry *cur_block;
+ /* Buffer for reading a page directly from a tablespace file. */
+ byte *page_buffer;
+ /*
+ Points to either binlog_cur_durable_offset, for readers that should not
+ see binlog data until it has become durable on disk; or
+ binlog_cur_end_offset otherwise.
+ */
+ std::atomic * const limit_offset;
+ /* Open file handle to tablespace file_no, or -1. */
+ File cur_file_handle;
+ /*
+ Flag used to skip the rest of any partial chunk we might be starting in
+ the middle of.
+ */
+ bool skipping_partial;
+
+ binlog_chunk_reader(std::atomic *limit_offset_);
+ void set_page_buf(byte *in_page_buf) { page_buffer= in_page_buf; }
+ ~binlog_chunk_reader();
+
+ /* Current type, or FSP_BINLOG_TYPE_FILLER if between records. */
+ byte cur_type() { return (byte)(s.chunk_type & FSP_BINLOG_TYPE_MASK); }
+ bool cur_is_cont() { return (s.chunk_type & FSP_BINLOG_FLAG_CONT) != 0; }
+ bool end_of_record() { return !s.in_record; }
+ bool is_end_of_page() noexcept
+ {
+ return s.in_page_offset >= ibb_page_size - (BINLOG_PAGE_DATA_END + 3);
+ }
+ static int read_error_corruption(uint64_t file_no, uint64_t page_no,
+ const char *msg);
+ int read_error_corruption(const char *msg)
+ {
+ return read_error_corruption(s.file_no, s.page_no, msg);
+ }
+ enum chunk_reader_status fetch_current_page();
+ /*
+ Try to read max_len bytes from a record into buffer.
+
+ If multipage is true, will move across pages to read following
+ continuation chunks, if any, to try and read max_len total bytes. Only if
+ the record ends before max_len bytes is less amount of bytes returned.
+
+ If multipage is false, will read as much is available on one page (up to
+ max of max_len), and then return.
+
+ Returns number of bytes read, or -1 for error.
+ Returns 0 if the chunk_reader is pointing to start of a chunk at the end
+ of the current binlog (ie. end-of-file).
+ */
+ int read_data(byte *buffer, int max_len, bool multipage);
+ /*
+ Find a chunk boundary at or after specified offset in current page. Used
+ for init_legacy_pos() to find a valid starting position.
+ */
+ int find_offset_in_page(uint32_t off);
+ /* Read the file header of current file_no. */
+ int get_file_header(binlog_header_data *out_header);
+
+ /* Save current position, and restore it later. */
+ void save_pos(saved_position *out_pos) { *out_pos= s; }
+ void restore_pos(saved_position *pos);
+ void seek(uint64_t file_no, uint64_t offset);
+
+ /*
+ Make next read_data() skip any data from the current chunk (if any), and
+ start reading data only from the beginning of the next chunk. */
+ void skip_current() { if (s.in_record) s.skip_current= true; }
+ /*
+ Used initially, after seeking potentially into the middle of a (commit)
+ record, to skip any continuation chunks until we reach the start of the
+ first real record.
+ */
+ void skip_partial(bool skip) { skipping_partial= skip; }
+ /* Release any buffer pool page latch. */
+ void release(bool release_file_page= false);
+ bool data_available();
+ bool is_before_pos(uint64_t file_no, uint64_t offset);
+ uint64_t current_file_no() { return s.file_no; }
+ uint64_t current_pos() {
+ return (s.page_no << ibb_page_size_shift) + s.in_page_offset;
+ }
+};
+
+
+/** The state interval (in pages) used for active_binlog_file_no. */
+extern uint64_t current_binlog_state_interval;
+extern mysql_mutex_t active_binlog_mutex;
+extern pthread_cond_t active_binlog_cond;
+extern mysql_mutex_t binlog_durable_mutex;
+extern mysql_cond_t binlog_durable_cond;
+extern std::atomic active_binlog_file_no;
+extern uint64_t first_open_binlog_file_no;
+extern uint64_t last_created_binlog_file_no;
+extern std::atomic binlog_cur_durable_offset[4];
+extern std::atomic binlog_cur_end_offset[4];
+extern fsp_binlog_page_fifo *binlog_page_fifo;
+
+extern ibb_file_oob_refs ibb_file_hash;
+
+
+static inline void
+fsp_binlog_release(fsp_binlog_page_entry *page)
+{
+ binlog_page_fifo->release_page(page);
+}
+
+extern size_t crc32_pwrite_page(File fd, byte *buf, uint32_t page_no,
+ myf MyFlags) noexcept;
+extern int crc32_pread_page(File fd, byte *buf, uint32_t page_no,
+ myf MyFlags) noexcept;
+extern int crc32_pread_page(pfs_os_file_t fh, byte *buf, uint32_t page_no,
+ myf MyFlags) noexcept;
+extern bool ibb_record_in_file_hash(uint64_t file_no, uint64_t oob_ref,
+ uint64_t xa_ref, LF_PINS *in_pins=nullptr);
+extern void binlog_write_up_to_now() noexcept;
+extern void fsp_binlog_extract_header_page(const byte *page_buf,
+ binlog_header_data *out_header_data)
+ noexcept;
+extern void fsp_log_binlog_write(mtr_t *mtr, fsp_binlog_page_entry *page,
+ uint64_t file_no, uint32_t page_no,
+ uint32_t page_offset, uint32_t len);
+extern void fsp_log_header_page(mtr_t *mtr, fsp_binlog_page_entry *page,
+ uint64_t file_no, uint32_t len) noexcept;
+extern dberr_t fsp_binlog_init();
+extern void fsp_binlog_shutdown();
+extern dberr_t fsp_binlog_tablespace_close(uint64_t file_no);
+extern bool fsp_binlog_open(const char *file_name, pfs_os_file_t fh,
+ uint64_t file_no, size_t file_size,
+ uint32_t init_page, byte *partial_page);
+extern dberr_t fsp_binlog_tablespace_create(uint64_t file_no,
+ uint32_t size_in_pages,
+ LF_PINS *pins);
+extern std::pair fsp_binlog_write_rec(
+ struct chunk_data_base *chunk_data, mtr_t *mtr, byte chunk_type,
+ LF_PINS *pins);
+extern bool fsp_binlog_flush();
+
+#endif /* fsp_binlog_h */
diff --git a/storage/innobase/include/innodb_binlog.h b/storage/innobase/include/innodb_binlog.h
new file mode 100644
index 0000000000000..0539cd07c8ac7
--- /dev/null
+++ b/storage/innobase/include/innodb_binlog.h
@@ -0,0 +1,317 @@
+/*****************************************************************************
+
+Copyright (c) 2024, Kristian Nielsen
+
+This program is free software; you can redistribute it and/or modify it under
+the terms of the GNU General Public License as published by the Free Software
+Foundation; version 2 of the License.
+
+This program is distributed in the hope that it will be useful, but WITHOUT
+ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or FITNESS
+FOR A PARTICULAR PURPOSE. See the GNU General Public License for more details.
+
+You should have received a copy of the GNU General Public License along with
+this program; if not, write to the Free Software Foundation, Inc.,
+51 Franklin Street, Fifth Floor, Boston, MA 02110-1335 USA
+
+*****************************************************************************/
+
+/**************************************************//**
+@file include/innodb_binlog.h
+InnoDB implementation of binlog.
+*******************************************************/
+
+#ifndef innodb_binlog_h
+#define innodb_binlog_h
+
+#include "univ.i"
+#include "fsp_binlog.h"
+
+
+struct mtr_t;
+struct rpl_binlog_state_base;
+struct rpl_gtid;
+struct handler_binlog_event_group_info;
+class handler_binlog_reader;
+struct handler_binlog_purge_info;
+struct binlog_oob_context;
+
+
+/**
+ The struct chunk_data_base is a simple encapsulation of data for a chunk that
+ is to be written to the binlog. Used to separate the generic code that
+ handles binlog writing with page format and so on, from the details of the
+ data being written, avoiding an intermediary buffer holding consecutive data.
+
+ Currently used for:
+ - chunk_data_cache: A binlog trx cache to be binlogged as a commit record.
+ - chunk_data_oob: An out-of-band piece of event group data.
+ - chunk_data_flush: For dummy filler data.
+*/
+struct chunk_data_base {
+ /*
+ Copy at most max_len bytes to address p.
+ Returns a pair with amount copied, and a bool if this is the last data.
+ Should return the maximum amount of data available (up to max_len). Thus
+ the size returned should only be less than max_len if the last-data flag
+ is returned as true.
+ */
+ virtual std::pair copy_data(byte *p, uint32_t max_len) = 0;
+ virtual ~chunk_data_base() {};
+};
+
+
+/**
+ Empty chunk data, used to pass a dummy record to fsp_binlog_write_rec()
+ in fsp_binlog_flush().
+*/
+struct chunk_data_flush : public chunk_data_base {
+ ~chunk_data_flush() { }
+
+ virtual std::pair copy_data(byte *p, uint32_t max_len) final
+ {
+ memset(p, 0xff, max_len);
+ return {max_len, true};
+ }
+};
+
+
+static constexpr size_t IBB_BINLOG_HEADER_SIZE= 64;
+
+/**
+ Data stored at the start of each binlog file.
+ (The data is stored as little-engian values in the first page of the file;
+ this is just a struct to pass around the values in-memory).
+*/
+struct binlog_header_data {
+ /*
+ The LSN corresponding to the start of the binlog file. Any redo record
+ with smaller start (or end) LSN than this should be ignored during recovery
+ and not applied to this file.
+ */
+ lsn_t start_lsn;
+ /*
+ The file_no of the binlog file. This is written into the header to be able
+ to recover it in the case where no binlog files are present at server
+ start (could be due to FLUSH BINARY LOGS or RESET MASTER).
+ */
+ uint64_t file_no;
+ /* The length of this binlog file, in pages. */
+ uint64_t page_count;
+ /*
+ The interval (in pages) at which the (differential) binlog GTID state is
+ written into the binlog file, for faster GTID position search. This
+ corresponds to the value of --innodb-binlog-state-interval at the time the
+ binlog file was created.
+ */
+ uint64_t diff_state_interval;
+ /* The earliest file_no that we have oob references into. */
+ uint64_t oob_ref_file_no;
+ /* The earliest file_no that we have XA oob references into. */
+ uint64_t xa_ref_file_no;
+ /* The log_2 of the page size (eg. ibb_page_size_shift). */
+ uint32_t page_size_shift;
+ /*
+ Major and minor file format version number. The idea is that minor version
+ increments are backwards compatible, major version upgrades are not.
+ */
+ uint32_t vers_major, vers_minor;
+ /* Whether the page was found empty. */
+ bool is_empty;
+ /*
+ Whether the page was found invalid, bad magic or major version, or CRC32
+ error (and not empty).
+ */
+ bool is_invalid;
+};
+
+
+/**
+ The class pending_lsn_fifo keeps track of pending LSNs - and their
+ corresponding binlog file_no/offset - that have been mtr-committed, but have
+ not yet become durable.
+
+ Used to delay sending to slaves any data that might be lost in case the
+ master crashes just after sending.
+*/
+class pending_lsn_fifo {
+ static constexpr uint32_t fixed_size_log2= 10;
+ static constexpr uint32_t fixed_size= (1 << fixed_size_log2);
+ static constexpr uint32_t mask= (1 << fixed_size_log2) - 1;
+public:
+ struct entry {
+ lsn_t lsn;
+ uint64_t file_no;
+ uint64_t offset;
+ } fifo[fixed_size];
+ /*
+ Set while we are duing a durable sync of the redo log to the LSN that we
+ are requesting to become durable. Used to avoid multiple threads
+ needlessly trying to sync the redo log on top of one another.
+ */
+ lsn_t flushing_lsn;
+ /*
+ The last added (and thus largest) lsn. Equal to cur_head().lsn when the
+ fifo is not empty (and the lsn of the previous head when it is empty).
+ */
+ lsn_t last_lsn_added;
+ /*
+ The current file_no that has any durable data. Used to detect when an LSN
+ moves the current durable end point to the next file, so that the previous
+ file can then be marked as fully durable.
+ The value ~0 is used as a marker for "not yet initialized".
+ */
+ uint64_t cur_file_no;
+ /* The `head' points one past the most recent element. */
+ uint32_t head;
+ /* The `tail' points to the earliest element. */
+ uint32_t tail;
+
+ pending_lsn_fifo();
+ void init(uint64_t start_file_no);
+ void reset();
+ bool is_empty() { return head == tail; }
+ bool is_full() { return head == tail + fixed_size; }
+ entry &cur_head() { ut_ad(!is_empty()); return fifo[(head - 1) & mask]; }
+ entry &cur_tail() { ut_ad(!is_empty()); return fifo[tail & mask]; }
+ void drop_tail() { ut_ad(!is_empty()); ++tail; }
+ void new_head() { ut_ad(!is_full()); ++head; }
+ void record_commit(binlog_oob_context *c);
+ void add_to_fifo(uint64_t lsn, uint64_t file_no, uint64_t offset);
+ bool process_durable_lsn(lsn_t lsn);
+};
+
+
+/**
+ Class that keeps track of the oob references etc. for each
+ XA PREPAREd XID.
+*/
+class ibb_xid_hash {
+public:
+ struct xid_elem {
+ XID xid;
+ uint64_t refcnt_file_no;
+ uint64_t oob_num_nodes;
+ uint64_t oob_first_file_no;
+ uint64_t oob_first_offset;
+ uint64_t oob_last_file_no;
+ uint64_t oob_last_offset;
+ };
+ HASH xid_hash;
+ mysql_mutex_t xid_mutex;
+
+ ibb_xid_hash();
+ ~ibb_xid_hash();
+ bool add_xid(const XID *xid, const binlog_oob_context *c);
+ bool add_xid(const XID *xid, uint64_t refcnt_file_no, LF_PINS *pins,
+ uint64_t num_nodes,
+ uint64_t first_file_no, uint64_t first_offset,
+ uint64_t last_file_no, uint64_t last_offset);
+ xid_elem *grab_xid(const XID *xid);
+ template bool run_on_xid(const XID *xid, F callback);
+};
+
+
+#define BINLOG_NAME_BASE "binlog-"
+#define BINLOG_NAME_EXT ".ibb"
+/* '/' + "binlog-" + (<=20 digits) + '.' + "ibb" + '\0'. */
+#define BINLOG_NAME_MAX_LEN 1 + 1 + 7 + 20 + 1 + 3 + 1
+
+
+extern pending_lsn_fifo ibb_pending_lsn_fifo;
+extern uint32_t innodb_binlog_size_in_pages;
+extern const char *innodb_binlog_directory;
+extern uint32_t binlog_cur_page_no;
+extern uint32_t binlog_cur_page_offset;
+extern ulonglong innodb_binlog_state_interval;
+extern rpl_binlog_state_base binlog_full_state;
+extern rpl_binlog_state_base binlog_diff_state;
+extern mysql_mutex_t purge_binlog_mutex;
+extern size_t total_binlog_used_size;
+extern ibb_xid_hash *ibb_xa_xid_hash;
+
+
+static inline void
+binlog_name_make(char name_buf[OS_FILE_MAX_PATH], uint64_t file_no,
+ const char *binlog_dir)
+{
+ snprintf(name_buf, OS_FILE_MAX_PATH,
+ "%s/" BINLOG_NAME_BASE "%06" PRIu64 BINLOG_NAME_EXT,
+ binlog_dir, file_no);
+}
+
+
+static inline void
+binlog_name_make(char name_buf[OS_FILE_MAX_PATH], uint64_t file_no)
+{
+ binlog_name_make(name_buf, file_no, innodb_binlog_directory);
+}
+
+
+static inline void
+binlog_name_make_short(char *name_buf, uint64_t file_no)
+{
+ sprintf(name_buf, BINLOG_NAME_BASE "%06" PRIu64 BINLOG_NAME_EXT, file_no);
+}
+
+
+extern bool is_binlog_name(const char *name, uint64_t *out_idx);
+extern int get_binlog_header(const char *binlog_path, byte *page_buf,
+ lsn_t &out_lsn, bool &out_empty) noexcept;
+extern dberr_t innodb_binlog_startup_init();
+extern void ibb_set_max_size(size_t binlog_size);
+extern bool innodb_binlog_init(size_t binlog_size, const char *directory,
+ HASH *recovery_hash);
+extern void innodb_binlog_close(bool shutdown);
+extern bool ibb_write_header_page(mtr_t *mtr, uint64_t file_no,
+ uint64_t file_size_in_pages, lsn_t start_lsn,
+ uint64_t gtid_state_interval_in_pages,
+ LF_PINS *pins);
+extern bool binlog_gtid_state(rpl_binlog_state_base *state, mtr_t *mtr,
+ fsp_binlog_page_entry * &block, uint32_t &page_no,
+ uint32_t &page_offset, uint64_t file_no);
+extern bool innodb_binlog_oob_ordered(THD *thd, const unsigned char *data,
+ size_t data_len, void **engine_data,
+ void **stm_start_data,
+ void **savepoint_data);
+extern bool innodb_binlog_oob(THD *thd, const unsigned char *data,
+ size_t data_len, void **engine_data);
+void ibb_savepoint_rollback(THD *thd, void **engine_data,
+ void **stmt_start_data, void **savepoint_data);
+extern void innodb_reset_oob(void **engine_data);
+extern void innodb_free_oob(void *engine_data);
+extern handler_binlog_reader *innodb_get_binlog_reader(bool wait_durable);
+extern void ibb_wait_durable_offset(uint64_t file_no, uint64_t wait_offset);
+extern void ibb_get_filename(char name[FN_REFLEN], uint64_t file_no);
+extern binlog_oob_context *innodb_binlog_trx(trx_t *trx, mtr_t *mtr);
+extern void innodb_binlog_post_commit(mtr_t *mtr, binlog_oob_context *c);
+extern bool innobase_binlog_write_direct_ordered
+ (IO_CACHE *cache, handler_binlog_event_group_info *binlog_info,
+ const rpl_gtid *gtid);
+extern bool innobase_binlog_write_direct
+ (IO_CACHE *cache, handler_binlog_event_group_info *binlog_info,
+ const rpl_gtid *gtid);
+extern void ibb_group_commit(THD *thd,
+ handler_binlog_event_group_info *binlog_info);
+extern bool ibb_write_xa_prepare_ordered(THD *thd,
+ handler_binlog_event_group_info *binlog_info,
+ uchar engine_count);
+extern bool ibb_write_xa_prepare(THD *thd,
+ handler_binlog_event_group_info *binlog_info,
+ uchar engine_count);
+extern bool ibb_xa_rollback_ordered(THD *thd, const XID *xid,
+ void **engine_data);
+extern bool ibb_xa_rollback(THD *thd, const XID *xid, void **engine_data);
+extern void ibb_binlog_unlog(const XID *xid, void **engine_data);
+extern bool innodb_find_binlogs(uint64_t *out_first, uint64_t *out_last);
+extern void innodb_binlog_status(uint64_t *out_file_no, uint64_t *out_pos);
+extern bool innodb_binlog_get_init_state(rpl_binlog_state_base *out_state);
+extern bool innodb_reset_binlogs();
+extern int innodb_binlog_purge(handler_binlog_purge_info *purge_info);
+extern bool binlog_recover_write_data(bool space_id, uint32_t page_no,
+ uint16_t offset,
+ lsn_t start_lsn, lsn_t lsn,
+ const byte *buf, size_t size) noexcept;
+extern void binlog_recover_end(lsn_t lsn) noexcept;
+
+#endif /* innodb_binlog_h */
diff --git a/storage/innobase/include/log0log.h b/storage/innobase/include/log0log.h
index e80011a9c4c50..11b034bb1a4af 100644
--- a/storage/innobase/include/log0log.h
+++ b/storage/innobase/include/log0log.h
@@ -102,6 +102,10 @@ or the MySQL version that created the redo log file. */
#define LOG_HEADER_CREATOR_END 48
/* @} */
+/** Fake tablespace id for InnoDB-implemented binlog files. */
+static constexpr uint32_t LOG_BINLOG_ID_0= SRV_SPACE_ID_UPPER_BOUND;
+static constexpr uint32_t LOG_BINLOG_ID_1= SRV_SPACE_ID_UPPER_BOUND + 1;
+
struct log_t;
/** File abstraction */
diff --git a/storage/innobase/include/log0recv.h b/storage/innobase/include/log0recv.h
index 457218656f439..f87ca043697a8 100644
--- a/storage/innobase/include/log0recv.h
+++ b/storage/innobase/include/log0recv.h
@@ -333,6 +333,17 @@ struct recv_sys_t
void parse_page0(const page_id_t id, const byte *b, bool size, bool flags)
noexcept;
+ /** Pass a binlog recovery record to the binlog implementation.
+ @param space_id binlog file identifier
+ @paral l log record
+ @param rlen record length
+ @param page_no page modified by the record
+ @param start_lsn LSN at start of record
+ @param lsn LSN at end of record
+ @return whether record was found corrupt */
+ bool parse_store_binlog(uint32_t space_id, const byte *l, uint32_t rlen,
+ uint32_t page_no, lsn_t start_lsn, lsn_t lsn);
+
/** @return whether parse_store() needs to be invoked
@param space_id tablespace identifier */
bool parse_store_if_exists(uint32_t space_id) const noexcept;
diff --git a/storage/innobase/include/mtr0log.h b/storage/innobase/include/mtr0log.h
index 84dd2fff4feb7..518b2523f1748 100644
--- a/storage/innobase/include/mtr0log.h
+++ b/storage/innobase/include/mtr0log.h
@@ -313,9 +313,13 @@ inline byte *mtr_t::log_write(const page_id_t id, const buf_page_t *bpage,
{
static_assert(!(type & 15) && type != RESERVED &&
type <= FILE_CHECKPOINT, "invalid type");
- ut_ad(type >= FILE_CREATE || is_named_space(id.space()));
+ ut_ad(type >= FILE_CREATE || is_named_space(id.space()) ||
+ id.space() == LOG_BINLOG_ID_0 ||
+ id.space() == LOG_BINLOG_ID_1);
ut_ad(!bpage || bpage->id() == id);
- ut_ad(id < end_page_id);
+ ut_ad(id < end_page_id ||
+ id.space() == LOG_BINLOG_ID_0 ||
+ id.space() == LOG_BINLOG_ID_1);
constexpr bool have_len= type != INIT_PAGE && type != FREE_PAGE;
constexpr bool have_offset= type == WRITE || type == MEMSET ||
type == MEMMOVE;
@@ -323,9 +327,11 @@ inline byte *mtr_t::log_write(const page_id_t id, const buf_page_t *bpage,
ut_ad(have_len || len == 0);
ut_ad(have_len || !alloc);
ut_ad(have_offset || offset == 0);
- ut_ad(offset + len <= srv_page_size);
+ ut_ad(offset + len <= srv_page_size ||
+ (type == WRITE && id.space() >= LOG_BINLOG_ID_0));
static_assert(MIN_4BYTE >= UNIV_PAGE_SIZE_MAX, "consistency");
ut_ad(type == FREE_PAGE || type == OPTION || (type == EXTENDED && !bpage) ||
+ (type == WRITE && id.space() >= LOG_BINLOG_ID_0) ||
memo_contains_flagged(bpage, MTR_MEMO_MODIFY));
size_t max_len;
if (!have_len)
diff --git a/storage/innobase/include/mtr0mtr.h b/storage/innobase/include/mtr0mtr.h
index 82756ee43cf0d..540a7ea8a5bb0 100644
--- a/storage/innobase/include/mtr0mtr.h
+++ b/storage/innobase/include/mtr0mtr.h
@@ -31,6 +31,9 @@ Created 11/26/1995 Heikki Tuuri
#include "buf0buf.h"
#include "small_vector.h"
+struct fsp_binlog_page_entry;
+
+
/** Start a mini-transaction. */
#define mtr_start(m) (m)->start()
@@ -644,6 +647,10 @@ struct mtr_t {
/** Note that log_sys.latch is no longer being held exclusively. */
void flag_wr_unlock() noexcept { ut_ad(m_latch_ex); m_latch_ex= false; }
+ /* Binlog page release at mtr commit. */
+ fsp_binlog_page_entry *get_binlog_page() { return m_binlog_page; }
+ void set_binlog_page(fsp_binlog_page_entry *page) { m_binlog_page= page; }
+
private:
/** Handle any pages that were freed during the mini-transaction. */
void process_freed_pages();
@@ -725,6 +732,16 @@ struct mtr_t {
@param size total size of the record
@return the log record payload after the encoded length */
static const byte *parse_length(const byte *l, uint32_t *size) noexcept;
+
+ /** Write binlog data
+ @param page_id binlog file id and page number
+ @param offset offset within the page
+ @param buf data
+ @param size size of data
+ @return */
+ void write_binlog(page_id_t page_id, uint16_t offset,
+ const void *buf, size_t size) noexcept;
+
private:
/** Release all latches. */
@@ -800,4 +817,6 @@ struct mtr_t {
fil_space_t *m_freed_space= nullptr;
/** set of freed page ids */
range_set *m_freed_pages= nullptr;
+ /** Latched binlog page to release at mtr commit*/
+ fsp_binlog_page_entry *m_binlog_page;
};
diff --git a/storage/innobase/include/small_vector.h b/storage/innobase/include/small_vector.h
index 2acdc49f6682f..9060fedca2106 100644
--- a/storage/innobase/include/small_vector.h
+++ b/storage/innobase/include/small_vector.h
@@ -19,6 +19,7 @@ this program; if not, write to the Free Software Foundation, Inc.,
#pragma once
/* A normally small vector, inspired by llvm::SmallVector */
#include "my_global.h"
+#include "my_valgrind.h"
#include
#include
diff --git a/storage/innobase/include/trx0trx.h b/storage/innobase/include/trx0trx.h
index 438d8828ab97e..791717e9e6c48 100644
--- a/storage/innobase/include/trx0trx.h
+++ b/storage/innobase/include/trx0trx.h
@@ -830,6 +830,8 @@ struct trx_t : ilist_node<>
rollback. */
/** whether this is holding the prepare mutex */
bool active_commit_ordered;
+ /** whether innobase_xa_prepare() was done. */
+ bool active_prepare;
/*------------------------------*/
bool flush_log_later;/* In 2PC, we hold the
prepare_commit mutex across
diff --git a/storage/innobase/include/univ.i b/storage/innobase/include/univ.i
index 490f71653f70b..dc453ac9936a8 100644
--- a/storage/innobase/include/univ.i
+++ b/storage/innobase/include/univ.i
@@ -481,6 +481,12 @@ extern mysql_pfs_key_t trx_pool_mutex_key;
extern mysql_pfs_key_t trx_pool_manager_mutex_key;
extern mysql_pfs_key_t lock_wait_mutex_key;
extern mysql_pfs_key_t srv_threads_mutex_key;
+extern mysql_pfs_key_t fsp_active_binlog_mutex_key;
+extern mysql_pfs_key_t fsp_binlog_durable_mutex_key;
+extern mysql_pfs_key_t fsp_binlog_durable_cond_key;
+extern mysql_pfs_key_t fsp_purge_binlog_mutex_key;
+extern mysql_pfs_key_t fsp_page_fifo_mutex_key;
+extern mysql_pfs_key_t ibb_xid_hash_mutex_key;
# endif /* UNIV_PFS_MUTEX */
# ifdef UNIV_PFS_RWLOCK
diff --git a/storage/innobase/include/ut0new.h b/storage/innobase/include/ut0new.h
index 398dd0dcc9ecd..bcc129601d11f 100644
--- a/storage/innobase/include/ut0new.h
+++ b/storage/innobase/include/ut0new.h
@@ -165,6 +165,7 @@ ut_allocator::get_mem_key()):
happens then that means that the list of predefined names must be extended.
Keep this list alphabetically sorted. */
extern PSI_memory_key mem_key_ahi;
+extern PSI_memory_key mem_key_binlog;
extern PSI_memory_key mem_key_buf_buf_pool;
extern PSI_memory_key mem_key_dict_stats_bg_recalc_pool_t;
extern PSI_memory_key mem_key_dict_stats_index_map_t;
@@ -849,6 +850,7 @@ constexpr const char* const auto_event_names[] =
"fil0crypt",
"fil0fil",
"fsp0file",
+ "fsp_binlog",
"fts0ast",
"fts0blex",
"fts0config",
@@ -860,6 +862,7 @@ constexpr const char* const auto_event_names[] =
"fts0sql",
"fts0tlex",
"gis0sea",
+ "innodb_binlog",
"ha_innodb",
"handler0alter",
"hash0hash",
diff --git a/storage/innobase/log/log0recv.cc b/storage/innobase/log/log0recv.cc
index 9eb286b4d5516..1ed17f20fd835 100644
--- a/storage/innobase/log/log0recv.cc
+++ b/storage/innobase/log/log0recv.cc
@@ -53,6 +53,7 @@ Created 9/20/1997 Heikki Tuuri
#include "srv0srv.h"
#include "srv0start.h"
#include "fil0pagecompress.h"
+#include "innodb_binlog.h"
#include "log.h"
/** The recovery system */
@@ -2579,6 +2580,25 @@ void recv_sys_t::parse_page0(const page_id_t id, const byte *b,
fil_space_set_recv_size_and_flags(space_id, s, f);
}
+ATTRIBUTE_COLD
+ATTRIBUTE_NOINLINE
+bool recv_sys_t::parse_store_binlog(uint32_t space_id, const byte *l,
+ uint32_t rlen, uint32_t page_no,
+ lsn_t start_lsn, lsn_t lsn)
+{
+ const size_t olen= mlog_decode_varint_length(*l);
+ if (UNIV_UNLIKELY(olen >= rlen) || UNIV_UNLIKELY(olen > 3))
+ return true;
+ const uint32_t offset= mlog_decode_varint(l);
+ ut_ad(offset != MLOG_DECODE_ERROR);
+ if (UNIV_UNLIKELY(offset + rlen - olen >= 65535))
+ return true;
+ if (binlog_recover_write_data(space_id & 1, page_no, uint16_t(offset),
+ start_lsn, lsn, l + olen, rlen - olen))
+ return true;
+ return false;
+}
+
ATTRIBUTE_NOINLINE
bool recv_sys_t::parse_store_if_exists(uint32_t space_id) const noexcept
{
@@ -2773,7 +2793,8 @@ log_parse_file(const page_id_t id, bool if_exists,
if (space_id == TRX_SYS_SPACE || srv_is_undo_tablespace(space_id))
goto file_rec_error;
- if (fnend - l < 4 || memcmp(fnend - 4, DOT_IBD, 4))
+ if (fnend - l < 4 ||
+ (memcmp(fnend - 4, DOT_IBD, 4) && memcmp(fnend - 4, DOT_IBB, 4)))
goto file_rec_error;
if (UNIV_UNLIKELY(!recv_needed_recovery && srv_read_only_mode))
@@ -2825,7 +2846,7 @@ static recv_sys_t::parse_mtr_result
log_page_modify(uint32_t space_id, uint32_t page_no) noexcept
{
ut_ad(space_id);
- if (srv_is_undo_tablespace(space_id))
+ if (space_id >= SRV_SPACE_ID_UPPER_BOUND || srv_is_undo_tablespace(space_id))
return recv_sys_t::OK;
recv_spaces_t::iterator i= recv_spaces.lower_bound(space_id);
const lsn_t lsn{recv_sys.lsn};
@@ -2910,6 +2931,7 @@ recv_sys_t::parse_tail(const byte *begin, bool if_exists, size_t size) noexcept
return r;
l= mtr_t::parse_length(l, &rlen);
+ bool is_binlog= false;
uint32_t idlen;
if ((b & 0x80) && got_page_op)
{
@@ -2945,6 +2967,8 @@ recv_sys_t::parse_tail(const byte *begin, bool if_exists, size_t size) noexcept
space_id= mlog_decode_varint(l);
if (UNIV_UNLIKELY(space_id == MLOG_DECODE_ERROR))
goto page_id_corrupted;
+ static_assert((LOG_BINLOG_ID_0 | 1) == LOG_BINLOG_ID_1, "");
+ is_binlog= !ENC_10_8 && (space_id | 1) == LOG_BINLOG_ID_1;
l+= idlen;
rlen-= idlen;
idlen= mlog_decode_varint_length(*l);
@@ -2995,7 +3019,7 @@ recv_sys_t::parse_tail(const byte *begin, bool if_exists, size_t size) noexcept
}
same_page:
if (!rlen);
- else if (UNIV_UNLIKELY(size_t(l - recs) + rlen > srv_page_size))
+ else if (!is_binlog && UNIV_UNLIKELY(size_t(l - recs) + rlen > srv_page_size))
goto record_corrupted;
const page_id_t id{space_id, page_no};
ut_d(if ((b & 0x70) == INIT_PAGE || (b & 0x70) == OPTION)
@@ -3085,12 +3109,15 @@ recv_sys_t::parse_tail(const byte *begin, bool if_exists, size_t size) noexcept
/* fall through */
case RESERVED:
continue;
- case WRITE:
case MEMMOVE:
case MEMSET:
+ if (storing == YES && !ENC_10_8 && is_binlog)
+ goto record_corrupted;
+ /* fall through */
+ case WRITE:
if (storing == BACKUP)
continue;
- if (storing == NO && UNIV_LIKELY(page_no != 0))
+ if (storing == NO && UNIV_LIKELY((page_no | (uint32_t)is_binlog) != 0))
/* fil_space_set_recv_size_and_flags() is mandatory for storing==NO.
It is only applicable to page_no == 0. Other than that, we can just
ignore the payload and only compute the mini-transaction checksum;
@@ -3100,24 +3127,28 @@ recv_sys_t::parse_tail(const byte *begin, bool if_exists, size_t size) noexcept
goto record_corrupted;
if (ENC_10_8)
cl= log_decrypt_legacy(iv, recs, l, rlen, decrypt_buf);
- const uint32_t olen= mlog_decode_varint_length(*(ENC_10_8 ? cl : l));
- if (UNIV_UNLIKELY(olen >= rlen) || UNIV_UNLIKELY(olen > 3))
- goto record_corrupted;
- const uint32_t offset= mlog_decode_varint(ENC_10_8 ? cl : l);
- ut_ad(offset != MLOG_DECODE_ERROR);
- static_assert(FIL_PAGE_OFFSET == 4, "compatibility");
- if (UNIV_UNLIKELY(offset >= srv_page_size))
- goto record_corrupted;
- last_offset+= offset;
- if (UNIV_UNLIKELY(last_offset < 8 || last_offset >= srv_page_size))
- goto record_corrupted;
- (ENC_10_8 ? cl : l)+= olen;
- rlen-= olen;
+ if (!is_binlog)
+ {
+ const uint32_t olen= mlog_decode_varint_length(*(ENC_10_8 ? cl : l));
+ if (UNIV_UNLIKELY(olen >= rlen) || UNIV_UNLIKELY(olen > 3))
+ goto record_corrupted;
+ const uint32_t offset= mlog_decode_varint(ENC_10_8 ? cl : l);
+ ut_ad(offset != MLOG_DECODE_ERROR);
+ static_assert(FIL_PAGE_OFFSET == 4, "compatibility");
+ if (UNIV_UNLIKELY(offset >= srv_page_size))
+ goto record_corrupted;
+ last_offset+= offset;
+ if (UNIV_UNLIKELY(last_offset < 8 || last_offset >= srv_page_size))
+ goto record_corrupted;
+ (ENC_10_8 ? cl : l)+= olen;
+ rlen-= olen;
+ }
if ((b & 0x70) == WRITE)
{
- if (UNIV_UNLIKELY(rlen + last_offset > srv_page_size))
+ if (!ENC_10_8 && is_binlog);
+ else if (UNIV_UNLIKELY(rlen + last_offset > srv_page_size))
goto record_corrupted;
- if (UNIV_UNLIKELY(!page_no) && file_checkpoint)
+ else if (UNIV_UNLIKELY(!page_no) && file_checkpoint)
{
const bool has_size= last_offset <= FSP_HEADER_OFFSET + FSP_SIZE &&
last_offset + rlen >= FSP_HEADER_OFFSET + FSP_SIZE + 4;
@@ -3142,7 +3173,7 @@ recv_sys_t::parse_tail(const byte *begin, bool if_exists, size_t size) noexcept
goto record_corrupted;
const uint32_t len= mlog_decode_varint(ENC_10_8 ? cl : l);
ut_ad(len != MLOG_DECODE_ERROR);
- if (UNIV_UNLIKELY(last_offset + len > srv_page_size))
+ if (UNIV_UNLIKELY(last_offset + len > srv_page_size) || is_binlog)
goto record_corrupted;
(ENC_10_8 ? cl : l)+= llen;
rlen-= llen;
@@ -3179,6 +3210,11 @@ recv_sys_t::parse_tail(const byte *begin, bool if_exists, size_t size) noexcept
}
#endif
if (storing != YES);
+ else if (!ENC_10_8 && is_binlog)
+ {
+ if (parse_store_binlog(space_id, l, rlen, page_no, start_lsn, lsn))
+ goto record_corrupted;
+ }
else if (if_exists && !parse_store_if_exists(space_id));
else if (UNIV_UNLIKELY(parse_store(id, ENC_10_8 && l != cl
? decrypt_buf : recs,
@@ -4400,6 +4436,7 @@ static bool recv_scan_log(bool last_phase, const recv_sys_t::parser *parser)
ut_ad(!rewound_lsn);
ut_ad(recv_sys.lsn >= recv_sys.file_checkpoint);
log_sys.set_recovered_lsn(recv_sys.lsn);
+ binlog_recover_end(recv_sys.lsn);
}
else if (rewound_lsn)
{
@@ -4407,6 +4444,9 @@ static bool recv_scan_log(bool last_phase, const recv_sys_t::parser *parser)
ut_ad(recv_sys.file_checkpoint);
recv_sys.lsn= rewound_lsn;
}
+ else if (store)
+ binlog_recover_end(recv_sys.lsn);
+
func_exit:
ut_d(recv_sys.after_apply= last_phase);
mysql_mutex_unlock(&recv_sys.mutex);
diff --git a/storage/innobase/mtr/mtr0mtr.cc b/storage/innobase/mtr/mtr0mtr.cc
index 952850b3f499d..46944962bafce 100644
--- a/storage/innobase/mtr/mtr0mtr.cc
+++ b/storage/innobase/mtr/mtr0mtr.cc
@@ -36,6 +36,7 @@ Created 11/26/1995 Heikki Tuuri
#endif
#include "srv0start.h"
#include "trx0trx.h"
+#include "fsp_binlog.h"
#include "log.h"
#include "my_cpu.h"
@@ -208,6 +209,7 @@ void mtr_t::start()
m_user_space= nullptr;
m_commit_lsn= 0;
m_trim_pages= false;
+ m_binlog_page= nullptr;
}
/** Release the resources */
@@ -217,6 +219,11 @@ inline void mtr_t::release_resources()
ut_ad(m_memo.empty());
m_log.erase();
ut_d(m_commit= true);
+ if (m_binlog_page)
+ {
+ fsp_binlog_release(m_binlog_page);
+ m_binlog_page= nullptr;
+ }
}
/** Handle any pages that were freed during the mini-transaction. */
diff --git a/storage/innobase/os/os0file.cc b/storage/innobase/os/os0file.cc
index 8cf761545f3f1..29b78b8a24eda 100644
--- a/storage/innobase/os/os0file.cc
+++ b/storage/innobase/os/os0file.cc
@@ -1541,6 +1541,12 @@ bool os_file_set_size(const char *name, os_file_t file, os_offset_t size,
return success;
}
+ DBUG_EXECUTE_IF(
+ "ib_alloc_file_disk_full",
+ errno = ENOSPC;
+ return(false);
+ );
+
# ifdef HAVE_POSIX_FALLOCATE
int err;
os_offset_t current_size;
diff --git a/storage/innobase/srv/srv0start.cc b/storage/innobase/srv/srv0start.cc
index 318357432cfdf..df2b2c1753e96 100644
--- a/storage/innobase/srv/srv0start.cc
+++ b/storage/innobase/srv/srv0start.cc
@@ -87,6 +87,7 @@ Created 2/16/1996 Heikki Tuuri
#include "row0mysql.h"
#include "btr0pcur.h"
#include "ibuf0ibuf.h"
+#include "innodb_binlog.h"
#include "zlib.h"
#include "log.h"
@@ -1970,6 +1971,14 @@ dberr_t srv_start(bool create_new_db)
return(srv_init_abort(err));
}
+ err= innodb_binlog_startup_init();
+ if (UNIV_UNLIKELY(err != DB_SUCCESS))
+ {
+ sql_print_error("InnoDB: Could not initialize the binlog in InnoDB, "
+ "aborting");
+ return(srv_init_abort(DB_ERROR));
+ }
+
if (!srv_read_only_mode
&& srv_operation <= SRV_OPERATION_EXPORT_RESTORED) {
/* Initialize the innodb_temporary tablespace and keep
@@ -2088,6 +2097,7 @@ void innodb_shutdown()
logs_empty_and_mark_files_at_shutdown();
}
+ innodb_binlog_close(true);
os_aio_free();
fil_space_t::close_all();
/* Exit any remaining threads. */
diff --git a/storage/innobase/trx/trx0trx.cc b/storage/innobase/trx/trx0trx.cc
index a774901dc91a7..6cf78810af323 100644
--- a/storage/innobase/trx/trx0trx.cc
+++ b/storage/innobase/trx/trx0trx.cc
@@ -49,6 +49,7 @@ Created 3/26/1996 Heikki Tuuri
#include "trx0xa.h"
#include "ut0pool.h"
#include "ut0vec.h"
+#include "innodb_binlog.h"
#include "log.h"
#include
@@ -108,6 +109,8 @@ trx_init(
trx->active_commit_ordered = false;
+ trx->active_prepare = false;
+
trx->isolation_level = TRX_ISO_REPEATABLE_READ;
trx->check_foreigns = true;
@@ -414,6 +417,7 @@ void trx_t::free()
bulk_insert */);
MEM_NOACCESS(&is_registered, sizeof is_registered);
MEM_NOACCESS(&active_commit_ordered, sizeof active_commit_ordered);
+ MEM_NOACCESS(&active_prepare, sizeof active_prepare);
MEM_NOACCESS(&flush_log_later, sizeof flush_log_later);
MEM_NOACCESS(&duplicates, sizeof duplicates);
MEM_NOACCESS(&dict_operation, sizeof dict_operation);
@@ -1130,6 +1134,7 @@ inline void trx_t::write_serialisation_history(mtr_t *mtr)
ut_ad(!read_only);
trx_rseg_t *rseg= rsegs.m_redo.rseg;
trx_undo_t *&undo= rsegs.m_redo.undo;
+ binlog_oob_context *binlog_ctx= nullptr;
if (UNIV_LIKELY(undo != nullptr))
{
MONITOR_INC(MONITOR_TRX_COMMIT_UNDO);
@@ -1169,6 +1174,11 @@ inline void trx_t::write_serialisation_history(mtr_t *mtr)
}
else
trx_sys.assign_new_trx_no(this);
+
+ /* Include binlog data in the commit record, if any. */
+ if (active_commit_ordered)
+ binlog_ctx= innodb_binlog_trx(this, mtr);
+
UT_LIST_REMOVE(rseg->undo_list, undo);
/* Change the undo log segment state from TRX_UNDO_ACTIVE, to
define the transaction as committed in the file based domain,
@@ -1182,6 +1192,7 @@ inline void trx_t::write_serialisation_history(mtr_t *mtr)
rseg->release();
mtr->commit();
commit_lsn= undo_no || !xid.is_null() ? mtr->commit_lsn() : 0;
+ innodb_binlog_post_commit(mtr, binlog_ctx);
}
/********************************************************************
@@ -1736,7 +1747,7 @@ void trx_commit_complete_for_mysql(trx_t *trx)
case 0:
return;
case 1:
- if (trx->active_commit_ordered)
+ if (trx->active_commit_ordered && trx->active_prepare)
return;
}
trx_flush_log_if_needed(lsn, trx);
diff --git a/storage/innobase/ut/ut0new.cc b/storage/innobase/ut/ut0new.cc
index a3ce1bdf3c767..ad575016aeca7 100644
--- a/storage/innobase/ut/ut0new.cc
+++ b/storage/innobase/ut/ut0new.cc
@@ -38,6 +38,7 @@ Keep this list alphabetically sorted. */
#ifdef BTR_CUR_HASH_ADAPT
PSI_memory_key mem_key_ahi;
#endif /* BTR_CUR_HASH_ADAPT */
+PSI_memory_key mem_key_binlog;
PSI_memory_key mem_key_buf_buf_pool;
PSI_memory_key mem_key_dict_stats_bg_recalc_pool_t;
PSI_memory_key mem_key_dict_stats_index_map_t;
@@ -65,6 +66,7 @@ static PSI_memory_info pfs_info[] = {
#ifdef BTR_CUR_HASH_ADAPT
{&mem_key_ahi, "adaptive hash index", 0},
#endif /* BTR_CUR_HASH_ADAPT */
+ {&mem_key_binlog, "innodb binlog implementation", 0},
{&mem_key_buf_buf_pool, "buf_buf_pool", 0},
{&mem_key_dict_stats_bg_recalc_pool_t, "dict_stats_bg_recalc_pool_t", 0},
{&mem_key_dict_stats_index_map_t, "dict_stats_index_map_t", 0},
diff --git a/support-files/magic b/support-files/magic
index accc3999ddbfa..0a9da4ca24391 100644
--- a/support-files/magic
+++ b/support-files/magic
@@ -24,3 +24,7 @@
>3 byte x Version %d
0 belong&0xffffff00 0xfefe0b00 MariaDB DDL recovery log
>3 byte x Version %d
+0 belong&0xffffff00 0xfefe0c00 MariaDB GTID index file
+>3 byte x Version %d
+0 belong&0xffffff00 0xfefe0d01 MariaDB InnoDB new binlog format
+>3 byte x Version %d