Uploaded image for project: 'MariaDB Server'
  1. MariaDB Server
  2. MDEV-14425

Change the InnoDB redo log format to reduce write amplification

Details

    Description

      The InnoDB redo log format is not optimal in many respects:

      • At the start of ib_logfile0, there are two log checkpoint blocks, only 1024 bytes apart, while there exist devices with 4096-byte block size. The rest of the log file is written in a circular fashion.
      • On log checkpoint, some file name information needs to be appended to the log.
      • File names that were first changed since the latest checkpoint must be appended to the log. The bookkeeping causes some contention on log_sys.mutex and fil_system.mutex. Edit: The contention on fil_system.mutex was practically removed in MDEV-23855, and the contention on log_sys.mutex due to this is minimal.
      • The log file was unnecessarily split into multiple files, logically treated as one big circular file. (MDEV-20907 in MariaDB Server 10.5.0 change the default to 1 file, and later the parameter was deprecated and ignored.)
      • Log records are divided into tiny blocks of 512 bytes, with 12+4 bytes of header and footer (12+8 bytes with MDEV-12041 innodb_encrypt_log (10.4.0)).
      • We are holding a mutex while zero-filling unused parts of log blocks, encrypting log blocks, or computing checksums.
      • We were holding an exclusive latch while copying log blocks; this was fixed in MDEV-27774.
      • Mariabackup cannot copy the log without having access to the encryption keys. (It can copy data file pages without encrypting them.)

      We had some ideas to move to an append-only file and to partition the log into multiple files, but it turned out that a single fixed-size circular log file would perform best in typical scenarios.

      To address the fil_system.mutex contention whose root cause was later fixed in MDEV-23855, we were considering to split the log as follows:

      • ib_logfile0 (after the 512-byte header) will be append-only, unencrypted, for records containing file names and checkpoint information. A checkpoint record will comprise an LSN and a byte offset in a separate, optionally encrypted, circular log file ib_logdata. The length of each record is explicitly tagged and the payload will be followed by CRC-32C.
      • The ib_logdata file can be append-only or circular. If it is circular, its fixed size must be an integer multiple of 512 bytes.

      One problem would have had to be solved: When would the ib_logfile0 be shrunk? No storage is unlimited.

      We will retain the ib_logfile0 and the basic format of its first 512 bytes for compatibility purposes, but other features could be improved.

      • We remove log block headers and footers. We really only need is to detect the logical end of the circular log. That can be achieved by making sure that mini-transactions are terminated by a sequence number (at least one bit) and a checksum. When the circular file wraps around, the sequence number will be incremented (or the sequence bit toggled).
      • For page-aligned I/O, we allow dummy records to be written, to indicate that the next bytes (until the end of the physical block, no matter what the I/O block size is) must be ignored. (The log parser will ignore these padding records, but we do not currently write them; we will keep overwriting the last physical block until it has been completely filled like we used to do until now.)
      • Encrypt and compute checksum on mtr_t::m_log before initiating a write to the circular log file. The log can be copied and checksum validated without access to encryption keys.
      • If the log is on a memory-mapped persistent memory device, then we will make log_sys.buf point directly to the persistent memory.

      Some old InnoDB redo log parameters were removed in MDEV-23397 (MariaDB 10.6.0). Some more parameters will removed or changed here:

      • innodb_log_write_ahead_size: Removed. On Linux and Microsoft Windows, we will detect and use the physical block size of the underlying storage. We will also remove the log_padded counter from INFORMATION_SCHEMA.INNODB_METRICS.
      • innodb_log_file_buffering: Added (MDEV-28766). This controls the use of O_DIRECT on the ib_logfile0 when the physical block size can be determined
      • innodb_log_buffer_size: The minimum value is raised to 2MiB and the granularity increased from 1024 to 4096 bytes. This buffer will also be used during recovery. Ignored when the log is memory-mapped (on PMEM or /dev/shm).
      • innodb_log_file_size: The allocation granularity is reduced from 1MiB to 4KiB.

      Some global variables will be adjusted as well:

      • Innodb_os_log_fsyncs: Removed. This will be included in Innodb_data_fsyncs.
      • Innodb_os_log_pending_fsyncs: Removed. This was limited to at most 1 by design.
      • Innodb_log_pending_writes: Removed. This was limited to at most 1 by design.

      The circular log file ib_logfile0

      The file name ib_logfile0 and the existing format of the first 512 bytes will be retained for the purpose of upgrading and preventing downgrading. In the first 512 bytes of the file, the following information will be present:

      • InnoDB redo log format version identifier (in the format introduced by MySQL 5.7.9/MariaDB 10.2.2)
      • CRC-32C checksum

      After the first 512 bytes, there will be two 64-byte checkpoint blocks at the byte offsets 4096 and 8192, containing:

      • The checkpoint LSN
      • The LSN at the time the checkpoint was created, pointing to an optional sequence of FILE_MODIFY records and a FILE_CHECKPOINT record

      The circular redo log record area starts at offset 12288 and extends to the end of the file. Unless the file was created by mariadb-backup, the file size will be a multiple of 4096 bytes.

      All writes to ib_logfile0 will be synchronous and durable (O_DSYNC, fdatasync() or O_SYNC, fsync() or pmem_persist()).

      Payload encoding

      The payload area will contain records in the MDEV-12353 format. Each mini-transaction will be followed by a sequence byte 0x00 or 0x01 (the value of the sequence bit), optionally (if the log is encrypted) a 8-byte nonce, and a CRC-32C of all the bytes (except the sequence byte), so that backup can avoid recomputing the checksum while copying the log to a new file.

      We want to be able to avoid overwriting the last log block, so we cannot have an explicit 'end of log' marker. We must associate each mini-transaction (atomic sequence of log records) with a sequence number (at the minimum, a sequence bit) and a checksum. The 4-byte CRC-32C is a good candidate, because it is already being used in data page checksums.

      Padding

      We might want to introduce a special mini-transaction 'Skip the next N bytes', encoded in sizeof(CRC)+2+log(N) bytes: CRC, record type and length, subtype and the value of the sequence bit, and variable-length encoded N. However, for a compressed storage device, it would be helpful to not have any garbage bytes in the log file. It would be better to initialize all those N bytes.

      If we need to pad a block with fewer bytes than the minimum size, we would write a record to skip the minimum size.

      This has been implemented with arbitrary-length FILE_CHECKPOINT mini-transactions whose payload consists of NUL bytes. The parser will ignore such records. We are not currently writing such records, but instead overwriting the last incomplete log block when more log is being appended, just like InnoDB always did.

      Mini-transaction encoding: Prepending or appending a CRC to each MDEV-12353 mini-transaction

      In the MDEV-12353 encoding, a record cannot start with the bytes 0x00 or 0x01. Mini-transactions are currently being terminated by the byte 0x00. We could store the sequence bit in the terminating byte of the mini-transaction. The checksum would exclude the terminating byte.

      Only the payload bytes would be encrypted (not record types or lengths, and not page identifiers either). In that way, records can be parsed and validated efficiently. Decryption would only have to be invoked when the log really needs to be applied on the page. The initialization vector for encryption and decryption can include the unencrypted record header bytes.

      It could be best to store the CRC before the mini-transaction payload, because the CRC of non-zero bytes cannot be 0. Hence, we can detect the end of the log without even parsing the mini-transaction bytes.

      Pros: Minimal overhead: sizeof(CRC) bytes per mini-transaction.
      Cons: Recovery may have to parse a lot of log before determining that the end of the log was reached.

      In the end, the CRC was written after the mini-transaction. The log parser can flag an inconsistency if the maximum mini-transaction size would be exceeded.

      Alternative encoding (scrapped idea): Prepending a mini-transaction header with length and CRC

      We could encapsulate MDEV-12353 records (without the mini-transaction terminating NUL byte) in the following structure:

      • variable-length encoded integer of total_length << 2 | sequence_bit
      • CRC of the data payload and the variable-length encoded integer
      • the data payload (MDEV-12353 records); could be encrypted in their entirety

      Skipped bytes (at least 5) would be indicated by the following:

      • variable-length encoded integer of skipped_length << 2 | 1 << 1 | sequence_bit
      • CRC of the variable-length encoded integer (not including the skipped bytes)

      Pros: Recovery can determine more quickly that the end of the circular log was reached, thanks to the length, sequence bit and (nonzero) CRC being stored at the start.
      Pros: More of the log could be encrypted (at the cost of recovery and backup restoration speed)
      Cons: Increased storage overhead: sizeof(CRC)+log(length * 4) bytes. For length<32 bytes, no change of overhead.
      Cons: If the encryption is based on the current LSN, then both encryption and the checksum would have to be computed while holding log_sys.mutex.

      Log writing and synchronous flushing

      For the bulk of the changes done by mini-transactions, we do not care about flushing. The file system can write log file blocks as it pleases.

      Some state changes of the database must be made durable at a specific time. Examples include user transaction COMMIT, XA PREPARE, XA ROLLBACK, and (in case the binlog is not enabled) XA COMMIT.

      Whenever we want to make a certain change durable, we must flush all log files up to the LSN of the mini-transaction commit that made the change.

      If redo log is physically replicated to the buffer pools of physical replicas (like in Amazon Aurora or Alibaba PolarDB), then we should first write to the local log and only then to the replicas, and we should assume that the writes to the files will always eventually be durable. If that assumption is broken, then all servers would have to be restarted and perform crash recovery.

      Crash recovery and backup

      The previous two-stage parsing (log block validation and log record parsing) was replaced with a single stage. The separate 2-megabyte buffer recv_sys.buf is no longer needed, because the bytes of the log records will be stored contiguously, except when the log file wraps around from its end to the offset 12,288.

      When the log file is memory-mapped, we will parse records directly from log_sys.buf that contains a view of the entire log file. For parsing the mini-transaction that wraps from the end of the file to the start, the record parser will use a special pointer wrapper. When not using memory-mapping, we will read from the log file to log_sys.buf in such a way that the records of each mini-transaction will be contiguous.

      Crash-upgrade from earlier versions will not be supported. Before upgrading, the old server must have been shut down, or mariadb-backup --prepare must have been executed using an appropriate older version of the backup tool.

      Starting up without ib_logfile0 will no longer be supported; see also MDEV-27199.

      Attachments

        1. 81cf92e9471.pdf
          29 kB
        2. append.c
          0.6 kB
        3. MDEV-14425.pdf
          29 kB
        4. NUMA_1.pdf
          37 kB
        5. NUMA_1vs2.pdf
          29 kB
        6. NUMA_2.pdf
          38 kB
        7. preallocate.c
          0.6 kB

        Issue Links

          Activity

            marko Marko Mäkelä created issue -
            marko Marko Mäkelä made changes -
            Field Original Value New Value
            marko Marko Mäkelä made changes -
            Description The InnoDB redo log format is not optimal in many respects:
            * The log files are not append-only. Modern journaled file systems handle append-only files more efficiently.
            * There is no possibility to archive the redo log.
            * The LSN is based on bytes, and the log is not partitioned, making it hard to parallelize writes of concurrent mini-transaction commits.

            Let us introduce the following parameters:
            * innodb_log_file_size (in multiples of 4096 bytes; 0 (default) disables log rotation/archiving)
            * innodb_log_partitions (default=1) for allowing concurrent writes to different partitions

            Some old InnoDB redo log parameters will be removed (or deprecated and ignored):
            * innodb_log_files_in_group
            * innodb_log_write_ahead_size
            * innodb_log_checksums (if it performs well enough with hardware-assisted CRC-32C)

            h2. The idea: Partition the log into append-only, truncate-the-start files
            Append-only files are more efficient in modern crash-safe file systems. For example, journaled file systems will not need to write the data twice (first to the journal, then to the data blocks).

            The ib_logfile0 will be repurposed for upgrade and downgrade-prevention purposes. The size of this file will be one file system block (padded with NUL bytes). In the first 512 bytes of the file, the following information will be present:
            * InnoDB redo log format version identifier (in the format introduced by MySQL 5.7.9/MariaDB 10.2.2)
            * The value of innodb_log_partitions at the latest checkpoint
            * The LSN of the latest checkpoint
            * CRC-32C checksum
            * Optional: If innodb_encrypt_log, write the encryption key ID and encrypt the checkpoint LSN with that key.

            This file will be overwritten (O_DIRECT) at every redo log checkpoint.

            The redo log will be partitioned into files like innodb_%u.log with 0..innodb_log_partitions-1.

            If innodb_log_max_size is not at the default (0), then as soon as one log file would exceed the maximum size, all log files will be rotated by renaming to innodb_%u.%06u and by creating empty innodb_%u.log files.

            If innodb_log_max_size=0 (the default), then at checkpoint, the start of each innodb_%u.log file will be discarded by punching a hole from 0 to the block that contains the first record at or after the checkpoint LSN. If the file system does not support hole-punching, then at the start of the file a header will be written that points to the first block.

            h3. Crash recovery:
            # Read start_checkpoint_lsn from ib_logfile0
            # Process each redo log up to the maximum contiguous LSN

            Each mini-transaction log start record contains the LSN (a mtr_t::commit() counter)
            If some file contains entries after the maximum contiguous LSN, truncate the excess before allowing writes to the redo log.

            h3: Log checkpoint
            {code:c++}
              pwrite(ib_logfile0, page_with_checkpoint_lsn, 4096, 0);

              if (innodb_log_max_size) {
                for (auto log : logs) {
                  mach_write_to_8(log_header, log->start_block(checkpoint_lsn));
                  pwrite(log->fd, log_header, 4096, 0);
                }
              } else {
                for (auto log : logs) {
                  uint64_t offs = log->start_block(checkpoint_lsn);
                  fallocate(log->fd, FALLOC_FL_PUNCH_HOLE, 0, offs);
                }
              }
            {code}
            The new function
            {code:c++}
            uint64_t log_file_t::start_block(lsn_t) const
            {code}
            returns the file offset of the block that contains the (introduced in this work) MLOG_START(lsn) record. It likely will require some additional main-memory data structure, because we might reuse log_file_t::buffer entries after they have been written to the redo log, but before the log checkpoint is initiated.

            Each log block header could contain the smallest start LSN in that block. The log block format will be completely new.

            h2. TODO: formulate the following notes properly
            * For multi-page mini-transactions, retain mtr_t::log.
            * Single-page mini-transactions can write directly to the buffer.
            * mtr_t::commit(bool sync = false):
              {log_file,lsn} = log_sys->acquire_file(); // Atomic or using log_sys->mutex
              write_mlog_start(mtr_log, lsn); // First record of each mini-transaction
              pos = log_file->append_and_release(mtr_log, len);
              if (sync) log_file->flush(pos);
            * log_file_t::flush(uint64_t file_pos) will (encrypt and) write buf to the file
            The InnoDB redo log format is not optimal in many respects:
            * The log files are not append-only. Modern journaled file systems handle append-only files more efficiently.
            * There is no possibility to archive the redo log.
            * The LSN is based on bytes, and the log is not partitioned, making it hard to parallelize writes of concurrent mini-transaction commits.

            Let us introduce the following parameters:
            * innodb_log_file_size (in multiples of 4096 bytes; 0 (default) disables log rotation/archiving)
            * innodb_log_partitions (default=1) for allowing concurrent writes to different partitions

            Some old InnoDB redo log parameters will be removed (or deprecated and ignored):
            * innodb_log_files_in_group
            * innodb_log_write_ahead_size
            * innodb_log_checksums (if it performs well enough with hardware-assisted CRC-32C)

            h2. The idea: Partition the log into append-only, truncate-the-start files
            Append-only files are more efficient in modern crash-safe file systems. For example, file systems can avoid writing the data twice (first to the journal, then to the data blocks).

            The ib_logfile0 will be repurposed for the purpose of upgrading and preventing downgrading. The size of this file will be one file system block (padded with NUL bytes). In the first 512 bytes of the file, the following information will be present:
            * InnoDB redo log format version identifier (in the format introduced by MySQL 5.7.9/MariaDB 10.2.2)
            * The value of innodb_log_partitions at the latest checkpoint
            * The LSN of the latest checkpoint
            * CRC-32C checksum
            * Optional: If innodb_encrypt_log, write the encryption key ID and encrypt the checkpoint LSN with that key.

            This file will be overwritten (O_DIRECT) at every redo log checkpoint.

            The redo log will be partitioned into files like innodb_%u.log with 0..innodb_log_partitions-1. The log file will be chosen by pthread_self() % innodb_log_partitions.

            If innodb_log_max_size is not at the default (0), then as soon as one log file would exceed the maximum size, all log files will be rotated by renaming to innodb_%u.%06u and by creating empty innodb_%u.log files.

            If innodb_log_max_size=0 (the default), then at checkpoint, the start of each innodb_%u.log file will be discarded by punching a hole from 0 to the block that contains the first record at or after the checkpoint LSN. If the file system does not support hole-punching, then at the start of the file a header will be written that points to the first block.

            The log block format will be redesigned. The log block header may contain the following:
            * Log block size (the physical block size of the system that wrote the block)
            * Checksum
            * Pointer to the first MLOG_START record (or 0 if there is no such record in the page)
            * The smallest log block size is 512 bytes.
            * All-zero log blocks are silently ignored (treated as 512 bytes)

            Mini-transactions will always start with an MLOG_START(lsn) entry. The lsn is a global sequence number that is atomically incremented whenever mtr_t::commit() is about to write redo log.

            h3. Page flushing
            Before writing any dirty page to data file, ensure that _all_ log files have been flushed up to the page LSN.
            Thanks to this, recovery will not require contiguous LSN in all log files.
            Thanks to this, a persistent mtr_t::commit() (which changes user transaction state) will only have to flush the current redo log file, not all log files.

            h3. Crash recovery
            # Read start_checkpoint_lsn from ib_logfile0.
            # In each redo log file: Find the first MLOG_START record with lsn>= checkpoint_lsn
            # Process each redo log up to the end

            We do not require the LSN to be contiguous from checkpoint_lsn to the end.
            The log_sys->lsn will be initialized to the maximum LSN that was found.

            The MLOG_START records will be found by scanning the redo log blocks from a start offset onwards until a qualifying record is found.
            On checkpoint, the start of each log file will be truncated until a block that is near the MLOG_START(checkpoint_lsn), so that we will not have to scan from the start of each file.

            h3. Log checkpoint
            {code:c++}
              pwrite(ib_logfile0, page_with_checkpoint_lsn, 4096, 0);

              if (innodb_log_max_size) {
                for (auto log : logs) {
                  mach_write_to_8(log_header, log->start_block(checkpoint_lsn));
                  pwrite(log->fd, log_header, 4096, 0);
                }
              } else {
                for (auto log : logs) {
                  uint64_t offs = log->start_block(checkpoint_lsn);
                  fallocate(log->fd, FALLOC_FL_PUNCH_HOLE, 0, offs);
                }
              }
            {code}
            The new function
            {code:c++}
            uint64_t log_file_t::start_block(lsn_t) const
            {code}
            returns the file offset of the block that contains the (introduced in this work) MLOG_START(lsn) record, or an offset that is not much smaller than that. It likely will require some additional main-memory data structure, because we might reuse log_file_t::buffer entries after they have been written to the redo log, but before the log checkpoint is initiated.

            h3. Log buffering
            For multi-page mini-transactions (such as B-tree page split or merge), we can have a local redo log record buffer in mtr_t, similar to the current mtr_t::log. This buffer would be copied to the log file buffer at mtr_t::commit().

            Short or single-page mini-transactions can directly write the log records to the log file buffer (avoid copying and heap memory allocation).

            Pseudo-code:
            {code:c++}
            void mtr_t::commit(bool sync = false)
            {
              {log_file,lsn} = log_sys->acquire_file(); // Atomic or using log_sys->mutex
              write_mlog_start(mtr_log, lsn); // First record of each mini-transaction
              pos = log_file->append_and_release(mtr_log, len);
              if (sync) log_file->flush(pos);
            }
            {code}
            The function log_file_t::flush(uint64_t file_pos) will (encrypt and) write buf to the file.

            h3. To be defined
            Which data structure to use for
            {code:c++}
            uint64_t log_file_t::start_block(lsn_t) const
            {code}
            and which accuracy to aim for? Maybe 1MiB granularity would be enough? Then, just remember the start LSN of each 1MiB segment since the latest checkpoint was written?
            marko Marko Mäkelä made changes -
            marko Marko Mäkelä made changes -
            marko Marko Mäkelä made changes -
            marko Marko Mäkelä made changes -
            marko Marko Mäkelä made changes -
            Attachment append.c [ 45146 ]
            marko Marko Mäkelä made changes -
            Attachment preallocate.c [ 45147 ]
            marko Marko Mäkelä made changes -
            Description The InnoDB redo log format is not optimal in many respects:
            * The log files are not append-only. Modern journaled file systems handle append-only files more efficiently.
            * There is no possibility to archive the redo log.
            * The LSN is based on bytes, and the log is not partitioned, making it hard to parallelize writes of concurrent mini-transaction commits.

            Let us introduce the following parameters:
            * innodb_log_file_size (in multiples of 4096 bytes; 0 (default) disables log rotation/archiving)
            * innodb_log_partitions (default=1) for allowing concurrent writes to different partitions

            Some old InnoDB redo log parameters will be removed (or deprecated and ignored):
            * innodb_log_files_in_group
            * innodb_log_write_ahead_size
            * innodb_log_checksums (if it performs well enough with hardware-assisted CRC-32C)

            h2. The idea: Partition the log into append-only, truncate-the-start files
            Append-only files are more efficient in modern crash-safe file systems. For example, file systems can avoid writing the data twice (first to the journal, then to the data blocks).

            The ib_logfile0 will be repurposed for the purpose of upgrading and preventing downgrading. The size of this file will be one file system block (padded with NUL bytes). In the first 512 bytes of the file, the following information will be present:
            * InnoDB redo log format version identifier (in the format introduced by MySQL 5.7.9/MariaDB 10.2.2)
            * The value of innodb_log_partitions at the latest checkpoint
            * The LSN of the latest checkpoint
            * CRC-32C checksum
            * Optional: If innodb_encrypt_log, write the encryption key ID and encrypt the checkpoint LSN with that key.

            This file will be overwritten (O_DIRECT) at every redo log checkpoint.

            The redo log will be partitioned into files like innodb_%u.log with 0..innodb_log_partitions-1. The log file will be chosen by pthread_self() % innodb_log_partitions.

            If innodb_log_max_size is not at the default (0), then as soon as one log file would exceed the maximum size, all log files will be rotated by renaming to innodb_%u.%06u and by creating empty innodb_%u.log files.

            If innodb_log_max_size=0 (the default), then at checkpoint, the start of each innodb_%u.log file will be discarded by punching a hole from 0 to the block that contains the first record at or after the checkpoint LSN. If the file system does not support hole-punching, then at the start of the file a header will be written that points to the first block.

            The log block format will be redesigned. The log block header may contain the following:
            * Log block size (the physical block size of the system that wrote the block)
            * Checksum
            * Pointer to the first MLOG_START record (or 0 if there is no such record in the page)
            * The smallest log block size is 512 bytes.
            * All-zero log blocks are silently ignored (treated as 512 bytes)

            Mini-transactions will always start with an MLOG_START(lsn) entry. The lsn is a global sequence number that is atomically incremented whenever mtr_t::commit() is about to write redo log.

            h3. Page flushing
            Before writing any dirty page to data file, ensure that _all_ log files have been flushed up to the page LSN.
            Thanks to this, recovery will not require contiguous LSN in all log files.
            Thanks to this, a persistent mtr_t::commit() (which changes user transaction state) will only have to flush the current redo log file, not all log files.

            h3. Crash recovery
            # Read start_checkpoint_lsn from ib_logfile0.
            # In each redo log file: Find the first MLOG_START record with lsn>= checkpoint_lsn
            # Process each redo log up to the end

            We do not require the LSN to be contiguous from checkpoint_lsn to the end.
            The log_sys->lsn will be initialized to the maximum LSN that was found.

            The MLOG_START records will be found by scanning the redo log blocks from a start offset onwards until a qualifying record is found.
            On checkpoint, the start of each log file will be truncated until a block that is near the MLOG_START(checkpoint_lsn), so that we will not have to scan from the start of each file.

            h3. Log checkpoint
            {code:c++}
              pwrite(ib_logfile0, page_with_checkpoint_lsn, 4096, 0);

              if (innodb_log_max_size) {
                for (auto log : logs) {
                  mach_write_to_8(log_header, log->start_block(checkpoint_lsn));
                  pwrite(log->fd, log_header, 4096, 0);
                }
              } else {
                for (auto log : logs) {
                  uint64_t offs = log->start_block(checkpoint_lsn);
                  fallocate(log->fd, FALLOC_FL_PUNCH_HOLE, 0, offs);
                }
              }
            {code}
            The new function
            {code:c++}
            uint64_t log_file_t::start_block(lsn_t) const
            {code}
            returns the file offset of the block that contains the (introduced in this work) MLOG_START(lsn) record, or an offset that is not much smaller than that. It likely will require some additional main-memory data structure, because we might reuse log_file_t::buffer entries after they have been written to the redo log, but before the log checkpoint is initiated.

            h3. Log buffering
            For multi-page mini-transactions (such as B-tree page split or merge), we can have a local redo log record buffer in mtr_t, similar to the current mtr_t::log. This buffer would be copied to the log file buffer at mtr_t::commit().

            Short or single-page mini-transactions can directly write the log records to the log file buffer (avoid copying and heap memory allocation).

            Pseudo-code:
            {code:c++}
            void mtr_t::commit(bool sync = false)
            {
              {log_file,lsn} = log_sys->acquire_file(); // Atomic or using log_sys->mutex
              write_mlog_start(mtr_log, lsn); // First record of each mini-transaction
              pos = log_file->append_and_release(mtr_log, len);
              if (sync) log_file->flush(pos);
            }
            {code}
            The function log_file_t::flush(uint64_t file_pos) will (encrypt and) write buf to the file.

            h3. To be defined
            Which data structure to use for
            {code:c++}
            uint64_t log_file_t::start_block(lsn_t) const
            {code}
            and which accuracy to aim for? Maybe 1MiB granularity would be enough? Then, just remember the start LSN of each 1MiB segment since the latest checkpoint was written?
            The InnoDB redo log format is not optimal in many respects:
            * The log files are not append-only. Modern journaled file systems handle append-only files more efficiently.
            * There is no possibility to archive the redo log.
            * The LSN is based on bytes, and the log is not partitioned, making it hard to parallelize writes of concurrent mini-transaction commits.

            Let us introduce the following parameters:
            * innodb_log_file_size (in multiples of 4096 bytes; 0 (default) disables log rotation/archiving)
            * innodb_log_partitions (default=1) for allowing concurrent writes to different partitions

            Some old InnoDB redo log parameters will be removed (or deprecated and ignored):
            * innodb_log_files_in_group
            * innodb_log_write_ahead_size
            * innodb_log_checksums (if it performs well enough with hardware-assisted CRC-32C)

            h2. The idea: Partition the log into append-only, truncate-the-start files
            Append-only files are more efficient in modern crash-safe file systems. For example, file systems can avoid writing the data twice (first to the journal, then to the data blocks).

            The ib_logfile0 will be repurposed for the purpose of upgrading and preventing downgrading. The size of this file will be one file system block (padded with NUL bytes). In the first 512 bytes of the file, the following information will be present:
            * InnoDB redo log format version identifier (in the format introduced by MySQL 5.7.9/MariaDB 10.2.2)
            * The value of innodb_log_partitions at the latest checkpoint
            * The LSN of the latest checkpoint
            * CRC-32C checksum
            * Optional: If innodb_encrypt_log, write the encryption key ID and encrypt the checkpoint LSN with that key.

            This file will be overwritten (O_DIRECT) at every redo log checkpoint.

            The redo log will be partitioned into files like innodb_%u.log with 0..innodb_log_partitions-1. The log file will be chosen by pthread_self() % innodb_log_partitions.

            If innodb_log_max_size is not at the default (0), then as soon as one log file would exceed the maximum size, all log files will be rotated by renaming to innodb_%u.%06u and by creating empty innodb_%u.log files.

            If innodb_log_max_size=0 (the default), then at checkpoint, the start of each innodb_%u.log file will be discarded by punching a hole from 0 to the block that contains the first record at or after the checkpoint LSN. If the file system does not support hole-punching, then at the start of the file a header will be written that points to the first block.

            The log block format will be redesigned. The log block header may contain the following:
            * Log block size (the physical block size of the system that wrote the block)
            * Checksum
            * Pointer to the first MLOG_START record (or 0 if there is no such record in the page)
            * The smallest log block size is 512 bytes.
            * All-zero log blocks are silently ignored (treated as 512 bytes)

            Mini-transactions will always start with an MLOG_START(lsn) entry. The lsn is a global sequence number that is atomically incremented whenever mtr_t::commit() is about to write redo log.

            For operations on clustered indexes, the MLOG_START entry could be followed by the user transaction start ID ({{DB_TRX_ID}}) so that the changes could be filtered by transaction.

            h3. Log writing and synchronous flushing
            For the bulk of the changes done by mini-transactions, we do not care about flushing. The file system can write log file blocks as it pleases.

            Some state changes of the database must be made durable at a specific time. Examples include user transaction {{COMMIT}}, {{XA PREPARE}}, {{XA ROLLBACK}}, and (in case the binlog is not enabled) {{XA COMMIT}}.

            Whenever we want to make a certain change durable, we must flush all log files up to the LSN of the mini-transaction commit that made the change. While doing this, we can pad each log file to the file system block size, so that the file system can always write full blocks. The padding could also be helpful when trying to resurrect a corrupted redo log file.

            If redo log is additionally replicated to the buffer pools of slaves (like in PolarDB), then we should first write to the redo log files and only then to the slaves, and we should assume that the writes to the files will always eventually be durable. If that assumption is broken, then all servers would have to be restarted and perform crash recovery.

            h3. Crash recovery
            # Read start_checkpoint_lsn from ib_logfile0.
            # In each redo log file: Find the first MLOG_START record with lsn>= checkpoint_lsn
            # Process each redo log up to the end

            The log_sys->lsn will be initialized to the maximum LSN that was found.

            The MLOG_START records will be found by scanning the redo log blocks from a start offset onwards until a qualifying record is found.
            On checkpoint, the start of each log file may be truncated until a block that is near the MLOG_START(checkpoint_lsn), so that we will not have to scan from the start of each file.

            h4. Ordering of mini-transaction commits
            Should we require the LSN to be contiguous from checkpoint_lsn to the end? This matters in a scenario where the server was killed while multiple log files were being written concurrently, or multiple {{mtr_t::commit()}} were executing at the same time.

            If we allowed gaps in the LSNs, we are essentially implying a partial ordering of mini-transaction commits. This would simplify the recovery algorithm and allow us to recover more mini-transactions from the redo log. But it would not allow to recover any more user transaction commits, because synchronous log flushing would guarantee the files to be in sync with each other. It would also break correctness issue in the following scenario:
            # Mini-transaction m1 updates page A via log file 1.
            # Mini-transaction m2 updates page A via log file 2.
            # The write to log file 2 is completed, but log file 1 was not written yet.
            # The system is killed.
            # Recovery sees the commit LSN of m2 but does not see m1, whose LSN would have been smaller.
            # If recovery ignores the gap of m1.LSN and applies the change of m2, page A may be inconsistent, because it missed the earlier change by m1.

            This scenario could be prevented with additional logic in the mini-transactions, adding {{fsync()}} of the ‘related log files’. But it would necessarily slow down the log writing, defeating the purpose of partitioning the log.

            It could make sense to introduce a disaster recovery option for ignoring LSN gaps between files.

            h3. Page flushing
            Before writing any dirty page to data file, ensure that _all_ log files have been flushed up to the page LSN.
            Thanks to this, recovery will not require contiguous LSN in all log files.
            Thanks to this, a persistent mtr_t::commit() (which changes user transaction state) will only have to flush the current redo log file, not all log files.

            h3. Log checkpoint
            {code:c++}
              pwrite(ib_logfile0, page_with_checkpoint_lsn, 4096, 0);

              if (innodb_log_max_size) {
                for (auto log : logs) {
                  mach_write_to_8(log_header, log->start_block(checkpoint_lsn));
                  pwrite(log->fd, log_header, 4096, 0);
                }
              } else {
                for (auto log : logs) {
                  uint64_t offs = log->start_block(checkpoint_lsn);
                  fallocate(log->fd, FALLOC_FL_PUNCH_HOLE, 0, offs);
                }
              }
            {code}
            The new function
            {code:c++}
            uint64_t log_file_t::start_block(lsn_t) const
            {code}
            returns the file offset of the block that contains the (introduced in this work) MLOG_START(lsn) record, or an offset that is not much smaller than that. It likely will require some additional main-memory data structure, because we might reuse log_file_t::buffer entries after they have been written to the redo log, but before the log checkpoint is initiated.

            h3. Log buffering
            For multi-page mini-transactions (such as B-tree page split or merge), we can have a local redo log record buffer in mtr_t, similar to the current mtr_t::log. This buffer would be copied to the log file buffer at mtr_t::commit().

            Short or single-page mini-transactions can directly write the log records to the log file buffer (avoid copying and heap memory allocation).

            Pseudo-code:
            {code:c++}
            void mtr_t::commit(bool sync = false)
            {
              {log_file,lsn} = log_sys->acquire_file(); // Atomic or using log_sys->mutex
              write_mlog_start(mtr_log, lsn); // First record of each mini-transaction
              pos = log_file->append_and_release(mtr_log, len);
              if (sync) log_file->flush(pos);
            }
            {code}
            The function log_file_t::flush(uint64_t file_pos) will (encrypt and) write buf to the file.

            h3. To be defined
            Which data structure to use for
            {code:c++}
            uint64_t log_file_t::start_block(lsn_t) const
            {code}
            and which accuracy to aim for? Maybe 1MiB granularity would be enough? Then, just remember the start LSN of each 1MiB segment since the latest checkpoint was written?
            marko Marko Mäkelä made changes -
            Description The InnoDB redo log format is not optimal in many respects:
            * The log files are not append-only. Modern journaled file systems handle append-only files more efficiently.
            * There is no possibility to archive the redo log.
            * The LSN is based on bytes, and the log is not partitioned, making it hard to parallelize writes of concurrent mini-transaction commits.

            Let us introduce the following parameters:
            * innodb_log_file_size (in multiples of 4096 bytes; 0 (default) disables log rotation/archiving)
            * innodb_log_partitions (default=1) for allowing concurrent writes to different partitions

            Some old InnoDB redo log parameters will be removed (or deprecated and ignored):
            * innodb_log_files_in_group
            * innodb_log_write_ahead_size
            * innodb_log_checksums (if it performs well enough with hardware-assisted CRC-32C)

            h2. The idea: Partition the log into append-only, truncate-the-start files
            Append-only files are more efficient in modern crash-safe file systems. For example, file systems can avoid writing the data twice (first to the journal, then to the data blocks).

            The ib_logfile0 will be repurposed for the purpose of upgrading and preventing downgrading. The size of this file will be one file system block (padded with NUL bytes). In the first 512 bytes of the file, the following information will be present:
            * InnoDB redo log format version identifier (in the format introduced by MySQL 5.7.9/MariaDB 10.2.2)
            * The value of innodb_log_partitions at the latest checkpoint
            * The LSN of the latest checkpoint
            * CRC-32C checksum
            * Optional: If innodb_encrypt_log, write the encryption key ID and encrypt the checkpoint LSN with that key.

            This file will be overwritten (O_DIRECT) at every redo log checkpoint.

            The redo log will be partitioned into files like innodb_%u.log with 0..innodb_log_partitions-1. The log file will be chosen by pthread_self() % innodb_log_partitions.

            If innodb_log_max_size is not at the default (0), then as soon as one log file would exceed the maximum size, all log files will be rotated by renaming to innodb_%u.%06u and by creating empty innodb_%u.log files.

            If innodb_log_max_size=0 (the default), then at checkpoint, the start of each innodb_%u.log file will be discarded by punching a hole from 0 to the block that contains the first record at or after the checkpoint LSN. If the file system does not support hole-punching, then at the start of the file a header will be written that points to the first block.

            The log block format will be redesigned. The log block header may contain the following:
            * Log block size (the physical block size of the system that wrote the block)
            * Checksum
            * Pointer to the first MLOG_START record (or 0 if there is no such record in the page)
            * The smallest log block size is 512 bytes.
            * All-zero log blocks are silently ignored (treated as 512 bytes)

            Mini-transactions will always start with an MLOG_START(lsn) entry. The lsn is a global sequence number that is atomically incremented whenever mtr_t::commit() is about to write redo log.

            For operations on clustered indexes, the MLOG_START entry could be followed by the user transaction start ID ({{DB_TRX_ID}}) so that the changes could be filtered by transaction.

            h3. Log writing and synchronous flushing
            For the bulk of the changes done by mini-transactions, we do not care about flushing. The file system can write log file blocks as it pleases.

            Some state changes of the database must be made durable at a specific time. Examples include user transaction {{COMMIT}}, {{XA PREPARE}}, {{XA ROLLBACK}}, and (in case the binlog is not enabled) {{XA COMMIT}}.

            Whenever we want to make a certain change durable, we must flush all log files up to the LSN of the mini-transaction commit that made the change. While doing this, we can pad each log file to the file system block size, so that the file system can always write full blocks. The padding could also be helpful when trying to resurrect a corrupted redo log file.

            If redo log is additionally replicated to the buffer pools of slaves (like in PolarDB), then we should first write to the redo log files and only then to the slaves, and we should assume that the writes to the files will always eventually be durable. If that assumption is broken, then all servers would have to be restarted and perform crash recovery.

            h3. Crash recovery
            # Read start_checkpoint_lsn from ib_logfile0.
            # In each redo log file: Find the first MLOG_START record with lsn>= checkpoint_lsn
            # Process each redo log up to the end

            The log_sys->lsn will be initialized to the maximum LSN that was found.

            The MLOG_START records will be found by scanning the redo log blocks from a start offset onwards until a qualifying record is found.
            On checkpoint, the start of each log file may be truncated until a block that is near the MLOG_START(checkpoint_lsn), so that we will not have to scan from the start of each file.

            h4. Ordering of mini-transaction commits
            Should we require the LSN to be contiguous from checkpoint_lsn to the end? This matters in a scenario where the server was killed while multiple log files were being written concurrently, or multiple {{mtr_t::commit()}} were executing at the same time.

            If we allowed gaps in the LSNs, we are essentially implying a partial ordering of mini-transaction commits. This would simplify the recovery algorithm and allow us to recover more mini-transactions from the redo log. But it would not allow to recover any more user transaction commits, because synchronous log flushing would guarantee the files to be in sync with each other. It would also break correctness issue in the following scenario:
            # Mini-transaction m1 updates page A via log file 1.
            # Mini-transaction m2 updates page A via log file 2.
            # The write to log file 2 is completed, but log file 1 was not written yet.
            # The system is killed.
            # Recovery sees the commit LSN of m2 but does not see m1, whose LSN would have been smaller.
            # If recovery ignores the gap of m1.LSN and applies the change of m2, page A may be inconsistent, because it missed the earlier change by m1.

            This scenario could be prevented with additional logic in the mini-transactions, adding {{fsync()}} of the ‘related log files’. But it would necessarily slow down the log writing, defeating the purpose of partitioning the log.

            It could make sense to introduce a disaster recovery option for ignoring LSN gaps between files.

            h3. Page flushing
            Before writing any dirty page to data file, ensure that _all_ log files have been flushed up to the page LSN.
            Thanks to this, recovery will not require contiguous LSN in all log files.
            Thanks to this, a persistent mtr_t::commit() (which changes user transaction state) will only have to flush the current redo log file, not all log files.

            h3. Log checkpoint
            {code:c++}
              pwrite(ib_logfile0, page_with_checkpoint_lsn, 4096, 0);

              if (innodb_log_max_size) {
                for (auto log : logs) {
                  mach_write_to_8(log_header, log->start_block(checkpoint_lsn));
                  pwrite(log->fd, log_header, 4096, 0);
                }
              } else {
                for (auto log : logs) {
                  uint64_t offs = log->start_block(checkpoint_lsn);
                  fallocate(log->fd, FALLOC_FL_PUNCH_HOLE, 0, offs);
                }
              }
            {code}
            The new function
            {code:c++}
            uint64_t log_file_t::start_block(lsn_t) const
            {code}
            returns the file offset of the block that contains the (introduced in this work) MLOG_START(lsn) record, or an offset that is not much smaller than that. It likely will require some additional main-memory data structure, because we might reuse log_file_t::buffer entries after they have been written to the redo log, but before the log checkpoint is initiated.

            h3. Log buffering
            For multi-page mini-transactions (such as B-tree page split or merge), we can have a local redo log record buffer in mtr_t, similar to the current mtr_t::log. This buffer would be copied to the log file buffer at mtr_t::commit().

            Short or single-page mini-transactions can directly write the log records to the log file buffer (avoid copying and heap memory allocation).

            Pseudo-code:
            {code:c++}
            void mtr_t::commit(bool sync = false)
            {
              {log_file,lsn} = log_sys->acquire_file(); // Atomic or using log_sys->mutex
              write_mlog_start(mtr_log, lsn); // First record of each mini-transaction
              pos = log_file->append_and_release(mtr_log, len);
              if (sync) log_file->flush(pos);
            }
            {code}
            The function log_file_t::flush(uint64_t file_pos) will (encrypt and) write buf to the file.

            h3. To be defined
            Which data structure to use for
            {code:c++}
            uint64_t log_file_t::start_block(lsn_t) const
            {code}
            and which accuracy to aim for? Maybe 1MiB granularity would be enough? Then, just remember the start LSN of each 1MiB segment since the latest checkpoint was written?
            The InnoDB redo log format is not optimal in many respects:
            * The log files are not append-only. Modern journaled file systems handle append-only files more efficiently.
            * There is no possibility to archive the redo log.
            * The LSN is based on bytes, and the log is not partitioned, making it hard to parallelize writes of concurrent mini-transaction commits.

            Let us introduce the following parameters:
            * innodb_log_file_size (in multiples of 4096 bytes; 0 (default) disables log rotation/archiving)
            * innodb_log_partitions (default=1) for allowing concurrent writes to different partitions

            Some old InnoDB redo log parameters will be removed (or deprecated and ignored):
            * innodb_log_files_in_group
            * innodb_log_write_ahead_size
            * innodb_log_checksums (if it performs well enough with hardware-assisted CRC-32C)

            h2. The idea: Partition the log into append-only, truncate-the-start files
            Append-only files are more efficient in modern crash-safe file systems. For example, file systems can avoid writing the data twice (first to the journal, then to the data blocks).

            The ib_logfile0 will be repurposed for the purpose of upgrading and preventing downgrading. The size of this file will be one file system block (padded with NUL bytes). In the first 512 bytes of the file, the following information will be present:
            * InnoDB redo log format version identifier (in the format introduced by MySQL 5.7.9/MariaDB 10.2.2)
            * The value of innodb_log_partitions at the latest checkpoint
            * The LSN of the latest checkpoint
            * CRC-32C checksum
            * Optional: If innodb_encrypt_log, write the encryption key ID and encrypt the checkpoint LSN with that key.

            This file will be overwritten (O_DIRECT) at every redo log checkpoint.

            The redo log will be partitioned into files like innodb_%u.log with 0..innodb_log_partitions-1. The log file will be chosen by pthread_self() % innodb_log_partitions.

            If innodb_log_max_size is not at the default (0), then as soon as one log file would exceed the maximum size, all log files will be rotated by renaming to innodb_%u.%06u and by creating empty innodb_%u.log files.

            If innodb_log_max_size=0 (the default), then at checkpoint, the start of each innodb_%u.log file will be discarded by punching a hole from 0 to the block that contains the first record at or after the checkpoint LSN. If the file system does not support hole-punching, then at the start of the file a header will be written that points to the first block.

            The log block format will be redesigned. The log block header may contain the following:
            * Log block size (the physical block size of the system that wrote the block)
            * Checksum
            * Pointer to the first MLOG_START record (or 0 if there is no such record in the page)
            * The smallest log block size is 512 bytes.
            * All-zero log blocks are silently ignored (treated as 512 bytes)

            Mini-transactions will always start with an MLOG_START(lsn) entry. The lsn is a global sequence number that is atomically incremented whenever mtr_t::commit() is about to write redo log.

            For operations on clustered indexes, the MLOG_START entry could be followed by the user transaction start ID ({{DB_TRX_ID}}) so that the changes could be filtered by transaction.

            h3. Log writing and synchronous flushing
            For the bulk of the changes done by mini-transactions, we do not care about flushing. The file system can write log file blocks as it pleases.

            Some state changes of the database must be made durable at a specific time. Examples include user transaction {{COMMIT}}, {{XA PREPARE}}, {{XA ROLLBACK}}, and (in case the binlog is not enabled) {{XA COMMIT}}.

            Whenever we want to make a certain change durable, we must flush all log files up to the LSN of the mini-transaction commit that made the change. While doing this, we can pad each log file to the file system block size, so that the file system can always write full blocks. The padding could also be helpful when trying to resurrect a corrupted redo log file.

            If redo log is additionally replicated to the buffer pools of slaves (like in PolarDB), then we should first write to the redo log files and only then to the slaves, and we should assume that the writes to the files will always eventually be durable. If that assumption is broken, then all servers would have to be restarted and perform crash recovery.

            h3. Crash recovery
            # Read start_checkpoint_lsn from ib_logfile0.
            # In each redo log file: Find the first MLOG_START record with lsn>= checkpoint_lsn
            # Process each redo log up to the end

            The log_sys->lsn will be initialized to the maximum LSN that was found.

            The MLOG_START records will be found by scanning the redo log blocks from a start offset onwards until a qualifying record is found.
            On checkpoint, the start of each log file may be truncated until a block that is near the MLOG_START(checkpoint_lsn), so that we will not have to scan from the start of each file.

            h4. Ordering of mini-transaction commits
            Should we require the LSN to be contiguous from checkpoint_lsn to the end? This matters in a scenario where the server was killed while multiple log files were being written concurrently, or multiple {{mtr_t::commit()}} were executing at the same time.

            If we allowed gaps in the LSNs, we are essentially implying a partial ordering of mini-transaction commits. This would simplify the recovery algorithm and allow us to recover more mini-transactions from the redo log. But it would not allow to recover any more user transaction commits, because synchronous log flushing would guarantee the files to be in sync with each other. It would also break correctness issue in the following scenario:
            # Mini-transaction m1 updates page A via log file 1.
            # Mini-transaction m2 updates page A via log file 2.
            # The write to log file 2 is completed, but log file 1 was not written yet.
            # The system is killed.
            # Recovery sees the commit LSN of m2 but does not see m1, whose LSN would have been smaller.
            # If recovery ignores the gap of m1.LSN and applies the change of m2, page A may be inconsistent, because it missed the earlier change by m1.

            This scenario could be prevented with additional logic in the mini-transactions, adding {{fsync()}} of the ‘related log files’. But it would necessarily slow down the log writing, defeating the purpose of partitioning the log.

            It could make sense to introduce a disaster recovery option for ignoring LSN gaps between files. A possibility (after fixing MDEV-12700) would be a special value {{innodb_read_only=2}} which would recover the database while ignoring the LSN gaps.

            h3. Page flushing
            Before writing any dirty page to data file, ensure that _all_ log files have been flushed up to the page LSN.
            Thanks to this, recovery will not require contiguous LSN in all log files.
            Thanks to this, a persistent mtr_t::commit() (which changes user transaction state) will only have to flush the current redo log file, not all log files.

            h3. Log checkpoint
            {code:c++}
              pwrite(ib_logfile0, page_with_checkpoint_lsn, 4096, 0);

              if (innodb_log_max_size) {
                for (auto log : logs) {
                  mach_write_to_8(log_header, log->start_block(checkpoint_lsn));
                  pwrite(log->fd, log_header, 4096, 0);
                }
              } else {
                for (auto log : logs) {
                  uint64_t offs = log->start_block(checkpoint_lsn);
                  fallocate(log->fd, FALLOC_FL_PUNCH_HOLE, 0, offs);
                }
              }
            {code}
            The new function
            {code:c++}
            uint64_t log_file_t::start_block(lsn_t) const
            {code}
            returns the file offset of the block that contains the (introduced in this work) MLOG_START(lsn) record, or an offset that is not much smaller than that. It likely will require some additional main-memory data structure, because we might reuse log_file_t::buffer entries after they have been written to the redo log, but before the log checkpoint is initiated.

            h3. Log buffering
            For multi-page mini-transactions (such as B-tree page split or merge), we can have a local redo log record buffer in mtr_t, similar to the current mtr_t::log. This buffer would be copied to the log file buffer at mtr_t::commit().

            Short or single-page mini-transactions can directly write the log records to the log file buffer (avoid copying and heap memory allocation).

            Pseudo-code:
            {code:c++}
            void mtr_t::commit(bool sync = false)
            {
              {log_file,lsn} = log_sys->acquire_file(); // Atomic or using log_sys->mutex
              write_mlog_start(mtr_log, lsn); // First record of each mini-transaction
              pos = log_file->append_and_release(mtr_log, len);
              if (sync) log_file->flush(pos);
            }
            {code}
            The function log_file_t::flush(uint64_t file_pos) will (encrypt and) write buf to the file.

            h3. To be defined
            Which data structure to use for
            {code:c++}
            uint64_t log_file_t::start_block(lsn_t) const
            {code}
            and which accuracy to aim for? Maybe 1MiB granularity would be enough? Then, just remember the start LSN of each 1MiB segment since the latest checkpoint was written?
            marko Marko Mäkelä made changes -
            marko Marko Mäkelä made changes -
            marko Marko Mäkelä made changes -
            marko Marko Mäkelä made changes -
            marko Marko Mäkelä made changes -
            marko Marko Mäkelä made changes -
            ralf.gebhardt Ralf Gebhardt made changes -
            Fix Version/s 10.5 [ 23123 ]
            Fix Version/s 10.4 [ 22408 ]
            marko Marko Mäkelä made changes -
            marko Marko Mäkelä made changes -
            marko Marko Mäkelä made changes -
            marko Marko Mäkelä made changes -
            marko Marko Mäkelä made changes -
            NRE Projects RM_105_CANDIDATE
            ralf.gebhardt Ralf Gebhardt made changes -
            Priority Major [ 3 ] Critical [ 2 ]
            marko Marko Mäkelä made changes -
            marko Marko Mäkelä made changes -
            marko Marko Mäkelä made changes -
            marko Marko Mäkelä made changes -
            marko Marko Mäkelä made changes -
            marko Marko Mäkelä made changes -
            marko Marko Mäkelä made changes -
            marko Marko Mäkelä made changes -
            marko Marko Mäkelä made changes -
            marko Marko Mäkelä made changes -
            kevg Eugene Kosov (Inactive) made changes -
            Assignee Marko Mäkelä [ marko ] Eugene Kosov [ kevg ]
            marko Marko Mäkelä made changes -
            marko Marko Mäkelä made changes -
            julien.fritsch Julien Fritsch made changes -
            julien.fritsch Julien Fritsch made changes -
            julien.fritsch Julien Fritsch made changes -
            marko Marko Mäkelä made changes -
            marko Marko Mäkelä made changes -
            Component/s Encryption [ 11200 ]
            Component/s mariabackup [ 14500 ]
            Fix Version/s 10.6 [ 24028 ]
            Fix Version/s 10.5 [ 23123 ]
            Assignee Eugene Kosov [ kevg ] Marko Mäkelä [ marko ]
            Description The InnoDB redo log format is not optimal in many respects:
            * The log files are not append-only. Modern journaled file systems handle append-only files more efficiently.
            * There is no possibility to archive the redo log.
            * The LSN is based on bytes, and the log is not partitioned, making it hard to parallelize writes of concurrent mini-transaction commits.

            Let us introduce the following parameters:
            * innodb_log_file_size (in multiples of 4096 bytes; 0 (default) disables log rotation/archiving)
            * innodb_log_partitions (default=1) for allowing concurrent writes to different partitions

            Some old InnoDB redo log parameters will be removed (or deprecated and ignored):
            * innodb_log_files_in_group
            * innodb_log_write_ahead_size
            * innodb_log_checksums (if it performs well enough with hardware-assisted CRC-32C)

            h2. The idea: Partition the log into append-only, truncate-the-start files
            Append-only files are more efficient in modern crash-safe file systems. For example, file systems can avoid writing the data twice (first to the journal, then to the data blocks).

            The ib_logfile0 will be repurposed for the purpose of upgrading and preventing downgrading. The size of this file will be one file system block (padded with NUL bytes). In the first 512 bytes of the file, the following information will be present:
            * InnoDB redo log format version identifier (in the format introduced by MySQL 5.7.9/MariaDB 10.2.2)
            * The value of innodb_log_partitions at the latest checkpoint
            * The LSN of the latest checkpoint
            * CRC-32C checksum
            * Optional: If innodb_encrypt_log, write the encryption key ID and encrypt the checkpoint LSN with that key.

            This file will be overwritten (O_DIRECT) at every redo log checkpoint.

            The redo log will be partitioned into files like innodb_%u.log with 0..innodb_log_partitions-1. The log file will be chosen by pthread_self() % innodb_log_partitions.

            If innodb_log_max_size is not at the default (0), then as soon as one log file would exceed the maximum size, all log files will be rotated by renaming to innodb_%u.%06u and by creating empty innodb_%u.log files.

            If innodb_log_max_size=0 (the default), then at checkpoint, the start of each innodb_%u.log file will be discarded by punching a hole from 0 to the block that contains the first record at or after the checkpoint LSN. If the file system does not support hole-punching, then at the start of the file a header will be written that points to the first block.

            The log block format will be redesigned. The log block header may contain the following:
            * Log block size (the physical block size of the system that wrote the block)
            * Checksum
            * Pointer to the first MLOG_START record (or 0 if there is no such record in the page)
            * The smallest log block size is 512 bytes.
            * All-zero log blocks are silently ignored (treated as 512 bytes)

            Mini-transactions will always start with an MLOG_START(lsn) entry. The lsn is a global sequence number that is atomically incremented whenever mtr_t::commit() is about to write redo log.

            For operations on clustered indexes, the MLOG_START entry could be followed by the user transaction start ID ({{DB_TRX_ID}}) so that the changes could be filtered by transaction.

            h3. Log writing and synchronous flushing
            For the bulk of the changes done by mini-transactions, we do not care about flushing. The file system can write log file blocks as it pleases.

            Some state changes of the database must be made durable at a specific time. Examples include user transaction {{COMMIT}}, {{XA PREPARE}}, {{XA ROLLBACK}}, and (in case the binlog is not enabled) {{XA COMMIT}}.

            Whenever we want to make a certain change durable, we must flush all log files up to the LSN of the mini-transaction commit that made the change. While doing this, we can pad each log file to the file system block size, so that the file system can always write full blocks. The padding could also be helpful when trying to resurrect a corrupted redo log file.

            If redo log is additionally replicated to the buffer pools of slaves (like in PolarDB), then we should first write to the redo log files and only then to the slaves, and we should assume that the writes to the files will always eventually be durable. If that assumption is broken, then all servers would have to be restarted and perform crash recovery.

            h3. Crash recovery
            # Read start_checkpoint_lsn from ib_logfile0.
            # In each redo log file: Find the first MLOG_START record with lsn>= checkpoint_lsn
            # Process each redo log up to the end

            The log_sys->lsn will be initialized to the maximum LSN that was found.

            The MLOG_START records will be found by scanning the redo log blocks from a start offset onwards until a qualifying record is found.
            On checkpoint, the start of each log file may be truncated until a block that is near the MLOG_START(checkpoint_lsn), so that we will not have to scan from the start of each file.

            h4. Ordering of mini-transaction commits
            Should we require the LSN to be contiguous from checkpoint_lsn to the end? This matters in a scenario where the server was killed while multiple log files were being written concurrently, or multiple {{mtr_t::commit()}} were executing at the same time.

            If we allowed gaps in the LSNs, we are essentially implying a partial ordering of mini-transaction commits. This would simplify the recovery algorithm and allow us to recover more mini-transactions from the redo log. But it would not allow to recover any more user transaction commits, because synchronous log flushing would guarantee the files to be in sync with each other. It would also break correctness issue in the following scenario:
            # Mini-transaction m1 updates page A via log file 1.
            # Mini-transaction m2 updates page A via log file 2.
            # The write to log file 2 is completed, but log file 1 was not written yet.
            # The system is killed.
            # Recovery sees the commit LSN of m2 but does not see m1, whose LSN would have been smaller.
            # If recovery ignores the gap of m1.LSN and applies the change of m2, page A may be inconsistent, because it missed the earlier change by m1.

            This scenario could be prevented with additional logic in the mini-transactions, adding {{fsync()}} of the ‘related log files’. But it would necessarily slow down the log writing, defeating the purpose of partitioning the log.

            It could make sense to introduce a disaster recovery option for ignoring LSN gaps between files. A possibility (after fixing MDEV-12700) would be a special value {{innodb_read_only=2}} which would recover the database while ignoring the LSN gaps.

            h3. Page flushing
            Before writing any dirty page to data file, ensure that _all_ log files have been flushed up to the page LSN.
            Thanks to this, recovery will not require contiguous LSN in all log files.
            Thanks to this, a persistent mtr_t::commit() (which changes user transaction state) will only have to flush the current redo log file, not all log files.

            h3. Log checkpoint
            {code:c++}
              pwrite(ib_logfile0, page_with_checkpoint_lsn, 4096, 0);

              if (innodb_log_max_size) {
                for (auto log : logs) {
                  mach_write_to_8(log_header, log->start_block(checkpoint_lsn));
                  pwrite(log->fd, log_header, 4096, 0);
                }
              } else {
                for (auto log : logs) {
                  uint64_t offs = log->start_block(checkpoint_lsn);
                  fallocate(log->fd, FALLOC_FL_PUNCH_HOLE, 0, offs);
                }
              }
            {code}
            The new function
            {code:c++}
            uint64_t log_file_t::start_block(lsn_t) const
            {code}
            returns the file offset of the block that contains the (introduced in this work) MLOG_START(lsn) record, or an offset that is not much smaller than that. It likely will require some additional main-memory data structure, because we might reuse log_file_t::buffer entries after they have been written to the redo log, but before the log checkpoint is initiated.

            h3. Log buffering
            For multi-page mini-transactions (such as B-tree page split or merge), we can have a local redo log record buffer in mtr_t, similar to the current mtr_t::log. This buffer would be copied to the log file buffer at mtr_t::commit().

            Short or single-page mini-transactions can directly write the log records to the log file buffer (avoid copying and heap memory allocation).

            Pseudo-code:
            {code:c++}
            void mtr_t::commit(bool sync = false)
            {
              {log_file,lsn} = log_sys->acquire_file(); // Atomic or using log_sys->mutex
              write_mlog_start(mtr_log, lsn); // First record of each mini-transaction
              pos = log_file->append_and_release(mtr_log, len);
              if (sync) log_file->flush(pos);
            }
            {code}
            The function log_file_t::flush(uint64_t file_pos) will (encrypt and) write buf to the file.

            h3. To be defined
            Which data structure to use for
            {code:c++}
            uint64_t log_file_t::start_block(lsn_t) const
            {code}
            and which accuracy to aim for? Maybe 1MiB granularity would be enough? Then, just remember the start LSN of each 1MiB segment since the latest checkpoint was written?
            The InnoDB redo log format is not optimal in many respects:
            * At the start of {{ib_logfile0}}, there are only two log checkpoint blocks. The rest of the log file is written in a circular fashion.
            * On log checkpoint, some file name information needs to be appended to the log.
            * File names that were first changed since the latest checkpoint must be appended to the log. The bookkeeping causes some contention on {{log_sys.mutex}} and {{fil_system.mutex}}.
            * The log file was unnecessarily split into multiple files, logically treated as one big circular file. (MDEV-20907 in MariaDB Server 10.5.0 change the default to 1 file, and later the parameter was deprecated and ignored.)
            * Log records are divided into tiny blocks of 512 bytes, with 12+4 bytes of header and footer (12+8 bytes with MDEV-12041 {{innodb_encrypt_log}} (10.4.0)).
            * We are holding a mutex while zero-filling unused parts of log blocks, encrypting log blocks, or computing checksums.
            * Mariabackup cannot copy the log without having access to the encryption keys.

            We had some ideas to move to an append-only file and to partition the log into multiple files, but it turned out that a single fixed-size circular log file would perform best.

            We should keep the first 512 bytes of the file {{ib_logfile0}} for compatibility purposes, but everything else could be improved.
            * {{ib_logfile0}} (after the 512-byte header) will be append-only, unencrypted, for records containing file names and checkpoint information. A checkpoint record will comprise an LSN and a byte offset in a separate, optionally encrypted, circular log file {{ib_logdata}}. The length of each record is explicitly tagged and the payload will be followed by CRC-32C.
            * The {{ib_logdata}} file can be append-only or circular. If it is circular, its fixed size must be an integer multiple of 512 bytes.
            * We remove log block headers and footers. We really only need is to detect the logical end of the circular log in {{ib_logdata}}. That can be achieved by making sure that mini-transactions are terminated by a sequence number (at least one bit) and a checksum. When the circular file wraps around, the sequence number will be incremented (or the sequence bit toggled).
            * For page-aligned I/O, allow dummy records to be written, to indicate that the next bytes (until the end of the physical page, no matter what the I/O block size is) must be ignored. Thus, there is no need to initialize any padding bytes or encrypt or compute checksums on them.
            * Encrypt and compute checksums on {{mtr_t::m_log}} before initiating a write to the circular log file. The log can be copied and checksums validated without access to encryption keys.
            * If {{ib_logdata}} is on a memory-mapped persistent memory device, then we could bypass {{log_sys.buf}} and the file system copy the data directly to the memory-mapped area and flush the CPU cache.

            Some old InnoDB redo log parameters will be deprecated and ignored:
            * {{innodb_log_files_in_group}} (deprecated & ignored in MariaDB 10.5.1)
            * {{innodb_log_checksums}} (already set by default, and forced when {{innodb_encrypt_log}})

            The parameter {{innodb_log_write_ahead_size}} may be repurposed to indicate the desired physical log block size. For PMEM, it might be 64 bytes (the width of a CPU cache line).

            h2. The checkpoint and file operations log file {{ib_logfile0}}
            The file name {{ib_logfile0}} and the existing format of the first 512 bytes will be retained for the purpose of upgrading and preventing downgrading. In the first 512 bytes of the file, the following information will be present:
            * InnoDB redo log format version identifier (in the format introduced by MySQL 5.7.9/MariaDB 10.2.2)
            * The size of the {{ib_logdata}} file (the maximum possible size of the file if it is append-only).
            * All encryption parameters for {{ib_logdata}}
            * CRC-32C checksum
            This file will be append-only. After the first 512 bytes, the format will be:
            * Optional: {{FILE_ID}} records (renamed from {{FILE_MODIFY}}) of all {{.ibd}} files that existed when the redo log file was created. Each record terminated by CRC-32C.
            * {{FILE_CHECKPOINT}} record with LSN and byte offset in {{ib_logdata}}, and sequence bit (in circular {{ib_logdata}} file), and CRC-32C. The byte offset will be at least 47 bits (128 TiB). Total size of a checkpoint: at least 1+8+6+4=19 bytes.
            * Optional: {{FILE_CREATE}}, {{FILE_DELETE}}, {{FILE_RENAME}} records and further {{FILE_CHECKPOINT}} records for any file operations or subsequent checkpoints since the redo log was created. Each record will be terminated by CRC-32C.

            All writes to {{ib_logfile0}} will be synchronous and durable ({{O_DSYNC}}, {{fdatasync()}} or {{O_SYNC}}, {{fsync()}}) .

            h2. The circular or append-only page-level log file {{ib_logdata}}
            The {{ib_logdata}} file will contain MDEV-12353 records except {{FILE_}} records that will be written into {{ib_logfile0}}. Each mini-transaction will be followed by a CRC-32C of all the bytes (using the value 0 for the sequence bit or number), so that Mariabackup can avoid recomputing the checksum while copying the log to a new file.

            If the {{ib_logdata}} is append-only, then we could enable log archiving by actually writing to files like {{ib_logdata.000000}}, {{ib_logdata.000001}} and so on.

            The exact encoding of {{ib_logdata}} is not decided yet. We want to avoid overwriting the last log block, so we cannot have an explicit 'end of log' marker.

            One possibility is that we will write MDEV-12353 records, with encryption applied to the payload bytes only (unencrypted record types, lengths, and possibly page identifiers). 4 bytes of CRC-32C would be appended after the mini-transaction-terminating byte 0x00 or 0x01 (the value of the sequence bit).

            The main drawback of this format would be that recovery may have to parse a lot of log before finding out whether the end of the log was reached.

            We would also have to introduce a special record of ‘skip the next N bytes’. For short sequences, we could simply say that the CRC-32C can be followed by any number of 0x00 or 0x01 bytes before the next mini-transaction starts. But, this would require us to fill the block to the end. If we want to avoid that, we might introduce a special record, something like 2 bytes for the record type, subtype and length, a variable-length encoded integer saying ‘skip N more bytes’ and a 4-byte CRC-32C checksum.

            Another possibility is that we will encapsulate MDEV-12353 records (possibly omitting the mini-transaction terminating NUL byte) in the following structure:
            * variable-length encoded integer of {{total_length << 2 | sequence_bit}}
            * the data payload (MDEV-12353 records), all encrypted if encryption is used
            * CRC-32C of the data payload
            Skipped bytes would be indicated by the following:
            * For skipped lengths 1 to 4 bytes, {{sequence_bit}} would be written.
            * For at least 5 skipped bytes: variable-length encoded integer of {{skipped_length << 2 | 1 << 1 | sequence_bit}} followed by CRC-32C

            h3. Log writing and synchronous flushing
            For the bulk of the changes done by mini-transactions, we do not care about flushing. The file system can write log file blocks as it pleases.

            Some state changes of the database must be made durable at a specific time. Examples include user transaction {{COMMIT}}, {{XA PREPARE}}, {{XA ROLLBACK}}, and (in case the binlog is not enabled) {{XA COMMIT}}.

            Whenever we want to make a certain change durable, we must flush all log files up to the LSN of the mini-transaction commit that made the change. While doing this, we can pad each log file to the file system block size (the 'skipped bytes'), so that the file system can always write full blocks. The padding could also be helpful when trying to resurrect a corrupted redo log file.

            If redo log is physically replicated to the buffer pools of slaves (like in PolarDB), then we should first write to the redo log files and only then to the slaves, and we should assume that the writes to the files will always eventually be durable. If that assumption is broken, then all servers would have to be restarted and perform crash recovery. Because the log sequence number will be counted in bytes, we will have to replicate the number of skipped bytes. (But we could omit the skipped bytes themselves when sending over the network.)

            h3. Crash recovery
            # Validate the {{ib_logfile0}} header, including checking the size of the {{ib_logdata}} file.
            # Read the last bytes of {{ib_logfile0}} to see if it ends in a valid {{FILE_CHECKPOINT}} record. If not, read the entire {{ib_logfile0}} to find the latest {{FILE_CHECKPOINT}} record.
            # If {{ib_logdata}} ends at the identified byte position (we got sequence bit mismatch or checksum mismatch), then the database was clean.
            # Else, recover the file name information based on the {{ib_logfile0}} contents and start recovery.
            # Parse the {{ib_logdata}} until the end. If any unknown tablespace identifiers exist, abort startup unless {{innodb_force_recovery≥1}}. Ignore records for those tablespaces for which {{FILE_DELETE}} had been recorded.
            # Replay {{FILE_DELETE}} and {{FILE_RENAME}} operations.
            # Shrink any files for which we parsed a {{TRIM_PAGES}} record, and extend any files according to changes to {{FSP_SIZE}}.
            # Apply the parsed log to all data files.

            h3. Considerations on checkpoints and writing {{ib_logdata}}
            After completing a checkpoint (the write of the {{FILE_CHECKPOINT}} record to {{ib_logfile0}}), we could punch a hole in {{ib_logdata}} to discard no-longer-needed log records.

            As an option, the redo log could be rebuilt on a checkpoint, by creating a logically empty set of log files, at the minimum consisting of an empty {{ib_logdata}} file and a {{ib_logfile0}} that contains {{FILE_ID}} records and a {{FILE_CHECKPOINT}} record.

            If redo log archiving is enabled, when the maximum configured size of a single redo log (which is stored in the {{ib_logfile0}} header) is reached, a new {{ib_logdata.%06u}} file will be created. The old file length must be exactly at the maximum length.

            Writes to {{ib_logfile0}} will *not* increment the LSN at all! This means that the redo log could be easily rebuilt at any LSN, and Mariabackup could write additional information to that file, removing the need for separate {{.delta}} files in incremental backups.
            Labels performance
            Priority Critical [ 2 ] Major [ 3 ]
            Summary InnoDB redo log format for better performance Change the InnoDB redo log format to reduce write amplification
            marko Marko Mäkelä made changes -
            Description The InnoDB redo log format is not optimal in many respects:
            * At the start of {{ib_logfile0}}, there are only two log checkpoint blocks. The rest of the log file is written in a circular fashion.
            * On log checkpoint, some file name information needs to be appended to the log.
            * File names that were first changed since the latest checkpoint must be appended to the log. The bookkeeping causes some contention on {{log_sys.mutex}} and {{fil_system.mutex}}.
            * The log file was unnecessarily split into multiple files, logically treated as one big circular file. (MDEV-20907 in MariaDB Server 10.5.0 change the default to 1 file, and later the parameter was deprecated and ignored.)
            * Log records are divided into tiny blocks of 512 bytes, with 12+4 bytes of header and footer (12+8 bytes with MDEV-12041 {{innodb_encrypt_log}} (10.4.0)).
            * We are holding a mutex while zero-filling unused parts of log blocks, encrypting log blocks, or computing checksums.
            * Mariabackup cannot copy the log without having access to the encryption keys.

            We had some ideas to move to an append-only file and to partition the log into multiple files, but it turned out that a single fixed-size circular log file would perform best.

            We should keep the first 512 bytes of the file {{ib_logfile0}} for compatibility purposes, but everything else could be improved.
            * {{ib_logfile0}} (after the 512-byte header) will be append-only, unencrypted, for records containing file names and checkpoint information. A checkpoint record will comprise an LSN and a byte offset in a separate, optionally encrypted, circular log file {{ib_logdata}}. The length of each record is explicitly tagged and the payload will be followed by CRC-32C.
            * The {{ib_logdata}} file can be append-only or circular. If it is circular, its fixed size must be an integer multiple of 512 bytes.
            * We remove log block headers and footers. We really only need is to detect the logical end of the circular log in {{ib_logdata}}. That can be achieved by making sure that mini-transactions are terminated by a sequence number (at least one bit) and a checksum. When the circular file wraps around, the sequence number will be incremented (or the sequence bit toggled).
            * For page-aligned I/O, allow dummy records to be written, to indicate that the next bytes (until the end of the physical page, no matter what the I/O block size is) must be ignored. Thus, there is no need to initialize any padding bytes or encrypt or compute checksums on them.
            * Encrypt and compute checksums on {{mtr_t::m_log}} before initiating a write to the circular log file. The log can be copied and checksums validated without access to encryption keys.
            * If {{ib_logdata}} is on a memory-mapped persistent memory device, then we could bypass {{log_sys.buf}} and the file system copy the data directly to the memory-mapped area and flush the CPU cache.

            Some old InnoDB redo log parameters will be deprecated and ignored:
            * {{innodb_log_files_in_group}} (deprecated & ignored in MariaDB 10.5.1)
            * {{innodb_log_checksums}} (already set by default, and forced when {{innodb_encrypt_log}})

            The parameter {{innodb_log_write_ahead_size}} may be repurposed to indicate the desired physical log block size. For PMEM, it might be 64 bytes (the width of a CPU cache line).

            h2. The checkpoint and file operations log file {{ib_logfile0}}
            The file name {{ib_logfile0}} and the existing format of the first 512 bytes will be retained for the purpose of upgrading and preventing downgrading. In the first 512 bytes of the file, the following information will be present:
            * InnoDB redo log format version identifier (in the format introduced by MySQL 5.7.9/MariaDB 10.2.2)
            * The size of the {{ib_logdata}} file (the maximum possible size of the file if it is append-only).
            * All encryption parameters for {{ib_logdata}}
            * CRC-32C checksum
            This file will be append-only. After the first 512 bytes, the format will be:
            * Optional: {{FILE_ID}} records (renamed from {{FILE_MODIFY}}) of all {{.ibd}} files that existed when the redo log file was created. Each record terminated by CRC-32C.
            * {{FILE_CHECKPOINT}} record with LSN and byte offset in {{ib_logdata}}, and sequence bit (in circular {{ib_logdata}} file), and CRC-32C. The byte offset will be at least 47 bits (128 TiB). Total size of a checkpoint: at least 1+8+6+4=19 bytes.
            * Optional: {{FILE_CREATE}}, {{FILE_DELETE}}, {{FILE_RENAME}} records and further {{FILE_CHECKPOINT}} records for any file operations or subsequent checkpoints since the redo log was created. Each record will be terminated by CRC-32C.

            All writes to {{ib_logfile0}} will be synchronous and durable ({{O_DSYNC}}, {{fdatasync()}} or {{O_SYNC}}, {{fsync()}}) .

            h2. The circular or append-only page-level log file {{ib_logdata}}
            The {{ib_logdata}} file will contain MDEV-12353 records except {{FILE_}} records that will be written into {{ib_logfile0}}. Each mini-transaction will be followed by a CRC-32C of all the bytes (using the value 0 for the sequence bit or number), so that Mariabackup can avoid recomputing the checksum while copying the log to a new file.

            If the {{ib_logdata}} is append-only, then we could enable log archiving by actually writing to files like {{ib_logdata.000000}}, {{ib_logdata.000001}} and so on.

            The exact encoding of {{ib_logdata}} is not decided yet. We want to avoid overwriting the last log block, so we cannot have an explicit 'end of log' marker.

            One possibility is that we will write MDEV-12353 records, with encryption applied to the payload bytes only (unencrypted record types, lengths, and possibly page identifiers). 4 bytes of CRC-32C would be appended after the mini-transaction-terminating byte 0x00 or 0x01 (the value of the sequence bit).

            The main drawback of this format would be that recovery may have to parse a lot of log before finding out whether the end of the log was reached.

            We would also have to introduce a special record of ‘skip the next N bytes’. For short sequences, we could simply say that the CRC-32C can be followed by any number of 0x00 or 0x01 bytes before the next mini-transaction starts. But, this would require us to fill the block to the end. If we want to avoid that, we might introduce a special record, something like 2 bytes for the record type, subtype and length, a variable-length encoded integer saying ‘skip N more bytes’ and a 4-byte CRC-32C checksum.

            Another possibility is that we will encapsulate MDEV-12353 records (possibly omitting the mini-transaction terminating NUL byte) in the following structure:
            * variable-length encoded integer of {{total_length << 2 | sequence_bit}}
            * the data payload (MDEV-12353 records), all encrypted if encryption is used
            * CRC-32C of the data payload
            Skipped bytes would be indicated by the following:
            * For skipped lengths 1 to 4 bytes, {{sequence_bit}} would be written.
            * For at least 5 skipped bytes: variable-length encoded integer of {{skipped_length << 2 | 1 << 1 | sequence_bit}} followed by CRC-32C

            h3. Log writing and synchronous flushing
            For the bulk of the changes done by mini-transactions, we do not care about flushing. The file system can write log file blocks as it pleases.

            Some state changes of the database must be made durable at a specific time. Examples include user transaction {{COMMIT}}, {{XA PREPARE}}, {{XA ROLLBACK}}, and (in case the binlog is not enabled) {{XA COMMIT}}.

            Whenever we want to make a certain change durable, we must flush all log files up to the LSN of the mini-transaction commit that made the change. While doing this, we can pad each log file to the file system block size (the 'skipped bytes'), so that the file system can always write full blocks. The padding could also be helpful when trying to resurrect a corrupted redo log file.

            If redo log is physically replicated to the buffer pools of slaves (like in PolarDB), then we should first write to the redo log files and only then to the slaves, and we should assume that the writes to the files will always eventually be durable. If that assumption is broken, then all servers would have to be restarted and perform crash recovery. Because the log sequence number will be counted in bytes, we will have to replicate the number of skipped bytes. (But we could omit the skipped bytes themselves when sending over the network.)

            h3. Crash recovery
            # Validate the {{ib_logfile0}} header, including checking the size of the {{ib_logdata}} file.
            # Read the last bytes of {{ib_logfile0}} to see if it ends in a valid {{FILE_CHECKPOINT}} record. If not, read the entire {{ib_logfile0}} to find the latest {{FILE_CHECKPOINT}} record.
            # If {{ib_logdata}} ends at the identified byte position (we got sequence bit mismatch or checksum mismatch), then the database was clean.
            # Else, recover the file name information based on the {{ib_logfile0}} contents and start recovery.
            # Parse the {{ib_logdata}} until the end. If any unknown tablespace identifiers exist, abort startup unless {{innodb_force_recovery≥1}}. Ignore records for those tablespaces for which {{FILE_DELETE}} had been recorded.
            # Replay {{FILE_DELETE}} and {{FILE_RENAME}} operations.
            # Shrink any files for which we parsed a {{TRIM_PAGES}} record, and extend any files according to changes to {{FSP_SIZE}}.
            # Apply the parsed log to all data files.

            h3. Considerations on checkpoints and writing {{ib_logdata}}
            After completing a checkpoint (the write of the {{FILE_CHECKPOINT}} record to {{ib_logfile0}}), we could punch a hole in {{ib_logdata}} to discard no-longer-needed log records.

            As an option, the redo log could be rebuilt on a checkpoint, by creating a logically empty set of log files, at the minimum consisting of an empty {{ib_logdata}} file and a {{ib_logfile0}} that contains {{FILE_ID}} records and a {{FILE_CHECKPOINT}} record.

            If redo log archiving is enabled, when the maximum configured size of a single redo log (which is stored in the {{ib_logfile0}} header) is reached, a new {{ib_logdata.%06u}} file will be created. The old file length must be exactly at the maximum length.

            Writes to {{ib_logfile0}} will *not* increment the LSN at all! This means that the redo log could be easily rebuilt at any LSN, and Mariabackup could write additional information to that file, removing the need for separate {{.delta}} files in incremental backups.
            The InnoDB redo log format is not optimal in many respects:
            * At the start of {{ib_logfile0}}, there are only two log checkpoint blocks. The rest of the log file is written in a circular fashion.
            * On log checkpoint, some file name information needs to be appended to the log.
            * File names that were first changed since the latest checkpoint must be appended to the log. The bookkeeping causes some contention on {{log_sys.mutex}} and {{fil_system.mutex}}.
            * The log file was unnecessarily split into multiple files, logically treated as one big circular file. (MDEV-20907 in MariaDB Server 10.5.0 change the default to 1 file, and later the parameter was deprecated and ignored.)
            * Log records are divided into tiny blocks of 512 bytes, with 12+4 bytes of header and footer (12+8 bytes with MDEV-12041 {{innodb_encrypt_log}} (10.4.0)).
            * We are holding a mutex while zero-filling unused parts of log blocks, encrypting log blocks, or computing checksums.
            * Mariabackup cannot copy the log without having access to the encryption keys. (It can copy data file pages without encrypting them.)

            We had some ideas to move to an append-only file and to partition the log into multiple files, but it turned out that a single fixed-size circular log file would perform best in typical scenarios.

            We should keep the first 512 bytes of the file {{ib_logfile0}} for compatibility purposes, but everything else could be improved.
            * {{ib_logfile0}} (after the 512-byte header) will be append-only, unencrypted, for records containing file names and checkpoint information. A checkpoint record will comprise an LSN and a byte offset in a separate, optionally encrypted, circular log file {{ib_logdata}}. The length of each record is explicitly tagged and the payload will be followed by CRC-32C.
            * The {{ib_logdata}} file can be append-only or circular. If it is circular, its fixed size must be an integer multiple of 512 bytes.
            * We remove log block headers and footers. We really only need is to detect the logical end of the circular log in {{ib_logdata}}. That can be achieved by making sure that mini-transactions are terminated by a sequence number (at least one bit) and a checksum. When the circular file wraps around, the sequence number will be incremented (or the sequence bit toggled).
            * For page-aligned I/O, allow dummy records to be written, to indicate that the next bytes (until the end of the physical page, no matter what the I/O block size is) must be ignored. Thus, there is no need to initialize any padding bytes or encrypt or compute checksums on them.
            * Encrypt and compute checksums on {{mtr_t::m_log}} before initiating a write to the circular log file. The log can be copied and checksums validated without access to encryption keys.
            * If {{ib_logdata}} is on a memory-mapped persistent memory device, then we could bypass {{log_sys.buf}} and the file system copy the data directly to the memory-mapped area and flush the CPU cache.

            Some old InnoDB redo log parameters will be deprecated and ignored:
            * {{innodb_log_files_in_group}} (deprecated & ignored in MariaDB 10.5.1)
            * {{innodb_log_checksums}} (already set by default, and forced when {{innodb_encrypt_log}})

            The parameter {{innodb_log_write_ahead_size}} may be repurposed to indicate the desired physical log block size. For PMEM, it might be 64 bytes (the width of a CPU cache line).

            h2. The checkpoint and file operations log file {{ib_logfile0}}
            The file name {{ib_logfile0}} and the existing format of the first 512 bytes will be retained for the purpose of upgrading and preventing downgrading. In the first 512 bytes of the file, the following information will be present:
            * InnoDB redo log format version identifier (in the format introduced by MySQL 5.7.9/MariaDB 10.2.2)
            * The size of the {{ib_logdata}} file (the maximum possible size of the file if it is append-only).
            * All encryption parameters for {{ib_logdata}}
            * CRC-32C checksum
            This file will be append-only. After the first 512 bytes, the format will be:
            * Optional: {{FILE_ID}} records (renamed from {{FILE_MODIFY}}) of all {{.ibd}} files that existed when the redo log file was created. Each record terminated by CRC-32C.
            * {{FILE_CHECKPOINT}} record with LSN and byte offset in {{ib_logdata}}, and sequence bit (in circular {{ib_logdata}} file), and CRC-32C. The byte offset will be at least 47 bits (128 TiB). Total size of a checkpoint: at least 1+8+6+4=19 bytes.
            * Optional: {{FILE_CREATE}}, {{FILE_DELETE}}, {{FILE_RENAME}} records and further {{FILE_CHECKPOINT}} records for any file operations or subsequent checkpoints since the redo log was created. Each record will be terminated by CRC-32C.

            All writes to {{ib_logfile0}} will be synchronous and durable ({{O_DSYNC}}, {{fdatasync()}} or {{O_SYNC}}, {{fsync()}}) .

            h2. The circular or append-only page-level log file {{ib_logdata}}
            The {{ib_logdata}} file will contain MDEV-12353 records except {{FILE_}} records that will be written into {{ib_logfile0}}. Each mini-transaction will be followed by a CRC-32C of all the bytes (using the value 0 for the sequence bit or number), so that Mariabackup can avoid recomputing the checksum while copying the log to a new file.

            If the {{ib_logdata}} is append-only, then we could enable log archiving by actually writing to files like {{ib_logdata.000000}}, {{ib_logdata.000001}} and so on.

            h3. Payload encoding
            The exact encoding of {{ib_logdata}} is not decided yet. We want to avoid overwriting the last log block, so we cannot have an explicit 'end of log' marker. We must associate each mini-transaction (atomic sequence of log records) with a sequence number (at the minimum, a sequence bit) and a checksum. The 4-byte CRC-32C is a good candidate, because it is already being used in data page checksums.

            We know that a CRC of nonzero bytes must be nonzero, and the mini-transaction payload cannot be zero, hence the CRC bytes can never be zero.
            Because of this, it would be beneficial to zero-initialize all the skipped bytes to increase the probability of quick detection of the end of the circular log. Compared to the existing format, we would avoid CRC computation or encryption of the skipped bytes, but the zero-filling would still have to be protected with a mutex.

            h4. Alternative: Prepending a CRC to each MDEV-12353 mini-transaction
            In the MDEV-12353 encoding, a record cannot start with the bytes {{0x00}} or {{0x01}}. Mini-transactions are currently being terminated by the byte {{0x00}}. We could store the sequence bit in the terminating byte of the mini-transaction. The checksum would exclude the terminating byte.

            Only the payload bytes would be encrypted (not record types or lengths, and not page identifiers either). In that way, records can be parsed and validated efficiently. Decryption would only have to be invoked when the log really needs to be applied on the page. The initialization vector for encryption and decryption can include the unencrypted record header bytes.

            It is best to store the CRC before the mini-transaction payload, because the CRC cannot be 0. Hence, we can detect the end of the log without even parsing the mini-transaction bytes.

            For circular log files, we can introduce a special mini-transaction 'Skip the next N bytes', encoded in sizeof(CRC)+2+log(N) bytes: CRC, record type and length, subtype and the value of the sequence bit, and variable-length encoded N. If we need to pad a block with fewer bytes than the minimum size, we would write a record to skip the minimum size.

            Pros: Minimal overhead: sizeof(CRC) bytes per mini-transaction.
            Cons: Recovery may have to parse a lot of log before determining that the end of the log was reached.

            h4. Alternative: Prepending a mini-transaction header with length and CRC
            We could encapsulate MDEV-12353 records (without the mini-transaction terminating NUL byte) in the following structure:
            * variable-length encoded integer of {{total_length << 2 | sequence_bit}}
            * CRC of the data payload and the variable-length encoded integer
            * the data payload (MDEV-12353 records); could be encrypted in their entirety

            Skipped bytes (at least 5) would be indicated by the following:
            * variable-length encoded integer of {{skipped_length << 2 | 1 << 1 | sequence_bit}}
            * CRC of the variable-length encoded integer (not including the skipped bytes)

            Pros: Recovery can determine more quickly that the end of the circular log was reached, thanks to the length, sequence bit and (nonzero) CRC being stored at the start.
            Pros: More of the log could be encrypted (at the cost of recovery and backup restoration speed)
            Cons: Slightly more overhead: sizeof(CRC)+log(length * 4) bytes. For length<32 bytes, no change of overhead.

            h3. Log writing and synchronous flushing
            For the bulk of the changes done by mini-transactions, we do not care about flushing. The file system can write log file blocks as it pleases.

            Some state changes of the database must be made durable at a specific time. Examples include user transaction {{COMMIT}}, {{XA PREPARE}}, {{XA ROLLBACK}}, and (in case the binlog is not enabled) {{XA COMMIT}}.

            Whenever we want to make a certain change durable, we must flush all log files up to the LSN of the mini-transaction commit that made the change. While doing this, we can pad each log file to the file system block size (the 'skipped bytes'), so that the file system can always write full blocks. The padding could also be helpful when trying to resurrect a corrupted redo log file.

            If redo log is physically replicated to the buffer pools of slaves (like in PolarDB), then we should first write to the redo log files and only then to the slaves, and we should assume that the writes to the files will always eventually be durable. If that assumption is broken, then all servers would have to be restarted and perform crash recovery. Because the log sequence number will be counted in bytes, we will have to replicate the number of skipped bytes. (But we could omit the skipped bytes themselves when sending over the network.)

            h3. Crash recovery
            # Validate the {{ib_logfile0}} header, including checking the size of the {{ib_logdata}} file.
            # Read the last bytes of {{ib_logfile0}} to see if it ends in a valid {{FILE_CHECKPOINT}} record. If not, read the entire {{ib_logfile0}} to find the latest {{FILE_CHECKPOINT}} record. (The last write to {{ib_logfile0}} could have been incomplete, and we may have to trim that file.)
            # If {{ib_logdata}} ends at the identified byte position (we got sequence bit mismatch or checksum mismatch), then the database was clean.
            # Else, recover the file name information based on the {{ib_logfile0}} contents and start recovery.
            # Parse the {{ib_logdata}} until the end. If any tablespace identifiers refer to unknown or inaccessible data files, abort startup unless {{innodb_force_recovery≥1}}. Ignore records for those tablespaces for which {{FILE_DELETE}} had been recorded.
            # Now that the log has been validated, start the modifications in recovery. First, trim {{ib_logfile0}} if needed, and replay {{FILE_DELETE}} and {{FILE_RENAME}} operations.
            # Shrink any files for which we parsed a {{TRIM_PAGES}} record, and extend any files according to changes to {{FSP_SIZE}}.
            # Apply the parsed log to all data files.

            h3. Considerations on checkpoints and writing {{ib_logdata}}
            After completing a checkpoint (the write of the {{FILE_CHECKPOINT}} record to {{ib_logfile0}}), we could punch a hole in {{ib_logdata}} to discard no-longer-needed log records.

            As an option, the redo log could be rebuilt on a checkpoint, by creating a logically empty set of log files, at the minimum consisting of an empty {{ib_logdata}} file and a {{ib_logfile0}} that contains {{FILE_ID}} records and a {{FILE_CHECKPOINT}} record.

            If redo log archiving is enabled, when the maximum configured size of a single redo log (which is stored in the {{ib_logfile0}} header) is reached, a new {{ib_logdata.%06u}} file will be created. The old file length must be exactly at the maximum length.

            Writes to {{ib_logfile0}} will *not* increment the LSN at all! This means that the redo log could be easily rebuilt at any LSN, and Mariabackup could write additional information to that file, removing the need for separate {{.delta}} files in incremental backups.
            julien.fritsch Julien Fritsch made changes -
            julien.fritsch Julien Fritsch made changes -
            Priority Major [ 3 ] Critical [ 2 ]
            marko Marko Mäkelä made changes -
            kevg Eugene Kosov (Inactive) made changes -
            marko Marko Mäkelä made changes -
            marko Marko Mäkelä made changes -
            marko Marko Mäkelä made changes -
            serg Sergei Golubchik made changes -
            Priority Critical [ 2 ] Major [ 3 ]
            ralf.gebhardt Ralf Gebhardt made changes -
            Priority Major [ 3 ] Critical [ 2 ]
            marko Marko Mäkelä made changes -
            Status Open [ 1 ] In Progress [ 3 ]
            marko Marko Mäkelä made changes -
            marko Marko Mäkelä made changes -
            Status In Progress [ 3 ] Stalled [ 10000 ]
            marko Marko Mäkelä made changes -
            Fix Version/s 10.7 [ 24805 ]
            Fix Version/s 10.6 [ 24028 ]
            marko Marko Mäkelä made changes -
            Status Stalled [ 10000 ] In Progress [ 3 ]
            serg Sergei Golubchik made changes -
            Priority Critical [ 2 ] Major [ 3 ]
            ralf.gebhardt Ralf Gebhardt made changes -
            Fix Version/s 10.8 [ 26121 ]
            Fix Version/s 10.7 [ 24805 ]
            serg Sergei Golubchik made changes -
            Priority Major [ 3 ] Critical [ 2 ]
            serg Sergei Golubchik made changes -
            Workflow MariaDB v3 [ 83863 ] MariaDB v4 [ 131821 ]
            marko Marko Mäkelä made changes -
            marko Marko Mäkelä made changes -
            Status In Progress [ 3 ] In Testing [ 10301 ]
            marko Marko Mäkelä made changes -
            axel Axel Schwenke made changes -
            Attachment MDEV-14425.pdf [ 61254 ]
            marko Marko Mäkelä made changes -
            Assignee Marko Mäkelä [ marko ] Matthias Leich [ mleich ]
            marko Marko Mäkelä made changes -
            Status In Testing [ 10301 ] Closed [ 6 ]
            marko Marko Mäkelä made changes -
            Status Closed [ 6 ] Stalled [ 10000 ]
            marko Marko Mäkelä made changes -
            Status Stalled [ 10000 ] In Testing [ 10301 ]
            danblack Daniel Black made changes -
            ramesh Ramesh Sivaraman made changes -
            marko Marko Mäkelä made changes -
            axel Axel Schwenke made changes -
            Attachment 81cf92e9471.pdf [ 61645 ]
            ramesh Ramesh Sivaraman made changes -
            marko Marko Mäkelä made changes -
            marko Marko Mäkelä made changes -
            axel Axel Schwenke made changes -
            Attachment NUMA_1.pdf [ 61713 ]
            Attachment NUMA_1vs2.pdf [ 61714 ]
            Attachment NUMA_2.pdf [ 61715 ]
            marko Marko Mäkelä made changes -
            Description The InnoDB redo log format is not optimal in many respects:
            * At the start of {{ib_logfile0}}, there are only two log checkpoint blocks. The rest of the log file is written in a circular fashion.
            * On log checkpoint, some file name information needs to be appended to the log.
            * File names that were first changed since the latest checkpoint must be appended to the log. The bookkeeping causes some contention on {{log_sys.mutex}} and {{fil_system.mutex}}.
            * The log file was unnecessarily split into multiple files, logically treated as one big circular file. (MDEV-20907 in MariaDB Server 10.5.0 change the default to 1 file, and later the parameter was deprecated and ignored.)
            * Log records are divided into tiny blocks of 512 bytes, with 12+4 bytes of header and footer (12+8 bytes with MDEV-12041 {{innodb_encrypt_log}} (10.4.0)).
            * We are holding a mutex while zero-filling unused parts of log blocks, encrypting log blocks, or computing checksums.
            * Mariabackup cannot copy the log without having access to the encryption keys. (It can copy data file pages without encrypting them.)

            We had some ideas to move to an append-only file and to partition the log into multiple files, but it turned out that a single fixed-size circular log file would perform best in typical scenarios.

            We should keep the first 512 bytes of the file {{ib_logfile0}} for compatibility purposes, but everything else could be improved.
            * {{ib_logfile0}} (after the 512-byte header) will be append-only, unencrypted, for records containing file names and checkpoint information. A checkpoint record will comprise an LSN and a byte offset in a separate, optionally encrypted, circular log file {{ib_logdata}}. The length of each record is explicitly tagged and the payload will be followed by CRC-32C.
            * The {{ib_logdata}} file can be append-only or circular. If it is circular, its fixed size must be an integer multiple of 512 bytes.
            * We remove log block headers and footers. We really only need is to detect the logical end of the circular log in {{ib_logdata}}. That can be achieved by making sure that mini-transactions are terminated by a sequence number (at least one bit) and a checksum. When the circular file wraps around, the sequence number will be incremented (or the sequence bit toggled).
            * For page-aligned I/O, allow dummy records to be written, to indicate that the next bytes (until the end of the physical page, no matter what the I/O block size is) must be ignored. Thus, there is no need to initialize any padding bytes or encrypt or compute checksums on them.
            * Encrypt and compute checksums on {{mtr_t::m_log}} before initiating a write to the circular log file. The log can be copied and checksums validated without access to encryption keys.
            * If {{ib_logdata}} is on a memory-mapped persistent memory device, then we could bypass {{log_sys.buf}} and the file system copy the data directly to the memory-mapped area and flush the CPU cache.

            Some old InnoDB redo log parameters will be deprecated and ignored:
            * {{innodb_log_files_in_group}} (deprecated & ignored in MariaDB 10.5.1)
            * {{innodb_log_checksums}} (already set by default, and forced when {{innodb_encrypt_log}})

            The parameter {{innodb_log_write_ahead_size}} may be repurposed to indicate the desired physical log block size. For PMEM, it might be 64 bytes (the width of a CPU cache line).

            h2. The checkpoint and file operations log file {{ib_logfile0}}
            The file name {{ib_logfile0}} and the existing format of the first 512 bytes will be retained for the purpose of upgrading and preventing downgrading. In the first 512 bytes of the file, the following information will be present:
            * InnoDB redo log format version identifier (in the format introduced by MySQL 5.7.9/MariaDB 10.2.2)
            * The size of the {{ib_logdata}} file (the maximum possible size of the file if it is append-only).
            * All encryption parameters for {{ib_logdata}}
            * CRC-32C checksum
            This file will be append-only. After the first 512 bytes, the format will be:
            * Optional: {{FILE_ID}} records (renamed from {{FILE_MODIFY}}) of all {{.ibd}} files that existed when the redo log file was created. Each record terminated by CRC-32C.
            * {{FILE_CHECKPOINT}} record with LSN and byte offset in {{ib_logdata}}, and sequence bit (in circular {{ib_logdata}} file), and CRC-32C. The byte offset will be at least 47 bits (128 TiB). Total size of a checkpoint: at least 1+8+6+4=19 bytes.
            * Optional: {{FILE_CREATE}}, {{FILE_DELETE}}, {{FILE_RENAME}} records and further {{FILE_CHECKPOINT}} records for any file operations or subsequent checkpoints since the redo log was created. Each record will be terminated by CRC-32C.

            All writes to {{ib_logfile0}} will be synchronous and durable ({{O_DSYNC}}, {{fdatasync()}} or {{O_SYNC}}, {{fsync()}}) .

            h2. The circular or append-only page-level log file {{ib_logdata}}
            The {{ib_logdata}} file will contain MDEV-12353 records except {{FILE_}} records that will be written into {{ib_logfile0}}. Each mini-transaction will be followed by a CRC-32C of all the bytes (using the value 0 for the sequence bit or number), so that Mariabackup can avoid recomputing the checksum while copying the log to a new file.

            If the {{ib_logdata}} is append-only, then we could enable log archiving by actually writing to files like {{ib_logdata.000000}}, {{ib_logdata.000001}} and so on.

            h3. Payload encoding
            The exact encoding of {{ib_logdata}} is not decided yet. We want to avoid overwriting the last log block, so we cannot have an explicit 'end of log' marker. We must associate each mini-transaction (atomic sequence of log records) with a sequence number (at the minimum, a sequence bit) and a checksum. The 4-byte CRC-32C is a good candidate, because it is already being used in data page checksums.

            We know that a CRC of nonzero bytes must be nonzero, and the mini-transaction payload cannot be zero, hence the CRC bytes can never be zero.
            Because of this, it would be beneficial to zero-initialize all the skipped bytes to increase the probability of quick detection of the end of the circular log. Compared to the existing format, we would avoid CRC computation or encryption of the skipped bytes, but the zero-filling would still have to be protected with a mutex.

            h4. Alternative: Prepending a CRC to each MDEV-12353 mini-transaction
            In the MDEV-12353 encoding, a record cannot start with the bytes {{0x00}} or {{0x01}}. Mini-transactions are currently being terminated by the byte {{0x00}}. We could store the sequence bit in the terminating byte of the mini-transaction. The checksum would exclude the terminating byte.

            Only the payload bytes would be encrypted (not record types or lengths, and not page identifiers either). In that way, records can be parsed and validated efficiently. Decryption would only have to be invoked when the log really needs to be applied on the page. The initialization vector for encryption and decryption can include the unencrypted record header bytes.

            It is best to store the CRC before the mini-transaction payload, because the CRC cannot be 0. Hence, we can detect the end of the log without even parsing the mini-transaction bytes.

            For circular log files, we can introduce a special mini-transaction 'Skip the next N bytes', encoded in sizeof(CRC)+2+log(N) bytes: CRC, record type and length, subtype and the value of the sequence bit, and variable-length encoded N. If we need to pad a block with fewer bytes than the minimum size, we would write a record to skip the minimum size.

            Pros: Minimal overhead: sizeof(CRC) bytes per mini-transaction.
            Cons: Recovery may have to parse a lot of log before determining that the end of the log was reached.

            h4. Alternative: Prepending a mini-transaction header with length and CRC
            We could encapsulate MDEV-12353 records (without the mini-transaction terminating NUL byte) in the following structure:
            * variable-length encoded integer of {{total_length << 2 | sequence_bit}}
            * CRC of the data payload and the variable-length encoded integer
            * the data payload (MDEV-12353 records); could be encrypted in their entirety

            Skipped bytes (at least 5) would be indicated by the following:
            * variable-length encoded integer of {{skipped_length << 2 | 1 << 1 | sequence_bit}}
            * CRC of the variable-length encoded integer (not including the skipped bytes)

            Pros: Recovery can determine more quickly that the end of the circular log was reached, thanks to the length, sequence bit and (nonzero) CRC being stored at the start.
            Pros: More of the log could be encrypted (at the cost of recovery and backup restoration speed)
            Cons: Slightly more overhead: sizeof(CRC)+log(length * 4) bytes. For length<32 bytes, no change of overhead.

            h3. Log writing and synchronous flushing
            For the bulk of the changes done by mini-transactions, we do not care about flushing. The file system can write log file blocks as it pleases.

            Some state changes of the database must be made durable at a specific time. Examples include user transaction {{COMMIT}}, {{XA PREPARE}}, {{XA ROLLBACK}}, and (in case the binlog is not enabled) {{XA COMMIT}}.

            Whenever we want to make a certain change durable, we must flush all log files up to the LSN of the mini-transaction commit that made the change. While doing this, we can pad each log file to the file system block size (the 'skipped bytes'), so that the file system can always write full blocks. The padding could also be helpful when trying to resurrect a corrupted redo log file.

            If redo log is physically replicated to the buffer pools of slaves (like in PolarDB), then we should first write to the redo log files and only then to the slaves, and we should assume that the writes to the files will always eventually be durable. If that assumption is broken, then all servers would have to be restarted and perform crash recovery. Because the log sequence number will be counted in bytes, we will have to replicate the number of skipped bytes. (But we could omit the skipped bytes themselves when sending over the network.)

            h3. Crash recovery
            # Validate the {{ib_logfile0}} header, including checking the size of the {{ib_logdata}} file.
            # Read the last bytes of {{ib_logfile0}} to see if it ends in a valid {{FILE_CHECKPOINT}} record. If not, read the entire {{ib_logfile0}} to find the latest {{FILE_CHECKPOINT}} record. (The last write to {{ib_logfile0}} could have been incomplete, and we may have to trim that file.)
            # If {{ib_logdata}} ends at the identified byte position (we got sequence bit mismatch or checksum mismatch), then the database was clean.
            # Else, recover the file name information based on the {{ib_logfile0}} contents and start recovery.
            # Parse the {{ib_logdata}} until the end. If any tablespace identifiers refer to unknown or inaccessible data files, abort startup unless {{innodb_force_recovery≥1}}. Ignore records for those tablespaces for which {{FILE_DELETE}} had been recorded.
            # Now that the log has been validated, start the modifications in recovery. First, trim {{ib_logfile0}} if needed, and replay {{FILE_DELETE}} and {{FILE_RENAME}} operations.
            # Shrink any files for which we parsed a {{TRIM_PAGES}} record, and extend any files according to changes to {{FSP_SIZE}}.
            # Apply the parsed log to all data files.

            h3. Considerations on checkpoints and writing {{ib_logdata}}
            After completing a checkpoint (the write of the {{FILE_CHECKPOINT}} record to {{ib_logfile0}}), we could punch a hole in {{ib_logdata}} to discard no-longer-needed log records.

            As an option, the redo log could be rebuilt on a checkpoint, by creating a logically empty set of log files, at the minimum consisting of an empty {{ib_logdata}} file and a {{ib_logfile0}} that contains {{FILE_ID}} records and a {{FILE_CHECKPOINT}} record.

            If redo log archiving is enabled, when the maximum configured size of a single redo log (which is stored in the {{ib_logfile0}} header) is reached, a new {{ib_logdata.%06u}} file will be created. The old file length must be exactly at the maximum length.

            Writes to {{ib_logfile0}} will *not* increment the LSN at all! This means that the redo log could be easily rebuilt at any LSN, and Mariabackup could write additional information to that file, removing the need for separate {{.delta}} files in incremental backups.
            The InnoDB redo log format is not optimal in many respects:
            * At the start of {{ib_logfile0}}, there are two log checkpoint blocks, only 1024 bytes apart, while there exist devices with 4096-byte block size. The rest of the log file is written in a circular fashion.
            * On log checkpoint, some file name information needs to be appended to the log.
            * File names that were first changed since the latest checkpoint must be appended to the log. The bookkeeping causes some contention on {{log_sys.mutex}}- and {{fil_system.mutex}}-. Edit: The contention on {{fil_system.mutex}} was practically removed in MDEV-23855, and the contention on {{log_sys.mutex}} due to this is minimal.
            * The log file was unnecessarily split into multiple files, logically treated as one big circular file. (MDEV-20907 in MariaDB Server 10.5.0 change the default to 1 file, and later the parameter was deprecated and ignored.)
            * Log records are divided into tiny blocks of 512 bytes, with 12+4 bytes of header and footer (12+8 bytes with MDEV-12041 {{innodb_encrypt_log}} (10.4.0)).
            * We are holding a mutex while zero-filling unused parts of log blocks, encrypting log blocks, or computing checksums.
            * Mariabackup cannot copy the log without having access to the encryption keys. (It can copy data file pages without encrypting them.)

            We had some ideas to move to an append-only file and to partition the log into multiple files, but it turned out that a single fixed-size circular log file would perform best in typical scenarios.

            To address the {{fil_system.mutex}} contention whose root cause was later fixed in MDEV-23855, we were considering to split the log as follows:
            * {{ib_logfile0}} (after the 512-byte header) will be append-only, unencrypted, for records containing file names and checkpoint information. A checkpoint record will comprise an LSN and a byte offset in a separate, optionally encrypted, circular log file {{ib_logdata}}. The length of each record is explicitly tagged and the payload will be followed by CRC-32C.
            * The {{ib_logdata}} file can be append-only or circular. If it is circular, its fixed size must be an integer multiple of 512 bytes.

            One problem would have had to be solved: When would the {{ib_logfile0}} be shrunk? No storage is unlimited.

            So, we will keep the {{ib_logfile0}} and the basic format of its first 512 bytes for compatibility purposes, but other features could be improved.
            * We remove log block headers and footers. We really only need is to detect the logical end of the circular log in {{ib_logdata}}. That can be achieved by making sure that mini-transactions are terminated by a sequence number (at least one bit) and a checksum. When the circular file wraps around, the sequence number will be incremented (or the sequence bit toggled).
            * For page-aligned I/O, we allow dummy records to be written, to indicate that the next bytes (until the end of the physical block, no matter what the I/O block size is) must be ignored. (The log parser will ignore these padding records, but we do not currently write them; we will keep overwriting the last physical block until it has been completely filled like we used to do until now.)
            * Encrypt and compute checksum on {{mtr_t::m_log}} before initiating a write to the circular log file. The log can be copied and checksum validated without access to encryption keys.
            * If the log is on a memory-mapped persistent memory device, then we will make {{log_sys.buf}} point directly to the persistent memory.

            Some old InnoDB redo log parameters will be deprecated and ignored:
            * {{innodb_log_files_in_group}} (deprecated & ignored in MariaDB 10.5.1; removed by )
            * {{innodb_log_checksums}} (removed by MDEV-23397 in MariaDB 10.6.0)
            *

            We should keep


            The parameter {{innodb_log_write_ahead_size}} may be repurposed to indicate the desired physical log block size. For PMEM, it might be 64 bytes (the width of a CPU cache line).

            h2. The checkpoint and file operations log file {{ib_logfile0}}
            The file name {{ib_logfile0}} and the existing format of the first 512 bytes will be retained for the purpose of upgrading and preventing downgrading. In the first 512 bytes of the file, the following information will be present:
            * InnoDB redo log format version identifier (in the format introduced by MySQL 5.7.9/MariaDB 10.2.2)
            * The size of the {{ib_logdata}} file (the maximum possible size of the file if it is append-only).
            * All encryption parameters for {{ib_logdata}}
            * CRC-32C checksum
            This file will be append-only. After the first 512 bytes, the format will be:
            * Optional: {{FILE_ID}} records (renamed from {{FILE_MODIFY}}) of all {{.ibd}} files that existed when the redo log file was created. Each record terminated by CRC-32C.
            * {{FILE_CHECKPOINT}} record with LSN and byte offset in {{ib_logdata}}, and sequence bit (in circular {{ib_logdata}} file), and CRC-32C. The byte offset will be at least 47 bits (128 TiB). Total size of a checkpoint: at least 1+8+6+4=19 bytes.
            * Optional: {{FILE_CREATE}}, {{FILE_DELETE}}, {{FILE_RENAME}} records and further {{FILE_CHECKPOINT}} records for any file operations or subsequent checkpoints since the redo log was created. Each record will be terminated by CRC-32C.

            All writes to {{ib_logfile0}} will be synchronous and durable ({{O_DSYNC}}, {{fdatasync()}} or {{O_SYNC}}, {{fsync()}}) .

            h2. The circular or append-only page-level log file {{ib_logdata}}
            The {{ib_logdata}} file will contain MDEV-12353 records except {{FILE_}} records that will be written into {{ib_logfile0}}. Each mini-transaction will be followed by a CRC-32C of all the bytes (using the value 0 for the sequence bit or number), so that Mariabackup can avoid recomputing the checksum while copying the log to a new file.

            If the {{ib_logdata}} is append-only, then we could enable log archiving by actually writing to files like {{ib_logdata.000000}}, {{ib_logdata.000001}} and so on.

            h3. Payload encoding
            The exact encoding of {{ib_logdata}} is not decided yet. We want to avoid overwriting the last log block, so we cannot have an explicit 'end of log' marker. We must associate each mini-transaction (atomic sequence of log records) with a sequence number (at the minimum, a sequence bit) and a checksum. The 4-byte CRC-32C is a good candidate, because it is already being used in data page checksums.

            We know that a CRC of nonzero bytes must be nonzero, and the mini-transaction payload cannot be zero, hence the CRC bytes can never be zero.
            Because of this, it would be beneficial to zero-initialize all the skipped bytes to increase the probability of quick detection of the end of the circular log. Compared to the existing format, we would avoid CRC computation or encryption of the skipped bytes, but the zero-filling would still have to be protected with a mutex.

            h4. Alternative: Prepending a CRC to each MDEV-12353 mini-transaction
            In the MDEV-12353 encoding, a record cannot start with the bytes {{0x00}} or {{0x01}}. Mini-transactions are currently being terminated by the byte {{0x00}}. We could store the sequence bit in the terminating byte of the mini-transaction. The checksum would exclude the terminating byte.

            Only the payload bytes would be encrypted (not record types or lengths, and not page identifiers either). In that way, records can be parsed and validated efficiently. Decryption would only have to be invoked when the log really needs to be applied on the page. The initialization vector for encryption and decryption can include the unencrypted record header bytes.

            It is best to store the CRC before the mini-transaction payload, because the CRC cannot be 0. Hence, we can detect the end of the log without even parsing the mini-transaction bytes.

            For circular log files, we can introduce a special mini-transaction 'Skip the next N bytes', encoded in sizeof(CRC)+2+log(N) bytes: CRC, record type and length, subtype and the value of the sequence bit, and variable-length encoded N. If we need to pad a block with fewer bytes than the minimum size, we would write a record to skip the minimum size.

            Pros: Minimal overhead: sizeof(CRC) bytes per mini-transaction.
            Cons: Recovery may have to parse a lot of log before determining that the end of the log was reached.

            h4. Alternative: Prepending a mini-transaction header with length and CRC
            We could encapsulate MDEV-12353 records (without the mini-transaction terminating NUL byte) in the following structure:
            * variable-length encoded integer of {{total_length << 2 | sequence_bit}}
            * CRC of the data payload and the variable-length encoded integer
            * the data payload (MDEV-12353 records); could be encrypted in their entirety

            Skipped bytes (at least 5) would be indicated by the following:
            * variable-length encoded integer of {{skipped_length << 2 | 1 << 1 | sequence_bit}}
            * CRC of the variable-length encoded integer (not including the skipped bytes)

            Pros: Recovery can determine more quickly that the end of the circular log was reached, thanks to the length, sequence bit and (nonzero) CRC being stored at the start.
            Pros: More of the log could be encrypted (at the cost of recovery and backup restoration speed)
            Cons: Slightly more overhead: sizeof(CRC)+log(length * 4) bytes. For length<32 bytes, no change of overhead.

            h3. Log writing and synchronous flushing
            For the bulk of the changes done by mini-transactions, we do not care about flushing. The file system can write log file blocks as it pleases.

            Some state changes of the database must be made durable at a specific time. Examples include user transaction {{COMMIT}}, {{XA PREPARE}}, {{XA ROLLBACK}}, and (in case the binlog is not enabled) {{XA COMMIT}}.

            Whenever we want to make a certain change durable, we must flush all log files up to the LSN of the mini-transaction commit that made the change. While doing this, we can pad each log file to the file system block size (the 'skipped bytes'), so that the file system can always write full blocks. The padding could also be helpful when trying to resurrect a corrupted redo log file.

            If redo log is physically replicated to the buffer pools of slaves (like in PolarDB), then we should first write to the redo log files and only then to the slaves, and we should assume that the writes to the files will always eventually be durable. If that assumption is broken, then all servers would have to be restarted and perform crash recovery. Because the log sequence number will be counted in bytes, we will have to replicate the number of skipped bytes. (But we could omit the skipped bytes themselves when sending over the network.)

            h3. Crash recovery
            # Validate the {{ib_logfile0}} header, including checking the size of the {{ib_logdata}} file.
            # Read the last bytes of {{ib_logfile0}} to see if it ends in a valid {{FILE_CHECKPOINT}} record. If not, read the entire {{ib_logfile0}} to find the latest {{FILE_CHECKPOINT}} record. (The last write to {{ib_logfile0}} could have been incomplete, and we may have to trim that file.)
            # If {{ib_logdata}} ends at the identified byte position (we got sequence bit mismatch or checksum mismatch), then the database was clean.
            # Else, recover the file name information based on the {{ib_logfile0}} contents and start recovery.
            # Parse the {{ib_logdata}} until the end. If any tablespace identifiers refer to unknown or inaccessible data files, abort startup unless {{innodb_force_recovery≥1}}. Ignore records for those tablespaces for which {{FILE_DELETE}} had been recorded.
            # Now that the log has been validated, start the modifications in recovery. First, trim {{ib_logfile0}} if needed, and replay {{FILE_DELETE}} and {{FILE_RENAME}} operations.
            # Shrink any files for which we parsed a {{TRIM_PAGES}} record, and extend any files according to changes to {{FSP_SIZE}}.
            # Apply the parsed log to all data files.

            h3. Considerations on checkpoints and writing {{ib_logdata}}
            After completing a checkpoint (the write of the {{FILE_CHECKPOINT}} record to {{ib_logfile0}}), we could punch a hole in {{ib_logdata}} to discard no-longer-needed log records.

            As an option, the redo log could be rebuilt on a checkpoint, by creating a logically empty set of log files, at the minimum consisting of an empty {{ib_logdata}} file and a {{ib_logfile0}} that contains {{FILE_ID}} records and a {{FILE_CHECKPOINT}} record.

            If redo log archiving is enabled, when the maximum configured size of a single redo log (which is stored in the {{ib_logfile0}} header) is reached, a new {{ib_logdata.%06u}} file will be created. The old file length must be exactly at the maximum length.

            Writes to {{ib_logfile0}} will *not* increment the LSN at all! This means that the redo log could be easily rebuilt at any LSN, and Mariabackup could write additional information to that file, removing the need for separate {{.delta}} files in incremental backups.
            marko Marko Mäkelä made changes -
            Description The InnoDB redo log format is not optimal in many respects:
            * At the start of {{ib_logfile0}}, there are two log checkpoint blocks, only 1024 bytes apart, while there exist devices with 4096-byte block size. The rest of the log file is written in a circular fashion.
            * On log checkpoint, some file name information needs to be appended to the log.
            * File names that were first changed since the latest checkpoint must be appended to the log. The bookkeeping causes some contention on {{log_sys.mutex}}- and {{fil_system.mutex}}-. Edit: The contention on {{fil_system.mutex}} was practically removed in MDEV-23855, and the contention on {{log_sys.mutex}} due to this is minimal.
            * The log file was unnecessarily split into multiple files, logically treated as one big circular file. (MDEV-20907 in MariaDB Server 10.5.0 change the default to 1 file, and later the parameter was deprecated and ignored.)
            * Log records are divided into tiny blocks of 512 bytes, with 12+4 bytes of header and footer (12+8 bytes with MDEV-12041 {{innodb_encrypt_log}} (10.4.0)).
            * We are holding a mutex while zero-filling unused parts of log blocks, encrypting log blocks, or computing checksums.
            * Mariabackup cannot copy the log without having access to the encryption keys. (It can copy data file pages without encrypting them.)

            We had some ideas to move to an append-only file and to partition the log into multiple files, but it turned out that a single fixed-size circular log file would perform best in typical scenarios.

            To address the {{fil_system.mutex}} contention whose root cause was later fixed in MDEV-23855, we were considering to split the log as follows:
            * {{ib_logfile0}} (after the 512-byte header) will be append-only, unencrypted, for records containing file names and checkpoint information. A checkpoint record will comprise an LSN and a byte offset in a separate, optionally encrypted, circular log file {{ib_logdata}}. The length of each record is explicitly tagged and the payload will be followed by CRC-32C.
            * The {{ib_logdata}} file can be append-only or circular. If it is circular, its fixed size must be an integer multiple of 512 bytes.

            One problem would have had to be solved: When would the {{ib_logfile0}} be shrunk? No storage is unlimited.

            So, we will keep the {{ib_logfile0}} and the basic format of its first 512 bytes for compatibility purposes, but other features could be improved.
            * We remove log block headers and footers. We really only need is to detect the logical end of the circular log in {{ib_logdata}}. That can be achieved by making sure that mini-transactions are terminated by a sequence number (at least one bit) and a checksum. When the circular file wraps around, the sequence number will be incremented (or the sequence bit toggled).
            * For page-aligned I/O, we allow dummy records to be written, to indicate that the next bytes (until the end of the physical block, no matter what the I/O block size is) must be ignored. (The log parser will ignore these padding records, but we do not currently write them; we will keep overwriting the last physical block until it has been completely filled like we used to do until now.)
            * Encrypt and compute checksum on {{mtr_t::m_log}} before initiating a write to the circular log file. The log can be copied and checksum validated without access to encryption keys.
            * If the log is on a memory-mapped persistent memory device, then we will make {{log_sys.buf}} point directly to the persistent memory.

            Some old InnoDB redo log parameters will be deprecated and ignored:
            * {{innodb_log_files_in_group}} (deprecated & ignored in MariaDB 10.5.1; removed by )
            * {{innodb_log_checksums}} (removed by MDEV-23397 in MariaDB 10.6.0)
            *

            We should keep


            The parameter {{innodb_log_write_ahead_size}} may be repurposed to indicate the desired physical log block size. For PMEM, it might be 64 bytes (the width of a CPU cache line).

            h2. The checkpoint and file operations log file {{ib_logfile0}}
            The file name {{ib_logfile0}} and the existing format of the first 512 bytes will be retained for the purpose of upgrading and preventing downgrading. In the first 512 bytes of the file, the following information will be present:
            * InnoDB redo log format version identifier (in the format introduced by MySQL 5.7.9/MariaDB 10.2.2)
            * The size of the {{ib_logdata}} file (the maximum possible size of the file if it is append-only).
            * All encryption parameters for {{ib_logdata}}
            * CRC-32C checksum
            This file will be append-only. After the first 512 bytes, the format will be:
            * Optional: {{FILE_ID}} records (renamed from {{FILE_MODIFY}}) of all {{.ibd}} files that existed when the redo log file was created. Each record terminated by CRC-32C.
            * {{FILE_CHECKPOINT}} record with LSN and byte offset in {{ib_logdata}}, and sequence bit (in circular {{ib_logdata}} file), and CRC-32C. The byte offset will be at least 47 bits (128 TiB). Total size of a checkpoint: at least 1+8+6+4=19 bytes.
            * Optional: {{FILE_CREATE}}, {{FILE_DELETE}}, {{FILE_RENAME}} records and further {{FILE_CHECKPOINT}} records for any file operations or subsequent checkpoints since the redo log was created. Each record will be terminated by CRC-32C.

            All writes to {{ib_logfile0}} will be synchronous and durable ({{O_DSYNC}}, {{fdatasync()}} or {{O_SYNC}}, {{fsync()}}) .

            h2. The circular or append-only page-level log file {{ib_logdata}}
            The {{ib_logdata}} file will contain MDEV-12353 records except {{FILE_}} records that will be written into {{ib_logfile0}}. Each mini-transaction will be followed by a CRC-32C of all the bytes (using the value 0 for the sequence bit or number), so that Mariabackup can avoid recomputing the checksum while copying the log to a new file.

            If the {{ib_logdata}} is append-only, then we could enable log archiving by actually writing to files like {{ib_logdata.000000}}, {{ib_logdata.000001}} and so on.

            h3. Payload encoding
            The exact encoding of {{ib_logdata}} is not decided yet. We want to avoid overwriting the last log block, so we cannot have an explicit 'end of log' marker. We must associate each mini-transaction (atomic sequence of log records) with a sequence number (at the minimum, a sequence bit) and a checksum. The 4-byte CRC-32C is a good candidate, because it is already being used in data page checksums.

            We know that a CRC of nonzero bytes must be nonzero, and the mini-transaction payload cannot be zero, hence the CRC bytes can never be zero.
            Because of this, it would be beneficial to zero-initialize all the skipped bytes to increase the probability of quick detection of the end of the circular log. Compared to the existing format, we would avoid CRC computation or encryption of the skipped bytes, but the zero-filling would still have to be protected with a mutex.

            h4. Alternative: Prepending a CRC to each MDEV-12353 mini-transaction
            In the MDEV-12353 encoding, a record cannot start with the bytes {{0x00}} or {{0x01}}. Mini-transactions are currently being terminated by the byte {{0x00}}. We could store the sequence bit in the terminating byte of the mini-transaction. The checksum would exclude the terminating byte.

            Only the payload bytes would be encrypted (not record types or lengths, and not page identifiers either). In that way, records can be parsed and validated efficiently. Decryption would only have to be invoked when the log really needs to be applied on the page. The initialization vector for encryption and decryption can include the unencrypted record header bytes.

            It is best to store the CRC before the mini-transaction payload, because the CRC cannot be 0. Hence, we can detect the end of the log without even parsing the mini-transaction bytes.

            For circular log files, we can introduce a special mini-transaction 'Skip the next N bytes', encoded in sizeof(CRC)+2+log(N) bytes: CRC, record type and length, subtype and the value of the sequence bit, and variable-length encoded N. If we need to pad a block with fewer bytes than the minimum size, we would write a record to skip the minimum size.

            Pros: Minimal overhead: sizeof(CRC) bytes per mini-transaction.
            Cons: Recovery may have to parse a lot of log before determining that the end of the log was reached.

            h4. Alternative: Prepending a mini-transaction header with length and CRC
            We could encapsulate MDEV-12353 records (without the mini-transaction terminating NUL byte) in the following structure:
            * variable-length encoded integer of {{total_length << 2 | sequence_bit}}
            * CRC of the data payload and the variable-length encoded integer
            * the data payload (MDEV-12353 records); could be encrypted in their entirety

            Skipped bytes (at least 5) would be indicated by the following:
            * variable-length encoded integer of {{skipped_length << 2 | 1 << 1 | sequence_bit}}
            * CRC of the variable-length encoded integer (not including the skipped bytes)

            Pros: Recovery can determine more quickly that the end of the circular log was reached, thanks to the length, sequence bit and (nonzero) CRC being stored at the start.
            Pros: More of the log could be encrypted (at the cost of recovery and backup restoration speed)
            Cons: Slightly more overhead: sizeof(CRC)+log(length * 4) bytes. For length<32 bytes, no change of overhead.

            h3. Log writing and synchronous flushing
            For the bulk of the changes done by mini-transactions, we do not care about flushing. The file system can write log file blocks as it pleases.

            Some state changes of the database must be made durable at a specific time. Examples include user transaction {{COMMIT}}, {{XA PREPARE}}, {{XA ROLLBACK}}, and (in case the binlog is not enabled) {{XA COMMIT}}.

            Whenever we want to make a certain change durable, we must flush all log files up to the LSN of the mini-transaction commit that made the change. While doing this, we can pad each log file to the file system block size (the 'skipped bytes'), so that the file system can always write full blocks. The padding could also be helpful when trying to resurrect a corrupted redo log file.

            If redo log is physically replicated to the buffer pools of slaves (like in PolarDB), then we should first write to the redo log files and only then to the slaves, and we should assume that the writes to the files will always eventually be durable. If that assumption is broken, then all servers would have to be restarted and perform crash recovery. Because the log sequence number will be counted in bytes, we will have to replicate the number of skipped bytes. (But we could omit the skipped bytes themselves when sending over the network.)

            h3. Crash recovery
            # Validate the {{ib_logfile0}} header, including checking the size of the {{ib_logdata}} file.
            # Read the last bytes of {{ib_logfile0}} to see if it ends in a valid {{FILE_CHECKPOINT}} record. If not, read the entire {{ib_logfile0}} to find the latest {{FILE_CHECKPOINT}} record. (The last write to {{ib_logfile0}} could have been incomplete, and we may have to trim that file.)
            # If {{ib_logdata}} ends at the identified byte position (we got sequence bit mismatch or checksum mismatch), then the database was clean.
            # Else, recover the file name information based on the {{ib_logfile0}} contents and start recovery.
            # Parse the {{ib_logdata}} until the end. If any tablespace identifiers refer to unknown or inaccessible data files, abort startup unless {{innodb_force_recovery≥1}}. Ignore records for those tablespaces for which {{FILE_DELETE}} had been recorded.
            # Now that the log has been validated, start the modifications in recovery. First, trim {{ib_logfile0}} if needed, and replay {{FILE_DELETE}} and {{FILE_RENAME}} operations.
            # Shrink any files for which we parsed a {{TRIM_PAGES}} record, and extend any files according to changes to {{FSP_SIZE}}.
            # Apply the parsed log to all data files.

            h3. Considerations on checkpoints and writing {{ib_logdata}}
            After completing a checkpoint (the write of the {{FILE_CHECKPOINT}} record to {{ib_logfile0}}), we could punch a hole in {{ib_logdata}} to discard no-longer-needed log records.

            As an option, the redo log could be rebuilt on a checkpoint, by creating a logically empty set of log files, at the minimum consisting of an empty {{ib_logdata}} file and a {{ib_logfile0}} that contains {{FILE_ID}} records and a {{FILE_CHECKPOINT}} record.

            If redo log archiving is enabled, when the maximum configured size of a single redo log (which is stored in the {{ib_logfile0}} header) is reached, a new {{ib_logdata.%06u}} file will be created. The old file length must be exactly at the maximum length.

            Writes to {{ib_logfile0}} will *not* increment the LSN at all! This means that the redo log could be easily rebuilt at any LSN, and Mariabackup could write additional information to that file, removing the need for separate {{.delta}} files in incremental backups.
            The InnoDB redo log format is not optimal in many respects:
            * At the start of {{ib_logfile0}}, there are two log checkpoint blocks, only 1024 bytes apart, while there exist devices with 4096-byte block size. The rest of the log file is written in a circular fashion.
            * On log checkpoint, some file name information needs to be appended to the log.
            * File names that were first changed since the latest checkpoint must be appended to the log. The bookkeeping causes some contention on {{log_sys.mutex}}- and {{fil_system.mutex}}-. Edit: The contention on {{fil_system.mutex}} was practically removed in MDEV-23855, and the contention on {{log_sys.mutex}} due to this is minimal.
            * The log file was unnecessarily split into multiple files, logically treated as one big circular file. (MDEV-20907 in MariaDB Server 10.5.0 change the default to 1 file, and later the parameter was deprecated and ignored.)
            * Log records are divided into tiny blocks of 512 bytes, with 12+4 bytes of header and footer (12+8 bytes with MDEV-12041 {{innodb_encrypt_log}} (10.4.0)).
            * We are holding a mutex while zero-filling unused parts of log blocks, encrypting log blocks, or computing checksums.
            * Mariabackup cannot copy the log without having access to the encryption keys. (It can copy data file pages without encrypting them.)

            We had some ideas to move to an append-only file and to partition the log into multiple files, but it turned out that a single fixed-size circular log file would perform best in typical scenarios.

            To address the {{fil_system.mutex}} contention whose root cause was later fixed in MDEV-23855, we were considering to split the log as follows:
            * {{ib_logfile0}} (after the 512-byte header) will be append-only, unencrypted, for records containing file names and checkpoint information. A checkpoint record will comprise an LSN and a byte offset in a separate, optionally encrypted, circular log file {{ib_logdata}}. The length of each record is explicitly tagged and the payload will be followed by CRC-32C.
            * The {{ib_logdata}} file can be append-only or circular. If it is circular, its fixed size must be an integer multiple of 512 bytes.

            One problem would have had to be solved: When would the {{ib_logfile0}} be shrunk? No storage is unlimited.

            So, we will keep the {{ib_logfile0}} and the basic format of its first 512 bytes for compatibility purposes, but other features could be improved.
            * We remove log block headers and footers. We really only need is to detect the logical end of the circular log in {{ib_logdata}}. That can be achieved by making sure that mini-transactions are terminated by a sequence number (at least one bit) and a checksum. When the circular file wraps around, the sequence number will be incremented (or the sequence bit toggled).
            * For page-aligned I/O, we allow dummy records to be written, to indicate that the next bytes (until the end of the physical block, no matter what the I/O block size is) must be ignored. (The log parser will ignore these padding records, but we do not currently write them; we will keep overwriting the last physical block until it has been completely filled like we used to do until now.)
            * Encrypt and compute checksum on {{mtr_t::m_log}} before initiating a write to the circular log file. The log can be copied and checksum validated without access to encryption keys.
            * If the log is on a memory-mapped persistent memory device, then we will make {{log_sys.buf}} point directly to the persistent memory.

            Some old InnoDB redo log parameters will be deprecated and ignored:
            * {{innodb_log_files_in_group}} (deprecated & ignored in MariaDB 10.5.1; removed by )
            * {{innodb_log_checksums}} (removed by MDEV-23397 in MariaDB 10.6.0)
            * {{innodb_log_write_ahead_size}}


            The parameter {{innodb_log_write_ahead_size}} may be repurposed to indicate the desired physical log block size. For PMEM, it might be 64 bytes (the width of a CPU cache line).

            h2. The checkpoint and file operations log file {{ib_logfile0}}
            The file name {{ib_logfile0}} and the existing format of the first 512 bytes will be retained for the purpose of upgrading and preventing downgrading. In the first 512 bytes of the file, the following information will be present:
            * InnoDB redo log format version identifier (in the format introduced by MySQL 5.7.9/MariaDB 10.2.2)
            * The size of the {{ib_logdata}} file (the maximum possible size of the file if it is append-only).
            * All encryption parameters for {{ib_logdata}}
            * CRC-32C checksum
            This file will be append-only. After the first 512 bytes, the format will be:
            * Optional: {{FILE_ID}} records (renamed from {{FILE_MODIFY}}) of all {{.ibd}} files that existed when the redo log file was created. Each record terminated by CRC-32C.
            * {{FILE_CHECKPOINT}} record with LSN and byte offset in {{ib_logdata}}, and sequence bit (in circular {{ib_logdata}} file), and CRC-32C. The byte offset will be at least 47 bits (128 TiB). Total size of a checkpoint: at least 1+8+6+4=19 bytes.
            * Optional: {{FILE_CREATE}}, {{FILE_DELETE}}, {{FILE_RENAME}} records and further {{FILE_CHECKPOINT}} records for any file operations or subsequent checkpoints since the redo log was created. Each record will be terminated by CRC-32C.

            All writes to {{ib_logfile0}} will be synchronous and durable ({{O_DSYNC}}, {{fdatasync()}} or {{O_SYNC}}, {{fsync()}}) .

            h2. The circular or append-only page-level log file {{ib_logdata}}
            The {{ib_logdata}} file will contain MDEV-12353 records except {{FILE_}} records that will be written into {{ib_logfile0}}. Each mini-transaction will be followed by a CRC-32C of all the bytes (using the value 0 for the sequence bit or number), so that Mariabackup can avoid recomputing the checksum while copying the log to a new file.

            If the {{ib_logdata}} is append-only, then we could enable log archiving by actually writing to files like {{ib_logdata.000000}}, {{ib_logdata.000001}} and so on.

            h3. Payload encoding
            The exact encoding of {{ib_logdata}} is not decided yet. We want to avoid overwriting the last log block, so we cannot have an explicit 'end of log' marker. We must associate each mini-transaction (atomic sequence of log records) with a sequence number (at the minimum, a sequence bit) and a checksum. The 4-byte CRC-32C is a good candidate, because it is already being used in data page checksums.

            We know that a CRC of nonzero bytes must be nonzero, and the mini-transaction payload cannot be zero, hence the CRC bytes can never be zero.
            Because of this, it would be beneficial to zero-initialize all the skipped bytes to increase the probability of quick detection of the end of the circular log. Compared to the existing format, we would avoid CRC computation or encryption of the skipped bytes, but the zero-filling would still have to be protected with a mutex.

            h4. Alternative: Prepending a CRC to each MDEV-12353 mini-transaction
            In the MDEV-12353 encoding, a record cannot start with the bytes {{0x00}} or {{0x01}}. Mini-transactions are currently being terminated by the byte {{0x00}}. We could store the sequence bit in the terminating byte of the mini-transaction. The checksum would exclude the terminating byte.

            Only the payload bytes would be encrypted (not record types or lengths, and not page identifiers either). In that way, records can be parsed and validated efficiently. Decryption would only have to be invoked when the log really needs to be applied on the page. The initialization vector for encryption and decryption can include the unencrypted record header bytes.

            It is best to store the CRC before the mini-transaction payload, because the CRC cannot be 0. Hence, we can detect the end of the log without even parsing the mini-transaction bytes.

            For circular log files, we can introduce a special mini-transaction 'Skip the next N bytes', encoded in sizeof(CRC)+2+log(N) bytes: CRC, record type and length, subtype and the value of the sequence bit, and variable-length encoded N. If we need to pad a block with fewer bytes than the minimum size, we would write a record to skip the minimum size.

            Pros: Minimal overhead: sizeof(CRC) bytes per mini-transaction.
            Cons: Recovery may have to parse a lot of log before determining that the end of the log was reached.

            h4. Alternative: Prepending a mini-transaction header with length and CRC
            We could encapsulate MDEV-12353 records (without the mini-transaction terminating NUL byte) in the following structure:
            * variable-length encoded integer of {{total_length << 2 | sequence_bit}}
            * CRC of the data payload and the variable-length encoded integer
            * the data payload (MDEV-12353 records); could be encrypted in their entirety

            Skipped bytes (at least 5) would be indicated by the following:
            * variable-length encoded integer of {{skipped_length << 2 | 1 << 1 | sequence_bit}}
            * CRC of the variable-length encoded integer (not including the skipped bytes)

            Pros: Recovery can determine more quickly that the end of the circular log was reached, thanks to the length, sequence bit and (nonzero) CRC being stored at the start.
            Pros: More of the log could be encrypted (at the cost of recovery and backup restoration speed)
            Cons: Slightly more overhead: sizeof(CRC)+log(length * 4) bytes. For length<32 bytes, no change of overhead.

            h3. Log writing and synchronous flushing
            For the bulk of the changes done by mini-transactions, we do not care about flushing. The file system can write log file blocks as it pleases.

            Some state changes of the database must be made durable at a specific time. Examples include user transaction {{COMMIT}}, {{XA PREPARE}}, {{XA ROLLBACK}}, and (in case the binlog is not enabled) {{XA COMMIT}}.

            Whenever we want to make a certain change durable, we must flush all log files up to the LSN of the mini-transaction commit that made the change. While doing this, we can pad each log file to the file system block size (the 'skipped bytes'), so that the file system can always write full blocks. The padding could also be helpful when trying to resurrect a corrupted redo log file.

            If redo log is physically replicated to the buffer pools of slaves (like in PolarDB), then we should first write to the redo log files and only then to the slaves, and we should assume that the writes to the files will always eventually be durable. If that assumption is broken, then all servers would have to be restarted and perform crash recovery. Because the log sequence number will be counted in bytes, we will have to replicate the number of skipped bytes. (But we could omit the skipped bytes themselves when sending over the network.)

            h3. Crash recovery
            # Validate the {{ib_logfile0}} header, including checking the size of the {{ib_logdata}} file.
            # Read the last bytes of {{ib_logfile0}} to see if it ends in a valid {{FILE_CHECKPOINT}} record. If not, read the entire {{ib_logfile0}} to find the latest {{FILE_CHECKPOINT}} record. (The last write to {{ib_logfile0}} could have been incomplete, and we may have to trim that file.)
            # If {{ib_logdata}} ends at the identified byte position (we got sequence bit mismatch or checksum mismatch), then the database was clean.
            # Else, recover the file name information based on the {{ib_logfile0}} contents and start recovery.
            # Parse the {{ib_logdata}} until the end. If any tablespace identifiers refer to unknown or inaccessible data files, abort startup unless {{innodb_force_recovery≥1}}. Ignore records for those tablespaces for which {{FILE_DELETE}} had been recorded.
            # Now that the log has been validated, start the modifications in recovery. First, trim {{ib_logfile0}} if needed, and replay {{FILE_DELETE}} and {{FILE_RENAME}} operations.
            # Shrink any files for which we parsed a {{TRIM_PAGES}} record, and extend any files according to changes to {{FSP_SIZE}}.
            # Apply the parsed log to all data files.

            h3. Considerations on checkpoints and writing {{ib_logdata}}
            After completing a checkpoint (the write of the {{FILE_CHECKPOINT}} record to {{ib_logfile0}}), we could punch a hole in {{ib_logdata}} to discard no-longer-needed log records.

            As an option, the redo log could be rebuilt on a checkpoint, by creating a logically empty set of log files, at the minimum consisting of an empty {{ib_logdata}} file and a {{ib_logfile0}} that contains {{FILE_ID}} records and a {{FILE_CHECKPOINT}} record.

            If redo log archiving is enabled, when the maximum configured size of a single redo log (which is stored in the {{ib_logfile0}} header) is reached, a new {{ib_logdata.%06u}} file will be created. The old file length must be exactly at the maximum length.

            Writes to {{ib_logfile0}} will *not* increment the LSN at all! This means that the redo log could be easily rebuilt at any LSN, and Mariabackup could write additional information to that file, removing the need for separate {{.delta}} files in incremental backups.
            marko Marko Mäkelä made changes -
            Description The InnoDB redo log format is not optimal in many respects:
            * At the start of {{ib_logfile0}}, there are two log checkpoint blocks, only 1024 bytes apart, while there exist devices with 4096-byte block size. The rest of the log file is written in a circular fashion.
            * On log checkpoint, some file name information needs to be appended to the log.
            * File names that were first changed since the latest checkpoint must be appended to the log. The bookkeeping causes some contention on {{log_sys.mutex}}- and {{fil_system.mutex}}-. Edit: The contention on {{fil_system.mutex}} was practically removed in MDEV-23855, and the contention on {{log_sys.mutex}} due to this is minimal.
            * The log file was unnecessarily split into multiple files, logically treated as one big circular file. (MDEV-20907 in MariaDB Server 10.5.0 change the default to 1 file, and later the parameter was deprecated and ignored.)
            * Log records are divided into tiny blocks of 512 bytes, with 12+4 bytes of header and footer (12+8 bytes with MDEV-12041 {{innodb_encrypt_log}} (10.4.0)).
            * We are holding a mutex while zero-filling unused parts of log blocks, encrypting log blocks, or computing checksums.
            * Mariabackup cannot copy the log without having access to the encryption keys. (It can copy data file pages without encrypting them.)

            We had some ideas to move to an append-only file and to partition the log into multiple files, but it turned out that a single fixed-size circular log file would perform best in typical scenarios.

            To address the {{fil_system.mutex}} contention whose root cause was later fixed in MDEV-23855, we were considering to split the log as follows:
            * {{ib_logfile0}} (after the 512-byte header) will be append-only, unencrypted, for records containing file names and checkpoint information. A checkpoint record will comprise an LSN and a byte offset in a separate, optionally encrypted, circular log file {{ib_logdata}}. The length of each record is explicitly tagged and the payload will be followed by CRC-32C.
            * The {{ib_logdata}} file can be append-only or circular. If it is circular, its fixed size must be an integer multiple of 512 bytes.

            One problem would have had to be solved: When would the {{ib_logfile0}} be shrunk? No storage is unlimited.

            So, we will keep the {{ib_logfile0}} and the basic format of its first 512 bytes for compatibility purposes, but other features could be improved.
            * We remove log block headers and footers. We really only need is to detect the logical end of the circular log in {{ib_logdata}}. That can be achieved by making sure that mini-transactions are terminated by a sequence number (at least one bit) and a checksum. When the circular file wraps around, the sequence number will be incremented (or the sequence bit toggled).
            * For page-aligned I/O, we allow dummy records to be written, to indicate that the next bytes (until the end of the physical block, no matter what the I/O block size is) must be ignored. (The log parser will ignore these padding records, but we do not currently write them; we will keep overwriting the last physical block until it has been completely filled like we used to do until now.)
            * Encrypt and compute checksum on {{mtr_t::m_log}} before initiating a write to the circular log file. The log can be copied and checksum validated without access to encryption keys.
            * If the log is on a memory-mapped persistent memory device, then we will make {{log_sys.buf}} point directly to the persistent memory.

            Some old InnoDB redo log parameters will be deprecated and ignored:
            * {{innodb_log_files_in_group}} (deprecated & ignored in MariaDB 10.5.1; removed by )
            * {{innodb_log_checksums}} (removed by MDEV-23397 in MariaDB 10.6.0)
            * {{innodb_log_write_ahead_size}}


            The parameter {{innodb_log_write_ahead_size}} may be repurposed to indicate the desired physical log block size. For PMEM, it might be 64 bytes (the width of a CPU cache line).

            h2. The checkpoint and file operations log file {{ib_logfile0}}
            The file name {{ib_logfile0}} and the existing format of the first 512 bytes will be retained for the purpose of upgrading and preventing downgrading. In the first 512 bytes of the file, the following information will be present:
            * InnoDB redo log format version identifier (in the format introduced by MySQL 5.7.9/MariaDB 10.2.2)
            * The size of the {{ib_logdata}} file (the maximum possible size of the file if it is append-only).
            * All encryption parameters for {{ib_logdata}}
            * CRC-32C checksum
            This file will be append-only. After the first 512 bytes, the format will be:
            * Optional: {{FILE_ID}} records (renamed from {{FILE_MODIFY}}) of all {{.ibd}} files that existed when the redo log file was created. Each record terminated by CRC-32C.
            * {{FILE_CHECKPOINT}} record with LSN and byte offset in {{ib_logdata}}, and sequence bit (in circular {{ib_logdata}} file), and CRC-32C. The byte offset will be at least 47 bits (128 TiB). Total size of a checkpoint: at least 1+8+6+4=19 bytes.
            * Optional: {{FILE_CREATE}}, {{FILE_DELETE}}, {{FILE_RENAME}} records and further {{FILE_CHECKPOINT}} records for any file operations or subsequent checkpoints since the redo log was created. Each record will be terminated by CRC-32C.

            All writes to {{ib_logfile0}} will be synchronous and durable ({{O_DSYNC}}, {{fdatasync()}} or {{O_SYNC}}, {{fsync()}}) .

            h2. The circular or append-only page-level log file {{ib_logdata}}
            The {{ib_logdata}} file will contain MDEV-12353 records except {{FILE_}} records that will be written into {{ib_logfile0}}. Each mini-transaction will be followed by a CRC-32C of all the bytes (using the value 0 for the sequence bit or number), so that Mariabackup can avoid recomputing the checksum while copying the log to a new file.

            If the {{ib_logdata}} is append-only, then we could enable log archiving by actually writing to files like {{ib_logdata.000000}}, {{ib_logdata.000001}} and so on.

            h3. Payload encoding
            The exact encoding of {{ib_logdata}} is not decided yet. We want to avoid overwriting the last log block, so we cannot have an explicit 'end of log' marker. We must associate each mini-transaction (atomic sequence of log records) with a sequence number (at the minimum, a sequence bit) and a checksum. The 4-byte CRC-32C is a good candidate, because it is already being used in data page checksums.

            We know that a CRC of nonzero bytes must be nonzero, and the mini-transaction payload cannot be zero, hence the CRC bytes can never be zero.
            Because of this, it would be beneficial to zero-initialize all the skipped bytes to increase the probability of quick detection of the end of the circular log. Compared to the existing format, we would avoid CRC computation or encryption of the skipped bytes, but the zero-filling would still have to be protected with a mutex.

            h4. Alternative: Prepending a CRC to each MDEV-12353 mini-transaction
            In the MDEV-12353 encoding, a record cannot start with the bytes {{0x00}} or {{0x01}}. Mini-transactions are currently being terminated by the byte {{0x00}}. We could store the sequence bit in the terminating byte of the mini-transaction. The checksum would exclude the terminating byte.

            Only the payload bytes would be encrypted (not record types or lengths, and not page identifiers either). In that way, records can be parsed and validated efficiently. Decryption would only have to be invoked when the log really needs to be applied on the page. The initialization vector for encryption and decryption can include the unencrypted record header bytes.

            It is best to store the CRC before the mini-transaction payload, because the CRC cannot be 0. Hence, we can detect the end of the log without even parsing the mini-transaction bytes.

            For circular log files, we can introduce a special mini-transaction 'Skip the next N bytes', encoded in sizeof(CRC)+2+log(N) bytes: CRC, record type and length, subtype and the value of the sequence bit, and variable-length encoded N. If we need to pad a block with fewer bytes than the minimum size, we would write a record to skip the minimum size.

            Pros: Minimal overhead: sizeof(CRC) bytes per mini-transaction.
            Cons: Recovery may have to parse a lot of log before determining that the end of the log was reached.

            h4. Alternative: Prepending a mini-transaction header with length and CRC
            We could encapsulate MDEV-12353 records (without the mini-transaction terminating NUL byte) in the following structure:
            * variable-length encoded integer of {{total_length << 2 | sequence_bit}}
            * CRC of the data payload and the variable-length encoded integer
            * the data payload (MDEV-12353 records); could be encrypted in their entirety

            Skipped bytes (at least 5) would be indicated by the following:
            * variable-length encoded integer of {{skipped_length << 2 | 1 << 1 | sequence_bit}}
            * CRC of the variable-length encoded integer (not including the skipped bytes)

            Pros: Recovery can determine more quickly that the end of the circular log was reached, thanks to the length, sequence bit and (nonzero) CRC being stored at the start.
            Pros: More of the log could be encrypted (at the cost of recovery and backup restoration speed)
            Cons: Slightly more overhead: sizeof(CRC)+log(length * 4) bytes. For length<32 bytes, no change of overhead.

            h3. Log writing and synchronous flushing
            For the bulk of the changes done by mini-transactions, we do not care about flushing. The file system can write log file blocks as it pleases.

            Some state changes of the database must be made durable at a specific time. Examples include user transaction {{COMMIT}}, {{XA PREPARE}}, {{XA ROLLBACK}}, and (in case the binlog is not enabled) {{XA COMMIT}}.

            Whenever we want to make a certain change durable, we must flush all log files up to the LSN of the mini-transaction commit that made the change. While doing this, we can pad each log file to the file system block size (the 'skipped bytes'), so that the file system can always write full blocks. The padding could also be helpful when trying to resurrect a corrupted redo log file.

            If redo log is physically replicated to the buffer pools of slaves (like in PolarDB), then we should first write to the redo log files and only then to the slaves, and we should assume that the writes to the files will always eventually be durable. If that assumption is broken, then all servers would have to be restarted and perform crash recovery. Because the log sequence number will be counted in bytes, we will have to replicate the number of skipped bytes. (But we could omit the skipped bytes themselves when sending over the network.)

            h3. Crash recovery
            # Validate the {{ib_logfile0}} header, including checking the size of the {{ib_logdata}} file.
            # Read the last bytes of {{ib_logfile0}} to see if it ends in a valid {{FILE_CHECKPOINT}} record. If not, read the entire {{ib_logfile0}} to find the latest {{FILE_CHECKPOINT}} record. (The last write to {{ib_logfile0}} could have been incomplete, and we may have to trim that file.)
            # If {{ib_logdata}} ends at the identified byte position (we got sequence bit mismatch or checksum mismatch), then the database was clean.
            # Else, recover the file name information based on the {{ib_logfile0}} contents and start recovery.
            # Parse the {{ib_logdata}} until the end. If any tablespace identifiers refer to unknown or inaccessible data files, abort startup unless {{innodb_force_recovery≥1}}. Ignore records for those tablespaces for which {{FILE_DELETE}} had been recorded.
            # Now that the log has been validated, start the modifications in recovery. First, trim {{ib_logfile0}} if needed, and replay {{FILE_DELETE}} and {{FILE_RENAME}} operations.
            # Shrink any files for which we parsed a {{TRIM_PAGES}} record, and extend any files according to changes to {{FSP_SIZE}}.
            # Apply the parsed log to all data files.

            h3. Considerations on checkpoints and writing {{ib_logdata}}
            After completing a checkpoint (the write of the {{FILE_CHECKPOINT}} record to {{ib_logfile0}}), we could punch a hole in {{ib_logdata}} to discard no-longer-needed log records.

            As an option, the redo log could be rebuilt on a checkpoint, by creating a logically empty set of log files, at the minimum consisting of an empty {{ib_logdata}} file and a {{ib_logfile0}} that contains {{FILE_ID}} records and a {{FILE_CHECKPOINT}} record.

            If redo log archiving is enabled, when the maximum configured size of a single redo log (which is stored in the {{ib_logfile0}} header) is reached, a new {{ib_logdata.%06u}} file will be created. The old file length must be exactly at the maximum length.

            Writes to {{ib_logfile0}} will *not* increment the LSN at all! This means that the redo log could be easily rebuilt at any LSN, and Mariabackup could write additional information to that file, removing the need for separate {{.delta}} files in incremental backups.
            The InnoDB redo log format is not optimal in many respects:
            * At the start of {{ib_logfile0}}, there are two log checkpoint blocks, only 1024 bytes apart, while there exist devices with 4096-byte block size. The rest of the log file is written in a circular fashion.
            * On log checkpoint, some file name information needs to be appended to the log.
            * File names that were first changed since the latest checkpoint must be appended to the log. The bookkeeping causes some contention on {{log_sys.mutex}}- and {{fil_system.mutex}}-. Edit: The contention on {{fil_system.mutex}} was practically removed in MDEV-23855, and the contention on {{log_sys.mutex}} due to this is minimal.
            * The log file was unnecessarily split into multiple files, logically treated as one big circular file. (MDEV-20907 in MariaDB Server 10.5.0 change the default to 1 file, and later the parameter was deprecated and ignored.)
            * Log records are divided into tiny blocks of 512 bytes, with 12+4 bytes of header and footer (12+8 bytes with MDEV-12041 {{innodb_encrypt_log}} (10.4.0)).
            * We are holding a mutex while zero-filling unused parts of log blocks, encrypting log blocks, or computing checksums.
            * Mariabackup cannot copy the log without having access to the encryption keys. (It can copy data file pages without encrypting them.)

            We had some ideas to move to an append-only file and to partition the log into multiple files, but it turned out that a single fixed-size circular log file would perform best in typical scenarios.

            To address the {{fil_system.mutex}} contention whose root cause was later fixed in MDEV-23855, we were considering to split the log as follows:
            * {{ib_logfile0}} (after the 512-byte header) will be append-only, unencrypted, for records containing file names and checkpoint information. A checkpoint record will comprise an LSN and a byte offset in a separate, optionally encrypted, circular log file {{ib_logdata}}. The length of each record is explicitly tagged and the payload will be followed by CRC-32C.
            * The {{ib_logdata}} file can be append-only or circular. If it is circular, its fixed size must be an integer multiple of 512 bytes.

            One problem would have had to be solved: When would the {{ib_logfile0}} be shrunk? No storage is unlimited.

            So, we will keep the {{ib_logfile0}} and the basic format of its first 512 bytes for compatibility purposes, but other features could be improved.
            * We remove log block headers and footers. We really only need is to detect the logical end of the circular log in {{ib_logdata}}. That can be achieved by making sure that mini-transactions are terminated by a sequence number (at least one bit) and a checksum. When the circular file wraps around, the sequence number will be incremented (or the sequence bit toggled).
            * For page-aligned I/O, we allow dummy records to be written, to indicate that the next bytes (until the end of the physical block, no matter what the I/O block size is) must be ignored. (The log parser will ignore these padding records, but we do not currently write them; we will keep overwriting the last physical block until it has been completely filled like we used to do until now.)
            * Encrypt and compute checksum on {{mtr_t::m_log}} before initiating a write to the circular log file. The log can be copied and checksum validated without access to encryption keys.
            * If the log is on a memory-mapped persistent memory device, then we will make {{log_sys.buf}} point directly to the persistent memory.

            Some old InnoDB redo log parameters were removed in MDEV-23397 (MariaDB 10.6.0). Some more parameters will removed or changed here:
            * {{innodb_log_write_ahead_size}}: Removed. On Linux and Microsoft Windows, we will detect and use the physical block size of the underlying storage. We will also remove the {{log_padded}} counter from {{INFORMATION_SCHEMA.INNODB_METRICS}}.
            * {{innodb_log_buffer_size}}: The minimum value is raised to 2MiB and the granularity increased from 1024 to 4096 bytes. This buffer will also be used during recovery. Ignored when the log is memory-mapped (on PMEM or {{/dev/shm}}).
            * {{innodb_log_file_size}}: The allocation granularity is reduced from 1MiB to 4KiB.

            Some global variables will be adjusted as well:
            * {{Innodb_os_log_fsyncs}}: Removed. This will be included in {{Innodb_data_fsyncs}}.
            * {{Innodb_os_log_pending_fsyncs}}: Removed. This was limited to at most 1 by design.
            * {{Innodb_log_pending_writes}}: Removed. This was limited to at most 1 by design.

            h2. The circular log file {{ib_logfile0}}
            The file name {{ib_logfile0}} and the existing format of the first 512 bytes will be retained for the purpose of upgrading and preventing downgrading. In the first 512 bytes of the file, the following information will be present:
            * InnoDB redo log format version identifier (in the format introduced by MySQL 5.7.9/MariaDB 10.2.2)
            * CRC-32C checksum

            After the first 512 bytes, there will be two 64-byte checkpoint blocks at the byte offsets 4096 and 8192, containing:
            * The checkpoint LSN
            * The LSN at the time the checkpoint was created, pointing to an optional sequence of {{FILE_MODIFY}} records and a {{FILE_CHECKPOINT}} record

            The circular redo log record area starts at offset 12288 and extends to the end of the file. Unless the file was created by {{mariadb-backup}}, the file size will be a multiple of 4096 bytes.

            All writes to {{ib_logfile0}} will be synchronous and durable ({{O_DSYNC}}, {{fdatasync()}} or {{O_SYNC}}, {{fsync()}} or {{pmem_persist()}}) .

            h3. Payload encoding

            The payload area will contain records in the MDEV-12353 format. Each mini-transaction will be followed by a sequence byte 0x00 or 0x01 (the value of the sequence bit), optionally (if the log is encrypted) a 8-byte nonce, and a CRC-32C of all the bytes (except the sequence byte), so that backup can avoid recomputing the checksum while copying the log to a new file.

            We want to be able to avoid overwriting the last log block, so we cannot have an explicit 'end of log' marker. We must associate each mini-transaction (atomic sequence of log records) with a sequence number (at the minimum, a sequence bit) and a checksum. The 4-byte CRC-32C is a good candidate, because it is already being used in data page checksums.

            h4. Padding

            We might want to introduce a special mini-transaction 'Skip the next N bytes', encoded in sizeof(CRC)+2+log(N) bytes: CRC, record type and length, subtype and the value of the sequence bit, and variable-length encoded N. However, for a compressed storage device, it would be helpful to not have any garbage bytes in the log file. It would be better to initialize all those N bytes.

            If we need to pad a block with fewer bytes than the minimum size, we would write a record to skip the minimum size.

            This has been implemented with arbitrary-length {{FILE_CHECKPOINT}} mini-transactions whose payload consists of NUL bytes. The parser will ignore such records. We are not currently writing such records, but instead overwriting the last incomplete log block when more log is being appended.

            h4. Mini-transaction encoding: Prepending or appending a CRC to each MDEV-12353 mini-transaction
            In the MDEV-12353 encoding, a record cannot start with the bytes {{0x00}} or {{0x01}}. Mini-transactions are currently being terminated by the byte {{0x00}}. We could store the sequence bit in the terminating byte of the mini-transaction. The checksum would exclude the terminating byte.

            Only the payload bytes would be encrypted (not record types or lengths, and not page identifiers either). In that way, records can be parsed and validated efficiently. Decryption would only have to be invoked when the log really needs to be applied on the page. The initialization vector for encryption and decryption can include the unencrypted record header bytes.

            It could be best to store the CRC before the mini-transaction payload, because the CRC of non-zero bytes cannot be 0. Hence, we can detect the end of the log without even parsing the mini-transaction bytes.

            Pros: Minimal overhead: sizeof(CRC) bytes per mini-transaction.
            Cons: Recovery may have to parse a lot of log before determining that the end of the log was reached.

            In the end, the CRC was written after the mini-transaction. The log parser can flag an inconsistency if the maximum mini-transaction size would be exceeded.

            h4. Alternative encoding (scrapped idea): Prepending a mini-transaction header with length and CRC
            We could encapsulate MDEV-12353 records (without the mini-transaction terminating NUL byte) in the following structure:
            * variable-length encoded integer of {{total_length << 2 | sequence_bit}}
            * CRC of the data payload and the variable-length encoded integer
            * the data payload (MDEV-12353 records); could be encrypted in their entirety

            Skipped bytes (at least 5) would be indicated by the following:
            * variable-length encoded integer of {{skipped_length << 2 | 1 << 1 | sequence_bit}}
            * CRC of the variable-length encoded integer (not including the skipped bytes)

            Pros: Recovery can determine more quickly that the end of the circular log was reached, thanks to the length, sequence bit and (nonzero) CRC being stored at the start.
            Pros: More of the log could be encrypted (at the cost of recovery and backup restoration speed)
            Cons: Increased storage overhead: sizeof(CRC)+log(length * 4) bytes. For length<32 bytes, no change of overhead.
            Cons: If the encryption is based on the current LSN, then both encryption and the checksum would have to be computed while holding {{log_sys.mutex}}.

            h3. Log writing and synchronous flushing
            For the bulk of the changes done by mini-transactions, we do not care about flushing. The file system can write log file blocks as it pleases.

            Some state changes of the database must be made durable at a specific time. Examples include user transaction {{COMMIT}}, {{XA PREPARE}}, {{XA ROLLBACK}}, and (in case the binlog is not enabled) {{XA COMMIT}}.

            Whenever we want to make a certain change durable, we must flush all log files up to the LSN of the mini-transaction commit that made the change. While doing this, we can pad each log file to the file system block size (the 'skipped bytes'), so that the file system can always write full blocks. The padding could also be helpful when trying to resurrect a corrupted redo log file.

            If redo log is physically replicated to the buffer pools of slaves (like in PolarDB), then we should first write to the redo log files and only then to the slaves, and we should assume that the writes to the files will always eventually be durable. If that assumption is broken, then all servers would have to be restarted and perform crash recovery. Because the log sequence number will be counted in bytes, we will have to replicate the number of skipped bytes. (But we could omit the skipped bytes themselves when sending over the network.)

            h3. Crash recovery and backup
            The previous two-stage parsing (log block validation and log record parsing) was replaced with a single stage. The separate 2-megabyte buffer {{recv_sys.buf}} is no longer needed, because the bytes of the log records will be stored contiguously, except when the log file wraps around from its end to the offset 12,288.

            When the log file is memory-mapped, we will parse records directly from {{log_sys.buf}} that contains a view of the entire log file. For parsing the mini-transaction that wraps from the end of the file to the start, the record parser will use a special pointer wrapper. When not using memory-mapping, we will read from the log file to {{log_sys.buf}} in such a way that the records of each mini-transaction will be contiguous.

            Crash-upgrade from earlier versions will not be supported.

            Starting up without {{ib_logfile0}} will no longer be supported; see
            marko Marko Mäkelä made changes -
            Description The InnoDB redo log format is not optimal in many respects:
            * At the start of {{ib_logfile0}}, there are two log checkpoint blocks, only 1024 bytes apart, while there exist devices with 4096-byte block size. The rest of the log file is written in a circular fashion.
            * On log checkpoint, some file name information needs to be appended to the log.
            * File names that were first changed since the latest checkpoint must be appended to the log. The bookkeeping causes some contention on {{log_sys.mutex}}- and {{fil_system.mutex}}-. Edit: The contention on {{fil_system.mutex}} was practically removed in MDEV-23855, and the contention on {{log_sys.mutex}} due to this is minimal.
            * The log file was unnecessarily split into multiple files, logically treated as one big circular file. (MDEV-20907 in MariaDB Server 10.5.0 change the default to 1 file, and later the parameter was deprecated and ignored.)
            * Log records are divided into tiny blocks of 512 bytes, with 12+4 bytes of header and footer (12+8 bytes with MDEV-12041 {{innodb_encrypt_log}} (10.4.0)).
            * We are holding a mutex while zero-filling unused parts of log blocks, encrypting log blocks, or computing checksums.
            * Mariabackup cannot copy the log without having access to the encryption keys. (It can copy data file pages without encrypting them.)

            We had some ideas to move to an append-only file and to partition the log into multiple files, but it turned out that a single fixed-size circular log file would perform best in typical scenarios.

            To address the {{fil_system.mutex}} contention whose root cause was later fixed in MDEV-23855, we were considering to split the log as follows:
            * {{ib_logfile0}} (after the 512-byte header) will be append-only, unencrypted, for records containing file names and checkpoint information. A checkpoint record will comprise an LSN and a byte offset in a separate, optionally encrypted, circular log file {{ib_logdata}}. The length of each record is explicitly tagged and the payload will be followed by CRC-32C.
            * The {{ib_logdata}} file can be append-only or circular. If it is circular, its fixed size must be an integer multiple of 512 bytes.

            One problem would have had to be solved: When would the {{ib_logfile0}} be shrunk? No storage is unlimited.

            So, we will keep the {{ib_logfile0}} and the basic format of its first 512 bytes for compatibility purposes, but other features could be improved.
            * We remove log block headers and footers. We really only need is to detect the logical end of the circular log in {{ib_logdata}}. That can be achieved by making sure that mini-transactions are terminated by a sequence number (at least one bit) and a checksum. When the circular file wraps around, the sequence number will be incremented (or the sequence bit toggled).
            * For page-aligned I/O, we allow dummy records to be written, to indicate that the next bytes (until the end of the physical block, no matter what the I/O block size is) must be ignored. (The log parser will ignore these padding records, but we do not currently write them; we will keep overwriting the last physical block until it has been completely filled like we used to do until now.)
            * Encrypt and compute checksum on {{mtr_t::m_log}} before initiating a write to the circular log file. The log can be copied and checksum validated without access to encryption keys.
            * If the log is on a memory-mapped persistent memory device, then we will make {{log_sys.buf}} point directly to the persistent memory.

            Some old InnoDB redo log parameters were removed in MDEV-23397 (MariaDB 10.6.0). Some more parameters will removed or changed here:
            * {{innodb_log_write_ahead_size}}: Removed. On Linux and Microsoft Windows, we will detect and use the physical block size of the underlying storage. We will also remove the {{log_padded}} counter from {{INFORMATION_SCHEMA.INNODB_METRICS}}.
            * {{innodb_log_buffer_size}}: The minimum value is raised to 2MiB and the granularity increased from 1024 to 4096 bytes. This buffer will also be used during recovery. Ignored when the log is memory-mapped (on PMEM or {{/dev/shm}}).
            * {{innodb_log_file_size}}: The allocation granularity is reduced from 1MiB to 4KiB.

            Some global variables will be adjusted as well:
            * {{Innodb_os_log_fsyncs}}: Removed. This will be included in {{Innodb_data_fsyncs}}.
            * {{Innodb_os_log_pending_fsyncs}}: Removed. This was limited to at most 1 by design.
            * {{Innodb_log_pending_writes}}: Removed. This was limited to at most 1 by design.

            h2. The circular log file {{ib_logfile0}}
            The file name {{ib_logfile0}} and the existing format of the first 512 bytes will be retained for the purpose of upgrading and preventing downgrading. In the first 512 bytes of the file, the following information will be present:
            * InnoDB redo log format version identifier (in the format introduced by MySQL 5.7.9/MariaDB 10.2.2)
            * CRC-32C checksum

            After the first 512 bytes, there will be two 64-byte checkpoint blocks at the byte offsets 4096 and 8192, containing:
            * The checkpoint LSN
            * The LSN at the time the checkpoint was created, pointing to an optional sequence of {{FILE_MODIFY}} records and a {{FILE_CHECKPOINT}} record

            The circular redo log record area starts at offset 12288 and extends to the end of the file. Unless the file was created by {{mariadb-backup}}, the file size will be a multiple of 4096 bytes.

            All writes to {{ib_logfile0}} will be synchronous and durable ({{O_DSYNC}}, {{fdatasync()}} or {{O_SYNC}}, {{fsync()}} or {{pmem_persist()}}) .

            h3. Payload encoding

            The payload area will contain records in the MDEV-12353 format. Each mini-transaction will be followed by a sequence byte 0x00 or 0x01 (the value of the sequence bit), optionally (if the log is encrypted) a 8-byte nonce, and a CRC-32C of all the bytes (except the sequence byte), so that backup can avoid recomputing the checksum while copying the log to a new file.

            We want to be able to avoid overwriting the last log block, so we cannot have an explicit 'end of log' marker. We must associate each mini-transaction (atomic sequence of log records) with a sequence number (at the minimum, a sequence bit) and a checksum. The 4-byte CRC-32C is a good candidate, because it is already being used in data page checksums.

            h4. Padding

            We might want to introduce a special mini-transaction 'Skip the next N bytes', encoded in sizeof(CRC)+2+log(N) bytes: CRC, record type and length, subtype and the value of the sequence bit, and variable-length encoded N. However, for a compressed storage device, it would be helpful to not have any garbage bytes in the log file. It would be better to initialize all those N bytes.

            If we need to pad a block with fewer bytes than the minimum size, we would write a record to skip the minimum size.

            This has been implemented with arbitrary-length {{FILE_CHECKPOINT}} mini-transactions whose payload consists of NUL bytes. The parser will ignore such records. We are not currently writing such records, but instead overwriting the last incomplete log block when more log is being appended.

            h4. Mini-transaction encoding: Prepending or appending a CRC to each MDEV-12353 mini-transaction
            In the MDEV-12353 encoding, a record cannot start with the bytes {{0x00}} or {{0x01}}. Mini-transactions are currently being terminated by the byte {{0x00}}. We could store the sequence bit in the terminating byte of the mini-transaction. The checksum would exclude the terminating byte.

            Only the payload bytes would be encrypted (not record types or lengths, and not page identifiers either). In that way, records can be parsed and validated efficiently. Decryption would only have to be invoked when the log really needs to be applied on the page. The initialization vector for encryption and decryption can include the unencrypted record header bytes.

            It could be best to store the CRC before the mini-transaction payload, because the CRC of non-zero bytes cannot be 0. Hence, we can detect the end of the log without even parsing the mini-transaction bytes.

            Pros: Minimal overhead: sizeof(CRC) bytes per mini-transaction.
            Cons: Recovery may have to parse a lot of log before determining that the end of the log was reached.

            In the end, the CRC was written after the mini-transaction. The log parser can flag an inconsistency if the maximum mini-transaction size would be exceeded.

            h4. Alternative encoding (scrapped idea): Prepending a mini-transaction header with length and CRC
            We could encapsulate MDEV-12353 records (without the mini-transaction terminating NUL byte) in the following structure:
            * variable-length encoded integer of {{total_length << 2 | sequence_bit}}
            * CRC of the data payload and the variable-length encoded integer
            * the data payload (MDEV-12353 records); could be encrypted in their entirety

            Skipped bytes (at least 5) would be indicated by the following:
            * variable-length encoded integer of {{skipped_length << 2 | 1 << 1 | sequence_bit}}
            * CRC of the variable-length encoded integer (not including the skipped bytes)

            Pros: Recovery can determine more quickly that the end of the circular log was reached, thanks to the length, sequence bit and (nonzero) CRC being stored at the start.
            Pros: More of the log could be encrypted (at the cost of recovery and backup restoration speed)
            Cons: Increased storage overhead: sizeof(CRC)+log(length * 4) bytes. For length<32 bytes, no change of overhead.
            Cons: If the encryption is based on the current LSN, then both encryption and the checksum would have to be computed while holding {{log_sys.mutex}}.

            h3. Log writing and synchronous flushing
            For the bulk of the changes done by mini-transactions, we do not care about flushing. The file system can write log file blocks as it pleases.

            Some state changes of the database must be made durable at a specific time. Examples include user transaction {{COMMIT}}, {{XA PREPARE}}, {{XA ROLLBACK}}, and (in case the binlog is not enabled) {{XA COMMIT}}.

            Whenever we want to make a certain change durable, we must flush all log files up to the LSN of the mini-transaction commit that made the change. While doing this, we can pad each log file to the file system block size (the 'skipped bytes'), so that the file system can always write full blocks. The padding could also be helpful when trying to resurrect a corrupted redo log file.

            If redo log is physically replicated to the buffer pools of slaves (like in PolarDB), then we should first write to the redo log files and only then to the slaves, and we should assume that the writes to the files will always eventually be durable. If that assumption is broken, then all servers would have to be restarted and perform crash recovery. Because the log sequence number will be counted in bytes, we will have to replicate the number of skipped bytes. (But we could omit the skipped bytes themselves when sending over the network.)

            h3. Crash recovery and backup
            The previous two-stage parsing (log block validation and log record parsing) was replaced with a single stage. The separate 2-megabyte buffer {{recv_sys.buf}} is no longer needed, because the bytes of the log records will be stored contiguously, except when the log file wraps around from its end to the offset 12,288.

            When the log file is memory-mapped, we will parse records directly from {{log_sys.buf}} that contains a view of the entire log file. For parsing the mini-transaction that wraps from the end of the file to the start, the record parser will use a special pointer wrapper. When not using memory-mapping, we will read from the log file to {{log_sys.buf}} in such a way that the records of each mini-transaction will be contiguous.

            Crash-upgrade from earlier versions will not be supported.

            Starting up without {{ib_logfile0}} will no longer be supported; see
            The InnoDB redo log format is not optimal in many respects:
            * At the start of {{ib_logfile0}}, there are two log checkpoint blocks, only 1024 bytes apart, while there exist devices with 4096-byte block size. The rest of the log file is written in a circular fashion.
            * On log checkpoint, some file name information needs to be appended to the log.
            * File names that were first changed since the latest checkpoint must be appended to the log. The bookkeeping causes some contention on {{log_sys.mutex}}- and {{fil_system.mutex}}-. Edit: The contention on {{fil_system.mutex}} was practically removed in MDEV-23855, and the contention on {{log_sys.mutex}} due to this is minimal.
            * The log file was unnecessarily split into multiple files, logically treated as one big circular file. (MDEV-20907 in MariaDB Server 10.5.0 change the default to 1 file, and later the parameter was deprecated and ignored.)
            * Log records are divided into tiny blocks of 512 bytes, with 12+4 bytes of header and footer (12+8 bytes with MDEV-12041 {{innodb_encrypt_log}} (10.4.0)).
            * We are holding a mutex while zero-filling unused parts of log blocks, encrypting log blocks, or computing checksums.
            * Mariabackup cannot copy the log without having access to the encryption keys. (It can copy data file pages without encrypting them.)

            We had some ideas to move to an append-only file and to partition the log into multiple files, but it turned out that a single fixed-size circular log file would perform best in typical scenarios.

            To address the {{fil_system.mutex}} contention whose root cause was later fixed in MDEV-23855, we were considering to split the log as follows:
            * {{ib_logfile0}} (after the 512-byte header) will be append-only, unencrypted, for records containing file names and checkpoint information. A checkpoint record will comprise an LSN and a byte offset in a separate, optionally encrypted, circular log file {{ib_logdata}}. The length of each record is explicitly tagged and the payload will be followed by CRC-32C.
            * The {{ib_logdata}} file can be append-only or circular. If it is circular, its fixed size must be an integer multiple of 512 bytes.

            One problem would have had to be solved: When would the {{ib_logfile0}} be shrunk? No storage is unlimited.

            So, we will keep the {{ib_logfile0}} and the basic format of its first 512 bytes for compatibility purposes, but other features could be improved.
            * We remove log block headers and footers. We really only need is to detect the logical end of the circular log in {{ib_logdata}}. That can be achieved by making sure that mini-transactions are terminated by a sequence number (at least one bit) and a checksum. When the circular file wraps around, the sequence number will be incremented (or the sequence bit toggled).
            * For page-aligned I/O, we allow dummy records to be written, to indicate that the next bytes (until the end of the physical block, no matter what the I/O block size is) must be ignored. (The log parser will ignore these padding records, but we do not currently write them; we will keep overwriting the last physical block until it has been completely filled like we used to do until now.)
            * Encrypt and compute checksum on {{mtr_t::m_log}} before initiating a write to the circular log file. The log can be copied and checksum validated without access to encryption keys.
            * If the log is on a memory-mapped persistent memory device, then we will make {{log_sys.buf}} point directly to the persistent memory.

            Some old InnoDB redo log parameters were removed in MDEV-23397 (MariaDB 10.6.0). Some more parameters will removed or changed here:
            * {{innodb_log_write_ahead_size}}: Removed. On Linux and Microsoft Windows, we will detect and use the physical block size of the underlying storage. We will also remove the {{log_padded}} counter from {{INFORMATION_SCHEMA.INNODB_METRICS}}.
            * {{innodb_log_buffer_size}}: The minimum value is raised to 2MiB and the granularity increased from 1024 to 4096 bytes. This buffer will also be used during recovery. Ignored when the log is memory-mapped (on PMEM or {{/dev/shm}}).
            * {{innodb_log_file_size}}: The allocation granularity is reduced from 1MiB to 4KiB.

            Some global variables will be adjusted as well:
            * {{Innodb_os_log_fsyncs}}: Removed. This will be included in {{Innodb_data_fsyncs}}.
            * {{Innodb_os_log_pending_fsyncs}}: Removed. This was limited to at most 1 by design.
            * {{Innodb_log_pending_writes}}: Removed. This was limited to at most 1 by design.

            h2. The circular log file {{ib_logfile0}}
            The file name {{ib_logfile0}} and the existing format of the first 512 bytes will be retained for the purpose of upgrading and preventing downgrading. In the first 512 bytes of the file, the following information will be present:
            * InnoDB redo log format version identifier (in the format introduced by MySQL 5.7.9/MariaDB 10.2.2)
            * CRC-32C checksum

            After the first 512 bytes, there will be two 64-byte checkpoint blocks at the byte offsets 4096 and 8192, containing:
            * The checkpoint LSN
            * The LSN at the time the checkpoint was created, pointing to an optional sequence of {{FILE_MODIFY}} records and a {{FILE_CHECKPOINT}} record

            The circular redo log record area starts at offset 12288 and extends to the end of the file. Unless the file was created by {{mariadb-backup}}, the file size will be a multiple of 4096 bytes.

            All writes to {{ib_logfile0}} will be synchronous and durable ({{O_DSYNC}}, {{fdatasync()}} or {{O_SYNC}}, {{fsync()}} or {{pmem_persist()}}) .

            h3. Payload encoding

            The payload area will contain records in the MDEV-12353 format. Each mini-transaction will be followed by a sequence byte 0x00 or 0x01 (the value of the sequence bit), optionally (if the log is encrypted) a 8-byte nonce, and a CRC-32C of all the bytes (except the sequence byte), so that backup can avoid recomputing the checksum while copying the log to a new file.

            We want to be able to avoid overwriting the last log block, so we cannot have an explicit 'end of log' marker. We must associate each mini-transaction (atomic sequence of log records) with a sequence number (at the minimum, a sequence bit) and a checksum. The 4-byte CRC-32C is a good candidate, because it is already being used in data page checksums.

            h4. Padding

            We might want to introduce a special mini-transaction 'Skip the next N bytes', encoded in sizeof(CRC)+2+log(N) bytes: CRC, record type and length, subtype and the value of the sequence bit, and variable-length encoded N. However, for a compressed storage device, it would be helpful to not have any garbage bytes in the log file. It would be better to initialize all those N bytes.

            If we need to pad a block with fewer bytes than the minimum size, we would write a record to skip the minimum size.

            This has been implemented with arbitrary-length {{FILE_CHECKPOINT}} mini-transactions whose payload consists of NUL bytes. The parser will ignore such records. We are not currently writing such records, but instead overwriting the last incomplete log block when more log is being appended.

            h4. Mini-transaction encoding: Prepending or appending a CRC to each MDEV-12353 mini-transaction
            In the MDEV-12353 encoding, a record cannot start with the bytes {{0x00}} or {{0x01}}. Mini-transactions are currently being terminated by the byte {{0x00}}. We could store the sequence bit in the terminating byte of the mini-transaction. The checksum would exclude the terminating byte.

            Only the payload bytes would be encrypted (not record types or lengths, and not page identifiers either). In that way, records can be parsed and validated efficiently. Decryption would only have to be invoked when the log really needs to be applied on the page. The initialization vector for encryption and decryption can include the unencrypted record header bytes.

            It could be best to store the CRC before the mini-transaction payload, because the CRC of non-zero bytes cannot be 0. Hence, we can detect the end of the log without even parsing the mini-transaction bytes.

            Pros: Minimal overhead: sizeof(CRC) bytes per mini-transaction.
            Cons: Recovery may have to parse a lot of log before determining that the end of the log was reached.

            In the end, the CRC was written after the mini-transaction. The log parser can flag an inconsistency if the maximum mini-transaction size would be exceeded.

            h4. Alternative encoding (scrapped idea): Prepending a mini-transaction header with length and CRC
            We could encapsulate MDEV-12353 records (without the mini-transaction terminating NUL byte) in the following structure:
            * variable-length encoded integer of {{total_length << 2 | sequence_bit}}
            * CRC of the data payload and the variable-length encoded integer
            * the data payload (MDEV-12353 records); could be encrypted in their entirety

            Skipped bytes (at least 5) would be indicated by the following:
            * variable-length encoded integer of {{skipped_length << 2 | 1 << 1 | sequence_bit}}
            * CRC of the variable-length encoded integer (not including the skipped bytes)

            Pros: Recovery can determine more quickly that the end of the circular log was reached, thanks to the length, sequence bit and (nonzero) CRC being stored at the start.
            Pros: More of the log could be encrypted (at the cost of recovery and backup restoration speed)
            Cons: Increased storage overhead: sizeof(CRC)+log(length * 4) bytes. For length<32 bytes, no change of overhead.
            Cons: If the encryption is based on the current LSN, then both encryption and the checksum would have to be computed while holding {{log_sys.mutex}}.

            h3. Log writing and synchronous flushing
            For the bulk of the changes done by mini-transactions, we do not care about flushing. The file system can write log file blocks as it pleases.

            Some state changes of the database must be made durable at a specific time. Examples include user transaction {{COMMIT}}, {{XA PREPARE}}, {{XA ROLLBACK}}, and (in case the binlog is not enabled) {{XA COMMIT}}.

            Whenever we want to make a certain change durable, we must flush all log files up to the LSN of the mini-transaction commit that made the change. While doing this, we can pad each log file to the file system block size (the 'skipped bytes'), so that the file system can always write full blocks. The padding could also be helpful when trying to resurrect a corrupted redo log file.

            If redo log is physically replicated to the buffer pools of slaves (like in PolarDB), then we should first write to the redo log files and only then to the slaves, and we should assume that the writes to the files will always eventually be durable. If that assumption is broken, then all servers would have to be restarted and perform crash recovery. Because the log sequence number will be counted in bytes, we will have to replicate the number of skipped bytes. (But we could omit the skipped bytes themselves when sending over the network.)

            h3. Crash recovery and backup
            The previous two-stage parsing (log block validation and log record parsing) was replaced with a single stage. The separate 2-megabyte buffer {{recv_sys.buf}} is no longer needed, because the bytes of the log records will be stored contiguously, except when the log file wraps around from its end to the offset 12,288.

            When the log file is memory-mapped, we will parse records directly from {{log_sys.buf}} that contains a view of the entire log file. For parsing the mini-transaction that wraps from the end of the file to the start, the record parser will use a special pointer wrapper. When not using memory-mapping, we will read from the log file to {{log_sys.buf}} in such a way that the records of each mini-transaction will be contiguous.

            Crash-upgrade from earlier versions will not be supported. Before upgrading, the old server must have been shut down, or {{mariadb-backup --prepare}} must have been executed using an appropriate older version of the backup tool.

            Starting up without {{ib_logfile0}} will no longer be supported; see also MDEV-27199.
            axel Axel Schwenke made changes -
            axel Axel Schwenke made changes -
            marko Marko Mäkelä made changes -
            mleich Matthias Leich made changes -
            Status In Testing [ 10301 ] Stalled [ 10000 ]
            mleich Matthias Leich made changes -
            Assignee Matthias Leich [ mleich ] Marko Mäkelä [ marko ]
            serg Sergei Golubchik made changes -
            serg Sergei Golubchik made changes -
            Priority Critical [ 2 ] Blocker [ 1 ]
            marko Marko Mäkelä made changes -
            issue.field.resolutiondate 2022-01-21 15:06:50.0 2022-01-21 15:06:50.284
            marko Marko Mäkelä made changes -
            Fix Version/s 10.8.1 [ 26815 ]
            Fix Version/s 10.8 [ 26121 ]
            Resolution Fixed [ 1 ]
            Status Stalled [ 10000 ] Closed [ 6 ]
            elenst Elena Stepanova made changes -
            marko Marko Mäkelä made changes -
            marko Marko Mäkelä made changes -
            marko Marko Mäkelä made changes -
            danblack Daniel Black made changes -
            marko Marko Mäkelä made changes -
            marko Marko Mäkelä made changes -
            marko Marko Mäkelä made changes -
            marko Marko Mäkelä made changes -
            marko Marko Mäkelä made changes -
            marko Marko Mäkelä made changes -
            marko Marko Mäkelä made changes -
            marko Marko Mäkelä made changes -
            marko Marko Mäkelä made changes -
            marko Marko Mäkelä made changes -
            marko Marko Mäkelä made changes -
            Description The InnoDB redo log format is not optimal in many respects:
            * At the start of {{ib_logfile0}}, there are two log checkpoint blocks, only 1024 bytes apart, while there exist devices with 4096-byte block size. The rest of the log file is written in a circular fashion.
            * On log checkpoint, some file name information needs to be appended to the log.
            * File names that were first changed since the latest checkpoint must be appended to the log. The bookkeeping causes some contention on {{log_sys.mutex}}- and {{fil_system.mutex}}-. Edit: The contention on {{fil_system.mutex}} was practically removed in MDEV-23855, and the contention on {{log_sys.mutex}} due to this is minimal.
            * The log file was unnecessarily split into multiple files, logically treated as one big circular file. (MDEV-20907 in MariaDB Server 10.5.0 change the default to 1 file, and later the parameter was deprecated and ignored.)
            * Log records are divided into tiny blocks of 512 bytes, with 12+4 bytes of header and footer (12+8 bytes with MDEV-12041 {{innodb_encrypt_log}} (10.4.0)).
            * We are holding a mutex while zero-filling unused parts of log blocks, encrypting log blocks, or computing checksums.
            * Mariabackup cannot copy the log without having access to the encryption keys. (It can copy data file pages without encrypting them.)

            We had some ideas to move to an append-only file and to partition the log into multiple files, but it turned out that a single fixed-size circular log file would perform best in typical scenarios.

            To address the {{fil_system.mutex}} contention whose root cause was later fixed in MDEV-23855, we were considering to split the log as follows:
            * {{ib_logfile0}} (after the 512-byte header) will be append-only, unencrypted, for records containing file names and checkpoint information. A checkpoint record will comprise an LSN and a byte offset in a separate, optionally encrypted, circular log file {{ib_logdata}}. The length of each record is explicitly tagged and the payload will be followed by CRC-32C.
            * The {{ib_logdata}} file can be append-only or circular. If it is circular, its fixed size must be an integer multiple of 512 bytes.

            One problem would have had to be solved: When would the {{ib_logfile0}} be shrunk? No storage is unlimited.

            So, we will keep the {{ib_logfile0}} and the basic format of its first 512 bytes for compatibility purposes, but other features could be improved.
            * We remove log block headers and footers. We really only need is to detect the logical end of the circular log in {{ib_logdata}}. That can be achieved by making sure that mini-transactions are terminated by a sequence number (at least one bit) and a checksum. When the circular file wraps around, the sequence number will be incremented (or the sequence bit toggled).
            * For page-aligned I/O, we allow dummy records to be written, to indicate that the next bytes (until the end of the physical block, no matter what the I/O block size is) must be ignored. (The log parser will ignore these padding records, but we do not currently write them; we will keep overwriting the last physical block until it has been completely filled like we used to do until now.)
            * Encrypt and compute checksum on {{mtr_t::m_log}} before initiating a write to the circular log file. The log can be copied and checksum validated without access to encryption keys.
            * If the log is on a memory-mapped persistent memory device, then we will make {{log_sys.buf}} point directly to the persistent memory.

            Some old InnoDB redo log parameters were removed in MDEV-23397 (MariaDB 10.6.0). Some more parameters will removed or changed here:
            * {{innodb_log_write_ahead_size}}: Removed. On Linux and Microsoft Windows, we will detect and use the physical block size of the underlying storage. We will also remove the {{log_padded}} counter from {{INFORMATION_SCHEMA.INNODB_METRICS}}.
            * {{innodb_log_buffer_size}}: The minimum value is raised to 2MiB and the granularity increased from 1024 to 4096 bytes. This buffer will also be used during recovery. Ignored when the log is memory-mapped (on PMEM or {{/dev/shm}}).
            * {{innodb_log_file_size}}: The allocation granularity is reduced from 1MiB to 4KiB.

            Some global variables will be adjusted as well:
            * {{Innodb_os_log_fsyncs}}: Removed. This will be included in {{Innodb_data_fsyncs}}.
            * {{Innodb_os_log_pending_fsyncs}}: Removed. This was limited to at most 1 by design.
            * {{Innodb_log_pending_writes}}: Removed. This was limited to at most 1 by design.

            h2. The circular log file {{ib_logfile0}}
            The file name {{ib_logfile0}} and the existing format of the first 512 bytes will be retained for the purpose of upgrading and preventing downgrading. In the first 512 bytes of the file, the following information will be present:
            * InnoDB redo log format version identifier (in the format introduced by MySQL 5.7.9/MariaDB 10.2.2)
            * CRC-32C checksum

            After the first 512 bytes, there will be two 64-byte checkpoint blocks at the byte offsets 4096 and 8192, containing:
            * The checkpoint LSN
            * The LSN at the time the checkpoint was created, pointing to an optional sequence of {{FILE_MODIFY}} records and a {{FILE_CHECKPOINT}} record

            The circular redo log record area starts at offset 12288 and extends to the end of the file. Unless the file was created by {{mariadb-backup}}, the file size will be a multiple of 4096 bytes.

            All writes to {{ib_logfile0}} will be synchronous and durable ({{O_DSYNC}}, {{fdatasync()}} or {{O_SYNC}}, {{fsync()}} or {{pmem_persist()}}) .

            h3. Payload encoding

            The payload area will contain records in the MDEV-12353 format. Each mini-transaction will be followed by a sequence byte 0x00 or 0x01 (the value of the sequence bit), optionally (if the log is encrypted) a 8-byte nonce, and a CRC-32C of all the bytes (except the sequence byte), so that backup can avoid recomputing the checksum while copying the log to a new file.

            We want to be able to avoid overwriting the last log block, so we cannot have an explicit 'end of log' marker. We must associate each mini-transaction (atomic sequence of log records) with a sequence number (at the minimum, a sequence bit) and a checksum. The 4-byte CRC-32C is a good candidate, because it is already being used in data page checksums.

            h4. Padding

            We might want to introduce a special mini-transaction 'Skip the next N bytes', encoded in sizeof(CRC)+2+log(N) bytes: CRC, record type and length, subtype and the value of the sequence bit, and variable-length encoded N. However, for a compressed storage device, it would be helpful to not have any garbage bytes in the log file. It would be better to initialize all those N bytes.

            If we need to pad a block with fewer bytes than the minimum size, we would write a record to skip the minimum size.

            This has been implemented with arbitrary-length {{FILE_CHECKPOINT}} mini-transactions whose payload consists of NUL bytes. The parser will ignore such records. We are not currently writing such records, but instead overwriting the last incomplete log block when more log is being appended.

            h4. Mini-transaction encoding: Prepending or appending a CRC to each MDEV-12353 mini-transaction
            In the MDEV-12353 encoding, a record cannot start with the bytes {{0x00}} or {{0x01}}. Mini-transactions are currently being terminated by the byte {{0x00}}. We could store the sequence bit in the terminating byte of the mini-transaction. The checksum would exclude the terminating byte.

            Only the payload bytes would be encrypted (not record types or lengths, and not page identifiers either). In that way, records can be parsed and validated efficiently. Decryption would only have to be invoked when the log really needs to be applied on the page. The initialization vector for encryption and decryption can include the unencrypted record header bytes.

            It could be best to store the CRC before the mini-transaction payload, because the CRC of non-zero bytes cannot be 0. Hence, we can detect the end of the log without even parsing the mini-transaction bytes.

            Pros: Minimal overhead: sizeof(CRC) bytes per mini-transaction.
            Cons: Recovery may have to parse a lot of log before determining that the end of the log was reached.

            In the end, the CRC was written after the mini-transaction. The log parser can flag an inconsistency if the maximum mini-transaction size would be exceeded.

            h4. Alternative encoding (scrapped idea): Prepending a mini-transaction header with length and CRC
            We could encapsulate MDEV-12353 records (without the mini-transaction terminating NUL byte) in the following structure:
            * variable-length encoded integer of {{total_length << 2 | sequence_bit}}
            * CRC of the data payload and the variable-length encoded integer
            * the data payload (MDEV-12353 records); could be encrypted in their entirety

            Skipped bytes (at least 5) would be indicated by the following:
            * variable-length encoded integer of {{skipped_length << 2 | 1 << 1 | sequence_bit}}
            * CRC of the variable-length encoded integer (not including the skipped bytes)

            Pros: Recovery can determine more quickly that the end of the circular log was reached, thanks to the length, sequence bit and (nonzero) CRC being stored at the start.
            Pros: More of the log could be encrypted (at the cost of recovery and backup restoration speed)
            Cons: Increased storage overhead: sizeof(CRC)+log(length * 4) bytes. For length<32 bytes, no change of overhead.
            Cons: If the encryption is based on the current LSN, then both encryption and the checksum would have to be computed while holding {{log_sys.mutex}}.

            h3. Log writing and synchronous flushing
            For the bulk of the changes done by mini-transactions, we do not care about flushing. The file system can write log file blocks as it pleases.

            Some state changes of the database must be made durable at a specific time. Examples include user transaction {{COMMIT}}, {{XA PREPARE}}, {{XA ROLLBACK}}, and (in case the binlog is not enabled) {{XA COMMIT}}.

            Whenever we want to make a certain change durable, we must flush all log files up to the LSN of the mini-transaction commit that made the change. While doing this, we can pad each log file to the file system block size (the 'skipped bytes'), so that the file system can always write full blocks. The padding could also be helpful when trying to resurrect a corrupted redo log file.

            If redo log is physically replicated to the buffer pools of slaves (like in PolarDB), then we should first write to the redo log files and only then to the slaves, and we should assume that the writes to the files will always eventually be durable. If that assumption is broken, then all servers would have to be restarted and perform crash recovery. Because the log sequence number will be counted in bytes, we will have to replicate the number of skipped bytes. (But we could omit the skipped bytes themselves when sending over the network.)

            h3. Crash recovery and backup
            The previous two-stage parsing (log block validation and log record parsing) was replaced with a single stage. The separate 2-megabyte buffer {{recv_sys.buf}} is no longer needed, because the bytes of the log records will be stored contiguously, except when the log file wraps around from its end to the offset 12,288.

            When the log file is memory-mapped, we will parse records directly from {{log_sys.buf}} that contains a view of the entire log file. For parsing the mini-transaction that wraps from the end of the file to the start, the record parser will use a special pointer wrapper. When not using memory-mapping, we will read from the log file to {{log_sys.buf}} in such a way that the records of each mini-transaction will be contiguous.

            Crash-upgrade from earlier versions will not be supported. Before upgrading, the old server must have been shut down, or {{mariadb-backup --prepare}} must have been executed using an appropriate older version of the backup tool.

            Starting up without {{ib_logfile0}} will no longer be supported; see also MDEV-27199.
            The InnoDB redo log format is not optimal in many respects:
            * At the start of {{ib_logfile0}}, there are two log checkpoint blocks, only 1024 bytes apart, while there exist devices with 4096-byte block size. The rest of the log file is written in a circular fashion.
            * On log checkpoint, some file name information needs to be appended to the log.
            * File names that were first changed since the latest checkpoint must be appended to the log. The bookkeeping causes some contention on {{log_sys.mutex}} and {{fil_system.mutex}}. Edit: The contention on {{fil_system.mutex}} was practically removed in MDEV-23855, and the contention on {{log_sys.mutex}} due to this is minimal.
            * The log file was unnecessarily split into multiple files, logically treated as one big circular file. (MDEV-20907 in MariaDB Server 10.5.0 change the default to 1 file, and later the parameter was deprecated and ignored.)
            * Log records are divided into tiny blocks of 512 bytes, with 12+4 bytes of header and footer (12+8 bytes with MDEV-12041 {{innodb_encrypt_log}} (10.4.0)).
            * We are holding a mutex while zero-filling unused parts of log blocks, encrypting log blocks, or computing checksums.
            * We were holding an exclusive latch while copying log blocks; this was fixed in MDEV-27774.
            * Mariabackup cannot copy the log without having access to the encryption keys. (It can copy data file pages without encrypting them.)

            We had some ideas to move to an append-only file and to partition the log into multiple files, but it turned out that a single fixed-size circular log file would perform best in typical scenarios.

            To address the {{fil_system.mutex}} contention whose root cause was later fixed in MDEV-23855, we were considering to split the log as follows:
            * {{ib_logfile0}} (after the 512-byte header) will be append-only, unencrypted, for records containing file names and checkpoint information. A checkpoint record will comprise an LSN and a byte offset in a separate, optionally encrypted, circular log file {{ib_logdata}}. The length of each record is explicitly tagged and the payload will be followed by CRC-32C.
            * The {{ib_logdata}} file can be append-only or circular. If it is circular, its fixed size must be an integer multiple of 512 bytes.

            One problem would have had to be solved: When would the {{ib_logfile0}} be shrunk? No storage is unlimited.

            We will retain the {{ib_logfile0}} and the basic format of its first 512 bytes for compatibility purposes, but other features could be improved.
            * We remove log block headers and footers. We really only need is to detect the logical end of the circular log. That can be achieved by making sure that mini-transactions are terminated by a sequence number (at least one bit) and a checksum. When the circular file wraps around, the sequence number will be incremented (or the sequence bit toggled).
            * For page-aligned I/O, we allow dummy records to be written, to indicate that the next bytes (until the end of the physical block, no matter what the I/O block size is) must be ignored. (The log parser will ignore these padding records, but we do not currently write them; we will keep overwriting the last physical block until it has been completely filled like we used to do until now.)
            * Encrypt and compute checksum on {{mtr_t::m_log}} before initiating a write to the circular log file. The log can be copied and checksum validated without access to encryption keys.
            * If the log is on a memory-mapped persistent memory device, then we will make {{log_sys.buf}} point directly to the persistent memory.

            Some old InnoDB redo log parameters were removed in MDEV-23397 (MariaDB 10.6.0). Some more parameters will removed or changed here:
            * {{innodb_log_write_ahead_size}}: Removed. On Linux and Microsoft Windows, we will detect and use the physical block size of the underlying storage. We will also remove the {{log_padded}} counter from {{INFORMATION_SCHEMA.INNODB_METRICS}}.
            * {{innodb_log_file_buffering}}: Added (MDEV-28766). This controls the use of {{O_DIRECT}} on the {{ib_logfile0}} when the physical block size can be determined
            * {{innodb_log_buffer_size}}: The minimum value is raised to 2MiB and the granularity increased from 1024 to 4096 bytes. This buffer will also be used during recovery. Ignored when the log is memory-mapped (on PMEM or {{/dev/shm}}).
            * {{innodb_log_file_size}}: The allocation granularity is reduced from 1MiB to 4KiB.

            Some global variables will be adjusted as well:
            * {{Innodb_os_log_fsyncs}}: Removed. This will be included in {{Innodb_data_fsyncs}}.
            * {{Innodb_os_log_pending_fsyncs}}: Removed. This was limited to at most 1 by design.
            * {{Innodb_log_pending_writes}}: Removed. This was limited to at most 1 by design.

            h2. The circular log file {{ib_logfile0}}
            The file name {{ib_logfile0}} and the existing format of the first 512 bytes will be retained for the purpose of upgrading and preventing downgrading. In the first 512 bytes of the file, the following information will be present:
            * InnoDB redo log format version identifier (in the format introduced by MySQL 5.7.9/MariaDB 10.2.2)
            * CRC-32C checksum

            After the first 512 bytes, there will be two 64-byte checkpoint blocks at the byte offsets 4096 and 8192, containing:
            * The checkpoint LSN
            * The LSN at the time the checkpoint was created, pointing to an optional sequence of {{FILE_MODIFY}} records and a {{FILE_CHECKPOINT}} record

            The circular redo log record area starts at offset 12288 and extends to the end of the file. Unless the file was created by {{mariadb-backup}}, the file size will be a multiple of 4096 bytes.

            All writes to {{ib_logfile0}} will be synchronous and durable ({{O_DSYNC}}, {{fdatasync()}} or {{O_SYNC}}, {{fsync()}} or {{pmem_persist()}}).

            h3. Payload encoding

            The payload area will contain records in the MDEV-12353 format. Each mini-transaction will be followed by a sequence byte 0x00 or 0x01 (the value of the sequence bit), optionally (if the log is encrypted) a 8-byte nonce, and a CRC-32C of all the bytes (except the sequence byte), so that backup can avoid recomputing the checksum while copying the log to a new file.

            We want to be able to avoid overwriting the last log block, so we cannot have an explicit 'end of log' marker. We must associate each mini-transaction (atomic sequence of log records) with a sequence number (at the minimum, a sequence bit) and a checksum. The 4-byte CRC-32C is a good candidate, because it is already being used in data page checksums.

            h4. Padding

            We might want to introduce a special mini-transaction 'Skip the next N bytes', encoded in sizeof(CRC)+2+log(N) bytes: CRC, record type and length, subtype and the value of the sequence bit, and variable-length encoded N. However, for a compressed storage device, it would be helpful to not have any garbage bytes in the log file. It would be better to initialize all those N bytes.

            If we need to pad a block with fewer bytes than the minimum size, we would write a record to skip the minimum size.

            This has been implemented with arbitrary-length {{FILE_CHECKPOINT}} mini-transactions whose payload consists of NUL bytes. The parser will ignore such records. We are not currently writing such records, but instead overwriting the last incomplete log block when more log is being appended, just like InnoDB always did.

            h4. Mini-transaction encoding: Prepending or appending a CRC to each MDEV-12353 mini-transaction
            In the MDEV-12353 encoding, a record cannot start with the bytes {{0x00}} or {{0x01}}. Mini-transactions are currently being terminated by the byte {{0x00}}. We could store the sequence bit in the terminating byte of the mini-transaction. The checksum would exclude the terminating byte.

            Only the payload bytes would be encrypted (not record types or lengths, and not page identifiers either). In that way, records can be parsed and validated efficiently. Decryption would only have to be invoked when the log really needs to be applied on the page. The initialization vector for encryption and decryption can include the unencrypted record header bytes.

            It could be best to store the CRC before the mini-transaction payload, because the CRC of non-zero bytes cannot be 0. Hence, we can detect the end of the log without even parsing the mini-transaction bytes.

            Pros: Minimal overhead: sizeof(CRC) bytes per mini-transaction.
            Cons: Recovery may have to parse a lot of log before determining that the end of the log was reached.

            In the end, the CRC was written after the mini-transaction. The log parser can flag an inconsistency if the maximum mini-transaction size would be exceeded.

            h4. Alternative encoding (scrapped idea): Prepending a mini-transaction header with length and CRC
            We could encapsulate MDEV-12353 records (without the mini-transaction terminating NUL byte) in the following structure:
            * variable-length encoded integer of {{total_length << 2 | sequence_bit}}
            * CRC of the data payload and the variable-length encoded integer
            * the data payload (MDEV-12353 records); could be encrypted in their entirety

            Skipped bytes (at least 5) would be indicated by the following:
            * variable-length encoded integer of {{skipped_length << 2 | 1 << 1 | sequence_bit}}
            * CRC of the variable-length encoded integer (not including the skipped bytes)

            Pros: Recovery can determine more quickly that the end of the circular log was reached, thanks to the length, sequence bit and (nonzero) CRC being stored at the start.
            Pros: More of the log could be encrypted (at the cost of recovery and backup restoration speed)
            Cons: Increased storage overhead: sizeof(CRC)+log(length * 4) bytes. For length<32 bytes, no change of overhead.
            Cons: If the encryption is based on the current LSN, then both encryption and the checksum would have to be computed while holding {{log_sys.mutex}}.

            h3. Log writing and synchronous flushing
            For the bulk of the changes done by mini-transactions, we do not care about flushing. The file system can write log file blocks as it pleases.

            Some state changes of the database must be made durable at a specific time. Examples include user transaction {{COMMIT}}, {{XA PREPARE}}, {{XA ROLLBACK}}, and (in case the binlog is not enabled) {{XA COMMIT}}.

            Whenever we want to make a certain change durable, we must flush all log files up to the LSN of the mini-transaction commit that made the change.

            If redo log is physically replicated to the buffer pools of physical replicas (like in Amazon Aurora or Alibaba PolarDB), then we should first write to the local log and only then to the replicas, and we should assume that the writes to the files will always eventually be durable. If that assumption is broken, then all servers would have to be restarted and perform crash recovery.

            h3. Crash recovery and backup
            The previous two-stage parsing (log block validation and log record parsing) was replaced with a single stage. The separate 2-megabyte buffer {{recv_sys.buf}} is no longer needed, because the bytes of the log records will be stored contiguously, except when the log file wraps around from its end to the offset 12,288.

            When the log file is memory-mapped, we will parse records directly from {{log_sys.buf}} that contains a view of the entire log file. For parsing the mini-transaction that wraps from the end of the file to the start, the record parser will use a special pointer wrapper. When not using memory-mapping, we will read from the log file to {{log_sys.buf}} in such a way that the records of each mini-transaction will be contiguous.

            Crash-upgrade from earlier versions will not be supported. Before upgrading, the old server must have been shut down, or {{mariadb-backup --prepare}} must have been executed using an appropriate older version of the backup tool.

            Starting up without {{ib_logfile0}} will no longer be supported; see also MDEV-27199.
            marko Marko Mäkelä made changes -
            ralf.gebhardt Ralf Gebhardt made changes -
            Labels performance Preview_10.8 performance
            marko Marko Mäkelä made changes -
            marko Marko Mäkelä made changes -
            rob.schwyzer@mariadb.com Rob Schwyzer (Inactive) made changes -
            marko Marko Mäkelä made changes -
            marko Marko Mäkelä made changes -
            marko Marko Mäkelä made changes -
            marko Marko Mäkelä made changes -
            marko Marko Mäkelä made changes -
            julien.fritsch Julien Fritsch made changes -
            julien.fritsch Julien Fritsch made changes -
            julien.fritsch Julien Fritsch made changes -
            julien.fritsch Julien Fritsch made changes -
            marko Marko Mäkelä made changes -
            rob.schwyzer@mariadb.com Rob Schwyzer (Inactive) made changes -
            rob.schwyzer@mariadb.com Rob Schwyzer (Inactive) made changes -
            marko Marko Mäkelä made changes -
            marko Marko Mäkelä made changes -
            julien.fritsch Julien Fritsch made changes -
            julien.fritsch Julien Fritsch made changes -
            marko Marko Mäkelä made changes -
            marko Marko Mäkelä made changes -
            marko Marko Mäkelä made changes -
            axel Axel Schwenke made changes -

            People

              marko Marko Mäkelä
              marko Marko Mäkelä
              Votes:
              8 Vote for this issue
              Watchers:
              33 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved:

                Git Integration

                  Error rendering 'com.xiplink.jira.git.jira_git_plugin:git-issue-webpanel'. Please contact your Jira administrators.