diff --git a/fdbbackup/backup.actor.cpp b/fdbbackup/backup.actor.cpp index 205d7aa566..2e9019969a 100644 --- a/fdbbackup/backup.actor.cpp +++ b/fdbbackup/backup.actor.cpp @@ -1837,16 +1837,16 @@ ACTOR Future expireBackupData(const char *name, std::string destinationCon ACTOR Future deleteBackupContainer(const char *name, std::string destinationContainer) { try { state Reference c = openBackupContainer(name, destinationContainer); + state int numDeleted = 0; + state Future done = c->deleteContainer(&numDeleted); loop { choose { - when ( Void _ = wait(c->deleteContainer()) ) { + when ( Void _ = wait(done) ) { printf("The entire container has been deleted.\n"); break; } when ( Void _ = wait(delay(3)) ) { - int numDeleted = 0; - c->deleteContainer(&numDeleted); printf("%d files have been deleted so far...\n", numDeleted); } } diff --git a/fdbclient/BackupContainer.actor.cpp b/fdbclient/BackupContainer.actor.cpp index af6e8bcd59..88f2365850 100644 --- a/fdbclient/BackupContainer.actor.cpp +++ b/fdbclient/BackupContainer.actor.cpp @@ -149,36 +149,42 @@ public: return writeFile(format("ranges/%s/range,%lld,%s,%d", rangeVersionFolderString(version).c_str(), version, g_random->randomUniqueID().toString().c_str(), blockSize)); } - static RangeFile pathToRangeFile(std::string path, int64_t size) { + static bool pathToRangeFile(RangeFile &out, std::string path, int64_t size) { std::string name = basename(path); RangeFile f; f.fileName = path; f.fileSize = size; int len; - if(sscanf(name.c_str(), "range,%lld,%*[^,],%u%n", &f.version, &f.blockSize, &len) == 2 && len == name.size()) - return f; - throw restore_unknown_file_type(); + if(sscanf(name.c_str(), "range,%lld,%*[^,],%u%n", &f.version, &f.blockSize, &len) == 2 && len == name.size()) { + out = f; + return true; + } + return false; } - static LogFile pathToLogFile(std::string path, int64_t size) { + static bool pathToLogFile(LogFile &out, std::string path, int64_t size) { std::string name = basename(path); LogFile f; f.fileName = path; f.fileSize = size; int len; - if(sscanf(name.c_str(), "log,%lld,%lld,%*[^,],%u%n", &f.beginVersion, &f.endVersion, &f.blockSize, &len) == 3 && len == name.size()) - return f; - throw restore_unknown_file_type(); + if(sscanf(name.c_str(), "log,%lld,%lld,%*[^,],%u%n", &f.beginVersion, &f.endVersion, &f.blockSize, &len) == 3 && len == name.size()) { + out = f; + return true; + } + return false; } - static KeyspaceSnapshotFile pathToKeyspaceSnapshotFile(std::string path) { + static bool pathToKeyspaceSnapshotFile(KeyspaceSnapshotFile &out, std::string path) { std::string name = basename(path); KeyspaceSnapshotFile f; f.fileName = path; int len; - if(sscanf(name.c_str(), "snapshot,%lld,%lld,%lld%n", &f.beginVersion, &f.endVersion, &f.totalSize, &len) == 3 && len == name.size()) - return f; - throw restore_unknown_file_type(); + if(sscanf(name.c_str(), "snapshot,%lld,%lld,%lld%n", &f.beginVersion, &f.endVersion, &f.totalSize, &len) == 3 && len == name.size()) { + out = f; + return true; + } + return false; } // TODO: Do this more efficiently, as the range file list for a snapshot could potentially be hundreds of megabytes. @@ -236,14 +242,18 @@ public: Version minVer = std::numeric_limits::max(); Version maxVer = std::numeric_limits::min(); + RangeFile rf; for(auto &f : fileNames) { - array.push_back(f); - RangeFile rf = pathToRangeFile(f, 0); - if(rf.version < minVer) - minVer = rf.version; - if(rf.version > maxVer) - maxVer = rf.version; + if(pathToRangeFile(rf, f, 0)) { + array.push_back(f); + if(rf.version < minVer) + minVer = rf.version; + if(rf.version > maxVer) + maxVer = rf.version; + } + else + throw restore_unknown_file_type(); } doc.create("totalBytes") = totalBytes; @@ -267,9 +277,9 @@ public: Future> listLogFiles(Version beginVersion = std::numeric_limits::min(), Version endVersion = std::numeric_limits::max()) { return map(listFiles("logs/"), [=](const FilesAndSizesT &files) { std::vector results; + LogFile lf; for(auto &f : files) { - LogFile lf = pathToLogFile(f.first, f.second); - if(lf.endVersion > beginVersion && lf.beginVersion < endVersion) + if(pathToLogFile(lf, f.first, f.second) && lf.endVersion > beginVersion && lf.beginVersion < endVersion) results.push_back(lf); } std::sort(results.begin(), results.end()); @@ -279,9 +289,9 @@ public: Future> listRangeFiles(Version beginVersion = std::numeric_limits::min(), Version endVersion = std::numeric_limits::max()) { return map(listFiles("ranges/"), [=](const FilesAndSizesT &files) { std::vector results; + RangeFile rf; for(auto &f : files) { - RangeFile rf = pathToRangeFile(f.first, f.second); - if(rf.version >= beginVersion && rf.version <= endVersion) + if(pathToRangeFile(rf, f.first, f.second) && rf.version >= beginVersion && rf.version <= endVersion) results.push_back(rf); } std::sort(results.begin(), results.end()); @@ -291,9 +301,9 @@ public: Future> listKeyspaceSnapshots(Version beginVersion = std::numeric_limits::min(), Version endVersion = std::numeric_limits::max()) { return map(listFiles("snapshots/"), [=](const FilesAndSizesT &files) { std::vector results; + KeyspaceSnapshotFile sf; for(auto &f : files) { - KeyspaceSnapshotFile sf = pathToKeyspaceSnapshotFile(f.first); - if(sf.endVersion > beginVersion && sf.beginVersion < endVersion) + if(pathToKeyspaceSnapshotFile(sf, f.first) && sf.endVersion > beginVersion && sf.beginVersion < endVersion) results.push_back(sf); } std::sort(results.begin(), results.end()); @@ -724,8 +734,8 @@ public: // TODO: If there is a need, this can be made faster by discovering common prefixes and listing levels of the folder structure in parallel. ACTOR static Future deleteContainer_impl(Reference bc, int *pNumDeleted) { state PromiseStream resultStream; - state Future done = bc->m_bstore->listBucketStream(BACKUP_BUCKET, resultStream, bc->m_name + "/"); - state std::vector> deleteFutures; + state Future done = bc->m_bstore->listBucketStream(BACKUP_BUCKET, resultStream, bc->m_name + "/", '/', std::numeric_limits::max()); + state std::list> deleteFutures; loop { choose { when(Void _ = wait(done)) { @@ -740,11 +750,20 @@ public: return Void(); })); } + + while(deleteFutures.size() > CLIENT_KNOBS->BLOBSTORE_CONCURRENT_REQUESTS) { + Void _ = wait(deleteFutures.front()); + deleteFutures.pop_front(); + } } } } - Void _ = wait(waitForAll(deleteFutures)); + while(deleteFutures.size() > 0) { + Void _ = wait(deleteFutures.front()); + deleteFutures.pop_front(); + } + return Void(); } diff --git a/fdbclient/DatabaseBackupAgent.actor.cpp b/fdbclient/DatabaseBackupAgent.actor.cpp index 436b49a80a..09f74a76dc 100644 --- a/fdbclient/DatabaseBackupAgent.actor.cpp +++ b/fdbclient/DatabaseBackupAgent.actor.cpp @@ -1445,8 +1445,8 @@ public: tr3.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); tr3.setOption(FDBTransactionOptions::LOCK_AWARE); Version destVersion = wait(tr3.getReadVersion()); + TraceEvent("DBA_switchover_version_upgrade").detail("src", commitVersion).detail("dest", destVersion); if (destVersion <= commitVersion) { - TraceEvent("DBA_switchover_version_upgrade").detail("src", commitVersion).detail("dest", destVersion); TEST(true); // Forcing dest backup cluster to higher version tr3.set(minRequiredCommitVersionKey, BinaryWriter::toValue(commitVersion+1, Unversioned())); Void _ = wait(tr3.commit()); @@ -1549,12 +1549,20 @@ public: if (lastApplied.present()) { Version current = tr->getReadVersion().get(); Version applied = BinaryReader::fromStringRef(lastApplied.get(), Unversioned()); + TraceEvent("DBA_abort_version_upgrade").detail("src", applied).detail("dest", current); if (current <= applied) { - TraceEvent("DBA_abort_version_upgrade").detail("src", applied).detail("dest", current); TEST(true); // Upgrading version of local database. // The +1 is because we want to make sure that a versionstamped operation can't reuse // the same version as an already-applied transaction. tr->set(minRequiredCommitVersionKey, BinaryWriter::toValue(applied+1, Unversioned())); + } else { + // We need to enforce that the read we did of the applyMutationsBeginKey is the most + // recent and up to date value, as the proxy might have accepted a commit previously + // queued by dumpData after our read. Transactions that don't have write conflict ranges + // have a no-op commit(), as they become snapshot transactions to which we don't promise + // strict serializability. Therefore, we add an arbitrary write conflict range to + // request the strict serializability guarantee that is required. + tr->addWriteConflictRange(singleKeyRange(minRequiredCommitVersionKey)); } } Void _ = wait(tr->commit()); diff --git a/fdbclient/FileBackupAgent.actor.cpp b/fdbclient/FileBackupAgent.actor.cpp index dd4bc3defa..3615ecc17f 100644 --- a/fdbclient/FileBackupAgent.actor.cpp +++ b/fdbclient/FileBackupAgent.actor.cpp @@ -674,6 +674,86 @@ namespace fileBackup { return Void(); } + ACTOR static Future abortOldBackup(FileBackupAgent* backupAgent, Reference tr, std::string tagName) { + tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); + tr->setOption(FDBTransactionOptions::LOCK_AWARE); + + state Subspace tagNames = backupAgent->subspace.get(BackupAgentBase::keyTagName); + Optional uidStr = wait(tr->get(tagNames.pack(Key(tagName)))); + if (!uidStr.present()) { + TraceEvent(SevError, "BackupTagMissing").detail("tagName", tagName.c_str()); + throw backup_error(); + } + state UID uid = BinaryReader::fromStringRef(uidStr.get(), Unversioned()); + + state Subspace statusSpace = backupAgent->subspace.get(BackupAgentBase::keyStates).get(uid.toString()); + state Subspace globalConfig = backupAgent->subspace.get(BackupAgentBase::keyConfig).get(uid.toString()); + state Subspace newConfigSpace = uidPrefixKey(LiteralStringRef("uid->config/").withPrefix(fileBackupPrefixRange.begin), uid); + + Optional statusStr = wait(tr->get(statusSpace.pack(FileBackupAgent::keyStateStatus))); + state EBackupState status = !statusStr.present() ? FileBackupAgent::STATE_NEVERRAN : BackupAgentBase::getState(statusStr.get().toString()); + + if (!backupAgent->isRunnable(status)) { + throw backup_unneeded(); + } + + TraceEvent(SevInfo, "FileBackupAbortOld") + .detail("tagName", tagName.c_str()) + .detail("status", BackupAgentBase::getStateText(status)); + + // Clear the folder id will prevent future tasks from executing + tr->clear(singleKeyRange(StringRef(globalConfig.pack(FileBackupAgent::keyFolderId)))); + + Key configPath = uidPrefixKey(logRangesRange.begin, uid); + Key logsPath = uidPrefixKey(backupLogKeys.begin, uid); + + tr->clear(KeyRangeRef(configPath, strinc(configPath))); + tr->clear(KeyRangeRef(logsPath, strinc(logsPath))); + tr->clear(newConfigSpace.range()); + + Key statusKey = StringRef(statusSpace.pack(FileBackupAgent::keyStateStatus)); + tr->set(statusKey, StringRef(FileBackupAgent::getStateText(BackupAgentBase::STATE_ABORTED))); + + return Void(); + } + + struct AbortOldBackupTask : TaskFuncBase { + static StringRef name; + ACTOR static Future _finish(Reference tr, Reference taskBucket, Reference futureBucket, Reference task) { + state FileBackupAgent backupAgent; + state std::string tagName = task->params[BackupAgentBase::keyConfigBackupTag].toString(); + + TEST(true); // Canceling old backup task + + TraceEvent(SevInfo, "FileBackupCancelOldTask") + .detail("task", printable(task->params[Task::reservedTaskParamKeyType])) + .detail("tagName", tagName); + Void _ = wait(abortOldBackup(&backupAgent, tr, tagName)); + + Void _ = wait(taskBucket->finish(tr, task)); + return Void(); + } + + virtual StringRef getName() const { + TraceEvent(SevError, "FileBackupError").detail("cause", "AbortOldBackupTaskFunc::name() should never be called"); + ASSERT(false); + return StringRef(); + } + + Future execute(Database cx, Reference tb, Reference fb, Reference task) { return Future(Void()); }; + Future finish(Reference tr, Reference tb, Reference fb, Reference task) { return _finish(tr, tb, fb, task); }; + }; + StringRef AbortOldBackupTask::name = LiteralStringRef("abort_legacy_backup"); + REGISTER_TASKFUNC(AbortOldBackupTask); + REGISTER_TASKFUNC_ALIAS(AbortOldBackupTask, file_backup_diff_logs); + REGISTER_TASKFUNC_ALIAS(AbortOldBackupTask, file_backup_log_range); + REGISTER_TASKFUNC_ALIAS(AbortOldBackupTask, file_backup_logs); + REGISTER_TASKFUNC_ALIAS(AbortOldBackupTask, file_backup_range); + REGISTER_TASKFUNC_ALIAS(AbortOldBackupTask, file_backup_restorable); + REGISTER_TASKFUNC_ALIAS(AbortOldBackupTask, file_finish_full_backup); + REGISTER_TASKFUNC_ALIAS(AbortOldBackupTask, file_finished_full_backup); + REGISTER_TASKFUNC_ALIAS(AbortOldBackupTask, file_start_full_backup); + std::function)> NOP_SETUP_TASK_FN = [](Reference task) { /* NOP */ }; ACTOR static Future addBackupTask(StringRef name, uint32_t version, @@ -1011,7 +1091,7 @@ namespace fileBackup { } }; - StringRef BackupRangeTaskFunc::name = LiteralStringRef("file_backup_range"); + StringRef BackupRangeTaskFunc::name = LiteralStringRef("file_backup_write_range"); const uint32_t BackupRangeTaskFunc::version = 1; REGISTER_TASKFUNC(BackupRangeTaskFunc); @@ -1237,11 +1317,11 @@ namespace fileBackup { // timeElapsed is between 0 and 1 and represents what portion of the shards we should have completed by now double timeElapsed; if(snapshotTargetEndVersion > snapshotBeginVersion) - timeElapsed = std::max(1.0, (double)(nextDispatchVersion - snapshotBeginVersion) / (snapshotTargetEndVersion - snapshotBeginVersion)); + timeElapsed = std::min(1.0, (double)(nextDispatchVersion - snapshotBeginVersion) / (snapshotTargetEndVersion - snapshotBeginVersion)); else timeElapsed = 1.0; - state int countExpectedShardsDone = std::min(countAllShards, countAllShards * timeElapsed); + state int countExpectedShardsDone = countAllShards * timeElapsed; state int countShardsToDispatch = std::max(0, countExpectedShardsDone - countShardsDone); TraceEvent("FileBackupSnapshotDispatchStats") @@ -1435,7 +1515,7 @@ namespace fileBackup { } }; - StringRef BackupSnapshotDispatchTask::name = LiteralStringRef("file_backup_snapshot_dispatch"); + StringRef BackupSnapshotDispatchTask::name = LiteralStringRef("file_backup_dispatch_ranges"); const uint32_t BackupSnapshotDispatchTask::version = 1; REGISTER_TASKFUNC(BackupSnapshotDispatchTask); @@ -1634,7 +1714,7 @@ namespace fileBackup { } }; - StringRef BackupLogRangeTaskFunc::name = LiteralStringRef("file_backup_log_range"); + StringRef BackupLogRangeTaskFunc::name = LiteralStringRef("file_backup_write_logs"); const uint32_t BackupLogRangeTaskFunc::version = 1; REGISTER_TASKFUNC(BackupLogRangeTaskFunc); @@ -1722,18 +1802,18 @@ namespace fileBackup { Future execute(Database cx, Reference tb, Reference fb, Reference task) { return Void(); }; Future finish(Reference tr, Reference tb, Reference fb, Reference task) { return _finish(tr, tb, fb, task); }; }; - StringRef BackupLogsDispatchTask::name = LiteralStringRef("file_backup_logs"); + StringRef BackupLogsDispatchTask::name = LiteralStringRef("file_backup_dispatch_logs"); const uint32_t BackupLogsDispatchTask::version = 1; REGISTER_TASKFUNC(BackupLogsDispatchTask); - struct FinishedFullBackupTaskFunc : BackupTaskFuncBase { + struct FileBackupFinishedTask : BackupTaskFuncBase { static StringRef name; static const uint32_t version; StringRef getName() const { return name; }; ACTOR static Future _finish(Reference tr, Reference taskBucket, Reference futureBucket, Reference task) { - Void _ = wait(checkTaskVersion(tr->getDatabase(), task, FinishedFullBackupTaskFunc::name, FinishedFullBackupTaskFunc::version)); + Void _ = wait(checkTaskVersion(tr->getDatabase(), task, FileBackupFinishedTask::name, FileBackupFinishedTask::version)); state BackupConfig backup(task); state UID uid = backup.getUid(); @@ -1754,8 +1834,8 @@ namespace fileBackup { } ACTOR static Future addTask(Reference tr, Reference taskBucket, Reference parentTask, TaskCompletionKey completionKey, Reference waitFor = Reference()) { - Key key = wait(addBackupTask(FinishedFullBackupTaskFunc::name, - FinishedFullBackupTaskFunc::version, + Key key = wait(addBackupTask(FileBackupFinishedTask::name, + FileBackupFinishedTask::version, tr, taskBucket, completionKey, BackupConfig(parentTask), waitFor)); return key; @@ -1764,13 +1844,9 @@ namespace fileBackup { Future execute(Database cx, Reference tb, Reference fb, Reference task) { return Void(); }; Future finish(Reference tr, Reference tb, Reference fb, Reference task) { return _finish(tr, tb, fb, task); }; }; - StringRef FinishedFullBackupTaskFunc::name = LiteralStringRef("file_finished_full_backup"); - const uint32_t FinishedFullBackupTaskFunc::version = 1; - REGISTER_TASKFUNC(FinishedFullBackupTaskFunc); - - // TODO: Register a task that will finish/delete any tasks of these types: - //LiteralStringRef("file_backup_diff_logs"); - //LiteralStringRef("file_finish_full_backup"); + StringRef FileBackupFinishedTask::name = LiteralStringRef("file_backup_finished"); + const uint32_t FileBackupFinishedTask::version = 1; + REGISTER_TASKFUNC(FileBackupFinishedTask); struct BackupSnapshotManifest : BackupTaskFuncBase { static StringRef name; @@ -1918,7 +1994,7 @@ namespace fileBackup { Future execute(Database cx, Reference tb, Reference fb, Reference task) { return _execute(cx, tb, fb, task); }; Future finish(Reference tr, Reference tb, Reference fb, Reference task) { return _finish(tr, tb, fb, task); }; }; - StringRef BackupSnapshotManifest::name = LiteralStringRef("file_backup_snapshot_manifest"); + StringRef BackupSnapshotManifest::name = LiteralStringRef("file_backup_write_snapshot_manifest"); const uint32_t BackupSnapshotManifest::version = 1; REGISTER_TASKFUNC(BackupSnapshotManifest); @@ -1978,7 +2054,7 @@ namespace fileBackup { // If a clean stop is requested, the log and snapshot tasks will quit after the backup is restorable, then the following // task will clean up and set the completed state. - Key _ = wait(FinishedFullBackupTaskFunc::addTask(tr, taskBucket, task, TaskCompletionKey::noSignal(), backupFinished)); + Key _ = wait(FileBackupFinishedTask::addTask(tr, taskBucket, task, TaskCompletionKey::noSignal(), backupFinished)); Void _ = wait(taskBucket->finish(tr, task)); return Void(); @@ -1998,7 +2074,7 @@ namespace fileBackup { Future execute(Database cx, Reference tb, Reference fb, Reference task) { return _execute(cx, tb, fb, task); }; Future finish(Reference tr, Reference tb, Reference fb, Reference task) { return _finish(tr, tb, fb, task); }; }; - StringRef StartFullBackupTaskFunc::name = LiteralStringRef("file_start_full_backup"); + StringRef StartFullBackupTaskFunc::name = LiteralStringRef("file_backup_start"); const uint32_t StartFullBackupTaskFunc::version = 1; REGISTER_TASKFUNC(StartFullBackupTaskFunc); diff --git a/fdbclient/Knobs.cpp b/fdbclient/Knobs.cpp index 64bd546cdb..f5ff24aa85 100644 --- a/fdbclient/Knobs.cpp +++ b/fdbclient/Knobs.cpp @@ -145,9 +145,11 @@ ClientKnobs::ClientKnobs(bool randomize) { init( BLOBSTORE_CONNECT_TIMEOUT, 10 ); init( BLOBSTORE_MAX_CONNECTION_LIFE, 120 ); init( BLOBSTORE_REQUEST_TRIES, 10 ); - init( BLOBSTORE_REQUEST_TIMEOUT, 30 ); + init( BLOBSTORE_REQUEST_TIMEOUT, 60 ); init( BLOBSTORE_CONCURRENT_UPLOADS, BACKUP_TASKS_PER_AGENT*2 ); + init( BLOBSTORE_CONCURRENT_LISTS, 20 ); + init( BLOBSTORE_CONCURRENT_REQUESTS, BLOBSTORE_CONCURRENT_UPLOADS + BLOBSTORE_CONCURRENT_LISTS + 5); init( BLOBSTORE_CONCURRENT_WRITES_PER_FILE, 5 ); init( BLOBSTORE_CONCURRENT_READS_PER_FILE, 3 ); @@ -159,7 +161,6 @@ ClientKnobs::ClientKnobs(bool randomize) { // These are basically unlimited by default but can be used to reduce blob IO if needed init( BLOBSTORE_REQUESTS_PER_SECOND, 200 ); - init( BLOBSTORE_CONCURRENT_REQUESTS, (BACKUP_TASKS_PER_AGENT*2)+5 ); init( BLOBSTORE_MAX_SEND_BYTES_PER_SECOND, 1e9 ); init( BLOBSTORE_MAX_RECV_BYTES_PER_SECOND, 1e9 ); diff --git a/fdbclient/Knobs.h b/fdbclient/Knobs.h index 4557cda9e3..5fc52988cd 100644 --- a/fdbclient/Knobs.h +++ b/fdbclient/Knobs.h @@ -155,6 +155,7 @@ public: int BLOBSTORE_MULTIPART_MAX_PART_SIZE; int BLOBSTORE_MULTIPART_MIN_PART_SIZE; int BLOBSTORE_CONCURRENT_UPLOADS; + int BLOBSTORE_CONCURRENT_LISTS; int BLOBSTORE_CONCURRENT_WRITES_PER_FILE; int BLOBSTORE_CONCURRENT_READS_PER_FILE; int BLOBSTORE_READ_BLOCK_SIZE; diff --git a/fdbclient/NativeAPI.actor.cpp b/fdbclient/NativeAPI.actor.cpp index 6695ec9dfd..2761c497eb 100644 --- a/fdbclient/NativeAPI.actor.cpp +++ b/fdbclient/NativeAPI.actor.cpp @@ -490,7 +490,7 @@ ACTOR static Future monitorClientInfo( ReferencefileContentsUpToDate(fileConnectionString)) { - req.issues = LiteralStringRef("unable_to_write_cluster_file"); + req.issues = LiteralStringRef("incorrect_cluster_file_contents"); if(ccf->canGetFilename()) { TraceEvent(SevWarnAlways, "IncorrectClusterFileContents").detail("Filename", ccf->getFilename()) .detail("ConnectionStringFromFile", fileConnectionString.toString()) diff --git a/fdbclient/StatusClient.actor.cpp b/fdbclient/StatusClient.actor.cpp index d1e2b9da68..66841502bc 100644 --- a/fdbclient/StatusClient.actor.cpp +++ b/fdbclient/StatusClient.actor.cpp @@ -305,12 +305,13 @@ ACTOR Future clientStatusFetcher(Reference statusObj["cluster_file"] = statusObjClusterFile; if (!contentsUpToDate){ - std::string description = "Cluster file is not up to date.\nIt contains the connection string: "; + std::string description = "Cluster file contents do not match current cluster connection string."; + description += "\nThe file contains the connection string: "; description += ClusterConnectionFile(f->getFilename()).getConnectionString().toString().c_str(); - description += "\nThe original connection string is: "; + description += "\nThe current connection string is: "; description += f->getConnectionString().toString().c_str(); - description += "\nThis must mean that file permissions or other platform issues have prevented the file from being updated. To change coordinators without manual intervention, the cluster file and its containing folder must be writable by all servers and clients. If a majority of the coordinators referenced by the old connection string are lost, the database will stop working until the correct cluster file is distributed to all processes."; - messages->push_back(makeMessage("inconsistent_cluster_file", description.c_str())); + description += "\nVerify cluster file is writable and has not been overwritten externally. To change coordinators without manual intervention, the cluster file and its containing folder must be writable by all servers and clients. If a majority of the coordinators referenced by the old connection string are lost, the database will stop working until the correct cluster file is distributed to all processes."; + messages->push_back(makeMessage("incorrect_cluster_file_contents", description.c_str())); } return statusObj; diff --git a/fdbclient/TaskBucket.h b/fdbclient/TaskBucket.h index f9f13bfea5..714808fadc 100644 --- a/fdbclient/TaskBucket.h +++ b/fdbclient/TaskBucket.h @@ -391,6 +391,7 @@ struct TaskFuncBase : IDispatched, std::func }; }; #define REGISTER_TASKFUNC(TaskFunc) REGISTER_FACTORY(TaskFuncBase, TaskFunc, name) +#define REGISTER_TASKFUNC_ALIAS(TaskFunc, Alias) REGISTER_DISPATCHED_ALIAS(TaskFunc, Alias, TaskFunc::name, LiteralStringRef(#Alias)) struct TaskCompletionKey { Future get(Reference tr, Reference taskBucket); diff --git a/fdbclient/vexillographer/java.cs b/fdbclient/vexillographer/java.cs index fe7de3e228..abf0d38eb3 100644 --- a/fdbclient/vexillographer/java.cs +++ b/fdbclient/vexillographer/java.cs @@ -168,7 +168,6 @@ namespace vexillographer private static void writePredicateClass(TextWriter outFile, Scope scope, IEnumerable