WL#9536: InnoDB_New_DD: Support crash-safe DDL

Affects: Server-8.0   —   Status: Complete

This is to support crash-safe DDL for new DD project. With the
single persistent system table in new DD, it's easier to support crash-safe DDL.
There is no chance that Server and InnoDB have different metadata for database
objects.

It's necessary to write DDL logs to support rollback. To make rollback
absolutely safe, all data file operations should be done in the last phase of
corresponding DDL.
Correctness guarantees:

FR1. READ COMMITTED and stronger isolation levels must see a consistent view
of the data dictionary tables (except DDL_LOG).

FR2. After completing crash recovery and before starting any DDL operation,
any transaction at any isolation level (including READ UNCOMMITTED)
observes the following:
FR2a. The DDL_LOG table is empty.
FR3b. All data dictionary tables are consistent.

FR1. FR2a and FR2b hold after a clean shutdown.

FR3. In read-only mode, the server should emit messages if any incomplete DDL
transactions were recovered, or if there exist any records in the DDL_LOG
table.
NOTE: An incomplete DDL transaction will be treated as if it had not been
committed. This is OK; there is no reason to prevent startup.

FR4. DDL (Create/Drop/Alter/Rename/Truncate) should either be committed or 
completely rolled back in case of a crash during DDL operation.


FR5. innodb_flush_log_at_trx_commit would be ignored for DDL, and redo logs for
DDL would be always flushed on transaction commit.

NOTE: For FR1, this is more like a precondition for this(and other InnoDB new DD
worklogs). Only with this guaranteed, metadata for a single table can be got
correctly, and the table can be accessed in a right way. So this is not a 
feature that is introduced by this worklog.

Some DDL operations consist of multiple subtransactions and multiple
records in the new DDL_LOG table. For example, ALTER TABLE may have to
create some storage objects in the ‘prepare’ phase, and
these changes would have to be made durable before the actual work
starts. The ‘prepared’ data will be either ‘committed’ or ‘rolled
back’ in the main transaction, or it will be ‘rolled back’ as part of
crash recovery.

Note: All DD related chagnes should be in one transaction(DD transaction).

Note that our use of ‘Prepare’ and ‘Commit’ is not to be confused with
the XA 2-phase commit. A DDL operation can be structured as follows:


1. ‘Prepare’: create any needed objects (files, B-trees)
This will write type='FREE' or type='DROP' or type='DELETE' records
to DDL_LOG.
2. ‘Perform’: for example, populate index trees
3. ‘Commit’: update the data dictionary tables and commit DD transaction; -- or 
XA PREPARE; XA COMMIT;
4. ‘Post-ddl’ (garbage collection): replay and delete all records of DDL_LOG
(actually perform B-tree free, file delete, etc),

The DDL_LOG is a hidden InnoDB table mysql.innodb_ddl_log
in the data dictionary tablespace.

§0 The DDL_LOG schema definition and operation types

DDL_LOG is a hard-coded InnoDB table in the data dictionary
tablespace. No DDL is allowed on it, and SQL DML access is normally
blocked as well.

The hard-coded schema definition corresponds to the following SQL:

CREATE TABLE mysql.innodb_ddl_log (
  id BIGINT UNSIGNED NOT NULL AUTO_INCREMENT PRIMARY KEY,
  thread_id BIGINT UNSIGNED NOT NULL,
  type INT UNSIGNED NOT NULL,-- Types of 'FREE', 'DELETE', 'RENAME', and 'DROP'
  space_id INT UNSIGNED,
  page_no INT UNSIGNED,
  index_id BIGINT UNSIGNED,
  table_id BIGINT UNSIGNED,
  old_file_path VARCHAR(512) COLLATE UTF8_BIN,
  new_file_path VARCHAR(512) COLLATE UTF8_BIN,
  KEY(thread_id)
);

All operations will be implemented in an idempotent way, that is, it
must be possible to execute a given operation multiple times in a row,
leading to the same end result as executing it only once.

