33#ifndef CLONE_CLONE_INCLUDE 
   34#define CLONE_CLONE_INCLUDE 
   51#define CLONE_FILES_DIR OS_FILE_PREFIX "clone" OS_PATH_SEPARATOR_STR
 
  224    ib::info(ER_IB_CLONE_OPERATION) << 
"Clone Set Error code: " << 
err 
  301                         uint32_t &ret_block);
 
  376                  int in_err, uint32_t &num_wait);
 
  481    if (
err == ER_NET_ERROR_ON_WRITE || 
err == ER_NET_READ_ERROR ||
 
  482        err == ER_NET_WRITE_INTERRUPTED || 
err == ER_NET_READ_INTERRUPTED ||
 
  483        err == ER_NET_WAIT_ERROR) {
 
  501    if (chunks.empty()) {
 
  512    auto it = chunks.begin();
 
  513    auto min_incomplete_chunk = it->first;
 
  515    ut_ad(min_complete_chunk != min_incomplete_chunk);
 
  516    return (min_incomplete_chunk < min_complete_chunk);
 
  595           const char *data_dir);
 
  655    bool has_thd = (thd != 
nullptr);
 
  665    return (
err == ER_QUERY_INTERRUPTED);
 
  773                        uint32_t &percent_done,
 
  774                        std::chrono::steady_clock::time_point &disp_time);
 
  808                         bool &handle_deleted);
 
  891                uint64_t offset, 
byte *
buffer, uint32_t size,
 
  986                           uint32_t block_size, uint32_t &compressed_len);
 
 1005                       unsigned char *
buffer, uint32_t buf_len);
 
 1128                      uint64_t snapshot_id, 
bool is_pfs_monitor,
 
 1173                       bool check_intr, uint32_t &blocked_state, 
int &
error);
 
 1180                     uint32_t blocked_state);
 
 1221    auto alert_count = 
static_cast<int>(alert_interval / sleep_time);
 
 1222    auto total_count = 
