Merge pull request #1625 from jzhou77/backup-worker
Create a backup role in the cluster
This commit is contained in:
commit
fe0e10c312
|
@ -4,6 +4,18 @@ set(FDBBACKUP_SRCS
|
|||
add_flow_target(EXECUTABLE NAME fdbbackup SRCS ${FDBBACKUP_SRCS})
|
||||
target_link_libraries(fdbbackup PRIVATE fdbclient)
|
||||
|
||||
set(FDBCONVERT_SRCS
|
||||
FileConverter.actor.cpp
|
||||
FileConverter.h)
|
||||
add_flow_target(EXECUTABLE NAME fdbconvert SRCS ${FDBCONVERT_SRCS})
|
||||
target_link_libraries(fdbconvert PRIVATE fdbclient)
|
||||
|
||||
set(FDBDECODE_SRCS
|
||||
FileDecoder.actor.cpp
|
||||
FileConverter.h)
|
||||
add_flow_target(EXECUTABLE NAME fdbdecode SRCS ${FDBDECODE_SRCS})
|
||||
target_link_libraries(fdbdecode PRIVATE fdbclient)
|
||||
|
||||
if(NOT OPEN_FOR_IDE)
|
||||
fdb_install(TARGETS fdbbackup DESTINATION bin COMPONENT clients)
|
||||
install_symlink(
|
||||
|
|
|
@ -0,0 +1,596 @@
|
|||
/*
|
||||
* FileConverter.actor.cpp
|
||||
*
|
||||
* This source file is part of the FoundationDB open source project
|
||||
*
|
||||
* Copyright 2013-2019 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.
|
||||
*/
|
||||
|
||||
#include "fdbbackup/FileConverter.h"
|
||||
|
||||
#include <algorithm>
|
||||
#include <iostream>
|
||||
#include <cinttypes>
|
||||
#include <cstdio>
|
||||
#include <vector>
|
||||
|
||||
#include "fdbclient/BackupAgent.actor.h"
|
||||
#include "fdbclient/BackupContainer.h"
|
||||
#include "fdbclient/MutationList.h"
|
||||
#include "flow/flow.h"
|
||||
#include "flow/serialize.h"
|
||||
|
||||
namespace file_converter {
|
||||
|
||||
void printConvertUsage() {
|
||||
std::cout << "\n"
|
||||
<< " -r, --container Container URL.\n"
|
||||
<< " -b, --begin BEGIN\n"
|
||||
<< " Begin version.\n"
|
||||
<< " -e, --end END End version.\n"
|
||||
<< " --log Enables trace file logging for the CLI session.\n"
|
||||
<< " --logdir PATH Specifes the output directory for trace files. If\n"
|
||||
<< " unspecified, defaults to the current directory. Has\n"
|
||||
<< " no effect unless --log is specified.\n"
|
||||
<< " --loggroup LOG_GROUP\n"
|
||||
<< " Sets the LogGroup field with the specified value for all\n"
|
||||
<< " events in the trace output (defaults to `default').\n"
|
||||
<< " --trace_format FORMAT\n"
|
||||
<< " Select the format of the trace files. xml (the default) and json are supported.\n"
|
||||
<< " Has no effect unless --log is specified.\n"
|
||||
<< " -h, --help Display this help and exit.\n"
|
||||
<< "\n";
|
||||
|
||||
return;
|
||||
}
|
||||
|
||||
void printLogFiles(std::string msg, const std::vector<LogFile>& files) {
|
||||
std::cout << msg << " " << files.size() << " log files\n";
|
||||
for (const auto& file : files) {
|
||||
std::cout << file.toString() << "\n";
|
||||
}
|
||||
std::cout << std::endl;
|
||||
}
|
||||
|
||||
std::vector<LogFile> getRelevantLogFiles(const std::vector<LogFile>& files, Version begin, Version end) {
|
||||
std::vector<LogFile> filtered;
|
||||
for (const auto& file : files) {
|
||||
if (file.beginVersion <= end && file.endVersion >= begin && file.tagId >= 0) {
|
||||
filtered.push_back(file);
|
||||
}
|
||||
}
|
||||
std::sort(filtered.begin(), filtered.end());
|
||||
|
||||
// Remove duplicates. This is because backup workers may store the log for
|
||||
// old epochs successfully, but do not update the progress before another
|
||||
// recovery happened. As a result, next epoch will retry and creates
|
||||
// duplicated log files.
|
||||
std::vector<LogFile> sorted;
|
||||
int i = 0;
|
||||
for (int j = 1; j < filtered.size(); j++) {
|
||||
if (!filtered[i].sameContent(filtered[j])) {
|
||||
sorted.push_back(filtered[i]);
|
||||
i = j;
|
||||
}
|
||||
}
|
||||
if (i < filtered.size()) {
|
||||
sorted.push_back(filtered[i]);
|
||||
}
|
||||
|
||||
return sorted;
|
||||
}
|
||||
|
||||
struct ConvertParams {
|
||||
std::string container_url;
|
||||
Version begin = invalidVersion;
|
||||
Version end = invalidVersion;
|
||||
bool log_enabled = false;
|
||||
std::string log_dir, trace_format, trace_log_group;
|
||||
|
||||
bool isValid() { return begin != invalidVersion && end != invalidVersion && !container_url.empty(); }
|
||||
|
||||
std::string toString() {
|
||||
std::string s;
|
||||
s.append("ContainerURL:");
|
||||
s.append(container_url);
|
||||
s.append(" Begin:");
|
||||
s.append(format("%" PRId64, begin));
|
||||
s.append(" End:");
|
||||
s.append(format("%" PRId64, end));
|
||||
if (log_enabled) {
|
||||
if (!log_dir.empty()) {
|
||||
s.append(" LogDir:").append(log_dir);
|
||||
}
|
||||
if (!trace_format.empty()) {
|
||||
s.append(" Format:").append(trace_format);
|
||||
}
|
||||
if (!trace_log_group.empty()) {
|
||||
s.append(" LogGroup:").append(trace_log_group);
|
||||
}
|
||||
}
|
||||
return s;
|
||||
}
|
||||
};
|
||||
|
||||
struct VersionedData {
|
||||
LogMessageVersion version;
|
||||
StringRef message; // Serialized mutation.
|
||||
Arena arena; // The arena that contains mutation.
|
||||
|
||||
VersionedData() : version(invalidVersion, -1) {}
|
||||
VersionedData(LogMessageVersion v, StringRef m, Arena a) : version(v), message(m), arena(a) {}
|
||||
};
|
||||
|
||||
struct MutationFilesReadProgress : public ReferenceCounted<MutationFilesReadProgress> {
|
||||
MutationFilesReadProgress(std::vector<LogFile>& logs, Version begin, Version end)
|
||||
: files(logs), beginVersion(begin), endVersion(end) {}
|
||||
|
||||
struct FileProgress : public ReferenceCounted<FileProgress> {
|
||||
FileProgress(Reference<IAsyncFile> f, int index) : fd(f), idx(index), offset(0), eof(false) {}
|
||||
|
||||
bool operator<(const FileProgress& rhs) const {
|
||||
if (rhs.mutations.empty()) return true;
|
||||
if (mutations.empty()) return false;
|
||||
return mutations[0].version < rhs.mutations[0].version;
|
||||
}
|
||||
bool operator<=(const FileProgress& rhs) const {
|
||||
if (rhs.mutations.empty()) return true;
|
||||
if (mutations.empty()) return false;
|
||||
return mutations[0].version <= rhs.mutations[0].version;
|
||||
}
|
||||
bool empty() { return eof && mutations.empty(); }
|
||||
|
||||
// Decodes the block into mutations and save them if >= minVersion and < maxVersion.
|
||||
// Returns true if new mutations has been saved.
|
||||
bool decodeBlock(const Standalone<StringRef>& buf, int len, Version minVersion, Version maxVersion) {
|
||||
StringRef block(buf.begin(), len);
|
||||
StringRefReader reader(block, restore_corrupted_data());
|
||||
int count = 0, inserted = 0;
|
||||
Version msgVersion = invalidVersion;
|
||||
|
||||
try {
|
||||
while (1) {
|
||||
// If eof reached or first key len bytes is 0xFF then end of block was reached.
|
||||
if (reader.eof() || *reader.rptr == 0xFF) break;
|
||||
|
||||
// Deserialize messages written in saveMutationsToFile().
|
||||
msgVersion = bigEndian64(reader.consume<Version>());
|
||||
uint32_t sub = bigEndian32(reader.consume<uint32_t>());
|
||||
int msgSize = bigEndian32(reader.consume<int>());
|
||||
const uint8_t* message = reader.consume(msgSize);
|
||||
|
||||
BinaryReader rd(message, msgSize, AssumeVersion(currentProtocolVersion));
|
||||
MutationRef m;
|
||||
rd >> m;
|
||||
count++;
|
||||
if (msgVersion >= maxVersion) {
|
||||
TraceEvent("FileDecodeEnd")
|
||||
.detail("MaxV", maxVersion)
|
||||
.detail("Version", msgVersion)
|
||||
.detail("File", fd->getFilename());
|
||||
eof = true;
|
||||
break; // skip
|
||||
}
|
||||
if (msgVersion >= minVersion) {
|
||||
mutations.emplace_back(LogMessageVersion(msgVersion, sub), StringRef(message, msgSize),
|
||||
buf.arena());
|
||||
inserted++;
|
||||
}
|
||||
}
|
||||
offset += len;
|
||||
|
||||
TraceEvent("Decoded")
|
||||
.detail("Name", fd->getFilename())
|
||||
.detail("Count", count)
|
||||
.detail("Insert", inserted)
|
||||
.detail("BlockOffset", reader.rptr - buf.begin())
|
||||
.detail("Total", mutations.size())
|
||||
.detail("EOF", eof)
|
||||
.detail("Version", msgVersion)
|
||||
.detail("NewOffset", offset);
|
||||
return inserted > 0;
|
||||
} catch (Error& e) {
|
||||
TraceEvent(SevWarn, "CorruptLogFileBlock")
|
||||
.error(e)
|
||||
.detail("Filename", fd->getFilename())
|
||||
.detail("BlockOffset", offset)
|
||||
.detail("BlockLen", len)
|
||||
.detail("ErrorRelativeOffset", reader.rptr - buf.begin())
|
||||
.detail("ErrorAbsoluteOffset", reader.rptr - buf.begin() + offset);
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
Reference<IAsyncFile> fd;
|
||||
int idx; // index in the MutationFilesReadProgress::files vector
|
||||
int64_t offset; // offset of the file to be read
|
||||
bool eof; // If EOF is seen so far or endVersion is encountered. If true, the file can't be read further.
|
||||
std::vector<VersionedData> mutations; // Buffered mutations read so far
|
||||
};
|
||||
|
||||
bool hasMutations() {
|
||||
for (const auto& fp : fileProgress) {
|
||||
if (!fp->empty()) return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
void dumpProgress(std::string msg) {
|
||||
std::cout << msg << "\n ";
|
||||
for (const auto fp : fileProgress) {
|
||||
std::cout << fp->fd->getFilename() << " " << fp->mutations.size() << " mutations";
|
||||
if (fp->mutations.size() > 0) {
|
||||
std::cout << ", range " << fp->mutations[0].version.toString() << " "
|
||||
<< fp->mutations.back().version.toString() << "\n";
|
||||
} else {
|
||||
std::cout << "\n\n";
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Sorts files according to their first mutation version and removes files without mutations.
|
||||
void sortAndRemoveEmpty() {
|
||||
std::sort(fileProgress.begin(), fileProgress.end(),
|
||||
[](const Reference<FileProgress>& a, const Reference<FileProgress>& b) { return (*a) < (*b); });
|
||||
while (!fileProgress.empty() && fileProgress.back()->empty()) {
|
||||
fileProgress.pop_back();
|
||||
}
|
||||
}
|
||||
|
||||
// Requires hasMutations() return true before calling this function.
|
||||
// The caller must hold on the the arena associated with the mutation.
|
||||
Future<VersionedData> getNextMutation() { return getMutationImpl(this); }
|
||||
|
||||
ACTOR static Future<VersionedData> getMutationImpl(MutationFilesReadProgress* self) {
|
||||
ASSERT(!self->fileProgress.empty() && !self->fileProgress[0]->mutations.empty());
|
||||
|
||||
state Reference<FileProgress> fp = self->fileProgress[0];
|
||||
state VersionedData data = fp->mutations[0];
|
||||
fp->mutations.erase(fp->mutations.begin());
|
||||
if (fp->mutations.empty()) {
|
||||
// decode one more block
|
||||
wait(decodeToVersion(fp, /*version=*/0, self->endVersion, self->getLogFile(fp->idx)));
|
||||
}
|
||||
|
||||
if (fp->empty()) {
|
||||
self->fileProgress.erase(self->fileProgress.begin());
|
||||
} else {
|
||||
// Keep fileProgress sorted
|
||||
for (int i = 1; i < self->fileProgress.size(); i++) {
|
||||
if (*self->fileProgress[i - 1] <= *self->fileProgress[i]) {
|
||||
break;
|
||||
}
|
||||
std::swap(self->fileProgress[i - 1], self->fileProgress[i]);
|
||||
}
|
||||
}
|
||||
return data;
|
||||
}
|
||||
|
||||
LogFile& getLogFile(int index) { return files[index]; }
|
||||
|
||||
Future<Void> openLogFiles(Reference<IBackupContainer> container) { return openLogFilesImpl(this, container); }
|
||||
|
||||
// Opens log files in the progress and starts decoding until the beginVersion is seen.
|
||||
ACTOR static Future<Void> openLogFilesImpl(MutationFilesReadProgress* progress,
|
||||
Reference<IBackupContainer> container) {
|
||||
state std::vector<Future<Reference<IAsyncFile>>> asyncFiles;
|
||||
for (const auto& file : progress->files) {
|
||||
asyncFiles.push_back(container->readFile(file.fileName));
|
||||
}
|
||||
wait(waitForAll(asyncFiles)); // open all files
|
||||
|
||||
// Attempt decode the first few blocks of log files until beginVersion is consumed
|
||||
std::vector<Future<Void>> fileDecodes;
|
||||
for (int i = 0; i < asyncFiles.size(); i++) {
|
||||
Reference<FileProgress> fp(new FileProgress(asyncFiles[i].get(), i));
|
||||
progress->fileProgress.push_back(fp);
|
||||
fileDecodes.push_back(
|
||||
decodeToVersion(fp, progress->beginVersion, progress->endVersion, progress->getLogFile(i)));
|
||||
}
|
||||
|
||||
wait(waitForAll(fileDecodes));
|
||||
|
||||
progress->sortAndRemoveEmpty();
|
||||
|
||||
return Void();
|
||||
}
|
||||
|
||||
// Decodes the file until EOF or an mutation >= minVersion and saves these mutations.
|
||||
// Skip mutations >= maxVersion.
|
||||
ACTOR static Future<Void> decodeToVersion(Reference<FileProgress> fp, Version minVersion, Version maxVersion,
|
||||
LogFile file) {
|
||||
if (fp->empty()) return Void();
|
||||
|
||||
if (!fp->mutations.empty() && fp->mutations.back().version.version >= minVersion) return Void();
|
||||
|
||||
try {
|
||||
// Read block by block until we see the minVersion
|
||||
loop {
|
||||
state int64_t len = std::min<int64_t>(file.blockSize, file.fileSize - fp->offset);
|
||||
if (len == 0) {
|
||||
fp->eof = true;
|
||||
return Void();
|
||||
}
|
||||
|
||||
state Standalone<StringRef> buf = makeString(len);
|
||||
int rLen = wait(fp->fd->read(mutateString(buf), len, fp->offset));
|
||||
if (len != rLen) throw restore_bad_read();
|
||||
|
||||
TraceEvent("ReadFile")
|
||||
.detail("Name", fp->fd->getFilename())
|
||||
.detail("Length", rLen)
|
||||
.detail("Offset", fp->offset);
|
||||
if (fp->decodeBlock(buf, rLen, minVersion, maxVersion)) break;
|
||||
}
|
||||
return Void();
|
||||
} catch (Error& e) {
|
||||
TraceEvent(SevWarn, "CorruptedLogFileBlock")
|
||||
.error(e)
|
||||
.detail("Filename", fp->fd->getFilename())
|
||||
.detail("BlockOffset", fp->offset)
|
||||
.detail("BlockLen", len);
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
std::vector<LogFile> files;
|
||||
const Version beginVersion, endVersion;
|
||||
std::vector<Reference<FileProgress>> fileProgress;
|
||||
};
|
||||
|
||||
// Writes a log file in the old backup format, described in backup-dataFormat.md.
|
||||
// This is similar to the LogFileWriter in FileBackupAgent.actor.cpp.
|
||||
struct LogFileWriter {
|
||||
LogFileWriter() : blockSize(-1) {}
|
||||
LogFileWriter(Reference<IBackupFile> f, int bsize) : file(f), blockSize(bsize) {}
|
||||
|
||||
// Returns the block key, i.e., `Param1`, in the back file. The format is
|
||||
// `hash_value|commitVersion|part`.
|
||||
static Standalone<StringRef> getBlockKey(Version commitVersion, int part) {
|
||||
const int32_t version = commitVersion / CLIENT_KNOBS->LOG_RANGE_BLOCK_SIZE;
|
||||
|
||||
BinaryWriter wr(Unversioned());
|
||||
wr << (uint8_t)hashlittle(&version, sizeof(version), 0);
|
||||
wr << bigEndian64(commitVersion);
|
||||
wr << bigEndian32(part);
|
||||
return wr.toValue();
|
||||
}
|
||||
|
||||
// Return a block of contiguous padding bytes, growing if needed.
|
||||
static Value makePadding(int size) {
|
||||
static Value pad;
|
||||
if (pad.size() < size) {
|
||||
pad = makeString(size);
|
||||
memset(mutateString(pad), '\xff', pad.size());
|
||||
}
|
||||
|
||||
return pad.substr(0, size);
|
||||
}
|
||||
|
||||
// Start a new block if needed, then write the key and value
|
||||
ACTOR static Future<Void> writeKV_impl(LogFileWriter* self, Key k, Value v) {
|
||||
// If key and value do not fit in this block, end it and start a new one
|
||||
int toWrite = sizeof(int32_t) + k.size() + sizeof(int32_t) + v.size();
|
||||
if (self->file->size() + toWrite > self->blockEnd) {
|
||||
// Write padding if needed
|
||||
int bytesLeft = self->blockEnd - self->file->size();
|
||||
if (bytesLeft > 0) {
|
||||
state Value paddingFFs = makePadding(bytesLeft);
|
||||
wait(self->file->append(paddingFFs.begin(), bytesLeft));
|
||||
}
|
||||
|
||||
// Set new blockEnd
|
||||
self->blockEnd += self->blockSize;
|
||||
|
||||
// write Header
|
||||
wait(self->file->append((uint8_t*)&BACKUP_AGENT_MLOG_VERSION, sizeof(BACKUP_AGENT_MLOG_VERSION)));
|
||||
}
|
||||
|
||||
wait(self->file->appendStringRefWithLen(k));
|
||||
wait(self->file->appendStringRefWithLen(v));
|
||||
|
||||
// At this point we should be in whatever the current block is or the block size is too small
|
||||
if (self->file->size() > self->blockEnd) throw backup_bad_block_size();
|
||||
|
||||
return Void();
|
||||
}
|
||||
|
||||
Future<Void> writeKV(Key k, Value v) { return writeKV_impl(this, k, v); }
|
||||
|
||||
// Adds a new mutation to an interal buffer and writes out when encountering
|
||||
// a new commitVersion or exceeding the block size.
|
||||
ACTOR static Future<Void> addMutation(LogFileWriter* self, Version commitVersion, MutationListRef mutations) {
|
||||
state Standalone<StringRef> value = BinaryWriter::toValue(mutations, IncludeVersion());
|
||||
|
||||
state int part = 0;
|
||||
for (; part * CLIENT_KNOBS->MUTATION_BLOCK_SIZE < value.size(); part++) {
|
||||
StringRef partBuf = value.substr(
|
||||
part * CLIENT_KNOBS->MUTATION_BLOCK_SIZE,
|
||||
std::min(value.size() - part * CLIENT_KNOBS->MUTATION_BLOCK_SIZE, CLIENT_KNOBS->MUTATION_BLOCK_SIZE));
|
||||
Standalone<StringRef> key = getBlockKey(commitVersion, part);
|
||||
wait(writeKV_impl(self, key, partBuf));
|
||||
}
|
||||
return Void();
|
||||
}
|
||||
|
||||
private:
|
||||
Reference<IBackupFile> file;
|
||||
int blockSize;
|
||||
int64_t blockEnd = 0;
|
||||
};
|
||||
|
||||
ACTOR Future<Void> convert(ConvertParams params) {
|
||||
state Reference<IBackupContainer> container = IBackupContainer::openContainer(params.container_url);
|
||||
state BackupFileList listing = wait(container->dumpFileList());
|
||||
std::sort(listing.logs.begin(), listing.logs.end());
|
||||
TraceEvent("Container").detail("URL", params.container_url).detail("Logs", listing.logs.size());
|
||||
state BackupDescription desc = wait(container->describeBackup());
|
||||
std::cout << "\n" << desc.toString() << "\n";
|
||||
|
||||
// std::cout << "Using Protocol Version: 0x" << std::hex << currentProtocolVersion.version() << std::dec << "\n";
|
||||
|
||||
std::vector<LogFile> logs = getRelevantLogFiles(listing.logs, params.begin, params.end);
|
||||
printLogFiles("Range has", logs);
|
||||
|
||||
state Reference<MutationFilesReadProgress> progress(new MutationFilesReadProgress(logs, params.begin, params.end));
|
||||
|
||||
wait(progress->openLogFiles(container));
|
||||
|
||||
state int blockSize = CLIENT_KNOBS->BACKUP_LOGFILE_BLOCK_SIZE;
|
||||
state Reference<IBackupFile> outFile = wait(container->writeLogFile(params.begin, params.end, blockSize));
|
||||
state LogFileWriter logFile(outFile, blockSize);
|
||||
std::cout << "Output file: " << outFile->getFileName() << "\n";
|
||||
|
||||
state MutationList list;
|
||||
state Arena arena;
|
||||
state Version version = invalidVersion;
|
||||
while (progress->hasMutations()) {
|
||||
state VersionedData data = wait(progress->getNextMutation());
|
||||
|
||||
// emit a mutation batch to file when encounter a new version
|
||||
if (list.totalSize() > 0 && version != data.version.version) {
|
||||
wait(LogFileWriter::addMutation(&logFile, version, list));
|
||||
list = MutationList();
|
||||
arena = Arena();
|
||||
}
|
||||
|
||||
BinaryReader rd(data.message, AssumeVersion(currentProtocolVersion));
|
||||
MutationRef m;
|
||||
rd >> m;
|
||||
std::cout << data.version.toString() << " m = " << m.toString() << "\n";
|
||||
list.push_back_deep(arena, m);
|
||||
version = data.version.version;
|
||||
}
|
||||
if (list.totalSize() > 0) {
|
||||
wait(LogFileWriter::addMutation(&logFile, version, list));
|
||||
}
|
||||
|
||||
wait(outFile->finish());
|
||||
|
||||
return Void();
|
||||
}
|
||||
|
||||
int parseCommandLine(ConvertParams* param, CSimpleOpt* args) {
|
||||
while (args->Next()) {
|
||||
auto lastError = args->LastError();
|
||||
switch (lastError) {
|
||||
case SO_SUCCESS:
|
||||
break;
|
||||
|
||||
default:
|
||||
std::cerr << "ERROR: argument given for option: " << args->OptionText() << "\n";
|
||||
return FDB_EXIT_ERROR;
|
||||
break;
|
||||
}
|
||||
|
||||
int optId = args->OptionId();
|
||||
const char* arg = args->OptionArg();
|
||||
switch (optId) {
|
||||
case OPT_HELP:
|
||||
printConvertUsage();
|
||||
return FDB_EXIT_ERROR;
|
||||
|
||||
case OPT_BEGIN_VERSION:
|
||||
if (!sscanf(arg, "%" SCNd64, ¶m->begin)) {
|
||||
std::cerr << "ERROR: could not parse begin version " << arg << "\n";
|
||||
printConvertUsage();
|
||||
return FDB_EXIT_ERROR;
|
||||
}
|
||||
break;
|
||||
|
||||
case OPT_END_VERSION:
|
||||
if (!sscanf(arg, "%" SCNd64, ¶m->end)) {
|
||||
std::cerr << "ERROR: could not parse end version " << arg << "\n";
|
||||
printConvertUsage();
|
||||
return FDB_EXIT_ERROR;
|
||||
}
|
||||
break;
|
||||
|
||||
case OPT_CONTAINER:
|
||||
param->container_url = args->OptionArg();
|
||||
break;
|
||||
|
||||
case OPT_TRACE:
|
||||
param->log_enabled = true;
|
||||
break;
|
||||
|
||||
case OPT_TRACE_DIR:
|
||||
param->log_dir = args->OptionArg();
|
||||
break;
|
||||
|
||||
case OPT_TRACE_FORMAT:
|
||||
if (!validateTraceFormat(args->OptionArg())) {
|
||||
std::cerr << "ERROR: Unrecognized trace format " << args->OptionArg() << "\n";
|
||||
return FDB_EXIT_ERROR;
|
||||
}
|
||||
param->trace_format = args->OptionArg();
|
||||
break;
|
||||
|
||||
case OPT_TRACE_LOG_GROUP:
|
||||
param->trace_log_group = args->OptionArg();
|
||||
break;
|
||||
}
|
||||
}
|
||||
return FDB_EXIT_SUCCESS;
|
||||
}
|
||||
|
||||
} // namespace file_converter
|
||||
|
||||
int main(int argc, char** argv) {
|
||||
try {
|
||||
CSimpleOpt* args = new CSimpleOpt(argc, argv, file_converter::gConverterOptions, SO_O_EXACT);
|
||||
file_converter::ConvertParams param;
|
||||
int status = file_converter::parseCommandLine(¶m, args);
|
||||
std::cout << "Params: " << param.toString() << "\n";
|
||||
if (status != FDB_EXIT_SUCCESS || !param.isValid()) {
|
||||
file_converter::printConvertUsage();
|
||||
return status;
|
||||
}
|
||||
|
||||
if (param.log_enabled) {
|
||||
if (param.log_dir.empty()) {
|
||||
setNetworkOption(FDBNetworkOptions::TRACE_ENABLE);
|
||||
} else {
|
||||
setNetworkOption(FDBNetworkOptions::TRACE_ENABLE, StringRef(param.log_dir));
|
||||
}
|
||||
if (!param.trace_format.empty()) {
|
||||
setNetworkOption(FDBNetworkOptions::TRACE_FORMAT, StringRef(param.trace_format));
|
||||
}
|
||||
if (!param.trace_log_group.empty()) {
|
||||
setNetworkOption(FDBNetworkOptions::TRACE_LOG_GROUP, StringRef(param.trace_log_group));
|
||||
}
|
||||
}
|
||||
|
||||
platformInit();
|
||||
Error::init();
|
||||
|
||||
StringRef url(param.container_url);
|
||||
setupNetwork(0, true);
|
||||
|
||||
TraceEvent::setNetworkThread();
|
||||
openTraceFile(NetworkAddress(), 10 << 20, 10 << 20, param.log_dir, "convert", param.trace_log_group);
|
||||
|
||||
auto f = stopAfter(convert(param));
|
||||
|
||||
runNetwork();
|
||||
return status;
|
||||
} catch (Error& e) {
|
||||
fprintf(stderr, "ERROR: %s\n", e.what());
|
||||
return FDB_EXIT_ERROR;
|
||||
} catch (std::exception& e) {
|
||||
TraceEvent(SevError, "MainError").error(unknown_error()).detail("RootException", e.what());
|
||||
return FDB_EXIT_MAIN_EXCEPTION;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,65 @@
|
|||
/*
|
||||
* FileConverter.h
|
||||
*
|
||||
* This source file is part of the FoundationDB open source project
|
||||
*
|
||||
* Copyright 2013-2019 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.
|
||||
*/
|
||||
|
||||
#ifndef FDBBACKUP_FILECONVERTER_H
|
||||
#define FDBBACKUP_FILECONVERTER_H
|
||||
#pragma once
|
||||
|
||||
#include <cinttypes>
|
||||
#include "flow/SimpleOpt.h"
|
||||
|
||||
namespace file_converter {
|
||||
|
||||
// File format convertion constants
|
||||
enum {
|
||||
OPT_CONTAINER,
|
||||
OPT_BEGIN_VERSION,
|
||||
OPT_END_VERSION,
|
||||
OPT_TRACE,
|
||||
OPT_TRACE_DIR,
|
||||
OPT_TRACE_FORMAT,
|
||||
OPT_TRACE_LOG_GROUP,
|
||||
OPT_INPUT_FILE,
|
||||
OPT_HELP
|
||||
};
|
||||
|
||||
CSimpleOpt::SOption gConverterOptions[] = { { OPT_CONTAINER, "-r", SO_REQ_SEP },
|
||||
{ OPT_CONTAINER, "--container", SO_REQ_SEP },
|
||||
{ OPT_BEGIN_VERSION, "-b", SO_REQ_SEP },
|
||||
{ OPT_BEGIN_VERSION, "--begin", SO_REQ_SEP },
|
||||
{ OPT_END_VERSION, "-e", SO_REQ_SEP },
|
||||
{ OPT_END_VERSION, "--end", SO_REQ_SEP },
|
||||
{ OPT_TRACE, "--log", SO_NONE },
|
||||
{ OPT_TRACE_DIR, "--logdir", SO_REQ_SEP },
|
||||
{ OPT_TRACE_FORMAT, "--trace_format", SO_REQ_SEP },
|
||||
{ OPT_TRACE_LOG_GROUP, "--loggroup", SO_REQ_SEP },
|
||||
{ OPT_INPUT_FILE, "-i", SO_REQ_SEP },
|
||||
{ OPT_INPUT_FILE, "--input", SO_REQ_SEP },
|
||||
{ OPT_HELP, "-?", SO_NONE },
|
||||
{ OPT_HELP, "-h", SO_NONE },
|
||||
{ OPT_HELP, "--help", SO_NONE },
|
||||
SO_END_OF_OPTIONS };
|
||||
|
||||
} // namespace file_converter
|
||||
|
||||
// Mutation log version written by old FileBackupAgent
|
||||
static const uint32_t BACKUP_AGENT_MLOG_VERSION = 2001;
|
||||
|
||||
#endif // FDBBACKUP_FILECONVERTER_H
|
|
@ -0,0 +1,440 @@
|
|||
/*
|
||||
* FileDecoder.actor.cpp
|
||||
*
|
||||
* This source file is part of the FoundationDB open source project
|
||||
*
|
||||
* Copyright 2013-2019 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.
|
||||
*/
|
||||
|
||||
#include <algorithm>
|
||||
#include <iostream>
|
||||
#include <vector>
|
||||
|
||||
#include "fdbclient/BackupAgent.actor.h"
|
||||
#include "fdbclient/BackupContainer.h"
|
||||
#include "fdbbackup/FileConverter.h"
|
||||
#include "fdbclient/MutationList.h"
|
||||
#include "flow/flow.h"
|
||||
#include "flow/serialize.h"
|
||||
|
||||
namespace file_converter {
|
||||
|
||||
void printDecodeUsage() {
|
||||
std::cout << "\n"
|
||||
" -r, --container Container URL.\n"
|
||||
" -i, --input FILE Log file to be decoded.\n"
|
||||
"\n";
|
||||
return;
|
||||
}
|
||||
|
||||
struct DecodeParams {
|
||||
std::string container_url;
|
||||
std::string file;
|
||||
bool log_enabled = false;
|
||||
std::string log_dir, trace_format, trace_log_group;
|
||||
|
||||
std::string toString() {
|
||||
std::string s;
|
||||
s.append("ContainerURL: ");
|
||||
s.append(container_url);
|
||||
s.append(", File: ");
|
||||
s.append(file);
|
||||
if (log_enabled) {
|
||||
if (!log_dir.empty()) {
|
||||
s.append(" LogDir:").append(log_dir);
|
||||
}
|
||||
if (!trace_format.empty()) {
|
||||
s.append(" Format:").append(trace_format);
|
||||
}
|
||||
if (!trace_log_group.empty()) {
|
||||
s.append(" LogGroup:").append(trace_log_group);
|
||||
}
|
||||
}
|
||||
return s;
|
||||
}
|
||||
};
|
||||
|
||||
int parseDecodeCommandLine(DecodeParams* param, CSimpleOpt* args) {
|
||||
while (args->Next()) {
|
||||
auto lastError = args->LastError();
|
||||
switch (lastError) {
|
||||
case SO_SUCCESS:
|
||||
break;
|
||||
|
||||
default:
|
||||
std::cerr << "ERROR: argument given for option: " << args->OptionText() << "\n";
|
||||
return FDB_EXIT_ERROR;
|
||||
break;
|
||||
}
|
||||
int optId = args->OptionId();
|
||||
switch (optId) {
|
||||
case OPT_HELP:
|
||||
printDecodeUsage();
|
||||
return FDB_EXIT_ERROR;
|
||||
|
||||
case OPT_CONTAINER:
|
||||
param->container_url = args->OptionArg();
|
||||
break;
|
||||
|
||||
case OPT_INPUT_FILE:
|
||||
param->file = args->OptionArg();
|
||||
break;
|
||||
|
||||
case OPT_TRACE:
|
||||
param->log_enabled = true;
|
||||
break;
|
||||
|
||||
case OPT_TRACE_DIR:
|
||||
param->log_dir = args->OptionArg();
|
||||
break;
|
||||
|
||||
case OPT_TRACE_FORMAT:
|
||||
if (!validateTraceFormat(args->OptionArg())) {
|
||||
std::cerr << "ERROR: Unrecognized trace format " << args->OptionArg() << "\n";
|
||||
return FDB_EXIT_ERROR;
|
||||
}
|
||||
param->trace_format = args->OptionArg();
|
||||
break;
|
||||
|
||||
case OPT_TRACE_LOG_GROUP:
|
||||
param->trace_log_group = args->OptionArg();
|
||||
break;
|
||||
}
|
||||
}
|
||||
return FDB_EXIT_SUCCESS;
|
||||
}
|
||||
|
||||
void printLogFiles(std::string msg, const std::vector<LogFile>& files) {
|
||||
std::cout << msg << " " << files.size() << " log files\n";
|
||||
for (const auto& file : files) {
|
||||
std::cout << file.toString() << "\n";
|
||||
}
|
||||
std::cout << std::endl;
|
||||
}
|
||||
|
||||
std::vector<LogFile> getRelevantLogFiles(const std::vector<LogFile>& files, const DecodeParams& params) {
|
||||
std::vector<LogFile> filtered;
|
||||
for (const auto& file : files) {
|
||||
if (file.fileName.find(params.file) != std::string::npos) {
|
||||
filtered.push_back(file);
|
||||
}
|
||||
}
|
||||
return filtered;
|
||||
}
|
||||
|
||||
std::pair<Version, int32_t> decode_key(const StringRef& key) {
|
||||
ASSERT(key.size() == sizeof(uint8_t) + sizeof(Version) + sizeof(int32_t));
|
||||
|
||||
uint8_t hash;
|
||||
Version version;
|
||||
int32_t part;
|
||||
BinaryReader rd(key, Unversioned());
|
||||
rd >> hash >> version >> part;
|
||||
version = bigEndian64(version);
|
||||
part = bigEndian32(part);
|
||||
|
||||
int32_t v = version / CLIENT_KNOBS->LOG_RANGE_BLOCK_SIZE;
|
||||
ASSERT(((uint8_t)hashlittle(&v, sizeof(v), 0)) == hash);
|
||||
|
||||
return std::make_pair(version, part);
|
||||
}
|
||||
|
||||
// Decodes an encoded list of mutations in the format of:
|
||||
// [includeVersion:uint64_t][val_length:uint32_t][mutation_1][mutation_2]...[mutation_k],
|
||||
// where a mutation is encoded as:
|
||||
// [type:uint32_t][keyLength:uint32_t][valueLength:uint32_t][key][value]
|
||||
std::vector<MutationRef> decode_value(const StringRef& value) {
|
||||
StringRefReader reader(value, restore_corrupted_data());
|
||||
|
||||
reader.consume<uint64_t>(); // Consume the includeVersion
|
||||
uint32_t val_length = reader.consume<uint32_t>();
|
||||
ASSERT(val_length == value.size() - sizeof(uint64_t) - sizeof(uint32_t));
|
||||
|
||||
std::vector<MutationRef> mutations;
|
||||
while (1) {
|
||||
if (reader.eof()) break;
|
||||
|
||||
// Deserialization of a MutationRef, which was packed by MutationListRef::push_back_deep()
|
||||
uint32_t type, p1len, p2len;
|
||||
type = reader.consume<uint32_t>();
|
||||
p1len = reader.consume<uint32_t>();
|
||||
p2len = reader.consume<uint32_t>();
|
||||
|
||||
const uint8_t* key = reader.consume(p1len);
|
||||
const uint8_t* val = reader.consume(p2len);
|
||||
|
||||
mutations.emplace_back((MutationRef::Type)type, StringRef(key, p1len), StringRef(val, p2len));
|
||||
}
|
||||
return mutations;
|
||||
}
|
||||
|
||||
struct VersionedMutations {
|
||||
Version version;
|
||||
std::vector<MutationRef> mutations;
|
||||
Arena arena; // The arena that contains the mutations.
|
||||
};
|
||||
|
||||
/*
|
||||
* Model a decoding progress for a mutation file. Usage is:
|
||||
*
|
||||
* DecodeProgress progress(logfile);
|
||||
* wait(progress->openFile(container));
|
||||
* while (!progress->finished()) {
|
||||
* VersionedMutations m = wait(progress->getNextBatch());
|
||||
* ...
|
||||
* }
|
||||
*
|
||||
* Internally, the decoding process is done block by block -- each block is
|
||||
* decoded into a list of key/value pairs, which are then decoded into batches
|
||||
* of mutations. Because a version's mutations can be split into many key/value
|
||||
* pairs, the decoding of mutation batch needs to look ahead one more pair. So
|
||||
* at any time this object might have two blocks of data in memory.
|
||||
*/
|
||||
struct DecodeProgress {
|
||||
DecodeProgress() = default;
|
||||
DecodeProgress(const LogFile& file) : file(file) {}
|
||||
|
||||
// If there are no more mutations to pull.
|
||||
bool finished() { return eof && keyValues.empty(); }
|
||||
|
||||
// Returns all mutations of the next version in a batch.
|
||||
Future<VersionedMutations> getNextBatch() { return getNextBatchImpl(this); }
|
||||
|
||||
Future<Void> openFile(Reference<IBackupContainer> container) { return openFileImpl(this, container); }
|
||||
|
||||
// The following are private APIs:
|
||||
|
||||
// PRECONDITION: finished() must return false before calling this function.
|
||||
// Returns the next batch of mutations along with the arena backing it.
|
||||
ACTOR static Future<VersionedMutations> getNextBatchImpl(DecodeProgress* self) {
|
||||
ASSERT(!self->finished());
|
||||
|
||||
state std::pair<Arena, KeyValueRef> arena_kv = self->keyValues[0];
|
||||
|
||||
// decode this batch's version
|
||||
state std::pair<Version, int32_t> version_part = decode_key(arena_kv.second.key);
|
||||
ASSERT(version_part.second == 0); // first part number must be 0.
|
||||
|
||||
// decode next versions, check if they are continuous parts
|
||||
state int idx = 1; // next kv pair in "keyValues"
|
||||
state int bufSize = arena_kv.second.value.size();
|
||||
state int lastPart = 0;
|
||||
loop {
|
||||
// Try to decode another block if needed
|
||||
if (idx == self->keyValues.size()) {
|
||||
wait(readAndDecodeFile(self));
|
||||
}
|
||||
if (idx == self->keyValues.size()) break;
|
||||
|
||||
std::pair<Version, int32_t> next_version_part = decode_key(self->keyValues[idx].second.key);
|
||||
if (version_part.first != next_version_part.first) break;
|
||||
|
||||
if (lastPart + 1 != next_version_part.second) {
|
||||
TraceEvent("DecodeError").detail("Part1", lastPart).detail("Part2", next_version_part.second);
|
||||
throw restore_corrupted_data();
|
||||
}
|
||||
bufSize += self->keyValues[idx].second.value.size();
|
||||
idx++;
|
||||
lastPart++;
|
||||
}
|
||||
|
||||
VersionedMutations m;
|
||||
m.version = version_part.first;
|
||||
if (idx > 1) {
|
||||
// Stitch parts into one and then decode one by one
|
||||
Standalone<StringRef> buf = self->combineValues(idx, bufSize);
|
||||
m.mutations = decode_value(buf);
|
||||
m.arena = buf.arena();
|
||||
} else {
|
||||
m.mutations = decode_value(arena_kv.second.value);
|
||||
m.arena = arena_kv.first;
|
||||
}
|
||||
self->keyValues.erase(self->keyValues.begin(), self->keyValues.begin() + idx);
|
||||
|
||||
return m;
|
||||
}
|
||||
|
||||
// Returns a buffer which stitches first "idx" values into one.
|
||||
// "len" MUST equal the summation of these values.
|
||||
Standalone<StringRef> combineValues(const int idx, const int len) {
|
||||
ASSERT(idx <= keyValues.size() && idx > 1);
|
||||
|
||||
Standalone<StringRef> buf = makeString(len);
|
||||
int n = 0;
|
||||
for (int i = 0; i < idx; i++) {
|
||||
const auto& value = keyValues[i].second.value;
|
||||
memcpy(mutateString(buf) + n, value.begin(), value.size());
|
||||
n += value.size();
|
||||
}
|
||||
|
||||
ASSERT(n == len);
|
||||
return buf;
|
||||
}
|
||||
|
||||
// Decodes a block into KeyValueRef stored in "keyValues".
|
||||
void decode_block(const Standalone<StringRef>& buf, int len) {
|
||||
StringRef block(buf.begin(), len);
|
||||
StringRefReader reader(block, restore_corrupted_data());
|
||||
|
||||
try {
|
||||
// Read header, currently only decoding version 2001
|
||||
if (reader.consume<int32_t>() != BACKUP_AGENT_MLOG_VERSION) throw restore_unsupported_file_version();
|
||||
|
||||
// Read k/v pairs. Block ends either at end of last value exactly or with 0xFF as first key len byte.
|
||||
while (1) {
|
||||
// If eof reached or first key len bytes is 0xFF then end of block was reached.
|
||||
if (reader.eof() || *reader.rptr == 0xFF) break;
|
||||
|
||||
// Read key and value. If anything throws then there is a problem.
|
||||
uint32_t kLen = reader.consumeNetworkUInt32();
|
||||
const uint8_t* k = reader.consume(kLen);
|
||||
uint32_t vLen = reader.consumeNetworkUInt32();
|
||||
const uint8_t* v = reader.consume(vLen);
|
||||
keyValues.emplace_back(buf.arena(), KeyValueRef(StringRef(k, kLen), StringRef(v, vLen)));
|
||||
}
|
||||
|
||||
// Make sure any remaining bytes in the block are 0xFF
|
||||
for (auto b : reader.remainder()) {
|
||||
if (b != 0xFF) throw restore_corrupted_data_padding();
|
||||
}
|
||||
|
||||
return;
|
||||
} catch (Error& e) {
|
||||
TraceEvent(SevWarn, "CorruptBlock").error(e).detail("Offset", reader.rptr - buf.begin());
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
ACTOR static Future<Void> openFileImpl(DecodeProgress* self, Reference<IBackupContainer> container) {
|
||||
Reference<IAsyncFile> fd = wait(container->readFile(self->file.fileName));
|
||||
self->fd = fd;
|
||||
wait(readAndDecodeFile(self));
|
||||
return Void();
|
||||
}
|
||||
|
||||
// Reads a file block, decodes it into key/value pairs, and stores these pairs.
|
||||
ACTOR static Future<Void> readAndDecodeFile(DecodeProgress* self) {
|
||||
try {
|
||||
state int64_t len = std::min<int64_t>(self->file.blockSize, self->file.fileSize - self->offset);
|
||||
if (len == 0) {
|
||||
self->eof = true;
|
||||
return Void();
|
||||
}
|
||||
|
||||
state Standalone<StringRef> buf = makeString(len);
|
||||
state int rLen = wait(self->fd->read(mutateString(buf), len, self->offset));
|
||||
TraceEvent("ReadFile")
|
||||
.detail("Name", self->file.fileName)
|
||||
.detail("Len", rLen)
|
||||
.detail("Offset", self->offset);
|
||||
if (rLen != len) {
|
||||
throw restore_corrupted_data();
|
||||
}
|
||||
self->decode_block(buf, rLen);
|
||||
self->offset += rLen;
|
||||
return Void();
|
||||
} catch (Error& e) {
|
||||
TraceEvent(SevWarn, "CorruptLogFileBlock")
|
||||
.error(e)
|
||||
.detail("Filename", self->file.fileName)
|
||||
.detail("BlockOffset", self->offset)
|
||||
.detail("BlockLen", self->file.blockSize);
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
LogFile file;
|
||||
Reference<IAsyncFile> fd;
|
||||
int64_t offset = 0;
|
||||
bool eof = false;
|
||||
// Key value pairs and their memory arenas.
|
||||
std::vector<std::pair<Arena, KeyValueRef>> keyValues;
|
||||
};
|
||||
|
||||
ACTOR Future<Void> decode_logs(DecodeParams params) {
|
||||
state Reference<IBackupContainer> container = IBackupContainer::openContainer(params.container_url);
|
||||
|
||||
state BackupFileList listing = wait(container->dumpFileList());
|
||||
std::sort(listing.logs.begin(), listing.logs.end());
|
||||
TraceEvent("Container").detail("URL", params.container_url).detail("Logs", listing.logs.size());
|
||||
|
||||
BackupDescription desc = wait(container->describeBackup());
|
||||
std::cout << "\n" << desc.toString() << "\n";
|
||||
|
||||
state std::vector<LogFile> logs = getRelevantLogFiles(listing.logs, params);
|
||||
printLogFiles("Relevant files are: ", logs);
|
||||
|
||||
state int i = 0;
|
||||
for (; i < logs.size(); i++) {
|
||||
state DecodeProgress progress(logs[i]);
|
||||
wait(progress.openFile(container));
|
||||
while (!progress.finished()) {
|
||||
VersionedMutations vms = wait(progress.getNextBatch());
|
||||
for (const auto& m : vms.mutations) {
|
||||
std::cout << vms.version << " " << m.toString() << "\n";
|
||||
}
|
||||
}
|
||||
}
|
||||
return Void();
|
||||
}
|
||||
|
||||
} // namespace file_converter
|
||||
|
||||
int main(int argc, char** argv) {
|
||||
try {
|
||||
CSimpleOpt* args = new CSimpleOpt(argc, argv, file_converter::gConverterOptions, SO_O_EXACT);
|
||||
file_converter::DecodeParams param;
|
||||
int status = file_converter::parseDecodeCommandLine(¶m, args);
|
||||
std::cout << "Params: " << param.toString() << "\n";
|
||||
if (status != FDB_EXIT_SUCCESS) {
|
||||
file_converter::printDecodeUsage();
|
||||
return status;
|
||||
}
|
||||
|
||||
if (param.log_enabled) {
|
||||
if (param.log_dir.empty()) {
|
||||
setNetworkOption(FDBNetworkOptions::TRACE_ENABLE);
|
||||
} else {
|
||||
setNetworkOption(FDBNetworkOptions::TRACE_ENABLE, StringRef(param.log_dir));
|
||||
}
|
||||
if (!param.trace_format.empty()) {
|
||||
setNetworkOption(FDBNetworkOptions::TRACE_FORMAT, StringRef(param.trace_format));
|
||||
}
|
||||
if (!param.trace_log_group.empty()) {
|
||||
setNetworkOption(FDBNetworkOptions::TRACE_LOG_GROUP, StringRef(param.trace_log_group));
|
||||
}
|
||||
}
|
||||
|
||||
platformInit();
|
||||
Error::init();
|
||||
|
||||
StringRef url(param.container_url);
|
||||
setupNetwork(0, true);
|
||||
|
||||
TraceEvent::setNetworkThread();
|
||||
openTraceFile(NetworkAddress(), 10 << 20, 10 << 20, param.log_dir, "decode", param.trace_log_group);
|
||||
|
||||
auto f = stopAfter(decode_logs(param));
|
||||
|
||||
runNetwork();
|
||||
return status;
|
||||
} catch (Error& e) {
|
||||
fprintf(stderr, "ERROR: %s\n", e.what());
|
||||
return FDB_EXIT_ERROR;
|
||||
} catch (std::exception& e) {
|
||||
TraceEvent(SevError, "MainError").error(unknown_error()).detail("RootException", e.what());
|
||||
return FDB_EXIT_MAIN_EXCEPTION;
|
||||
}
|
||||
}
|
|
@ -487,7 +487,7 @@ Standalone<VectorRef<KeyRangeRef>> getLogRanges(Version beginVersion, Version en
|
|||
Standalone<VectorRef<KeyRangeRef>> getApplyRanges(Version beginVersion, Version endVersion, Key backupUid);
|
||||
Future<Void> eraseLogData(Reference<ReadYourWritesTransaction> tr, Key logUidValue, Key destUidValue, Optional<Version> endVersion = Optional<Version>(), bool checkBackupUid = false, Version backupUid = 0);
|
||||
Key getApplyKey( Version version, Key backupUid );
|
||||
std::pair<uint64_t, uint32_t> decodeBKMutationLogKey(Key key);
|
||||
std::pair<Version, uint32_t> decodeBKMutationLogKey(Key key);
|
||||
Standalone<VectorRef<MutationRef>> decodeBackupLogValue(StringRef value);
|
||||
void decodeBackupLogValue(Arena& arena, VectorRef<MutationRef>& result, int64_t& mutationSize, StringRef value, StringRef addPrefix = StringRef(), StringRef removePrefix = StringRef());
|
||||
Future<Void> logError(Database cx, Key keyErrors, const std::string& message);
|
||||
|
@ -848,5 +848,38 @@ ACTOR Future<Version> fastRestore(Database cx, Standalone<StringRef> tagName, St
|
|||
bool waitForComplete, long targetVersion, bool verbose, Standalone<KeyRangeRef> range,
|
||||
Standalone<StringRef> addPrefix, Standalone<StringRef> removePrefix);
|
||||
|
||||
// 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 <typename T>
|
||||
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.
|
||||
const int32_t consumeNetworkInt32() { return (int32_t)bigEndian32((uint32_t)consume<int32_t>()); }
|
||||
const uint32_t consumeNetworkUInt32() { return bigEndian32(consume<uint32_t>()); }
|
||||
|
||||
bool eof() { return rptr == end; }
|
||||
|
||||
const uint8_t *rptr, *end;
|
||||
Error failure_error;
|
||||
};
|
||||
|
||||
#include "flow/unactorcompiler.h"
|
||||
#endif
|
||||
|
|
|
@ -131,7 +131,7 @@ bool copyParameter(Reference<Task> source, Reference<Task> dest, Key key) {
|
|||
}
|
||||
|
||||
Version getVersionFromString(std::string const& value) {
|
||||
Version version(-1);
|
||||
Version version = invalidVersion;
|
||||
int n = 0;
|
||||
if (sscanf(value.c_str(), "%lld%n", (long long*)&version, &n) != 1 || n != value.size()) {
|
||||
TraceEvent(SevWarnAlways, "GetVersionFromString").detail("InvalidVersion", value);
|
||||
|
@ -204,7 +204,7 @@ Key getApplyKey( Version version, Key backupUid ) {
|
|||
//returns(version, part) where version is the database version number of
|
||||
//the transaction log data in the value, and part is 0 for the first such
|
||||
//data for a given version, 1 for the second block of data, etc.
|
||||
std::pair<uint64_t, uint32_t> decodeBKMutationLogKey(Key key) {
|
||||
std::pair<Version, uint32_t> decodeBKMutationLogKey(Key key) {
|
||||
return std::make_pair(bigEndian64(*(int64_t*)(key.begin() + backupLogPrefixBytes + sizeof(UID) + sizeof(uint8_t))),
|
||||
bigEndian32(*(int32_t*)(key.begin() + backupLogPrefixBytes + sizeof(UID) + sizeof(uint8_t) + sizeof(int64_t))));
|
||||
}
|
||||
|
@ -379,7 +379,9 @@ void decodeBackupLogValue(Arena& arena, VectorRef<MutationRef>& result, int& mut
|
|||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
static double lastErrorTime = 0;
|
||||
|
||||
void logErrorWorker(Reference<ReadYourWritesTransaction> tr, Key keyErrors, std::string message) {
|
||||
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
|
||||
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
|
||||
|
|
|
@ -39,7 +39,7 @@
|
|||
|
||||
namespace IBackupFile_impl {
|
||||
|
||||
ACTOR Future<Void> appendStringRefWithLen(Reference<IBackupFile> file, Standalone<StringRef> s) {
|
||||
ACTOR Future<Void> appendStringRefWithLen(Reference<IBackupFile> file, Standalone<StringRef> s) {
|
||||
state uint32_t lenBuf = bigEndian32((uint32_t)s.size());
|
||||
wait(file->append(&lenBuf, sizeof(lenBuf)));
|
||||
wait(file->append(s.begin(), s.size()));
|
||||
|
@ -258,15 +258,15 @@ std::string BackupDescription::toJSON() const {
|
|||
*/
|
||||
class BackupContainerFileSystem : public IBackupContainer {
|
||||
public:
|
||||
virtual void addref() = 0;
|
||||
virtual void delref() = 0;
|
||||
void addref() override = 0;
|
||||
void delref() override = 0;
|
||||
|
||||
BackupContainerFileSystem() {}
|
||||
virtual ~BackupContainerFileSystem() {}
|
||||
|
||||
// Create the container
|
||||
virtual Future<Void> create() = 0;
|
||||
virtual Future<bool> exists() = 0;
|
||||
Future<Void> create() override = 0;
|
||||
Future<bool> exists() override = 0;
|
||||
|
||||
// Get a list of fileNames and their sizes in the container under the given path
|
||||
// Although not required, an implementation can avoid traversing unwanted subfolders
|
||||
|
@ -275,7 +275,7 @@ public:
|
|||
virtual Future<FilesAndSizesT> listFiles(std::string path = "", std::function<bool(std::string const &)> folderPathFilter = nullptr) = 0;
|
||||
|
||||
// Open a file for read by fileName
|
||||
virtual Future<Reference<IAsyncFile>> readFile(std::string fileName) = 0;
|
||||
Future<Reference<IAsyncFile>> readFile(std::string fileName) override = 0;
|
||||
|
||||
// Open a file for write by fileName
|
||||
virtual Future<Reference<IBackupFile>> writeFile(std::string fileName) = 0;
|
||||
|
@ -285,7 +285,7 @@ public:
|
|||
|
||||
// Delete entire container. During the process, if pNumDeleted is not null it will be
|
||||
// updated with the count of deleted files so that progress can be seen.
|
||||
virtual Future<Void> deleteContainer(int *pNumDeleted) = 0;
|
||||
Future<Void> deleteContainer(int* pNumDeleted) override = 0;
|
||||
|
||||
// Creates a 2-level path (x/y) where v should go such that x/y/* contains (10^smallestBucket) possible versions
|
||||
static std::string versionFolderString(Version v, int smallestBucket) {
|
||||
|
@ -333,11 +333,18 @@ public:
|
|||
return format("logs/%s/", versionFolderString(v, 11).c_str());
|
||||
}
|
||||
|
||||
Future<Reference<IBackupFile>> writeLogFile(Version beginVersion, Version endVersion, int blockSize) {
|
||||
Future<Reference<IBackupFile>> writeLogFile(Version beginVersion, Version endVersion, int blockSize) override {
|
||||
return writeFile(logVersionFolderString(beginVersion) + format("log,%lld,%lld,%s,%d", beginVersion, endVersion, deterministicRandom()->randomUniqueID().toString().c_str(), blockSize));
|
||||
}
|
||||
|
||||
Future<Reference<IBackupFile>> writeRangeFile(Version snapshotBeginVersion, int snapshotFileCount, Version fileVersion, int blockSize) {
|
||||
Future<Reference<IBackupFile>> writeTaggedLogFile(Version beginVersion, Version endVersion, int blockSize,
|
||||
uint16_t tagId) override {
|
||||
return writeFile(logVersionFolderString(beginVersion) +
|
||||
format("log,%lld,%lld,%s,%d,%d", beginVersion, endVersion,
|
||||
deterministicRandom()->randomUniqueID().toString().c_str(), blockSize, tagId));
|
||||
}
|
||||
|
||||
Future<Reference<IBackupFile>> writeRangeFile(Version snapshotBeginVersion, int snapshotFileCount, Version fileVersion, int blockSize) override {
|
||||
std::string fileName = format("range,%" PRId64 ",%s,%d", fileVersion, deterministicRandom()->randomUniqueID().toString().c_str(), blockSize);
|
||||
|
||||
// In order to test backward compatibility in simulation, sometimes write to the old path format
|
||||
|
@ -370,6 +377,11 @@ public:
|
|||
if(sscanf(name.c_str(), "log,%" SCNd64 ",%" SCNd64 ",%*[^,],%u%n", &f.beginVersion, &f.endVersion, &f.blockSize, &len) == 3 && len == name.size()) {
|
||||
out = f;
|
||||
return true;
|
||||
} else if (sscanf(name.c_str(), "log,%" SCNd64 ",%" SCNd64 ",%*[^,],%u,%d%n", &f.beginVersion, &f.endVersion,
|
||||
&f.blockSize, &f.tagId, &len) == 4 &&
|
||||
len == name.size() && f.tagId >= 0) {
|
||||
out = f;
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
@ -495,7 +507,7 @@ public:
|
|||
return Void();
|
||||
}
|
||||
|
||||
Future<Void> writeKeyspaceSnapshotFile(std::vector<std::string> fileNames, int64_t totalBytes) {
|
||||
Future<Void> writeKeyspaceSnapshotFile(std::vector<std::string> fileNames, int64_t totalBytes) override {
|
||||
return writeKeyspaceSnapshotFile_impl(Reference<BackupContainerFileSystem>::addRef(this), fileNames, totalBytes);
|
||||
};
|
||||
|
||||
|
@ -612,7 +624,7 @@ public:
|
|||
return BackupFileList({fRanges.get(), fLogs.get(), fSnapshots.get()});
|
||||
}
|
||||
|
||||
Future<BackupFileList> dumpFileList(Version begin, Version end) {
|
||||
Future<BackupFileList> dumpFileList(Version begin, Version end) override {
|
||||
return dumpFileList_impl(Reference<BackupContainerFileSystem>::addRef(this), begin, end);
|
||||
}
|
||||
|
||||
|
@ -829,7 +841,7 @@ public:
|
|||
}
|
||||
|
||||
// Uses the virtual methods to describe the backup contents
|
||||
Future<BackupDescription> describeBackup(bool deepScan, Version logStartVersionOverride) {
|
||||
Future<BackupDescription> describeBackup(bool deepScan, Version logStartVersionOverride) override {
|
||||
return describeBackup_impl(Reference<BackupContainerFileSystem>::addRef(this), deepScan, logStartVersionOverride);
|
||||
}
|
||||
|
||||
|
@ -1009,7 +1021,7 @@ public:
|
|||
}
|
||||
|
||||
// Delete all data up to (but not including endVersion)
|
||||
Future<Void> expireData(Version expireEndVersion, bool force, ExpireProgress *progress, Version restorableBeginVersion) {
|
||||
Future<Void> expireData(Version expireEndVersion, bool force, ExpireProgress *progress, Version restorableBeginVersion) override {
|
||||
return expireData_impl(Reference<BackupContainerFileSystem>::addRef(this), expireEndVersion, force, progress, restorableBeginVersion);
|
||||
}
|
||||
|
||||
|
@ -1067,7 +1079,7 @@ public:
|
|||
return Optional<RestorableFileSet>();
|
||||
}
|
||||
|
||||
Future<Optional<RestorableFileSet>> getRestoreSet(Version targetVersion){
|
||||
Future<Optional<RestorableFileSet>> getRestoreSet(Version targetVersion) override {
|
||||
return getRestoreSet_impl(Reference<BackupContainerFileSystem>::addRef(this), targetVersion);
|
||||
}
|
||||
|
||||
|
@ -1152,8 +1164,8 @@ public:
|
|||
|
||||
class BackupContainerLocalDirectory : public BackupContainerFileSystem, ReferenceCounted<BackupContainerLocalDirectory> {
|
||||
public:
|
||||
void addref() { return ReferenceCounted<BackupContainerLocalDirectory>::addref(); }
|
||||
void delref() { return ReferenceCounted<BackupContainerLocalDirectory>::delref(); }
|
||||
void addref() override { return ReferenceCounted<BackupContainerLocalDirectory>::addref(); }
|
||||
void delref() override { return ReferenceCounted<BackupContainerLocalDirectory>::delref(); }
|
||||
|
||||
static std::string getURLFormat() { return "file://</path/to/base/dir/>"; }
|
||||
|
||||
|
@ -1202,7 +1214,7 @@ public:
|
|||
return results;
|
||||
}
|
||||
|
||||
Future<Void> create() {
|
||||
Future<Void> create() override {
|
||||
// Nothing should be done here because create() can be called by any process working with the container URL, such as fdbbackup.
|
||||
// Since "local directory" containers are by definition local to the machine they are accessed from,
|
||||
// the container's creation (in this case the creation of a directory) must be ensured prior to every file creation,
|
||||
|
@ -1212,11 +1224,11 @@ public:
|
|||
}
|
||||
|
||||
// The container exists if the folder it resides in exists
|
||||
Future<bool> exists() {
|
||||
Future<bool> exists() override {
|
||||
return directoryExists(m_path);
|
||||
}
|
||||
|
||||
Future<Reference<IAsyncFile>> readFile(std::string path) {
|
||||
Future<Reference<IAsyncFile>> readFile(std::string path) override {
|
||||
int flags = IAsyncFile::OPEN_NO_AIO | IAsyncFile::OPEN_READONLY | IAsyncFile::OPEN_UNCACHED;
|
||||
// Simulation does not properly handle opening the same file from multiple machines using a shared filesystem,
|
||||
// so create a symbolic link to make each file opening appear to be unique. This could also work in production
|
||||
|
@ -1285,15 +1297,15 @@ public:
|
|||
return finish_impl(Reference<BackupFile>::addRef(this));
|
||||
}
|
||||
|
||||
void addref() { return ReferenceCounted<BackupFile>::addref(); }
|
||||
void delref() { return ReferenceCounted<BackupFile>::delref(); }
|
||||
void addref() override { return ReferenceCounted<BackupFile>::addref(); }
|
||||
void delref() override { return ReferenceCounted<BackupFile>::delref(); }
|
||||
|
||||
private:
|
||||
Reference<IAsyncFile> m_file;
|
||||
std::string m_finalFullPath;
|
||||
};
|
||||
|
||||
Future<Reference<IBackupFile>> writeFile(std::string path) {
|
||||
Future<Reference<IBackupFile>> writeFile(std::string path) override {
|
||||
int flags = IAsyncFile::OPEN_NO_AIO | IAsyncFile::OPEN_CREATE | IAsyncFile::OPEN_ATOMIC_WRITE_AND_CREATE | IAsyncFile::OPEN_READWRITE;
|
||||
std::string fullPath = joinPath(m_path, path);
|
||||
platform::createDirectory(parentDirectory(fullPath));
|
||||
|
@ -1304,7 +1316,7 @@ public:
|
|||
});
|
||||
}
|
||||
|
||||
Future<Void> deleteFile(std::string path) {
|
||||
Future<Void> deleteFile(std::string path) override {
|
||||
::deleteFile(joinPath(m_path, path));
|
||||
return Void();
|
||||
}
|
||||
|
@ -1329,7 +1341,7 @@ public:
|
|||
return results;
|
||||
}
|
||||
|
||||
Future<Void> deleteContainer(int *pNumDeleted) {
|
||||
Future<Void> deleteContainer(int* pNumDeleted) override {
|
||||
// In order to avoid deleting some random directory due to user error, first describe the backup
|
||||
// and make sure it has something in it.
|
||||
return map(describeBackup(false, invalidVersion), [=](BackupDescription const &desc) {
|
||||
|
@ -1389,8 +1401,8 @@ public:
|
|||
}
|
||||
}
|
||||
|
||||
void addref() { return ReferenceCounted<BackupContainerBlobStore>::addref(); }
|
||||
void delref() { return ReferenceCounted<BackupContainerBlobStore>::delref(); }
|
||||
void addref() override { return ReferenceCounted<BackupContainerBlobStore>::addref(); }
|
||||
void delref() override { return ReferenceCounted<BackupContainerBlobStore>::delref(); }
|
||||
|
||||
static std::string getURLFormat() {
|
||||
return BlobStoreEndpoint::getURLFormat(true) + " (Note: The 'bucket' parameter is required.)";
|
||||
|
@ -1398,7 +1410,7 @@ public:
|
|||
|
||||
virtual ~BackupContainerBlobStore() {}
|
||||
|
||||
Future<Reference<IAsyncFile>> readFile(std::string path) {
|
||||
Future<Reference<IAsyncFile>> readFile(std::string path) override {
|
||||
return Reference<IAsyncFile>(
|
||||
new AsyncFileReadAheadCache(
|
||||
Reference<IAsyncFile>(new AsyncFileBlobStoreRead(m_bstore, m_bucket, dataPath(path))),
|
||||
|
@ -1435,17 +1447,17 @@ public:
|
|||
return map(m_file->sync(), [=](Void _) { self->m_file.clear(); return Void(); });
|
||||
}
|
||||
|
||||
void addref() { return ReferenceCounted<BackupFile>::addref(); }
|
||||
void delref() { return ReferenceCounted<BackupFile>::delref(); }
|
||||
void addref() override { return ReferenceCounted<BackupFile>::addref(); }
|
||||
void delref() override { return ReferenceCounted<BackupFile>::delref(); }
|
||||
private:
|
||||
Reference<IAsyncFile> m_file;
|
||||
};
|
||||
|
||||
Future<Reference<IBackupFile>> writeFile(std::string path) {
|
||||
Future<Reference<IBackupFile>> writeFile(std::string path) override {
|
||||
return Reference<IBackupFile>(new BackupFile(path, Reference<IAsyncFile>(new AsyncFileBlobStoreWrite(m_bstore, m_bucket, dataPath(path)))));
|
||||
}
|
||||
|
||||
Future<Void> deleteFile(std::string path) {
|
||||
Future<Void> deleteFile(std::string path) override {
|
||||
return m_bstore->deleteObject(m_bucket, dataPath(path));
|
||||
}
|
||||
|
||||
|
@ -1483,12 +1495,12 @@ public:
|
|||
return Void();
|
||||
}
|
||||
|
||||
Future<Void> create() {
|
||||
Future<Void> create() override {
|
||||
return create_impl(Reference<BackupContainerBlobStore>::addRef(this));
|
||||
}
|
||||
|
||||
// The container exists if the index entry in the blob bucket exists
|
||||
Future<bool> exists() {
|
||||
Future<bool> exists() override {
|
||||
return m_bstore->objectExists(m_bucket, indexEntry());
|
||||
}
|
||||
|
||||
|
@ -1508,7 +1520,7 @@ public:
|
|||
return Void();
|
||||
}
|
||||
|
||||
Future<Void> deleteContainer(int *pNumDeleted) {
|
||||
Future<Void> deleteContainer(int* pNumDeleted) override {
|
||||
return deleteContainer_impl(Reference<BackupContainerBlobStore>::addRef(this), pNumDeleted);
|
||||
}
|
||||
|
||||
|
|
|
@ -68,17 +68,24 @@ struct LogFile {
|
|||
uint32_t blockSize;
|
||||
std::string fileName;
|
||||
int64_t fileSize;
|
||||
int tagId = -1; // Log router tag. Non-negative for new backup format.
|
||||
|
||||
// Order by beginVersion, break ties with endVersion
|
||||
bool operator< (const LogFile &rhs) const {
|
||||
return beginVersion == rhs.beginVersion ? endVersion < rhs.endVersion : beginVersion < rhs.beginVersion;
|
||||
}
|
||||
|
||||
// Returns if two log files have the same content by comparing version range and tag ID.
|
||||
bool sameContent(const LogFile& rhs) const {
|
||||
return beginVersion == rhs.beginVersion && endVersion == rhs.endVersion && tagId == rhs.tagId;
|
||||
}
|
||||
|
||||
std::string toString() const {
|
||||
std::stringstream ss;
|
||||
ss << "beginVersion:" << std::to_string(beginVersion) << " endVersion:" << std::to_string(endVersion) <<
|
||||
" blockSize:" << std::to_string(blockSize) << " filename:" << fileName <<
|
||||
" fileSize:" << std::to_string(fileSize);
|
||||
ss << "beginVersion:" << std::to_string(beginVersion) << " endVersion:" << std::to_string(endVersion)
|
||||
<< " blockSize:" << std::to_string(blockSize) << " filename:" << fileName
|
||||
<< " fileSize:" << std::to_string(fileSize)
|
||||
<< " tagId: " << (tagId >= 0 ? std::to_string(tagId) : std::string("(None)"));
|
||||
return ss.str();
|
||||
}
|
||||
};
|
||||
|
@ -205,6 +212,10 @@ public:
|
|||
virtual Future<Reference<IBackupFile>> writeLogFile(Version beginVersion, Version endVersion, int blockSize) = 0;
|
||||
virtual Future<Reference<IBackupFile>> writeRangeFile(Version snapshotBeginVersion, int snapshotFileCount, Version fileVersion, int blockSize) = 0;
|
||||
|
||||
// Open a tagged log file for writing, where tagId is the log router tag's id.
|
||||
virtual Future<Reference<IBackupFile>> writeTaggedLogFile(Version beginVersion, Version endVersion, int blockSize,
|
||||
uint16_t tagId) = 0;
|
||||
|
||||
// Write a KeyspaceSnapshotFile of range file names representing a full non overlapping
|
||||
// snapshot of the key ranges this backup is targeting.
|
||||
virtual Future<Void> writeKeyspaceSnapshotFile(std::vector<std::string> fileNames, int64_t totalBytes) = 0;
|
||||
|
|
|
@ -42,11 +42,16 @@ enum {
|
|||
tagLocalityRemoteLog = -3,
|
||||
tagLocalityUpgraded = -4,
|
||||
tagLocalitySatellite = -5,
|
||||
tagLocalityLogRouterMapped = -6,
|
||||
tagLocalityLogRouterMapped = -6, // used by log router to pop from TLogs
|
||||
tagLocalityTxs = -7,
|
||||
tagLocalityBackup = -8, // used by backup role to pop from TLogs
|
||||
tagLocalityInvalid = -99
|
||||
}; //The TLog and LogRouter require these number to be as compact as possible
|
||||
|
||||
inline bool isPseudoLocality(int8_t locality) {
|
||||
return locality == tagLocalityLogRouterMapped || locality == tagLocalityBackup;
|
||||
}
|
||||
|
||||
#pragma pack(push, 1)
|
||||
struct Tag {
|
||||
int8_t locality;
|
||||
|
@ -966,4 +971,18 @@ struct HealthMetrics {
|
|||
}
|
||||
};
|
||||
|
||||
struct WorkerBackupStatus {
|
||||
LogEpoch epoch;
|
||||
Version version;
|
||||
Tag tag;
|
||||
|
||||
WorkerBackupStatus() : epoch(0), version(invalidVersion) {}
|
||||
WorkerBackupStatus(LogEpoch e, Version v, Tag t) : epoch(e), version(v), tag(t) {}
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, epoch, version, tag);
|
||||
}
|
||||
};
|
||||
|
||||
#endif
|
||||
|
|
|
@ -545,42 +545,6 @@ namespace fileBackup {
|
|||
Key lastValue;
|
||||
};
|
||||
|
||||
// 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<typename T> 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< int32_t>());}
|
||||
uint32_t consumeNetworkUInt32() { return bigEndian32( consume<uint32_t>());}
|
||||
|
||||
bool eof() { return rptr == end; }
|
||||
|
||||
const uint8_t *rptr, *end;
|
||||
Error failure_error;
|
||||
};
|
||||
|
||||
ACTOR Future<Standalone<VectorRef<KeyValueRef>>> decodeRangeFileBlock(Reference<IAsyncFile> file, int64_t offset, int len) {
|
||||
state Standalone<StringRef> buf = makeString(len);
|
||||
int rLen = wait(file->read(mutateString(buf), len, offset));
|
||||
|
|
|
@ -173,6 +173,7 @@ struct GetReadVersionRequest : TimedRequest {
|
|||
PRIORITY_BATCH = 1 << 24
|
||||
};
|
||||
enum {
|
||||
FLAG_USE_MIN_KNOWN_COMMITTED_VERSION = 4,
|
||||
FLAG_USE_PROVISIONAL_PROXIES = 2,
|
||||
FLAG_CAUSAL_READ_RISKY = 1,
|
||||
FLAG_PRIORITY_MASK = PRIORITY_SYSTEM_IMMEDIATE,
|
||||
|
|
|
@ -491,6 +491,38 @@ ProcessData decodeWorkerListValue( ValueRef const& value ) {
|
|||
return s;
|
||||
}
|
||||
|
||||
const KeyRangeRef backupProgressKeys(LiteralStringRef("\xff\x02/backupProgress/"),
|
||||
LiteralStringRef("\xff\x02/backupProgress0"));
|
||||
const KeyRef backupProgressPrefix = backupProgressKeys.begin;
|
||||
const KeyRef backupStartedKey = LiteralStringRef("\xff\x02/backupStarted");
|
||||
|
||||
const Key backupProgressKeyFor(UID workerID) {
|
||||
BinaryWriter wr(Unversioned());
|
||||
wr.serializeBytes(backupProgressPrefix);
|
||||
wr << workerID;
|
||||
return wr.toValue();
|
||||
}
|
||||
|
||||
const Value backupProgressValue(const WorkerBackupStatus& status) {
|
||||
BinaryWriter wr(IncludeVersion());
|
||||
wr << status;
|
||||
return wr.toValue();
|
||||
}
|
||||
|
||||
UID decodeBackupProgressKey(const KeyRef& key) {
|
||||
UID serverID;
|
||||
BinaryReader rd(key.removePrefix(backupProgressPrefix), Unversioned());
|
||||
rd >> serverID;
|
||||
return serverID;
|
||||
}
|
||||
|
||||
WorkerBackupStatus decodeBackupProgressValue(const ValueRef& value) {
|
||||
WorkerBackupStatus status;
|
||||
BinaryReader reader(value, IncludeVersion());
|
||||
reader >> status;
|
||||
return status;
|
||||
}
|
||||
|
||||
const KeyRef coordinatorsKey = LiteralStringRef("\xff/coordinators");
|
||||
const KeyRef logsKey = LiteralStringRef("\xff/logs");
|
||||
const KeyRef minRequiredCommitVersionKey = LiteralStringRef("\xff/minRequiredCommitVersion");
|
||||
|
|
|
@ -174,6 +174,17 @@ const Value workerListValue( ProcessData const& );
|
|||
Key decodeWorkerListKey( KeyRef const& );
|
||||
ProcessData decodeWorkerListValue( ValueRef const& );
|
||||
|
||||
// "\xff/backupProgress/[[workerID]]" := "[[WorkerBackupStatus]]"
|
||||
extern const KeyRangeRef backupProgressKeys;
|
||||
extern const KeyRef backupProgressPrefix;
|
||||
const Key backupProgressKeyFor(UID workerID);
|
||||
const Value backupProgressValue(const WorkerBackupStatus& status);
|
||||
UID decodeBackupProgressKey(const KeyRef& key);
|
||||
WorkerBackupStatus decodeBackupProgressValue(const ValueRef& value);
|
||||
|
||||
// "\xff/backupStarted"
|
||||
extern const KeyRef backupStartedKey;
|
||||
|
||||
extern const KeyRef coordinatorsKey;
|
||||
extern const KeyRef logsKey;
|
||||
extern const KeyRef minRequiredCommitVersionKey;
|
||||
|
|
|
@ -135,6 +135,26 @@ ProcessClass::Fitness ProcessClass::machineClassFitness( ClusterRole role ) cons
|
|||
default:
|
||||
return ProcessClass::WorstFit;
|
||||
}
|
||||
case ProcessClass::Backup:
|
||||
switch( _class ) {
|
||||
case ProcessClass::BackupClass:
|
||||
return ProcessClass::BestFit;
|
||||
case ProcessClass::StatelessClass:
|
||||
case ProcessClass::LogRouterClass:
|
||||
return ProcessClass::GoodFit;
|
||||
case ProcessClass::UnsetClass:
|
||||
return ProcessClass::UnsetFit;
|
||||
case ProcessClass::ResolutionClass:
|
||||
return ProcessClass::OkayFit;
|
||||
case ProcessClass::TransactionClass:
|
||||
return ProcessClass::OkayFit;
|
||||
case ProcessClass::CoordinatorClass:
|
||||
case ProcessClass::MasterClass:
|
||||
case ProcessClass::TesterClass:
|
||||
return ProcessClass::NeverAssign;
|
||||
default:
|
||||
return ProcessClass::WorstFit;
|
||||
}
|
||||
case ProcessClass::ClusterController:
|
||||
switch( _class ) {
|
||||
case ProcessClass::ClusterControllerClass:
|
||||
|
|
|
@ -44,11 +44,12 @@ struct ProcessClass {
|
|||
CoordinatorClass,
|
||||
RatekeeperClass,
|
||||
StorageCacheClass,
|
||||
BackupClass,
|
||||
InvalidClass = -1
|
||||
};
|
||||
|
||||
enum Fitness { BestFit, GoodFit, UnsetFit, OkayFit, WorstFit, ExcludeFit, NeverAssign }; //cannot be larger than 7 because of leader election mask
|
||||
enum ClusterRole { Storage, TLog, Proxy, Master, Resolver, LogRouter, ClusterController, DataDistributor, Ratekeeper, StorageCache, NoRole };
|
||||
enum ClusterRole { Storage, TLog, Proxy, Master, Resolver, LogRouter, ClusterController, DataDistributor, Ratekeeper, StorageCache, Backup, NoRole };
|
||||
enum ClassSource { CommandLineSource, AutoSource, DBSource, InvalidSource = -1 };
|
||||
int16_t _class;
|
||||
int16_t _source;
|
||||
|
@ -74,6 +75,7 @@ public:
|
|||
else if (s=="coordinator") _class = CoordinatorClass;
|
||||
else if (s=="ratekeeper") _class = RatekeeperClass;
|
||||
else if (s=="storage_cache") _class = StorageCacheClass;
|
||||
else if (s=="backup") _class = BackupClass;
|
||||
else _class = InvalidClass;
|
||||
}
|
||||
|
||||
|
@ -94,6 +96,7 @@ public:
|
|||
else if (classStr=="coordinator") _class = CoordinatorClass;
|
||||
else if (classStr=="ratekeeper") _class = RatekeeperClass;
|
||||
else if (classStr=="storage_cache") _class = StorageCacheClass;
|
||||
else if (classStr=="backup") _class = BackupClass;
|
||||
else _class = InvalidClass;
|
||||
|
||||
if (sourceStr=="command_line") _source = CommandLineSource;
|
||||
|
@ -129,6 +132,7 @@ public:
|
|||
case CoordinatorClass: return "coordinator";
|
||||
case RatekeeperClass: return "ratekeeper";
|
||||
case StorageCacheClass: return "storage_cache";
|
||||
case BackupClass: return "backup";
|
||||
default: return "invalid";
|
||||
}
|
||||
}
|
||||
|
|
|
@ -99,6 +99,7 @@ public:
|
|||
case ProcessClass::DataDistributorClass: return false;
|
||||
case ProcessClass::RatekeeperClass: return false;
|
||||
case ProcessClass::StorageCacheClass: return false;
|
||||
case ProcessClass::BackupClass: return false;
|
||||
default: return false;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,54 @@
|
|||
/*
|
||||
* BackupInterface.h
|
||||
*
|
||||
* This source file is part of the FoundationDB open source project
|
||||
*
|
||||
* Copyright 2013-2019 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.
|
||||
*/
|
||||
|
||||
#ifndef FDBSERVER_BACKUPINTERFACE_H
|
||||
#define FDBSERVER_BACKUPINTERFACE_H
|
||||
|
||||
#include "fdbclient/FDBTypes.h"
|
||||
#include "fdbrpc/fdbrpc.h"
|
||||
#include "fdbrpc/Locality.h"
|
||||
|
||||
// The interface for backup workers.
|
||||
struct BackupInterface {
|
||||
constexpr static FileIdentifier file_identifier = 6762745;
|
||||
RequestStream<ReplyPromise<Void>> waitFailure;
|
||||
struct LocalityData locality;
|
||||
|
||||
BackupInterface() = default;
|
||||
explicit BackupInterface(const struct LocalityData& l) : locality(l) {}
|
||||
|
||||
void initEndpoints() {}
|
||||
UID id() const { return getToken(); }
|
||||
NetworkAddress address() const { return waitFailure.getEndpoint().getPrimaryAddress(); }
|
||||
UID getToken() const { return waitFailure.getEndpoint().token; }
|
||||
bool operator== (const BackupInterface& r) const {
|
||||
return getToken() == r.getToken();
|
||||
}
|
||||
bool operator!= (const BackupInterface& r) const {
|
||||
return !(*this == r);
|
||||
}
|
||||
|
||||
template <class Archive>
|
||||
void serialize(Archive& ar) {
|
||||
serializer(ar, waitFailure, locality);
|
||||
}
|
||||
};
|
||||
|
||||
#endif //FDBSERVER_BACKUPINTERFACE_H
|
|
@ -0,0 +1,117 @@
|
|||
#include "fdbserver/BackupProgress.actor.h"
|
||||
|
||||
#include "fdbclient/NativeAPI.actor.h"
|
||||
#include "fdbclient/SystemData.h"
|
||||
#include "flow/UnitTest.h"
|
||||
#include "flow/actorcompiler.h" // This must be the last #include.
|
||||
|
||||
void BackupProgress::addBackupStatus(const WorkerBackupStatus& status) {
|
||||
auto& it = progress[status.epoch];
|
||||
auto lb = it.lower_bound(status.tag);
|
||||
if (lb != it.end() && status.tag == lb->first) {
|
||||
if (lb->second < status.version) {
|
||||
lb->second = status.version;
|
||||
}
|
||||
} else {
|
||||
it.insert(lb, { status.tag, status.version });
|
||||
}
|
||||
}
|
||||
|
||||
std::map<std::pair<LogEpoch, Version>, std::map<Tag, Version>> BackupProgress::getUnfinishedBackup() {
|
||||
std::map<std::pair<LogEpoch, Version>, std::map<Tag, Version>> toRecruit;
|
||||
|
||||
if (!backupStartedValue.present()) return toRecruit; // No active backups
|
||||
|
||||
for (const auto& [epoch, info] : epochInfos) {
|
||||
std::set<Tag> tags = enumerateLogRouterTags(info.logRouterTags);
|
||||
std::map<Tag, Version> tagVersions;
|
||||
auto progressIt = progress.find(epoch);
|
||||
if (progressIt != progress.end()) {
|
||||
for (const auto& [tag, savedVersion] : progressIt->second) {
|
||||
tags.erase(tag);
|
||||
if (savedVersion < info.epochEnd - 1) {
|
||||
tagVersions.insert({ tag, savedVersion + 1 });
|
||||
TraceEvent("BW", dbgid)
|
||||
.detail("OldEpoch", epoch)
|
||||
.detail("Tag", tag.toString())
|
||||
.detail("BeginVersion", savedVersion + 1)
|
||||
.detail("EndVersion", info.epochEnd);
|
||||
}
|
||||
}
|
||||
}
|
||||
for (const Tag tag : tags) { // tags without progress data
|
||||
tagVersions.insert({ tag, info.epochBegin });
|
||||
TraceEvent("BW", dbgid)
|
||||
.detail("OldEpoch", epoch)
|
||||
.detail("Tag", tag.toString())
|
||||
.detail("BeginVersion", info.epochBegin)
|
||||
.detail("EndVersion", info.epochEnd);
|
||||
}
|
||||
if (!tagVersions.empty()) {
|
||||
toRecruit[{ epoch, info.epochEnd }] = tagVersions;
|
||||
}
|
||||
}
|
||||
return toRecruit;
|
||||
}
|
||||
|
||||
// Save each tag's savedVersion for all epochs into "bStatus".
|
||||
ACTOR Future<Void> getBackupProgress(Database cx, UID dbgid, Reference<BackupProgress> bStatus) {
|
||||
state Transaction tr(cx);
|
||||
|
||||
loop {
|
||||
try {
|
||||
tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
|
||||
tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE);
|
||||
tr.setOption(FDBTransactionOptions::LOCK_AWARE);
|
||||
state Future<Optional<Value>> fValue = tr.get(backupStartedKey);
|
||||
state Standalone<RangeResultRef> results = wait(tr.getRange(backupProgressKeys, CLIENT_KNOBS->TOO_MANY));
|
||||
ASSERT(!results.more && results.size() < CLIENT_KNOBS->TOO_MANY);
|
||||
|
||||
Optional<Value> value = wait(fValue);
|
||||
bStatus->setBackupStartedValue(value);
|
||||
for (auto& it : results) {
|
||||
const UID workerID = decodeBackupProgressKey(it.key);
|
||||
const WorkerBackupStatus status = decodeBackupProgressValue(it.value);
|
||||
bStatus->addBackupStatus(status);
|
||||
TraceEvent("GotBackupProgress", dbgid)
|
||||
.detail("W", workerID)
|
||||
.detail("Epoch", status.epoch)
|
||||
.detail("Version", status.version)
|
||||
.detail("Tag", status.tag.toString());
|
||||
}
|
||||
return Void();
|
||||
} catch (Error& e) {
|
||||
wait(tr.onError(e));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
TEST_CASE("/BackupProgress/Unfinished") {
|
||||
std::map<LogEpoch, ILogSystem::EpochTagsVersionsInfo> epochInfos;
|
||||
|
||||
const int epoch1 = 2, begin1 = 1, end1 = 100;
|
||||
const Tag tag1(tagLocalityLogRouter, 0);
|
||||
epochInfos.insert({ epoch1, ILogSystem::EpochTagsVersionsInfo(1, begin1, end1) });
|
||||
BackupProgress progress(UID(0, 0), epochInfos);
|
||||
progress.setBackupStartedValue(Optional<Value>(LiteralStringRef("1")));
|
||||
|
||||
std::map<std::pair<LogEpoch, Version>, std::map<Tag, Version>> unfinished = progress.getUnfinishedBackup();
|
||||
|
||||
ASSERT(unfinished.size() == 1);
|
||||
for (const auto [epochVersion, tagVersion] : unfinished) {
|
||||
ASSERT(epochVersion.first == epoch1 && epochVersion.second == end1);
|
||||
ASSERT(tagVersion.size() == 1 && tagVersion.begin()->first == tag1 && tagVersion.begin()->second == begin1);
|
||||
}
|
||||
|
||||
const int saved1 = 50;
|
||||
WorkerBackupStatus status1(epoch1, saved1, tag1);
|
||||
progress.addBackupStatus(status1);
|
||||
unfinished = progress.getUnfinishedBackup();
|
||||
ASSERT(unfinished.size() == 1);
|
||||
for (const auto [epochVersion, tagVersion] : unfinished) {
|
||||
ASSERT(epochVersion.first == epoch1 && epochVersion.second == end1);
|
||||
ASSERT(tagVersion.size() == 1 && tagVersion.begin()->first == tag1 && tagVersion.begin()->second == saved1 + 1);
|
||||
}
|
||||
|
||||
return Void();
|
||||
}
|
|
@ -0,0 +1,89 @@
|
|||
/*
|
||||
* BackupProgress.h
|
||||
*
|
||||
* This source file is part of the FoundationDB open source project
|
||||
*
|
||||
* Copyright 2013-2019 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.
|
||||
*/
|
||||
|
||||
#if defined(NO_INTELLISENSE) && !defined(FDBSERVER_BACKUPPROGRESS_ACTOR_G_H)
|
||||
#define FDBSERVER_BACKUPPROGRESS_ACTOR_G_H
|
||||
#include "fdbserver/BackupProgress.actor.g.h"
|
||||
#elif !defined(FDBSERVER_BACKUPPROGRESS_ACTOR_H)
|
||||
#define FDBSERVER_BACKUPPROGRESS_ACTOR_H
|
||||
|
||||
#include <map>
|
||||
#include "fdbclient/FDBTypes.h"
|
||||
#include "fdbserver/LogSystem.h"
|
||||
#include "flow/Arena.h"
|
||||
#include "flow/FastRef.h"
|
||||
#include "flow/actorcompiler.h" // This must be the last #include.
|
||||
|
||||
class BackupProgress : NonCopyable, ReferenceCounted<BackupProgress> {
|
||||
public:
|
||||
BackupProgress(UID id, const std::map<LogEpoch, ILogSystem::EpochTagsVersionsInfo>& infos)
|
||||
: dbgid(id), epochInfos(infos) {}
|
||||
~BackupProgress() {}
|
||||
|
||||
// Adds a backup status. If the tag already has an entry, then the max of
|
||||
// savedVersion is used.
|
||||
void addBackupStatus(const WorkerBackupStatus& status);
|
||||
|
||||
// Returns a map of pair<Epoch, endVersion> : map<tag, savedVersion>, so that
|
||||
// the backup range should be [savedVersion + 1, endVersion) for the "tag" of the "Epoch".
|
||||
//
|
||||
// Specifically, the backup ranges for each old epoch are:
|
||||
// if tag in tags_without_backup_progress:
|
||||
// backup [epochBegin, endVersion)
|
||||
// else if savedVersion < endVersion - 1 = knownCommittedVersion
|
||||
// backup [savedVersion + 1, endVersion)
|
||||
std::map<std::pair<LogEpoch, Version>, std::map<Tag, Version>> getUnfinishedBackup();
|
||||
|
||||
// Set the value for "backupStartedKey"
|
||||
void setBackupStartedValue(Optional<Value> value) {
|
||||
backupStartedValue = value;
|
||||
}
|
||||
|
||||
void addref() { ReferenceCounted<BackupProgress>::addref(); }
|
||||
|
||||
void delref() { ReferenceCounted<BackupProgress>::delref(); }
|
||||
|
||||
private:
|
||||
std::set<Tag> enumerateLogRouterTags(int logRouterTags) {
|
||||
std::set<Tag> tags;
|
||||
for (int i = 0; i < logRouterTags; i++) {
|
||||
tags.insert(Tag(tagLocalityLogRouter, i));
|
||||
}
|
||||
return tags;
|
||||
}
|
||||
|
||||
const UID dbgid;
|
||||
|
||||
// Note this MUST be iterated in ascending order.
|
||||
const std::map<LogEpoch, ILogSystem::EpochTagsVersionsInfo> epochInfos;
|
||||
|
||||
// Backup progress saved in the system keyspace. Note there can be multiple
|
||||
// progress status for a tag in an epoch due to later epoch trying to fill
|
||||
// the gap. "progress" MUST be iterated in ascending order.
|
||||
std::map<LogEpoch, std::map<Tag, Version>> progress;
|
||||
|
||||
// Value of the "backupStartedKey".
|
||||
Optional<Value> backupStartedValue;
|
||||
};
|
||||
|
||||
ACTOR Future<Void> getBackupProgress(Database cx, UID dbgid, Reference<BackupProgress> bStatus);
|
||||
|
||||
#include "flow/unactorcompiler.h"
|
||||
#endif
|
|
@ -0,0 +1,459 @@
|
|||
/*
|
||||
* BackupWorker.actor.cpp
|
||||
*
|
||||
* This source file is part of the FoundationDB open source project
|
||||
*
|
||||
* Copyright 2013-2019 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.
|
||||
*/
|
||||
|
||||
#include "fdbclient/BackupContainer.h"
|
||||
#include "fdbclient/DatabaseContext.h"
|
||||
#include "fdbclient/MasterProxyInterface.h"
|
||||
#include "fdbclient/SystemData.h"
|
||||
#include "fdbserver/BackupInterface.h"
|
||||
#include "fdbserver/LogProtocolMessage.h"
|
||||
#include "fdbserver/LogSystem.h"
|
||||
#include "fdbserver/ServerDBInfo.h"
|
||||
#include "fdbserver/WaitFailure.h"
|
||||
#include "fdbserver/WorkerInterface.actor.h"
|
||||
#include "flow/Error.h"
|
||||
|
||||
#include "flow/actorcompiler.h" // This must be the last #include.
|
||||
|
||||
struct VersionedMessage {
|
||||
LogMessageVersion version;
|
||||
StringRef message;
|
||||
VectorRef<Tag> tags;
|
||||
Arena arena; // Keep a reference to the memory containing the message
|
||||
|
||||
VersionedMessage(LogMessageVersion v, StringRef m, const VectorRef<Tag>& t, const Arena& a)
|
||||
: version(v), message(m), tags(t), arena(a) {}
|
||||
const Version getVersion() const { return version.version; }
|
||||
const uint32_t getSubVersion() const { return version.sub; }
|
||||
};
|
||||
|
||||
struct BackupData {
|
||||
const UID myId;
|
||||
const Tag tag; // LogRouter tag for this worker, i.e., (-2, i)
|
||||
const Version startVersion;
|
||||
const Optional<Version> endVersion; // old epoch's end version (inclusive), or empty for current epoch
|
||||
const LogEpoch recruitedEpoch;
|
||||
const LogEpoch backupEpoch;
|
||||
Version minKnownCommittedVersion;
|
||||
Version savedVersion, lastSeenVersion;
|
||||
AsyncVar<Reference<ILogSystem>> logSystem;
|
||||
Database cx;
|
||||
std::vector<VersionedMessage> messages;
|
||||
Reference<IBackupContainer> container;
|
||||
AsyncVar<bool> pullFinished;
|
||||
|
||||
CounterCollection cc;
|
||||
Future<Void> logger;
|
||||
|
||||
explicit BackupData(Reference<AsyncVar<ServerDBInfo>> db, const InitializeBackupRequest& req)
|
||||
: myId(req.reqId), tag(req.routerTag), startVersion(req.startVersion), endVersion(req.endVersion),
|
||||
recruitedEpoch(req.recruitedEpoch), backupEpoch(req.backupEpoch), minKnownCommittedVersion(invalidVersion),
|
||||
savedVersion(invalidVersion), lastSeenVersion(invalidVersion), cc("BackupWorker", myId.toString()) {
|
||||
cx = openDBOnServer(db, TaskPriority::DefaultEndpoint, true, true);
|
||||
pullFinished.set(false);
|
||||
|
||||
specialCounter(cc, "SavedVersion", [this]() { return this->savedVersion; });
|
||||
specialCounter(cc, "MinKnownCommittedVersion", [this]() { return this->minKnownCommittedVersion; });
|
||||
specialCounter(cc, "MsgQ", [this]() { return this->messages.size(); });
|
||||
logger = traceCounters("BackupWorkerMetrics", myId, SERVER_KNOBS->WORKER_LOGGING_INTERVAL, &cc,
|
||||
"BackupWorkerMetrics");
|
||||
if (g_network->isSimulated()) {
|
||||
container = IBackupContainer::openContainer("file://simfdb/mutation_backups/");
|
||||
} else {
|
||||
// TODO: use blobstore URL passed from somewhere.
|
||||
ASSERT(false);
|
||||
container = IBackupContainer::openContainer("blobstore://");
|
||||
}
|
||||
}
|
||||
|
||||
void pop() {
|
||||
const LogEpoch oldest = logSystem.get()->getOldestBackupEpoch();
|
||||
if (backupEpoch > oldest) {
|
||||
// Defer pop if old epoch hasn't finished popping yet.
|
||||
TraceEvent("BackupWorkerPopDeferred", myId)
|
||||
.suppressFor(1.0)
|
||||
.detail("BackupEpoch", backupEpoch)
|
||||
.detail("OldestEpoch", oldest)
|
||||
.detail("Version", savedVersion);
|
||||
return;
|
||||
}
|
||||
const Tag popTag = logSystem.get()->getPseudoPopTag(tag, ProcessClass::BackupClass);
|
||||
logSystem.get()->pop(savedVersion, popTag);
|
||||
}
|
||||
|
||||
void eraseMessagesAfterEndVersion() {
|
||||
ASSERT(endVersion.present());
|
||||
const Version ver = endVersion.get();
|
||||
while (!messages.empty()) {
|
||||
if (messages.back().getVersion() > ver) {
|
||||
messages.pop_back();
|
||||
} else {
|
||||
return;
|
||||
}
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
ACTOR Future<Void> monitorBackupStarted(BackupData* self) {
|
||||
loop {
|
||||
state ReadYourWritesTransaction tr(self->cx);
|
||||
|
||||
loop {
|
||||
try {
|
||||
tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
|
||||
tr.setOption(FDBTransactionOptions::LOCK_AWARE);
|
||||
Optional<Standalone<StringRef>> value = wait(tr.get(backupStartedKey));
|
||||
if (value.present()) return Void();
|
||||
|
||||
state Future<Void> watchFuture = tr.watch(backupStartedKey);
|
||||
wait(tr.commit());
|
||||
wait(watchFuture);
|
||||
break;
|
||||
} catch (Error& e) {
|
||||
wait(tr.onError(e));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
ACTOR Future<Void> saveProgress(BackupData* self, Version backupVersion) {
|
||||
state Transaction tr(self->cx);
|
||||
state Key key = backupProgressKeyFor(self->myId);
|
||||
|
||||
loop {
|
||||
try {
|
||||
tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
|
||||
tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE);
|
||||
tr.setOption(FDBTransactionOptions::LOCK_AWARE);
|
||||
|
||||
WorkerBackupStatus status(self->backupEpoch, backupVersion, self->tag);
|
||||
tr.set(key, backupProgressValue(status));
|
||||
tr.addReadConflictRange(singleKeyRange(key));
|
||||
wait(tr.commit());
|
||||
return Void();
|
||||
} catch (Error& e) {
|
||||
wait(tr.onError(e));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
static std::string tagsToString(const VectorRef<Tag>& tags) {
|
||||
std::string s;
|
||||
bool first = true;
|
||||
for (auto t = tags.begin(); t != tags.end(); t++) {
|
||||
if (first) {
|
||||
first = false;
|
||||
} else {
|
||||
s.append(", ");
|
||||
}
|
||||
s.append((*t).toString());
|
||||
}
|
||||
return s;
|
||||
}
|
||||
|
||||
// Returns true if the message is a mutation that should be backuped, i.e.,
|
||||
// either key is not in system key space or is not a metadataVersionKey.
|
||||
bool isBackupMessage(const VersionedMessage& msg) {
|
||||
for (Tag tag : msg.tags) {
|
||||
if (tag.locality == tagLocalitySpecial || tag.locality == tagLocalityTxs) {
|
||||
return false; // skip Txs mutations
|
||||
}
|
||||
}
|
||||
|
||||
BinaryReader reader(msg.message.begin(), msg.message.size(), AssumeVersion(currentProtocolVersion));
|
||||
|
||||
// Return false for LogProtocolMessage.
|
||||
if (LogProtocolMessage::isNextIn(reader)) return false;
|
||||
|
||||
MutationRef m;
|
||||
reader >> m;
|
||||
|
||||
// check for metadataVersionKey and special metadata mutations
|
||||
if (!normalKeys.contains(m.param1) && m.param1 != metadataVersionKey) {
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
// Return a block of contiguous padding bytes, growing if needed.
|
||||
static Value makePadding(int size) {
|
||||
static Value pad;
|
||||
if (pad.size() < size) {
|
||||
pad = makeString(size);
|
||||
memset(mutateString(pad), '\xff', pad.size());
|
||||
}
|
||||
|
||||
return pad.substr(0, size);
|
||||
}
|
||||
|
||||
// Saves messages in the range of [0, numMsg) to a file and then remove these
|
||||
// messages. The file format is a sequence of (Version, sub#, msgSize, message),
|
||||
ACTOR Future<Void> saveMutationsToFile(BackupData* self, Version popVersion, int numMsg) {
|
||||
state int blockSize = SERVER_KNOBS->BACKUP_FILE_BLOCK_BYTES;
|
||||
state Reference<IBackupFile> logFile =
|
||||
wait(self->container->writeTaggedLogFile(self->messages[0].getVersion(), popVersion, blockSize, self->tag.id));
|
||||
TraceEvent("OpenMutationFile", self->myId)
|
||||
.detail("StartVersion", self->messages[0].getVersion())
|
||||
.detail("EndVersion", popVersion)
|
||||
.detail("BlockSize", blockSize)
|
||||
.detail("TagId", self->tag.id)
|
||||
.detail("File", logFile->getFileName());
|
||||
state int idx = 0;
|
||||
state int64_t blockEnd = 0;
|
||||
for (; idx < numMsg; idx++) {
|
||||
if (!isBackupMessage(self->messages[idx])) continue;
|
||||
|
||||
const int bytes = sizeof(Version) + sizeof(uint32_t) + sizeof(int) + self->messages[idx].message.size();
|
||||
// Start a new block if needed
|
||||
if (logFile->size() + bytes > blockEnd) {
|
||||
// Write padding if needed
|
||||
const int bytesLeft = blockEnd - logFile->size();
|
||||
if (bytesLeft > 0) {
|
||||
state Value paddingFFs = makePadding(bytesLeft);
|
||||
wait(logFile->append(paddingFFs.begin(), bytesLeft));
|
||||
}
|
||||
|
||||
blockEnd += blockSize;
|
||||
// TODO: add block header
|
||||
}
|
||||
|
||||
// Convert to big Endianness for version.version, version.sub, and msgSize
|
||||
// The decoder assumes 0xFF is the end, so little endian can easily be
|
||||
// mistaken as the end. In contrast, big endian for version almost guarantee
|
||||
// the first byte is not 0xFF (should always be 0x00).
|
||||
state int msgSize = self->messages[idx].message.size();
|
||||
BinaryWriter wr(Unversioned());
|
||||
wr << bigEndian64(self->messages[idx].version.version)
|
||||
<< bigEndian32(self->messages[idx].version.sub)
|
||||
<< bigEndian32(msgSize);
|
||||
Standalone<StringRef> buf = wr.toValue();
|
||||
wait(logFile->append((void*)buf.begin(), buf.size()));
|
||||
wait(logFile->append(self->messages[idx].message.begin(), msgSize));
|
||||
}
|
||||
|
||||
self->messages.erase(self->messages.begin(), self->messages.begin() + numMsg);
|
||||
wait(logFile->finish());
|
||||
TraceEvent("CloseMutationFile", self->myId)
|
||||
.detail("FileSize", logFile->size())
|
||||
.detail("TagId", self->tag.id)
|
||||
.detail("File", logFile->getFileName());
|
||||
|
||||
return Void();
|
||||
}
|
||||
|
||||
// Uploads self->messages to cloud storage and updates savedVersion.
|
||||
ACTOR Future<Void> uploadData(BackupData* self) {
|
||||
state Version popVersion = invalidVersion;
|
||||
|
||||
loop {
|
||||
if (self->endVersion.present() && self->savedVersion >= self->endVersion.get()) {
|
||||
self->messages.clear();
|
||||
return Void();
|
||||
}
|
||||
|
||||
// FIXME: knobify the delay of 10s. This delay is sensitive, as it is the
|
||||
// lag TLog might have. Changing to 20s may fail consistency check.
|
||||
state Future<Void> uploadDelay = delay(10);
|
||||
|
||||
if (self->messages.empty()) {
|
||||
// Even though messages is empty, we still want to advance popVersion.
|
||||
popVersion = std::max(popVersion, self->lastSeenVersion);
|
||||
} else {
|
||||
const Version maxPopVersion =
|
||||
self->endVersion.present() ? self->endVersion.get() : self->minKnownCommittedVersion;
|
||||
int numMsg = 0;
|
||||
for (const auto& message : self->messages) {
|
||||
if (message.getVersion() > maxPopVersion) break;
|
||||
popVersion = std::max(popVersion, message.getVersion());
|
||||
numMsg++;
|
||||
}
|
||||
if (numMsg > 0) {
|
||||
wait(saveMutationsToFile(self, popVersion, numMsg));
|
||||
}
|
||||
}
|
||||
if (self->pullFinished.get() && self->messages.empty()) {
|
||||
// Advance popVersion to the endVersion to avoid gap between last
|
||||
// message version and the endVersion.
|
||||
popVersion = self->endVersion.get();
|
||||
}
|
||||
|
||||
if (popVersion > self->savedVersion) {
|
||||
wait(saveProgress(self, popVersion));
|
||||
TraceEvent("BackupWorkerSavedProgress", self->myId)
|
||||
.detail("Tag", self->tag.toString())
|
||||
.detail("Version", popVersion)
|
||||
.detail("MsgQ", self->messages.size());
|
||||
self->savedVersion = std::max(popVersion, self->savedVersion);
|
||||
self->pop();
|
||||
}
|
||||
|
||||
if (!self->pullFinished.get()) {
|
||||
wait(uploadDelay || self->pullFinished.onChange());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Pulls data from TLog servers using LogRouter tag.
|
||||
ACTOR Future<Void> pullAsyncData(BackupData* self) {
|
||||
state Future<Void> logSystemChange = Void();
|
||||
state Reference<ILogSystem::IPeekCursor> r;
|
||||
state Version tagAt = self->startVersion;
|
||||
state Version lastVersion = 0;
|
||||
|
||||
loop {
|
||||
loop choose {
|
||||
when (wait(r ? r->getMore(TaskPriority::TLogCommit) : Never())) {
|
||||
break;
|
||||
}
|
||||
when (wait(logSystemChange)) {
|
||||
if (self->logSystem.get()) {
|
||||
r = self->logSystem.get()->peekLogRouter(self->myId, tagAt, self->tag);
|
||||
} else {
|
||||
r = Reference<ILogSystem::IPeekCursor>();
|
||||
}
|
||||
logSystemChange = self->logSystem.onChange();
|
||||
}
|
||||
}
|
||||
self->minKnownCommittedVersion = std::max(self->minKnownCommittedVersion, r->getMinKnownCommittedVersion());
|
||||
|
||||
// Note we aggressively peek (uncommitted) messages, but only committed
|
||||
// messages/mutations will be flushed to disk/blob in uploadData().
|
||||
while (r->hasMessage()) {
|
||||
self->messages.emplace_back(r->version(), r->getMessage(), r->getTags(), r->arena());
|
||||
r->nextMessage();
|
||||
}
|
||||
|
||||
tagAt = std::max(r->version().version, lastVersion);
|
||||
self->lastSeenVersion = std::max(tagAt, self->lastSeenVersion);
|
||||
TraceEvent("BackupWorkerGot", self->myId).suppressFor(1.0).detail("V", tagAt);
|
||||
if (self->endVersion.present() && tagAt > self->endVersion.get()) {
|
||||
self->eraseMessagesAfterEndVersion();
|
||||
TraceEvent("BackupWorkerFinishPull", self->myId)
|
||||
.detail("Tag", self->tag.toString())
|
||||
.detail("VersionGot", tagAt)
|
||||
.detail("EndVersion", self->endVersion.get())
|
||||
.detail("MsgQ", self->messages.size());
|
||||
self->pullFinished.set(true);
|
||||
return Void();
|
||||
}
|
||||
wait(yield());
|
||||
}
|
||||
}
|
||||
|
||||
ACTOR Future<Void> monitorBackupKeyOrPullData(BackupData* self) {
|
||||
loop {
|
||||
state Future<Void> started = monitorBackupStarted(self);
|
||||
|
||||
loop {
|
||||
GetReadVersionRequest request(1, GetReadVersionRequest::PRIORITY_DEFAULT |
|
||||
GetReadVersionRequest::FLAG_USE_MIN_KNOWN_COMMITTED_VERSION);
|
||||
|
||||
choose {
|
||||
when(wait(started)) { break; }
|
||||
when(wait(self->cx->onMasterProxiesChanged())) {}
|
||||
when(GetReadVersionReply reply = wait(loadBalance(self->cx->getMasterProxies(false),
|
||||
&MasterProxyInterface::getConsistentReadVersion,
|
||||
request, self->cx->taskID))) {
|
||||
self->savedVersion = std::max(reply.version, self->savedVersion);
|
||||
self->minKnownCommittedVersion = std::max(reply.version, self->minKnownCommittedVersion);
|
||||
self->pop(); // Pop while the worker is in this NOOP state.
|
||||
wait(delay(SERVER_KNOBS->BACKUP_NOOP_POP_DELAY, self->cx->taskID));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
TraceEvent("BackupWorkerStartPullData", self->myId);
|
||||
wait(pullAsyncData(self));
|
||||
}
|
||||
}
|
||||
|
||||
ACTOR Future<Void> checkRemoved(Reference<AsyncVar<ServerDBInfo>> db, LogEpoch recoveryCount,
|
||||
BackupData* self) {
|
||||
loop {
|
||||
bool isDisplaced =
|
||||
db->get().recoveryCount > recoveryCount && db->get().recoveryState != RecoveryState::UNINITIALIZED;
|
||||
if (isDisplaced) {
|
||||
TraceEvent("BackupWorkerDisplaced", self->myId)
|
||||
.detail("RecoveryCount", recoveryCount)
|
||||
.detail("SavedVersion", self->savedVersion)
|
||||
.detail("BackupWorkers", describe(db->get().logSystemConfig.tLogs))
|
||||
.detail("DBRecoveryCount", db->get().recoveryCount)
|
||||
.detail("RecoveryState", (int)db->get().recoveryState);
|
||||
throw worker_removed();
|
||||
}
|
||||
wait(db->onChange());
|
||||
}
|
||||
}
|
||||
|
||||
ACTOR Future<Void> backupWorker(BackupInterface interf, InitializeBackupRequest req,
|
||||
Reference<AsyncVar<ServerDBInfo>> db) {
|
||||
state BackupData self(db, req);
|
||||
state PromiseStream<Future<Void>> addActor;
|
||||
state Future<Void> error = actorCollection(addActor.getFuture());
|
||||
state Future<Void> dbInfoChange = Void();
|
||||
|
||||
TraceEvent("BackupWorkerStart", self.myId)
|
||||
.detail("Tag", req.routerTag.toString())
|
||||
.detail("StartVersion", req.startVersion)
|
||||
.detail("EndVersion", req.endVersion.present() ? req.endVersion.get() : -1)
|
||||
.detail("LogEpoch", req.recruitedEpoch)
|
||||
.detail("BackupEpoch", req.backupEpoch);
|
||||
try {
|
||||
addActor.send(monitorBackupKeyOrPullData(&self));
|
||||
addActor.send(checkRemoved(db, req.recruitedEpoch, &self));
|
||||
addActor.send(waitFailureServer(interf.waitFailure.getFuture()));
|
||||
|
||||
state Future<Void> done = uploadData(&self);
|
||||
|
||||
loop choose {
|
||||
when(wait(dbInfoChange)) {
|
||||
dbInfoChange = db->onChange();
|
||||
Reference<ILogSystem> ls = ILogSystem::fromServerDBInfo(self.myId, db->get(), true);
|
||||
bool hasPseudoLocality = ls.isValid() && ls->hasPseudoLocality(tagLocalityBackup);
|
||||
LogEpoch oldestBackupEpoch = 0;
|
||||
if (hasPseudoLocality) {
|
||||
self.logSystem.set(ls);
|
||||
self.pop();
|
||||
oldestBackupEpoch = ls->getOldestBackupEpoch();
|
||||
}
|
||||
TraceEvent("BackupWorkerLogSystem", self.myId)
|
||||
.detail("HasBackupLocality", hasPseudoLocality)
|
||||
.detail("OldestBackupEpoch", oldestBackupEpoch)
|
||||
.detail("Tag", self.tag.toString());
|
||||
}
|
||||
when(wait(done)) {
|
||||
TraceEvent("BackupWorkerDone", self.myId).detail("BackupEpoch", self.backupEpoch);
|
||||
// Notify master so that this worker can be removed from log system, then this
|
||||
// worker (for an old epoch's unfinished work) can safely exit.
|
||||
wait(brokenPromiseToNever(db->get().master.notifyBackupWorkerDone.getReply(
|
||||
BackupWorkerDoneRequest(self.myId, self.backupEpoch))));
|
||||
break;
|
||||
}
|
||||
when(wait(error)) {}
|
||||
}
|
||||
} catch (Error& e) {
|
||||
TraceEvent("BackupWorkerTerminated", self.myId).error(e, true);
|
||||
if (e.code() != error_code_actor_cancelled && e.code() != error_code_worker_removed) {
|
||||
throw;
|
||||
}
|
||||
}
|
||||
return Void();
|
||||
}
|
|
@ -1,6 +1,10 @@
|
|||
set(FDBSERVER_SRCS
|
||||
ApplyMetadataMutation.h
|
||||
ApplyMetadataMutation.cpp
|
||||
BackupInterface.h
|
||||
BackupProgress.actor.cpp
|
||||
BackupProgress.actor.h
|
||||
BackupWorker.actor.cpp
|
||||
ClusterController.actor.cpp
|
||||
ClusterRecruitmentInterface.h
|
||||
ConflictSet.h
|
||||
|
|
|
@ -18,9 +18,16 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
#include <algorithm>
|
||||
#include <iterator>
|
||||
#include <map>
|
||||
#include <set>
|
||||
#include <vector>
|
||||
|
||||
#include "fdbrpc/FailureMonitor.h"
|
||||
#include "flow/ActorCollection.h"
|
||||
#include "fdbclient/NativeAPI.actor.h"
|
||||
#include "fdbserver/BackupInterface.h"
|
||||
#include "fdbserver/CoordinationInterface.h"
|
||||
#include "fdbserver/DataDistributorInterface.h"
|
||||
#include "fdbserver/Knobs.h"
|
||||
|
@ -34,7 +41,6 @@
|
|||
#include "fdbserver/ServerDBInfo.h"
|
||||
#include "fdbserver/Status.h"
|
||||
#include "fdbserver/LatencyBandConfig.h"
|
||||
#include <algorithm>
|
||||
#include "fdbclient/DatabaseContext.h"
|
||||
#include "fdbserver/RecoveryState.h"
|
||||
#include "fdbclient/ReadYourWrites.h"
|
||||
|
@ -186,6 +192,7 @@ public:
|
|||
}
|
||||
serverInfo->set( newInfoCache );
|
||||
}
|
||||
|
||||
};
|
||||
|
||||
struct UpdateWorkerList {
|
||||
|
@ -773,6 +780,14 @@ public:
|
|||
}
|
||||
}
|
||||
|
||||
const int nBackup = std::max<int>(
|
||||
(req.configuration.desiredLogRouterCount > 0 ? req.configuration.desiredLogRouterCount : tlogs.size()),
|
||||
req.maxOldLogRouters);
|
||||
auto backupWorkers =
|
||||
getWorkersForRoleInDatacenter(dcId, ProcessClass::Backup, nBackup, req.configuration, id_used);
|
||||
std::transform(backupWorkers.begin(), backupWorkers.end(), std::back_inserter(result.backupWorkers),
|
||||
[](const WorkerDetails& w) { return w.interf; });
|
||||
|
||||
if( now() - startTime < SERVER_KNOBS->WAIT_FOR_GOOD_RECRUITMENT_DELAY &&
|
||||
( RoleFitness(SERVER_KNOBS->EXPECTED_TLOG_FITNESS, req.configuration.getDesiredLogs(), ProcessClass::TLog).betterCount(RoleFitness(tlogs, ProcessClass::TLog)) ||
|
||||
( region.satelliteTLogReplicationFactor > 0 && RoleFitness(SERVER_KNOBS->EXPECTED_TLOG_FITNESS, req.configuration.getDesiredSatelliteLogs(dcId), ProcessClass::TLog).betterCount(RoleFitness(satelliteLogs, ProcessClass::TLog)) ) ||
|
||||
|
@ -898,6 +913,14 @@ public:
|
|||
result.resolvers.push_back(resolvers[i].interf);
|
||||
for(int i = 0; i < proxies.size(); i++)
|
||||
result.proxies.push_back(proxies[i].interf);
|
||||
|
||||
const int nBackup = std::max<int>(tlogs.size(), req.maxOldLogRouters);
|
||||
auto backupWorkers = getWorkersForRoleInDatacenter(dcId, ProcessClass::Backup, nBackup,
|
||||
req.configuration, id_used);
|
||||
std::transform(backupWorkers.begin(), backupWorkers.end(),
|
||||
std::back_inserter(result.backupWorkers),
|
||||
[](const WorkerDetails& w) { return w.interf; });
|
||||
|
||||
break;
|
||||
} else {
|
||||
if(fitness < bestFitness) {
|
||||
|
@ -1014,6 +1037,8 @@ public:
|
|||
std::vector<WorkerDetails> satellite_tlogs;
|
||||
std::vector<WorkerDetails> log_routers;
|
||||
std::set<NetworkAddress> logRouterAddresses;
|
||||
std::vector<WorkerDetails> backup_workers;
|
||||
std::set<NetworkAddress> backup_addresses;
|
||||
|
||||
for( auto& logSet : dbi.logSystemConfig.tLogs ) {
|
||||
for( auto& it : logSet.tLogs ) {
|
||||
|
@ -1044,6 +1069,16 @@ public:
|
|||
log_routers.push_back(tlogWorker->second.details);
|
||||
}
|
||||
}
|
||||
|
||||
for (const auto& worker : logSet.backupWorkers) {
|
||||
auto workerIt = id_worker.find(worker.interf().locality.processId());
|
||||
if (workerIt == id_worker.end()) return false;
|
||||
if (workerIt->second.priorityInfo.isExcluded) return true;
|
||||
if (backup_addresses.count(workerIt->second.details.interf.address()) == 0) {
|
||||
backup_addresses.insert(workerIt->second.details.interf.address());
|
||||
backup_workers.push_back(workerIt->second.details);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Get proxy classes
|
||||
|
@ -1201,15 +1236,36 @@ public:
|
|||
if(oldInFit.proxy.betterFitness(newInFit.proxy) || oldInFit.resolver.betterFitness(newInFit.resolver)) {
|
||||
return false;
|
||||
}
|
||||
if(oldTLogFit > newTLogFit || oldInFit > newInFit || oldSatelliteTLogFit > newSatelliteTLogFit || oldRemoteTLogFit > newRemoteTLogFit || oldLogRoutersFit > newLogRoutersFit) {
|
||||
TraceEvent("BetterMasterExists", id).detail("OldMasterFit", oldMasterFit).detail("NewMasterFit", newMasterFit)
|
||||
.detail("OldTLogFit", oldTLogFit.toString()).detail("NewTLogFit", newTLogFit.toString())
|
||||
.detail("OldProxyFit", oldInFit.proxy.toString()).detail("NewProxyFit", newInFit.proxy.toString())
|
||||
.detail("OldResolverFit", oldInFit.resolver.toString()).detail("NewResolverFit", newInFit.resolver.toString())
|
||||
.detail("OldSatelliteFit", oldSatelliteTLogFit.toString()).detail("NewSatelliteFit", newSatelliteTLogFit.toString())
|
||||
.detail("OldRemoteFit", oldRemoteTLogFit.toString()).detail("NewRemoteFit", newRemoteTLogFit.toString())
|
||||
.detail("OldRouterFit", oldLogRoutersFit.toString()).detail("NewRouterFit", newLogRoutersFit.toString())
|
||||
.detail("OldSatelliteFallback", oldSatelliteFallback).detail("NewSatelliteFallback", newSatelliteFallback);
|
||||
|
||||
// Check backup worker fitness
|
||||
RoleFitness oldBackupWorkersFit(backup_workers, ProcessClass::Backup);
|
||||
const int nBackup = backup_addresses.size();
|
||||
RoleFitness newBackupWorkersFit(
|
||||
getWorkersForRoleInDatacenter(clusterControllerDcId, ProcessClass::Backup, nBackup, db.config, id_used),
|
||||
ProcessClass::Backup);
|
||||
|
||||
if (oldTLogFit > newTLogFit || oldInFit > newInFit || oldSatelliteTLogFit > newSatelliteTLogFit ||
|
||||
oldRemoteTLogFit > newRemoteTLogFit || oldLogRoutersFit > newLogRoutersFit ||
|
||||
oldBackupWorkersFit > newBackupWorkersFit) {
|
||||
TraceEvent("BetterMasterExists", id)
|
||||
.detail("OldMasterFit", oldMasterFit)
|
||||
.detail("NewMasterFit", newMasterFit)
|
||||
.detail("OldTLogFit", oldTLogFit.toString())
|
||||
.detail("NewTLogFit", newTLogFit.toString())
|
||||
.detail("OldProxyFit", oldInFit.proxy.toString())
|
||||
.detail("NewProxyFit", newInFit.proxy.toString())
|
||||
.detail("OldResolverFit", oldInFit.resolver.toString())
|
||||
.detail("NewResolverFit", newInFit.resolver.toString())
|
||||
.detail("OldSatelliteFit", oldSatelliteTLogFit.toString())
|
||||
.detail("NewSatelliteFit", newSatelliteTLogFit.toString())
|
||||
.detail("OldRemoteFit", oldRemoteTLogFit.toString())
|
||||
.detail("NewRemoteFit", newRemoteTLogFit.toString())
|
||||
.detail("OldRouterFit", oldLogRoutersFit.toString())
|
||||
.detail("NewRouterFit", newLogRoutersFit.toString())
|
||||
.detail("OldBackupWorkerFit", oldBackupWorkersFit.toString())
|
||||
.detail("NewBackupWorkerFit", newBackupWorkersFit.toString())
|
||||
.detail("OldSatelliteFallback", oldSatelliteFallback)
|
||||
.detail("NewSatelliteFallback", newSatelliteFallback);
|
||||
return true;
|
||||
}
|
||||
|
||||
|
@ -1293,8 +1349,8 @@ public:
|
|||
double startTime;
|
||||
Optional<double> remoteStartTime;
|
||||
Version datacenterVersionDifference;
|
||||
bool versionDifferenceUpdated;
|
||||
PromiseStream<Future<Void>> addActor;
|
||||
bool versionDifferenceUpdated;
|
||||
bool recruitingDistributor;
|
||||
Optional<UID> recruitingRatekeeperID;
|
||||
AsyncVar<bool> recruitRatekeeper;
|
||||
|
@ -2155,12 +2211,15 @@ void registerWorker( RegisterWorkerRequest req, ClusterControllerData *self ) {
|
|||
if (req.ratekeeperInterf.present()) {
|
||||
if((self->recruitingRatekeeperID.present() && self->recruitingRatekeeperID.get() != req.ratekeeperInterf.get().id()) ||
|
||||
self->clusterControllerDcId != w.locality.dcId()) {
|
||||
TraceEvent("CCHaltRegisteringRatekeeper", self->id).detail("RKID", req.ratekeeperInterf.get().id())
|
||||
.detail("DcID", printable(self->clusterControllerDcId))
|
||||
.detail("ReqDcID", printable(w.locality.dcId()))
|
||||
.detail("RecruitingRKID", self->recruitingRatekeeperID.present() ? self->recruitingRatekeeperID.get() : UID());
|
||||
self->id_worker[w.locality.processId()].haltRatekeeper = brokenPromiseToNever(req.ratekeeperInterf.get().haltRatekeeper.getReply(HaltRatekeeperRequest(self->id)));
|
||||
} else if(!self->recruitingRatekeeperID.present()) {
|
||||
TraceEvent("CCHaltRegisteringRatekeeper", self->id)
|
||||
.detail("RKID", req.ratekeeperInterf.get().id())
|
||||
.detail("DcID", printable(self->clusterControllerDcId))
|
||||
.detail("ReqDcID", printable(w.locality.dcId()))
|
||||
.detail("RecruitingRKID",
|
||||
self->recruitingRatekeeperID.present() ? self->recruitingRatekeeperID.get() : UID());
|
||||
self->id_worker[w.locality.processId()].haltRatekeeper = brokenPromiseToNever(
|
||||
req.ratekeeperInterf.get().haltRatekeeper.getReply(HaltRatekeeperRequest(self->id)));
|
||||
} else if (!self->recruitingRatekeeperID.present()) {
|
||||
const RatekeeperInterface& rki = req.ratekeeperInterf.get();
|
||||
const auto& ratekeeper = self->db.serverInfo->get().read().ratekeeper;
|
||||
TraceEvent("CCRegisterRatekeeper", self->id).detail("RKID", rki.id());
|
||||
|
|
|
@ -22,10 +22,13 @@
|
|||
#define FDBSERVER_CLUSTERRECRUITMENTINTERFACE_H
|
||||
#pragma once
|
||||
|
||||
#include <vector>
|
||||
|
||||
#include "fdbclient/ClusterInterface.h"
|
||||
#include "fdbclient/StorageServerInterface.h"
|
||||
#include "fdbclient/MasterProxyInterface.h"
|
||||
#include "fdbclient/DatabaseConfiguration.h"
|
||||
#include "fdbserver/BackupInterface.h"
|
||||
#include "fdbserver/DataDistributorInterface.h"
|
||||
#include "fdbserver/MasterInterface.h"
|
||||
#include "fdbserver/RecoveryState.h"
|
||||
|
@ -84,20 +87,22 @@ struct ClusterControllerFullInterface {
|
|||
|
||||
struct RecruitFromConfigurationReply {
|
||||
constexpr static FileIdentifier file_identifier = 2224085;
|
||||
vector<WorkerInterface> tLogs;
|
||||
vector<WorkerInterface> satelliteTLogs;
|
||||
vector<WorkerInterface> proxies;
|
||||
vector<WorkerInterface> resolvers;
|
||||
vector<WorkerInterface> storageServers;
|
||||
vector<WorkerInterface> oldLogRouters;
|
||||
std::vector<WorkerInterface> backupWorkers;
|
||||
std::vector<WorkerInterface> tLogs;
|
||||
std::vector<WorkerInterface> satelliteTLogs;
|
||||
std::vector<WorkerInterface> proxies;
|
||||
std::vector<WorkerInterface> resolvers;
|
||||
std::vector<WorkerInterface> storageServers;
|
||||
std::vector<WorkerInterface> oldLogRouters;
|
||||
Optional<Key> dcId;
|
||||
bool satelliteFallback;
|
||||
|
||||
RecruitFromConfigurationReply() : satelliteFallback(false) {}
|
||||
|
||||
template <class Ar>
|
||||
void serialize( Ar& ar ) {
|
||||
serializer(ar, tLogs, satelliteTLogs, proxies, resolvers, storageServers, oldLogRouters, dcId, satelliteFallback);
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, tLogs, satelliteTLogs, proxies, resolvers, storageServers, oldLogRouters, dcId,
|
||||
satelliteFallback, backupWorkers);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -120,8 +125,8 @@ struct RecruitFromConfigurationRequest {
|
|||
|
||||
struct RecruitRemoteFromConfigurationReply {
|
||||
constexpr static FileIdentifier file_identifier = 9091392;
|
||||
vector<WorkerInterface> remoteTLogs;
|
||||
vector<WorkerInterface> logRouters;
|
||||
std::vector<WorkerInterface> remoteTLogs;
|
||||
std::vector<WorkerInterface> logRouters;
|
||||
|
||||
template <class Ar>
|
||||
void serialize( Ar& ar ) {
|
||||
|
@ -230,18 +235,18 @@ struct RegisterMasterRequest {
|
|||
UID id;
|
||||
LocalityData mi;
|
||||
LogSystemConfig logSystemConfig;
|
||||
vector<MasterProxyInterface> proxies;
|
||||
vector<ResolverInterface> resolvers;
|
||||
std::vector<MasterProxyInterface> proxies;
|
||||
std::vector<ResolverInterface> resolvers;
|
||||
DBRecoveryCount recoveryCount;
|
||||
int64_t registrationCount;
|
||||
Optional<DatabaseConfiguration> configuration;
|
||||
vector<UID> priorCommittedLogServers;
|
||||
std::vector<UID> priorCommittedLogServers;
|
||||
RecoveryState recoveryState;
|
||||
bool recoveryStalled;
|
||||
|
||||
ReplyPromise<Void> reply;
|
||||
|
||||
RegisterMasterRequest() {}
|
||||
RegisterMasterRequest() : logSystemConfig(0) {}
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
|
|
|
@ -41,9 +41,8 @@ struct OldLogData;
|
|||
// At the moment, transaction logs are replicated but not partitioned, so the topology is as simple as a list of
|
||||
// transaction log replicas and the write quorum that was used to commit to them. The read quorum required to
|
||||
// ensure durability of locking and recovery is therefore tLogWriteAntiQuorum + 1.
|
||||
|
||||
struct CoreTLogSet {
|
||||
std::vector< UID > tLogs;
|
||||
std::vector<UID> tLogs;
|
||||
int32_t tLogWriteAntiQuorum; // The write anti quorum previously used to write to tLogs, which might be different from the anti quorum suggested by the current configuration going forward!
|
||||
int32_t tLogReplicationFactor; // The replication factor previously used to write to tLogs, which might be different from the current configuration
|
||||
std::vector< LocalityData > tLogLocalities; // Stores the localities of the log servers
|
||||
|
@ -57,9 +56,13 @@ struct CoreTLogSet {
|
|||
CoreTLogSet() : tLogWriteAntiQuorum(0), tLogReplicationFactor(0), isLocal(true), locality(tagLocalityUpgraded), startVersion(invalidVersion) {}
|
||||
explicit CoreTLogSet(const LogSet& logset);
|
||||
|
||||
bool operator == (CoreTLogSet const& rhs) const {
|
||||
return tLogs == rhs.tLogs && tLogWriteAntiQuorum == rhs.tLogWriteAntiQuorum && tLogReplicationFactor == rhs.tLogReplicationFactor && isLocal == rhs.isLocal && satelliteTagLocations == rhs.satelliteTagLocations &&
|
||||
locality == rhs.locality && startVersion == rhs.startVersion && ((!tLogPolicy && !rhs.tLogPolicy) || (tLogPolicy && rhs.tLogPolicy && (tLogPolicy->info() == rhs.tLogPolicy->info())));
|
||||
bool operator==(CoreTLogSet const& rhs) const {
|
||||
return tLogs == rhs.tLogs && tLogWriteAntiQuorum == rhs.tLogWriteAntiQuorum &&
|
||||
tLogReplicationFactor == rhs.tLogReplicationFactor && isLocal == rhs.isLocal &&
|
||||
satelliteTagLocations == rhs.satelliteTagLocations && locality == rhs.locality &&
|
||||
startVersion == rhs.startVersion &&
|
||||
((!tLogPolicy && !rhs.tLogPolicy) ||
|
||||
(tLogPolicy && rhs.tLogPolicy && (tLogPolicy->info() == rhs.tLogPolicy->info())));
|
||||
}
|
||||
|
||||
template <class Archive>
|
||||
|
@ -77,14 +80,17 @@ struct OldTLogCoreData {
|
|||
std::vector<CoreTLogSet> tLogs;
|
||||
int32_t logRouterTags;
|
||||
int32_t txsTags;
|
||||
Version epochEnd;
|
||||
Version epochBegin, epochEnd;
|
||||
std::set<int8_t> pseudoLocalities;
|
||||
LogEpoch epoch;
|
||||
|
||||
OldTLogCoreData() : epochEnd(0), logRouterTags(0), txsTags(0) {}
|
||||
OldTLogCoreData() : epochBegin(0), epochEnd(0), logRouterTags(0), txsTags(0), epoch(0) {}
|
||||
explicit OldTLogCoreData(const OldLogData&);
|
||||
|
||||
bool operator == (OldTLogCoreData const& rhs) const {
|
||||
return tLogs == rhs.tLogs && logRouterTags == rhs.logRouterTags && txsTags == rhs.txsTags && epochEnd == rhs.epochEnd && pseudoLocalities == rhs.pseudoLocalities;
|
||||
bool operator==(const OldTLogCoreData& rhs) const {
|
||||
return tLogs == rhs.tLogs && logRouterTags == rhs.logRouterTags && txsTags == rhs.txsTags &&
|
||||
epochBegin == rhs.epochBegin && epochEnd == rhs.epochEnd && pseudoLocalities == rhs.pseudoLocalities &&
|
||||
epoch == rhs.epoch;
|
||||
}
|
||||
|
||||
template <class Archive>
|
||||
|
@ -103,6 +109,9 @@ struct OldTLogCoreData {
|
|||
if (ar.protocolVersion().hasShardedTxsTags()) {
|
||||
serializer(ar, txsTags);
|
||||
}
|
||||
if (ar.protocolVersion().hasBackupWorker()) {
|
||||
serializer(ar, epoch, epochBegin);
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -134,10 +143,12 @@ struct DBCoreState {
|
|||
return priorCommittedLogServers;
|
||||
}
|
||||
|
||||
bool isEqual(DBCoreState const& r) const {
|
||||
return logSystemType == r.logSystemType && recoveryCount == r.recoveryCount && tLogs == r.tLogs && oldTLogData == r.oldTLogData && logRouterTags == r.logRouterTags && txsTags == r.txsTags && pseudoLocalities == r.pseudoLocalities;
|
||||
bool isEqual(const DBCoreState& r) const {
|
||||
return logSystemType == r.logSystemType && recoveryCount == r.recoveryCount && tLogs == r.tLogs &&
|
||||
oldTLogData == r.oldTLogData && logRouterTags == r.logRouterTags && txsTags == r.txsTags &&
|
||||
pseudoLocalities == r.pseudoLocalities;
|
||||
}
|
||||
bool operator == ( const DBCoreState& rhs ) const { return isEqual(rhs); }
|
||||
bool operator==(const DBCoreState& rhs) const { return isEqual(rhs); }
|
||||
|
||||
template <class Archive>
|
||||
void serialize(Archive& ar) {
|
||||
|
|
|
@ -354,6 +354,11 @@ ServerKnobs::ServerKnobs(bool randomize, ClientKnobs* clientKnobs, bool isSimula
|
|||
init( RESOLVER_STATE_MEMORY_LIMIT, 1e6 );
|
||||
init( LAST_LIMITED_RATIO, 2.0 );
|
||||
|
||||
// Backup Worker
|
||||
init( BACKUP_TIMEOUT, 0.4 );
|
||||
init( BACKUP_NOOP_POP_DELAY, 5.0 );
|
||||
init( BACKUP_FILE_BLOCK_BYTES, 1024 * 1024 );
|
||||
|
||||
//Cluster Controller
|
||||
init( CLUSTER_CONTROLLER_LOGGING_DELAY, 5.0 );
|
||||
init( MASTER_FAILURE_REACTION_TIME, 0.4 ); if( randomize && BUGGIFY ) MASTER_FAILURE_REACTION_TIME = 10.0;
|
||||
|
|
|
@ -292,6 +292,11 @@ public:
|
|||
double SAMPLE_POLL_TIME;
|
||||
int64_t RESOLVER_STATE_MEMORY_LIMIT;
|
||||
|
||||
// Backup Worker
|
||||
double BACKUP_TIMEOUT; // master's reaction time for backup failure
|
||||
double BACKUP_NOOP_POP_DELAY;
|
||||
int BACKUP_FILE_BLOCK_BYTES;
|
||||
|
||||
//Cluster Controller
|
||||
double CLUSTER_CONTROLLER_LOGGING_DELAY;
|
||||
double MASTER_FAILURE_REACTION_TIME;
|
||||
|
|
|
@ -73,11 +73,11 @@ struct LogRouterData {
|
|||
}
|
||||
};
|
||||
|
||||
UID dbgid;
|
||||
const UID dbgid;
|
||||
Reference<AsyncVar<Reference<ILogSystem>>> logSystem;
|
||||
NotifiedVersion version;
|
||||
NotifiedVersion minPopped;
|
||||
Version startVersion;
|
||||
const Version startVersion;
|
||||
Version minKnownCommittedVersion;
|
||||
Version poppedVersion;
|
||||
Deque<std::pair<Version, Standalone<VectorRef<uint8_t>>>> messageBlocks;
|
||||
|
@ -108,7 +108,7 @@ struct LogRouterData {
|
|||
|
||||
//only callable after getTagData returns a null reference
|
||||
Reference<TagData> createTagData(Tag tag, Version popped, Version knownCommittedVersion) {
|
||||
Reference<TagData> newTagData = Reference<TagData>( new TagData(tag, popped, knownCommittedVersion) );
|
||||
Reference<TagData> newTagData(new TagData(tag, popped, knownCommittedVersion));
|
||||
tag_data[tag.id] = newTagData;
|
||||
return newTagData;
|
||||
}
|
||||
|
@ -221,21 +221,18 @@ ACTOR Future<Void> pullAsyncData( LogRouterData *self ) {
|
|||
state Reference<ILogSystem::IPeekCursor> r;
|
||||
state Version tagAt = self->version.get() + 1;
|
||||
state Version lastVer = 0;
|
||||
state std::vector<int> tags;
|
||||
state std::vector<int> tags; // an optimization to avoid reallocating vector memory in every loop
|
||||
|
||||
loop {
|
||||
loop {
|
||||
choose {
|
||||
when(wait( r ? r->getMore(TaskPriority::TLogCommit) : Never() ) ) {
|
||||
break;
|
||||
}
|
||||
when( wait( dbInfoChange ) ) { //FIXME: does this actually happen?
|
||||
if( self->logSystem->get() )
|
||||
r = self->logSystem->get()->peekLogRouter( self->dbgid, tagAt, self->routerTag );
|
||||
else
|
||||
r = Reference<ILogSystem::IPeekCursor>();
|
||||
dbInfoChange = self->logSystem->onChange();
|
||||
loop choose {
|
||||
when(wait(r ? r->getMore(TaskPriority::TLogCommit) : Never())) { break; }
|
||||
when(wait(dbInfoChange)) { // FIXME: does this actually happen?
|
||||
if (self->logSystem->get()) {
|
||||
r = self->logSystem->get()->peekLogRouter(self->dbgid, tagAt, self->routerTag);
|
||||
} else {
|
||||
r = Reference<ILogSystem::IPeekCursor>();
|
||||
}
|
||||
dbInfoChange = self->logSystem->onChange();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -537,26 +534,11 @@ ACTOR Future<Void> logRouterCore(
|
|||
}
|
||||
|
||||
ACTOR Future<Void> checkRemoved(Reference<AsyncVar<ServerDBInfo>> db, uint64_t recoveryCount, TLogInterface myInterface) {
|
||||
loop{
|
||||
bool isDisplaced = ( (db->get().recoveryCount > recoveryCount && db->get().recoveryState != RecoveryState::UNINITIALIZED) || (db->get().recoveryCount == recoveryCount && db->get().recoveryState == RecoveryState::FULLY_RECOVERED) );
|
||||
if(isDisplaced) {
|
||||
for(auto& log : db->get().logSystemConfig.tLogs) {
|
||||
if( std::count( log.logRouters.begin(), log.logRouters.end(), myInterface.id() ) ) {
|
||||
isDisplaced = false;
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
if(isDisplaced) {
|
||||
for(auto& old : db->get().logSystemConfig.oldTLogs) {
|
||||
for(auto& log : old.tLogs) {
|
||||
if( std::count( log.logRouters.begin(), log.logRouters.end(), myInterface.id() ) ) {
|
||||
isDisplaced = false;
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
loop {
|
||||
bool isDisplaced =
|
||||
((db->get().recoveryCount > recoveryCount && db->get().recoveryState != RecoveryState::UNINITIALIZED) ||
|
||||
(db->get().recoveryCount == recoveryCount && db->get().recoveryState == RecoveryState::FULLY_RECOVERED));
|
||||
isDisplaced = isDisplaced && !db->get().logSystemConfig.hasLogRouter(myInterface.id());
|
||||
if (isDisplaced) {
|
||||
throw worker_removed();
|
||||
}
|
||||
|
|
|
@ -41,6 +41,7 @@ class LogSet : NonCopyable, public ReferenceCounted<LogSet> {
|
|||
public:
|
||||
std::vector<Reference<AsyncVar<OptionalInterface<TLogInterface>>>> logServers;
|
||||
std::vector<Reference<AsyncVar<OptionalInterface<TLogInterface>>>> logRouters;
|
||||
std::vector<Reference<AsyncVar<OptionalInterface<BackupInterface>>>> backupWorkers;
|
||||
int32_t tLogWriteAntiQuorum;
|
||||
int32_t tLogReplicationFactor;
|
||||
std::vector< LocalityData > tLogLocalities; // Stores the localities of the log servers
|
||||
|
@ -70,7 +71,7 @@ public:
|
|||
return result;
|
||||
}
|
||||
|
||||
bool hasLogRouter(UID id) {
|
||||
bool hasLogRouter(UID id) const {
|
||||
for (const auto& router : logRouters) {
|
||||
if (router->get().id() == id) {
|
||||
return true;
|
||||
|
@ -79,6 +80,15 @@ public:
|
|||
return false;
|
||||
}
|
||||
|
||||
bool hasBackupWorker(UID id) const {
|
||||
for (const auto& worker : backupWorkers) {
|
||||
if (worker->get().id() == id) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
std::string logServerString() {
|
||||
std::string result;
|
||||
for(int i = 0; i < logServers.size(); i++) {
|
||||
|
@ -704,6 +714,19 @@ struct ILogSystem {
|
|||
// Call only on an ILogSystem obtained from recoverAndEndEpoch()
|
||||
// Returns the first unreadable version number of the recovered epoch (i.e. message version numbers < (get_end(), 0) will be readable)
|
||||
|
||||
virtual Version getStartVersion() const = 0; // Returns the start version of current epoch.
|
||||
|
||||
struct EpochTagsVersionsInfo {
|
||||
int32_t logRouterTags; // Number of log router tags.
|
||||
Version epochBegin, epochEnd;
|
||||
|
||||
explicit EpochTagsVersionsInfo(int32_t n, Version begin, Version end)
|
||||
: logRouterTags(n), epochBegin(begin), epochEnd(end) {}
|
||||
};
|
||||
|
||||
// Returns EpochTagVersionsInfo for old epochs that this log system is aware of, excluding the current epoch.
|
||||
virtual std::map<LogEpoch, EpochTagsVersionsInfo> getOldEpochTagsVersionsInfo() const = 0;
|
||||
|
||||
virtual Future<Reference<ILogSystem>> newEpoch( struct RecruitFromConfigurationReply const& recr, Future<struct RecruitRemoteFromConfigurationReply> const& fRemoteWorkers, DatabaseConfiguration const& config,
|
||||
LogEpoch recoveryCount, int8_t primaryLocality, int8_t remoteLocality, std::vector<Tag> const& allTags, Reference<AsyncVar<bool>> const& recruitmentStalled ) = 0;
|
||||
// Call only on an ILogSystem obtained from recoverAndEndEpoch()
|
||||
|
@ -726,6 +749,7 @@ struct ILogSystem {
|
|||
virtual bool hasRemoteLogs() const = 0;
|
||||
|
||||
virtual Tag getRandomRouterTag() const = 0;
|
||||
virtual int getLogRouterTags() const = 0; // Returns the number of router tags.
|
||||
|
||||
virtual Tag getRandomTxsTag() const = 0;
|
||||
|
||||
|
@ -739,9 +763,20 @@ struct ILogSystem {
|
|||
// process class doesn't use pseudo tag, return the same tag.
|
||||
virtual Tag getPseudoPopTag(Tag tag, ProcessClass::ClassType type) = 0;
|
||||
|
||||
virtual bool isPseudoLocality(int8_t locality) = 0;
|
||||
virtual bool hasPseudoLocality(int8_t locality) = 0;
|
||||
|
||||
virtual Version popPseudoLocalityTag(int8_t locality, Version upTo) = 0;
|
||||
// Returns the actual version to be popped from the log router tag for the given pseudo tag.
|
||||
// For instance, a pseudo tag (-8, 2) means the actual popping tag is (-2, 2). Assuming there
|
||||
// are multiple pseudo tags, the returned version is the min(all pseudo tags' "upTo" versions).
|
||||
virtual Version popPseudoLocalityTag(Tag tag, Version upTo) = 0;
|
||||
|
||||
virtual void setBackupWorkers(const std::vector<InitializeBackupReply>& replies) = 0;
|
||||
|
||||
// Removes a finished backup worker from log system and returns true. Returns false
|
||||
// if the worker is not found.
|
||||
virtual bool removeBackupWorker(const BackupWorkerDoneRequest& req) = 0;
|
||||
|
||||
virtual LogEpoch getOldestBackupEpoch() const = 0;
|
||||
};
|
||||
|
||||
struct LengthPrefixedStringRef {
|
||||
|
|
|
@ -22,6 +22,7 @@
|
|||
#define FDBSERVER_LOGSYSTEMCONFIG_H
|
||||
#pragma once
|
||||
|
||||
#include "fdbserver/BackupInterface.h"
|
||||
#include "fdbserver/TLogInterface.h"
|
||||
#include "fdbrpc/ReplicationPolicy.h"
|
||||
#include "fdbclient/DatabaseConfiguration.h"
|
||||
|
@ -82,6 +83,7 @@ struct TLogSet {
|
|||
constexpr static FileIdentifier file_identifier = 6302317;
|
||||
std::vector<OptionalInterface<TLogInterface>> tLogs;
|
||||
std::vector<OptionalInterface<TLogInterface>> logRouters;
|
||||
std::vector<OptionalInterface<BackupInterface>> backupWorkers;
|
||||
int32_t tLogWriteAntiQuorum, tLogReplicationFactor;
|
||||
std::vector< LocalityData > tLogLocalities; // Stores the localities of the log servers
|
||||
TLogVersion tLogVersion;
|
||||
|
@ -95,12 +97,16 @@ struct TLogSet {
|
|||
explicit TLogSet(const LogSet& rhs);
|
||||
|
||||
std::string toString() const {
|
||||
return format("anti: %d replication: %d local: %d routers: %d tLogs: %s locality: %d", tLogWriteAntiQuorum, tLogReplicationFactor, isLocal, logRouters.size(), describe(tLogs).c_str(), locality);
|
||||
return format("anti: %d replication: %d local: %d routers: %d tLogs: %s backupWorkers: %s locality: %d",
|
||||
tLogWriteAntiQuorum, tLogReplicationFactor, isLocal, logRouters.size(), describe(tLogs).c_str(),
|
||||
describe(backupWorkers).c_str(), locality);
|
||||
}
|
||||
|
||||
bool operator == ( const TLogSet& rhs ) const {
|
||||
if (tLogWriteAntiQuorum != rhs.tLogWriteAntiQuorum || tLogReplicationFactor != rhs.tLogReplicationFactor || isLocal != rhs.isLocal || satelliteTagLocations != rhs.satelliteTagLocations ||
|
||||
startVersion != rhs.startVersion || tLogs.size() != rhs.tLogs.size() || locality != rhs.locality || logRouters.size() != rhs.logRouters.size()) {
|
||||
if (tLogWriteAntiQuorum != rhs.tLogWriteAntiQuorum || tLogReplicationFactor != rhs.tLogReplicationFactor ||
|
||||
isLocal != rhs.isLocal || satelliteTagLocations != rhs.satelliteTagLocations ||
|
||||
startVersion != rhs.startVersion || tLogs.size() != rhs.tLogs.size() || locality != rhs.locality ||
|
||||
logRouters.size() != rhs.logRouters.size() || backupWorkers.size() != rhs.backupWorkers.size()) {
|
||||
return false;
|
||||
}
|
||||
if ((tLogPolicy && !rhs.tLogPolicy) || (!tLogPolicy && rhs.tLogPolicy) || (tLogPolicy && (tLogPolicy->info() != rhs.tLogPolicy->info()))) {
|
||||
|
@ -116,6 +122,14 @@ struct TLogSet {
|
|||
return false;
|
||||
}
|
||||
}
|
||||
for (int j = 0; j < backupWorkers.size(); j++) {
|
||||
if (backupWorkers[j].id() != rhs.backupWorkers[j].id() ||
|
||||
backupWorkers[j].present() != rhs.backupWorkers[j].present() ||
|
||||
(backupWorkers[j].present() &&
|
||||
backupWorkers[j].interf().getToken() != rhs.backupWorkers[j].interf().getToken())) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
|
@ -137,38 +151,31 @@ struct TLogSet {
|
|||
|
||||
template <class Ar>
|
||||
void serialize( Ar& ar ) {
|
||||
if constexpr (is_fb_function<Ar>) {
|
||||
serializer(ar, tLogs, logRouters, tLogWriteAntiQuorum, tLogReplicationFactor, tLogPolicy, tLogLocalities,
|
||||
isLocal, locality, startVersion, satelliteTagLocations, tLogVersion);
|
||||
} else {
|
||||
serializer(ar, tLogs, logRouters, tLogWriteAntiQuorum, tLogReplicationFactor, tLogPolicy, tLogLocalities, isLocal, locality, startVersion, satelliteTagLocations);
|
||||
if (ar.isDeserializing && !ar.protocolVersion().hasTLogVersion()) {
|
||||
tLogVersion = TLogVersion::V2;
|
||||
} else {
|
||||
serializer(ar, tLogVersion);
|
||||
}
|
||||
ASSERT(tLogPolicy.getPtr() == nullptr || tLogVersion != TLogVersion::UNSET);
|
||||
}
|
||||
serializer(ar, tLogs, logRouters, tLogWriteAntiQuorum, tLogReplicationFactor, tLogPolicy, tLogLocalities,
|
||||
isLocal, locality, startVersion, satelliteTagLocations, tLogVersion, backupWorkers);
|
||||
}
|
||||
};
|
||||
|
||||
struct OldTLogConf {
|
||||
constexpr static FileIdentifier file_identifier = 16233772;
|
||||
std::vector<TLogSet> tLogs;
|
||||
Version epochEnd;
|
||||
Version epochBegin, epochEnd;
|
||||
int32_t logRouterTags;
|
||||
int32_t txsTags;
|
||||
std::set<int8_t> pseudoLocalities; // Tracking pseudo localities, e.g., tagLocalityLogRouterMapped, used in the old epoch.
|
||||
LogEpoch epoch;
|
||||
|
||||
OldTLogConf() : epochEnd(0), logRouterTags(0), txsTags(0) {}
|
||||
OldTLogConf() : epochBegin(0), epochEnd(0), logRouterTags(0), txsTags(0), epoch(0) {}
|
||||
explicit OldTLogConf(const OldLogData&);
|
||||
|
||||
std::string toString() const {
|
||||
return format("end: %d tags: %d %s", epochEnd, logRouterTags, describe(tLogs).c_str());
|
||||
}
|
||||
|
||||
bool operator == ( const OldTLogConf& rhs ) const {
|
||||
return tLogs == rhs.tLogs && epochEnd == rhs.epochEnd && logRouterTags == rhs.logRouterTags && txsTags == rhs.txsTags && pseudoLocalities == rhs.pseudoLocalities;
|
||||
bool operator==(const OldTLogConf& rhs) const {
|
||||
return tLogs == rhs.tLogs && epochBegin == rhs.epochBegin && epochEnd == rhs.epochEnd &&
|
||||
logRouterTags == rhs.logRouterTags && txsTags == rhs.txsTags &&
|
||||
pseudoLocalities == rhs.pseudoLocalities && epoch == rhs.epoch;
|
||||
}
|
||||
|
||||
bool isEqualIds(OldTLogConf const& r) const {
|
||||
|
@ -185,7 +192,7 @@ struct OldTLogConf {
|
|||
|
||||
template <class Ar>
|
||||
void serialize( Ar& ar ) {
|
||||
serializer(ar, tLogs, epochEnd, logRouterTags, pseudoLocalities, txsTags);
|
||||
serializer(ar, tLogs, epochBegin, epochEnd, logRouterTags, pseudoLocalities, txsTags, epoch);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -210,8 +217,12 @@ struct LogSystemConfig {
|
|||
bool stopped;
|
||||
Optional<Version> recoveredAt;
|
||||
std::set<int8_t> pseudoLocalities;
|
||||
LogEpoch epoch;
|
||||
LogEpoch oldestBackupEpoch;
|
||||
|
||||
LogSystemConfig() : logSystemType(LogSystemType::empty), logRouterTags(0), txsTags(0), expectedLogSets(0), stopped(false) {}
|
||||
LogSystemConfig(LogEpoch e = 0)
|
||||
: logSystemType(LogSystemType::empty), logRouterTags(0), txsTags(0), expectedLogSets(0), stopped(false), epoch(e),
|
||||
oldestBackupEpoch(e) {}
|
||||
|
||||
std::string toString() const {
|
||||
return format("type: %d oldGenerations: %d tags: %d %s", logSystemType, oldTLogs.size(), logRouterTags, describe(tLogs).c_str());
|
||||
|
@ -349,7 +360,10 @@ struct LogSystemConfig {
|
|||
bool operator == ( const LogSystemConfig& rhs ) const { return isEqual(rhs); }
|
||||
|
||||
bool isEqual(LogSystemConfig const& r) const {
|
||||
return logSystemType == r.logSystemType && tLogs == r.tLogs && oldTLogs == r.oldTLogs && expectedLogSets == r.expectedLogSets && logRouterTags == r.logRouterTags && txsTags == r.txsTags && recruitmentID == r.recruitmentID && stopped == r.stopped && recoveredAt == r.recoveredAt && pseudoLocalities == r.pseudoLocalities;
|
||||
return logSystemType == r.logSystemType && tLogs == r.tLogs && oldTLogs == r.oldTLogs &&
|
||||
expectedLogSets == r.expectedLogSets && logRouterTags == r.logRouterTags && txsTags == r.txsTags &&
|
||||
recruitmentID == r.recruitmentID && stopped == r.stopped && recoveredAt == r.recoveredAt &&
|
||||
pseudoLocalities == r.pseudoLocalities && epoch == r.epoch && oldestBackupEpoch == r.oldestBackupEpoch;
|
||||
}
|
||||
|
||||
bool isEqualIds(LogSystemConfig const& r) const {
|
||||
|
@ -378,9 +392,67 @@ struct LogSystemConfig {
|
|||
return false;
|
||||
}
|
||||
|
||||
bool hasTLog(UID tid) const {
|
||||
for (const auto& log : tLogs) {
|
||||
if (std::count(log.tLogs.begin(), log.tLogs.end(), tid) > 0) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
for (const auto& old : oldTLogs) {
|
||||
for (const auto& log : old.tLogs) {
|
||||
if (std::count(log.tLogs.begin(), log.tLogs.end(), tid) > 0) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
bool hasLogRouter(UID rid) const {
|
||||
for (const auto& log : tLogs) {
|
||||
if (std::count(log.logRouters.begin(), log.logRouters.end(), rid) > 0) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
for (const auto& old : oldTLogs) {
|
||||
for (const auto& log : old.tLogs) {
|
||||
if (std::count(log.logRouters.begin(), log.logRouters.end(), rid) > 0) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
bool hasBackupWorker(UID bid) const {
|
||||
for (const auto& log : tLogs) {
|
||||
if (std::count(log.backupWorkers.begin(), log.backupWorkers.end(), bid) > 0) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
for (const auto& old : oldTLogs) {
|
||||
for (const auto& log : old.tLogs) {
|
||||
if (std::count(log.backupWorkers.begin(), log.backupWorkers.end(), bid) > 0) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
Version getEpochEndVersion(LogEpoch epoch) const {
|
||||
for (const auto& old : oldTLogs) {
|
||||
if (old.epoch == epoch) {
|
||||
return old.epochEnd;
|
||||
}
|
||||
}
|
||||
return invalidVersion;
|
||||
}
|
||||
|
||||
template <class Ar>
|
||||
void serialize( Ar& ar ) {
|
||||
serializer(ar, logSystemType, tLogs, logRouterTags, oldTLogs, expectedLogSets, recruitmentID, stopped, recoveredAt, pseudoLocalities, txsTags);
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, logSystemType, tLogs, logRouterTags, oldTLogs, expectedLogSets, recruitmentID, stopped,
|
||||
recoveredAt, pseudoLocalities, txsTags, epoch, oldestBackupEpoch);
|
||||
}
|
||||
};
|
||||
|
||||
|
|
|
@ -37,6 +37,7 @@ struct MasterInterface {
|
|||
RequestStream< struct TLogRejoinRequest > tlogRejoin; // sent by tlog (whether or not rebooted) to communicate with a new master
|
||||
RequestStream< struct ChangeCoordinatorsRequest > changeCoordinators;
|
||||
RequestStream< struct GetCommitVersionRequest > getCommitVersion;
|
||||
RequestStream<struct BackupWorkerDoneRequest> notifyBackupWorkerDone;
|
||||
|
||||
NetworkAddress address() const { return changeCoordinators.getEndpoint().getPrimaryAddress(); }
|
||||
|
||||
|
@ -44,9 +45,9 @@ struct MasterInterface {
|
|||
template <class Archive>
|
||||
void serialize(Archive& ar) {
|
||||
if constexpr (!is_fb_function<Archive>) {
|
||||
ASSERT( ar.protocolVersion().isValid() );
|
||||
}
|
||||
serializer(ar, locality, waitFailure, tlogRejoin, changeCoordinators, getCommitVersion);
|
||||
ASSERT(ar.protocolVersion().isValid());
|
||||
}
|
||||
serializer(ar, locality, waitFailure, tlogRejoin, changeCoordinators, getCommitVersion, notifyBackupWorkerDone);
|
||||
}
|
||||
|
||||
void initEndpoints() {
|
||||
|
@ -157,6 +158,21 @@ struct GetCommitVersionRequest {
|
|||
}
|
||||
};
|
||||
|
||||
struct BackupWorkerDoneRequest {
|
||||
constexpr static FileIdentifier file_identifier = 8736351;
|
||||
UID workerUID;
|
||||
LogEpoch backupEpoch;
|
||||
ReplyPromise<Void> reply;
|
||||
|
||||
BackupWorkerDoneRequest() : workerUID(), backupEpoch(-1) {}
|
||||
BackupWorkerDoneRequest(UID id, LogEpoch epoch) : workerUID(id), backupEpoch(epoch) {}
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, workerUID, backupEpoch, reply);
|
||||
}
|
||||
};
|
||||
|
||||
struct LifetimeToken {
|
||||
UID ccID;
|
||||
int64_t count;
|
||||
|
|
|
@ -487,7 +487,7 @@ bool isWhitelisted(const vector<Standalone<StringRef>>& binPathVec, StringRef bi
|
|||
}
|
||||
|
||||
ACTOR Future<Void> addBackupMutations(ProxyCommitData* self, std::map<Key, MutationListRef>* logRangeMutations,
|
||||
LogPushData* toCommit, Version commitVersion) {
|
||||
LogPushData* toCommit, Version commitVersion) {
|
||||
state std::map<Key, MutationListRef>::iterator logRangeMutation = logRangeMutations->begin();
|
||||
state int32_t version = commitVersion / CLIENT_KNOBS->LOG_RANGE_BLOCK_SIZE;
|
||||
state int yieldBytes = 0;
|
||||
|
@ -1174,14 +1174,23 @@ struct TransactionRateInfo {
|
|||
}
|
||||
};
|
||||
|
||||
ACTOR Future<Void> sendGrvReplies(Future<GetReadVersionReply> replyFuture, std::vector<GetReadVersionRequest> requests, ProxyStats *stats) {
|
||||
ACTOR Future<Void> sendGrvReplies(Future<GetReadVersionReply> replyFuture, std::vector<GetReadVersionRequest> requests,
|
||||
ProxyStats* stats, Version minKnownCommittedVersion) {
|
||||
GetReadVersionReply reply = wait(replyFuture);
|
||||
|
||||
double end = timer();
|
||||
for(GetReadVersionRequest const& request : requests) {
|
||||
if(request.priority() >= GetReadVersionRequest::PRIORITY_DEFAULT) {
|
||||
stats->grvLatencyBands.addMeasurement(end - request.requestTime());
|
||||
}
|
||||
request.reply.send(reply);
|
||||
if (request.flags & GetReadVersionRequest::FLAG_USE_MIN_KNOWN_COMMITTED_VERSION) {
|
||||
// Only backup worker may infrequently use this flag.
|
||||
GetReadVersionReply minKCVReply = reply;
|
||||
minKCVReply.version = minKnownCommittedVersion;
|
||||
request.reply.send(minKCVReply);
|
||||
} else {
|
||||
request.reply.send(reply);
|
||||
}
|
||||
}
|
||||
|
||||
return Void();
|
||||
|
@ -1299,7 +1308,8 @@ ACTOR static Future<Void> transactionStarter(
|
|||
for (int i = 0; i < start.size(); i++) {
|
||||
if (start[i].size()) {
|
||||
Future<GetReadVersionReply> readVersionReply = getLiveCommittedVersion(commitData, i, &otherProxies, debugID, transactionsStarted[i], systemTransactionsStarted[i], defaultPriTransactionsStarted[i], batchPriTransactionsStarted[i]);
|
||||
addActor.send(sendGrvReplies(readVersionReply, start[i], &commitData->stats));
|
||||
addActor.send(sendGrvReplies(readVersionReply, start[i], &commitData->stats,
|
||||
commitData->minKnownCommittedVersion));
|
||||
|
||||
// for now, base dynamic batching on the time for normal requests (not read_risky)
|
||||
if (i == 0) {
|
||||
|
|
|
@ -1087,28 +1087,9 @@ namespace oldTLog_4_6 {
|
|||
loop {
|
||||
auto const& inf = self->dbInfo->get();
|
||||
bool isDisplaced = !std::count( inf.priorCommittedLogServers.begin(), inf.priorCommittedLogServers.end(), tli.id() );
|
||||
isDisplaced = isDisplaced && inf.recoveryCount >= recoveryCount && inf.recoveryState != RecoveryState::UNINITIALIZED;
|
||||
|
||||
if(isDisplaced) {
|
||||
for(auto& log : inf.logSystemConfig.tLogs) {
|
||||
if( std::count( log.tLogs.begin(), log.tLogs.end(), tli.id() ) ) {
|
||||
isDisplaced = false;
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
if(isDisplaced) {
|
||||
for(auto& old : inf.logSystemConfig.oldTLogs) {
|
||||
for(auto& log : old.tLogs) {
|
||||
if( std::count( log.tLogs.begin(), log.tLogs.end(), tli.id() ) ) {
|
||||
isDisplaced = false;
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
if ( isDisplaced )
|
||||
{
|
||||
isDisplaced = isDisplaced && inf.recoveryCount >= recoveryCount &&
|
||||
inf.recoveryState != RecoveryState::UNINITIALIZED && !inf.logSystemConfig.hasTLog(tli.id());
|
||||
if (isDisplaced) {
|
||||
TraceEvent("TLogDisplaced", tli.id()).detail("Reason", "DBInfoDoesNotContain").detail("RecoveryCount", recoveryCount).detail("InfRecoveryCount", inf.recoveryCount).detail("RecoveryState", (int)inf.recoveryState)
|
||||
.detail("LogSysConf", describe(inf.logSystemConfig.tLogs)).detail("PriorLogs", describe(inf.priorCommittedLogServers)).detail("OldLogGens", inf.logSystemConfig.oldTLogs.size());
|
||||
if (BUGGIFY) wait( delay( SERVER_KNOBS->BUGGIFY_WORKER_REMOVED_MAX_LAG * deterministicRandom()->random01() ) );
|
||||
|
|
|
@ -695,9 +695,14 @@ ACTOR Future<Void> tLogPopCore( TLogData* self, Tag inputTag, Version to, Refere
|
|||
}
|
||||
state Version upTo = to;
|
||||
int8_t tagLocality = inputTag.locality;
|
||||
if (logData->logSystem->get().isValid() && logData->logSystem->get()->isPseudoLocality(tagLocality)) {
|
||||
upTo = logData->logSystem->get()->popPseudoLocalityTag(tagLocality, to);
|
||||
tagLocality = tagLocalityLogRouter;
|
||||
if (isPseudoLocality(tagLocality)) {
|
||||
if (logData->logSystem->get().isValid()) {
|
||||
upTo = logData->logSystem->get()->popPseudoLocalityTag(inputTag, to);
|
||||
tagLocality = tagLocalityLogRouter;
|
||||
} else {
|
||||
TraceEvent(SevWarn, "TLogPopNoLogSystem", self->dbgid).detail("Locality", tagLocality).detail("Version", upTo);
|
||||
return Void();
|
||||
}
|
||||
}
|
||||
state Tag tag(tagLocality, inputTag.id);
|
||||
auto tagData = logData->getTagData(tag);
|
||||
|
@ -912,7 +917,7 @@ void commitMessages( TLogData* self, Reference<LogData> logData, Version version
|
|||
|
||||
for(auto& msg : taggedMessages) {
|
||||
if(msg.message.size() > block.capacity() - block.size()) {
|
||||
logData->messageBlocks.push_back( std::make_pair(version, block) );
|
||||
logData->messageBlocks.emplace_back(version, block);
|
||||
addedBytes += int64_t(block.size()) * SERVER_KNOBS->TLOG_MESSAGE_BLOCK_OVERHEAD_FACTOR;
|
||||
block = Standalone<VectorRef<uint8_t>>();
|
||||
block.reserve(block.arena(), std::max<int64_t>(SERVER_KNOBS->TLOG_MESSAGE_BLOCK_BYTES, msgSize));
|
||||
|
@ -947,7 +952,7 @@ void commitMessages( TLogData* self, Reference<LogData> logData, Version version
|
|||
}
|
||||
|
||||
if (version >= tagData->popped) {
|
||||
tagData->versionMessages.push_back(std::make_pair(version, LengthPrefixedStringRef((uint32_t*)(block.end() - msg.message.size()))));
|
||||
tagData->versionMessages.emplace_back(version, LengthPrefixedStringRef((uint32_t*)(block.end() - msg.message.size())));
|
||||
if(tagData->versionMessages.back().second.expectedSize() > SERVER_KNOBS->MAX_MESSAGE_SIZE) {
|
||||
TraceEvent(SevWarnAlways, "LargeMessage").detail("Size", tagData->versionMessages.back().second.expectedSize());
|
||||
}
|
||||
|
@ -967,7 +972,7 @@ void commitMessages( TLogData* self, Reference<LogData> logData, Version version
|
|||
|
||||
msgSize -= msg.message.size();
|
||||
}
|
||||
logData->messageBlocks.push_back( std::make_pair(version, block) );
|
||||
logData->messageBlocks.emplace_back(version, block);
|
||||
addedBytes += int64_t(block.size()) * SERVER_KNOBS->TLOG_MESSAGE_BLOCK_OVERHEAD_FACTOR;
|
||||
addedBytes += overheadBytes;
|
||||
|
||||
|
@ -1459,28 +1464,16 @@ ACTOR Future<Void> rejoinMasters( TLogData* self, TLogInterface tli, DBRecoveryC
|
|||
} else {
|
||||
isDisplaced = isDisplaced && ( ( inf.recoveryCount > recoveryCount && inf.recoveryState != RecoveryState::UNINITIALIZED ) || ( inf.recoveryCount == recoveryCount && inf.recoveryState == RecoveryState::FULLY_RECOVERED ) );
|
||||
}
|
||||
if(isDisplaced) {
|
||||
for(auto& log : inf.logSystemConfig.tLogs) {
|
||||
if( std::count( log.tLogs.begin(), log.tLogs.end(), tli.id() ) ) {
|
||||
isDisplaced = false;
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
if(isDisplaced) {
|
||||
for(auto& old : inf.logSystemConfig.oldTLogs) {
|
||||
for(auto& log : old.tLogs) {
|
||||
if( std::count( log.tLogs.begin(), log.tLogs.end(), tli.id() ) ) {
|
||||
isDisplaced = false;
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
if ( isDisplaced )
|
||||
{
|
||||
TraceEvent("TLogDisplaced", tli.id()).detail("Reason", "DBInfoDoesNotContain").detail("RecoveryCount", recoveryCount).detail("InfRecoveryCount", inf.recoveryCount).detail("RecoveryState", (int)inf.recoveryState)
|
||||
.detail("LogSysConf", describe(inf.logSystemConfig.tLogs)).detail("PriorLogs", describe(inf.priorCommittedLogServers)).detail("OldLogGens", inf.logSystemConfig.oldTLogs.size());
|
||||
isDisplaced = isDisplaced && !inf.logSystemConfig.hasTLog(tli.id());
|
||||
if (isDisplaced) {
|
||||
TraceEvent("TLogDisplaced", tli.id())
|
||||
.detail("Reason", "DBInfoDoesNotContain")
|
||||
.detail("RecoveryCount", recoveryCount)
|
||||
.detail("InfRecoveryCount", inf.recoveryCount)
|
||||
.detail("RecoveryState", (int)inf.recoveryState)
|
||||
.detail("LogSysConf", describe(inf.logSystemConfig.tLogs))
|
||||
.detail("PriorLogs", describe(inf.priorCommittedLogServers))
|
||||
.detail("OldLogGens", inf.logSystemConfig.oldTLogs.size());
|
||||
if (BUGGIFY) wait( delay( SERVER_KNOBS->BUGGIFY_WORKER_REMOVED_MAX_LAG * deterministicRandom()->random01() ) );
|
||||
throw worker_removed();
|
||||
}
|
||||
|
@ -1856,7 +1849,7 @@ ACTOR Future<Void> pullAsyncData( TLogData* self, Reference<LogData> logData, st
|
|||
}
|
||||
}
|
||||
|
||||
messages.push_back( TagsAndMessage(r->getMessageWithTags(), r->getTags()) );
|
||||
messages.emplace_back(r->getMessageWithTags(), r->getTags());
|
||||
r->nextMessage();
|
||||
}
|
||||
|
||||
|
|
|
@ -1121,7 +1121,7 @@ void commitMessages( TLogData* self, Reference<LogData> logData, Version version
|
|||
|
||||
for(auto& msg : taggedMessages) {
|
||||
if(msg.message.size() > block.capacity() - block.size()) {
|
||||
logData->messageBlocks.push_back( std::make_pair(version, block) );
|
||||
logData->messageBlocks.emplace_back(version, block);
|
||||
addedBytes += int64_t(block.size()) * SERVER_KNOBS->TLOG_MESSAGE_BLOCK_OVERHEAD_FACTOR;
|
||||
block = Standalone<VectorRef<uint8_t>>();
|
||||
block.reserve(block.arena(), std::max<int64_t>(SERVER_KNOBS->TLOG_MESSAGE_BLOCK_BYTES, msgSize));
|
||||
|
@ -1156,7 +1156,7 @@ void commitMessages( TLogData* self, Reference<LogData> logData, Version version
|
|||
}
|
||||
|
||||
if (version >= tagData->popped) {
|
||||
tagData->versionMessages.push_back(std::make_pair(version, LengthPrefixedStringRef((uint32_t*)(block.end() - msg.message.size()))));
|
||||
tagData->versionMessages.emplace_back(version, LengthPrefixedStringRef((uint32_t*)(block.end() - msg.message.size())));
|
||||
if(tagData->versionMessages.back().second.expectedSize() > SERVER_KNOBS->MAX_MESSAGE_SIZE) {
|
||||
TraceEvent(SevWarnAlways, "LargeMessage").detail("Size", tagData->versionMessages.back().second.expectedSize());
|
||||
}
|
||||
|
@ -1176,7 +1176,7 @@ void commitMessages( TLogData* self, Reference<LogData> logData, Version version
|
|||
|
||||
msgSize -= msg.message.size();
|
||||
}
|
||||
logData->messageBlocks.push_back( std::make_pair(version, block) );
|
||||
logData->messageBlocks.emplace_back(version, block);
|
||||
addedBytes += int64_t(block.size()) * SERVER_KNOBS->TLOG_MESSAGE_BLOCK_OVERHEAD_FACTOR;
|
||||
addedBytes += overheadBytes;
|
||||
|
||||
|
@ -1236,9 +1236,14 @@ ACTOR Future<Void> tLogPopCore( TLogData* self, Tag inputTag, Version to, Refere
|
|||
}
|
||||
state Version upTo = to;
|
||||
int8_t tagLocality = inputTag.locality;
|
||||
if (logData->logSystem->get().isValid() && logData->logSystem->get()->isPseudoLocality(tagLocality)) {
|
||||
upTo = logData->logSystem->get()->popPseudoLocalityTag(tagLocality, to);
|
||||
tagLocality = tagLocalityLogRouter;
|
||||
if (isPseudoLocality(tagLocality)) {
|
||||
if (logData->logSystem->get().isValid()) {
|
||||
upTo = logData->logSystem->get()->popPseudoLocalityTag(inputTag, to);
|
||||
tagLocality = tagLocalityLogRouter;
|
||||
} else {
|
||||
TraceEvent(SevWarn, "TLogPopNoLogSystem", self->dbgid).detail("Locality", tagLocality).detail("Version", upTo);
|
||||
return Void();
|
||||
}
|
||||
}
|
||||
state Tag tag(tagLocality, inputTag.id);
|
||||
auto tagData = logData->getTagData(tag);
|
||||
|
@ -1516,7 +1521,7 @@ ACTOR Future<Void> tLogPeekMessages( TLogData* self, TLogPeekRequest req, Refere
|
|||
if (sd.version >= req.begin) {
|
||||
firstVersion = std::min(firstVersion, sd.version);
|
||||
const IDiskQueue::location end = sd.start.lo + sd.length;
|
||||
commitLocations.push_back( std::make_pair(sd.start, end) );
|
||||
commitLocations.emplace_back(sd.start, end);
|
||||
// This isn't perfect, because we aren't accounting for page boundaries, but should be
|
||||
// close enough.
|
||||
commitBytes += sd.length;
|
||||
|
@ -1850,28 +1855,16 @@ ACTOR Future<Void> rejoinMasters( TLogData* self, TLogInterface tli, DBRecoveryC
|
|||
} else {
|
||||
isDisplaced = isDisplaced && ( ( inf.recoveryCount > recoveryCount && inf.recoveryState != RecoveryState::UNINITIALIZED ) || ( inf.recoveryCount == recoveryCount && inf.recoveryState == RecoveryState::FULLY_RECOVERED ) );
|
||||
}
|
||||
if(isDisplaced) {
|
||||
for(auto& log : inf.logSystemConfig.tLogs) {
|
||||
if( std::count( log.tLogs.begin(), log.tLogs.end(), tli.id() ) ) {
|
||||
isDisplaced = false;
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
if(isDisplaced) {
|
||||
for(auto& old : inf.logSystemConfig.oldTLogs) {
|
||||
for(auto& log : old.tLogs) {
|
||||
if( std::count( log.tLogs.begin(), log.tLogs.end(), tli.id() ) ) {
|
||||
isDisplaced = false;
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
if ( isDisplaced )
|
||||
{
|
||||
TraceEvent("TLogDisplaced", tli.id()).detail("Reason", "DBInfoDoesNotContain").detail("RecoveryCount", recoveryCount).detail("InfRecoveryCount", inf.recoveryCount).detail("RecoveryState", (int)inf.recoveryState)
|
||||
.detail("LogSysConf", describe(inf.logSystemConfig.tLogs)).detail("PriorLogs", describe(inf.priorCommittedLogServers)).detail("OldLogGens", inf.logSystemConfig.oldTLogs.size());
|
||||
isDisplaced = isDisplaced && !inf.logSystemConfig.hasTLog(tli.id());
|
||||
if (isDisplaced) {
|
||||
TraceEvent("TLogDisplaced", tli.id())
|
||||
.detail("Reason", "DBInfoDoesNotContain")
|
||||
.detail("RecoveryCount", recoveryCount)
|
||||
.detail("InfRecoveryCount", inf.recoveryCount)
|
||||
.detail("RecoveryState", (int)inf.recoveryState)
|
||||
.detail("LogSysConf", describe(inf.logSystemConfig.tLogs))
|
||||
.detail("PriorLogs", describe(inf.priorCommittedLogServers))
|
||||
.detail("OldLogGens", inf.logSystemConfig.oldTLogs.size());
|
||||
if (BUGGIFY) wait( delay( SERVER_KNOBS->BUGGIFY_WORKER_REMOVED_MAX_LAG * deterministicRandom()->random01() ) );
|
||||
throw worker_removed();
|
||||
}
|
||||
|
@ -2258,7 +2251,7 @@ ACTOR Future<Void> pullAsyncData( TLogData* self, Reference<LogData> logData, st
|
|||
}
|
||||
}
|
||||
|
||||
messages.push_back( TagsAndMessage(r->getMessageWithTags(), r->getTags()) );
|
||||
messages.emplace_back(r->getMessageWithTags(), r->getTags());
|
||||
r->nextMessage();
|
||||
}
|
||||
|
||||
|
@ -2449,7 +2442,7 @@ ACTOR Future<Void> restorePersistentState( TLogData* self, LocalityData locality
|
|||
logData->recoveryCount = BinaryReader::fromStringRef<DBRecoveryCount>( fRecoverCounts.get()[idx].value, Unversioned() );
|
||||
logData->removed = rejoinMasters(self, recruited, logData->recoveryCount, registerWithMaster.getFuture(), false);
|
||||
removed.push_back(errorOr(logData->removed));
|
||||
logsByVersion.push_back(std::make_pair(ver, id1));
|
||||
logsByVersion.emplace_back(ver, id1);
|
||||
|
||||
TraceEvent("TLogPersistentStateRestore", self->dbgid).detail("LogId", logData->logId).detail("Ver", ver);
|
||||
// Restore popped keys. Pop operations that took place after the last (committed) updatePersistentDataVersion might be lost, but
|
||||
|
|
|
@ -577,12 +577,13 @@ ACTOR Future<Void> waitForQuietDatabase( Database cx, Reference<AsyncVar<ServerD
|
|||
if(++numSuccesses == 3) {
|
||||
TraceEvent(("QuietDatabase" + phase + "Done").c_str());
|
||||
break;
|
||||
} else {
|
||||
wait(delay(2.0));
|
||||
}
|
||||
else
|
||||
wait(delay( 2.0 ) );
|
||||
}
|
||||
} catch (Error& e) {
|
||||
if( e.code() != error_code_actor_cancelled && e.code() != error_code_attribute_not_found && e.code() != error_code_timed_out)
|
||||
if (e.code() != error_code_actor_cancelled && e.code() != error_code_attribute_not_found &&
|
||||
e.code() != error_code_timed_out)
|
||||
TraceEvent(("QuietDatabase" + phase + "Error").c_str()).error(e);
|
||||
|
||||
//Client invalid operation occurs if we don't get back a message from one of the servers, often corrected by retrying
|
||||
|
|
|
@ -42,7 +42,7 @@ struct ServerDBInfo {
|
|||
Optional<DataDistributorInterface> distributor; // The best guess of current data distributor.
|
||||
MasterInterface master; // The best guess as to the most recent master, which might still be recovering
|
||||
Optional<RatekeeperInterface> ratekeeper;
|
||||
vector<ResolverInterface> resolvers;
|
||||
std::vector<ResolverInterface> resolvers;
|
||||
DBRecoveryCount recoveryCount; // A recovery count from DBCoreState. A successful master recovery increments it twice; unsuccessful recoveries may increment it once. Depending on where the current master is in its recovery process, this might not have been written by the current master.
|
||||
RecoveryState recoveryState;
|
||||
LifetimeToken masterLifetime; // Used by masterserver to detect not being the currently chosen master
|
||||
|
@ -52,7 +52,7 @@ struct ServerDBInfo {
|
|||
Optional<LatencyBandConfig> latencyBandConfig;
|
||||
std::vector<std::pair<uint16_t,StorageServerInterface>> storageCaches;
|
||||
|
||||
explicit ServerDBInfo() : recoveryCount(0), recoveryState(RecoveryState::UNINITIALIZED) {}
|
||||
explicit ServerDBInfo() : recoveryCount(0), recoveryState(RecoveryState::UNINITIALIZED), logSystemConfig(0) {}
|
||||
|
||||
bool operator == (ServerDBInfo const& r) const { return id == r.id; }
|
||||
bool operator != (ServerDBInfo const& r) const { return id != r.id; }
|
||||
|
|
|
@ -26,6 +26,7 @@
|
|||
#include "fdbclient/KeyRangeMap.h"
|
||||
#include "fdbclient/RunTransaction.actor.h"
|
||||
#include "fdbclient/SystemData.h"
|
||||
#include "fdbclient/FDBTypes.h"
|
||||
#include "fdbserver/WorkerInterface.actor.h"
|
||||
#include "fdbserver/TLogInterface.h"
|
||||
#include "fdbserver/Knobs.h"
|
||||
|
@ -983,9 +984,14 @@ ACTOR Future<Void> tLogPopCore( TLogData* self, Tag inputTag, Version to, Refere
|
|||
}
|
||||
state Version upTo = to;
|
||||
int8_t tagLocality = inputTag.locality;
|
||||
if (logData->logSystem->get().isValid() && logData->logSystem->get()->isPseudoLocality(tagLocality)) {
|
||||
upTo = logData->logSystem->get()->popPseudoLocalityTag(tagLocality, to);
|
||||
tagLocality = tagLocalityLogRouter;
|
||||
if (isPseudoLocality(tagLocality)) {
|
||||
if (logData->logSystem->get().isValid()) {
|
||||
upTo = logData->logSystem->get()->popPseudoLocalityTag(inputTag, to);
|
||||
tagLocality = tagLocalityLogRouter;
|
||||
} else {
|
||||
TraceEvent(SevWarn, "TLogPopNoLogSystem", self->dbgid).detail("Locality", tagLocality).detail("Version", upTo);
|
||||
return Void();
|
||||
}
|
||||
}
|
||||
state Tag tag(tagLocality, inputTag.id);
|
||||
auto tagData = logData->getTagData(tag);
|
||||
|
@ -1006,7 +1012,7 @@ ACTOR Future<Void> tLogPopCore( TLogData* self, Tag inputTag, Version to, Refere
|
|||
|
||||
if (upTo > logData->persistentDataDurableVersion)
|
||||
wait(tagData->eraseMessagesBefore(upTo, self, logData, TaskPriority::TLogPop));
|
||||
//TraceEvent("TLogPop", self->dbgid).detail("Tag", tag.toString()).detail("To", upTo);
|
||||
TraceEvent("TLogPop", logData->logId).detail("Tag", tag.toString()).detail("To", upTo);
|
||||
}
|
||||
return Void();
|
||||
}
|
||||
|
@ -1211,7 +1217,7 @@ void commitMessages( TLogData* self, Reference<LogData> logData, Version version
|
|||
|
||||
for(auto& msg : taggedMessages) {
|
||||
if(msg.message.size() > block.capacity() - block.size()) {
|
||||
logData->messageBlocks.push_back( std::make_pair(version, block) );
|
||||
logData->messageBlocks.emplace_back(version, block);
|
||||
addedBytes += int64_t(block.size()) * SERVER_KNOBS->TLOG_MESSAGE_BLOCK_OVERHEAD_FACTOR;
|
||||
block = Standalone<VectorRef<uint8_t>>();
|
||||
block.reserve(block.arena(), std::max<int64_t>(SERVER_KNOBS->TLOG_MESSAGE_BLOCK_BYTES, msgSize));
|
||||
|
@ -1246,7 +1252,7 @@ void commitMessages( TLogData* self, Reference<LogData> logData, Version version
|
|||
}
|
||||
|
||||
if (version >= tagData->popped) {
|
||||
tagData->versionMessages.push_back(std::make_pair(version, LengthPrefixedStringRef((uint32_t*)(block.end() - msg.message.size()))));
|
||||
tagData->versionMessages.emplace_back(version, LengthPrefixedStringRef((uint32_t*)(block.end() - msg.message.size())));
|
||||
if(tagData->versionMessages.back().second.expectedSize() > SERVER_KNOBS->MAX_MESSAGE_SIZE) {
|
||||
TraceEvent(SevWarnAlways, "LargeMessage").detail("Size", tagData->versionMessages.back().second.expectedSize());
|
||||
}
|
||||
|
@ -1266,7 +1272,7 @@ void commitMessages( TLogData* self, Reference<LogData> logData, Version version
|
|||
|
||||
msgSize -= msg.message.size();
|
||||
}
|
||||
logData->messageBlocks.push_back( std::make_pair(version, block) );
|
||||
logData->messageBlocks.emplace_back(version, block);
|
||||
addedBytes += int64_t(block.size()) * SERVER_KNOBS->TLOG_MESSAGE_BLOCK_OVERHEAD_FACTOR;
|
||||
addedBytes += overheadBytes;
|
||||
|
||||
|
@ -1535,7 +1541,7 @@ ACTOR Future<Void> tLogPeekMessages( TLogData* self, TLogPeekRequest req, Refere
|
|||
if (sd.version >= req.begin) {
|
||||
firstVersion = std::min(firstVersion, sd.version);
|
||||
const IDiskQueue::location end = sd.start.lo + sd.length;
|
||||
commitLocations.push_back( std::make_pair(sd.start, end) );
|
||||
commitLocations.emplace_back(sd.start, end);
|
||||
// This isn't perfect, because we aren't accounting for page boundaries, but should be
|
||||
// close enough.
|
||||
commitBytes += sd.length;
|
||||
|
@ -1871,28 +1877,16 @@ ACTOR Future<Void> rejoinMasters( TLogData* self, TLogInterface tli, DBRecoveryC
|
|||
} else {
|
||||
isDisplaced = isDisplaced && ( ( inf.recoveryCount > recoveryCount && inf.recoveryState != RecoveryState::UNINITIALIZED ) || ( inf.recoveryCount == recoveryCount && inf.recoveryState == RecoveryState::FULLY_RECOVERED ) );
|
||||
}
|
||||
if(isDisplaced) {
|
||||
for(auto& log : inf.logSystemConfig.tLogs) {
|
||||
if( std::count( log.tLogs.begin(), log.tLogs.end(), tli.id() ) ) {
|
||||
isDisplaced = false;
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
if(isDisplaced) {
|
||||
for(auto& old : inf.logSystemConfig.oldTLogs) {
|
||||
for(auto& log : old.tLogs) {
|
||||
if( std::count( log.tLogs.begin(), log.tLogs.end(), tli.id() ) ) {
|
||||
isDisplaced = false;
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
if ( isDisplaced )
|
||||
{
|
||||
TraceEvent("TLogDisplaced", tli.id()).detail("Reason", "DBInfoDoesNotContain").detail("RecoveryCount", recoveryCount).detail("InfRecoveryCount", inf.recoveryCount).detail("RecoveryState", (int)inf.recoveryState)
|
||||
.detail("LogSysConf", describe(inf.logSystemConfig.tLogs)).detail("PriorLogs", describe(inf.priorCommittedLogServers)).detail("OldLogGens", inf.logSystemConfig.oldTLogs.size());
|
||||
isDisplaced = isDisplaced && !inf.logSystemConfig.hasTLog(tli.id());
|
||||
if (isDisplaced) {
|
||||
TraceEvent("TLogDisplaced", tli.id())
|
||||
.detail("Reason", "DBInfoDoesNotContain")
|
||||
.detail("RecoveryCount", recoveryCount)
|
||||
.detail("InfRecoveryCount", inf.recoveryCount)
|
||||
.detail("RecoveryState", (int)inf.recoveryState)
|
||||
.detail("LogSysConf", describe(inf.logSystemConfig.tLogs))
|
||||
.detail("PriorLogs", describe(inf.priorCommittedLogServers))
|
||||
.detail("OldLogGens", inf.logSystemConfig.oldTLogs.size());
|
||||
if (BUGGIFY) wait( delay( SERVER_KNOBS->BUGGIFY_WORKER_REMOVED_MAX_LAG * deterministicRandom()->random01() ) );
|
||||
throw worker_removed();
|
||||
}
|
||||
|
@ -2152,9 +2146,7 @@ void removeLog( TLogData* self, Reference<LogData> logData ) {
|
|||
self->popOrder.pop_front();
|
||||
}
|
||||
|
||||
if(self->id_data.size()) {
|
||||
return;
|
||||
} else {
|
||||
if (self->id_data.size() == 0) {
|
||||
throw worker_removed();
|
||||
}
|
||||
}
|
||||
|
@ -2278,7 +2270,7 @@ ACTOR Future<Void> pullAsyncData( TLogData* self, Reference<LogData> logData, st
|
|||
}
|
||||
}
|
||||
|
||||
messages.push_back( TagsAndMessage(r->getMessageWithTags(), r->getTags()) );
|
||||
messages.emplace_back(r->getMessageWithTags(), r->getTags());
|
||||
r->nextMessage();
|
||||
}
|
||||
|
||||
|
@ -2471,7 +2463,7 @@ ACTOR Future<Void> restorePersistentState( TLogData* self, LocalityData locality
|
|||
logData->recoveryCount = BinaryReader::fromStringRef<DBRecoveryCount>( fRecoverCounts.get()[idx].value, Unversioned() );
|
||||
logData->removed = rejoinMasters(self, recruited, logData->recoveryCount, registerWithMaster.getFuture(), false);
|
||||
removed.push_back(errorOr(logData->removed));
|
||||
logsByVersion.push_back(std::make_pair(ver, id1));
|
||||
logsByVersion.emplace_back(ver, id1);
|
||||
|
||||
TraceEvent("TLogPersistentStateRestore", self->dbgid).detail("LogId", logData->logId).detail("Ver", ver);
|
||||
// Restore popped keys. Pop operations that took place after the last (committed) updatePersistentDataVersion might be lost, but
|
||||
|
@ -2605,27 +2597,25 @@ bool tlogTerminated( TLogData* self, IKeyValueStore* persistentData, TLogQueue*
|
|||
|
||||
ACTOR Future<Void> updateLogSystem(TLogData* self, Reference<LogData> logData, LogSystemConfig recoverFrom, Reference<AsyncVar<Reference<ILogSystem>>> logSystem) {
|
||||
loop {
|
||||
bool found = false;
|
||||
if(self->dbInfo->get().logSystemConfig.recruitmentID == logData->recruitmentID) {
|
||||
if( self->dbInfo->get().logSystemConfig.isNextGenerationOf(recoverFrom) ) {
|
||||
bool found = self->dbInfo->get().logSystemConfig.recruitmentID == logData->recruitmentID;
|
||||
if (found) {
|
||||
if (self->dbInfo->get().logSystemConfig.isNextGenerationOf(recoverFrom)) {
|
||||
logSystem->set(ILogSystem::fromOldLogSystemConfig( logData->logId, self->dbInfo->get().myLocality, self->dbInfo->get().logSystemConfig ));
|
||||
found = true;
|
||||
} else if( self->dbInfo->get().logSystemConfig.isEqualIds(recoverFrom) ) {
|
||||
} else if (self->dbInfo->get().logSystemConfig.isEqualIds(recoverFrom)) {
|
||||
logSystem->set(ILogSystem::fromLogSystemConfig( logData->logId, self->dbInfo->get().myLocality, self->dbInfo->get().logSystemConfig, false, true ));
|
||||
found = true;
|
||||
}
|
||||
else if( self->dbInfo->get().recoveryState >= RecoveryState::ACCEPTING_COMMITS ) {
|
||||
} else if (self->dbInfo->get().recoveryState >= RecoveryState::ACCEPTING_COMMITS) {
|
||||
logSystem->set(ILogSystem::fromLogSystemConfig( logData->logId, self->dbInfo->get().myLocality, self->dbInfo->get().logSystemConfig, true ));
|
||||
found = true;
|
||||
} else {
|
||||
found = false;
|
||||
}
|
||||
}
|
||||
if( !found ) {
|
||||
if (!found) {
|
||||
logSystem->set(Reference<ILogSystem>());
|
||||
} else {
|
||||
logData->logSystem->get()->pop(logData->logRouterPoppedVersion, logData->remoteTag, logData->durableKnownCommittedVersion, logData->locality);
|
||||
}
|
||||
TraceEvent("TLogUpdate", self->dbgid).detail("LogId", logData->logId).detail("RecruitmentID", logData->recruitmentID).detail("DbRecruitmentID", self->dbInfo->get().logSystemConfig.recruitmentID).detail("RecoverFrom", recoverFrom.toString()).detail("DbInfo", self->dbInfo->get().logSystemConfig.toString()).detail("Found", found).detail("LogSystem", (bool) logSystem->get() ).detail("RecoveryState", (int)self->dbInfo->get().recoveryState);
|
||||
for(auto it : self->dbInfo->get().logSystemConfig.oldTLogs) {
|
||||
for (const auto& it : self->dbInfo->get().logSystemConfig.oldTLogs) {
|
||||
TraceEvent("TLogUpdateOld", self->dbgid).detail("LogId", logData->logId).detail("DbInfo", it.toString());
|
||||
}
|
||||
wait( self->dbInfo->onChange() );
|
||||
|
|
|
@ -45,17 +45,17 @@ struct OldLogData {
|
|||
std::vector<Reference<LogSet>> tLogs;
|
||||
int32_t logRouterTags;
|
||||
int32_t txsTags; // The number of txsTags, which may change across generations.
|
||||
Version epochEnd;
|
||||
Version epochBegin, epochEnd;
|
||||
std::set<int8_t> pseudoLocalities;
|
||||
LogEpoch epoch;
|
||||
|
||||
OldLogData() : epochEnd(0), logRouterTags(0), txsTags(0) {}
|
||||
OldLogData() : epochBegin(0), epochEnd(0), logRouterTags(0), txsTags(0), epoch(0) {}
|
||||
|
||||
// Constructor for T of OldTLogConf and OldTLogCoreData
|
||||
template<class T>
|
||||
template <class T>
|
||||
explicit OldLogData(const T& conf)
|
||||
: logRouterTags(conf.logRouterTags), txsTags(conf.txsTags), epochEnd(conf.epochEnd),
|
||||
pseudoLocalities(conf.pseudoLocalities)
|
||||
{
|
||||
: logRouterTags(conf.logRouterTags), txsTags(conf.txsTags), epochBegin(conf.epochBegin), epochEnd(conf.epochEnd),
|
||||
pseudoLocalities(conf.pseudoLocalities), epoch(conf.epoch) {
|
||||
tLogs.resize(conf.tLogs.size());
|
||||
for (int j = 0; j < conf.tLogs.size(); j++) {
|
||||
Reference<LogSet> logSet(new LogSet(conf.tLogs[j]));
|
||||
|
@ -81,12 +81,15 @@ LogSet::LogSet(const TLogSet& tLogSet) :
|
|||
locality(tLogSet.locality), startVersion(tLogSet.startVersion),
|
||||
satelliteTagLocations(tLogSet.satelliteTagLocations)
|
||||
{
|
||||
for(const auto& log : tLogSet.tLogs) {
|
||||
for (const auto& log : tLogSet.tLogs) {
|
||||
logServers.emplace_back(new AsyncVar<OptionalInterface<TLogInterface>>(log));
|
||||
}
|
||||
for(const auto& log : tLogSet.logRouters) {
|
||||
for (const auto& log : tLogSet.logRouters) {
|
||||
logRouters.emplace_back(new AsyncVar<OptionalInterface<TLogInterface>>(log));
|
||||
}
|
||||
for (const auto& log : tLogSet.backupWorkers) {
|
||||
backupWorkers.emplace_back(new AsyncVar<OptionalInterface<BackupInterface>>(log));
|
||||
}
|
||||
filterLocalityDataForPolicy(tLogPolicy, &tLogLocalities);
|
||||
updateLocalitySet(tLogLocalities);
|
||||
}
|
||||
|
@ -99,9 +102,10 @@ LogSet::LogSet(const CoreTLogSet& coreSet) :
|
|||
locality(coreSet.locality), startVersion(coreSet.startVersion),
|
||||
satelliteTagLocations(coreSet.satelliteTagLocations)
|
||||
{
|
||||
for(const auto& log : coreSet.tLogs) {
|
||||
for (const auto& log : coreSet.tLogs) {
|
||||
logServers.emplace_back(new AsyncVar<OptionalInterface<TLogInterface>>(OptionalInterface<TLogInterface>(log)));
|
||||
}
|
||||
// Do NOT recover coreSet.backupWorkers, because master will recruit new ones.
|
||||
filterLocalityDataForPolicy(tLogPolicy, &tLogLocalities);
|
||||
updateLocalitySet(tLogLocalities);
|
||||
}
|
||||
|
@ -117,16 +121,17 @@ TLogSet::TLogSet(const LogSet& rhs) :
|
|||
for (const auto& tlog : rhs.logServers) {
|
||||
tLogs.push_back(tlog->get());
|
||||
}
|
||||
|
||||
for (const auto& logRouter : rhs.logRouters) {
|
||||
logRouters.push_back(logRouter->get());
|
||||
}
|
||||
for (const auto& worker : rhs.backupWorkers) {
|
||||
backupWorkers.push_back(worker->get());
|
||||
}
|
||||
}
|
||||
|
||||
OldTLogConf::OldTLogConf(const OldLogData& oldLogData) :
|
||||
logRouterTags(oldLogData.logRouterTags), txsTags(oldLogData.txsTags), epochEnd(oldLogData.epochEnd),
|
||||
pseudoLocalities(oldLogData.pseudoLocalities)
|
||||
{
|
||||
OldTLogConf::OldTLogConf(const OldLogData& oldLogData)
|
||||
: logRouterTags(oldLogData.logRouterTags), txsTags(oldLogData.txsTags), epochBegin(oldLogData.epochBegin),
|
||||
epochEnd(oldLogData.epochEnd), pseudoLocalities(oldLogData.pseudoLocalities), epoch(oldLogData.epoch) {
|
||||
for (const Reference<LogSet>& logSet : oldLogData.tLogs) {
|
||||
tLogs.emplace_back(*logSet);
|
||||
}
|
||||
|
@ -144,12 +149,12 @@ CoreTLogSet::CoreTLogSet(const LogSet& logset) :
|
|||
for (const auto &log : logset.logServers) {
|
||||
tLogs.push_back(log->get().id());
|
||||
}
|
||||
// Do NOT store logset.backupWorkers, because master will recruit new ones.
|
||||
}
|
||||
|
||||
OldTLogCoreData::OldTLogCoreData(const OldLogData& oldData) :
|
||||
logRouterTags(oldData.logRouterTags), txsTags(oldData.txsTags), epochEnd(oldData.epochEnd),
|
||||
pseudoLocalities(oldData.pseudoLocalities)
|
||||
{
|
||||
OldTLogCoreData::OldTLogCoreData(const OldLogData& oldData)
|
||||
: logRouterTags(oldData.logRouterTags), txsTags(oldData.txsTags), epochBegin(oldData.epochBegin),
|
||||
epochEnd(oldData.epochEnd), pseudoLocalities(oldData.pseudoLocalities), epoch(oldData.epoch) {
|
||||
for (const Reference<LogSet>& logSet : oldData.tLogs) {
|
||||
if (logSet->logServers.size()) {
|
||||
tLogs.emplace_back(*logSet);
|
||||
|
@ -158,7 +163,7 @@ OldTLogCoreData::OldTLogCoreData(const OldLogData& oldData) :
|
|||
}
|
||||
|
||||
struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogSystem> {
|
||||
UID dbgid;
|
||||
const UID dbgid;
|
||||
LogSystemType logSystemType;
|
||||
std::vector<Reference<LogSet>> tLogs;
|
||||
int expectedLogSets;
|
||||
|
@ -168,9 +173,11 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
|
|||
int repopulateRegionAntiQuorum;
|
||||
bool stopped;
|
||||
std::set<int8_t> pseudoLocalities; // Represent special localities that will be mapped to tagLocalityLogRouter
|
||||
std::map<int8_t, Version> pseudoLocalityPopVersion;
|
||||
const LogEpoch epoch;
|
||||
LogEpoch oldestBackupEpoch;
|
||||
|
||||
// new members
|
||||
std::map<Tag, Version> pseudoLocalityPopVersion;
|
||||
Future<Void> rejoins;
|
||||
Future<Void> recoveryComplete;
|
||||
Future<Void> remoteRecovery;
|
||||
|
@ -179,6 +186,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
|
|||
AsyncVar<bool> recoveryCompleteWrittenToCoreState;
|
||||
bool remoteLogsWrittenToCoreState;
|
||||
bool hasRemoteServers;
|
||||
AsyncTrigger backupWorkerChanged;
|
||||
|
||||
Optional<Version> recoverAt;
|
||||
Optional<Version> recoveredAt;
|
||||
|
@ -190,21 +198,24 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
|
|||
std::vector<OldLogData> oldLogData;
|
||||
AsyncTrigger logSystemConfigChanged;
|
||||
|
||||
TagPartitionedLogSystem( UID dbgid, LocalityData locality, Optional<PromiseStream<Future<Void>>> addActor = Optional<PromiseStream<Future<Void>>>() ) : dbgid(dbgid), locality(locality), addActor(addActor), popActors(false), recoveryCompleteWrittenToCoreState(false), remoteLogsWrittenToCoreState(false), logSystemType(LogSystemType::empty), logRouterTags(0), txsTags(0), expectedLogSets(0), hasRemoteServers(false), stopped(false), repopulateRegionAntiQuorum(0) {}
|
||||
TagPartitionedLogSystem(UID dbgid, LocalityData locality, LogEpoch e,
|
||||
Optional<PromiseStream<Future<Void>>> addActor = Optional<PromiseStream<Future<Void>>>())
|
||||
: dbgid(dbgid), logSystemType(LogSystemType::empty), expectedLogSets(0), logRouterTags(0), txsTags(0),
|
||||
repopulateRegionAntiQuorum(0), epoch(e), oldestBackupEpoch(e), recoveryCompleteWrittenToCoreState(false),
|
||||
locality(locality), remoteLogsWrittenToCoreState(false), hasRemoteServers(false), stopped(false),
|
||||
addActor(addActor), popActors(false) {}
|
||||
|
||||
virtual void stopRejoins() {
|
||||
rejoins = Future<Void>();
|
||||
}
|
||||
void stopRejoins() override { rejoins = Future<Void>(); }
|
||||
|
||||
virtual void addref() {
|
||||
void addref() override {
|
||||
ReferenceCounted<TagPartitionedLogSystem>::addref();
|
||||
}
|
||||
|
||||
virtual void delref() {
|
||||
void delref() override {
|
||||
ReferenceCounted<TagPartitionedLogSystem>::delref();
|
||||
}
|
||||
|
||||
virtual std::string describe() {
|
||||
std::string describe() override {
|
||||
std::string result;
|
||||
for( int i = 0; i < tLogs.size(); i++ ) {
|
||||
result += format("%d: ", i);
|
||||
|
@ -215,42 +226,50 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
|
|||
return result;
|
||||
}
|
||||
|
||||
virtual UID getDebugID() {
|
||||
return dbgid;
|
||||
}
|
||||
UID getDebugID() override { return dbgid; }
|
||||
|
||||
void addPseudoLocality(int8_t locality) {
|
||||
ASSERT(locality < 0);
|
||||
pseudoLocalities.insert(locality);
|
||||
pseudoLocalityPopVersion[locality] = 0;
|
||||
for (uint16_t i = 0; i < logRouterTags; i++) {
|
||||
pseudoLocalityPopVersion[Tag(locality, i)] = 0;
|
||||
}
|
||||
}
|
||||
|
||||
Tag getPseudoPopTag(Tag tag, ProcessClass::ClassType type) override {
|
||||
switch (type) {
|
||||
case ProcessClass::LogRouterClass:
|
||||
if (tag.locality == tagLocalityLogRouter && pseudoLocalities.count(tag.locality) > 0) {
|
||||
if (tag.locality == tagLocalityLogRouter) {
|
||||
ASSERT(pseudoLocalities.count(tagLocalityLogRouterMapped) > 0);
|
||||
tag.locality = tagLocalityLogRouterMapped;
|
||||
}
|
||||
break;
|
||||
|
||||
default:
|
||||
case ProcessClass::BackupClass:
|
||||
if (tag.locality == tagLocalityLogRouter) {
|
||||
ASSERT(pseudoLocalities.count(tagLocalityBackup) > 0);
|
||||
tag.locality = tagLocalityBackup;
|
||||
}
|
||||
break;
|
||||
|
||||
default: // This should be an error at caller site.
|
||||
break;
|
||||
}
|
||||
return tag;
|
||||
}
|
||||
|
||||
bool isPseudoLocality(int8_t locality) override {
|
||||
return pseudoLocalities.count(locality) > 0;
|
||||
}
|
||||
bool hasPseudoLocality(int8_t locality) override { return pseudoLocalities.count(locality) > 0; }
|
||||
|
||||
Version popPseudoLocalityTag(int8_t locality, Version upTo) override {
|
||||
ASSERT(isPseudoLocality(locality));
|
||||
auto& localityVersion = pseudoLocalityPopVersion[locality];
|
||||
Version popPseudoLocalityTag(Tag tag, Version upTo) override {
|
||||
ASSERT(isPseudoLocality(tag.locality) && hasPseudoLocality(tag.locality));
|
||||
|
||||
Version& localityVersion = pseudoLocalityPopVersion[tag];
|
||||
localityVersion = std::max(localityVersion, upTo);
|
||||
Version minVersion = localityVersion;
|
||||
for (const auto& it : pseudoLocalityPopVersion) {
|
||||
minVersion = std::min(minVersion, it.second);
|
||||
for (const int8_t locality : pseudoLocalities) {
|
||||
minVersion = std::min(minVersion, pseudoLocalityPopVersion[Tag(locality, tag.id)]);
|
||||
}
|
||||
TraceEvent("Pop", dbgid).detail("Tag", tag.toString()).detail("Version", upTo).detail("PopVersion", minVersion);
|
||||
return minVersion;
|
||||
}
|
||||
|
||||
|
@ -261,7 +280,8 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
|
|||
static Reference<ILogSystem> fromLogSystemConfig( UID const& dbgid, LocalityData const& locality, LogSystemConfig const& lsConf, bool excludeRemote, bool useRecoveredAt, Optional<PromiseStream<Future<Void>>> addActor ) {
|
||||
ASSERT(lsConf.logSystemType == LogSystemType::tagPartitioned || (lsConf.logSystemType == LogSystemType::empty && !lsConf.tLogs.size()));
|
||||
//ASSERT(lsConf.epoch == epoch); //< FIXME
|
||||
Reference<TagPartitionedLogSystem> logSystem( new TagPartitionedLogSystem(dbgid, locality, addActor) );
|
||||
Reference<TagPartitionedLogSystem> logSystem(
|
||||
new TagPartitionedLogSystem(dbgid, locality, lsConf.epoch, addActor));
|
||||
|
||||
logSystem->tLogs.reserve(lsConf.tLogs.size());
|
||||
logSystem->expectedLogSets = lsConf.expectedLogSets;
|
||||
|
@ -281,6 +301,9 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
|
|||
|
||||
for (const auto& oldTlogConf : lsConf.oldTLogs) {
|
||||
logSystem->oldLogData.emplace_back(oldTlogConf);
|
||||
//TraceEvent("BWFromLSConf")
|
||||
// .detail("Epoch", logSystem->oldLogData.back().epoch)
|
||||
// .detail("Version", logSystem->oldLogData.back().epochEnd);
|
||||
}
|
||||
|
||||
logSystem->logSystemType = lsConf.logSystemType;
|
||||
|
@ -290,7 +313,8 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
|
|||
static Reference<ILogSystem> fromOldLogSystemConfig( UID const& dbgid, LocalityData const& locality, LogSystemConfig const& lsConf ) {
|
||||
ASSERT( lsConf.logSystemType == LogSystemType::tagPartitioned || (lsConf.logSystemType == LogSystemType::empty && !lsConf.tLogs.size()) );
|
||||
//ASSERT(lsConf.epoch == epoch); //< FIXME
|
||||
Reference<TagPartitionedLogSystem> logSystem( new TagPartitionedLogSystem(dbgid, locality) );
|
||||
const LogEpoch e = lsConf.oldTLogs.size() > 0 ? lsConf.oldTLogs[0].epoch : 0;
|
||||
Reference<TagPartitionedLogSystem> logSystem(new TagPartitionedLogSystem(dbgid, locality, e));
|
||||
|
||||
if (lsConf.oldTLogs.size()) {
|
||||
for (const TLogSet& tLogSet : lsConf.oldTLogs[0].tLogs) {
|
||||
|
@ -311,7 +335,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
|
|||
return logSystem;
|
||||
}
|
||||
|
||||
virtual void toCoreState( DBCoreState& newState ) {
|
||||
void toCoreState(DBCoreState& newState) override {
|
||||
if( recoveryComplete.isValid() && recoveryComplete.isError() )
|
||||
throw recoveryComplete.getError();
|
||||
|
||||
|
@ -336,17 +360,21 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
|
|||
if(!recoveryComplete.isValid() || !recoveryComplete.isReady() || (repopulateRegionAntiQuorum == 0 && (!remoteRecoveryComplete.isValid() || !remoteRecoveryComplete.isReady()))) {
|
||||
for (const auto& oldData : oldLogData) {
|
||||
newState.oldTLogData.emplace_back(oldData);
|
||||
TraceEvent("BWToCore")
|
||||
.detail("Epoch", newState.oldTLogData.back().epoch)
|
||||
.detail("BeginVersion", newState.oldTLogData.back().epochBegin)
|
||||
.detail("EndVersion", newState.oldTLogData.back().epochEnd);
|
||||
}
|
||||
}
|
||||
|
||||
newState.logSystemType = logSystemType;
|
||||
}
|
||||
|
||||
virtual bool remoteStorageRecovered() {
|
||||
bool remoteStorageRecovered() override {
|
||||
return remoteRecoveryComplete.isValid() && remoteRecoveryComplete.isReady();
|
||||
}
|
||||
|
||||
virtual Future<Void> onCoreStateChanged() {
|
||||
Future<Void> onCoreStateChanged() override {
|
||||
std::vector<Future<Void>> changes;
|
||||
changes.push_back(Never());
|
||||
if(recoveryComplete.isValid() && !recoveryComplete.isReady()) {
|
||||
|
@ -361,8 +389,8 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
|
|||
return waitForAny(changes);
|
||||
}
|
||||
|
||||
virtual void coreStateWritten( DBCoreState const& newState ) {
|
||||
if( !newState.oldTLogData.size() ) {
|
||||
void coreStateWritten(DBCoreState const& newState) override {
|
||||
if (!newState.oldTLogData.size()) {
|
||||
recoveryCompleteWrittenToCoreState.set(true);
|
||||
}
|
||||
for(auto& t : newState.tLogs) {
|
||||
|
@ -374,15 +402,16 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
|
|||
}
|
||||
}
|
||||
|
||||
virtual Future<Void> onError() {
|
||||
Future<Void> onError() override {
|
||||
return onError_internal(this);
|
||||
}
|
||||
|
||||
ACTOR static Future<Void> onError_internal( TagPartitionedLogSystem* self ) {
|
||||
// Never returns normally, but throws an error if the subsystem stops working
|
||||
loop {
|
||||
vector<Future<Void>> failed;
|
||||
vector<Future<Void>> changes;
|
||||
std::vector<Future<Void>> failed;
|
||||
std::vector<Future<Void>> backupFailed(1, Never());
|
||||
std::vector<Future<Void>> changes;
|
||||
|
||||
for(auto& it : self->tLogs) {
|
||||
for(auto &t : it->logServers) {
|
||||
|
@ -399,6 +428,15 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
|
|||
changes.push_back(t->onChange());
|
||||
}
|
||||
}
|
||||
for (const auto& worker : it->backupWorkers) {
|
||||
if (worker->get().present()) {
|
||||
backupFailed.push_back(waitFailureClient(
|
||||
worker->get().interf().waitFailure, SERVER_KNOBS->BACKUP_TIMEOUT,
|
||||
-SERVER_KNOBS->BACKUP_TIMEOUT / SERVER_KNOBS->SECONDS_BEFORE_NO_FAILURE_DELAY));
|
||||
} else {
|
||||
changes.push_back(worker->onChange());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if(!self->recoveryCompleteWrittenToCoreState.get()) {
|
||||
|
@ -412,6 +450,16 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
|
|||
}
|
||||
}
|
||||
}
|
||||
// Monitor changes of backup workers for old epochs.
|
||||
for (const auto& worker : old.tLogs[0]->backupWorkers) {
|
||||
if (worker->get().present()) {
|
||||
backupFailed.push_back(waitFailureClient(
|
||||
worker->get().interf().waitFailure, SERVER_KNOBS->BACKUP_TIMEOUT,
|
||||
-SERVER_KNOBS->BACKUP_TIMEOUT / SERVER_KNOBS->SECONDS_BEFORE_NO_FAILURE_DELAY));
|
||||
} else {
|
||||
changes.push_back(worker->onChange());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -420,13 +468,16 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
|
|||
}
|
||||
|
||||
changes.push_back(self->recoveryCompleteWrittenToCoreState.onChange());
|
||||
changes.push_back(self->backupWorkerChanged.onTrigger());
|
||||
|
||||
ASSERT( failed.size() >= 1 );
|
||||
wait( quorum(changes, 1) || tagError<Void>( quorum( failed, 1 ), master_tlog_failed() ) );
|
||||
wait(quorum(changes, 1) || tagError<Void>(quorum(failed, 1), master_tlog_failed()) ||
|
||||
tagError<Void>(quorum(backupFailed, 1), master_backup_worker_failed()));
|
||||
}
|
||||
}
|
||||
|
||||
virtual Future<Version> push( Version prevVersion, Version version, Version knownCommittedVersion, Version minKnownCommittedVersion, LogPushData& data, Optional<UID> debugID ) {
|
||||
Future<Version> push(Version prevVersion, Version version, Version knownCommittedVersion,
|
||||
Version minKnownCommittedVersion, LogPushData& data, Optional<UID> debugID) override {
|
||||
// FIXME: Randomize request order as in LegacyLogSystem?
|
||||
vector<Future<Void>> quorumResults;
|
||||
vector<Future<TLogCommitReply>> allReplies;
|
||||
|
@ -483,8 +534,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
|
|||
TraceEvent("TLogPeekAllAddingCurrent", dbgid).detail("Tag", tag.toString()).detail("Begin", begin).detail("End", end).detail("BestLogs", localSets[bestSet]->logServerString());
|
||||
cursors.emplace_back(new ILogSystem::SetPeekCursor( localSets, bestSet, localSets[bestSet]->bestLocationFor( tag ), tag, lastBegin, end, parallelGetMore));
|
||||
}
|
||||
int i = 0;
|
||||
while(begin < lastBegin) {
|
||||
for (int i = 0; begin < lastBegin; i++) {
|
||||
if(i == oldLogData.size()) {
|
||||
if(tag == txsTag || tag.locality == tagLocalityTxs || tag == cacheTag) {
|
||||
break;
|
||||
|
@ -514,7 +564,6 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
|
|||
if(!localOldSets.size()) {
|
||||
TraceEvent("TLogPeekAllNoLocalSets", dbgid).detail("Tag", tag.toString()).detail("Begin", begin).detail("End", end).detail("LastBegin", lastBegin);
|
||||
if(!cursors.size() && !foundSpecial) {
|
||||
i++;
|
||||
continue;
|
||||
}
|
||||
return Reference<ILogSystem::ServerPeekCursor>( new ILogSystem::ServerPeekCursor( Reference<AsyncVar<OptionalInterface<TLogInterface>>>(), tag, begin, getPeekEnd(), false, false ) );
|
||||
|
@ -531,7 +580,6 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
|
|||
}
|
||||
lastBegin = thisBegin;
|
||||
}
|
||||
i++;
|
||||
}
|
||||
|
||||
return Reference<ILogSystem::MultiCursor>( new ILogSystem::MultiCursor(cursors, epochEnds) );
|
||||
|
@ -601,7 +649,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
|
|||
}
|
||||
}
|
||||
|
||||
virtual Reference<IPeekCursor> peek( UID dbgid, Version begin, Optional<Version> end, Tag tag, bool parallelGetMore ) {
|
||||
Reference<IPeekCursor> peek( UID dbgid, Version begin, Optional<Version> end, Tag tag, bool parallelGetMore ) override {
|
||||
if(!tLogs.size()) {
|
||||
TraceEvent("TLogPeekNoLogSets", dbgid).detail("Tag", tag.toString()).detail("Begin", begin);
|
||||
return Reference<ILogSystem::ServerPeekCursor>( new ILogSystem::ServerPeekCursor( Reference<AsyncVar<OptionalInterface<TLogInterface>>>(), tag, begin, getPeekEnd(), false, false ) );
|
||||
|
@ -614,7 +662,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
|
|||
}
|
||||
}
|
||||
|
||||
virtual Reference<IPeekCursor> peek( UID dbgid, Version begin, Optional<Version> end, std::vector<Tag> tags, bool parallelGetMore ) {
|
||||
Reference<IPeekCursor> peek(UID dbgid, Version begin, Optional<Version> end, std::vector<Tag> tags, bool parallelGetMore) override {
|
||||
if(tags.empty()) {
|
||||
TraceEvent("TLogPeekNoTags", dbgid).detail("Begin", begin);
|
||||
return Reference<ILogSystem::ServerPeekCursor>( new ILogSystem::ServerPeekCursor( Reference<AsyncVar<OptionalInterface<TLogInterface>>>(), invalidTag, begin, getPeekEnd(), false, false ) );
|
||||
|
@ -683,8 +731,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
|
|||
}
|
||||
}
|
||||
Version lastBegin = tLogs[bestSet]->startVersion;
|
||||
int i = 0;
|
||||
while(begin < lastBegin) {
|
||||
for (int i = 0; begin < lastBegin; i++) {
|
||||
if(i == oldLogData.size()) {
|
||||
if((tag == txsTag || tag.locality == tagLocalityTxs) && cursors.size()) {
|
||||
break;
|
||||
|
@ -718,7 +765,6 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
|
|||
if(oldLogData[i].logRouterTags == 0 || logCount > 1 || foundSpecial) {
|
||||
throw worker_removed();
|
||||
}
|
||||
i++;
|
||||
continue;
|
||||
}
|
||||
|
||||
|
@ -735,14 +781,13 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
|
|||
}
|
||||
lastBegin = thisBegin;
|
||||
}
|
||||
i++;
|
||||
}
|
||||
|
||||
return Reference<ILogSystem::MultiCursor>( new ILogSystem::MultiCursor(cursors, epochEnds) );
|
||||
}
|
||||
}
|
||||
|
||||
virtual Reference<IPeekCursor> peekTxs( UID dbgid, Version begin, int8_t peekLocality, Version localEnd, bool canDiscardPopped ) {
|
||||
Reference<IPeekCursor> peekTxs(UID dbgid, Version begin, int8_t peekLocality, Version localEnd, bool canDiscardPopped) override {
|
||||
Version end = getEnd();
|
||||
if(!tLogs.size()) {
|
||||
TraceEvent("TLogPeekTxsNoLogs", dbgid);
|
||||
|
@ -824,7 +869,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
|
|||
}
|
||||
}
|
||||
|
||||
virtual Reference<IPeekCursor> peekSingle( UID dbgid, Version begin, Tag tag, std::vector<std::pair<Version,Tag>> history ) {
|
||||
Reference<IPeekCursor> peekSingle(UID dbgid, Version begin, Tag tag, std::vector<std::pair<Version,Tag>> history) override {
|
||||
while(history.size() && begin >= history.back().first) {
|
||||
history.pop_back();
|
||||
}
|
||||
|
@ -849,15 +894,15 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
|
|||
}
|
||||
}
|
||||
|
||||
virtual Reference<IPeekCursor> peekLogRouter( UID dbgid, Version begin, Tag tag ) {
|
||||
Reference<IPeekCursor> peekLogRouter(UID dbgid, Version begin, Tag tag) override {
|
||||
bool found = false;
|
||||
for( auto& log : tLogs ) {
|
||||
found = log->hasLogRouter(dbgid);
|
||||
if(found) {
|
||||
for (const auto& log : tLogs) {
|
||||
found = log->hasLogRouter(dbgid) || log->hasBackupWorker(dbgid);
|
||||
if (found) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
if( found ) {
|
||||
if (found) {
|
||||
if(stopped) {
|
||||
std::vector<Reference<LogSet>> localSets;
|
||||
int bestPrimarySet = 0;
|
||||
|
@ -909,11 +954,11 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
|
|||
}
|
||||
}
|
||||
bool firstOld = true;
|
||||
for(auto& old : oldLogData) {
|
||||
for (const auto& old : oldLogData) {
|
||||
found = false;
|
||||
for( auto& log : old.tLogs ) {
|
||||
found = log->hasLogRouter(dbgid);
|
||||
if(found) {
|
||||
for (const auto& log : old.tLogs) {
|
||||
found = log->hasLogRouter(dbgid) || log->hasBackupWorker(dbgid);
|
||||
if (found) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
@ -948,7 +993,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
|
|||
return Reference<ILogSystem::ServerPeekCursor>( new ILogSystem::ServerPeekCursor( Reference<AsyncVar<OptionalInterface<TLogInterface>>>(), tag, begin, getPeekEnd(), false, false ) );
|
||||
}
|
||||
|
||||
virtual Version getKnownCommittedVersion() {
|
||||
Version getKnownCommittedVersion() override {
|
||||
Version result = invalidVersion;
|
||||
for(auto& it : lockResults) {
|
||||
auto versions = TagPartitionedLogSystem::getDurableVersion(dbgid, it);
|
||||
|
@ -959,7 +1004,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
|
|||
return result;
|
||||
}
|
||||
|
||||
virtual Future<Void> onKnownCommittedVersionChange() {
|
||||
Future<Void> onKnownCommittedVersionChange() override {
|
||||
std::vector<Future<Void>> result;
|
||||
for(auto& it : lockResults) {
|
||||
result.push_back(TagPartitionedLogSystem::getDurableVersionChanged(it));
|
||||
|
@ -1000,7 +1045,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
|
|||
}
|
||||
}
|
||||
|
||||
virtual void popTxs( Version upTo, int8_t popLocality ) {
|
||||
void popTxs(Version upTo, int8_t popLocality) override {
|
||||
if( getTLogVersion() < TLogVersion::V4 ) {
|
||||
pop(upTo, txsTag, 0, popLocality);
|
||||
} else {
|
||||
|
@ -1010,7 +1055,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
|
|||
}
|
||||
}
|
||||
|
||||
virtual void pop( Version upTo, Tag tag, Version durableKnownCommittedVersion, int8_t popLocality ) {
|
||||
void pop(Version upTo, Tag tag, Version durableKnownCommittedVersion, int8_t popLocality) override {
|
||||
if (upTo <= 0) return;
|
||||
if( tag.locality == tagLocalityRemoteLog) {
|
||||
popLogRouter(upTo, tag, durableKnownCommittedVersion, popLocality);
|
||||
|
@ -1110,7 +1155,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
|
|||
return maxPopped;
|
||||
}
|
||||
|
||||
virtual Future<Version> getTxsPoppedVersion() {
|
||||
Future<Version> getTxsPoppedVersion() override {
|
||||
return getPoppedTxs(this);
|
||||
}
|
||||
|
||||
|
@ -1162,7 +1207,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
|
|||
}
|
||||
|
||||
// Returns success after confirming that pushes in the current epoch are still possible
|
||||
virtual Future<Void> confirmEpochLive(Optional<UID> debugID) {
|
||||
Future<Void> confirmEpochLive(Optional<UID> debugID) override {
|
||||
vector<Future<Void>> quorumResults;
|
||||
for(auto& it : tLogs) {
|
||||
if(it->isLocal && it->logServers.size()) {
|
||||
|
@ -1173,7 +1218,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
|
|||
return waitForAll(quorumResults);
|
||||
}
|
||||
|
||||
virtual Future<Void> endEpoch() {
|
||||
Future<Void> endEpoch() override {
|
||||
std::vector<Future<Void>> lockResults;
|
||||
for( auto& logSet : tLogs ) {
|
||||
for( auto& log : logSet->logServers ) {
|
||||
|
@ -1183,14 +1228,19 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
|
|||
return waitForAll(lockResults);
|
||||
}
|
||||
|
||||
virtual Future<Reference<ILogSystem>> newEpoch( RecruitFromConfigurationReply const& recr, Future<RecruitRemoteFromConfigurationReply> const& fRemoteWorkers, DatabaseConfiguration const& config, LogEpoch recoveryCount, int8_t primaryLocality, int8_t remoteLocality, std::vector<Tag> const& allTags, Reference<AsyncVar<bool>> const& recruitmentStalled ) {
|
||||
// Call only after end_epoch() has successfully completed. Returns a new epoch immediately following this one. The new epoch
|
||||
// is only provisional until the caller updates the coordinated DBCoreState
|
||||
// Call only after end_epoch() has successfully completed. Returns a new epoch immediately following this one.
|
||||
// The new epoch is only provisional until the caller updates the coordinated DBCoreState.
|
||||
Future<Reference<ILogSystem>> newEpoch(RecruitFromConfigurationReply const& recr,
|
||||
Future<RecruitRemoteFromConfigurationReply> const& fRemoteWorkers,
|
||||
DatabaseConfiguration const& config, LogEpoch recoveryCount,
|
||||
int8_t primaryLocality, int8_t remoteLocality,
|
||||
std::vector<Tag> const& allTags,
|
||||
Reference<AsyncVar<bool>> const& recruitmentStalled) override {
|
||||
return newEpoch( Reference<TagPartitionedLogSystem>::addRef(this), recr, fRemoteWorkers, config, recoveryCount, primaryLocality, remoteLocality, allTags, recruitmentStalled );
|
||||
}
|
||||
|
||||
virtual LogSystemConfig getLogSystemConfig() {
|
||||
LogSystemConfig logSystemConfig;
|
||||
LogSystemConfig getLogSystemConfig() override {
|
||||
LogSystemConfig logSystemConfig(epoch);
|
||||
logSystemConfig.logSystemType = logSystemType;
|
||||
logSystemConfig.expectedLogSets = expectedLogSets;
|
||||
logSystemConfig.logRouterTags = logRouterTags;
|
||||
|
@ -1198,6 +1248,8 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
|
|||
logSystemConfig.recruitmentID = recruitmentID;
|
||||
logSystemConfig.stopped = stopped;
|
||||
logSystemConfig.recoveredAt = recoveredAt;
|
||||
logSystemConfig.pseudoLocalities = pseudoLocalities;
|
||||
logSystemConfig.oldestBackupEpoch = oldestBackupEpoch;
|
||||
for (const Reference<LogSet>& logSet : tLogs) {
|
||||
if (logSet->isLocal || remoteLogsWrittenToCoreState) {
|
||||
logSystemConfig.tLogs.emplace_back(*logSet);
|
||||
|
@ -1212,7 +1264,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
|
|||
return logSystemConfig;
|
||||
}
|
||||
|
||||
virtual Standalone<StringRef> getLogsValue() {
|
||||
Standalone<StringRef> getLogsValue() override {
|
||||
vector<std::pair<UID, NetworkAddress>> logs;
|
||||
vector<std::pair<UID, NetworkAddress>> oldLogs;
|
||||
for(auto& t : tLogs) {
|
||||
|
@ -1234,7 +1286,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
|
|||
return logsValue( logs, oldLogs );
|
||||
}
|
||||
|
||||
virtual Future<Void> onLogSystemConfigChange() {
|
||||
Future<Void> onLogSystemConfigChange() override {
|
||||
std::vector<Future<Void>> changes;
|
||||
changes.push_back(logSystemConfigChanged.onTrigger());
|
||||
for(auto& t : tLogs) {
|
||||
|
@ -1257,7 +1309,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
|
|||
return waitForAny(changes);
|
||||
}
|
||||
|
||||
virtual Version getEnd() {
|
||||
Version getEnd() override {
|
||||
ASSERT( recoverAt.present() );
|
||||
return recoverAt.get() + 1;
|
||||
}
|
||||
|
@ -1269,7 +1321,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
|
|||
return std::numeric_limits<Version>::max();
|
||||
}
|
||||
|
||||
virtual void getPushLocations(VectorRef<Tag> tags, std::vector<int>& locations, bool allLocations) {
|
||||
void getPushLocations(VectorRef<Tag> tags, std::vector<int>& locations, bool allLocations) override {
|
||||
int locationOffset = 0;
|
||||
for(auto& log : tLogs) {
|
||||
if(log->isLocal && log->logServers.size()) {
|
||||
|
@ -1279,22 +1331,103 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
|
|||
}
|
||||
}
|
||||
|
||||
virtual bool hasRemoteLogs() const {
|
||||
bool hasRemoteLogs() const override {
|
||||
return logRouterTags > 0 || pseudoLocalities.size() > 0;
|
||||
}
|
||||
|
||||
virtual Tag getRandomRouterTag() const {
|
||||
Tag getRandomRouterTag() const override {
|
||||
return Tag(tagLocalityLogRouter, deterministicRandom()->randomInt(0, logRouterTags));
|
||||
}
|
||||
|
||||
virtual Tag getRandomTxsTag() const {
|
||||
Tag getRandomTxsTag() const override {
|
||||
return Tag(tagLocalityTxs, deterministicRandom()->randomInt(0, txsTags));
|
||||
}
|
||||
|
||||
virtual TLogVersion getTLogVersion() const {
|
||||
TLogVersion getTLogVersion() const override {
|
||||
return tLogs[0]->tLogVersion;
|
||||
}
|
||||
|
||||
int getLogRouterTags() const override { return logRouterTags; }
|
||||
|
||||
Version getStartVersion() const override {
|
||||
ASSERT(tLogs.size() > 0);
|
||||
return tLogs[0]->startVersion;
|
||||
}
|
||||
|
||||
std::map<LogEpoch, ILogSystem::EpochTagsVersionsInfo> getOldEpochTagsVersionsInfo() const override {
|
||||
std::map<LogEpoch, EpochTagsVersionsInfo> epochInfos;
|
||||
for (const auto& old : oldLogData) {
|
||||
epochInfos.insert(
|
||||
{ old.epoch, ILogSystem::EpochTagsVersionsInfo(old.logRouterTags, old.epochBegin, old.epochEnd) });
|
||||
TraceEvent("OldEpochTagsVersions", dbgid)
|
||||
.detail("Epoch", old.epoch)
|
||||
.detail("Tags", old.logRouterTags)
|
||||
.detail("BeginVersion", old.epochBegin)
|
||||
.detail("EndVersion", old.epochEnd);
|
||||
}
|
||||
return epochInfos;
|
||||
}
|
||||
|
||||
inline Reference<LogSet> getEpochLogSet(LogEpoch epoch) const {
|
||||
for (const auto& old : oldLogData) {
|
||||
if (epoch == old.epoch) return old.tLogs[0];
|
||||
}
|
||||
return Reference<LogSet>(nullptr);
|
||||
}
|
||||
|
||||
void setBackupWorkers(const std::vector<InitializeBackupReply>& replies) override {
|
||||
ASSERT(tLogs.size() > 0);
|
||||
|
||||
Reference<LogSet> logset = tLogs[0]; // Master recruits this epoch's worker first.
|
||||
LogEpoch logsetEpoch = this->epoch;
|
||||
oldestBackupEpoch = this->epoch;
|
||||
for (const auto& reply : replies) {
|
||||
Reference<AsyncVar<OptionalInterface<BackupInterface>>> worker(new AsyncVar<OptionalInterface<BackupInterface>>(OptionalInterface<BackupInterface>(reply.interf)));
|
||||
if (reply.backupEpoch != logsetEpoch) {
|
||||
// find the logset from oldLogData
|
||||
logsetEpoch = reply.backupEpoch;
|
||||
oldestBackupEpoch = std::min(oldestBackupEpoch, logsetEpoch);
|
||||
logset = getEpochLogSet(logsetEpoch);
|
||||
ASSERT(logset.isValid());
|
||||
}
|
||||
logset->backupWorkers.push_back(worker);
|
||||
}
|
||||
backupWorkerChanged.trigger();
|
||||
}
|
||||
|
||||
bool removeBackupWorker(const BackupWorkerDoneRequest& req) override {
|
||||
bool removed = false;
|
||||
Reference<LogSet> logset = getEpochLogSet(req.backupEpoch);
|
||||
if (logset.isValid()) {
|
||||
for (auto it = logset->backupWorkers.begin(); it != logset->backupWorkers.end(); it++) {
|
||||
if (it->getPtr()->get().interf().id() == req.workerUID) {
|
||||
logset->backupWorkers.erase(it);
|
||||
removed = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (removed) {
|
||||
oldestBackupEpoch = epoch;
|
||||
for (const auto& old : oldLogData) {
|
||||
if (old.epoch < oldestBackupEpoch && old.tLogs[0]->backupWorkers.size() > 0) {
|
||||
oldestBackupEpoch = old.epoch;
|
||||
}
|
||||
}
|
||||
backupWorkerChanged.trigger();
|
||||
}
|
||||
|
||||
TraceEvent("RemoveBackupWorker", dbgid)
|
||||
.detail("Removed", removed)
|
||||
.detail("BackupEpoch", req.backupEpoch)
|
||||
.detail("WorkerID", req.workerUID)
|
||||
.detail("OldestBackupEpoch", oldestBackupEpoch);
|
||||
return removed;
|
||||
}
|
||||
|
||||
LogEpoch getOldestBackupEpoch() const override { return oldestBackupEpoch; }
|
||||
|
||||
ACTOR static Future<Void> monitorLog(Reference<AsyncVar<OptionalInterface<TLogInterface>>> logServer, Reference<AsyncVar<bool>> failed) {
|
||||
state Future<Void> waitFailure;
|
||||
loop {
|
||||
|
@ -1403,7 +1536,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
|
|||
|
||||
if (!prevState.tLogs.size()) {
|
||||
// This is a brand new database
|
||||
Reference<TagPartitionedLogSystem> logSystem( new TagPartitionedLogSystem(dbgid, locality) );
|
||||
Reference<TagPartitionedLogSystem> logSystem( new TagPartitionedLogSystem(dbgid, locality, 0) );
|
||||
logSystem->logSystemType = prevState.logSystemType;
|
||||
logSystem->recoverAt = 0;
|
||||
logSystem->knownCommittedVersion = 0;
|
||||
|
@ -1473,7 +1606,11 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
|
|||
modifiedState.oldTLogData[i].tLogs.push_back(coreSet);
|
||||
modifiedState.oldTLogData[i].tLogs[0].isLocal = true;
|
||||
modifiedState.oldTLogData[i].logRouterTags = 0;
|
||||
modifiedState.oldTLogData[i].epochEnd = ( i == 0 ? modifiedState.tLogs[0].startVersion : modifiedState.oldTLogData[i-1].tLogs[0].startVersion );
|
||||
modifiedState.oldTLogData[i].epochBegin =
|
||||
modifiedState.oldTLogData[i].tLogs[0].startVersion;
|
||||
modifiedState.oldTLogData[i].epochEnd =
|
||||
(i == 0 ? modifiedState.tLogs[0].startVersion
|
||||
: modifiedState.oldTLogData[i - 1].tLogs[0].startVersion);
|
||||
modifiedLogSets++;
|
||||
}
|
||||
break;
|
||||
|
@ -1642,7 +1779,8 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
|
|||
if(maxEnd > 0 && (!lastEnd.present() || maxEnd < lastEnd.get())) {
|
||||
TEST( lastEnd.present() ); // Restarting recovery at an earlier point
|
||||
|
||||
Reference<TagPartitionedLogSystem> logSystem( new TagPartitionedLogSystem(dbgid, locality) );
|
||||
Reference<TagPartitionedLogSystem> logSystem(
|
||||
new TagPartitionedLogSystem(dbgid, locality, prevState.recoveryCount));
|
||||
|
||||
lastEnd = minEnd;
|
||||
logSystem->tLogs = logServers;
|
||||
|
@ -1819,10 +1957,10 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
|
|||
return Void();
|
||||
}
|
||||
|
||||
static Version getMaxLocalStartVersion(std::vector<Reference<LogSet>>& tLogs) {
|
||||
static Version getMaxLocalStartVersion(const std::vector<Reference<LogSet>>& tLogs) {
|
||||
Version maxStart = 0;
|
||||
for (const auto& logSet : tLogs) {
|
||||
if(logSet->isLocal) {
|
||||
if (logSet->isLocal) {
|
||||
maxStart = std::max(maxStart, logSet->startVersion);
|
||||
}
|
||||
}
|
||||
|
@ -1880,11 +2018,14 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
|
|||
}
|
||||
|
||||
state vector<Future<TLogInterface>> logRouterInitializationReplies;
|
||||
for( int i = 0; i < self->logRouterTags; i++) {
|
||||
const Version startVersion = oldLogSystem->logRouterTags == 0
|
||||
? oldLogSystem->recoverAt.get() + 1
|
||||
: std::max(self->tLogs[0]->startVersion, logSet->startVersion);
|
||||
for (int i = 0; i < self->logRouterTags; i++) {
|
||||
InitializeLogRouterRequest req;
|
||||
req.recoveryCount = recoveryCount;
|
||||
req.routerTag = Tag(tagLocalityLogRouter, i);
|
||||
req.startVersion = oldLogSystem->logRouterTags == 0 ? oldLogSystem->recoverAt.get() + 1 : std::max(self->tLogs[0]->startVersion, logSet->startVersion);
|
||||
req.startVersion = startVersion;
|
||||
req.tLogLocalities = localities;
|
||||
req.tLogPolicy = logSet->tLogPolicy;
|
||||
req.locality = remoteLocality;
|
||||
|
@ -1987,7 +2128,8 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
|
|||
ACTOR static Future<Reference<ILogSystem>> newEpoch( Reference<TagPartitionedLogSystem> oldLogSystem, RecruitFromConfigurationReply recr, Future<RecruitRemoteFromConfigurationReply> fRemoteWorkers, DatabaseConfiguration configuration, LogEpoch recoveryCount,
|
||||
int8_t primaryLocality, int8_t remoteLocality, std::vector<Tag> allTags, Reference<AsyncVar<bool>> recruitmentStalled ) {
|
||||
state double startTime = now();
|
||||
state Reference<TagPartitionedLogSystem> logSystem( new TagPartitionedLogSystem(oldLogSystem->getDebugID(), oldLogSystem->locality) );
|
||||
state Reference<TagPartitionedLogSystem> logSystem(
|
||||
new TagPartitionedLogSystem(oldLogSystem->getDebugID(), oldLogSystem->locality, recoveryCount));
|
||||
logSystem->logSystemType = LogSystemType::tagPartitioned;
|
||||
logSystem->expectedLogSets = 1;
|
||||
logSystem->recoveredAt = oldLogSystem->recoverAt;
|
||||
|
@ -1996,10 +2138,18 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
|
|||
logSystem->txsTags = configuration.tLogVersion >= TLogVersion::V4 ? recr.tLogs.size() : 0;
|
||||
oldLogSystem->recruitmentID = logSystem->recruitmentID;
|
||||
|
||||
logSystem->logRouterTags = recr.tLogs.size() * std::max<int>(1, configuration.desiredLogRouterCount / std::max<int>(1, recr.tLogs.size()));
|
||||
if(configuration.usableRegions > 1) {
|
||||
logSystem->logRouterTags = recr.tLogs.size() * std::max<int>(1, configuration.desiredLogRouterCount / std::max<int>(1,recr.tLogs.size()));
|
||||
logSystem->expectedLogSets++;
|
||||
logSystem->addPseudoLocality(tagLocalityLogRouterMapped);
|
||||
logSystem->addPseudoLocality(tagLocalityBackup);
|
||||
TraceEvent("AddPseudoLocality", logSystem->getDebugID())
|
||||
.detail("Locality1", "LogRouterMapped")
|
||||
.detail("Locality2", "Backup");
|
||||
} else {
|
||||
// Single region uses log router tag for backup workers.
|
||||
logSystem->addPseudoLocality(tagLocalityBackup);
|
||||
TraceEvent("AddPseudoLocality", logSystem->getDebugID()).detail("Locality", "Backup");
|
||||
}
|
||||
|
||||
logSystem->tLogs.emplace_back(new LogSet());
|
||||
|
@ -2050,10 +2200,12 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
|
|||
if(oldLogSystem->tLogs.size()) {
|
||||
logSystem->oldLogData.emplace_back();
|
||||
logSystem->oldLogData[0].tLogs = oldLogSystem->tLogs;
|
||||
logSystem->oldLogData[0].epochBegin = oldLogSystem->tLogs[0]->startVersion;
|
||||
logSystem->oldLogData[0].epochEnd = oldLogSystem->knownCommittedVersion + 1;
|
||||
logSystem->oldLogData[0].logRouterTags = oldLogSystem->logRouterTags;
|
||||
logSystem->oldLogData[0].txsTags = oldLogSystem->txsTags;
|
||||
logSystem->oldLogData[0].pseudoLocalities = oldLogSystem->pseudoLocalities;
|
||||
logSystem->oldLogData[0].epoch = oldLogSystem->epoch;
|
||||
}
|
||||
logSystem->oldLogData.insert(logSystem->oldLogData.end(), oldLogSystem->oldLogData.begin(), oldLogSystem->oldLogData.end());
|
||||
|
||||
|
@ -2255,7 +2407,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
|
|||
if(configuration.usableRegions > 1) {
|
||||
logSystem->hasRemoteServers = true;
|
||||
logSystem->remoteRecovery = TagPartitionedLogSystem::newRemoteEpoch(logSystem.getPtr(), oldLogSystem, fRemoteWorkers, configuration, recoveryCount, remoteLocality, allTags);
|
||||
if(oldLogSystem->logRouterTags == 0) {
|
||||
if (oldLogSystem->tLogs.size() > 0 && oldLogSystem->tLogs[0]->locality == tagLocalitySpecial) {
|
||||
//The wait is required so that we know both primary logs and remote logs have copied the data between the known committed version and the recovery version.
|
||||
//FIXME: we can remove this wait once we are able to have log routers which can ship data to the remote logs without using log router tags.
|
||||
wait(logSystem->remoteRecovery);
|
||||
|
|
|
@ -25,6 +25,7 @@
|
|||
#elif !defined(FDBSERVER_WORKERINTERFACE_ACTOR_H)
|
||||
#define FDBSERVER_WORKERINTERFACE_ACTOR_H
|
||||
|
||||
#include "fdbserver/BackupInterface.h"
|
||||
#include "fdbserver/DataDistributorInterface.h"
|
||||
#include "fdbserver/MasterInterface.h"
|
||||
#include "fdbserver/TLogInterface.h"
|
||||
|
@ -50,6 +51,7 @@ struct WorkerInterface {
|
|||
RequestStream< struct InitializeResolverRequest > resolver;
|
||||
RequestStream< struct InitializeStorageRequest > storage;
|
||||
RequestStream< struct InitializeLogRouterRequest > logRouter;
|
||||
RequestStream< struct InitializeBackupRequest > backup;
|
||||
|
||||
RequestStream< struct LoadedPingRequest > debugPing;
|
||||
RequestStream< struct CoordinationPingMessage > coordinationPing;
|
||||
|
@ -82,7 +84,7 @@ struct WorkerInterface {
|
|||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, clientInterface, locality, tLog, master, masterProxy, dataDistributor, ratekeeper, resolver, storage, logRouter, debugPing, coordinationPing, waitFailure, setMetricsRate, eventLogRequest, traceBatchDumpRequest, testerInterface, diskStoreRequest, execReq, workerSnapReq);
|
||||
serializer(ar, clientInterface, locality, tLog, master, masterProxy, dataDistributor, ratekeeper, resolver, storage, logRouter, debugPing, coordinationPing, waitFailure, setMetricsRate, eventLogRequest, traceBatchDumpRequest, testerInterface, diskStoreRequest, execReq, workerSnapReq, backup);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -122,7 +124,7 @@ struct InitializeTLogRequest {
|
|||
|
||||
ReplyPromise< struct TLogInterface > reply;
|
||||
|
||||
InitializeTLogRequest() {}
|
||||
InitializeTLogRequest() : recoverFrom(0) {}
|
||||
|
||||
template <class Ar>
|
||||
void serialize( Ar& ar ) {
|
||||
|
@ -146,6 +148,40 @@ struct InitializeLogRouterRequest {
|
|||
}
|
||||
};
|
||||
|
||||
struct InitializeBackupReply {
|
||||
constexpr static FileIdentifier file_identifier = 63843557;
|
||||
struct BackupInterface interf;
|
||||
LogEpoch backupEpoch;
|
||||
|
||||
InitializeBackupReply() = default;
|
||||
InitializeBackupReply(BackupInterface interface, LogEpoch e) : interf(interface), backupEpoch(e) {}
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, interf, backupEpoch);
|
||||
}
|
||||
};
|
||||
|
||||
struct InitializeBackupRequest {
|
||||
constexpr static FileIdentifier file_identifier = 68354279;
|
||||
UID reqId;
|
||||
LogEpoch recruitedEpoch; // The epoch the worker is recruited.
|
||||
LogEpoch backupEpoch; // The epoch the worker should work on. If different from the recruitedEpoch, then it refers
|
||||
// to some previous epoch with unfinished work.
|
||||
Tag routerTag;
|
||||
Version startVersion;
|
||||
Optional<Version> endVersion;
|
||||
ReplyPromise<struct InitializeBackupReply> reply;
|
||||
|
||||
InitializeBackupRequest() = default;
|
||||
explicit InitializeBackupRequest(UID id) : reqId(id) {}
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, reqId, recruitedEpoch, backupEpoch, routerTag, startVersion, endVersion, reply);
|
||||
}
|
||||
};
|
||||
|
||||
// FIXME: Rename to InitializeMasterRequest, etc
|
||||
struct RecruitMasterRequest {
|
||||
constexpr static FileIdentifier file_identifier = 12684574;
|
||||
|
@ -399,6 +435,7 @@ struct Role {
|
|||
static const Role RATEKEEPER;
|
||||
static const Role STORAGE_CACHE;
|
||||
static const Role COORDINATOR;
|
||||
static const Role BACKUP;
|
||||
|
||||
std::string roleName;
|
||||
std::string abbreviation;
|
||||
|
@ -468,6 +505,7 @@ ACTOR Future<Void> logRouter(TLogInterface interf, InitializeLogRouterRequest re
|
|||
ACTOR Future<Void> dataDistributor(DataDistributorInterface ddi, Reference<AsyncVar<ServerDBInfo>> db);
|
||||
ACTOR Future<Void> ratekeeper(RatekeeperInterface rki, Reference<AsyncVar<ServerDBInfo>> db);
|
||||
ACTOR Future<Void> storageCache(StorageServerInterface interf, uint16_t id, Reference<AsyncVar<ServerDBInfo>> db);
|
||||
ACTOR Future<Void> backupWorker(BackupInterface bi, InitializeBackupRequest req, Reference<AsyncVar<ServerDBInfo>> db);
|
||||
|
||||
void registerThreadForProfiling();
|
||||
void updateCpuProfiler(ProfilerRequest req);
|
||||
|
|
|
@ -1719,8 +1719,11 @@ int main(int argc, char* argv[]) {
|
|||
std::vector<std::string> directories = platform::listDirectories( dataFolder );
|
||||
for(int i = 0; i < directories.size(); i++)
|
||||
if (directories[i].size() != 32 && directories[i] != "." && directories[i] != ".." &&
|
||||
directories[i] != "backups" && directories[i].find("snap") == std::string::npos) {
|
||||
TraceEvent(SevError, "IncompatibleDirectoryFound").detail("DataFolder", dataFolder).detail("SuspiciousFile", directories[i]);
|
||||
directories[i] != "backups" && directories[i].find("snap") == std::string::npos &&
|
||||
directories[i] != "mutation_backups") {
|
||||
TraceEvent(SevError, "IncompatibleDirectoryFound")
|
||||
.detail("DataFolder", dataFolder)
|
||||
.detail("SuspiciousFile", directories[i]);
|
||||
fprintf(stderr, "ERROR: Data folder `%s' had non fdb file `%s'; please use clean, fdb-only folder\n", dataFolder.c_str(), directories[i].c_str());
|
||||
flushAndExit(FDB_EXIT_ERROR);
|
||||
}
|
||||
|
|
|
@ -34,6 +34,8 @@
|
|||
<ActorCompiler Include="worker.actor.cpp" />
|
||||
</ItemGroup>
|
||||
<ItemGroup>
|
||||
<ActorCompiler Include="BackupProgress.actor.cpp" />
|
||||
<ActorCompiler Include="BackupWorker.actor.cpp" />
|
||||
<ActorCompiler Include="VersionedBTree.actor.cpp" />
|
||||
<ActorCompiler Include="Coordination.actor.cpp" />
|
||||
<ActorCompiler Include="CoordinatedState.actor.cpp" />
|
||||
|
@ -172,6 +174,10 @@
|
|||
</ItemGroup>
|
||||
<ItemGroup>
|
||||
<ClInclude Include="ApplyMetadataMutation.h" />
|
||||
<ClInclude Include="BackupInterface.h" />
|
||||
<ActorCompiler Include="BackupProgress.actor.h">
|
||||
<EnableCompile>false</EnableCompile>
|
||||
</ActorCompiler>
|
||||
<ClInclude Include="ClusterRecruitmentInterface.h" />
|
||||
<ClInclude Include="ConflictSet.h" />
|
||||
<ClInclude Include="CoordinatedState.h" />
|
||||
|
|
|
@ -1,6 +1,8 @@
|
|||
<?xml version="1.0" encoding="utf-8"?>
|
||||
<Project ToolsVersion="4.0" xmlns="http://schemas.microsoft.com/developer/msbuild/2003">
|
||||
<ItemGroup>
|
||||
<ActorCompiler Include="BackupProgress.actor.cpp" />
|
||||
<ActorCompiler Include="BackupWorker.actor.cpp" />
|
||||
<ActorCompiler Include="ClusterController.actor.cpp" />
|
||||
<ActorCompiler Include="DataDistribution.actor.cpp" />
|
||||
<ActorCompiler Include="DataDistributionQueue.actor.cpp" />
|
||||
|
@ -322,6 +324,8 @@
|
|||
<ClCompile Include="LatencyBandConfig.cpp" />
|
||||
</ItemGroup>
|
||||
<ItemGroup>
|
||||
<ClInclude Include="BackupInterface.h" />
|
||||
<ClInclude Include="BackupProgress.h" />
|
||||
<ClInclude Include="ConflictSet.h" />
|
||||
<ClInclude Include="DataDistribution.actor.h" />
|
||||
<ClInclude Include="DataDistributorInterface.h" />
|
||||
|
|
|
@ -29,6 +29,8 @@
|
|||
#include "fdbserver/DataDistribution.actor.h"
|
||||
#include "fdbserver/Knobs.h"
|
||||
#include <iterator>
|
||||
#include "fdbserver/BackupProgress.actor.h"
|
||||
#include "fdbserver/MasterInterface.h"
|
||||
#include "fdbserver/WaitFailure.h"
|
||||
#include "fdbserver/WorkerInterface.actor.h"
|
||||
#include "fdbserver/ClusterRecruitmentInterface.h"
|
||||
|
@ -128,7 +130,7 @@ private:
|
|||
if(finalWrite) {
|
||||
self->finalWriteStarted = true;
|
||||
}
|
||||
|
||||
|
||||
try {
|
||||
wait( self->cstate.setExclusive( BinaryWriter::toValue(newState, IncludeVersion()) ) );
|
||||
} catch (Error& e) {
|
||||
|
@ -194,16 +196,16 @@ struct MasterData : NonCopyable, ReferenceCounted<MasterData> {
|
|||
return maxLocality + 1;
|
||||
}
|
||||
|
||||
vector< MasterProxyInterface > proxies;
|
||||
vector< MasterProxyInterface > provisionalProxies;
|
||||
vector< ResolverInterface > resolvers;
|
||||
std::vector<MasterProxyInterface> proxies;
|
||||
std::vector<MasterProxyInterface> provisionalProxies;
|
||||
std::vector<ResolverInterface> resolvers;
|
||||
|
||||
std::map<UID, ProxyVersionReplies> lastProxyVersionReplies;
|
||||
|
||||
Standalone<StringRef> dbId;
|
||||
|
||||
MasterInterface myInterface;
|
||||
ClusterControllerFullInterface clusterController; // If the cluster controller changes, this master will die, so this is immutable.
|
||||
const ClusterControllerFullInterface clusterController; // If the cluster controller changes, this master will die, so this is immutable.
|
||||
|
||||
ReusableCoordinatedState cstate;
|
||||
Promise<Void> cstateUpdated;
|
||||
|
@ -219,9 +221,11 @@ struct MasterData : NonCopyable, ReferenceCounted<MasterData> {
|
|||
PromiseStream<Future<Void>> addActor;
|
||||
Reference<AsyncVar<bool>> recruitmentStalled;
|
||||
bool forceRecovery;
|
||||
bool neverCreated;
|
||||
int8_t safeLocality;
|
||||
int8_t primaryLocality;
|
||||
bool neverCreated;
|
||||
|
||||
std::vector<WorkerInterface> backupWorkers; // Recruited backup workers from cluster controller.
|
||||
|
||||
MasterData(
|
||||
Reference<AsyncVar<ServerDBInfo>> const& dbInfo,
|
||||
|
@ -465,14 +469,14 @@ Future<Void> sendMasterRegistration( MasterData* self, LogSystemConfig const& lo
|
|||
|
||||
ACTOR Future<Void> updateRegistration( Reference<MasterData> self, Reference<ILogSystem> logSystem ) {
|
||||
state Database cx = openDBOnServer(self->dbInfo, TaskPriority::DefaultEndpoint, true, true);
|
||||
state Future<Void> trigger = self->registrationTrigger.onTrigger();
|
||||
state Future<Void> trigger = self->registrationTrigger.onTrigger();
|
||||
state Future<Void> updateLogsKey;
|
||||
|
||||
loop {
|
||||
wait( trigger );
|
||||
wait( delay( .001 ) ); // Coalesce multiple changes
|
||||
|
||||
trigger = self->registrationTrigger.onTrigger();
|
||||
trigger = self->registrationTrigger.onTrigger();
|
||||
|
||||
TraceEvent("MasterUpdateRegistration", self->dbgid).detail("RecoveryCount", self->cstate.myDBState.recoveryCount).detail("Logs", describe(logSystem->getLogSystemConfig().tLogs));
|
||||
|
||||
|
@ -592,20 +596,23 @@ ACTOR Future<vector<Standalone<CommitTransactionRef>>> recruitEverything( Refere
|
|||
self->remoteDcIds.push_back(recruits.dcId.get() == self->configuration.regions[0].dcId ? self->configuration.regions[1].dcId : self->configuration.regions[0].dcId);
|
||||
}
|
||||
}
|
||||
|
||||
self->backupWorkers.swap(recruits.backupWorkers);
|
||||
|
||||
TraceEvent("MasterRecoveryState", self->dbgid)
|
||||
.detail("StatusCode", RecoveryStatus::initializing_transaction_servers)
|
||||
.detail("Status", RecoveryStatus::names[RecoveryStatus::initializing_transaction_servers])
|
||||
.detail("Proxies", recruits.proxies.size())
|
||||
.detail("TLogs", recruits.tLogs.size())
|
||||
.detail("Resolvers", recruits.resolvers.size())
|
||||
.detail("BackupWorkers", self->backupWorkers.size())
|
||||
.trackLatest("MasterRecoveryState");
|
||||
|
||||
// Actually, newSeedServers does both the recruiting and initialization of the seed servers; so if this is a brand new database we are sort of lying that we are
|
||||
// past the recruitment phase. In a perfect world we would split that up so that the recruitment part happens above (in parallel with recruiting the transaction servers?).
|
||||
wait( newSeedServers( self, recruits, seedServers ) );
|
||||
state vector<Standalone<CommitTransactionRef>> confChanges;
|
||||
wait( newProxies( self, recruits ) && newResolvers( self, recruits ) && newTLogServers( self, recruits, oldLogSystem, &confChanges ) );
|
||||
wait(newProxies(self, recruits) && newResolvers(self, recruits) &&
|
||||
newTLogServers(self, recruits, oldLogSystem, &confChanges));
|
||||
return confChanges;
|
||||
}
|
||||
|
||||
|
@ -1092,6 +1099,7 @@ static std::set<int> const& normalMasterErrors() {
|
|||
s.insert( error_code_master_tlog_failed );
|
||||
s.insert( error_code_master_proxy_failed );
|
||||
s.insert( error_code_master_resolver_failed );
|
||||
s.insert( error_code_master_backup_worker_failed );
|
||||
s.insert( error_code_recruitment_failed );
|
||||
s.insert( error_code_no_more_servers );
|
||||
s.insert( error_code_master_recovery_failed );
|
||||
|
@ -1194,8 +1202,7 @@ ACTOR Future<Void> trackTlogRecovery( Reference<MasterData> self, Reference<Asyn
|
|||
}
|
||||
}
|
||||
|
||||
ACTOR Future<Void> configurationMonitor( Reference<MasterData> self ) {
|
||||
state Database cx = openDBOnServer(self->dbInfo, TaskPriority::DefaultEndpoint, true, true);
|
||||
ACTOR Future<Void> configurationMonitor(Reference<MasterData> self, Database cx) {
|
||||
loop {
|
||||
state ReadYourWritesTransaction tr(cx);
|
||||
|
||||
|
@ -1227,6 +1234,75 @@ ACTOR Future<Void> configurationMonitor( Reference<MasterData> self ) {
|
|||
}
|
||||
}
|
||||
|
||||
ACTOR static Future<Void> recruitBackupWorkers(Reference<MasterData> self, Database cx) {
|
||||
ASSERT(self->backupWorkers.size() > 0);
|
||||
|
||||
state LogEpoch epoch = self->cstate.myDBState.recoveryCount;
|
||||
state Reference<BackupProgress> backupProgress(
|
||||
new BackupProgress(self->dbgid, self->logSystem->getOldEpochTagsVersionsInfo()));
|
||||
state Future<Void> gotProgress = getBackupProgress(cx, self->dbgid, backupProgress);
|
||||
state std::vector<Future<InitializeBackupReply>> initializationReplies;
|
||||
|
||||
state std::vector<std::pair<UID, Tag>> idsTags; // worker IDs and tags for current epoch
|
||||
state int logRouterTags = self->logSystem->getLogRouterTags();
|
||||
for (int i = 0; i < logRouterTags; i++) {
|
||||
idsTags.emplace_back(deterministicRandom()->randomUniqueID(), Tag(tagLocalityLogRouter, i));
|
||||
}
|
||||
|
||||
const Version startVersion = self->logSystem->getStartVersion();
|
||||
state int i = 0;
|
||||
for (; i < logRouterTags; i++) {
|
||||
const auto& worker = self->backupWorkers[i % self->backupWorkers.size()];
|
||||
InitializeBackupRequest req(idsTags[i].first);
|
||||
req.recruitedEpoch = epoch;
|
||||
req.backupEpoch = epoch;
|
||||
req.routerTag = idsTags[i].second;
|
||||
req.startVersion = startVersion;
|
||||
TraceEvent("BackupRecruitment", self->dbgid)
|
||||
.detail("BKID", req.reqId)
|
||||
.detail("Tag", req.routerTag.toString())
|
||||
.detail("Epoch", epoch)
|
||||
.detail("BackupEpoch", epoch)
|
||||
.detail("StartVersion", req.startVersion);
|
||||
initializationReplies.push_back(
|
||||
transformErrors(throwErrorOr(worker.backup.getReplyUnlessFailedFor(
|
||||
req, SERVER_KNOBS->BACKUP_TIMEOUT, SERVER_KNOBS->MASTER_FAILURE_SLOPE_DURING_RECOVERY)),
|
||||
master_backup_worker_failed()));
|
||||
}
|
||||
|
||||
wait(gotProgress);
|
||||
std::map<std::pair<LogEpoch, Version>, std::map<Tag, Version>> toRecruit = backupProgress->getUnfinishedBackup();
|
||||
for (const auto& [epochVersion, tagVersions] : toRecruit) {
|
||||
for (const auto& [tag, version] : tagVersions) {
|
||||
const auto& worker = self->backupWorkers[i % self->backupWorkers.size()];
|
||||
i++;
|
||||
InitializeBackupRequest req(deterministicRandom()->randomUniqueID());
|
||||
req.recruitedEpoch = epoch;
|
||||
req.backupEpoch = epochVersion.first;
|
||||
req.routerTag = tag;
|
||||
req.startVersion = version; // savedVersion + 1
|
||||
req.endVersion = epochVersion.second - 1;
|
||||
TraceEvent("BackupRecruitment", self->dbgid)
|
||||
.detail("BKID", req.reqId)
|
||||
.detail("Tag", req.routerTag.toString())
|
||||
.detail("Epoch", epoch)
|
||||
.detail("BackupEpoch", req.backupEpoch)
|
||||
.detail("StartVersion", req.startVersion)
|
||||
.detail("EndVersion", req.endVersion.get());
|
||||
initializationReplies.push_back(transformErrors(
|
||||
throwErrorOr(worker.backup.getReplyUnlessFailedFor(req, SERVER_KNOBS->BACKUP_TIMEOUT,
|
||||
SERVER_KNOBS->MASTER_FAILURE_SLOPE_DURING_RECOVERY)),
|
||||
master_backup_worker_failed()));
|
||||
}
|
||||
}
|
||||
|
||||
std::vector<InitializeBackupReply> newRecruits = wait(getAll(initializationReplies));
|
||||
self->logSystem->setBackupWorkers(newRecruits);
|
||||
TraceEvent("BackupRecruitmentDone", self->dbgid);
|
||||
self->registrationTrigger.trigger();
|
||||
return Void();
|
||||
}
|
||||
|
||||
ACTOR Future<Void> masterCore( Reference<MasterData> self ) {
|
||||
state TraceInterval recoveryInterval("MasterRecovery");
|
||||
state double recoverStartTime = now();
|
||||
|
@ -1251,6 +1327,9 @@ ACTOR Future<Void> masterCore( Reference<MasterData> self ) {
|
|||
.detail("MyRecoveryCount", self->cstate.prevDBState.recoveryCount+2)
|
||||
.detail("ForceRecovery", self->forceRecovery)
|
||||
.trackLatest("MasterRecoveryState");
|
||||
//for (const auto& old : self->cstate.prevDBState.oldTLogData) {
|
||||
// TraceEvent("BWReadCoreState", self->dbgid).detail("Epoch", old.epoch).detail("Version", old.epochEnd);
|
||||
//}
|
||||
|
||||
state Reference<AsyncVar<Reference<ILogSystem>>> oldLogSystems( new AsyncVar<Reference<ILogSystem>> );
|
||||
state Future<Void> recoverAndEndEpoch = ILogSystem::recoverAndEndEpoch(oldLogSystems, self->dbgid, self->cstate.prevDBState, self->myInterface.tlogRejoin.getFuture(), self->myInterface.locality, &self->forceRecovery);
|
||||
|
@ -1444,7 +1523,9 @@ ACTOR Future<Void> masterCore( Reference<MasterData> self ) {
|
|||
self->addActor.send( resolutionBalancing(self) );
|
||||
|
||||
self->addActor.send( changeCoordinators(self) );
|
||||
self->addActor.send( configurationMonitor( self ) );
|
||||
Database cx = openDBOnServer(self->dbInfo, TaskPriority::DefaultEndpoint, true, true);
|
||||
self->addActor.send(configurationMonitor(self, cx));
|
||||
self->addActor.send(recruitBackupWorkers(self, cx));
|
||||
|
||||
wait( Future<Void>(Never()) );
|
||||
throw internal_error();
|
||||
|
@ -1473,6 +1554,12 @@ ACTOR Future<Void> masterServer( MasterInterface mi, Reference<AsyncVar<ServerDB
|
|||
throw worker_removed();
|
||||
}
|
||||
}
|
||||
when(BackupWorkerDoneRequest req = waitNext(mi.notifyBackupWorkerDone.getFuture())) {
|
||||
if (self->logSystem.isValid() && self->logSystem->removeBackupWorker(req)) {
|
||||
self->registrationTrigger.trigger();
|
||||
}
|
||||
req.reply.send(Void());
|
||||
}
|
||||
when (wait(collection) ) { ASSERT(false); throw internal_error(); }
|
||||
}
|
||||
} catch (Error& e) {
|
||||
|
@ -1488,9 +1575,9 @@ ACTOR Future<Void> masterServer( MasterInterface mi, Reference<AsyncVar<ServerDB
|
|||
TEST(err.code() == error_code_master_tlog_failed); // Master: terminated because of a tLog failure
|
||||
TEST(err.code() == error_code_master_proxy_failed); // Master: terminated because of a proxy failure
|
||||
TEST(err.code() == error_code_master_resolver_failed); // Master: terminated because of a resolver failure
|
||||
TEST(err.code() == error_code_master_backup_worker_failed); // Master: terminated because of a backup worker failure
|
||||
|
||||
if (normalMasterErrors().count(err.code()))
|
||||
{
|
||||
if (normalMasterErrors().count(err.code())) {
|
||||
TraceEvent("MasterTerminated", mi.id()).error(err);
|
||||
return Void();
|
||||
}
|
||||
|
|
|
@ -26,6 +26,7 @@
|
|||
#include "fdbrpc/simulator.h"
|
||||
#include "fdbclient/NativeAPI.actor.h"
|
||||
#include "fdbclient/MetricLogger.h"
|
||||
#include "fdbserver/BackupInterface.h"
|
||||
#include "fdbserver/WorkerInterface.actor.h"
|
||||
#include "fdbserver/IKeyValueStore.h"
|
||||
#include "fdbserver/WaitFailure.h"
|
||||
|
@ -1114,6 +1115,19 @@ ACTOR Future<Void> workerServer(
|
|||
TraceEvent("Ratekeeper_InitRequest", req.reqId).detail("RatekeeperId", recruited.id());
|
||||
req.reply.send(recruited);
|
||||
}
|
||||
when (InitializeBackupRequest req = waitNext(interf.backup.getFuture())) {
|
||||
BackupInterface recruited(locality);
|
||||
recruited.initEndpoints();
|
||||
|
||||
startRole(Role::BACKUP, recruited.id(), interf.id());
|
||||
DUMPTOKEN(recruited.waitFailure);
|
||||
|
||||
Future<Void> backupProcess = backupWorker(recruited, req, dbInfo);
|
||||
errorForwarders.add(forwardError(errors, Role::BACKUP, recruited.id(), backupProcess));
|
||||
TraceEvent("Backup_InitRequest", req.reqId).detail("BackupId", recruited.id());
|
||||
InitializeBackupReply reply(recruited, req.backupEpoch);
|
||||
req.reply.send(reply);
|
||||
}
|
||||
when( InitializeTLogRequest req = waitNext(interf.tLog.getFuture()) ) {
|
||||
// For now, there's a one-to-one mapping of spill type to TLogVersion.
|
||||
// With future work, a particular version of the TLog can support multiple
|
||||
|
@ -1545,3 +1559,4 @@ const Role Role::DATA_DISTRIBUTOR("DataDistributor", "DD");
|
|||
const Role Role::RATEKEEPER("Ratekeeper", "RK");
|
||||
const Role Role::STORAGE_CACHE("StorageCache", "SC");
|
||||
const Role Role::COORDINATOR("Coordinator", "CD");
|
||||
const Role Role::BACKUP("Backup", "BK");
|
||||
|
|
|
@ -121,7 +121,8 @@ struct ConsistencyCheckWorkload : TestWorkload
|
|||
}
|
||||
|
||||
try {
|
||||
wait(timeoutError(quietDatabase(cx, self->dbInfo, "ConsistencyCheckStart", 0, 1e5, 0, 0), self->quiescentWaitTimeout)); // FIXME: should be zero?
|
||||
wait(timeoutError(quietDatabase(cx, self->dbInfo, "ConsistencyCheckStart", 0, 1e5, 0, 0),
|
||||
self->quiescentWaitTimeout)); // FIXME: should be zero?
|
||||
}
|
||||
catch (Error& e) {
|
||||
TraceEvent("ConsistencyCheck_QuietDatabaseError").error(e);
|
||||
|
@ -272,14 +273,12 @@ struct ConsistencyCheckWorkload : TestWorkload
|
|||
try
|
||||
{
|
||||
int64_t maxStorageServerQueueSize = wait(getMaxStorageServerQueueSize(cx, self->dbInfo));
|
||||
if(maxStorageServerQueueSize > 0)
|
||||
{
|
||||
TraceEvent("ConsistencyCheck_NonZeroStorageServerQueue").detail("MaxQueueSize", maxStorageServerQueueSize);
|
||||
self->testFailure("Non-zero storage server queue size");
|
||||
if (maxStorageServerQueueSize > 0) {
|
||||
TraceEvent("ConsistencyCheck_ExceedStorageServerQueueLimit")
|
||||
.detail("MaxQueueSize", maxStorageServerQueueSize);
|
||||
self->testFailure("Storage server queue size exceeds limit");
|
||||
}
|
||||
}
|
||||
catch(Error& e)
|
||||
{
|
||||
} catch (Error& e) {
|
||||
if(e.code() == error_code_attribute_not_found)
|
||||
{
|
||||
TraceEvent("ConsistencyCheck_StorageQueueSizeError").error(e).detail("Reason", "Could not read queue size");
|
||||
|
|
|
@ -89,6 +89,7 @@ public: // introduced features
|
|||
PROTOCOL_VERSION_FEATURE(0x0FDB00B061070000LL, PseudoLocalities);
|
||||
PROTOCOL_VERSION_FEATURE(0x0FDB00B061070000LL, ShardedTxsTags);
|
||||
PROTOCOL_VERSION_FEATURE(0x0FDB00B063000000LL, UnifiedTLogSpilling);
|
||||
PROTOCOL_VERSION_FEATURE(0x0FDB00B063010000LL, BackupWorker);
|
||||
};
|
||||
|
||||
// These impact both communications and the deserialization of certain database and IKeyValueStore keys.
|
||||
|
|
|
@ -86,6 +86,7 @@ ERROR( please_reboot_delete, 1208, "Reboot of server process requested, with del
|
|||
ERROR( master_proxy_failed, 1209, "Master terminating because a Proxy failed" )
|
||||
ERROR( master_resolver_failed, 1210, "Master terminating because a Resolver failed" )
|
||||
ERROR( server_overloaded, 1211, "Server is under too much load and cannot respond" )
|
||||
ERROR( master_backup_worker_failed, 1212, "Master terminating because a backup worker failed")
|
||||
|
||||
// 15xx Platform errors
|
||||
ERROR( platform_error, 1500, "Platform error" )
|
||||
|
|
Loading…
Reference in New Issue