we shall get auto inc id before we insert a ddl log, and set id and thread_id 
for each insert. thread_id is used in ‘Post_ddl’ to replay and delete all ddl 
log records belonging to a DDL.

Changes to this table should be persisted ASAP. That is the redo logs of
changes to this table should be flushed to disk once the transaction gets
committed, regardless of what value innodb_flush_log_at_trx_commit is.
This prevents the scenario that data files are modified but related redo logs
were not flushed, thus some error in recovery/rollback.

§0.1 innodb_ddl_log.type='FREE': Dropping an index tree

One of the most common operations will be dropping an index tree:

INSERT INTO innodb_ddl_log
SET type='FREE', tablespace_id=…, page_number=…, index_id=…;

The page_number refers to the page that contains the allocation
information, which currently is the root page containing BTR_SEG_TOP
and BTR_SEG_LEAF.

If the index_id matches the PAGE_INDEX_ID on the page, the operation will
be applied by freeing the pages on BTR_SEG_TOP and BTR_SEG_LEAF, and by
replacing the (tablespace_id,page_number) with an all-zero page.

IF PAGE_TYPE=FIL_PAGE_INDEX AND PAGE_LEVEL=0 AND PAGE_INDEX_ID=index_id THEN
   btr_free_but_not_root(block);
   mtr_start(mtr);
   btr_free_root(block, mtr);
   fsp_init_file_page(block, mtr);
   mtr_commit(mtr);
END IF

NOTE: We can execute btr_free_but_not_root() multiple times on an
index root page, but we can execute btr_free_root() only once. The
fsp_init_file_page() and the consistency check will make this
operation idempotent.

NOTE: We shall drop ibuf pages for the index, or we can't drop the index with 
reference count > 0.

§0.2 innodb_ddl_log.type='DELETE': Deleting a file

Another common operation is deleting a file:

INSERT INTO innodb_ddl_log
SET type='DELETE', tablespace_id=…, page_number=…, old_file_path=…;

If the old_file_path is not found, no warning will be issued and the
DDL_LOG record will be deleted.

If the first page of the file identified by old_file_path is found but
does not contain (tablespace_id,page_number), a warning will be issued
to the server log and the DDL_LOG record will be deleted.

If old_file_path with (tablespace_id,page_number) is found,
the file will be deleted.

For single-table tablespaces, file deletion is needed for recovering from:

• CREATE TABLE(drop the newly created tablespace file)
• DROP TABLE(drop the tablespace file)
• TRUNCATE TABLE (drop the old or truncated tablespace file)
• ALTER TABLE (table-rebuilding; drop the old or altered tablespace file)

For user-created tablespaces, file deletion is needed for recovering from:

• CREATE TABLESPACE, ALTER TABLESPACE…ADD DATA FILE (rollback)
• DROP TABLESPACE, ALTER TABLESPACE…DROP DATA FILE (drop tablespace file)
• ALTER TABLESPACE (rebuilding; drop the old or altered tablespace file)

§0.3 innodb_ddl_log.type='RENAME': Renaming a file

Renaming files is presently only covered by the MLOG_FILE_RENAME2 redo
log record. This is not sufficient, because we also need to be able to
roll back an operation. So, we will write both MLOG_FILE_RENAME2 redo log
records to inform crash recovery and hot backup tools of the operation,
and DDL_LOG records to guarantee that rollback will be done:

INSERT INTO innodb_ddl_log
SET type='RENAME', tablespace_id=…, page_number=…, table_id=…,
 old_file_path=…, new_file_path=…;

The table_id may be left as NULL. For executing ha_innobase::rename_table()
that affect innodb_file_per_table tablespaces, we may fill in table_id
for diagnostic purposes.

If the old_file_path is not found, no warning will be issued and the
DDL_LOG record will be deleted.

If the first page of the file identified by old_file_path is found but
does not contain (tablespace_id,page_number), a warning will be issued
to the server log and the DDL_LOG record will be deleted.

