Skip to content

[DO NOT MERGE] bulkload try grpc #12028

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Draft
wants to merge 1 commit into
base: main
Choose a base branch
from
Draft
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions fdbclient/ServerKnobs.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -386,6 +386,7 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi
init( CC_ENFORCE_USE_UNFIT_DD_IN_SIM, false );
init( DISABLE_AUDIT_STORAGE_FINAL_REPLICA_CHECK_IN_SIM, false );
init( SS_BULKLOAD_GETRANGE_BATCH_SIZE, 10000 ); if (isSimulated) SS_BULKLOAD_GETRANGE_BATCH_SIZE = deterministicRandom()->randomInt(1, 10);
init( SS_BULKLOAD_BACKGROUND_TASK_COUNT, 1 );
init( BULKLOAD_ASYNC_READ_WRITE_BLOCK_SIZE, 1024*1024 ); if (isSimulated) BULKLOAD_ASYNC_READ_WRITE_BLOCK_SIZE = deterministicRandom()->randomInt(1024, 10240);

// BulkDumping
Expand Down
1 change: 1 addition & 0 deletions fdbclient/include/fdbclient/ServerKnobs.h
Original file line number Diff line number Diff line change
Expand Up @@ -419,6 +419,7 @@ class SWIFT_CXX_IMMORTAL_SINGLETON_TYPE ServerKnobs : public KnobsImpl<ServerKno
int SS_SERVE_BULKDUMP_PARALLELISM; // the number of bulk dump tasks that can concurrently happen at a SS
int64_t SS_BULKDUMP_BATCH_BYTES; // the max bytes when SS creates a batch to dump
int SS_BULKLOAD_GETRANGE_BATCH_SIZE; // the max number of keys to scan before do context switch
int SS_BULKLOAD_BACKGROUND_TASK_COUNT; // the max number of background tasks count for bulkload at a SS

// Run storage engine on a child process on the same machine with storage process
bool REMOTE_KV_STORE;
Expand Down
2 changes: 1 addition & 1 deletion fdbrpc/include/fdbrpc/grpc/AsyncTaskExecutor.h
Original file line number Diff line number Diff line change
Expand Up @@ -137,7 +137,7 @@ struct AsyncTaskExecutor::Action<Func, typename std::enable_if_t<!IsVoidReturn<F
// (including C++ coroutines).
promise_.sendError(unknown_error());
}
delete this;
// delete this;
}

