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

Reply via email to