If old_file_path with (tablespace_id,page_number) is found and
also new_file_path exists, recovery will be aborted, asking for
user intervention.

If old_file_path with (tablespace_id,page_number) is found and
new_file_path does not exist, the file will be renamed.

NOTE: type='RENAME' must be applied in descending order of
innodb_ddl_log.id, because there can be multiple
RENAME operations in a transaction. See §6.3.2.

§0.4 innodb_ddl_log.type='DROP': Dropping dynamic metadata

DDL operations that drop or rebuild InnoDB tables must remove any buffered 
dynamic persistent metadata from the persistent but non-transactional table 
called innodb_dynamic_metadata. Because we do not want to lose any dynamic 
persistent metadata in case of a rollback, the removal must be done at the 
‘Post_ddl’ step.

innodb_dynamic_metadata entries will only be written during DML, never during 
DDL. Thus, operations like CREATE TABLE never need to write a 'DROP' entry.
Before Commit, DROP TABLE, TRUNCATE TABLE and other table-rebuilding
operations will have to write a 'DROP' entry for any metadata that
post ddl should remove from the innodb_dynamic_metadata:

INSERT INTO innodb_ddl_log
SET type='DROP', tablespace_id=…, table_id=…;

§1 CREATE TABLE

§1.1 Invoke ha_innobase::create() on each table (‘Prepare’ and ‘Perform’):

IF single_tablespace(table) THEN
  BEGIN; -- InnoDB-internal subtransaction
    INSERT INTO innodb_ddl_log
    SET type='DELETE', tablespace_id=…, old_file_path=…;
  COMMIT;
  -- In the main transaction, remove the DDL_LOG record (no commit yet).
  DELETE FROM innodb_ddl_log WHERE id=LAST_INSERT_ID();
END IF
For each index tree:
  mtr_start(mtr); -- Start a mini-transaction inside the engine.
  Within mtr, allocate the root page and determine its number.
  btr_create(root, mtr); -- Initializes the root page.
  mtr_commit(mtr);
  IF NOT single_tablespace(table) THEN
    BEGIN; -- InnoDB-internal subtransaction
      INSERT INTO innodb_ddl_log
      SET type='FREE', tablespace_id=…, page_number=…;
    COMMIT;
    -- In the main transaction, remove the DDL_LOG record (no commit yet).
    DELETE FROM innodb_ddl_log WHERE id=LAST_INSERT_ID();
  END IF
End for

NOTE: The above can consist of multiple subtransactions
(one for each created index tree).

NOTE: If the server crashes after B-tree creation and FREE ddl log trx
commit, the B-tree will get lost(two segments lost). In order to avoid
latching issues, we discard the old design:
     We must commit the write to DDL_LOG before committing the
mini-transaction that allocates and initializes the root page.  We
could also do this by invoking trx_commit_low(trx, mtr) to perform the
first mtr_commit() of the btr_create() atomically with the
subtransaction commit.
     We shall release latches acquired in btr_create() before updating
innodb_ddl_log. This is because in btr_create(), some low-level latches
would be acquired, while in later updating innodb_ddl_log, higher level
latches for index tree, undo log, etc. would be required. This violates
the latching order. In the worst case, tablespace latch could be acquired
twice if a table in tablespace 'mysql' is under DDL. To handle this, we have
to release the latches acquired in btr_create(), except the page lock.
Then we can safely start background transaction and update innodb_ddl_log.

NOTE: Currently there is no handler API for starting or committing a
nested transaction. There only exist InnoDB-internal APIs for that.
Thus InnoDB can only support the DDL_LOG based recovery if the DDL_LOG
table is an InnoDB table.

§1.2 ‘Commit’ of CREATE TABLE (driven by MySQL):

INSERT INTO dd…; -- driven by MySQL; uses se_private_data from §1.1
COMMIT DICTIONARY TRANSACTION;

