This is an automated email from the ASF dual-hosted git repository. gavinchou pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/doris.git
The following commit(s) were added to refs/heads/master by this push: new c45ac4fbcdf [feature](restore) support atomic restore (#40353) c45ac4fbcdf is described below commit c45ac4fbcdf9cac839b3a9fc99d038ef52167d47 Author: walter <w41te...@gmail.com> AuthorDate: Sun Sep 8 01:21:20 2024 +0800 [feature](restore) support atomic restore (#40353) Previously, during a restore job, if the OLAP table already existed, it would be marked with a RESTORE status, preventing any read or write operations. This PR introduces an atomic restore feature. If enabled, data will first be loaded into a temporary table during the restore, and then atomically replace the original table, ensuring that the target table's read and write operations remain unaffected during the restore process. --- be/src/olap/snapshot_manager.cpp | 36 ++- be/src/olap/snapshot_manager.h | 1 + be/src/olap/tablet_manager.cpp | 16 +- .../org/apache/doris/analysis/RestoreStmt.java | 9 + .../org/apache/doris/backup/BackupHandler.java | 6 +- .../apache/doris/backup/RestoreFileMapping.java | 18 +- .../java/org/apache/doris/backup/RestoreJob.java | 313 ++++++++++++++++++--- .../apache/doris/service/FrontendServiceImpl.java | 3 + .../java/org/apache/doris/task/SnapshotTask.java | 13 +- .../doris/backup/RestoreFileMappingTest.java | 6 +- .../org/apache/doris/backup/RestoreJobTest.java | 3 +- gensrc/thrift/AgentService.thrift | 1 + gensrc/thrift/FrontendService.thrift | 1 + .../backup_restore/test_backup_restore_atomic.out | 78 +++++ .../test_backup_restore_atomic.groovy | 209 ++++++++++++++ 15 files changed, 646 insertions(+), 67 deletions(-) diff --git a/be/src/olap/snapshot_manager.cpp b/be/src/olap/snapshot_manager.cpp index 1aa0229ee65..c7790f6f354 100644 --- a/be/src/olap/snapshot_manager.cpp +++ b/be/src/olap/snapshot_manager.cpp @@ -82,15 +82,26 @@ Status SnapshotManager::make_snapshot(const TSnapshotRequest& request, string* s return Status::Error<INVALID_ARGUMENT>("output parameter cannot be null"); } - TabletSharedPtr ref_tablet = _engine.tablet_manager()->get_tablet(request.tablet_id); + TabletSharedPtr target_tablet = _engine.tablet_manager()->get_tablet(request.tablet_id); - DBUG_EXECUTE_IF("SnapshotManager::make_snapshot.inject_failure", { ref_tablet = nullptr; }) + DBUG_EXECUTE_IF("SnapshotManager::make_snapshot.inject_failure", { target_tablet = nullptr; }) - if (ref_tablet == nullptr) { + if (target_tablet == nullptr) { return Status::Error<TABLE_NOT_FOUND>("failed to get tablet. tablet={}", request.tablet_id); } - res = _create_snapshot_files(ref_tablet, request, snapshot_path, allow_incremental_clone); + TabletSharedPtr ref_tablet = target_tablet; + if (request.__isset.ref_tablet_id) { + int64_t ref_tablet_id = request.ref_tablet_id; + ref_tablet = _engine.tablet_manager()->get_tablet(ref_tablet_id); + if (ref_tablet == nullptr) { + return Status::Error<TABLE_NOT_FOUND>("failed to get ref tablet. tablet={}", + ref_tablet_id); + } + } + + res = _create_snapshot_files(ref_tablet, target_tablet, request, snapshot_path, + allow_incremental_clone); if (!res.ok()) { LOG(WARNING) << "failed to make snapshot. res=" << res << " tablet=" << request.tablet_id; @@ -347,6 +358,7 @@ Status SnapshotManager::_link_index_and_data_files( } Status SnapshotManager::_create_snapshot_files(const TabletSharedPtr& ref_tablet, + const TabletSharedPtr& target_tablet, const TSnapshotRequest& request, string* snapshot_path, bool* allow_incremental_clone) { @@ -366,10 +378,10 @@ Status SnapshotManager::_create_snapshot_files(const TabletSharedPtr& ref_tablet timeout_s = request.timeout; } std::string snapshot_id_path; - res = _calc_snapshot_id_path(ref_tablet, timeout_s, &snapshot_id_path); + res = _calc_snapshot_id_path(target_tablet, timeout_s, &snapshot_id_path); if (!res.ok()) { - LOG(WARNING) << "failed to calc snapshot_id_path, ref tablet=" - << ref_tablet->data_dir()->path(); + LOG(WARNING) << "failed to calc snapshot_id_path, tablet=" + << target_tablet->data_dir()->path(); return res; } @@ -377,12 +389,12 @@ Status SnapshotManager::_create_snapshot_files(const TabletSharedPtr& ref_tablet // schema_full_path_desc.filepath: // /snapshot_id_path/tablet_id/schema_hash/ - auto schema_full_path = get_schema_hash_full_path(ref_tablet, snapshot_id_path); + auto schema_full_path = get_schema_hash_full_path(target_tablet, snapshot_id_path); // header_path: // /schema_full_path/tablet_id.hdr - auto header_path = _get_header_full_path(ref_tablet, schema_full_path); + auto header_path = _get_header_full_path(target_tablet, schema_full_path); // /schema_full_path/tablet_id.hdr.json - auto json_header_path = _get_json_header_full_path(ref_tablet, schema_full_path); + auto json_header_path = _get_json_header_full_path(target_tablet, schema_full_path); bool exists = true; RETURN_IF_ERROR(io::global_local_filesystem()->exists(schema_full_path, &exists)); if (exists) { @@ -564,7 +576,9 @@ Status SnapshotManager::_create_snapshot_files(const TabletSharedPtr& ref_tablet << rs->rowset_meta()->empty(); } if (!res.ok()) { - LOG(WARNING) << "fail to create hard link. [path=" << snapshot_id_path << "]"; + LOG(WARNING) << "fail to create hard link. path=" << snapshot_id_path + << " tablet=" << target_tablet->tablet_id() + << " ref tablet=" << ref_tablet->tablet_id(); break; } diff --git a/be/src/olap/snapshot_manager.h b/be/src/olap/snapshot_manager.h index df2b1b33b27..dd10f7f3550 100644 --- a/be/src/olap/snapshot_manager.h +++ b/be/src/olap/snapshot_manager.h @@ -72,6 +72,7 @@ private: const std::vector<RowsetSharedPtr>& consistent_rowsets); Status _create_snapshot_files(const TabletSharedPtr& ref_tablet, + const TabletSharedPtr& target_tablet, const TSnapshotRequest& request, std::string* snapshot_path, bool* allow_incremental_clone); diff --git a/be/src/olap/tablet_manager.cpp b/be/src/olap/tablet_manager.cpp index 6525e9185da..bc883185465 100644 --- a/be/src/olap/tablet_manager.cpp +++ b/be/src/olap/tablet_manager.cpp @@ -279,9 +279,11 @@ Status TabletManager::create_tablet(const TCreateTabletReq& request, std::vector // we need use write lock on shard-1 and then use read lock on shard-2 // if there have create rollup tablet C(assume on shard-2) from tablet D(assume on shard-1) at the same time, we will meet deadlock std::unique_lock two_tablet_lock(_two_tablet_mtx, std::defer_lock); - bool is_schema_change = request.__isset.base_tablet_id && request.base_tablet_id > 0; - bool need_two_lock = is_schema_change && ((_tablets_shards_mask & request.base_tablet_id) != - (_tablets_shards_mask & tablet_id)); + bool is_schema_change_or_atomic_restore = + request.__isset.base_tablet_id && request.base_tablet_id > 0; + bool need_two_lock = + is_schema_change_or_atomic_restore && + ((_tablets_shards_mask & request.base_tablet_id) != (_tablets_shards_mask & tablet_id)); if (need_two_lock) { SCOPED_TIMER(ADD_TIMER(profile, "GetTwoTableLock")); two_tablet_lock.lock(); @@ -310,7 +312,7 @@ Status TabletManager::create_tablet(const TCreateTabletReq& request, std::vector TabletSharedPtr base_tablet = nullptr; // If the CreateTabletReq has base_tablet_id then it is a alter-tablet request - if (is_schema_change) { + if (is_schema_change_or_atomic_restore) { // if base_tablet_id's lock diffrent with new_tablet_id, we need lock it. if (need_two_lock) { SCOPED_TIMER(ADD_TIMER(profile, "GetBaseTablet")); @@ -327,7 +329,7 @@ Status TabletManager::create_tablet(const TCreateTabletReq& request, std::vector "new_tablet_id={}, base_tablet_id={}", tablet_id, request.base_tablet_id); } - // If we are doing schema-change, we should use the same data dir + // If we are doing schema-change or atomic-restore, we should use the same data dir // TODO(lingbin): A litter trick here, the directory should be determined before // entering this method if (request.storage_medium == base_tablet->data_dir()->storage_medium()) { @@ -337,8 +339,8 @@ Status TabletManager::create_tablet(const TCreateTabletReq& request, std::vector } // set alter type to schema-change. it is useless - TabletSharedPtr tablet = _internal_create_tablet_unlocked(request, is_schema_change, - base_tablet.get(), stores, profile); + TabletSharedPtr tablet = _internal_create_tablet_unlocked( + request, is_schema_change_or_atomic_restore, base_tablet.get(), stores, profile); if (tablet == nullptr) { DorisMetrics::instance()->create_tablet_requests_failed->increment(1); return Status::Error<CE_CMD_PARAMS_ERROR>("fail to create tablet. tablet_id={}", diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/RestoreStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/RestoreStmt.java index 35120b5fa11..bc38cfe09e5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/RestoreStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/RestoreStmt.java @@ -43,6 +43,7 @@ public class RestoreStmt extends AbstractBackupStmt implements NotFallbackInPars public static final String PROP_RESERVE_DYNAMIC_PARTITION_ENABLE = "reserve_dynamic_partition_enable"; public static final String PROP_CLEAN_TABLES = "clean_tables"; public static final String PROP_CLEAN_PARTITIONS = "clean_partitions"; + public static final String PROP_ATOMIC_RESTORE = "atomic_restore"; private boolean allowLoad = false; private ReplicaAllocation replicaAlloc = ReplicaAllocation.DEFAULT_ALLOCATION; @@ -54,6 +55,7 @@ public class RestoreStmt extends AbstractBackupStmt implements NotFallbackInPars private boolean isBeingSynced = false; private boolean isCleanTables = false; private boolean isCleanPartitions = false; + private boolean isAtomicRestore = false; private byte[] meta = null; private byte[] jobInfo = null; @@ -121,6 +123,10 @@ public class RestoreStmt extends AbstractBackupStmt implements NotFallbackInPars return isCleanPartitions; } + public boolean isAtomicRestore() { + return isAtomicRestore; + } + @Override public void analyze(Analyzer analyzer) throws UserException { if (repoName.equals(Repository.KEEP_ON_LOCAL_REPO_NAME)) { @@ -203,6 +209,9 @@ public class RestoreStmt extends AbstractBackupStmt implements NotFallbackInPars // is clean partitions isCleanPartitions = eatBooleanProperty(copiedProperties, PROP_CLEAN_PARTITIONS, isCleanPartitions); + // is atomic restore + isAtomicRestore = eatBooleanProperty(copiedProperties, PROP_ATOMIC_RESTORE, isAtomicRestore); + if (!copiedProperties.isEmpty()) { ErrorReport.reportAnalysisException(ErrorCode.ERR_COMMON_ERROR, "Unknown restore job properties: " + copiedProperties.keySet()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/backup/BackupHandler.java b/fe/fe-core/src/main/java/org/apache/doris/backup/BackupHandler.java index a2cb84e7b8a..25cc08f93f7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/backup/BackupHandler.java +++ b/fe/fe-core/src/main/java/org/apache/doris/backup/BackupHandler.java @@ -531,12 +531,14 @@ public class BackupHandler extends MasterDaemon implements Writable { db.getId(), db.getFullName(), jobInfo, stmt.allowLoad(), stmt.getReplicaAlloc(), stmt.getTimeoutMs(), metaVersion, stmt.reserveReplica(), stmt.reserveDynamicPartitionEnable(), stmt.isBeingSynced(), - stmt.isCleanTables(), stmt.isCleanPartitions(), env, Repository.KEEP_ON_LOCAL_REPO_ID, backupMeta); + stmt.isCleanTables(), stmt.isCleanPartitions(), stmt.isAtomicRestore(), + env, Repository.KEEP_ON_LOCAL_REPO_ID, backupMeta); } else { restoreJob = new RestoreJob(stmt.getLabel(), stmt.getBackupTimestamp(), db.getId(), db.getFullName(), jobInfo, stmt.allowLoad(), stmt.getReplicaAlloc(), stmt.getTimeoutMs(), stmt.getMetaVersion(), stmt.reserveReplica(), stmt.reserveDynamicPartitionEnable(), - stmt.isBeingSynced(), stmt.isCleanTables(), stmt.isCleanPartitions(), env, repository.getId()); + stmt.isBeingSynced(), stmt.isCleanTables(), stmt.isCleanPartitions(), stmt.isAtomicRestore(), + env, repository.getId()); } env.getEditLog().logRestoreJob(restoreJob); diff --git a/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreFileMapping.java b/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreFileMapping.java index 4bb791d84d7..78314602d5c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreFileMapping.java +++ b/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreFileMapping.java @@ -45,7 +45,7 @@ public class RestoreFileMapping implements Writable { } public IdChain(Long... ids) { - Preconditions.checkState(ids.length == 5); + Preconditions.checkState(ids.length == 6); chain = ids; } @@ -69,6 +69,14 @@ public class RestoreFileMapping implements Writable { return chain[4]; } + public boolean hasRefTabletId() { + return chain.length >= 6 && chain[5] != -1L; + } + + public long getRefTabletId() { + return chain[5]; + } + @Override public String toString() { StringBuilder sb = new StringBuilder(); @@ -84,8 +92,12 @@ public class RestoreFileMapping implements Writable { return false; } + if (((IdChain) obj).chain.length != chain.length) { + return false; + } + IdChain other = (IdChain) obj; - for (int i = 0; i < 5; i++) { + for (int i = 0; i < chain.length; i++) { // DO NOT use ==, Long_1 != Long_2 if (!chain[i].equals(other.chain[i])) { return false; @@ -98,7 +110,7 @@ public class RestoreFileMapping implements Writable { @Override public int hashCode() { int code = chain[0].hashCode(); - for (int i = 1; i < 5; i++) { + for (int i = 1; i < chain.length; i++) { code ^= chain[i].hashCode(); } return code; diff --git a/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java b/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java index 44ccdd44f74..b499a9b525e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java @@ -118,6 +118,8 @@ public class RestoreJob extends AbstractJob implements GsonPostProcessable { private static final String PROP_IS_BEING_SYNCED = PropertyAnalyzer.PROPERTIES_IS_BEING_SYNCED; private static final String PROP_CLEAN_TABLES = RestoreStmt.PROP_CLEAN_TABLES; private static final String PROP_CLEAN_PARTITIONS = RestoreStmt.PROP_CLEAN_PARTITIONS; + private static final String PROP_ATOMIC_RESTORE = RestoreStmt.PROP_ATOMIC_RESTORE; + private static final String ATOMIC_RESTORE_TABLE_PREFIX = "__doris_atomic_restore_prefix__"; private static final Logger LOG = LogManager.getLogger(RestoreJob.class); @@ -201,6 +203,8 @@ public class RestoreJob extends AbstractJob implements GsonPostProcessable { private boolean isCleanTables = false; // Whether to delete existing partitions that are not involved in the restore. private boolean isCleanPartitions = false; + // Whether to restore the data into a temp table, and then replace the origin one. + private boolean isAtomicRestore = false; // restore properties @SerializedName("prop") @@ -213,7 +217,7 @@ public class RestoreJob extends AbstractJob implements GsonPostProcessable { public RestoreJob(String label, String backupTs, long dbId, String dbName, BackupJobInfo jobInfo, boolean allowLoad, ReplicaAllocation replicaAlloc, long timeoutMs, int metaVersion, boolean reserveReplica, boolean reserveDynamicPartitionEnable, boolean isBeingSynced, boolean isCleanTables, - boolean isCleanPartitions, Env env, long repoId) { + boolean isCleanPartitions, boolean isAtomicRestore, Env env, long repoId) { super(JobType.RESTORE, label, dbId, dbName, timeoutMs, env, repoId); this.backupTimestamp = backupTs; this.jobInfo = jobInfo; @@ -230,19 +234,22 @@ public class RestoreJob extends AbstractJob implements GsonPostProcessable { this.isBeingSynced = isBeingSynced; this.isCleanTables = isCleanTables; this.isCleanPartitions = isCleanPartitions; + this.isAtomicRestore = isAtomicRestore; properties.put(PROP_RESERVE_REPLICA, String.valueOf(reserveReplica)); properties.put(PROP_RESERVE_DYNAMIC_PARTITION_ENABLE, String.valueOf(reserveDynamicPartitionEnable)); properties.put(PROP_IS_BEING_SYNCED, String.valueOf(isBeingSynced)); properties.put(PROP_CLEAN_TABLES, String.valueOf(isCleanTables)); properties.put(PROP_CLEAN_PARTITIONS, String.valueOf(isCleanPartitions)); + properties.put(PROP_ATOMIC_RESTORE, String.valueOf(isAtomicRestore)); } public RestoreJob(String label, String backupTs, long dbId, String dbName, BackupJobInfo jobInfo, boolean allowLoad, ReplicaAllocation replicaAlloc, long timeoutMs, int metaVersion, boolean reserveReplica, boolean reserveDynamicPartitionEnable, boolean isBeingSynced, boolean isCleanTables, - boolean isCleanPartitions, Env env, long repoId, BackupMeta backupMeta) { + boolean isCleanPartitions, boolean isAtomicRestore, Env env, long repoId, BackupMeta backupMeta) { this(label, backupTs, dbId, dbName, jobInfo, allowLoad, replicaAlloc, timeoutMs, metaVersion, reserveReplica, - reserveDynamicPartitionEnable, isBeingSynced, isCleanTables, isCleanPartitions, env, repoId); + reserveDynamicPartitionEnable, isBeingSynced, isCleanTables, isCleanPartitions, isAtomicRestore, env, + repoId); this.backupMeta = backupMeta; } @@ -538,8 +545,10 @@ public class RestoreJob extends AbstractJob implements GsonPostProcessable { } Preconditions.checkNotNull(backupMeta); - // Set all restored tbls' state to RESTORE - // Table's origin state must be NORMAL and does not have unfinished load job. + // Check the olap table state. + // + // If isAtomicRestore is not set, set all restored tbls' state to RESTORE, + // the table's origin state must be NORMAL and does not have unfinished load job. for (String tableName : jobInfo.backupOlapTableObjects.keySet()) { Table tbl = db.getTableNullable(jobInfo.getAliasByOriginNameIfSet(tableName)); if (tbl == null) { @@ -567,6 +576,11 @@ public class RestoreJob extends AbstractJob implements GsonPostProcessable { return; } + if (isAtomicRestore) { + // We will create new OlapTable in atomic restore, so does not set the RESTORE state. + continue; + } + for (Partition partition : olapTbl.getPartitions()) { if (!env.getLoadInstance().checkPartitionLoadFinished(partition.getId(), null)) { status = new Status(ErrCode.COMMON_ERROR, @@ -628,6 +642,9 @@ public class RestoreJob extends AbstractJob implements GsonPostProcessable { } } + // the new tablets -> { local tablet, schema hash }, used in atomic restore. + Map<Long, Pair<Long, Integer>> tabletBases = null; + // Check and prepare meta objects. AgentBatchTask batchTask = new AgentBatchTask(); db.readLock(); @@ -638,14 +655,15 @@ public class RestoreJob extends AbstractJob implements GsonPostProcessable { Table remoteTbl = backupMeta.getTable(tableName); Preconditions.checkNotNull(remoteTbl); Table localTbl = db.getTableNullable(jobInfo.getAliasByOriginNameIfSet(tableName)); + if (localTbl != null && localTbl.getType() != TableType.OLAP) { + // table already exist, but is not OLAP + status = new Status(ErrCode.COMMON_ERROR, + "The type of local table should be same as type of remote table: " + + remoteTbl.getName()); + return; + } + if (localTbl != null) { - // table already exist, check schema - if (localTbl.getType() != TableType.OLAP) { - status = new Status(ErrCode.COMMON_ERROR, - "The type of local table should be same as type of remote table: " - + remoteTbl.getName()); - return; - } OlapTable localOlapTbl = (OlapTable) localTbl; OlapTable remoteOlapTbl = (OlapTable) remoteTbl; @@ -691,28 +709,26 @@ public class RestoreJob extends AbstractJob implements GsonPostProcessable { PartitionItem localItem = localPartInfo.getItem(localPartition.getId()); PartitionItem remoteItem = remoteOlapTbl .getPartitionInfo().getItem(backupPartInfo.id); - if (localItem.equals(remoteItem)) { - // Same partition, same range - if (genFileMappingWhenBackupReplicasEqual(localPartInfo, localPartition, - localTbl, backupPartInfo, partitionName, tblInfo, remoteReplicaAlloc)) { - return; - } - } else { + if (!localItem.equals(remoteItem)) { // Same partition name, different range status = new Status(ErrCode.COMMON_ERROR, "Partition " + partitionName + " in table " + localTbl.getName() + " has different partition item with partition in repository"); return; } - } else { - // If this is a single partitioned table. - if (genFileMappingWhenBackupReplicasEqual(localPartInfo, localPartition, localTbl, - backupPartInfo, partitionName, tblInfo, remoteReplicaAlloc)) { - return; - } } - } else { + if (isAtomicRestore) { + // skip gen file mapping for atomic restore. + continue; + } + + // Same partition, same range or a single partitioned table. + if (genFileMappingWhenBackupReplicasEqual(localPartInfo, localPartition, + localTbl, backupPartInfo, partitionName, tblInfo, remoteReplicaAlloc)) { + return; + } + } else if (!isAtomicRestore) { // partitions does not exist PartitionInfo localPartitionInfo = localOlapTbl.getPartitionInfo(); if (localPartitionInfo.getType() == PartitionType.RANGE @@ -752,8 +768,10 @@ public class RestoreJob extends AbstractJob implements GsonPostProcessable { } finally { localOlapTbl.readUnlock(); } - } else { - // Table does not exist + } + + // Table does not exist or atomic restore + if (localTbl == null || isAtomicRestore) { OlapTable remoteOlapTbl = (OlapTable) remoteTbl; // Retain only expected restore partitions in this table; Set<String> allPartNames = remoteOlapTbl.getPartitionNames(); @@ -781,6 +799,15 @@ public class RestoreJob extends AbstractJob implements GsonPostProcessable { // DO NOT set remote table's new name here, cause we will still need the origin name later // remoteOlapTbl.setName(jobInfo.getAliasByOriginNameIfSet(tblInfo.name)); remoteOlapTbl.setState(allowLoad ? OlapTableState.RESTORE_WITH_LOAD : OlapTableState.RESTORE); + + if (isAtomicRestore && localTbl != null) { + // bind the backends and base tablets from local tbl. + tabletBases = bindLocalAndRemoteOlapTableReplicas((OlapTable) localTbl, remoteOlapTbl); + if (!status.ok()) { + return; + } + } + if (LOG.isDebugEnabled()) { LOG.debug("put remote table {} to restoredTbls", remoteOlapTbl.getName()); } @@ -837,6 +864,9 @@ public class RestoreJob extends AbstractJob implements GsonPostProcessable { // for now, nothing is modified in catalog // generate create replica tasks for all restored partitions + if (isAtomicRestore && !restoredPartitions.isEmpty()) { + throw new RuntimeException("atomic restore is set, but the restored partitions is not empty"); + } for (Pair<String, Partition> entry : restoredPartitions) { OlapTable localTbl = (OlapTable) db.getTableNullable(entry.first); Preconditions.checkNotNull(localTbl, localTbl.getName()); @@ -860,7 +890,8 @@ public class RestoreJob extends AbstractJob implements GsonPostProcessable { BackupOlapTableInfo backupOlapTableInfo = jobInfo.getOlapTableInfo(restoreOlapTable.getName()); genFileMapping(restoreOlapTable, restorePart, backupOlapTableInfo.id, backupOlapTableInfo.getPartInfo(restorePart.getName()), - !allowLoad /* if allow load, do not overwrite when commit */); + !allowLoad /* if allow load, do not overwrite when commit */, + tabletBases); } } // set restored table's new name after all 'genFileMapping' @@ -868,6 +899,9 @@ public class RestoreJob extends AbstractJob implements GsonPostProcessable { if (Env.isStoredTableNamesLowerCase()) { tableName = tableName.toLowerCase(); } + if (isAtomicRestore) { + tableName = tableAliasWithAtomicRestore(tableName); + } restoreTbl.setName(tableName); } @@ -991,6 +1025,77 @@ public class RestoreJob extends AbstractJob implements GsonPostProcessable { // No log here, PENDING state restore job will redo this method } + private Map<Long, Pair<Long, Integer>> bindLocalAndRemoteOlapTableReplicas( + OlapTable localOlapTbl, OlapTable remoteOlapTbl) { + Map<Long, Pair<Long, Integer>> tabletBases = new HashMap<>(); + + localOlapTbl.readLock(); + try { + for (Partition partition : remoteOlapTbl.getPartitions()) { + Partition localPartition = localOlapTbl.getPartition(partition.getName()); + if (localPartition == null) { + continue; + } + for (MaterializedIndex index : partition.getMaterializedIndices(IndexExtState.VISIBLE)) { + String indexName = remoteOlapTbl.getIndexNameById(index.getId()); + Long localIndexId = localOlapTbl.getIndexIdByName(indexName); + MaterializedIndex localIndex = localIndexId == null ? null : localPartition.getIndex(localIndexId); + if (localIndex == null) { + continue; + } + int schemaHash = localOlapTbl.getSchemaHashByIndexId(localIndexId); + if (schemaHash == -1) { + status = new Status(ErrCode.COMMON_ERROR, String.format( + "schema hash of local index %d is not found, remote table=%d, remote index=%d, " + + "local table=%d, local index=%d", localIndexId, remoteOlapTbl.getId(), index.getId(), + localOlapTbl.getId(), localIndexId)); + return null; + } + + List<Tablet> localTablets = localIndex.getTablets(); + List<Tablet> remoteTablets = index.getTablets(); + if (localTablets.size() != remoteTablets.size()) { + status = new Status(ErrCode.COMMON_ERROR, String.format( + "the size of local tablet %s is not equals to the remote %s, " + + "is_atomic_restore=true, remote table=%d, remote index=%d, " + + "local table=%d, local index=%d", localTablets.size(), remoteTablets.size(), + remoteOlapTbl.getId(), index.getId(), localOlapTbl.getId(), localIndexId)); + return null; + } + for (int i = 0; i < remoteTablets.size(); i++) { + Tablet localTablet = localTablets.get(i); + Tablet remoteTablet = remoteTablets.get(i); + List<Replica> localReplicas = localTablet.getReplicas(); + List<Replica> remoteReplicas = remoteTablet.getReplicas(); + if (localReplicas.size() != remoteReplicas.size()) { + status = new Status(ErrCode.COMMON_ERROR, String.format( + "the size of local replicas %s is not equals to the remote %s, " + + "is_atomic_restore=true, remote table=%d, remote index=%d, " + + "local table=%d, local index=%d, local replicas=%d, remote replicas=%d", + localTablets.size(), remoteTablets.size(), remoteOlapTbl.getId(), + index.getId(), localOlapTbl.getId(), localIndexId, localReplicas.size(), + remoteReplicas.size())); + return null; + } + for (int j = 0; j < remoteReplicas.size(); j++) { + long backendId = localReplicas.get(j).getBackendId(); + remoteReplicas.get(j).setBackendId(backendId); + if (LOG.isDebugEnabled()) { + LOG.debug("bind local replica {} and remote replica {} with same backend {}, table={}", + localReplicas.get(j).getId(), remoteReplicas.get(j).getId(), backendId, + localOlapTbl.getName()); + } + } + tabletBases.put(remoteTablet.getId(), Pair.of(localTablet.getId(), schemaHash)); + } + } + } + } finally { + localOlapTbl.readUnlock(); + } + return tabletBases; + } + private void prepareAndSendSnapshotTaskForOlapTable(Database db) { LOG.info("begin to make snapshot. {} when restore content is ALL", this); // begin to make snapshots for all replicas @@ -1002,7 +1107,8 @@ public class RestoreJob extends AbstractJob implements GsonPostProcessable { AgentBatchTask batchTask = new AgentBatchTask(); db.readLock(); try { - for (IdChain idChain : fileMapping.getMapping().keySet()) { + for (Map.Entry<IdChain, IdChain> entry : fileMapping.getMapping().entrySet()) { + IdChain idChain = entry.getKey(); OlapTable tbl = (OlapTable) db.getTableNullable(idChain.getTblId()); tbl.readLock(); try { @@ -1011,9 +1117,15 @@ public class RestoreJob extends AbstractJob implements GsonPostProcessable { Tablet tablet = index.getTablet(idChain.getTabletId()); Replica replica = tablet.getReplicaById(idChain.getReplicaId()); long signature = env.getNextId(); + boolean isRestoreTask = true; + // We don't care the visible version in restore job, the end version is used. + long visibleVersion = -1L; SnapshotTask task = new SnapshotTask(null, replica.getBackendId(), signature, jobId, db.getId(), - tbl.getId(), part.getId(), index.getId(), tablet.getId(), part.getVisibleVersion(), - tbl.getSchemaHashByIndexId(index.getId()), timeoutMs, true /* is restore task*/); + tbl.getId(), part.getId(), index.getId(), tablet.getId(), visibleVersion, + tbl.getSchemaHashByIndexId(index.getId()), timeoutMs, isRestoreTask); + if (entry.getValue().hasRefTabletId()) { + task.setRefTabletId(entry.getValue().getRefTabletId()); + } batchTask.addTask(task); unfinishedSignatureToId.put(signature, tablet.getId()); bePathsMap.put(replica.getBackendId(), replica.getPathHash()); @@ -1101,6 +1213,11 @@ public class RestoreJob extends AbstractJob implements GsonPostProcessable { } private void createReplicas(Database db, AgentBatchTask batchTask, OlapTable localTbl, Partition restorePart) { + createReplicas(db, batchTask, localTbl, restorePart, null); + } + + private void createReplicas(Database db, AgentBatchTask batchTask, OlapTable localTbl, Partition restorePart, + Map<Long, Pair<Long, Integer>> tabletBases) { Set<String> bfColumns = localTbl.getCopiedBfColumns(); double bfFpp = localTbl.getBfFpp(); @@ -1153,6 +1270,11 @@ public class RestoreJob extends AbstractJob implements GsonPostProcessable { localTbl.variantEnableFlattenNested()); task.setInvertedIndexFileStorageFormat(localTbl.getInvertedIndexFileStorageFormat()); task.setInRestoreMode(true); + if (tabletBases != null && tabletBases.containsKey(restoreTablet.getId())) { + // ensure this replica is bound to the same backend disk as the origin table's replica. + Pair<Long, Integer> baseTablet = tabletBases.get(restoreTablet.getId()); + task.setBaseTablet(baseTablet.first, baseTablet.second); + } batchTask.addTask(task); } } @@ -1235,6 +1357,11 @@ public class RestoreJob extends AbstractJob implements GsonPostProcessable { // files in repo to files in local private void genFileMapping(OlapTable localTbl, Partition localPartition, Long remoteTblId, BackupPartitionInfo backupPartInfo, boolean overwrite) { + genFileMapping(localTbl, localPartition, remoteTblId, backupPartInfo, overwrite, null); + } + + private void genFileMapping(OlapTable localTbl, Partition localPartition, Long remoteTblId, + BackupPartitionInfo backupPartInfo, boolean overwrite, Map<Long, Pair<Long, Integer>> tabletBases) { for (MaterializedIndex localIdx : localPartition.getMaterializedIndices(IndexExtState.VISIBLE)) { if (LOG.isDebugEnabled()) { LOG.debug("get index id: {}, index name: {}", localIdx.getId(), @@ -1249,10 +1376,17 @@ public class RestoreJob extends AbstractJob implements GsonPostProcessable { LOG.debug("get tablet mapping: {} to {}, index {}", backupTabletInfo.id, localTablet.getId(), i); } for (Replica localReplica : localTablet.getReplicas()) { - IdChain src = new IdChain(remoteTblId, backupPartInfo.id, backupIdxInfo.id, backupTabletInfo.id, - -1L /* no replica id */); - IdChain dest = new IdChain(localTbl.getId(), localPartition.getId(), - localIdx.getId(), localTablet.getId(), localReplica.getId()); + long refTabletId = -1L; + if (tabletBases != null && tabletBases.containsKey(localTablet.getId())) { + refTabletId = tabletBases.get(localTablet.getId()).first; + } + + long noReplicaId = -1L; + long noRefTabletId = -1L; + IdChain src = new IdChain(remoteTblId, backupPartInfo.id, backupIdxInfo.id, + backupTabletInfo.id, noReplicaId, refTabletId); + IdChain dest = new IdChain(localTbl.getId(), localPartition.getId(), localIdx.getId(), + localTablet.getId(), localReplica.getId(), noRefTabletId); fileMapping.putMapping(dest, src, overwrite); } } @@ -1292,6 +1426,11 @@ public class RestoreJob extends AbstractJob implements GsonPostProcessable { // replay set all existing tables's state to RESTORE for (String tableName : jobInfo.backupOlapTableObjects.keySet()) { + if (isAtomicRestore) { + // Atomic restore will creates new replica of the OlapTable. + continue; + } + Table tbl = db.getTableNullable(jobInfo.getAliasByOriginNameIfSet(tableName)); if (tbl == null) { continue; @@ -1419,7 +1558,7 @@ public class RestoreJob extends AbstractJob implements GsonPostProcessable { } private void downloadRemoteSnapshots() { - // Categorize snapshot onfos by db id. + // Categorize snapshot infos by db id. ArrayListMultimap<Long, SnapshotInfo> dbToSnapshotInfos = ArrayListMultimap.create(); for (SnapshotInfo info : snapshotInfos.values()) { dbToSnapshotInfos.put(info.getDbId(), info); @@ -1519,8 +1658,9 @@ public class RestoreJob extends AbstractJob implements GsonPostProcessable { return; } + long refTabletId = -1L; // no ref tablet id IdChain catalogIds = new IdChain(tbl.getId(), part.getId(), idx.getId(), - info.getTabletId(), replica.getId()); + info.getTabletId(), replica.getId(), refTabletId); IdChain repoIds = fileMapping.get(catalogIds); if (repoIds == null) { status = new Status(ErrCode.NOT_FOUND, @@ -1667,8 +1807,9 @@ public class RestoreJob extends AbstractJob implements GsonPostProcessable { return; } + long refTabletId = -1L; // no ref tablet id IdChain catalogIds = new IdChain(tbl.getId(), part.getId(), idx.getId(), - info.getTabletId(), replica.getId()); + info.getTabletId(), replica.getId(), refTabletId); IdChain repoIds = fileMapping.get(catalogIds); if (repoIds == null) { status = new Status(ErrCode.NOT_FOUND, @@ -1810,6 +1951,14 @@ public class RestoreJob extends AbstractJob implements GsonPostProcessable { return new Status(ErrCode.NOT_FOUND, "database " + dbId + " does not exist"); } + // replace the origin tables in atomic. + if (isAtomicRestore) { + Status st = atomicReplaceOlapTables(db, isReplay); + if (!st.ok()) { + return st; + } + } + // set all restored partition version and version hash // set all tables' state to NORMAL setTableStateToNormalAndUpdateProperties(db, true, isReplay); @@ -2135,6 +2284,86 @@ public class RestoreJob extends AbstractJob implements GsonPostProcessable { LOG.info("finished to cancel restore job. is replay: {}. {}", isReplay, this); } + private Status atomicReplaceOlapTables(Database db, boolean isReplay) { + assert isAtomicRestore; + for (String tableName : jobInfo.backupOlapTableObjects.keySet()) { + String originName = jobInfo.getAliasByOriginNameIfSet(tableName); + if (Env.isStoredTableNamesLowerCase()) { + originName = originName.toLowerCase(); + } + String aliasName = tableAliasWithAtomicRestore(originName); + + if (!db.writeLockIfExist()) { + return Status.OK; + } + try { + Table newTbl = db.getTableNullable(aliasName); + if (newTbl == null) { + LOG.warn("replace table from {} to {}, but the temp table is not found", aliasName, originName); + return new Status(ErrCode.COMMON_ERROR, "replace table failed, the temp table " + + aliasName + " is not found"); + } + if (newTbl.getType() != TableType.OLAP) { + LOG.warn("replace table from {} to {}, but the temp table is not OLAP, it type is {}", + aliasName, originName, newTbl.getType()); + return new Status(ErrCode.COMMON_ERROR, "replace table failed, the temp table " + aliasName + + " is not OLAP table, it is " + newTbl.getType()); + } + + OlapTable originOlapTbl = null; + Table originTbl = db.getTableNullable(originName); + if (originTbl != null) { + if (originTbl.getType() != TableType.OLAP) { + LOG.warn("replace table from {} to {}, but the origin table is not OLAP, it type is {}", + aliasName, originName, originTbl.getType()); + return new Status(ErrCode.COMMON_ERROR, "replace table failed, the origin table " + + originName + " is not OLAP table, it is " + originTbl.getType()); + } + originOlapTbl = (OlapTable) originTbl; // save the origin olap table, then drop it. + } + + // replace the table. + OlapTable newOlapTbl = (OlapTable) newTbl; + newOlapTbl.writeLock(); + try { + // rename new table name to origin table name and add the new table to database. + db.unregisterTable(aliasName); + newOlapTbl.checkAndSetName(originName, false); + db.unregisterTable(originName); + db.registerTable(newOlapTbl); + + // set the olap table state to normal immediately for querying + newOlapTbl.setState(OlapTableState.NORMAL); + LOG.info("atomic restore replace table {} name to {}, and set state to normal, origin table={}", + newOlapTbl.getId(), originName, originOlapTbl == null ? -1L : originOlapTbl.getId()); + } catch (DdlException e) { + LOG.warn("atomic restore replace table {} name from {} to {}", + newOlapTbl.getId(), aliasName, originName, e); + return new Status(ErrCode.COMMON_ERROR, "replace table from " + aliasName + " to " + originName + + " failed, reason=" + e.getMessage()); + } finally { + newOlapTbl.writeUnlock(); + } + + if (originOlapTbl != null) { + // The origin table is not used anymore, need to drop all its tablets. + originOlapTbl.writeLock(); + try { + LOG.info("drop the origin olap table {} by atomic restore. table={}", + originOlapTbl.getName(), originOlapTbl.getId()); + Env.getCurrentEnv().onEraseOlapTable(originOlapTbl, isReplay); + } finally { + originOlapTbl.writeUnlock(); + } + } + } finally { + db.writeUnlock(); + } + } + + return Status.OK; + } + private void setTableStateToNormalAndUpdateProperties(Database db, boolean committed, boolean isReplay) { for (String tableName : jobInfo.backupOlapTableObjects.keySet()) { Table tbl = db.getTableNullable(jobInfo.getAliasByOriginNameIfSet(tableName)); @@ -2279,6 +2508,7 @@ public class RestoreJob extends AbstractJob implements GsonPostProcessable { isBeingSynced = Boolean.parseBoolean(properties.get(PROP_IS_BEING_SYNCED)); isCleanTables = Boolean.parseBoolean(properties.get(PROP_CLEAN_TABLES)); isCleanPartitions = Boolean.parseBoolean(properties.get(PROP_CLEAN_PARTITIONS)); + isAtomicRestore = Boolean.parseBoolean(properties.get(PROP_ATOMIC_RESTORE)); } @Override @@ -2288,6 +2518,7 @@ public class RestoreJob extends AbstractJob implements GsonPostProcessable { isBeingSynced = Boolean.parseBoolean(properties.get(PROP_IS_BEING_SYNCED)); isCleanTables = Boolean.parseBoolean(properties.get(PROP_CLEAN_TABLES)); isCleanPartitions = Boolean.parseBoolean(properties.get(PROP_CLEAN_PARTITIONS)); + isAtomicRestore = Boolean.parseBoolean(properties.get(PROP_ATOMIC_RESTORE)); } @Override @@ -2297,4 +2528,8 @@ public class RestoreJob extends AbstractJob implements GsonPostProcessable { sb.append(", state: ").append(state.name()); return sb.toString(); } + + private String tableAliasWithAtomicRestore(String tableName) { + return ATOMIC_RESTORE_TABLE_PREFIX + tableName; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java b/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java index 77f79bc5768..6c0077ee311 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java +++ b/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java @@ -3007,6 +3007,9 @@ public class FrontendServiceImpl implements FrontendService.Iface { if (request.isCleanTables()) { properties.put(RestoreStmt.PROP_CLEAN_TABLES, "true"); } + if (request.isAtomicRestore()) { + properties.put(RestoreStmt.PROP_ATOMIC_RESTORE, "true"); + } AbstractBackupTableRefClause restoreTableRefClause = null; if (request.isSetTableRefs()) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/task/SnapshotTask.java b/fe/fe-core/src/main/java/org/apache/doris/task/SnapshotTask.java index 71b3570f288..81177305683 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/task/SnapshotTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/task/SnapshotTask.java @@ -29,6 +29,7 @@ public class SnapshotTask extends AgentTask { private int schemaHash; private long timeoutMs; private boolean isRestoreTask; + private Long refTabletId; // Set to true if this task for AdminCopyTablet. // Otherwise, it is for Backup/Restore operation. @@ -98,13 +99,23 @@ public class SnapshotTask extends AgentTask { return resultSnapshotPath; } + public void setRefTabletId(long refTabletId) { + assert refTabletId > 0; + this.refTabletId = refTabletId; + } + public TSnapshotRequest toThrift() { TSnapshotRequest request = new TSnapshotRequest(tabletId, schemaHash); - request.setVersion(version); request.setListFiles(true); request.setPreferredSnapshotVersion(TypesConstants.TPREFER_SNAPSHOT_REQ_VERSION); request.setTimeout(timeoutMs / 1000); request.setIsCopyTabletTask(isCopyTabletTask); + if (refTabletId != null) { + request.setRefTabletId(refTabletId); + } + if (version > 0L) { + request.setVersion(version); + } return request; } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/backup/RestoreFileMappingTest.java b/fe/fe-core/src/test/java/org/apache/doris/backup/RestoreFileMappingTest.java index d37a63f6d14..85de627fa44 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/backup/RestoreFileMappingTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/backup/RestoreFileMappingTest.java @@ -31,14 +31,14 @@ public class RestoreFileMappingTest { @Before public void setUp() { - src = new IdChain(10005L, 10006L, 10005L, 10007L, 10008L); - dest = new IdChain(10004L, 10003L, 10004L, 10007L, -1L); + src = new IdChain(10005L, 10006L, 10005L, 10007L, 10008L, -1L); + dest = new IdChain(10004L, 10003L, 10004L, 10007L, -1L, -1L); fileMapping.putMapping(src, dest, true); } @Test public void test() { - IdChain key = new IdChain(10005L, 10006L, 10005L, 10007L, 10008L); + IdChain key = new IdChain(10005L, 10006L, 10005L, 10007L, 10008L, -1L); Assert.assertEquals(key, src); Assert.assertEquals(src, key); IdChain val = fileMapping.get(key); diff --git a/fe/fe-core/src/test/java/org/apache/doris/backup/RestoreJobTest.java b/fe/fe-core/src/test/java/org/apache/doris/backup/RestoreJobTest.java index 43eb00e3807..dadfdb632e3 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/backup/RestoreJobTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/backup/RestoreJobTest.java @@ -256,7 +256,8 @@ public class RestoreJobTest { db.unregisterTable(expectedRestoreTbl.getName()); job = new RestoreJob(label, "2018-01-01 01:01:01", db.getId(), db.getFullName(), jobInfo, false, - new ReplicaAllocation((short) 3), 100000, -1, false, false, false, false, false, env, repo.getId()); + new ReplicaAllocation((short) 3), 100000, -1, false, false, false, false, false, false, + env, repo.getId()); List<Table> tbls = Lists.newArrayList(); List<Resource> resources = Lists.newArrayList(); diff --git a/gensrc/thrift/AgentService.thrift b/gensrc/thrift/AgentService.thrift index 4e9ecdcce60..f02b8c0f886 100644 --- a/gensrc/thrift/AgentService.thrift +++ b/gensrc/thrift/AgentService.thrift @@ -389,6 +389,7 @@ struct TSnapshotRequest { 11: optional Types.TVersion start_version 12: optional Types.TVersion end_version 13: optional bool is_copy_binlog + 14: optional Types.TTabletId ref_tablet_id } struct TReleaseSnapshotRequest { diff --git a/gensrc/thrift/FrontendService.thrift b/gensrc/thrift/FrontendService.thrift index 4b3d90ca762..9077dbd3cec 100644 --- a/gensrc/thrift/FrontendService.thrift +++ b/gensrc/thrift/FrontendService.thrift @@ -1260,6 +1260,7 @@ struct TRestoreSnapshotRequest { 12: optional binary job_info 13: optional bool clean_tables 14: optional bool clean_partitions + 15: optional bool atomic_restore } struct TRestoreSnapshotResult { diff --git a/regression-test/data/backup_restore/test_backup_restore_atomic.out b/regression-test/data/backup_restore/test_backup_restore_atomic.out new file mode 100644 index 00000000000..bee7a4da44f --- /dev/null +++ b/regression-test/data/backup_restore/test_backup_restore_atomic.out @@ -0,0 +1,78 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !sql -- +10 10 +20 20 +30 30 +40 40 +50 50 +60 60 +70 70 +80 80 +90 90 +100 100 + +-- !sql -- +10 10 +20 20 + +-- !sql -- +10 10 +20 20 +30 30 +40 40 +50 50 +60 60 +70 70 +80 80 +90 90 +100 100 + +-- !sql -- +10 10 +20 20 +30 30 +40 40 +50 50 +60 60 +70 70 +80 80 +90 90 +100 100 + +-- !sql -- +10 10 +20 20 +30 30 +40 40 +50 50 +60 60 +70 70 +80 80 +90 90 +100 100 + +-- !sql -- +10 10 +20 20 +30 30 +40 40 +50 50 +60 60 +70 70 +80 80 +90 90 +100 100 + +-- !sql -- +10 20 +20 40 +30 60 +40 80 +50 100 +60 120 +70 140 +80 160 +90 180 +100 200 +200 200 + diff --git a/regression-test/suites/backup_restore/test_backup_restore_atomic.groovy b/regression-test/suites/backup_restore/test_backup_restore_atomic.groovy new file mode 100644 index 00000000000..4b87340fb35 --- /dev/null +++ b/regression-test/suites/backup_restore/test_backup_restore_atomic.groovy @@ -0,0 +1,209 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("test_backup_restore_atomic", "backup_restore") { + String suiteName = "test_backup_restore_atomic" + String dbName = "${suiteName}_db_1" + String dbName1 = "${suiteName}_db_2" + String repoName = "repo_" + UUID.randomUUID().toString().replace("-", "") + String snapshotName = "${suiteName}_snapshot" + String tableNamePrefix = "${suiteName}_tables" + + def syncer = getSyncer() + syncer.createS3Repository(repoName) + sql "CREATE DATABASE IF NOT EXISTS ${dbName}" + sql "CREATE DATABASE IF NOT EXISTS ${dbName1}" + + // 1. restore to not exists table_0 + // 2. restore partial data to table_1 + // 3. restore less data to table_2 + // 4. restore incremental data to table_3 + int numTables = 4; + List<String> tables = [] + for (int i = 0; i < numTables; ++i) { + String tableName = "${tableNamePrefix}_${i}" + tables.add(tableName) + sql "DROP TABLE IF EXISTS ${dbName}.${tableName}" + sql """ + CREATE TABLE ${dbName}.${tableName} ( + `id` LARGEINT NOT NULL, + `count` LARGEINT SUM DEFAULT "0" + ) + AGGREGATE KEY(`id`) + PARTITION BY RANGE(`id`) + ( + PARTITION p1 VALUES LESS THAN ("10"), + PARTITION p2 VALUES LESS THAN ("20"), + PARTITION p3 VALUES LESS THAN ("30"), + PARTITION p4 VALUES LESS THAN ("40"), + PARTITION p5 VALUES LESS THAN ("50"), + PARTITION p6 VALUES LESS THAN ("60"), + PARTITION p7 VALUES LESS THAN ("120") + ) + DISTRIBUTED BY HASH(`id`) BUCKETS 2 + PROPERTIES + ( + "replication_num" = "1" + ) + """ + } + + // 5. the len of table name equals to the config table_name_length_limit + def maxLabelLen = getFeConfig("table_name_length_limit").toInteger() + def maxTableName = "".padRight(maxLabelLen, "x") + logger.info("config table_name_length_limit = ${maxLabelLen}, table name = ${maxTableName}") + sql "DROP TABLE IF EXISTS ${dbName}.${maxTableName}" + sql """ + CREATE TABLE ${dbName}.${maxTableName} ( + `id` LARGEINT NOT NULL, + `count` LARGEINT SUM DEFAULT "0" + ) + AGGREGATE KEY(`id`) + PARTITION BY RANGE(`id`) + ( + PARTITION p1 VALUES LESS THAN ("10"), + PARTITION p2 VALUES LESS THAN ("20"), + PARTITION p3 VALUES LESS THAN ("30"), + PARTITION p4 VALUES LESS THAN ("40"), + PARTITION p5 VALUES LESS THAN ("50"), + PARTITION p6 VALUES LESS THAN ("60"), + PARTITION p7 VALUES LESS THAN ("120") + ) + DISTRIBUTED BY HASH(`id`) BUCKETS 2 + PROPERTIES + ( + "replication_num" = "1" + ) + """ + tables.add(maxTableName) + + int numRows = 10; + List<String> values = [] + for (int j = 1; j <= numRows; ++j) { + values.add("(${j}0, ${j}0)") + } + + sql "INSERT INTO ${dbName}.${tableNamePrefix}_0 VALUES ${values.join(",")}" + sql "INSERT INTO ${dbName}.${tableNamePrefix}_1 VALUES ${values.join(",")}" + sql "INSERT INTO ${dbName}.${tableNamePrefix}_2 VALUES ${values.join(",")}" + sql "INSERT INTO ${dbName}.${tableNamePrefix}_3 VALUES ${values.join(",")}" + sql "INSERT INTO ${dbName}.${maxTableName} VALUES ${values.join(",")}" + + // the other partitions of table_1 will be drop + sql """ + BACKUP SNAPSHOT ${dbName}.${snapshotName} + TO `${repoName}` + ON ( + ${tableNamePrefix}_0, + ${tableNamePrefix}_1 PARTITION (p1, p2, p3), + ${tableNamePrefix}_2, + ${tableNamePrefix}_3, + ${maxTableName} + ) + """ + + syncer.waitSnapshotFinish(dbName) + + def snapshot = syncer.getSnapshotTimestamp(repoName, snapshotName) + assertTrue(snapshot != null) + + // drop table_0 + sql "DROP TABLE ${dbName}.${tableNamePrefix}_0 FORCE" + + // insert external data to table_2 + sql "INSERT INTO ${dbName}.${tableNamePrefix}_2 VALUES ${values.join(",")}" + + sql "TRUNCATE TABLE ${dbName}.${tableNamePrefix}_3" + + sql """ + RESTORE SNAPSHOT ${dbName}.${snapshotName} + FROM `${repoName}` + PROPERTIES + ( + "backup_timestamp" = "${snapshot}", + "reserve_replica" = "true", + "atomic_restore" = "true" + ) + """ + + syncer.waitAllRestoreFinish(dbName) + + for (def tableName in tables) { + qt_sql "SELECT * FROM ${dbName}.${tableName} ORDER BY id" + } + + // restore table_3 to new db + sql """ + RESTORE SNAPSHOT ${dbName1}.${snapshotName} + FROM `${repoName}` + ON (${tableNamePrefix}_3) + PROPERTIES + ( + "backup_timestamp" = "${snapshot}", + "reserve_replica" = "true", + "atomic_restore" = "true" + ) + """ + + syncer.waitAllRestoreFinish(dbName1) + + qt_sql "SELECT * FROM ${dbName1}.${tableNamePrefix}_3 ORDER BY id" + + // add partition and insert some data. + sql "ALTER TABLE ${dbName}.${tableNamePrefix}_3 ADD PARTITION p8 VALUES LESS THAN MAXVALUE" + sql "INSERT INTO ${dbName}.${tableNamePrefix}_3 VALUES ${values.join(",")}" + sql "INSERT INTO ${dbName}.${tableNamePrefix}_3 VALUES (200, 200)" + + // backup again + snapshotName = "${snapshotName}_1" + sql """ + BACKUP SNAPSHOT ${dbName}.${snapshotName} + TO `${repoName}` + ON (${tableNamePrefix}_3) + """ + + syncer.waitSnapshotFinish(dbName) + + snapshot = syncer.getSnapshotTimestamp(repoName, snapshotName) + assertTrue(snapshot != null) + + // restore with incremental data + sql """ + RESTORE SNAPSHOT ${dbName1}.${snapshotName} + FROM `${repoName}` + ON (${tableNamePrefix}_3) + PROPERTIES + ( + "backup_timestamp" = "${snapshot}", + "reserve_replica" = "true", + "atomic_restore" = "true" + ) + """ + + syncer.waitAllRestoreFinish(dbName1) + + qt_sql "SELECT * FROM ${dbName1}.${tableNamePrefix}_3 ORDER BY id" + + for (def tableName in tables) { + sql "DROP TABLE ${dbName}.${tableName} FORCE" + } + sql "DROP DATABASE ${dbName} FORCE" + sql "DROP DATABASE ${dbName1} FORCE" + sql "DROP REPOSITORY `${repoName}`" +} + + --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org