854 lines
28 KiB
C++
854 lines
28 KiB
C++
/*
|
|
*RocksDBCheckpointUtils.actor.cpp
|
|
*
|
|
* This source file is part of the FoundationDB open source project
|
|
*
|
|
* Copyright 2013-2022 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 "fdbserver/RocksDBCheckpointUtils.actor.h"
|
|
|
|
#ifdef SSD_ROCKSDB_EXPERIMENTAL
|
|
#include <rocksdb/db.h>
|
|
#include <rocksdb/env.h>
|
|
#include <rocksdb/options.h>
|
|
#include <rocksdb/slice.h>
|
|
#include <rocksdb/slice_transform.h>
|
|
#include <rocksdb/types.h>
|
|
#include <rocksdb/version.h>
|
|
#endif // SSD_ROCKSDB_EXPERIMENTAL
|
|
|
|
#include "fdbclient/FDBTypes.h"
|
|
#include "fdbclient/NativeAPI.actor.h"
|
|
#include "fdbclient/StorageCheckpoint.h"
|
|
#include "fdbserver/CoroFlow.h"
|
|
#include "fdbserver/Knobs.h"
|
|
#include "flow/IThreadPool.h"
|
|
#include "flow/ThreadHelper.actor.h"
|
|
#include "flow/Trace.h"
|
|
#include "flow/flow.h"
|
|
|
|
#include "flow/actorcompiler.h" // has to be last include
|
|
|
|
#ifdef SSD_ROCKSDB_EXPERIMENTAL
|
|
// Enforcing rocksdb version to be 6.22.1 or greater.
|
|
static_assert(ROCKSDB_MAJOR == 6 && ROCKSDB_MINOR >= 22 && ROCKSDB_PATCH >= 1,
|
|
"Unsupported rocksdb version. Update the rocksdb to at least 6.22.1 version");
|
|
|
|
namespace {
|
|
|
|
using DB = rocksdb::DB*;
|
|
using CF = rocksdb::ColumnFamilyHandle*;
|
|
|
|
const KeyRef persistVersion = "\xff\xffVersion"_sr;
|
|
|
|
rocksdb::Slice toSlice(StringRef s) {
|
|
return rocksdb::Slice(reinterpret_cast<const char*>(s.begin()), s.size());
|
|
}
|
|
|
|
StringRef toStringRef(rocksdb::Slice s) {
|
|
return StringRef(reinterpret_cast<const uint8_t*>(s.data()), s.size());
|
|
}
|
|
|
|
rocksdb::ColumnFamilyOptions getCFOptions() {
|
|
rocksdb::ColumnFamilyOptions options;
|
|
return options;
|
|
}
|
|
|
|
rocksdb::Options getOptions() {
|
|
rocksdb::Options options({}, getCFOptions());
|
|
options.create_if_missing = false;
|
|
options.db_log_dir = SERVER_KNOBS->LOG_DIRECTORY;
|
|
return options;
|
|
}
|
|
|
|
// Set some useful defaults desired for all reads.
|
|
rocksdb::ReadOptions getReadOptions() {
|
|
rocksdb::ReadOptions options;
|
|
options.background_purge_on_iterator_cleanup = true;
|
|
return options;
|
|
}
|
|
|
|
void logRocksDBError(const rocksdb::Status& status, const std::string& method) {
|
|
auto level = status.IsTimedOut() ? SevWarn : SevError;
|
|
TraceEvent e(level, "RocksDBCheckpointReaderError");
|
|
e.detail("Error", status.ToString()).detail("Method", method).detail("RocksDBSeverity", status.severity());
|
|
if (status.IsIOError()) {
|
|
e.detail("SubCode", status.subcode());
|
|
}
|
|
}
|
|
|
|
Error statusToError(const rocksdb::Status& s) {
|
|
if (s.IsIOError()) {
|
|
return io_error();
|
|
} else if (s.IsTimedOut()) {
|
|
return transaction_too_old();
|
|
} else {
|
|
return unknown_error();
|
|
}
|
|
}
|
|
|
|
// RocksDBCheckpointReader reads a RocksDB checkpoint, and returns the key-value pairs via nextKeyValues.
|
|
class RocksDBCheckpointReader : public ICheckpointReader {
|
|
public:
|
|
RocksDBCheckpointReader(const CheckpointMetaData& checkpoint, UID logID);
|
|
|
|
Future<Void> init(StringRef token) override;
|
|
|
|
Future<RangeResult> nextKeyValues(const int rowLimit, const int byteLimit) override;
|
|
|
|
Future<Standalone<StringRef>> nextChunk(const int byteLimit) { throw not_implemented(); }
|
|
|
|
Future<Void> close() { return doClose(this); }
|
|
|
|
private:
|
|
struct Reader : IThreadPoolReceiver {
|
|
struct OpenAction : TypedAction<Reader, OpenAction> {
|
|
OpenAction(std::string path, KeyRange range, Version version)
|
|
: path(std::move(path)), range(range), version(version) {}
|
|
|
|
double getTimeEstimate() const override { return SERVER_KNOBS->COMMIT_TIME_ESTIMATE; }
|
|
|
|
const std::string path;
|
|
const KeyRange range;
|
|
const Version version;
|
|
ThreadReturnPromise<Void> done;
|
|
};
|
|
|
|
struct CloseAction : TypedAction<Reader, CloseAction> {
|
|
CloseAction(std::string path, bool deleteOnClose) : path(path), deleteOnClose(deleteOnClose) {}
|
|
double getTimeEstimate() const override { return SERVER_KNOBS->COMMIT_TIME_ESTIMATE; }
|
|
|
|
std::string path;
|
|
bool deleteOnClose;
|
|
ThreadReturnPromise<Void> done;
|
|
};
|
|
|
|
struct ReadRangeAction : TypedAction<Reader, ReadRangeAction>, FastAllocated<ReadRangeAction> {
|
|
ReadRangeAction(int rowLimit, int byteLimit)
|
|
: rowLimit(rowLimit), byteLimit(byteLimit), startTime(timer_monotonic()) {}
|
|
|
|
double getTimeEstimate() const override { return SERVER_KNOBS->READ_RANGE_TIME_ESTIMATE; }
|
|
|
|
const int rowLimit, byteLimit;
|
|
const double startTime;
|
|
ThreadReturnPromise<RangeResult> result;
|
|
};
|
|
|
|
explicit Reader(DB& db);
|
|
~Reader() override {}
|
|
|
|
void init() override {}
|
|
|
|
void action(OpenAction& a);
|
|
|
|
void action(CloseAction& a);
|
|
|
|
void action(ReadRangeAction& a);
|
|
|
|
DB& db;
|
|
CF cf;
|
|
Key begin;
|
|
Key end;
|
|
double readRangeTimeout;
|
|
std::unique_ptr<rocksdb::Iterator> cursor;
|
|
};
|
|
|
|
ACTOR static Future<Void> doClose(RocksDBCheckpointReader* self);
|
|
|
|
DB db = nullptr;
|
|
std::string path;
|
|
const UID id;
|
|
Version version;
|
|
Reference<IThreadPool> readThreads;
|
|
Future<Void> openFuture;
|
|
};
|
|
|
|
RocksDBCheckpointReader::RocksDBCheckpointReader(const CheckpointMetaData& checkpoint, UID logID)
|
|
: id(logID), version(checkpoint.version) {
|
|
RocksDBCheckpoint rocksCheckpoint = getRocksCheckpoint(checkpoint);
|
|
this->path = rocksCheckpoint.checkpointDir;
|
|
if (g_network->isSimulated()) {
|
|
readThreads = CoroThreadPool::createThreadPool();
|
|
} else {
|
|
readThreads = createGenericThreadPool();
|
|
}
|
|
readThreads->addThread(new Reader(db), "fdb-rocks-rd");
|
|
}
|
|
|
|
Future<Void> RocksDBCheckpointReader::init(StringRef token) {
|
|
if (openFuture.isValid()) {
|
|
return openFuture;
|
|
}
|
|
|
|
KeyRange range = BinaryReader::fromStringRef<KeyRange>(token, IncludeVersion());
|
|
auto a = std::make_unique<Reader::OpenAction>(this->path, range, this->version);
|
|
openFuture = a->done.getFuture();
|
|
readThreads->post(a.release());
|
|
return openFuture;
|
|
}
|
|
|
|
Future<RangeResult> RocksDBCheckpointReader::nextKeyValues(const int rowLimit, const int byteLimit) {
|
|
auto a = std::make_unique<Reader::ReadRangeAction>(rowLimit, byteLimit);
|
|
auto res = a->result.getFuture();
|
|
readThreads->post(a.release());
|
|
return res;
|
|
}
|
|
|
|
RocksDBCheckpointReader::Reader::Reader(DB& db) : db(db), cf(nullptr) {
|
|
if (g_network->isSimulated()) {
|
|
// In simulation, increasing the read operation timeouts to 5 minutes, as some of the tests have
|
|
// very high load and single read thread cannot process all the load within the timeouts.
|
|
readRangeTimeout = 5 * 60;
|
|
} else {
|
|
readRangeTimeout = SERVER_KNOBS->ROCKSDB_READ_RANGE_TIMEOUT;
|
|
}
|
|
}
|
|
|
|
void RocksDBCheckpointReader::Reader::action(RocksDBCheckpointReader::Reader::OpenAction& a) {
|
|
ASSERT(cf == nullptr);
|
|
|
|
std::vector<std::string> columnFamilies;
|
|
rocksdb::Options options = getOptions();
|
|
rocksdb::Status status = rocksdb::DB::ListColumnFamilies(options, a.path, &columnFamilies);
|
|
if (std::find(columnFamilies.begin(), columnFamilies.end(), "default") == columnFamilies.end()) {
|
|
columnFamilies.push_back("default");
|
|
}
|
|
|
|
rocksdb::ColumnFamilyOptions cfOptions = getCFOptions();
|
|
std::vector<rocksdb::ColumnFamilyDescriptor> descriptors;
|
|
for (const std::string& name : columnFamilies) {
|
|
descriptors.push_back(rocksdb::ColumnFamilyDescriptor{ name, cfOptions });
|
|
}
|
|
|
|
std::vector<rocksdb::ColumnFamilyHandle*> handles;
|
|
status = rocksdb::DB::OpenForReadOnly(options, a.path, descriptors, &handles, &db);
|
|
|
|
if (!status.ok()) {
|
|
logRocksDBError(status, "OpenForReadOnly");
|
|
a.done.sendError(statusToError(status));
|
|
return;
|
|
}
|
|
|
|
for (rocksdb::ColumnFamilyHandle* handle : handles) {
|
|
if (handle->GetName() == SERVER_KNOBS->DEFAULT_FDB_ROCKSDB_COLUMN_FAMILY) {
|
|
cf = handle;
|
|
break;
|
|
}
|
|
}
|
|
|
|
ASSERT(db != nullptr && cf != nullptr);
|
|
|
|
begin = a.range.begin;
|
|
end = a.range.end;
|
|
|
|
TraceEvent(SevInfo, "RocksDBCheckpointReaderInit")
|
|
.detail("Path", a.path)
|
|
.detail("Method", "OpenForReadOnly")
|
|
.detail("ColumnFamily", cf->GetName())
|
|
.detail("Begin", begin)
|
|
.detail("End", end);
|
|
|
|
rocksdb::PinnableSlice value;
|
|
rocksdb::ReadOptions readOptions = getReadOptions();
|
|
status = db->Get(readOptions, cf, toSlice(persistVersion), &value);
|
|
|
|
if (!status.ok() && !status.IsNotFound()) {
|
|
logRocksDBError(status, "Checkpoint");
|
|
a.done.sendError(statusToError(status));
|
|
return;
|
|
}
|
|
|
|
const Version version =
|
|
status.IsNotFound() ? latestVersion : BinaryReader::fromStringRef<Version>(toStringRef(value), Unversioned());
|
|
|
|
ASSERT(version == a.version);
|
|
|
|
cursor = std::unique_ptr<rocksdb::Iterator>(db->NewIterator(readOptions, cf));
|
|
cursor->Seek(toSlice(begin));
|
|
|
|
a.done.send(Void());
|
|
}
|
|
|
|
void RocksDBCheckpointReader::Reader::action(RocksDBCheckpointReader::Reader::CloseAction& a) {
|
|
if (db == nullptr) {
|
|
a.done.send(Void());
|
|
return;
|
|
}
|
|
|
|
rocksdb::Status s = db->Close();
|
|
if (!s.ok()) {
|
|
logRocksDBError(s, "Close");
|
|
}
|
|
|
|
if (a.deleteOnClose) {
|
|
std::set<std::string> columnFamilies{ "default" };
|
|
columnFamilies.insert(SERVER_KNOBS->DEFAULT_FDB_ROCKSDB_COLUMN_FAMILY);
|
|
std::vector<rocksdb::ColumnFamilyDescriptor> descriptors;
|
|
for (const std::string& name : columnFamilies) {
|
|
descriptors.push_back(rocksdb::ColumnFamilyDescriptor{ name, getCFOptions() });
|
|
}
|
|
s = rocksdb::DestroyDB(a.path, getOptions(), descriptors);
|
|
if (!s.ok()) {
|
|
logRocksDBError(s, "Destroy");
|
|
} else {
|
|
TraceEvent("RocksDBCheckpointReader").detail("Path", a.path).detail("Method", "Destroy");
|
|
}
|
|
}
|
|
|
|
TraceEvent("RocksDBCheckpointReader").detail("Path", a.path).detail("Method", "Close");
|
|
a.done.send(Void());
|
|
}
|
|
|
|
void RocksDBCheckpointReader::Reader::action(RocksDBCheckpointReader::Reader::ReadRangeAction& a) {
|
|
const double readBeginTime = timer_monotonic();
|
|
|
|
if (readBeginTime - a.startTime > readRangeTimeout) {
|
|
TraceEvent(SevWarn, "RocksDBCheckpointReaderError")
|
|
.detail("Error", "Read range request timedout")
|
|
.detail("Method", "ReadRangeAction")
|
|
.detail("Timeout value", readRangeTimeout);
|
|
a.result.sendError(timed_out());
|
|
return;
|
|
}
|
|
|
|
RangeResult result;
|
|
if (a.rowLimit == 0 || a.byteLimit == 0) {
|
|
a.result.send(result);
|
|
return;
|
|
}
|
|
|
|
// For now, only forward scan is supported.
|
|
ASSERT(a.rowLimit > 0);
|
|
|
|
int accumulatedBytes = 0;
|
|
rocksdb::Status s;
|
|
while (cursor->Valid() && toStringRef(cursor->key()) < end) {
|
|
KeyValueRef kv(toStringRef(cursor->key()), toStringRef(cursor->value()));
|
|
accumulatedBytes += sizeof(KeyValueRef) + kv.expectedSize();
|
|
result.push_back_deep(result.arena(), kv);
|
|
cursor->Next();
|
|
if (result.size() >= a.rowLimit || accumulatedBytes >= a.byteLimit) {
|
|
break;
|
|
}
|
|
if (timer_monotonic() - a.startTime > readRangeTimeout) {
|
|
TraceEvent(SevWarn, "RocksDBCheckpointReaderError")
|
|
.detail("Error", "Read range request timedout")
|
|
.detail("Method", "ReadRangeAction")
|
|
.detail("Timeout value", readRangeTimeout);
|
|
a.result.sendError(transaction_too_old());
|
|
delete (cursor.release());
|
|
return;
|
|
}
|
|
}
|
|
|
|
s = cursor->status();
|
|
|
|
if (!s.ok()) {
|
|
logRocksDBError(s, "ReadRange");
|
|
a.result.sendError(statusToError(s));
|
|
delete (cursor.release());
|
|
return;
|
|
}
|
|
|
|
if (result.empty()) {
|
|
delete (cursor.release());
|
|
a.result.sendError(end_of_stream());
|
|
} else {
|
|
a.result.send(result);
|
|
}
|
|
}
|
|
|
|
ACTOR Future<Void> RocksDBCheckpointReader::doClose(RocksDBCheckpointReader* self) {
|
|
if (self == nullptr)
|
|
return Void();
|
|
|
|
auto a = new RocksDBCheckpointReader::Reader::CloseAction(self->path, false);
|
|
auto f = a->done.getFuture();
|
|
self->readThreads->post(a);
|
|
wait(f);
|
|
|
|
if (self != nullptr) {
|
|
wait(self->readThreads->stop());
|
|
}
|
|
|
|
if (self != nullptr) {
|
|
delete self;
|
|
}
|
|
|
|
return Void();
|
|
}
|
|
|
|
// RocksDBCFCheckpointReader reads an exported RocksDB Column Family checkpoint, and returns the serialized
|
|
// checkpoint via nextChunk.
|
|
class RocksDBCFCheckpointReader : public ICheckpointReader {
|
|
public:
|
|
RocksDBCFCheckpointReader(const CheckpointMetaData& checkpoint, UID logID)
|
|
: checkpoint_(checkpoint), id_(logID), file_(Reference<IAsyncFile>()), offset_(0) {}
|
|
|
|
Future<Void> init(StringRef token) override;
|
|
|
|
Future<RangeResult> nextKeyValues(const int rowLimit, const int byteLimit) override { throw not_implemented(); }
|
|
|
|
Future<Standalone<StringRef>> nextChunk(const int byteLimit) override;
|
|
|
|
Future<Void> close() override;
|
|
|
|
private:
|
|
ACTOR static Future<Void> doInit(RocksDBCFCheckpointReader* self) {
|
|
ASSERT(self != nullptr);
|
|
try {
|
|
state Reference<IAsyncFile> _file = wait(IAsyncFileSystem::filesystem()->open(
|
|
self->path_, IAsyncFile::OPEN_READONLY | IAsyncFile::OPEN_UNCACHED | IAsyncFile::OPEN_NO_AIO, 0));
|
|
self->file_ = _file;
|
|
TraceEvent("RocksDBCheckpointReaderOpenFile").detail("File", self->path_);
|
|
} catch (Error& e) {
|
|
TraceEvent(SevWarnAlways, "ServerGetCheckpointFileFailure")
|
|
.errorUnsuppressed(e)
|
|
.detail("File", self->path_);
|
|
throw e;
|
|
}
|
|
|
|
return Void();
|
|
}
|
|
|
|
ACTOR static Future<Standalone<StringRef>> getNextChunk(RocksDBCFCheckpointReader* self, int byteLimit) {
|
|
int blockSize = std::min(64 * 1024, byteLimit); // Block size read from disk.
|
|
state Standalone<StringRef> buf = makeAlignedString(_PAGE_SIZE, blockSize);
|
|
int bytesRead = wait(self->file_->read(mutateString(buf), blockSize, self->offset_));
|
|
if (bytesRead == 0) {
|
|
throw end_of_stream();
|
|
}
|
|
|
|
self->offset_ += bytesRead;
|
|
return buf.substr(0, bytesRead);
|
|
}
|
|
|
|
ACTOR static Future<Void> doClose(RocksDBCFCheckpointReader* self) {
|
|
wait(delay(0, TaskPriority::FetchKeys));
|
|
delete self;
|
|
return Void();
|
|
}
|
|
|
|
CheckpointMetaData checkpoint_;
|
|
UID id_;
|
|
Reference<IAsyncFile> file_;
|
|
int offset_;
|
|
std::string path_;
|
|
};
|
|
|
|
Future<Void> RocksDBCFCheckpointReader::init(StringRef token) {
|
|
ASSERT_EQ(this->checkpoint_.getFormat(), RocksDBColumnFamily);
|
|
const std::string name = token.toString();
|
|
this->offset_ = 0;
|
|
this->path_.clear();
|
|
const RocksDBColumnFamilyCheckpoint rocksCF = getRocksCF(this->checkpoint_);
|
|
for (const auto& sstFile : rocksCF.sstFiles) {
|
|
if (sstFile.name == name) {
|
|
this->path_ = sstFile.db_path + sstFile.name;
|
|
break;
|
|
}
|
|
}
|
|
|
|
if (this->path_.empty()) {
|
|
TraceEvent("RocksDBCheckpointReaderInitFileNotFound").detail("File", this->path_);
|
|
return checkpoint_not_found();
|
|
}
|
|
|
|
return doInit(this);
|
|
}
|
|
|
|
Future<Standalone<StringRef>> RocksDBCFCheckpointReader::nextChunk(const int byteLimit) {
|
|
return getNextChunk(this, byteLimit);
|
|
}
|
|
|
|
Future<Void> RocksDBCFCheckpointReader::close() {
|
|
return doClose(this);
|
|
}
|
|
|
|
// Fetch a single sst file from storage server. If the file is fetch successfully, it will be recorded via cFun.
|
|
ACTOR Future<Void> fetchCheckpointFile(Database cx,
|
|
std::shared_ptr<CheckpointMetaData> metaData,
|
|
int idx,
|
|
std::string dir,
|
|
std::function<Future<Void>(const CheckpointMetaData&)> cFun,
|
|
int maxRetries = 3) {
|
|
state RocksDBColumnFamilyCheckpoint rocksCF;
|
|
ObjectReader reader(metaData->serializedCheckpoint.begin(), IncludeVersion());
|
|
reader.deserialize(rocksCF);
|
|
|
|
// Skip fetched file.
|
|
if (rocksCF.sstFiles[idx].fetched && rocksCF.sstFiles[idx].db_path == dir) {
|
|
return Void();
|
|
}
|
|
|
|
state std::string remoteFile = rocksCF.sstFiles[idx].name;
|
|
state std::string localFile = dir + rocksCF.sstFiles[idx].name;
|
|
state UID ssID = metaData->ssID;
|
|
|
|
state Transaction tr(cx);
|
|
state StorageServerInterface ssi;
|
|
loop {
|
|
try {
|
|
tr.setOption(FDBTransactionOptions::READ_SYSTEM_KEYS);
|
|
Optional<Value> ss = wait(tr.get(serverListKeyFor(ssID)));
|
|
if (!ss.present()) {
|
|
throw checkpoint_not_found();
|
|
}
|
|
ssi = decodeServerListValue(ss.get());
|
|
break;
|
|
} catch (Error& e) {
|
|
wait(tr.onError(e));
|
|
}
|
|
}
|
|
|
|
state int attempt = 0;
|
|
loop {
|
|
try {
|
|
++attempt;
|
|
TraceEvent("FetchCheckpointFileBegin")
|
|
.detail("RemoteFile", remoteFile)
|
|
.detail("TargetUID", ssID.toString())
|
|
.detail("StorageServer", ssi.id().toString())
|
|
.detail("LocalFile", localFile)
|
|
.detail("Attempt", attempt);
|
|
|
|
wait(IAsyncFileSystem::filesystem()->deleteFile(localFile, true));
|
|
const int64_t flags = IAsyncFile::OPEN_ATOMIC_WRITE_AND_CREATE | IAsyncFile::OPEN_READWRITE |
|
|
IAsyncFile::OPEN_CREATE | IAsyncFile::OPEN_UNCACHED | IAsyncFile::OPEN_NO_AIO;
|
|
state int64_t offset = 0;
|
|
state Reference<IAsyncFile> asyncFile = wait(IAsyncFileSystem::filesystem()->open(localFile, flags, 0666));
|
|
|
|
state ReplyPromiseStream<FetchCheckpointReply> stream =
|
|
ssi.fetchCheckpoint.getReplyStream(FetchCheckpointRequest(metaData->checkpointID, remoteFile));
|
|
TraceEvent("FetchCheckpointFileReceivingData")
|
|
.detail("RemoteFile", remoteFile)
|
|
.detail("TargetUID", ssID.toString())
|
|
.detail("StorageServer", ssi.id().toString())
|
|
.detail("LocalFile", localFile)
|
|
.detail("Attempt", attempt);
|
|
loop {
|
|
state FetchCheckpointReply rep = waitNext(stream.getFuture());
|
|
wait(asyncFile->write(rep.data.begin(), rep.data.size(), offset));
|
|
wait(asyncFile->flush());
|
|
offset += rep.data.size();
|
|
}
|
|
} catch (Error& e) {
|
|
if (e.code() != error_code_end_of_stream) {
|
|
TraceEvent("FetchCheckpointFileError")
|
|
.errorUnsuppressed(e)
|
|
.detail("RemoteFile", remoteFile)
|
|
.detail("StorageServer", ssi.toString())
|
|
.detail("LocalFile", localFile)
|
|
.detail("Attempt", attempt);
|
|
if (attempt >= maxRetries) {
|
|
throw e;
|
|
}
|
|
} else {
|
|
wait(asyncFile->sync());
|
|
int64_t fileSize = wait(asyncFile->size());
|
|
TraceEvent("FetchCheckpointFileEnd")
|
|
.detail("RemoteFile", remoteFile)
|
|
.detail("StorageServer", ssi.toString())
|
|
.detail("LocalFile", localFile)
|
|
.detail("Attempt", attempt)
|
|
.detail("DataSize", offset)
|
|
.detail("FileSize", fileSize);
|
|
rocksCF.sstFiles[idx].db_path = dir;
|
|
rocksCF.sstFiles[idx].fetched = true;
|
|
metaData->serializedCheckpoint = ObjectWriter::toValue(rocksCF, IncludeVersion());
|
|
if (cFun) {
|
|
wait(cFun(*metaData));
|
|
}
|
|
return Void();
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
// TODO: Return when a file exceeds a limit.
|
|
ACTOR Future<Void> fetchCheckpointRange(Database cx,
|
|
std::shared_ptr<CheckpointMetaData> metaData,
|
|
KeyRange range,
|
|
std::string dir,
|
|
std::shared_ptr<rocksdb::SstFileWriter> writer,
|
|
std::function<Future<Void>(const CheckpointMetaData&)> cFun,
|
|
int maxRetries = 3) {
|
|
state std::string localFile = dir + "/" + metaData->checkpointID.toString() + ".sst";
|
|
RocksDBCheckpoint rcp = getRocksCheckpoint(*metaData);
|
|
TraceEvent("FetchCheckpointRange")
|
|
.detail("InitialState", metaData->toString())
|
|
.detail("RocksCheckpoint", rcp.toString());
|
|
|
|
for (const auto& file : rcp.fetchedFiles) {
|
|
ASSERT(!file.range.intersects(range));
|
|
}
|
|
|
|
state UID ssID = metaData->ssID;
|
|
state Transaction tr(cx);
|
|
state StorageServerInterface ssi;
|
|
loop {
|
|
tr.setOption(FDBTransactionOptions::LOCK_AWARE);
|
|
tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
|
|
try {
|
|
Optional<Value> ss = wait(tr.get(serverListKeyFor(ssID)));
|
|
if (!ss.present()) {
|
|
TraceEvent(SevWarnAlways, "FetchCheckpointRangeStorageServerNotFound")
|
|
.detail("SSID", ssID)
|
|
.detail("InitialState", metaData->toString());
|
|
throw checkpoint_not_found();
|
|
}
|
|
ssi = decodeServerListValue(ss.get());
|
|
break;
|
|
} catch (Error& e) {
|
|
wait(tr.onError(e));
|
|
}
|
|
}
|
|
|
|
ASSERT(ssi.id() == ssID);
|
|
|
|
state int attempt = 0;
|
|
state int64_t totalBytes = 0;
|
|
state rocksdb::Status status;
|
|
state Optional<Error> error;
|
|
loop {
|
|
totalBytes = 0;
|
|
++attempt;
|
|
try {
|
|
TraceEvent(SevInfo, "FetchCheckpointRangeBegin")
|
|
.detail("CheckpointID", metaData->checkpointID)
|
|
.detail("Range", range.toString())
|
|
.detail("TargetStorageServerUID", ssID)
|
|
.detail("LocalFile", localFile)
|
|
.detail("Attempt", attempt)
|
|
.log();
|
|
|
|
wait(IAsyncFileSystem::filesystem()->deleteFile(localFile, true));
|
|
status = writer->Open(localFile);
|
|
if (!status.ok()) {
|
|
Error e = statusToError(status);
|
|
TraceEvent(SevError, "FetchCheckpointRangeOpenFileError")
|
|
.detail("LocalFile", localFile)
|
|
.detail("Status", status.ToString());
|
|
throw e;
|
|
}
|
|
|
|
state ReplyPromiseStream<FetchCheckpointKeyValuesStreamReply> stream =
|
|
ssi.fetchCheckpointKeyValues.getReplyStream(
|
|
FetchCheckpointKeyValuesRequest(metaData->checkpointID, range));
|
|
TraceEvent(SevDebug, "FetchCheckpointKeyValuesReceivingData")
|
|
.detail("CheckpointID", metaData->checkpointID)
|
|
.detail("Range", range.toString())
|
|
.detail("TargetStorageServerUID", ssID.toString())
|
|
.detail("LocalFile", localFile)
|
|
.detail("Attempt", attempt)
|
|
.log();
|
|
|
|
loop {
|
|
FetchCheckpointKeyValuesStreamReply rep = waitNext(stream.getFuture());
|
|
for (int i = 0; i < rep.data.size(); ++i) {
|
|
status = writer->Put(toSlice(rep.data[i].key), toSlice(rep.data[i].value));
|
|
if (!status.ok()) {
|
|
Error e = statusToError(status);
|
|
TraceEvent(SevError, "FetchCheckpointRangeWriteError")
|
|
.detail("LocalFile", localFile)
|
|
.detail("Key", rep.data[i].key.toString())
|
|
.detail("Value", rep.data[i].value.toString())
|
|
.detail("Status", status.ToString());
|
|
throw e;
|
|
}
|
|
totalBytes += rep.data[i].expectedSize();
|
|
}
|
|
}
|
|
} catch (Error& e) {
|
|
Error err = e;
|
|
if (totalBytes > 0) {
|
|
status = writer->Finish();
|
|
if (!status.ok()) {
|
|
err = statusToError(status);
|
|
}
|
|
}
|
|
if (err.code() != error_code_end_of_stream) {
|
|
TraceEvent(SevWarn, "FetchCheckpointFileError")
|
|
.errorUnsuppressed(err)
|
|
.detail("CheckpointID", metaData->checkpointID)
|
|
.detail("Range", range.toString())
|
|
.detail("TargetStorageServerUID", ssID.toString())
|
|
.detail("LocalFile", localFile)
|
|
.detail("Attempt", attempt);
|
|
if (attempt >= maxRetries) {
|
|
error = err;
|
|
break;
|
|
}
|
|
} else {
|
|
if (totalBytes > 0) {
|
|
RocksDBCheckpoint rcp = getRocksCheckpoint(*metaData);
|
|
rcp.fetchedFiles.emplace_back(localFile, range, totalBytes);
|
|
rcp.checkpointDir = dir;
|
|
metaData->serializedCheckpoint = ObjectWriter::toValue(rcp, IncludeVersion());
|
|
}
|
|
if (!fileExists(localFile)) {
|
|
TraceEvent(SevWarn, "FetchCheckpointRangeEndFileNotFound")
|
|
.detail("CheckpointID", metaData->checkpointID)
|
|
.detail("Range", range.toString())
|
|
.detail("TargetStorageServerUID", ssID.toString())
|
|
.detail("LocalFile", localFile)
|
|
.detail("Attempt", attempt)
|
|
.detail("TotalBytes", totalBytes);
|
|
} else {
|
|
TraceEvent(SevInfo, "FetchCheckpointRangeEnd")
|
|
.detail("CheckpointID", metaData->checkpointID)
|
|
.detail("Range", range.toString())
|
|
.detail("TargetStorageServerUID", ssID.toString())
|
|
.detail("LocalFile", localFile)
|
|
.detail("Attempt", attempt)
|
|
.detail("TotalBytes", totalBytes);
|
|
break;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
if (error.present()) {
|
|
throw error.get();
|
|
}
|
|
|
|
return Void();
|
|
}
|
|
|
|
} // namespace
|
|
|
|
ACTOR Future<CheckpointMetaData> fetchRocksDBCheckpoint(Database cx,
|
|
CheckpointMetaData initialState,
|
|
std::string dir,
|
|
std::function<Future<Void>(const CheckpointMetaData&)> cFun) {
|
|
TraceEvent(SevInfo, "FetchRocksCheckpointBegin")
|
|
.detail("InitialState", initialState.toString())
|
|
.detail("CheckpointDir", dir);
|
|
|
|
state std::shared_ptr<CheckpointMetaData> metaData = std::make_shared<CheckpointMetaData>(initialState);
|
|
|
|
if (metaData->format == RocksDBColumnFamily) {
|
|
state RocksDBColumnFamilyCheckpoint rocksCF = getRocksCF(initialState);
|
|
TraceEvent(SevDebug, "RocksDBCheckpointMetaData").detail("RocksCF", rocksCF.toString());
|
|
|
|
state int i = 0;
|
|
state std::vector<Future<Void>> fs;
|
|
for (; i < rocksCF.sstFiles.size(); ++i) {
|
|
fs.push_back(fetchCheckpointFile(cx, metaData, i, dir, cFun));
|
|
TraceEvent(SevDebug, "GetCheckpointFetchingFile")
|
|
.detail("FileName", rocksCF.sstFiles[i].name)
|
|
.detail("Server", metaData->ssID.toString());
|
|
}
|
|
wait(waitForAll(fs));
|
|
} else if (metaData->format == RocksDB) {
|
|
std::shared_ptr<rocksdb::SstFileWriter> writer =
|
|
std::make_shared<rocksdb::SstFileWriter>(rocksdb::EnvOptions(), rocksdb::Options());
|
|
wait(fetchCheckpointRange(cx, metaData, metaData->range, dir, writer, cFun));
|
|
}
|
|
|
|
return *metaData;
|
|
}
|
|
|
|
ACTOR Future<Void> deleteRocksCheckpoint(CheckpointMetaData checkpoint) {
|
|
state CheckpointFormat format = checkpoint.getFormat();
|
|
state std::unordered_set<std::string> dirs;
|
|
if (format == RocksDBColumnFamily) {
|
|
RocksDBColumnFamilyCheckpoint rocksCF = getRocksCF(checkpoint);
|
|
TraceEvent(SevInfo, "DeleteRocksColumnFamilyCheckpoint", checkpoint.checkpointID)
|
|
.detail("CheckpointID", checkpoint.checkpointID)
|
|
.detail("RocksCF", rocksCF.toString());
|
|
|
|
for (const LiveFileMetaData& file : rocksCF.sstFiles) {
|
|
dirs.insert(file.db_path);
|
|
}
|
|
} else if (format == RocksDB) {
|
|
RocksDBCheckpoint rocksCheckpoint = getRocksCheckpoint(checkpoint);
|
|
TraceEvent(SevInfo, "DeleteRocksCheckpoint", checkpoint.checkpointID)
|
|
.detail("CheckpointID", checkpoint.checkpointID)
|
|
.detail("RocksCheckpoint", rocksCheckpoint.toString());
|
|
dirs.insert(rocksCheckpoint.checkpointDir);
|
|
} else {
|
|
ASSERT(false);
|
|
}
|
|
|
|
state std::unordered_set<std::string>::iterator it = dirs.begin();
|
|
for (; it != dirs.end(); ++it) {
|
|
const std::string dir = *it;
|
|
platform::eraseDirectoryRecursive(dir);
|
|
TraceEvent(SevInfo, "DeleteCheckpointRemovedDir", checkpoint.checkpointID)
|
|
.detail("CheckpointID", checkpoint.checkpointID)
|
|
.detail("Dir", dir);
|
|
wait(delay(0, TaskPriority::FetchKeys));
|
|
}
|
|
|
|
return Void();
|
|
}
|
|
#else
|
|
ACTOR Future<CheckpointMetaData> fetchRocksDBCheckpoint(Database cx,
|
|
CheckpointMetaData initialState,
|
|
std::string dir,
|
|
std::function<Future<Void>(const CheckpointMetaData&)> cFun) {
|
|
wait(delay(0));
|
|
return initialState;
|
|
}
|
|
|
|
ACTOR Future<Void> deleteRocksCheckpoint(CheckpointMetaData checkpoint) {
|
|
wait(delay(0));
|
|
return Void();
|
|
}
|
|
#endif // SSD_ROCKSDB_EXPERIMENTAL
|
|
|
|
int64_t getTotalFetchedBytes(const std::vector<CheckpointMetaData>& checkpoints) {
|
|
int64_t totalBytes = 0;
|
|
for (const auto& checkpoint : checkpoints) {
|
|
const CheckpointFormat format = checkpoint.getFormat();
|
|
if (format == RocksDBColumnFamily) {
|
|
// TODO: Returns the checkpoint size of a RocksDB Column Family.
|
|
} else if (format == RocksDB) {
|
|
auto rcp = getRocksCheckpoint(checkpoint);
|
|
for (const auto& file : rcp.fetchedFiles) {
|
|
totalBytes += file.size;
|
|
}
|
|
}
|
|
}
|
|
return totalBytes;
|
|
}
|
|
|
|
ICheckpointReader* newRocksDBCheckpointReader(const CheckpointMetaData& checkpoint, UID logID) {
|
|
#ifdef SSD_ROCKSDB_EXPERIMENTAL
|
|
const CheckpointFormat format = checkpoint.getFormat();
|
|
if (format == RocksDBColumnFamily) {
|
|
return new RocksDBCFCheckpointReader(checkpoint, logID);
|
|
} else if (format == RocksDB) {
|
|
return new RocksDBCheckpointReader(checkpoint, logID);
|
|
}
|
|
#endif // SSD_ROCKSDB_EXPERIMENTAL
|
|
return nullptr;
|
|
}
|
|
|
|
RocksDBColumnFamilyCheckpoint getRocksCF(const CheckpointMetaData& checkpoint) {
|
|
RocksDBColumnFamilyCheckpoint rocksCF;
|
|
ObjectReader reader(checkpoint.serializedCheckpoint.begin(), IncludeVersion());
|
|
reader.deserialize(rocksCF);
|
|
return rocksCF;
|
|
}
|
|
|
|
RocksDBCheckpoint getRocksCheckpoint(const CheckpointMetaData& checkpoint) {
|
|
RocksDBCheckpoint rocksCheckpoint;
|
|
ObjectReader reader(checkpoint.serializedCheckpoint.begin(), IncludeVersion());
|
|
reader.deserialize(rocksCheckpoint);
|
|
return rocksCheckpoint;
|
|
} |