// Returns `Future` representing the value returned by `fn_`.
Expand Down
2 changes: 1 addition & 1 deletion fdbserver/DataDistribution.actor.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -2362,7 +2362,7 @@ ACTOR Future<Void> bulkDumpManager(Reference<DataDistributor> self) {
loop {
bool allComplete = wait(checkBulkDumpJobComplete(self));
if (allComplete) {
TraceEvent(SevInfo, "DDBulkDumpManagerJobNotComplete", self->ddId).detail("JobId", jobId);
TraceEvent(SevInfo, "DDBulkDumpManagerJobAllTaskComplete", self->ddId).detail("JobId", jobId);
// Generate the job manifest file for bulkload.
// The job manifest file is the global map between ranges and their corresponding manifest file.
// When bulkload job loads a range, the job relies on this map to find the correct
Expand Down
12 changes: 6 additions & 6 deletions fdbserver/RocksDBCheckpointUtils.actor.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -733,7 +733,7 @@ ACTOR Future<Void> RocksDBColumnFamilyReader::doClose(RocksDBColumnFamilyReader*
class RocksDBSstFileWriter : public IRocksDBSstFileWriter {
public:
RocksDBSstFileWriter()
: writer(std::make_unique<rocksdb::SstFileWriter>(rocksdb::EnvOptions(), rocksdb::Options())), hasData(false){};
: writer(std::make_unique<rocksdb::SstFileWriter>(rocksdb::EnvOptions(), rocksdb::Options())), hasData(false) {};

void open(const std::string localFile) override;

Expand Down Expand Up @@ -789,7 +789,7 @@ bool RocksDBSstFileWriter::finish() {

class RocksDBSstFileReader : public IRocksDBSstFileReader {
public:
RocksDBSstFileReader() : sstReader(std::make_unique<rocksdb::SstFileReader>(rocksdb::Options())){};
RocksDBSstFileReader() : sstReader(std::make_unique<rocksdb::SstFileReader>(rocksdb::Options())) {};

RocksDBSstFileReader(const KeyRange& rangeBoundary, size_t rowLimit, size_t byteLimit)
: sstReader(std::make_unique<rocksdb::SstFileReader>(rocksdb::Options())), rowLimit(rowLimit),
Expand Down Expand Up @@ -872,10 +872,10 @@ RangeResult RocksDBSstFileReader::getRange(const KeyRange& range) {
rep.push_back_deep(rep.arena(), kv);
expectedSize = expectedSize + kv.expectedSize();
keyCount++;
if (g_network->isSimulated() && deterministicRandom()->random01() < 0.1) {
TraceEvent(SevWarnAlways, "TryGetRangeForBulkLoadInjectError");
throw operation_failed();
}
// if (g_network->isSimulated() && deterministicRandom()->random01() < 0.1) {
// TraceEvent(SevWarnAlways, "TryGetRangeForBulkLoadInjectError");
// throw operation_failed();
//}
if ((byteLimit > 0 && expectedSize >= byteLimit) || (rowLimit > 0 && keyCount >= rowLimit)) {
break;
}
Expand Down
38 changes: 32 additions & 6 deletions fdbserver/storageserver.actor.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -19,12 +19,15 @@
*/

#include <cinttypes>
#include <cstdint>
#include <functional>
#include <iterator>
#include <limits>
#include <memory>
#include <type_traits>
#include <unordered_map>
#include <thread>
#include <chrono>

#include "fdbclient/BlobCipher.h"
#include "fdbclient/BlobGranuleCommon.h"
Expand Down Expand Up @@ -75,6 +78,7 @@
#include "fdbrpc/sim_validation.h"
#include "fdbrpc/Smoother.h"
#include "fdbrpc/Stats.h"
#include "fdbrpc/grpc/AsyncTaskExecutor.h"
#include "fdbserver/AccumulativeChecksumUtil.h"
#include "fdbserver/BulkDumpUtil.actor.h"
#include "fdbserver/BulkLoadUtil.actor.h"
Expand Down Expand Up @@ -1393,6 +1397,8 @@ struct StorageServer : public IStorageMetricsService {
std::string bulkDumpFolder;
std::string bulkLoadFolder;

AsyncTaskExecutor bulkLoadBackgroundTaskPool;

// defined only during splitMutations()/addMutation()
UpdateEagerReadInfo* updateEagerReads;

Expand Down Expand Up @@ -1743,7 +1749,8 @@ struct StorageServer : public IStorageMetricsService {
/*maxTagsTracked=*/SERVER_KNOBS->SS_THROTTLE_TAGS_TRACKED,
/*minRateTracked=*/SERVER_KNOBS->MIN_TAG_READ_PAGES_RATE *
CLIENT_KNOBS->TAG_THROTTLING_PAGE_SIZE),
busiestWriteTagContext(ssi.id()), getEncryptCipherKeysMonitor(encryptionMonitor), counters(this),
bulkLoadBackgroundTaskPool(SERVER_KNOBS->SS_BULKLOAD_BACKGROUND_TASK_COUNT), busiestWriteTagContext(ssi.id()),
getEncryptCipherKeysMonitor(encryptionMonitor), counters(this),
storageServerSourceTLogIDEventHolder(
makeReference<EventCacheHolder>(ssi.id().toString() + "/StorageServerSourceTLogID")),
tenantData(db),
Expand Down Expand Up @@ -8770,18 +8777,37 @@ ACTOR Future<BulkLoadFileSet> bulkLoadFetchKeyValueFileToLoad(StorageServer* dat
return toLocalFileSet;
}

ACTOR Future<Void> tryGetRangeForBulkLoad(PromiseStream<RangeResult> results, KeyRange keys, std::string dataPath) {
ACTOR Future<Void> tryGetRangeForBulkLoad(StorageServer* data,
PromiseStream<RangeResult> results,
KeyRange keys,
std::string dataPath) {
try {
// TODO(BulkLoad): what if the data file is empty but the totalKeyCount is not zero
state Key beginKey = keys.begin;
state Key endKey = keys.end;
state std::unique_ptr<IRocksDBSstFileReader> reader = newRocksDBSstFileReader(
state std::shared_ptr<IRocksDBSstFileReader> reader = newRocksDBSstFileReader(
keys, SERVER_KNOBS->SS_BULKLOAD_GETRANGE_BATCH_SIZE, SERVER_KNOBS->FETCH_BLOCK_BYTES);
// TODO(BulkLoad): this can be a slow task. We will make this as async call.
reader->open(abspath(dataPath));
state RangeResult rep;
loop {
// TODO(BulkLoad): this is a blocking call. We will make this as async call.
RangeResult rep = reader->getRange(KeyRangeRef(beginKey, endKey));
rep.clear();
/*if (SERVER_KNOBS->SS_BULKLOAD_BACKGROUND_TASK_COUNT > 0) {
RangeResult rep_ = wait(data->bulkLoadBackgroundTaskPool.post(
[=]() { return reader->getRange(KeyRangeRef(beginKey, endKey)); }));
rep = rep_;
} else {
rep = reader->getRange(KeyRangeRef(beginKey, endKey));
}*/
wait(data->bulkLoadBackgroundTaskPool.post([=]() {
int64_t s = 0;
for (int64_t i = 0; i < 999999; i++) {
s += i;
}
std::cout << s;
return Void();
}));
rep = reader->getRange(KeyRangeRef(beginKey, endKey));
results.send(rep);
if (!rep.more) {
results.sendError(end_of_stream());
Expand Down Expand Up @@ -9041,7 +9067,7 @@ ACTOR Future<Void> fetchKeys(StorageServer* data, AddingShard* shard) {
// We download the data file to local disk and pass the data file path to read in the next step.
BulkLoadFileSet localFileSet =
wait(bulkLoadFetchKeyValueFileToLoad(data, bulkLoadLocalDir, bulkLoadTaskState));
hold = tryGetRangeForBulkLoad(results, keys, localFileSet.getDataFileFullPath());
hold = tryGetRangeForBulkLoad(data, results, keys, localFileSet.getDataFileFullPath());
rangeEnd = keys.end;
} else {
hold = tryGetRange(results, &tr, keys);
Expand Down