Because in step §1.1 the main transaction removed the type='DELETE' or
type='FREE' records that were written by subtransactions, on commit of
the main transaction, the DDL_LOG table will not contain any
additional records.

The DDL_LOG table will be updated in the same transaction with
the data dictionary tables. In the event of a crash, we will rely on
InnoDB crash recovery and InnoDB transactions. If and only if the
final COMMIT is recovered, the table will exist in the DD tables
and the DDL_LOG will not contain records to be recovered.

§1.3 Roll back or crash recovery of CREATE TABLE

If CREATE TABLE needs to be rolled back, we simply
replay the DDL_LOG operations that were written in §1.1 ‘Prepare’,
as noted in §0.1 and §0.2:

type='FREE': Replace the index tree root with an all-zero page.
type='DELETE': Delete the files if they exist.

As part of rolling back a CREATE TABLE operation before crash, this
clean-up will only be performed on the DDL_LOG records that were
written by the DDL operation. There could exist other DDL_LOG records
written by other concurrently running DDL operations.

NOTE: even though the main transaction of the CREATE TABLE did DELETE the
DDL_LOG records straight after inserting them, main transaction is not
committed until the end of step §1.2. So, crash
recovery as well as post-ddl cleanup will see all DDL_LOG for the
non-committed DDL operation.

§2 DROP TABLE

§2.1: ha_innobase::delete_table(): ‘Commit’ of DROP TABLE:

BEGIN DICTIONARY TRANSACTION;
DELETE FROM dd…; -- driven by MySQL
IF single_tablespace(dropped_table) THEN
  INSERT INTO innodb_ddl_log
  SET type='DELETE', tablespace_id=…, old_file_path=…;
ELSE
    FOR EACH INDEX
      INSERT INTO innodb_ddl_log
      SET type='FREE', tablespace_id=…, page_number=…, index_id=…;
    END FOR
END IF
COMMIT DICTIONARY TRANSACTION;

The COMMIT marks the DROP operation completed.  No changes were made
inside the storage engine yet.  The space will be reclaimed in a
subsequent step, which can be performed while holding a MDL on the
DDL_LOG table only.

§2.2: ‘Post-ddl’ of DROP TABLE (also, crash recovery roll-forward):

We simply replay the DDL_LOG operations that were written in §2.1 ‘Commit’,
as noted in §0.1, and §0.2:

type='FREE': Replace the index tree root with an all-zero page.
type='DELETE': Delete the files if they exist.

§3 ALTER TABLE, not rebuilding the table

For schema-only changes, such as renaming a column, extending a VARCHAR, etc.,
crash recovery works ‘out of the box’ by simply performing the DDL operations
within a single transaction at handler::commit_inplace_alter_table().

ALTER TABLE…ADD INDEX would create and delete the DDL_LOG entries in
prepare_inplace_alter_table_dict(), as noted in §1.

ALTER TABLE…DROP INDEX would do nothing until
handler::commit_inplace_alter_table(commit=true). 

In summary, this may comprise n_add_index subtransactions in ‘Prepare’
and n_drop_index subtransactions in ‘Post-ddl’:

§3.1 ‘Prepare’: handler::prepare_inplace_alter_table():

BEGIN DICTIONARY TRANSACTION; -- main DDL transaction
For each ADD INDEX:
  BEGIN; -- InnoDB-internal subtransaction
    INSERT INTO innodb_ddl_log
    SET type='FREE', tablespace_id=…, page_number=…, index_id=…;
  COMMIT;
  -- In the main transaction, remove the DDL_LOG record (no commit yet).
  DELETE FROM innodb_ddl_log WHERE id=LAST_INSERT_ID();
End for

§3.1.1 ‘Perform’: handler::inplace_alter_table()

This phase does not affect the data dictionary.

§3.2 ‘Commit’: handler::commit_inplace_alter_table(commit=true):

  UPDATE DD…; -- perform the schema changes as driven by MySQL
  -- write an entry for each DROP INDEX, in the main transaction
  INSERT INTO innodb_ddl_log
  SET type='FREE', tablespace_id=…, page_number=…, index_id=…;
  -- up until this point, we can roll back at any step
  COMMIT DICTIONARY TRANSACTION; -- marks the DDL completed

