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 <[email protected]>
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: [email protected]
For additional commands, e-mail: [email protected]