/* * BackupAgent.actor.h * * This source file is part of the FoundationDB open source project * * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors * * Licensed 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. */ #pragma once #if defined(NO_INTELLISENSE) && !defined(FDBCLIENT_BACKUP_AGENT_ACTOR_G_H) #define FDBCLIENT_BACKUP_AGENT_ACTOR_G_H #include "fdbclient/BackupAgent.actor.g.h" #elif !defined(FDBCLIENT_BACKUP_AGENT_ACTOR_H) #define FDBCLIENT_BACKUP_AGENT_ACTOR_H #include "flow/flow.h" #include "fdbclient/NativeAPI.actor.h" #include "fdbclient/TaskBucket.h" #include "fdbclient/Notified.h" #include "fdbrpc/IAsyncFile.h" #include "fdbclient/KeyBackedTypes.h" #include #include #include "fdbclient/BackupContainer.h" #include "flow/actorcompiler.h" // has to be last include class BackupAgentBase : NonCopyable { public: // Time formatter for anything backup or restore related static std::string formatTime(int64_t epochs); static int64_t parseTime(std::string timestamp); static std::string timeFormat() { return "YYYY/MM/DD.HH:MI:SS[+/-]HHMM"; } enum class EnumState { STATE_ERRORED = 0, STATE_SUBMITTED = 1, STATE_RUNNING = 2, STATE_RUNNING_DIFFERENTIAL = 3, STATE_COMPLETED = 4, STATE_NEVERRAN = 5, STATE_ABORTED = 6, STATE_PARTIALLY_ABORTED = 7 }; static const Key keyFolderId; static const Key keyBeginVersion; static const Key keyEndVersion; static const Key keyPrevBeginVersion; static const Key keyConfigBackupTag; static const Key keyConfigLogUid; static const Key keyConfigBackupRanges; static const Key keyConfigStopWhenDoneKey; static const Key keyStateStatus; static const Key keyStateStop; static const Key keyLastUid; static const Key keyBeginKey; static const Key keyEndKey; static const Key keyDrVersion; static const Key destUid; static const Key backupStartVersion; static const Key keyTagName; static const Key keyStates; static const Key keyConfig; static const Key keyErrors; static const Key keyRanges; static const Key keyTasks; static const Key keyFutures; static const Key keySourceStates; static const Key keySourceTagName; static const int logHeaderSize; // Convert the status text to an enumerated value static EnumState getState(std::string stateText) { auto enState = EnumState::STATE_ERRORED; if (stateText.empty()) { enState = EnumState::STATE_NEVERRAN; } else if (!stateText.compare("has been submitted")) { enState = EnumState::STATE_SUBMITTED; } else if (!stateText.compare("has been started")) { enState = EnumState::STATE_RUNNING; } else if (!stateText.compare("is differential")) { enState = EnumState::STATE_RUNNING_DIFFERENTIAL; } else if (!stateText.compare("has been completed")) { enState = EnumState::STATE_COMPLETED; } else if (!stateText.compare("has been aborted")) { enState = EnumState::STATE_ABORTED; } else if (!stateText.compare("has been partially aborted")) { enState = EnumState::STATE_PARTIALLY_ABORTED; } return enState; } // Convert the status enum to a text description static const char* getStateText(EnumState enState) { const char* stateText; switch (enState) { case EnumState::STATE_ERRORED: stateText = "has errored"; break; case EnumState::STATE_NEVERRAN: stateText = "has never been started"; break; case EnumState::STATE_SUBMITTED: stateText = "has been submitted"; break; case EnumState::STATE_RUNNING: stateText = "has been started"; break; case EnumState::STATE_RUNNING_DIFFERENTIAL: stateText = "is differential"; break; case EnumState::STATE_COMPLETED: stateText = "has been completed"; break; case EnumState::STATE_ABORTED: stateText = "has been aborted"; break; case EnumState::STATE_PARTIALLY_ABORTED: stateText = "has been partially aborted"; break; default: stateText = ""; break; } return stateText; } // Convert the status enum to a name static const char* getStateName(EnumState enState) { const char* s; switch (enState) { case EnumState::STATE_ERRORED: s = "Errored"; break; case EnumState::STATE_NEVERRAN: s = "NeverRan"; break; case EnumState::STATE_SUBMITTED: s = "Submitted"; break; case EnumState::STATE_RUNNING: s = "Running"; break; case EnumState::STATE_RUNNING_DIFFERENTIAL: s = "RunningDifferentially"; break; case EnumState::STATE_COMPLETED: s = "Completed"; break; case EnumState::STATE_ABORTED: s = "Aborted"; break; case EnumState::STATE_PARTIALLY_ABORTED: s = "Aborting"; break; default: s = ""; break; } return s; } // Determine if the specified state is runnable static bool isRunnable(EnumState enState) { bool isRunnable = false; switch (enState) { case EnumState::STATE_SUBMITTED: case EnumState::STATE_RUNNING: case EnumState::STATE_RUNNING_DIFFERENTIAL: case EnumState::STATE_PARTIALLY_ABORTED: isRunnable = true; break; default: break; } return isRunnable; } static const KeyRef getDefaultTag() { return StringRef(defaultTagName); } static const std::string getDefaultTagName() { return defaultTagName; } // This is only used for automatic backup name generation static Standalone getCurrentTime() { double t = now(); time_t curTime = t; char buffer[128]; struct tm* timeinfo; timeinfo = localtime(&curTime); strftime(buffer, 128, "%Y-%m-%d-%H-%M-%S", timeinfo); std::string time(buffer); return StringRef(time + format(".%06d", (int)(1e6 * (t - curTime)))); } protected: static const std::string defaultTagName; }; class FileBackupAgent : public BackupAgentBase { public: FileBackupAgent(); FileBackupAgent(FileBackupAgent&& r) noexcept : subspace(std::move(r.subspace)), config(std::move(r.config)), lastRestorable(std::move(r.lastRestorable)), taskBucket(std::move(r.taskBucket)), futureBucket(std::move(r.futureBucket)) {} void operator=(FileBackupAgent&& r) noexcept { subspace = std::move(r.subspace); config = std::move(r.config); lastRestorable = std::move(r.lastRestorable), taskBucket = std::move(r.taskBucket); futureBucket = std::move(r.futureBucket); } KeyBackedProperty lastBackupTimestamp() { return config.pack(LiteralStringRef(__FUNCTION__)); } Future run(Database cx, double* pollDelay, int maxConcurrentTasks) { return taskBucket->run(cx, futureBucket, pollDelay, maxConcurrentTasks); } /** RESTORE **/ enum ERestoreState { UNITIALIZED = 0, QUEUED = 1, STARTING = 2, RUNNING = 3, COMPLETED = 4, ABORTED = 5 }; static StringRef restoreStateText(ERestoreState id); static Key getPauseKey(); // parallel restore Future parallelRestoreFinish(Database cx, UID randomUID, bool unlockDB = true); Future submitParallelRestore(Database cx, Key backupTag, Standalone> backupRanges, Key bcUrl, Version targetVersion, bool lockDB, UID randomUID, Key addPrefix, Key removePrefix); Future atomicParallelRestore(Database cx, Key tagName, Standalone> ranges, Key addPrefix, Key removePrefix); // restore() will // - make sure that url is readable and appears to be a complete backup // - make sure the requested TargetVersion is valid // - submit a restore on the given tagName // - Optionally wait for the restore's completion. Will restore_error if restore fails or is aborted. // restore() will return the targetVersion which will be either the valid version passed in or the max restorable // version for the given url. Future restore(Database cx, Optional cxOrig, Key tagName, Key url, Standalone> ranges, bool waitForComplete = true, Version targetVersion = -1, bool verbose = true, Key addPrefix = Key(), Key removePrefix = Key(), bool lockDB = true, bool incrementalBackupOnly = false, Version beginVersion = -1); Future restore(Database cx, Optional cxOrig, Key tagName, Key url, bool waitForComplete = true, Version targetVersion = -1, bool verbose = true, KeyRange range = normalKeys, Key addPrefix = Key(), Key removePrefix = Key(), bool lockDB = true, bool incrementalBackupOnly = false, Version beginVersion = -1) { Standalone> rangeRef; rangeRef.push_back_deep(rangeRef.arena(), range); return restore(cx, cxOrig, tagName, url, rangeRef, waitForComplete, targetVersion, verbose, addPrefix, removePrefix, lockDB, incrementalBackupOnly, beginVersion); } Future atomicRestore(Database cx, Key tagName, Standalone> ranges, Key addPrefix = Key(), Key removePrefix = Key()); Future atomicRestore(Database cx, Key tagName, KeyRange range = normalKeys, Key addPrefix = Key(), Key removePrefix = Key()) { Standalone> rangeRef; rangeRef.push_back_deep(rangeRef.arena(), range); return atomicRestore(cx, tagName, rangeRef, addPrefix, removePrefix); } // Tries to abort the restore for a tag. Returns the final (stable) state of the tag. Future abortRestore(Reference tr, Key tagName); Future abortRestore(Database cx, Key tagName); // Waits for a restore tag to reach a final (stable) state. Future waitRestore(Database cx, Key tagName, bool verbose); // Get a string describing the status of a tag Future restoreStatus(Reference tr, Key tagName); Future restoreStatus(Database cx, Key tagName) { return runRYWTransaction(cx, [=](Reference tr) { return restoreStatus(tr, tagName); }); } /** BACKUP METHODS **/ Future submitBackup(Reference tr, Key outContainer, int snapshotIntervalSeconds, std::string tagName, Standalone> backupRanges, bool stopWhenDone = true, bool partitionedLog = false, bool incrementalBackupOnly = false); Future submitBackup(Database cx, Key outContainer, int snapshotIntervalSeconds, std::string tagName, Standalone> backupRanges, bool stopWhenDone = true, bool partitionedLog = false, bool incrementalBackupOnly = false) { return runRYWTransactionFailIfLocked(cx, [=](Reference tr) { return submitBackup(tr, outContainer, snapshotIntervalSeconds, tagName, backupRanges, stopWhenDone, partitionedLog, incrementalBackupOnly); }); } Future discontinueBackup(Reference tr, Key tagName); Future discontinueBackup(Database cx, Key tagName) { return runRYWTransaction( cx, [=](Reference tr) { return discontinueBackup(tr, tagName); }); } // Terminate an ongoing backup, without waiting for the backup to finish. // Preconditions: // A backup is running with the tag of `tagName`. // Otherwise `backup_unneeded` will be thrown indicating that the backup never existed or already finished. // Postconditions: // No more tasks will be spawned to backup ranges of the database. // logRangesRange and backupLogKeys will be cleared for this backup. Future abortBackup(Reference tr, std::string tagName); Future abortBackup(Database cx, std::string tagName) { return runRYWTransaction(cx, [=](Reference tr) { return abortBackup(tr, tagName); }); } Future getStatus(Database cx, bool showErrors, std::string tagName); Future getStatusJSON(Database cx, std::string tagName); Future getLastRestorable(Reference tr, Key tagName, bool snapshot = false); void setLastRestorable(Reference tr, Key tagName, Version version); // stopWhenDone will return when the backup is stopped, if enabled. Otherwise, it // will return when the backup directory is restorable. Future waitBackup(Database cx, std::string tagName, bool stopWhenDone = true, Reference* pContainer = nullptr, UID* pUID = nullptr); static const Key keyLastRestorable; Future getTaskCount(Reference tr) { return taskBucket->getTaskCount(tr); } Future getTaskCount(Database cx) { return taskBucket->getTaskCount(cx); } Future watchTaskCount(Reference tr) { return taskBucket->watchTaskCount(tr); } Future checkActive(Database cx) { return taskBucket->checkActive(cx); } // If "pause" is true, pause all backups; otherwise, resume all. Future changePause(Database db, bool pause); friend class FileBackupAgentImpl; static const int dataFooterSize; Subspace subspace; Subspace config; Subspace lastRestorable; Reference taskBucket; Reference futureBucket; }; template <> inline Tuple Codec::pack(FileBackupAgent::ERestoreState const& val) { return Tuple().append(val); } template <> inline FileBackupAgent::ERestoreState Codec::unpack(Tuple const& val) { return (FileBackupAgent::ERestoreState)val.getInt(0); } class DatabaseBackupAgent : public BackupAgentBase { public: DatabaseBackupAgent(); explicit DatabaseBackupAgent(Database src); DatabaseBackupAgent(DatabaseBackupAgent&& r) noexcept : subspace(std::move(r.subspace)), states(std::move(r.states)), config(std::move(r.config)), errors(std::move(r.errors)), ranges(std::move(r.ranges)), tagNames(std::move(r.tagNames)), taskBucket(std::move(r.taskBucket)), futureBucket(std::move(r.futureBucket)), sourceStates(std::move(r.sourceStates)), sourceTagNames(std::move(r.sourceTagNames)) {} void operator=(DatabaseBackupAgent&& r) noexcept { subspace = std::move(r.subspace); states = std::move(r.states); config = std::move(r.config); errors = std::move(r.errors); ranges = std::move(r.ranges); tagNames = std::move(r.tagNames); taskBucket = std::move(r.taskBucket); futureBucket = std::move(r.futureBucket); sourceStates = std::move(r.sourceStates); sourceTagNames = std::move(r.sourceTagNames); } Future run(Database cx, double* pollDelay, int maxConcurrentTasks) { return taskBucket->run(cx, futureBucket, pollDelay, maxConcurrentTasks); } Future atomicSwitchover(Database dest, Key tagName, Standalone> backupRanges, Key addPrefix, Key removePrefix, bool forceAction = false); Future unlockBackup(Reference tr, Key tagName); Future unlockBackup(Database cx, Key tagName) { return runRYWTransaction(cx, [=](Reference tr) { return unlockBackup(tr, tagName); }); } Future submitBackup(Reference tr, Key tagName, Standalone> backupRanges, bool stopWhenDone = true, Key addPrefix = StringRef(), Key removePrefix = StringRef(), bool lockDatabase = false, bool databasesInSync = false); Future submitBackup(Database cx, Key tagName, Standalone> backupRanges, bool stopWhenDone = true, Key addPrefix = StringRef(), Key removePrefix = StringRef(), bool lockDatabase = false, bool databasesInSync = false) { return runRYWTransaction(cx, [=](Reference tr) { return submitBackup( tr, tagName, backupRanges, stopWhenDone, addPrefix, removePrefix, lockDatabase, databasesInSync); }); } Future discontinueBackup(Reference tr, Key tagName); Future discontinueBackup(Database cx, Key tagName) { return runRYWTransaction( cx, [=](Reference tr) { return discontinueBackup(tr, tagName); }); } Future abortBackup(Database cx, Key tagName, bool partial = false, bool abortOldBackup = false, bool dstOnly = false, bool waitForDestUID = false); Future getStatus(Database cx, int errorLimit, Key tagName); Future getStateValue(Reference tr, UID logUid, bool snapshot = false); Future getStateValue(Database cx, UID logUid) { return runRYWTransaction(cx, [=](Reference tr) { return getStateValue(tr, logUid); }); } Future getDestUid(Reference tr, UID logUid, bool snapshot = false); Future getDestUid(Database cx, UID logUid) { return runRYWTransaction(cx, [=](Reference tr) { return getDestUid(tr, logUid); }); } Future getLogUid(Reference tr, Key tagName, bool snapshot = false); Future getLogUid(Database cx, Key tagName) { return runRYWTransaction(cx, [=](Reference tr) { return getLogUid(tr, tagName); }); } Future getRangeBytesWritten(Reference tr, UID logUid, bool snapshot = false); Future getLogBytesWritten(Reference tr, UID logUid, bool snapshot = false); // stopWhenDone will return when the backup is stopped, if enabled. Otherwise, it // will return when the backup directory is restorable. Future waitBackup(Database cx, Key tagName, bool stopWhenDone = true); Future waitSubmitted(Database cx, Key tagName); Future waitUpgradeToLatestDrVersion(Database cx, Key tagName); static const Key keyAddPrefix; static const Key keyRemovePrefix; static const Key keyRangeVersions; static const Key keyCopyStop; static const Key keyDatabasesInSync; static const int LATEST_DR_VERSION; Future getTaskCount(Reference tr) { return taskBucket->getTaskCount(tr); } Future getTaskCount(Database cx) { return taskBucket->getTaskCount(cx); } Future watchTaskCount(Reference tr) { return taskBucket->watchTaskCount(tr); } Future checkActive(Database cx) { return taskBucket->checkActive(cx); } friend class DatabaseBackupAgentImpl; Subspace subspace; Subspace states; Subspace config; Subspace errors; Subspace ranges; Subspace tagNames; Subspace sourceStates; Subspace sourceTagNames; Reference taskBucket; Reference futureBucket; }; typedef std::pair, Version> RangeResultWithVersion; struct RCGroup { Standalone items; Version version; uint64_t groupKey; RCGroup() : version(-1), groupKey(ULLONG_MAX){}; template void serialize(Ar& ar) { serializer(ar, items, version, groupKey); } }; bool copyParameter(Reference source, Reference dest, Key key); Version getVersionFromString(std::string const& value); Standalone> getLogRanges(Version beginVersion, Version endVersion, Key destUidValue, int blockSize = CLIENT_KNOBS->LOG_RANGE_BLOCK_SIZE); Standalone> getApplyRanges(Version beginVersion, Version endVersion, Key backupUid); Future eraseLogData(Reference tr, Key logUidValue, Key destUidValue, Optional endVersion = Optional(), bool checkBackupUid = false, Version backupUid = 0); Key getApplyKey(Version version, Key backupUid); Version getLogKeyVersion(Key key); std::pair decodeBKMutationLogKey(Key key); Future logError(Database cx, Key keyErrors, const std::string& message); Future logError(Reference tr, Key keyErrors, const std::string& message); Future checkVersion(Reference const& tr); ACTOR Future readCommitted(Database cx, PromiseStream results, Reference lock, KeyRangeRef range, bool terminator = true, bool systemAccess = false, bool lockAware = false); ACTOR Future readCommitted(Database cx, PromiseStream results, Future active, Reference lock, KeyRangeRef range, std::function(Key key)> groupBy, bool terminator = true, bool systemAccess = false, bool lockAware = false); ACTOR Future applyMutations(Database cx, Key uid, Key addPrefix, Key removePrefix, Version beginVersion, Version* endVersion, RequestStream commit, NotifiedVersion* committedVersion, Reference> keyVersion); ACTOR Future cleanupBackup(Database cx, bool deleteData); using EBackupState = BackupAgentBase::EnumState; template <> inline Tuple Codec::pack(EBackupState const& val) { return Tuple().append(static_cast(val)); } template <> inline EBackupState Codec::unpack(Tuple const& val) { return static_cast(val.getInt(0)); } // Key backed tags are a single-key slice of the TagUidMap, defined below. // The Value type of the key is a UidAndAbortedFlagT which is a pair of {UID, aborted_flag} // All tasks on the UID will have a validation key/value that requires aborted_flag to be // false, so changing that value, such as changing the UID or setting aborted_flag to true, // will kill all of the active tasks on that backup/restore UID. typedef std::pair UidAndAbortedFlagT; class KeyBackedTag : public KeyBackedProperty { public: KeyBackedTag() : KeyBackedProperty(StringRef()) {} KeyBackedTag(std::string tagName, StringRef tagMapPrefix); Future cancel(Reference tr) { std::string tag = tagName; Key _tagMapPrefix = tagMapPrefix; return map(get(tr), [tag, _tagMapPrefix, tr](Optional up) -> Void { if (up.present()) { // Set aborted flag to true up.get().second = true; KeyBackedTag(tag, _tagMapPrefix).set(tr, up.get()); } return Void(); }); } std::string tagName; Key tagMapPrefix; }; typedef KeyBackedMap TagMap; // Map of tagName to {UID, aborted_flag} located in the fileRestorePrefixRange keyspace. class TagUidMap : public KeyBackedMap { public: TagUidMap(const StringRef& prefix) : TagMap(LiteralStringRef("tag->uid/").withPrefix(prefix)), prefix(prefix) {} ACTOR static Future> getAll_impl(TagUidMap* tagsMap, Reference tr, bool snapshot); Future> getAll(Reference tr, bool snapshot = false) { return getAll_impl(this, tr, snapshot); } Key prefix; }; static inline KeyBackedTag makeRestoreTag(std::string tagName) { return KeyBackedTag(tagName, fileRestorePrefixRange.begin); } static inline KeyBackedTag makeBackupTag(std::string tagName) { return KeyBackedTag(tagName, fileBackupPrefixRange.begin); } static inline Future> getAllRestoreTags(Reference tr, bool snapshot = false) { return TagUidMap(fileRestorePrefixRange.begin).getAll(tr, snapshot); } static inline Future> getAllBackupTags(Reference tr, bool snapshot = false) { return TagUidMap(fileBackupPrefixRange.begin).getAll(tr, snapshot); } class KeyBackedConfig { public: static struct { static TaskParam uid() { return LiteralStringRef(__FUNCTION__); } } TaskParams; KeyBackedConfig(StringRef prefix, UID uid = UID()) : uid(uid), prefix(prefix), configSpace(uidPrefixKey(LiteralStringRef("uid->config/").withPrefix(prefix), uid)) {} KeyBackedConfig(StringRef prefix, Reference task) : KeyBackedConfig(prefix, TaskParams.uid().get(task)) {} Future toTask(Reference tr, Reference task, bool setValidation = true) { // Set the uid task parameter TaskParams.uid().set(task, uid); if (!setValidation) { return Void(); } // Set the validation condition for the task which is that the restore uid's tag's uid is the same as the // restore uid. Get this uid's tag, then get the KEY for the tag's uid but don't read it. That becomes the // validation key which TaskBucket will check, and its value must be this restore config's uid. UID u = uid; // 'this' could be invalid in lambda Key p = prefix; return map(tag().get(tr), [u, p, task](Optional const& tag) -> Void { if (!tag.present()) throw restore_error(); // Validation contition is that the uidPair key must be exactly {u, false} TaskBucket::setValidationCondition( task, KeyBackedTag(tag.get(), p).key, Codec::pack({ u, false }).pack()); return Void(); }); } KeyBackedProperty tag() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } UID getUid() { return uid; } Key getUidAsKey() { return BinaryWriter::toValue(uid, Unversioned()); } void clear(Reference tr) { tr->clear(configSpace.range()); } // lastError is a pair of error message and timestamp expressed as an int64_t KeyBackedProperty> lastError() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } KeyBackedMap> lastErrorPerType() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } // Updates the error per type map and the last error property Future updateErrorInfo(Database cx, Error e, std::string message) { // Avoid capture of this ptr auto& copy = *this; return runRYWTransaction(cx, [=](Reference tr) mutable { tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); tr->setOption(FDBTransactionOptions::LOCK_AWARE); return map(tr->getReadVersion(), [=](Version v) mutable { copy.lastError().set(tr, { message, v }); copy.lastErrorPerType().set(tr, e.code(), { message, v }); return Void(); }); }); } protected: UID uid; Key prefix; Subspace configSpace; }; template <> inline Tuple Codec>::pack(Reference const& bc) { return Tuple().append(StringRef(bc->getURL())); } template <> inline Reference Codec>::unpack(Tuple const& val) { return IBackupContainer::openContainer(val.getString(0).toString()); } class BackupConfig : public KeyBackedConfig { public: BackupConfig(UID uid = UID()) : KeyBackedConfig(fileBackupPrefixRange.begin, uid) {} BackupConfig(Reference task) : KeyBackedConfig(fileBackupPrefixRange.begin, task) {} // rangeFileMap maps a keyrange file's End to its Begin and Filename struct RangeSlice { Key begin; Version version; std::string fileName; int64_t fileSize; Tuple pack() const { return Tuple().append(begin).append(version).append(StringRef(fileName)).append(fileSize); } static RangeSlice unpack(Tuple const& t) { RangeSlice r; int i = 0; r.begin = t.getString(i++); r.version = t.getInt(i++); r.fileName = t.getString(i++).toString(); r.fileSize = t.getInt(i++); return r; } }; // Map of range end boundaries to info about the backup file written for that range. typedef KeyBackedMap RangeFileMapT; RangeFileMapT snapshotRangeFileMap() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } // Number of kv range files that were both committed to persistent storage AND inserted into // the snapshotRangeFileMap. Note that since insertions could replace 1 or more existing // map entries this is not necessarily the number of entries currently in the map. // This value exists to help with sizing of kv range folders for BackupContainers that // require it. KeyBackedBinaryValue snapshotRangeFileCount() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } // Coalesced set of ranges already dispatched for writing. typedef KeyBackedMap RangeDispatchMapT; RangeDispatchMapT snapshotRangeDispatchMap() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } // Interval to use for determining the target end version for new snapshots KeyBackedProperty snapshotIntervalSeconds() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } // When the current snapshot began KeyBackedProperty snapshotBeginVersion() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } // When the current snapshot is desired to end. // This can be changed at runtime to speed up or slow down a snapshot KeyBackedProperty snapshotTargetEndVersion() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } KeyBackedProperty snapshotBatchSize() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } KeyBackedProperty snapshotBatchFuture() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } KeyBackedProperty snapshotBatchDispatchDoneKey() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } KeyBackedProperty snapshotDispatchLastShardsBehind() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } KeyBackedProperty snapshotDispatchLastVersion() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } Future initNewSnapshot(Reference tr, int64_t intervalSeconds = -1) { BackupConfig& copy = *this; // Capture this by value instead of this ptr Future beginVersion = tr->getReadVersion(); Future defaultInterval = 0; if (intervalSeconds < 0) defaultInterval = copy.snapshotIntervalSeconds().getOrThrow(tr); // Make sure read version and possibly the snapshot interval value are ready, then clear/init the snapshot // config members return map(success(beginVersion) && success(defaultInterval), [=](Void) mutable { copy.snapshotRangeFileMap().clear(tr); copy.snapshotRangeDispatchMap().clear(tr); copy.snapshotBatchSize().clear(tr); copy.snapshotBatchFuture().clear(tr); copy.snapshotBatchDispatchDoneKey().clear(tr); if (intervalSeconds < 0) intervalSeconds = defaultInterval.get(); Version endVersion = beginVersion.get() + intervalSeconds * CLIENT_KNOBS->CORE_VERSIONSPERSECOND; copy.snapshotBeginVersion().set(tr, beginVersion.get()); copy.snapshotTargetEndVersion().set(tr, endVersion); copy.snapshotRangeFileCount().set(tr, 0); copy.snapshotDispatchLastVersion().clear(tr); copy.snapshotDispatchLastShardsBehind().clear(tr); return Void(); }); } KeyBackedBinaryValue rangeBytesWritten() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } KeyBackedBinaryValue logBytesWritten() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } KeyBackedProperty stateEnum() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } KeyBackedProperty> backupContainer() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } // Set to true when all backup workers for saving mutation logs have been started. KeyBackedProperty allWorkerStarted() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } // Each backup worker adds its (epoch, tag.id) to this property. KeyBackedProperty>> startedBackupWorkers() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } // Set to true if backup worker is enabled. KeyBackedProperty backupWorkerEnabled() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } // Set to true if partitioned log is enabled (only useful if backup worker is also enabled). KeyBackedProperty partitionedLogEnabled() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } // Set to true if only requesting incremental backup without base snapshot. KeyBackedProperty incrementalBackupOnly() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } // Latest version for which all prior versions have saved by backup workers. KeyBackedProperty latestBackupWorkerSavedVersion() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } // Stop differntial logging if already started or don't start after completing KV ranges KeyBackedProperty stopWhenDone() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } // Latest version for which all prior versions have had their log copy tasks completed KeyBackedProperty latestLogEndVersion() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } // The end version of the last complete snapshot KeyBackedProperty latestSnapshotEndVersion() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } // The end version of the first complete snapshot KeyBackedProperty firstSnapshotEndVersion() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } KeyBackedProperty destUidValue() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } Future> getLatestRestorableVersion(Reference tr) { tr->setOption(FDBTransactionOptions::READ_SYSTEM_KEYS); tr->setOption(FDBTransactionOptions::READ_LOCK_AWARE); auto lastLog = latestLogEndVersion().get(tr); auto firstSnapshot = firstSnapshotEndVersion().get(tr); auto workerEnabled = backupWorkerEnabled().get(tr); auto plogEnabled = partitionedLogEnabled().get(tr); auto workerVersion = latestBackupWorkerSavedVersion().get(tr); auto incrementalBackup = incrementalBackupOnly().get(tr); return map(success(lastLog) && success(firstSnapshot) && success(workerEnabled) && success(plogEnabled) && success(workerVersion) && success(incrementalBackup), [=](Void) -> Optional { // The latest log greater than the oldest snapshot is the restorable version Optional logVersion = workerEnabled.get().present() && workerEnabled.get().get() && plogEnabled.get().present() && plogEnabled.get().get() ? workerVersion.get() : lastLog.get(); if (logVersion.present() && firstSnapshot.get().present() && logVersion.get() > firstSnapshot.get().get()) { return std::max(logVersion.get() - 1, firstSnapshot.get().get()); } if (logVersion.present() && incrementalBackup.isReady() && incrementalBackup.get().present() && incrementalBackup.get().get()) { return logVersion.get() - 1; } return {}; }); } KeyBackedProperty> backupRanges() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } void startMutationLogs(Reference tr, KeyRangeRef backupRange, Key destUidValue) { Key mutationLogsDestKey = destUidValue.withPrefix(backupLogKeys.begin); tr->set(logRangesEncodeKey(backupRange.begin, BinaryReader::fromStringRef(destUidValue, Unversioned())), logRangesEncodeValue(backupRange.end, mutationLogsDestKey)); } Future logError(Database cx, Error e, std::string details, void* taskInstance = nullptr) { if (!uid.isValid()) { TraceEvent(SevError, "FileBackupErrorNoUID").error(e).detail("Description", details); return Void(); } TraceEvent t(SevWarn, "FileBackupError"); t.error(e) .detail("BackupUID", uid) .detail("Description", details) .detail("TaskInstance", (uint64_t)taskInstance); // key_not_found could happen if (e.code() == error_code_key_not_found) t.backtrace(); return updateErrorInfo(cx, e, details); } }; // Helper class for reading restore data from a buffer and throwing the right errors. struct StringRefReader { StringRefReader(StringRef s = StringRef(), Error e = Error()) : rptr(s.begin()), end(s.end()), failure_error(e) {} // Return remainder of data as a StringRef StringRef remainder() { return StringRef(rptr, end - rptr); } // Return a pointer to len bytes at the current read position and advance read pos const uint8_t* consume(unsigned int len) { if (rptr == end && len != 0) throw end_of_stream(); const uint8_t* p = rptr; rptr += len; if (rptr > end) throw failure_error; return p; } // Return a T from the current read position and advance read pos template const T consume() { return *(const T*)consume(sizeof(T)); } // Functions for consuming big endian (network byte order) integers. // Consumes a big endian number, swaps it to little endian, and returns it. int32_t consumeNetworkInt32() { return (int32_t)bigEndian32((uint32_t)consume()); } uint32_t consumeNetworkUInt32() { return bigEndian32(consume()); } // Convert big Endian value (e.g., encoded in log file) into a littleEndian uint64_t value. int64_t consumeNetworkInt64() { return (int64_t)bigEndian64((uint32_t)consume()); } uint64_t consumeNetworkUInt64() { return bigEndian64(consume()); } bool eof() { return rptr == end; } const uint8_t *rptr, *end; Error failure_error; }; namespace fileBackup { ACTOR Future>> decodeRangeFileBlock(Reference file, int64_t offset, int len); // Return a block of contiguous padding bytes "\0xff" for backup files, growing if needed. Value makePadding(int size); } // namespace fileBackup // For fast restore simulation test // For testing addPrefix feature in fast restore. // Transform db content in restoreRanges by removePrefix and then addPrefix. // Assume: DB is locked ACTOR Future transformRestoredDatabase(Database cx, Standalone> backupRanges, Key addPrefix, Key removePrefix); void simulateBlobFailure(); #include "flow/unactorcompiler.h" #endif