§3.3 ‘Post-ddl’ of ALTER TABLE and crash recovery roll-forward

We simply replay the type='FREE' operations that were written in §2.1
‘commit’ for DROP INDEX, as noted in §0.1:

type='FREE': Replace the index tree root with an all-zero page.

§3.4 ‘Post-ddl’ and crash recovery roll-back

The COMMIT DICTIONARY TRANSACTION in §3.2 marks the operation
committed. If we run into an error, the changes made in the main
transaction since §3.1 will be rolled back (full transaction rollback,
or rollback to savepoint), and we may have some DDL_LOG for undoing
ADD INDEX operations.

We simply replay the type='FREE' operations that were written in §3.1
‘Prepare’ for ADD INDEX, as noted in §0.1:

type='FREE': Replace the index tree root with an all-zero page.

§4 ALTER TABLE, rebuilding the table

If the ALTER TABLE involves rebuilding the table, from the InnoDB
point of view it will look like CREATE TABLE (§1) and DROP TABLE (§2).

This is the case both for ALGORITHM=COPY and for the cases of
ALGORITHM=INPLACE.

When the table is altered to a single-table tablespace, we will create
a single-table tablespace (§6.1) on the new table. If the table was a
single-table tablespace before the operation, we will drop the old
single-table tablespace (§6.2) when committing the operation.

NOTE: InnoDB requires that the new copy of the table be assigned a new
table_id, so that purge will simply drop any old undo log entries for
the table.

§4.1: ALTER TABLE...ALGORITHM=COPY

The ALGORITHM=COPY is implemented like CREATE TABLE, INSERT…SELECT,
RENAME TABLE, DROP TABLE, RENAME TABLE. It will be atomic if we do
not commit the transaction between the steps.

§4.1.1: ‘Prepare’ CREATE TABLE inside InnoDB: See §1.1.

§4.1.2: Report the INDEXES.se_private_data back to MySQL: See §1.1.

§4.1.3: ‘Perform’: Copy the rows from the original table to the new one.

This uses ha_innobase::write_row() as usual.

§4.1.4: ‘commit’: Atomically replace the old table with the new one

This consists of three operations in the same main transaction:
§1.2 ‘Commit’ phase of CREATE TABLE
§2.1 ‘Commit’ phase of DROP TABLE
§6.3 RENAME TABLE

NOTE: We will have to reassign TABLES.se_private_id. Also, the
INDEXES.se_private_data will have to be reassigned.
Other id inside the DD may remain unchanged if desired.

§4.1.5: ‘Post-ddl’ the pending type='FREE', type='DELETE', type='DROP'.

We simply replay the DDL_LOG operations that were written in §4.1.4 ‘Commit’,
as noted in §0.1, §0.2 and §0.4:

type='FREE': Replace the index tree root with an all-zero page.
type='DELETE': Delete the files if they exist.

NOTE: If an explicitly created (shared) tablespace becomes empty as a
result of DROP INDEX or DROP TABLE or any DDL operation, it will not
be dropped. Only implicitly created (file-per-table) tablespaces will
be dropped implicitly.

§4.2: ALTER TABLE...ALGORITHM=INPLACE

This is similar to §4.1 (ALGORITHM=COPY).

§4.2.1: handler::prepare_inplace_alter_table() does §4.1.1, §4.1.2
§4.2.2: handler::inplace_alter() copies the data, as in §4.1.3
§4.2.3: handler::commit_inplace_alter(commit=true): switch as in §4.1.4
§4.2.4: handler::post_ddl() applies the DDL_LOG as in §4.1.5

§4.3: ALTER TABLE...PARTITION, ALGORITHM=INPLACE 

§4.3.1: handler::prepare_inplace_alter_table() does §4.1.1, §4.1.2