static_cast<int>(
timeout / sleep_time);
 
 1232      if (mutex != 
nullptr) {
 
 1238      if (cur_sleep_time > sleep_time) {
 
 1239        cur_sleep_time = sleep_time;
 
 1242      std::this_thread::sleep_for(cur_sleep_time);
 
 1244      if (cur_sleep_time < sleep_time) {
 
 1246        cur_sleep_time *= 2;
 
 1253      if (mutex != 
nullptr) {
 
 1258      if (loop_count == 0) {
 
 1263      auto alert = (alert_count > 0) ? (loop_count % alert_count == 0) : 
true;
 
 1281                 std::forward<Wait_Cond_Cbk_Func>(func), mutex, 
is_timeout));
 
uint32_t space_id_t
Tablespace identifier.
Definition: api0api.h:52
 
Clone Handle for copying or applying data.
Definition: clone0clone.h:576
 
int send_file_metadata(Clone_Task *task, const Clone_File_Meta *file_meta, bool is_redo, Ha_clone_cbk *callback)
Send current file information via callback.
Definition: clone0copy.cc:1134
 
bool match_hdl_type(Clone_Handle_Type other_handle_type)
Check if clone type matches.
Definition: clone0clone.h:686
 
int sparse_file_write(Clone_File_Meta *file_meta, unsigned char *buffer, uint32_t len, pfs_os_file_t file, uint64_t start_off)
Write pages to file and punch holes.
Definition: clone0apply.cc:1266
 
void close_master_file()
Close master task file if open and unpin.
Definition: clone0clone.cc:687
 
void build_descriptor(Clone_Desc_Locator *loc_desc)
Build locator descriptor for the clone handle.
Definition: clone0clone.cc:2083
 
int set_compression(Clone_file_ctx *file_ctx)
Set compression type based on local capability.
Definition: clone0apply.cc:1009
 
int receive_data(Clone_Task *task, uint64_t offset, uint64_t file_size, uint32_t size, Ha_clone_cbk *callback)
Receive data from callback and apply.
Definition: clone0apply.cc:1392
 
int send_state_metadata(Clone_Task *task, Ha_clone_cbk *callback, bool is_start)
Send current state information via callback.
Definition: clone0copy.cc:1040
 
void notify_state_change(Clone_Task *task, Ha_clone_cbk *callback, Clone_Desc_State *state_desc)
Notify state change via callback.
Definition: clone0apply.cc:699
 
byte * m_clone_locator
Serialized locator.
Definition: clone0clone.h:1018
 
bool is_active()
Check if clone state is active.
Definition: clone0clone.h:699
 
int copy(uint task_id, Ha_clone_cbk *callback)
Transfer snapshot data via callback.
Definition: clone0copy.cc:1312
 
bool is_copy_clone() const
Check if it is copy clone.
Definition: clone0clone.h:681
 
uint m_clone_desc_version
Clone descriptor version in use.
Definition: clone0clone.h:1030
 
void display_progress(uint32_t cur_chunk, uint32_t max_chunk, uint32_t &percent_done, std::chrono::steady_clock::time_point &disp_time)
Display clone progress.
Definition: clone0copy.cc:1295
 
int apply_data(Clone_Task *task, Ha_clone_cbk *callback)
Apply data received via callback.
Definition: clone0apply.cc:1516
 
int process_chunk(Clone_Task *task, uint32_t chunk_num, uint32_t block_num, Ha_clone_cbk *callback)
Process a data chunk and send data blocks via callback.
Definition: clone0copy.cc:1439
 
uint m_restart_loc_len
Restart locator length in bytes.
Definition: clone0clone.h:1027
 
int ack_state_metadata(Clone_Task *task, Ha_clone_cbk *callback, Clone_Desc_State *state_desc)
Move to next state based on state metadata and set state information.
Definition: clone0apply.cc:709
 
int open_file(Clone_Task *task, const Clone_file_ctx *file_ctx, ulint file_type, bool create_file, File_init_cbk &init_cbk)
Open file for the task.
Definition: clone0clone.cc:2226
 
bool is_interrupted()
Definition: clone0clone.h:663
 
int apply_file_delete(Clone_Task *task, Clone_file_ctx *file_ctx, const Clone_File_Meta *new_meta)
Apply DDL delete to existing file to update chunk and block information.
Definition: clone0apply.cc:730
 
std::tuple< bool, bool > pins_file(const Clone_Task *task, const Clone_file_ctx *file_ctx)
Check if the task pins a file context.
Definition: clone0copy.cc:1752
 
int send_data(Clone_Task *task, const Clone_file_ctx *file_ctx, uint64_t offset, byte *buffer, uint32_t size, uint64_t new_file_size, Ha_clone_cbk *callback)
Send cloned data via callback.
Definition: clone0copy.cc:1200
 
uint m_clone_arr_index
Index in global array.
Definition: clone0clone.h:1033
 
void set_state(Clone_Handle_State state)
Set current clone state.
Definition: clone0clone.h:692
 
int close_and_unpin_file(Clone_Task *task)
Unpin and close currently pinned file.
Definition: clone0copy.cc:1724
 
int move_to_next_state(Clone_Task *task, Ha_clone_cbk *callback, Clone_Desc_State *state_desc)
Move to next state.
Definition: clone0clone.cc:2132
 
void set_abort()
Set clone to ABORT state end any attached snapshot.
Definition: clone0clone.cc:2212
 
int check_and_pin_file(Clone_Task *task, Clone_file_ctx *file_ctx, bool &handle_deleted)
Check and pin a file context if not already pinned.
Definition: clone0copy.cc:1688
 
bool drop_task(THD *thd, uint task_id, bool &is_master)
Drop task from clone handle.
Definition: clone0clone.cc:2099
 
int restart_copy(THD *thd, const byte *loc, uint loc_len)
Restart copy after a network failure.
Definition: clone0copy.cc:1620
 
int apply(THD *thd, uint task_id, Ha_clone_cbk *callback)
Apply snapshot data received via callback.
Definition: clone0apply.cc:1559
 
int fix_all_renamed(const Clone_Task *task)
Fix the file name and meta information for all files that are renamed with DDL extension.
Definition: clone0apply.cc:936
 
int check_space(const Clone_Task *task)
Check if enough space is there to clone.
Definition: clone0apply.cc:552
 
bool abort_by_ddl() const
Definition: clone0clone.h:747
 
int close_file(Clone_Task *task)
Close file for the task.
Definition: clone0clone.cc:2319
 
bool is_idle()
Check if clone is idle waiting for restart.
Definition: clone0clone.h:707
 
bool m_abort_ddl
If concurrent DDL should abort clone.
Definition: clone0clone.h:1045
 
bool replace_datadir() const
Definition: clone0clone.h:618
 
bool is_init()
Check if clone is initialized.
Definition: clone0clone.h:703
 
void attach()
Attach to the clone handle.
Definition: clone0clone.h:598
 
void set_ddl_abort()
Allow concurrent DDL to abort clone.
Definition: clone0clone.h:750
 
int apply_file_metadata(Clone_Task *task, Ha_clone_cbk *callback)
Create file metadata based on callback.
Definition: clone0apply.cc:1122
 
int send_all_ddl_metadata(Clone_Task *task, Ha_clone_cbk *callback)
Send all DDL metadata generated.
Definition: clone0copy.cc:1769
 
uint detach()
Detach from the clone handle.
Definition: clone0clone.h:602
 
int restart_apply(THD *thd, const byte *&loc, uint &loc_len)
Build locator with current state and restart apply.
Definition: clone0apply.cc:1608
 
byte * get_locator(uint &loc_len)
Get locator for the clone handle.
Definition: clone0clone.cc:2059
 
int apply_ddl(const Clone_File_Meta *new_meta, Clone_file_ctx *file_ctx)
Apply DDL changes to file at the end of FILE_COPY stage.
Definition: clone0apply.cc:797
 
~Clone_Handle()
Destructor: Detach from snapshot.
Definition: clone0clone.cc:1940
 
int modify_and_write(const Clone_Task *task, uint64_t offset, unsigned char *buffer, uint32_t buf_len)
Modify page encryption attribute and/or punch hole.
Definition: clone0apply.cc:1333
 
int apply_state_metadata(Clone_Task *task, Ha_clone_cbk *callback)
Move to next state based on state metadata and set state information.
Definition: clone0apply.cc:591
 
int send_all_file_metadata(Clone_Task *task, Ha_clone_cbk *callback)
Send all file information via callback.
Definition: clone0copy.cc:1106
 
uint m_ref_count
Reference count.
Definition: clone0clone.h:1039
 
const char * m_clone_dir
Clone data directory.
Definition: clone0clone.h:1048
 
uint m_locator_length
Locator length in bytes.
Definition: clone0clone.h:1021
 
Clone_Task_Manager m_clone_task_manager
Clone task manager.
Definition: clone0clone.h:1051
 
bool read_compressed_len(unsigned char *buffer, uint32_t len, uint32_t block_size, uint32_t &compressed_len)
Read compressed length from the page.
Definition: clone0apply.cc:1245
 
int apply_task_metadata(Clone_Task *task, Ha_clone_cbk *callback)
Create apply task based on task metadata in callback.
Definition: clone0apply.cc:533
 
byte * m_restart_loc
Serialized Restart locator.
Definition: clone0clone.h:1024
 
int create_clone_directory()
Create clone data directory.
Definition: clone0clone.cc:1950
 
void save_error(int err)
Save current error number.
Definition: clone0clone.h:645
 
const char * get_datadir() const
Definition: clone0clone.h:615
 
int send_keep_alive(Clone_Task *task, Ha_clone_cbk *callback)
Send keep alive while during long wait.
Definition: clone0copy.cc:1014
 
int send_task_metadata(Clone_Task *task, Ha_clone_cbk *callback)
Send current task information via callback.
Definition: clone0copy.cc:993
 
Clone_Snapshot * get_snapshot()
Definition: clone0clone.h:677
 
Clone_Handle_State m_clone_handle_state
Clone handle state.
Definition: clone0clone.h:1012
 
uint get_version()
Get clone data descriptor version.
Definition: clone0clone.h:674
 
Clone_Handle_Type m_clone_handle_type
Clone handle type: Copy, Apply.
Definition: clone0clone.h:1009
 
int file_callback(Ha_clone_cbk *cbk, Clone_Task *task, uint len, bool buf_cbk, uint64_t offset, ut::Location location)
Callback providing the file reference and data length to copy.
Definition: clone0clone.cc:2338
 
bool m_allow_restart
Allow restart of clone operation after network failure.
Definition: clone0clone.h:1042
 
int file_create_init(const Clone_file_ctx *file_ctx, ulint file_type, bool init)
Create a tablespace file and initialize.
Definition: clone0apply.cc:1058
 
uint64_t m_clone_id
Unique clone identifier.
Definition: clone0clone.h:1036
 
std::function< dberr_t(pfs_os_file_t)> File_init_cbk
Definition: clone0clone.h:784
 
Clone_Handle(Clone_Handle_Type handle_type, uint clone_version, uint clone_index)
Construct clone handle.
Definition: clone0clone.cc:1909
 
int init(const byte *ref_loc, uint ref_len, Ha_clone_type type, const char *data_dir)
Initialize clone handle.
Definition: clone0clone.cc:1994
 
uint get_index()
Get clone handle index in clone array.
Definition: clone0clone.h:670
 
int add_task(THD *thd, const byte *ref_loc, uint ref_len, uint &task_id)
Add a task to clone handle.
Definition: clone0clone.h:632
 
bool is_abort()
Check if clone is aborted.
Definition: clone0clone.h:711
 
int check_error(THD *thd)
Check for error from other tasks and DDL.
Definition: clone0clone.h:654
 
byte m_version_locator[CLONE_DESC_MAX_BASE_LEN]
Fixed locator for version negotiation.
Definition: clone0clone.h:1015
 
void update_work(uint size)
Update the progress of the clone operation.
Definition: clone0monitor.h:123
 
Dynamic database snapshot: Holds metadata and handle to data.
Definition: clone0snapshot.h:263
 
Clone_Monitor & get_clone_monitor()
Get performance schema accounting object used to monitor stage progress.
Definition: clone0snapshot.h:348
 
Definition: clone0clone.h:1072
 
Acquire_clone()
Constructor to get and pin clone handle.
Definition: clone0clone.cc:387
 
~Acquire_clone()
Destructor to release and free clone handle if necessary.
Definition: clone0clone.cc:395
 
Clone_Handle * m_clone
Acquired clone handle.
Definition: clone0clone.h:1085
 
Clone_Snapshot * get_snapshot()
Get current clone snapshot.
Definition: clone0clone.cc:402
 
RAII style wrapper to enter and exit wait stage.
Definition: clone0clone.h:1058
 
Wait_stage(const char *new_info)
Constructor to change the THD information string.
Definition: clone0api.cc:2792
 
const char * m_saved_info
Saved old THD information string.
Definition: clone0clone.h:1069
 
~Wait_stage()
Destructor to revert back the old information string.
Definition: clone0api.cc:2802
 
Clone System.
Definition: clone0clone.h:1055
 
void debug_wait_clone_begin()
Debug wait while starting clone and waiting for free marker.
Definition: clone0clone.cc:520
 
Clone_Handle * m_clone_arr[CLONE_ARR_SIZE]
Array of clone handles.
Definition: clone0clone.h:1319
 
uint m_num_snapshots
Number of copy snapshots.
Definition: clone0clone.h:1331
 
Clone_Handle * find_clone(const byte *ref_loc, uint loc_len, Clone_Handle_Type hdl_type)
Find if a clone is already running for the reference locator.
Definition: clone0clone.cc:85
 
void end_ddl_state(Clone_notify::Type type, space_id_t space, uint32_t blocked_state)
End restricted state during some critical ddl phase.
Definition: clone0clone.cc:616
 
void mark_wait()
Mark to indicate that new clone operations should wait.
Definition: clone0clone.cc:507
 
void drop_clone(Clone_Handle *clone_handle)
drop a clone handle from clone system
Definition: clone0clone.cc:267
 
uint m_num_apply_clones
Number of apply clones.
Definition: clone0clone.h:1325
 
static uint s_clone_wait_count
Number of active wait requests.
Definition: clone0clone.h:1197
 
ib_mutex_t * get_mutex()
Get clone sys mutex.
Definition: clone0clone.h:1188
 
int add_clone(const byte *loc, Clone_Handle_Type hdl_type, Clone_Handle *&clone_hdl)
Create and add a new clone handle to clone system.
Definition: clone0clone.cc:229
 
uint m_num_clones
Number of copy clones.
Definition: clone0clone.h:1322
 
static int wait(Clone_Msec sleep_time, Clone_Sec timeout, Clone_Sec alert_interval, Wait_Cond_Cbk_Func &&func, ib_mutex_t *mutex, bool &is_timeout)
Wait till the condition is satisfied or timeout.
Definition: clone0clone.h:1213
 
~Clone_Sys()
Destructor: Call during system shutdown.
Definition: clone0clone.cc:64
 
bool is_space_initialized() const
Definition: clone0clone.h:1300
 
uint64_t get_next_id()
Get next unique ID.
Definition: clone0clone.cc:641
 
bool begin_ddl_state(Clone_notify::Type type, space_id_t space, bool no_wait, bool check_intr, uint32_t &blocked_state, int &error)
Begin restricted state during some critical ddl phase.
Definition: clone0clone.cc:585
 
Clone_Snapshot * m_snapshot_arr[SNAPSHOT_ARR_SIZE]
Array of clone snapshots.
Definition: clone0clone.h:1328
 
int wait_for_free(THD *thd)
Wait for marker to get freed.
Definition: clone0clone.cc:527
 
void mark_free()
Free the wait marker.
Definition: clone0clone.cc:513
 
bool handle_ddl_state(Clone_notify::Type type, space_id_t space, bool begin)
Handle restricted state during critical ddl phase.
 
void mark_active()
Mark clone state to active if no other abort request.
Definition: clone0clone.cc:496
 
bool mark_abort(bool force)
Mark clone state to abort if no active clone.
Definition: clone0clone.cc:440
 
Clone_persist_gtid & get_gtid_persistor()
Definition: clone0clone.h:1294
 
uint m_num_apply_snapshots
Number of apply snapshots.
Definition: clone0clone.h:1334
 
uint64_t m_clone_id_generator
Clone unique ID generator.
Definition: clone0clone.h:1340
 
std::atomic< bool > m_space_initialized
If all innodb tablespaces are initialized.
Definition: clone0clone.h:1343
 
std::function< int(bool is_alert, bool &result)> Wait_Cond_Cbk_Func
Function to check wait condition.
Definition: clone0clone.h:1203
 
std::tuple< bool, Clone_Handle * > check_active_clone()
Check if any active clone is running.
Definition: clone0clone.cc:421
 
static uint s_clone_abort_count
Number of active abort requests.
Definition: clone0clone.h:1194
 
Clone_persist_gtid m_gtid_persister
GTID persister.
Definition: clone0clone.h:1346
 
ib_mutex_t m_clone_sys_mutex
Clone system mutex.
Definition: clone0clone.h:1337
 
static int wait_default(Wait_Cond_Cbk_Func &&func, ib_mutex_t *mutex, bool &is_timeout)
Wait till the condition is satisfied or default timeout.
Definition: clone0clone.h:1277
 
int find_free_index(Clone_Handle_Type hdl_type, uint &free_index)
Find free index to allocate new clone handle.
Definition: clone0clone.cc:130
 
static Clone_Sys_State s_clone_sys_state
Clone System state.
Definition: clone0clone.h:1191
 
Clone_Sys()
Construct clone system.
Definition: clone0clone.cc:52
 
void close_donor_master_file()
Close donor master task file if open and unpin.
Definition: clone0clone.cc:692
 
Clone_Handle * get_clone_by_index(const byte *loc, uint loc_len)
Get the clone handle from locator by index.
Definition: clone0clone.cc:292
 
int attach_snapshot(Clone_Handle_Type hdl_type, Ha_clone_type clone_type, uint64_t snapshot_id, bool is_pfs_monitor, Clone_Snapshot *&snapshot)
Get or create a snapshot for clone and attach.
Definition: clone0clone.cc:309
 
void set_space_initialized()
Remember that all innodb spaces are initialized after last startup.
Definition: clone0clone.h:1297
 
void detach_snapshot(Clone_Snapshot *snapshot, Clone_Handle_Type hdl_type)
Detach clone handle from snapshot.
Definition: clone0clone.cc:361
 
Task manager for manging the tasks for a clone operation.
Definition: clone0clone.h:203
 
bool is_restart_metadata(Clone_Task *task)
Check if needs to send state metadata once.
Definition: clone0clone.h:381
 
int alloc_buffer(Clone_Task *task)
Allocate buffers for current task.
Definition: clone0clone.cc:858
 
int set_chunk(Clone_Task *task, Clone_Task_Meta *new_meta)
Set current chunk and block information.
Definition: clone0clone.cc:1275
 
uint32_t get_next_chunk()
Get next unreserved chunk.
Definition: clone0clone.cc:1185
 
int finish_state(Clone_Task *task)
Mark state finished for current task.
Definition: clone0clone.cc:1687
 
bool drop_task(THD *thd, uint task_id, bool &is_master)
Drop task from task manager.
Definition: clone0clone.cc:1108
 
void reset_transition()
Reset state transition information.
Definition: clone0clone.h:331
 
bool is_restarted()
Check if clone is restarted after failure.
Definition: clone0clone.h:436
 
void reinit_apply_state(const byte *ref_loc, uint ref_len, byte *&new_loc, uint &new_len, uint &alloc_len)
Reinitialize state using locator.
Definition: clone0clone.cc:1388
 
bool m_transferred_file_meta
Definition: clone0clone.h:560
 
int wait_ack(Clone_Handle *clone, Clone_Task *task, Ha_clone_cbk *callback)
Wait for acknowledgement.
Definition: clone0clone.cc:1626
 
Chunk_Info m_chunk_info
Finished and incomplete chunk information.
Definition: clone0clone.h:533
 
Clone_Snapshot * get_snapshot()
Get attached snapshot.
Definition: clone0clone.h:353
 
bool m_send_state_meta
Send state metadata before starting: Used for restart.
Definition: clone0clone.h:563
 
int change_state(Clone_Task *task, Clone_Desc_State *state_desc, Snapshot_State new_state, Clone_Alert_Func cbk, uint &num_wait)
Move to next snapshot state.
Definition: clone0clone.cc:1767
 
uint m_num_tasks_finished
Number of tasks finished current state.
Definition: clone0clone.h:542
 
bool is_network_error(int err)
Check if network error.
Definition: clone0clone.h:480
 
int check_state(Clone_Task *task, Snapshot_State new_state, bool exit_on_wait, int in_err, uint32_t &num_wait)
Check if state transition is over and all tasks moved to next state.
Definition: clone0clone.cc:1862
 
void init_state()
Initialize task manager for current state.
Definition: clone0clone.cc:1593
 
void reset_error()
Reset error information.
Definition: clone0clone.h:338
 
void ack_state(const Clone_Desc_State *state_desc)
Set acknowledged state.
Definition: clone0clone.cc:1615
 
Clone_Snapshot * m_clone_snapshot
Attached snapshot handle.
Definition: clone0clone.h:572
 
int handle_error_other_task(bool raise_error)
Handle any error raised by concurrent tasks.
Definition: clone0clone.cc:917
 
uint32_t get_next_incomplete_chunk(uint32_t &block_num)
Get next in complete chunk if any.
Definition: clone0clone.cc:1226
 
std::string m_err_file_name
File name related to the saved error.
Definition: clone0clone.h:569
 
bool process_inclomplete_chunk()
Check if we should process incomplete chunk next.
Definition: clone0clone.h:498
 
void reserve_task(THD *thd, uint &task_id)
Reserve free task from task manager and initialize.
Definition: clone0clone.cc:821
 
uint m_num_tasks_transit
Number of tasks in transit state.
Definition: clone0clone.h:545
 
Clone_Task m_clone_tasks[CLONE_MAX_TASKS]
Clone task array.
Definition: clone0clone.h:536
 
int m_saved_error
Save any error raised by a task.
Definition: clone0clone.h:566
 
void reinit_copy_state(const byte *loc, uint loc_len)
Reinitialize state using locator.
Definition: clone0clone.cc:1496
 
uint m_num_tasks
Current number of tasks.
Definition: clone0clone.h:539
 
void debug_wait(uint chunk_num, Clone_Task *task)
Wait during clone operation.
Definition: clone0clone.cc:701
 
ib_mutex_t * get_mutex()
Get task state mutex.
Definition: clone0clone.h:211
 
int debug_restart(Clone_Task *task, int in_err, int restart_count)
Force restart clone operation by raising network error.
Definition: clone0clone.cc:730
 
void set_error(int err, const char *file_name)
Set error number.
Definition: clone0clone.h:221
 
Snapshot_State m_ack_state
Acknowledged state from client.
Definition: clone0clone.h:551
 
void add_incomplete_chunk(Clone_Task *task)
Track any incomplete chunks handled by the task.
Definition: clone0clone.cc:1322
 
bool is_chunk_reserved(uint32_t chunk_num)
Check if chunk is already reserved.
Definition: clone0clone.h:258
 
Clone_Task * find_master_task()
Definition: clone0clone.cc:675
 
Clone_Task * get_task_by_index(uint index)
Get task by index.
Definition: clone0clone.h:287
 
bool is_file_metadata_transferred() const
Definition: clone0clone.h:391
 
bool check_ack(const Clone_Desc_State *state_desc)
Check if state ACK is needed.
Definition: clone0clone.h:417
 
void set_file_meta_transferred()
Set sub-state: all file metadata is transferred.
Definition: clone0clone.h:396
 
void debug_wait_ddl_meta()
Wait before sending DDL metadata.
Definition: clone0clone.cc:663
 
Snapshot_State m_next_state
Next state: used during state transfer.
Definition: clone0clone.h:557
 
uint m_restart_count
Number of times clone is restarted.
Definition: clone0clone.h:548
 
int add_task(THD *thd, const byte *ref_loc, uint loc_len, uint &task_id)
Add a task to task manager.
Definition: clone0clone.cc:1032
 
bool in_transit_state()
Check if in state transition.
Definition: clone0clone.h:349
 
Snapshot_State m_current_state
Current state for clone.
Definition: clone0clone.h:554
 
void init(Clone_Snapshot *snapshot)
Initialize task manager for clone handle.
Definition: clone0clone.cc:761
 
Snapshot_State get_state()
Get current clone state.
Definition: clone0clone.h:345
 
bool wait_before_add(const byte *ref_loc, uint loc_len)
Check if we need to wait before adding current task.
Definition: clone0clone.cc:994
 
ib_mutex_t m_state_mutex
Mutex synchronizing access by concurrent tasks.
Definition: clone0clone.h:530
 
void reset_chunk(Clone_Task *task)
Reset chunk information for task.
Definition: clone0clone.h:264
 
int reserve_next_chunk(Clone_Task *task, uint32_t &ret_chunk, uint32_t &ret_block)
Reserve next chunk from task manager.
Definition: clone0clone.cc:1246
 
bool debug_sync_check(uint32_t chunk_num, Clone_Task *task)
Check if needs to wait for debug sync point.
Definition: clone0clone.cc:648
 
Type
Notification type.
Definition: clone0api.h:182
 
Persist GTID along with transaction commit.
Definition: clone0repl.h:69
 
Definition: handler.h:1012
 
For each client connection we create a separate thread with THD serving as a thread/connection descri...
Definition: sql_lexer_thd.h:34
 
A utility class which, if inherited from, prevents the descendant class from being copied,...
Definition: ut0class_life_cycle.h:41
 
Clone_System_State
Clone system state.
Definition: clone0clone.h:117
 
@ CLONE_SYS_ACTIVE
Definition: clone0clone.h:119
 
@ CLONE_SYS_ABORT
Definition: clone0clone.h:120
 
@ CLONE_SYS_INACTIVE
Definition: clone0clone.h:118
 
const char CLONE_INNODB_DDL_FILE_EXTN[]
Clone file extension for temporary renamed file.
Definition: clone0clone.h:101
 
Clone_Sys * clone_sys
Clone system global.
Definition: clone0clone.cc:41
 
const int CLONE_ARR_SIZE
Clone system array size.
Definition: clone0clone.h:143
 
const char CLONE_INNODB_RECOVERY_FILE[]
Clone recovery status.
Definition: clone0clone.h:75
 
const char CLONE_INNODB_OLD_FILES[]
Clone file name for list of old files to be removed.
Definition: clone0clone.h:87
 
const Clone_Msec CLONE_DEF_SLEEP
Default sleep time while waiting: 100 ms.
Definition: clone0clone.h:108
 
Clone_Handle_State
Clone Handle State.
Definition: clone0clone.h:126
 
@ CLONE_STATE_INIT
Definition: clone0clone.h:127
 
@ CLONE_STATE_IDLE
Definition: clone0clone.h:129
 
@ CLONE_STATE_ABORT
Definition: clone0clone.h:130
 
@ CLONE_STATE_ACTIVE
Definition: clone0clone.h:128
 
const char CLONE_INNODB_RECOVERY_CRASH_POINT[]
Clone simulate recovery error file name.
Definition: clone0clone.h:58
 
std::atomic< Clone_System_State > Clone_Sys_State
Definition: clone0clone.h:123
 
const char CLONE_INNODB_REPLACED_FILES[]
Clone file name for list of files to be replaced.
Definition: clone0clone.h:83
 
const char CLONE_INNODB_ERROR_FILE[]
Clone error file name.
Definition: clone0clone.h:67
 
Clone_Task_State
Clone task state.
Definition: clone0clone.h:134
 
@ CLONE_TASK_INACTIVE
Definition: clone0clone.h:134
 
@ CLONE_TASK_ACTIVE
Definition: clone0clone.h:134
 
const char CLONE_INNODB_DDL_FILES[]
Clone file name for list of temp files renamed by ddl.
Definition: clone0clone.h:91
 
const int MAX_SNAPSHOTS
Maximum number of concurrent snapshots.
Definition: clone0clone.h:137
 
std::chrono::milliseconds Clone_Msec
Definition: clone0clone.h:103
 
std::chrono::minutes Clone_Min
Definition: clone0clone.h:105
 
#define CLONE_FILES_DIR
Directory under data directory for all clone status files.
Definition: clone0clone.h:51
 
const char CLONE_INNODB_SAVED_FILE_EXTN[]
Clone file extension for saved old files.
Definition: clone0clone.h:98
 
std::chrono::seconds Clone_Sec
Definition: clone0clone.h:104
 
const char CLONE_INNODB_IN_PROGRESS_FILE[]
Clone in progress file name.
Definition: clone0clone.h:63
 
const int SNAPSHOT_ARR_SIZE
Snapshot system array size.
Definition: clone0clone.h:146
 
const int MAX_CLONES
Maximum number of concurrent clones.
Definition: clone0clone.h:140
 
const char CLONE_INNODB_REPLACED_FILE_EXTN[]
Clone file extension for files to be replaced.
Definition: clone0clone.h:95
 
const Clone_Min CLONE_DEF_TIMEOUT
Default timeout in multiple of sleep time: 30 minutes.
Definition: clone0clone.h:114
 
const char CLONE_INNODB_NEW_FILES[]
Clone file name for list of files cloned in place.
Definition: clone0clone.h:79
 
const char CLONE_INNODB_FIXUP_FILE[]
Clone fix up file name.
Definition: clone0clone.h:71
 
const size_t CLONE_INNODB_FILE_LEN
Clone in progress file name length.
Definition: clone0clone.h:54
 
const Clone_Sec CLONE_DEF_ALERT_INTERVAL
Default alert interval in multiple of sleep time: 5 seconds.
Definition: clone0clone.h:111
 
Innodb clone descriptors.
 
const int CLONE_MAX_TASKS
Maximum number of concurrent tasks for each clone.
Definition: clone0desc.h:51
 
Snapshot_State
Snapshot state transfer during clone.
Definition: clone0desc.h:93
 
@ CLONE_SNAPSHOT_DONE
Snapshot state at end after finishing transfer.
Definition: clone0desc.h:110
 
@ CLONE_SNAPSHOT_NONE
Invalid state.
Definition: clone0desc.h:95
 
@ CLONE_SNAPSHOT_INIT
Initialize state when snapshot object is created.
Definition: clone0desc.h:98
 
const uint32_t CLONE_DESC_MAX_BASE_LEN
Maximum base length for any serialized descriptor.
Definition: clone0desc.h:45
 
std::function< int()> Clone_Alert_Func
Function to alert caller for long wait.
Definition: clone0monitor.h:43
 
GTID persistence interface.
 
Database Physical Snapshot.
 
Clone_Handle_Type
Clone handle type.
Definition: clone0snapshot.h:239
 
@ CLONE_HDL_COPY
Clone Handle for COPY.
Definition: clone0snapshot.h:241
 
Clone_handler * clone_handle
Clone handler global.
Definition: clone_handler.cc:58
 
Global error codes for the database.
 
dberr_t
Definition: db0err.h:39
 
static int is_timeout(int e)
Definition: my_thread.h:57
 
static size_t file_size
Definition: mysql_config_editor.cc:71
 
static Value err()
Create a Value object that represents an error condition.
Definition: json_binary.cc:910
 
std::string file_name(Log_file_id file_id)
Provides name of the log file with the given file id, e.g.
Definition: log0pre_8_0_30.cc:94
 
static bool timeout(bool(*wait_condition)())
Timeout function.
Definition: log0meb.cc:496
 
mutable_buffer buffer(void *p, size_t n) noexcept
Definition: buffer.h:420
 
#define OS_FILE_PREFIX
Prefix all files and directory created under data directory with special string so that it never conf...
Definition: os0file.h:68
 
static File create_file(THD *thd, char *path, sql_exchange *exchange, IO_CACHE *cache)
Definition: query_result.cc:205
 
required string type
Definition: replication_group_member_actions.proto:34
 
Ha_clone_type
Clone operation types.
Definition: handler.h:960
 
Incomplete Chunk information.
Definition: clone0desc.h:318
 
uint32_t m_min_unres_chunk
Minimum chunk number that is not reserved yet.
Definition: clone0desc.h:329
 
Chnunk_Bitmap m_reserved_chunks
Information about chunks completed.
Definition: clone0desc.h:320
 
uint32_t m_total_chunks
Chunks for current state.
Definition: clone0desc.h:326
 
Chunk_Map m_incomplete_chunks
Information about unfinished chunks.
Definition: clone0desc.h:323
 
CLONE_DESC_LOCATOR: Descriptor for a task for clone operation.
Definition: clone0desc.h:362
 
CLONE_DESC_STATE: Descriptor for current snapshot state.
Definition: clone0desc.h:439
 
Snapshot_State m_state
Current snapshot State.
Definition: clone0desc.h:444
 
Task for clone operation.
Definition: clone0clone.h:150
 
bool m_file_cache
Data files are read using OS buffer cache.
Definition: clone0clone.h:174
 
uint m_current_file_index
Current file index.
Definition: clone0clone.h:171
 
uint32_t m_data_size
Data transferred for current chunk in bytes.
Definition: clone0clone.h:197
 
int m_debug_counter
Counter to restart in different state.
Definition: clone0clone.h:187
 
Clone_Task_Meta m_task_meta
Task Meta data.
Definition: clone0clone.h:152
 
uint m_alloc_len
Serial descriptor allocated length.
Definition: clone0clone.h:161
 
pfs_os_file_t m_current_file_des
Current file descriptor.
Definition: clone0clone.h:168
 
uint m_buffer_alloc_len
Allocated buffer length.
Definition: clone0clone.h:194
 
byte * m_current_buffer
Allocated buffer.
Definition: clone0clone.h:191
 
bool m_is_master
If master task.
Definition: clone0clone.h:177
 
bool m_pinned_file
If task is currently pinning file.
Definition: clone0clone.h:165
 
bool m_has_thd
If task has associated session.
Definition: clone0clone.h:180
 
bool m_ignore_sync
Ignore debug sync point.
Definition: clone0clone.h:184
 
Clone_Task_State m_task_state
Task state.
Definition: clone0clone.h:155
 
byte * m_serial_desc
Serial descriptor byte string.
Definition: clone0clone.h:158
 
Definition: clone0snapshot.h:49
 
Common file descriptor for file IO instrumentation with PFS on windows and other platforms.
Definition: os0file.h:169
 
double seconds()
Definition: task.cc:310
 
unsigned int uint
Definition: uca9-dump.cc:75
 
Version control for database, common definitions, and include files.
 
unsigned long int ulint
Definition: univ.i:406
 
#define UNIV_PFS_IO
Definition: univ.i:141
 
#define ut_ad(EXPR)
Debug assertion.
Definition: ut0dbg.h:69
 
#define ut_a(EXPR)
Abort execution if EXPR does not evaluate to nonzero.
Definition: ut0dbg.h:57
 
#define mutex_own(M)
Checks that the current thread owns the mutex.
Definition: ut0mutex.h:165
 
#define mutex_exit(M)
Definition: ut0mutex.h:123
 
#define mutex_enter(M)
Definition: ut0mutex.h:117
 
static ORDER * clone(THD *thd, ORDER *order)
Shallow clone the list of ORDER objects using mem_root and return the cloned list.
Definition: window.cc:84