This depends on what specific operation is going on which partitions.
For all operations that will create new partitions, such as ADD PARTITION, etc.
New partitions with some temporary names would be created. Others could be
no-op, such as DROP PARTITION, etc.

§4.3.2: handler::inplace_alter() copies the data, as in §4.1.3

It copies data between partitions if necessary. Not all operations do this.

§4.3.3: handler::commit_inplace_alter(commit=true)

This also depends on operations. First let's understand that ALTER PARTITION
may have some name conflicts. For example, REORGANIZE PARTITION can reorganize
two partitions with names p1 and p2 into new partitions with the same name
p1 and p2. Also ADD PARTITION for HASH partitioning will reorganize all
existing partitions, thus some new partitions would have the same name with
all old partitions. To prevent this kind of name conflicting, we have to
create some partitions with temporary names, and do some RENAME to adjust them.

If this is DROP PARTITION, do as §2.1 for every partition when no name
conflicts and do as §6.3 and §2.1 by renaming before deleting for every
partition.

If this is ADD PARTITION and no name conflicts, it's no-op, while if name
conflict, since the old partition has been renamed(§6.3) to temporary name,
just RENAME(§6.3) it to the original name of the old partition.

If this is about replacing an old partition with a new one of the same name,
such as COALESCE PARTITION, etc. Old partitions should be renamed(§6.3) to
temporary names, and newly created partitions should be renamed(§6.3) to
the correct names. Once all these done, old partitions with temporary names
can be finally dropped(§2.1)

§4.3.4: handler::post_ddl() applies the DDL_LOG

This may include type='FREE', type='DELETE', type='DROP' and type='RENAME'.

§5 TRUNCATE TABLE

TRUNCATE TABLE is implemented as CREATE TABLE (§1) and DROP TABLE
(§2) in a single atomic transaction. The ‘Commit’ phase of both will
be executed within a single transaction.

ha_innobase::truncate() will:

§5.1: RENAME the table(§6.3) to a temporary one to prepare for §5.2.
§5.2: CREATE a new table with the same name: See §1.
§5.3: DROP the old temporary table: See §2.

After handlerton::commit(), ha_innobase::post_ddl() will actually
perform the DROP TABLE part of TRUNCATE TABLE. See §2.2.

Once TRUNCATE starts destructing data, it cannot be stopped.
Our solution to that is that no data will be destroyed before
the transaction is committed. We have two possibilities for crash recovery:

(a) The server was killed before the DD transaction commit that
executed the ‘Commit’ steps of both DROP TABLE and CREATE TABLE.

The data dictionary tables will contain the table definition as it was
before the TRUNCATE started. We would process some type='DELETE' or
type='FREE' records that were written in the ‘Prepare’ phase of the
CREATE TABLE (§1.1), along with some type='RENAME' records written in §5.1.

(b) The server is killed after 'Commit' but before the ‘Post-ddl’
step for the CREATE TABLE completed.

The data dictionary tables will contain the state after the TRUNCATE
was marked committed. We may replay some type='DELETE' or type='FREE'
or type='DROP' records from the ‘Commit’ phase of the DROP TABLE
(§2.2).

§6 Tablespaces

§6.1 CREATE TABLESPACE, also creation of a single-table tablespace

This is analogous to CREATE TABLE or CREATE INDEX (§1), with the
exception that recovery will delete the tablespace file instead of
dropping index trees.

§6.1.1: ‘Prepare’ of CREATE TABLESPACE inside InnoDB (cf. §1.1):

BEGIN DICTIONARY TRANSACTION; -- driven by MySQL
  BEGIN; -- InnoDB-internal subtransaction
    INSERT INTO innodb_ddl_log
    SET type='DELETE', tablespace_id=…, page_no=…, old_file_path=…;
  COMMIT;
  -- In the main transaction, remove the DDL_LOG record (no commit yet).
  DELETE FROM innodb_ddl_log WHERE id=LAST_INSERT_ID();
-- no commit yet

§6.1.2 ‘Commit’ of CREATE TABLESPACE to the DD tables (driven by MySQL):

  INSERT INTO dd.tablespaces… VALUES (space_id, …);
COMMIT DICTIONARY TRANSACTION;

§6.2 DROP TABLESPACE, also dropping of a single-table tablespace

This is analogous to DROP TABLE or DROP INDEX (§2), with the
exception that recovery will delete the tablespace file instead of
dropping index trees.

§6.2.1: ‘Commit’ of DROP TABLESPACE (cf. §2.1):

BEGIN DICTIONARY TRANSACTION;
  DELETE FROM dd.tablespaces… WHERE … = space_id; -- driven by MySQL
  INSERT INTO innodb_ddl_log
  SET type='DELETE', tablespace_id=…, page_no=…, old_file_path=…;
COMMIT DICTIONARY TRANSACTION;

§6.2.2: ‘Post-ddl’ of DROP TABLESPACE (also, crash recovery roll-forward):

As noted in §0.2:

type='DELETE': Delete the files if they exist.

§6.3 RENAME TABLE in single-table tablespaces

RENAME TABLE on single-table tablespaces is currently only covered by
the InnoDB redo log (MLOG_FILE_RENAME2, which replaced MLOG_FILE_RENAME.
Currently, this is applied by both MEB and InnoDB crash recovery.

Both InnoDB and MEB will discover tablespaces based on MLOG_file_path,
MLOG_FILE_DELETE and MLOG_FILE_RENAME2 records.
InnoDB will not replay MLOG_FILE_RENAME2 records.

§6.3.1 Crash-safe renaming of files

RENAME operations will be covered by MLOG_FILE_RENAME2
and innodb_ddl_log.type='RENAME' records as follows:

BEGIN; -- InnoDB-internal subtransaction as in §1.1
  INSERT INTO innodb_ddl_log
  SET type='RENAME', tablespace_id=space_id, page_number=…,
  old_file_path=to, new_file_path=from; -- this is for rolling back!
COMMIT; -- persist by log_write_up_to(LSN_MAX, true)
-- In the main transaction, remove the DDL_LOG record (no commit yet).
DELETE FROM innodb_ddl_log WHERE id=LAST_INSERT_ID();
-- Do the low-level work:
Suspend writes to this tablespace (if possible).
Call sync_file_range() or fdatasync() or fsync() to sync the first pages.
error = rename(from, to) -- or MoveFileEx(from, to, MOVEFILE_WRITE_THROUGH)
IF NOT error THEN
  Call syncfs(fd) or fsync(directory) or fsync(file) to persist the rename.
  Write MLOG_FILE_RENAME2(space_id, first_page_number, from, to).
END IF
Resume writes to the tablespace.
RETURN !error

§6.3.2 Crash recovery of renaming files

This is a high-level description of a multi-file rename operation. We
are assuming that the page_number always matches and at most one of
the logged names exist in the file system. See §0.3 for details on
these low-level consistency checks.

§7 Notes

§7.1 Notes on TABLESPACE_FILES, RENAME TABLE

RENAME TABLE is somewhat similar to table-rebuilding ALTER TABLE where
we have to switch tables at the end of the operation.

The crash recovery of RENAME will be based on the DDL_LOG table, as
noted in §6.3. This has the advantage that the DBA can recover from
failures by using the innodb_files.txt. Now we may rely on InnoDB: Replace
MLOG_FILE_NAME with MLOG_FILE_OPEN, in which we should go back 
to the old way for tablespace discovery(e.g. we shall get missing tablespace in 
startup from crash just after RENAME file.)

§7.2 Notes on DDL recovery: undo logging

Any data dictionary transactions that were incomplete at the time of
crash, will be rolled back when InnoDB replays the redo log for the
data dictionary.

If and only if a COMMIT DICTIONARY TRANSACTION has made it into the
InnoDB redo log (and MySQL binlog), the transaction is considered
committed after recovery. What may remain to be done is the
‘Post-ddl’ roll-forward of committed DDL operations, or cleanup of some 
incomplete DDL operations. This is done by
processing and deleting DDL_LOG records as noted in §8.

NOTE: Typically, the DDL_LOG table will be empty. Only a crash or
server kill may leave records there.

NOTE: If there are multiple concurrent DDL transactions, there may
exist records for multiple tables in the DDL_LOG table.

NOTE: Any DDL operation that completes without a crash or kill will
collect the DDL_LOG garbage and roll forward their own changes
(‘post-ddl’).

DDL_LOG replay will not modify any tables. Only the replayed DDL_LOG
records will be deleted.

§7.2.1 Looking up tables for rollback and purge

There is a common undo log that covers operations both on the data
dictionary (metadata) and on user data. Even if we later introduced
catalog- or shard-specific undo logs, the undo logs in each shard
would cover both DDL and DML transactions.

A mixed DDL+DML transaction may write undo log records for tables
whose definitions have not been committed to the data dictionary
yet. For example, CREATE TABLE t…REPLACE SELECT… can generate both
insert_undo and update_undo records for the new table. The REPLACE
functionality (as well as IGNORE) depends on the DML undo logging.

On crash recovery, InnoDB will scan all undo log headers to resurrect
any incomplete transactions and put them into trx_sys->rw_trx_list.
We will also identify those undo logs that contain DDL operations. Server will 
provide interface thd_is_dd_update_stmt() for us to mark a DDL transaction.

During the rollback of DDL and DDL+DML transactions, we can
use READ COMMITTED for the table_id lookup, because the undo logs and
the dictionary tables were already recovered from the undo log. If we fail to 
find the table that an undo record is referring to, we can simply discard the 
undo record. In this way, rolling back the DML part of CREATE TABLE…SELECT will 
be fast, as the table_id will simply not be found.

The purge of delete-marked records and committed undo logs will be
started only after the redo log apply for user tablespaces has been
initiated. Theoretically, we could start the purge for DDL-only
transactions earlier, but this would add complexity for minimal gain.

§8 General recovery notes

Normal redo & undo, then ddl log replay.

Pseudocode for the DDL_LOG replay follows:

CREATE PROCEDURE ddl_log_replay()
BEGIN
  DECLARE c CURSOR FOR
  SELECT * FROM innodb_ddl_log ORDER BY id DESC;

  OPEN c;
  REPEAT
    FETCH c;
    CASE type
    WHEN 'FREE' THEN
      Drop the index tree and replace it with an all-zero page (§0.1)
    WHEN 'DELETE' THEN
      Delete the files if they exist (§0.2)
    WHEN 'RENAME' THEN
      -- NOTE: c.tablespace_id should refer to an existing dd.tablespaces.id
      Rename the file if needed (§0.3)
    END CASE
    WHEN 'DROP' THEN
      Drop the table dynamic metadata entries if needed (§0.4)
    END CASE
    COMMIT DICTIONARY TRANSACTION;
    BEGIN DICTIONARY TRANSACTION;
  UNTIL all_rows_fetched;
  CLOSE c;
END;
Two global variables are introduced by this worklog.

1. Variable innodb_print_ddl_logs to decide if DDL logs would be printed to
error log or not.

static MYSQL_SYSVAR_BOOL(print_ddl_logs, srv_print_ddl_logs,
  PLUGIN_VAR_OPCMDARG,
  "Print all DDl logs to MySQL error log (off by default)",
  NULL, NULL, FALSE);


2. Variable innodb_ddl_log_crash_reset_debug to reset crash point counters,
which is now only used in debug-mode for MTR.

static MYSQL_SYSVAR_BOOL(ddl_log_crash_reset_debug,
  innodb_ddl_log_crash_reset_debug,
  PLUGIN_VAR_OPCMDARG,
  "Reset all crash injection counters to 1",
  NULL, ddl_log_crash_reset, FALSE);