Bulk Loading Framework (#11369)

This commit is contained in:
Zhe Wang 2024-07-23 14:57:28 -07:00 committed by GitHub
parent 92ad7666c3
commit 74990e44bd
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
43 changed files with 3069 additions and 116 deletions

View File

@ -0,0 +1,186 @@
/*
* BulkLoadCommand.actor.cpp
*
* This source file is part of the FoundationDB open source project
*
* Copyright 2013-2024 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 "fdbcli/fdbcli.actor.h"
#include "fdbclient/IClientApi.h"
#include "fdbclient/ManagementAPI.actor.h"
#include "fdbclient/BulkLoading.h"
#include "flow/Arena.h"
#include "flow/FastRef.h"
#include "flow/ThreadHelper.actor.h"
#include "flow/actorcompiler.h" // This must be the last #include.
namespace fdb_cli {
ACTOR Future<Void> getBulkLoadStateByRange(Database cx,
KeyRange rangeToRead,
size_t countLimit,
Optional<BulkLoadPhase> phase) {
try {
std::vector<BulkLoadState> res = wait(getValidBulkLoadTasksWithinRange(cx, rangeToRead, countLimit, phase));
int64_t finishCount = 0;
int64_t unfinishedCount = 0;
for (const auto& bulkLoadState : res) {
if (bulkLoadState.phase == BulkLoadPhase::Complete) {
printf("[Complete]: %s\n", bulkLoadState.toString().c_str());
++finishCount;
} else if (bulkLoadState.phase == BulkLoadPhase::Running) {
printf("[Running]: %s\n", bulkLoadState.toString().c_str());
++unfinishedCount;
} else if (bulkLoadState.phase == BulkLoadPhase::Triggered) {
printf("[Triggered]: %s\n", bulkLoadState.toString().c_str());
++unfinishedCount;
} else if (bulkLoadState.phase == BulkLoadPhase::Submitted) {
printf("[Submitted] %s\n", bulkLoadState.toString().c_str());
++unfinishedCount;
} else if (bulkLoadState.phase == BulkLoadPhase::Acknowledged) {
printf("[Acknowledge] %s\n", bulkLoadState.toString().c_str());
++finishCount;
} else {
UNREACHABLE();
}
}
printf("Finished task count %ld of total %ld tasks\n", finishCount, finishCount + unfinishedCount);
} catch (Error& e) {
if (e.code() == error_code_timed_out) {
printf("timed out\n");
}
}
return Void();
}
ACTOR Future<UID> bulkLoadCommandActor(Reference<IClusterConnectionRecord> clusterFile,
Database cx,
std::vector<StringRef> tokens) {
if (tokencmp(tokens[1], "mode")) {
// Set bulk loading mode
if (tokens.size() != 3) {
printUsage(tokens[0]);
return UID();
}
if (tokencmp(tokens[2], "on")) {
int old = wait(setBulkLoadMode(cx, 1));
TraceEvent("SetBulkLoadModeCommand").detail("OldValue", old).detail("NewValue", 1);
return UID();
} else if (tokencmp(tokens[2], "off")) {
int old = wait(setBulkLoadMode(cx, 0));
TraceEvent("SetBulkLoadModeCommand").detail("OldValue", old).detail("NewValue", 0);
return UID();
} else {
printUsage(tokens[0]);
return UID();
}
} else if (tokencmp(tokens[1], "acknowledge")) {
// Acknowledge any completed bulk loading task and clear the corresponding metadata
if (tokens.size() != 5) {
printUsage(tokens[0]);
return UID();
}
state UID taskId = UID::fromString(tokens[2].toString());
Key rangeBegin = tokens[3];
Key rangeEnd = tokens[4];
if (rangeBegin > normalKeys.end || rangeEnd > normalKeys.end) {
printUsage(tokens[0]);
return UID();
}
KeyRange range = Standalone(KeyRangeRef(rangeBegin, rangeEnd));
wait(acknowledgeBulkLoadTask(cx, range, taskId));
return taskId;
} else if (tokencmp(tokens[1], "local")) {
// Generate spec of bulk loading local files and submit the bulk loading task
if (tokens.size() < 7) {
printUsage(tokens[0]);
return UID();
}
Key rangeBegin = tokens[2];
Key rangeEnd = tokens[3];
// Bulk load can only inject data to normal key space, aka "" ~ \xff
if (rangeBegin > normalKeys.end || rangeEnd > normalKeys.end) {
printUsage(tokens[0]);
return UID();
}
std::string folder = tokens[4].toString();
std::string dataFile = tokens[5].toString();
std::string byteSampleFile = tokens[6].toString(); // TODO(BulkLoad): reject if the input bytes sampling file is
// not same as the configuration as FDB cluster
KeyRange range = Standalone(KeyRangeRef(rangeBegin, rangeEnd));
state BulkLoadState bulkLoadTask = newBulkLoadTaskLocalSST(range, folder, dataFile, byteSampleFile);
wait(submitBulkLoadTask(cx, bulkLoadTask));
return bulkLoadTask.getTaskId();
} else if (tokencmp(tokens[1], "status")) {
// Get progress of existing bulk loading tasks intersecting the input range
// TODO(BulkLoad): check status by ID
if (tokens.size() < 6) {
printUsage(tokens[0]);
return UID();
}
Key rangeBegin = tokens[2];
Key rangeEnd = tokens[3];
// Bulk load can only inject data to normal key space, aka "" ~ \xff
if (rangeBegin > normalKeys.end || rangeEnd > normalKeys.end) {
printUsage(tokens[0]);
return UID();
}
KeyRange range = Standalone(KeyRangeRef(rangeBegin, rangeEnd));
std::string inputPhase = tokens[4].toString();
Optional<BulkLoadPhase> phase;
if (inputPhase == "all") {
phase = Optional<BulkLoadPhase>();
} else if (inputPhase == "submitted") {
phase = BulkLoadPhase::Submitted;
} else if (inputPhase == "triggered") {
phase = BulkLoadPhase::Triggered;
} else if (inputPhase == "running") {
phase = BulkLoadPhase::Running;
} else if (inputPhase == "complete") {
phase = BulkLoadPhase::Complete;
} else if (inputPhase == "acknowledged") {
phase = BulkLoadPhase::Acknowledged;
} else {
printUsage(tokens[0]);
return UID();
}
int countLimit = std::stoi(tokens[5].toString());
wait(getBulkLoadStateByRange(cx, range, countLimit, phase));
return UID();
} else {
printUsage(tokens[0]);
return UID();
}
}
CommandFactory bulkLoadFactory(
"bulkload",
CommandHelp("bulkload [mode|acknowledge|local|status] [ARGs]",
"bulkload commands",
"To set bulkLoad mode: `bulkload mode [on|off]'\n"
"To acknowledge completed tasks within a range: `bulkload acknowledge <TaskID> <BeginKey> <EndKey>'\n"
"To trigger a task injecting a SST file from local file system: `bulkload local <BeginKey> <EndKey> "
"<Folder> <DataFile> <ByteSampleFile>'\n"
"To get progress of tasks within a range: `bulkload status <BeginKey> <EndKey> "
"[all|submitted|triggered|running|complete] <limit>'\n"));
} // namespace fdb_cli

View File

@ -1714,6 +1714,14 @@ ACTOR Future<int> cli(CLIOptions opt, LineNoise* plinenoise, Reference<ClusterCo
continue;
}
if (tokencmp(tokens[0], "bulkload")) {
UID taskId = wait(makeInterruptable(bulkLoadCommandActor(ccf, localDb, tokens)));
if (taskId.isValid()) {
printf("Received bulkload task: %s\n", taskId.toString().c_str());
}
continue;
}
if (tokencmp(tokens[0], "force_recovery_with_data_loss")) {
bool _result = wait(makeInterruptable(forceRecoveryWithDataLossCommandActor(db, tokens)));
if (!_result)

View File

@ -204,7 +204,12 @@ ACTOR Future<UID> auditStorageCommandActor(Reference<IClusterConnectionRecord> c
std::vector<StringRef> tokens);
// Retrieve audit storage status
ACTOR Future<bool> getAuditStatusCommandActor(Database cx, std::vector<StringRef> tokens);
// Retrieve shard information command
ACTOR Future<bool> locationMetadataCommandActor(Database cx, std::vector<StringRef> tokens);
// Bulk loading command
ACTOR Future<UID> bulkLoadCommandActor(Reference<IClusterConnectionRecord> clusterFile,
Database cx,
std::vector<StringRef> tokens);
// force_recovery_with_data_loss command
ACTOR Future<bool> forceRecoveryWithDataLossCommandActor(Reference<IDatabase> db, std::vector<StringRef> tokens);
// include command

36
fdbclient/BulkLoading.cpp Normal file
View File

@ -0,0 +1,36 @@
/*
* BulkLoading.cpp
*
* This source file is part of the FoundationDB open source project
*
* Copyright 2013-2024 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/BulkLoading.h"
BulkLoadState newBulkLoadTaskLocalSST(KeyRange range,
std::string folder,
std::string dataFile,
std::string bytesSampleFile) {
std::unordered_set<std::string> dataFiles;
dataFiles.insert(dataFile);
return BulkLoadState(range,
BulkLoadType::SST,
BulkLoadTransportMethod::CP,
BulkLoadInjectMethod::File,
folder,
dataFiles,
bytesSampleFile);
}

View File

@ -2792,6 +2792,173 @@ ACTOR Future<UID> cancelAuditStorage(Reference<IClusterConnectionRecord> cluster
return auditId;
}
ACTOR Future<int> setBulkLoadMode(Database cx, int mode) {
state Transaction tr(cx);
state BinaryWriter wr(Unversioned());
wr << mode;
loop {
try {
tr.setOption(FDBTransactionOptions::LOCK_AWARE);
tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE);
state int oldMode = 0;
Optional<Value> oldModeValue = wait(tr.get(bulkLoadModeKey));
if (oldModeValue.present()) {
BinaryReader rd(oldModeValue.get(), Unversioned());
rd >> oldMode;
}
if (oldMode != mode) {
BinaryWriter wrMyOwner(Unversioned());
wrMyOwner << dataDistributionModeLock;
tr.set(moveKeysLockOwnerKey, wrMyOwner.toValue());
BinaryWriter wrLastWrite(Unversioned());
wrLastWrite << deterministicRandom()->randomUniqueID(); // triger DD restarts
tr.set(moveKeysLockWriteKey, wrLastWrite.toValue());
tr.set(bulkLoadModeKey, wr.toValue());
wait(tr.commit());
TraceEvent("DDBulkLoadModeKeyChanged").detail("NewMode", mode).detail("OldMode", oldMode);
}
return oldMode;
} catch (Error& e) {
wait(tr.onError(e));
}
}
}
ACTOR Future<std::vector<BulkLoadState>> getValidBulkLoadTasksWithinRange(
Database cx,
KeyRange rangeToRead,
size_t limit = 10,
Optional<BulkLoadPhase> phase = Optional<BulkLoadPhase>()) {
state Transaction tr(cx);
state Key readBegin = rangeToRead.begin;
state Key readEnd = rangeToRead.end;
state RangeResult rangeResult;
state std::vector<BulkLoadState> res;
while (readBegin < readEnd) {
state int retryCount = 0;
loop {
try {
rangeResult.clear();
tr.setOption(FDBTransactionOptions::READ_SYSTEM_KEYS);
tr.setOption(FDBTransactionOptions::LOCK_AWARE);
wait(store(rangeResult,
krmGetRanges(&tr,
bulkLoadPrefix,
KeyRangeRef(readBegin, readEnd),
CLIENT_KNOBS->KRM_GET_RANGE_LIMIT,
CLIENT_KNOBS->KRM_GET_RANGE_LIMIT_BYTES)));
break;
} catch (Error& e) {
if (retryCount > 30) {
throw timed_out();
}
wait(tr.onError(e));
retryCount++;
}
}
for (int i = 0; i < rangeResult.size() - 1; ++i) {
if (rangeResult[i].value.empty()) {
continue;
}
BulkLoadState bulkLoadState = decodeBulkLoadState(rangeResult[i].value);
KeyRange range = Standalone(KeyRangeRef(rangeResult[i].key, rangeResult[i + 1].key));
if (range != bulkLoadState.getRange()) {
ASSERT(bulkLoadState.getRange().contains(range));
continue;
}
if (!phase.present() || phase.get() == bulkLoadState.phase) {
res.push_back(bulkLoadState);
}
if (res.size() >= limit) {
return res;
}
}
readBegin = rangeResult.back().key;
}
return res;
}
// Submit bulkload task and overwrite any existing task
ACTOR Future<Void> submitBulkLoadTask(Database cx, BulkLoadState bulkLoadTask) {
loop {
state Transaction tr(cx);
try {
tr.setOption(FDBTransactionOptions::LOCK_AWARE);
tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
if (bulkLoadTask.phase != BulkLoadPhase::Submitted) {
TraceEvent(g_network->isSimulated() ? SevError : SevWarnAlways, "SubmitBulkLoadTaskError")
.setMaxEventLength(-1)
.setMaxFieldLength(-1)
.detail("Task", bulkLoadTask.toString());
throw bulkload_task_failed();
}
bulkLoadTask.submitTime = now();
wait(krmSetRange(&tr, bulkLoadPrefix, bulkLoadTask.getRange(), bulkLoadStateValue(bulkLoadTask)));
wait(tr.commit());
break;
} catch (Error& e) {
wait(tr.onError(e));
}
}
return Void();
}
// Get bulk load task metadata with range and taskId and phase selector
// Throw error if the task is outdated or the task is not in any input phase at the tr read version
ACTOR Future<BulkLoadState> getBulkLoadTask(Transaction* tr,
KeyRange range,
UID taskId,
std::vector<BulkLoadPhase> phases) {
state BulkLoadState bulkLoadState;
RangeResult result = wait(krmGetRanges(tr, bulkLoadPrefix, range));
if (result.size() > 2) {
throw bulkload_task_outdated();
} else if (result[0].value.empty()) {
throw bulkload_task_outdated();
}
ASSERT(result.size() == 2);
bulkLoadState = decodeBulkLoadState(result[0].value);
ASSERT(bulkLoadState.getTaskId().isValid());
if (taskId != bulkLoadState.getTaskId()) {
// This task is overwritten by a newer task
throw bulkload_task_outdated();
}
KeyRange currentRange = KeyRangeRef(result[0].key, result[1].key);
if (bulkLoadState.getRange() != currentRange) {
// This task is partially overwritten by a newer task
ASSERT(bulkLoadState.getRange().contains(currentRange));
throw bulkload_task_outdated();
}
if (phases.size() > 0 && !bulkLoadState.onAnyPhase(phases)) {
throw bulkload_task_outdated();
}
return bulkLoadState;
}
// Update bulkload task to acknowledge state
ACTOR Future<Void> acknowledgeBulkLoadTask(Database cx, KeyRange range, UID taskId) {
loop {
state Transaction tr(cx);
state BulkLoadState bulkLoadState;
try {
tr.setOption(FDBTransactionOptions::LOCK_AWARE);
tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
wait(store(bulkLoadState,
getBulkLoadTask(&tr, range, taskId, { BulkLoadPhase::Complete, BulkLoadPhase::Acknowledged })));
bulkLoadState.phase = BulkLoadPhase::Acknowledged;
ASSERT(range == bulkLoadState.getRange() && taskId == bulkLoadState.getTaskId());
wait(krmSetRange(&tr, bulkLoadPrefix, bulkLoadState.getRange(), bulkLoadStateValue(bulkLoadState)));
wait(tr.commit());
break;
} catch (Error& e) {
wait(tr.onError(e));
}
}
return Void();
}
ACTOR Future<Void> waitForPrimaryDC(Database cx, StringRef dcId) {
state ReadYourWritesTransaction tr(cx);

View File

@ -366,6 +366,13 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi
init( DD_MAXIMUM_LARGE_TEAM_CLEANUP, 10000 ); if( randomize && BUGGIFY ) DD_MAXIMUM_LARGE_TEAM_CLEANUP = 10;
init( DD_LARGE_TEAM_DELAY, 60.0 );
init( DD_FIX_WRONG_REPLICAS_DELAY, 60.0 );
// BulkLoading
init( BULKLOAD_FILE_BYTES_MAX, 1*1024*1024*1024 ); // 1GB
init( DD_BULKLOAD_SHARD_BOUNDARY_CHANGE_DELAY_SEC, 10.0 ); if( randomize && BUGGIFY ) DD_BULKLOAD_SHARD_BOUNDARY_CHANGE_DELAY_SEC = deterministicRandom()->random01() * 10 + 1;
init( DD_BULKLOAD_TASK_METADATA_READ_SIZE, 100 ); if( randomize && BUGGIFY ) DD_BULKLOAD_TASK_METADATA_READ_SIZE = deterministicRandom()->randomInt(2, 100);
init( DD_BULKLOAD_PARALLELISM, 10 ); if( randomize && BUGGIFY ) DD_BULKLOAD_PARALLELISM = deterministicRandom()->randomInt(1, 10);
init( DD_BULKLOAD_SCHEDULE_MIN_INTERVAL_SEC, 2.0 ); if( randomize && BUGGIFY ) DD_BULKLOAD_SCHEDULE_MIN_INTERVAL_SEC = deterministicRandom()->random01() * 10 + 1;
// TeamRemover
init( TR_FLAG_DISABLE_MACHINE_TEAM_REMOVER, false ); if( randomize && BUGGIFY ) TR_FLAG_DISABLE_MACHINE_TEAM_REMOVER = deterministicRandom()->random01() < 0.1 ? true : false; // false by default. disable the consistency check when it's true

View File

@ -1224,6 +1224,22 @@ const KeyRef moveKeysLockWriteKey = "\xff/moveKeysLock/Write"_sr;
const KeyRef dataDistributionModeKey = "\xff/dataDistributionMode"_sr;
const UID dataDistributionModeLock = UID(6345, 3425);
// Bulk loading keys
const KeyRef bulkLoadModeKey = "\xff/bulkLoadMode"_sr;
const KeyRangeRef bulkLoadKeys = KeyRangeRef("\xff/bulkLoad/"_sr, "\xff/bulkLoad0"_sr);
const KeyRef bulkLoadPrefix = bulkLoadKeys.begin;
const Value bulkLoadStateValue(const BulkLoadState& bulkLoadState) {
return ObjectWriter::toValue(bulkLoadState, IncludeVersion());
}
BulkLoadState decodeBulkLoadState(const ValueRef& value) {
BulkLoadState bulkLoadState;
ObjectReader reader(value.begin(), IncludeVersion());
reader.deserialize(bulkLoadState);
return bulkLoadState;
}
// Keys to view and control tag throttling
const KeyRangeRef tagThrottleKeys = KeyRangeRef("\xff\x02/throttledTags/tag/"_sr, "\xff\x02/throttledTags/tag0"_sr);
const KeyRef tagThrottleKeysPrefix = tagThrottleKeys.begin;

View File

@ -0,0 +1,222 @@
/*
* BulkLoading.h
*
* This source file is part of the FoundationDB open source project
*
* Copyright 2013-2024 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 FDBCLIENT_BULKLOADING_H
#define FDBCLIENT_BULKLOADING_H
#pragma once
#include "fdbclient/FDBTypes.h"
#include "fdbrpc/fdbrpc.h"
enum class BulkLoadPhase : uint8_t {
Invalid = 0, // Used to distinguish if a BulkLoadState is a valid task
Submitted = 1, // Set by users
Triggered = 2, // Update when DD trigger a data move for the task
Running = 3, // Update atomically with updating KeyServer dest servers in startMoveKey
Complete = 4, // Update atomically with updating KeyServer src servers in finishMoveKey
Acknowledged = 5, // Updated by users; DD automatically clear metadata with this phase
};
enum class BulkLoadType : uint8_t {
Invalid = 0,
SST = 1,
};
enum class BulkLoadTransportMethod : uint8_t {
Invalid = 0,
CP = 1, // Local file copy. Used when the data file is in the local file system for any storage server. Used for
// simulation test and local cluster test.
};
enum class BulkLoadInjectMethod : uint8_t {
Invalid = 0,
File = 1,
};
struct BulkLoadState {
constexpr static FileIdentifier file_identifier = 1384499;
BulkLoadState() = default;
// for acknowledging a completed task, where only taskId and range are used
BulkLoadState(UID taskId, KeyRange range) : taskId(taskId), range(range), phase(BulkLoadPhase::Invalid) {}
// for submitting a task
BulkLoadState(KeyRange range,
BulkLoadType loadType,
BulkLoadTransportMethod transportMethod,
BulkLoadInjectMethod injectMethod,
std::string folder,
std::unordered_set<std::string> dataFiles,
Optional<std::string> bytesSampleFile)
: taskId(deterministicRandom()->randomUniqueID()), range(range), loadType(loadType),
transportMethod(transportMethod), injectMethod(injectMethod), folder(folder), dataFiles(dataFiles),
bytesSampleFile(bytesSampleFile), phase(BulkLoadPhase::Submitted) {
ASSERT(isValid());
}
bool operator==(const BulkLoadState& rhs) const {
return taskId == rhs.taskId && range == rhs.range && dataFiles == rhs.dataFiles;
}
std::string toString() const {
std::string res =
"BulkLoadState: [Range]: " + Traceable<KeyRangeRef>::toString(range) +
", [Type]: " + std::to_string(static_cast<uint8_t>(loadType)) +
", [TransportMethod]: " + std::to_string(static_cast<uint8_t>(transportMethod)) +
", [InjectMethod]: " + std::to_string(static_cast<uint8_t>(injectMethod)) +
", [Phase]: " + std::to_string(static_cast<uint8_t>(phase)) + ", [Folder]: " + folder +
", [DataFiles]: " + describe(dataFiles) + ", [SubmitTime]: " + std::to_string(submitTime) +
", [TriggerTime]: " + std::to_string(triggerTime) + ", [StartTime]: " + std::to_string(startTime) +
", [CompleteTime]: " + std::to_string(completeTime) + ", [RestartCount]: " + std::to_string(restartCount);
if (bytesSampleFile.present()) {
res = res + ", [ByteSampleFile]: " + bytesSampleFile.get();
}
if (dataMoveId.present()) {
res = res + ", [DataMoveId]: " + dataMoveId.get().toString();
}
res = res + ", [TaskId]: " + taskId.toString();
return res;
}
KeyRange getRange() const { return range; }
UID getTaskId() const { return taskId; }
std::string getFolder() const { return folder; }
BulkLoadTransportMethod getTransportMethod() const { return transportMethod; }
std::unordered_set<std::string> getDataFiles() const { return dataFiles; }
Optional<std::string> getBytesSampleFile() const { return bytesSampleFile; }
bool onAnyPhase(const std::vector<BulkLoadPhase>& inputPhases) const {
for (const auto& inputPhase : inputPhases) {
if (inputPhase == phase) {
return true;
}
}
return false;
}
void setDataMoveId(UID id) {
if (dataMoveId.present() && dataMoveId.get() != id) {
TraceEvent(SevWarn, "DDBulkLoadTaskUpdateDataMoveId")
.detail("NewId", id)
.detail("BulkLoadTask", this->toString());
}
dataMoveId = id;
}
inline Optional<UID> getDataMoveId() const { return dataMoveId; }
inline void clearDataMoveId() { dataMoveId.reset(); }
bool isValid() const {
if (!taskId.isValid()) {
return false;
}
if (range.empty()) {
return false;
}
if (transportMethod == BulkLoadTransportMethod::Invalid) {
return false;
} else if (transportMethod != BulkLoadTransportMethod::CP) {
throw not_implemented();
}
if (injectMethod == BulkLoadInjectMethod::Invalid) {
return false;
} else if (injectMethod != BulkLoadInjectMethod::File) {
throw not_implemented();
}
if (dataFiles.empty()) {
return false;
}
for (const auto& filePath : dataFiles) {
if (filePath.substr(0, folder.size()) != folder) {
return false;
}
}
if (bytesSampleFile.present()) {
if (bytesSampleFile.get().substr(0, folder.size()) != folder) {
return false;
}
}
// TODO(BulkLoad): do some validation between methods and files
return true;
}
template <class Ar>
void serialize(Ar& ar) {
serializer(ar,
range,
loadType,
transportMethod,
injectMethod,
phase,
folder,
dataFiles,
bytesSampleFile,
dataMoveId,
taskId,
submitTime,
triggerTime,
startTime,
completeTime,
restartCount);
}
// Updated by DD
BulkLoadPhase phase;
double submitTime = 0;
double triggerTime = 0;
double startTime = 0;
double completeTime = 0;
int restartCount = -1;
private:
// Set by user
UID taskId; // Unique ID of the task
KeyRange range; // Load the key-value within this range "[begin, end)" from data file
// File inject config
BulkLoadType loadType;
BulkLoadTransportMethod transportMethod;
BulkLoadInjectMethod injectMethod;
// Folder includes all files to be injected
std::string folder;
// Files to inject
std::unordered_set<std::string> dataFiles;
Optional<std::string> bytesSampleFile;
// bytesSampleFile is Optional. If bytesSampleFile is not provided, storage server will go through all keys and
// conduct byte sampling, which will slow down the bulk loading rate.
// TODO(BulkLoad): add file checksum
// Set by DD
Optional<UID> dataMoveId;
};
BulkLoadState newBulkLoadTaskLocalSST(KeyRange range,
std::string folder,
std::string dataFile,
std::string bytesSampleFile);
#endif

View File

@ -168,6 +168,27 @@ ACTOR Future<UID> cancelAuditStorage(Reference<IClusterConnectionRecord> cluster
UID auditId,
double timeoutSeconds);
// Set bulk load mode
ACTOR Future<int> setBulkLoadMode(Database cx, int mode);
// Get valid bulk load task state within the input range
ACTOR Future<std::vector<BulkLoadState>> getValidBulkLoadTasksWithinRange(Database cx,
KeyRange rangeToRead,
size_t limit,
Optional<BulkLoadPhase> phase);
// Submit a bulk load task
ACTOR Future<Void> submitBulkLoadTask(Database cx, BulkLoadState bulkLoadTask);
// Acknowledge a bulk load task if it has been completed
ACTOR Future<Void> acknowledgeBulkLoadTask(Database cx, KeyRange range, UID taskId);
// Get bulk load task for the input range and taskId
ACTOR Future<BulkLoadState> getBulkLoadTask(Transaction* tr,
KeyRange range,
UID taskId,
std::vector<BulkLoadPhase> phases);
ACTOR Future<Void> printHealthyZone(Database cx);
ACTOR Future<bool> clearHealthyZone(Database cx, bool printWarning = false, bool clearSSFailureZoneString = false);
ACTOR Future<bool> setHealthyZone(Database cx, StringRef zoneId, double seconds, bool printWarning = false);

View File

@ -379,6 +379,13 @@ public:
// requested
double DD_FIX_WRONG_REPLICAS_DELAY; // the amount of time between attempts to increase the replication factor of
// under replicated shards
int BULKLOAD_FILE_BYTES_MAX; // the maximum bytes of files to inject by bulk loading
double DD_BULKLOAD_SHARD_BOUNDARY_CHANGE_DELAY_SEC; // seconds to delay shard boundary change when blocked by bulk
// loading
int DD_BULKLOAD_TASK_METADATA_READ_SIZE; // the number of bulk load tasks read from metadata at a time
int DD_BULKLOAD_PARALLELISM; // the maximum number of running bulk load tasks
double DD_BULKLOAD_SCHEDULE_MIN_INTERVAL_SEC; // the minimal seconds that the bulk load scheduler has to wait
// between two rounds
// Run storage engine on a child process on the same machine with storage process
bool REMOTE_KV_STORE;

View File

@ -22,6 +22,7 @@
#define FDBCLIENT_STORAGCHECKPOINT_H
#pragma once
#include "fdbclient/BulkLoading.h"
#include "fdbclient/FDBTypes.h"
const std::string checkpointBytesSampleFileName = "metadata_bytes.sst";
@ -173,6 +174,7 @@ struct DataMoveMetaData {
std::set<UID> checkpoints;
int16_t phase; // DataMoveMetaData::Phase.
int8_t mode;
Optional<BulkLoadState> bulkLoadState; // set if the data move is a bulk load data move
DataMoveMetaData() = default;
DataMoveMetaData(UID id, Version version, KeyRange range) : id(id), version(version), priority(0), mode(0) {
@ -192,12 +194,15 @@ struct DataMoveMetaData {
", [Phase]: " + std::to_string(static_cast<int>(phase)) +
", [Source Servers]: " + describe(src) + ", [Destination Servers]: " + describe(dest) +
", [Checkpoints]: " + describe(checkpoints);
if (bulkLoadState.present()) {
res = res + ", [BulkLoadState]: " + bulkLoadState.get().toString();
}
return res;
}
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, id, version, ranges, priority, src, dest, checkpoints, phase, mode);
serializer(ar, id, version, ranges, priority, src, dest, checkpoints, phase, mode, bulkLoadState);
}
};

View File

@ -25,6 +25,7 @@
// Functions and constants documenting the organization of the reserved keyspace in the database beginning with "\xFF"
#include "fdbclient/AccumulativeChecksum.h"
#include "fdbclient/BulkLoading.h"
#include "fdbclient/FDBTypes.h"
#include "fdbclient/BlobWorkerInterface.h" // TODO move the functions that depend on this out of here and into BlobWorkerInterface.h to remove this dependency
#include "fdbclient/StorageServerInterface.h"
@ -42,7 +43,9 @@ enum class DataMoveType : uint8_t {
LOGICAL = 0,
PHYSICAL = 1,
PHYSICAL_EXP = 2,
NUMBER_OF_TYPES = 3,
LOGICAL_BULKLOAD = 3,
PHYSICAL_BULKLOAD = 4,
NUMBER_OF_TYPES = 5,
};
// One-to-one relationship to the priority knobs
@ -518,6 +521,12 @@ extern const KeyRef moveKeysLockOwnerKey, moveKeysLockWriteKey;
extern const KeyRef dataDistributionModeKey;
extern const UID dataDistributionModeLock;
extern const KeyRef bulkLoadModeKey;
extern const KeyRangeRef bulkLoadKeys;
extern const KeyRef bulkLoadPrefix;
const Value bulkLoadStateValue(const BulkLoadState& bulkLoadState);
BulkLoadState decodeBulkLoadState(const ValueRef& value);
// Keys to view and control tag throttling
extern const KeyRangeRef tagThrottleKeys;
extern const KeyRef tagThrottleKeysPrefix;

View File

@ -0,0 +1,174 @@
/*
* BulkLoadUtils.actor.cpp
*
* This source file is part of the FoundationDB open source project
*
* Copyright 2013-2024 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/FDBTypes.h"
#include "fdbclient/NativeAPI.actor.h"
#include "fdbclient/ClientKnobs.h"
#include "fdbserver/BulkLoadUtil.actor.h"
#include "fdbserver/RocksDBCheckpointUtils.actor.h"
#include "fdbserver/StorageMetrics.actor.h"
#include <fmt/format.h>
#include "flow/actorcompiler.h" // has to be last include
std::string generateRandomBulkLoadDataFileName() {
return deterministicRandom()->randomUniqueID().toString() + "-data.sst";
}
std::string generateRandomBulkLoadBytesSampleFileName() {
return deterministicRandom()->randomUniqueID().toString() + "-bytesample.sst";
}
ACTOR Future<Optional<BulkLoadState>> getBulkLoadStateFromDataMove(Database cx, UID dataMoveId, UID logId) {
loop {
state Transaction tr(cx);
try {
Optional<Value> val = wait(tr.get(dataMoveKeyFor(dataMoveId)));
if (!val.present()) {
TraceEvent(SevWarn, "SSBulkLoadDataMoveIdNotExist", logId).detail("DataMoveID", dataMoveId);
return Optional<BulkLoadState>();
}
DataMoveMetaData dataMoveMetaData = decodeDataMoveValue(val.get());
return dataMoveMetaData.bulkLoadState;
} catch (Error& e) {
wait(tr.onError(e));
}
}
}
void bulkLoadFileCopy(std::string fromFile, std::string toFile, size_t fileBytesMax) {
std::string content = readFileBytes(fromFile, fileBytesMax);
writeFile(toFile, content);
// TODO(BulkLoad): Do file checksum for toFile
return;
}
ACTOR Future<SSBulkLoadFileSet> bulkLoadTransportCP_impl(std::string dir,
BulkLoadState bulkLoadState,
size_t fileBytesMax,
UID logId) {
ASSERT(bulkLoadState.getTransportMethod() == BulkLoadTransportMethod::CP);
loop {
state std::string toFile;
state std::string fromFile;
state SSBulkLoadFileSet fileSet;
try {
fileSet.folder = abspath(joinPath(dir, bulkLoadState.getFolder()));
// Clear existing folder
platform::eraseDirectoryRecursive(fileSet.folder);
if (!platform::createDirectory(fileSet.folder)) {
throw retry();
}
// Move bulk load files to loading folder
for (const auto& filePath : bulkLoadState.getDataFiles()) {
fromFile = abspath(filePath);
toFile = abspath(joinPath(fileSet.folder, generateRandomBulkLoadDataFileName()));
if (fileSet.dataFileList.find(toFile) != fileSet.dataFileList.end()) {
ASSERT_WE_THINK(false);
throw retry();
}
bulkLoadFileCopy(fromFile, toFile, fileBytesMax);
fileSet.dataFileList.insert(toFile);
TraceEvent(SevInfo, "SSBulkLoadSSTFileCopied", logId)
.detail("BulkLoadTask", bulkLoadState.toString())
.detail("FromFile", fromFile)
.detail("ToFile", toFile);
}
if (bulkLoadState.getBytesSampleFile().present()) {
fromFile = abspath(bulkLoadState.getBytesSampleFile().get());
if (fileExists(fromFile)) {
toFile = abspath(joinPath(fileSet.folder, generateRandomBulkLoadBytesSampleFileName()));
bulkLoadFileCopy(fromFile, toFile, fileBytesMax);
fileSet.bytesSampleFile = toFile;
TraceEvent(SevInfo, "SSBulkLoadSSTFileCopied", logId)
.detail("BulkLoadTask", bulkLoadState.toString())
.detail("FromFile", fromFile)
.detail("ToFile", toFile);
}
}
return fileSet;
} catch (Error& e) {
if (e.code() == error_code_actor_cancelled) {
throw e;
}
TraceEvent(SevInfo, "SSBulkLoadTaskFetchSSTFileCopyError", logId)
.errorUnsuppressed(e)
.detail("BulkLoadTask", bulkLoadState.toString())
.detail("FromFile", fromFile)
.detail("ToFile", toFile);
wait(delay(5.0));
}
}
}
ACTOR Future<Optional<std::string>> getBytesSamplingFromSSTFiles(std::string folderToGenerate,
std::unordered_set<std::string> dataFiles,
UID logId) {
loop {
try {
std::string bytesSampleFile =
abspath(joinPath(folderToGenerate, generateRandomBulkLoadBytesSampleFileName()));
std::unique_ptr<IRocksDBSstFileWriter> sstWriter = newRocksDBSstFileWriter();
sstWriter->open(bytesSampleFile);
bool anySampled = false;
for (const auto& filePath : dataFiles) {
std::unique_ptr<IRocksDBSstFileReader> reader = newRocksDBSstFileReader();
reader->open(filePath);
while (reader->hasNext()) {
KeyValue kv = reader->next();
ByteSampleInfo sampleInfo = isKeyValueInSample(kv);
if (sampleInfo.inSample) {
sstWriter->write(kv.key, kv.value); // TODO(BulkLoad): validate if kvs are sorted
anySampled = true;
}
}
}
// It is possible that no key is sampled
// This can happen when the data to sample is small
// In this case, no SST sample byte file is generated
if (anySampled) {
ASSERT(sstWriter->finish());
return bytesSampleFile;
} else {
return Optional<std::string>();
}
} catch (Error& e) {
if (e.code() == error_code_actor_cancelled) {
throw e;
}
TraceEvent(SevWarn, "SSBulkLoadTaskSamplingError", logId).errorUnsuppressed(e);
wait(delay(5.0));
}
}
}
void checkContent(std::unordered_set<std::string> dataFiles, UID logId) {
for (const auto& filePath : dataFiles) {
std::unique_ptr<IRocksDBSstFileReader> reader = newRocksDBSstFileReader();
reader->open(filePath);
while (reader->hasNext()) {
KeyValue kv = reader->next();
TraceEvent("CheckContent", logId).detail("Key", kv.key).detail("Value", kv.value);
}
}
return;
}

View File

@ -2267,6 +2267,12 @@ ACTOR Future<Void> monitorDataDistributor(ClusterControllerData* self) {
}
loop {
bool ddExist = self->db.serverInfo->get().distributor.present();
TraceEvent(SevInfo, "CCMonitorDataDistributor", self->id)
.detail("Recruiting", self->recruitDistributor.get())
.detail("Existing", ddExist)
.detail("ExistingDD", ddExist ? self->db.serverInfo->get().distributor.get().id().toString() : "");
if (self->db.serverInfo->get().distributor.present() && !self->recruitDistributor.get()) {
choose {
when(wait(waitFailureClient(self->db.serverInfo->get().distributor.get().waitFailure,

View File

@ -424,8 +424,10 @@ std::string Busyness::toString() {
// find the "workFactor" for this, were it launched now
int getSrcWorkFactor(RelocateData const& relocation, int singleRegionTeamSize) {
if (relocation.healthPriority == SERVER_KNOBS->PRIORITY_TEAM_1_LEFT ||
relocation.healthPriority == SERVER_KNOBS->PRIORITY_TEAM_0_LEFT)
if (relocation.bulkLoadTask.present())
return 0;
else if (relocation.healthPriority == SERVER_KNOBS->PRIORITY_TEAM_1_LEFT ||
relocation.healthPriority == SERVER_KNOBS->PRIORITY_TEAM_0_LEFT)
return WORK_FULL_UTILIZATION / SERVER_KNOBS->RELOCATION_PARALLELISM_PER_SOURCE_SERVER;
else if (relocation.healthPriority == SERVER_KNOBS->PRIORITY_TEAM_2_LEFT)
return WORK_FULL_UTILIZATION / 2 / SERVER_KNOBS->RELOCATION_PARALLELISM_PER_SOURCE_SERVER;
@ -463,6 +465,10 @@ bool canLaunchSrc(RelocateData& relocation,
// Blob migrator is backed by s3 so it can allow unlimited data movements
if (relocation.src.size() == 1 && BlobMigratorInterface::isBlobMigrator(relocation.src.back())) {
return true;
} else if (relocation.bulkLoadTask.present()) {
// workFactor for bulk load task on source is always 0, therefore, we can safely launch
// the data move with a bulk load task
return true;
}
// find the "workFactor" for this, were it launched now
@ -539,7 +545,7 @@ void completeDest(RelocateData const& relocation, std::map<UID, Busyness>& destB
}
void complete(RelocateData const& relocation, std::map<UID, Busyness>& busymap, std::map<UID, Busyness>& destBusymap) {
ASSERT(relocation.workFactor > 0);
ASSERT(relocation.bulkLoadTask.present() || relocation.workFactor > 0);
for (int i = 0; i < relocation.src.size(); i++)
busymap[relocation.src[i]].removeWork(relocation.priority, relocation.workFactor);
@ -579,7 +585,8 @@ ACTOR Future<Void> getSourceServersForRange(DDQueue* self,
DDQueue::DDQueue(DDQueueInitParams const& params)
: IDDRelocationQueue(), distributorId(params.id), lock(params.lock), txnProcessor(params.db),
teamCollections(params.teamCollections), shardsAffectedByTeamFailure(params.shardsAffectedByTeamFailure),
physicalShardCollection(params.physicalShardCollection), getAverageShardBytes(params.getAverageShardBytes),
physicalShardCollection(params.physicalShardCollection), bulkLoadTaskCollection(params.bulkLoadTaskCollection),
getAverageShardBytes(params.getAverageShardBytes),
startMoveKeysParallelismLock(SERVER_KNOBS->DD_MOVE_KEYS_PARALLELISM),
finishMoveKeysParallelismLock(SERVER_KNOBS->DD_MOVE_KEYS_PARALLELISM),
cleanUpDataMoveParallelismLock(SERVER_KNOBS->DD_MOVE_KEYS_PARALLELISM),
@ -988,7 +995,7 @@ void DDQueue::launchQueuedWork(RelocateData launchData, const DDEnabledState* dd
launchQueuedWork(combined, ddEnabledState);
}
DataMoveType newDataMoveType() {
DataMoveType newDataMoveType(bool doBulkLoading) {
DataMoveType type = DataMoveType::LOGICAL;
if (deterministicRandom()->random01() < SERVER_KNOBS->DD_PHYSICAL_SHARD_MOVE_PROBABILITY) {
type = DataMoveType::PHYSICAL;
@ -996,10 +1003,47 @@ DataMoveType newDataMoveType() {
if (type != DataMoveType::PHYSICAL && SERVER_KNOBS->ENABLE_PHYSICAL_SHARD_MOVE_EXPERIMENT) {
type = DataMoveType::PHYSICAL_EXP;
}
if (doBulkLoading) {
if (type == DataMoveType::LOGICAL) {
type = DataMoveType::LOGICAL_BULKLOAD;
} else if (type == DataMoveType::PHYSICAL || type == DataMoveType::PHYSICAL_EXP) {
type = DataMoveType::PHYSICAL_BULKLOAD;
} else {
UNREACHABLE();
}
}
return type;
}
bool runPendingBulkLoadTaskWithRelocateData(DDQueue* self, RelocateData& rd) {
bool doBulkLoading = false;
Optional<DDBulkLoadTask> task = self->bulkLoadTaskCollection->getTaskByRange(rd.keys);
if (task.present() && task.get().coreState.onAnyPhase({ BulkLoadPhase::Triggered, BulkLoadPhase::Running })) {
rd.bulkLoadTask = task.get();
doBulkLoading = true;
}
if (doBulkLoading) {
try {
self->bulkLoadTaskCollection->startTask(rd.bulkLoadTask.get().coreState);
} catch (Error& e) {
ASSERT_WE_THINK(e.code() == error_code_bulkload_task_outdated);
if (e.code() == error_code_bulkload_task_outdated) {
TraceEvent(g_network->isSimulated() ? SevError : SevWarnAlways,
"DDBulkLoadTaskOutdatedWhenStartRelocator",
self->distributorId) // unexpected
.detail("NewDataMoveID", rd.dataMoveId)
.detail("NewDataMovePriority", rd.priority)
.detail("NewDataMoveRange", rd.keys)
.detail("BulkLoadTask", rd.bulkLoadTask.get().toString());
throw movekeys_conflict();
} else {
throw e;
}
}
}
return doBulkLoading;
}
// For each relocateData rd in the queue, check if there exist inflight relocate data whose keyrange is overlapped
// with rd. If there exist, cancel them by cancelling their actors and reducing the src servers' busyness of those
// canceled inflight relocateData. Launch the relocation for the rd.
@ -1012,6 +1056,18 @@ void DDQueue::launchQueuedWork(std::set<RelocateData, std::greater<RelocateData>
for (; it != combined.end(); it++) {
RelocateData rd(*it);
// If having a bulk load task overlapping the rd range,
// attach bulk load task to the input rd if rd is not a data move
// for unhealthy. Make the bulk load task visible on the global task map
bool doBulkLoading = runPendingBulkLoadTaskWithRelocateData(this, rd);
if (doBulkLoading) {
TraceEvent(SevInfo, "DDBulkLoadRunTaskWithRelocateData", this->distributorId)
.detail("NewDataMoveId", rd.dataMoveId)
.detail("NewDataMovePriority", rd.priority)
.detail("NewDataMoveRange", rd.keys)
.detail("BulkLoadTask", rd.bulkLoadTask.get().toString());
}
// Check if there is an inflight shard that is overlapped with the queued relocateShard (rd)
bool overlappingInFlight = false;
auto intersectingInFlight = inFlight.intersectingRanges(rd.keys);
@ -1052,6 +1108,12 @@ void DDQueue::launchQueuedWork(std::set<RelocateData, std::greater<RelocateData>
// FIXME: we need spare capacity even when we're just going to be cancelling work via TEAM_HEALTHY
if (!rd.isRestore() && !canLaunchSrc(rd, teamSize, singleRegionTeamSize, busymap, cancellableRelocations)) {
// logRelocation( rd, "SkippingQueuedRelocation" );
if (rd.bulkLoadTask.present()) {
TraceEvent(g_network->isSimulated() ? SevError : SevWarnAlways,
"DDBulkLoadDelayedByBusySrc",
this->distributorId)
.detail("BulkLoadTask", rd.bulkLoadTask.get().toString());
}
continue;
}
@ -1101,6 +1163,14 @@ void DDQueue::launchQueuedWork(std::set<RelocateData, std::greater<RelocateData>
for (int r = 0; r < ranges.size(); r++) {
RelocateData& rrs = inFlight.rangeContaining(ranges[r].begin)->value();
rrs.keys = ranges[r];
if (rrs.bulkLoadTask.present() && rrs.bulkLoadTask.get().coreState.getRange() != rrs.keys) {
// The new bulk load data move partially overwrites an old bulk load data move.
// In this case, the old bulk load task is cancelled.
// For the range that is not covered by the new data move, drop the bulk load task and
// run it as a normal data move.
ASSERT(rrs.bulkLoadTask.get().coreState.getRange().contains(rrs.keys));
rrs.bulkLoadTask.reset();
}
if (rd.keys == ranges[r] && rd.isRestore()) {
ASSERT(rd.dataMove != nullptr);
ASSERT(SERVER_KNOBS->SHARD_ENCODE_LOCATION_METADATA);
@ -1115,9 +1185,9 @@ void DDQueue::launchQueuedWork(std::set<RelocateData, std::greater<RelocateData>
} else {
rrs.dataMoveId = newDataMoveId(deterministicRandom()->randomUInt64(),
AssignEmptyRange::False,
newDataMoveType(),
newDataMoveType(doBulkLoading),
rrs.dmReason);
TraceEvent(SevInfo, "NewDataMoveWithRandomDestID")
TraceEvent(SevInfo, "NewDataMoveWithRandomDestID", this->distributorId)
.detail("DataMoveID", rrs.dataMoveId.toString())
.detail("TrackID", rrs.randomId)
.detail("Range", rrs.keys)
@ -1126,6 +1196,12 @@ void DDQueue::launchQueuedWork(std::set<RelocateData, std::greater<RelocateData>
}
} else {
rrs.dataMoveId = anonymousShardId;
TraceEvent(SevInfo, "NewDataMoveWithAnonymousDestID", this->distributorId)
.detail("DataMoveID", rrs.dataMoveId.toString())
.detail("TrackID", rrs.randomId)
.detail("Range", rrs.keys)
.detail("Reason", rrs.reason.toString())
.detail("DataMoveReason", static_cast<int>(rrs.dmReason));
}
}
@ -1333,6 +1409,34 @@ static int nonOverlappedServerCount(const std::vector<UID>& srcIds, const std::v
return count;
}
void validateBulkLoadRelocateData(const RelocateData& rd, const std::vector<UID>& destIds, UID logId) {
BulkLoadState bulkLoadState = rd.bulkLoadTask.get().coreState;
if (rd.keys != bulkLoadState.getRange()) {
TraceEvent(g_network->isSimulated() ? SevError : SevWarnAlways, "DDBulkLoadTaskLaunchFailed", logId)
.detail("Reason", "Wrong data move range")
.detail("BulkLoadTask", bulkLoadState.toString())
.detail("DataMovePriority", rd.priority)
.detail("DataMoveId", rd.dataMoveId)
.detail("RelocatorRange", rd.keys);
throw movekeys_conflict();
// Very important invariant. If this error appears, check the logic
}
for (const auto& destId : destIds) {
if (std::find(rd.src.begin(), rd.src.end(), destId) != rd.src.end()) {
// In this case, getTeam has to select src as dest when remote team collection is not ready
// This is not expected
TraceEvent(g_network->isSimulated() ? SevError : SevWarnAlways, "DDBulkLoadTaskLaunchFailed", logId)
.detail("Reason", "Conflict src and destd due to remote recovery")
.detail("BulkLoadTask", bulkLoadState.toString())
.detail("DataMovePriority", rd.priority)
.detail("DataMoveId", rd.dataMoveId)
.detail("RelocatorRange", rd.keys);
throw movekeys_conflict();
}
}
return;
}
// This actor relocates the specified keys to a good place.
// The inFlightActor key range map stores the actor for each RelocateData
ACTOR Future<Void> dataDistributionRelocator(DDQueue* self,
@ -1359,6 +1463,7 @@ ACTOR Future<Void> dataDistributionRelocator(DDQueue* self,
state WantTrueBest wantTrueBest(isValleyFillerPriority(rd.priority));
state uint64_t debugID = deterministicRandom()->randomUInt64();
state bool enableShardMove = SERVER_KNOBS->SHARD_ENCODE_LOCATION_METADATA && SERVER_KNOBS->ENABLE_DD_PHYSICAL_SHARD;
state bool doBulkLoading = rd.bulkLoadTask.present();
try {
if (now() - self->lastInterval < 1.0) {
@ -1471,6 +1576,17 @@ ACTOR Future<Void> dataDistributionRelocator(DDQueue* self,
}
anyHealthy = true;
bestTeams.emplace_back(bestTeam.first.get(), bestTeam.second);
if (doBulkLoading) {
TraceEvent(SevInfo, "DDBulkLoadTaskSelectDestTeam", self->distributorId)
.detail("Context", "Restore")
.detail("SrcIds", describe(rd.src))
.detail("DestIds", bestTeam.first.get()->getServerIDs())
.detail("DestTeam", bestTeam.first.get()->getTeamID())
.detail("BulkLoadTask", rd.bulkLoadTask.get().toString())
.detail("Priority", rd.priority)
.detail("DataMoveId", rd.dataMoveId)
.detail("Primary", tciIndex == 0);
}
} else {
double inflightPenalty = SERVER_KNOBS->INFLIGHT_PENALTY_HEALTHY;
if (rd.healthPriority == SERVER_KNOBS->PRIORITY_TEAM_UNHEALTHY ||
@ -1505,6 +1621,7 @@ ACTOR Future<Void> dataDistributionRelocator(DDQueue* self,
req.src = rd.src;
req.completeSources = rd.completeSources;
req.storageQueueAware = SERVER_KNOBS->ENABLE_STORAGE_QUEUE_AWARE_TEAM_SELECTION;
req.findTeamForBulkLoad = doBulkLoading;
if (enableShardMove && tciIndex == 1) {
ASSERT(physicalShardIDCandidate != UID().first() &&
@ -1534,6 +1651,12 @@ ACTOR Future<Void> dataDistributionRelocator(DDQueue* self,
brokenPromiseToNever(self->teamCollections[tciIndex].getTeam.getReply(req));
bestTeamReady = fbestTeam.isReady();
std::pair<Optional<Reference<IDataDistributionTeam>>, bool> bestTeam = wait(fbestTeam);
if (doBulkLoading) {
TraceEvent(SevInfo, "DDBulkLoadTaskRelocatorBestTeamReceived", self->distributorId)
.detail("DataMoveID", rd.dataMoveId)
.detail("BulkLoadTask", rd.bulkLoadTask.get().toString())
.detail("BestTeamReady", bestTeamReady);
}
if (tciIndex > 0 && !bestTeamReady) {
// self->shardsAffectedByTeamFailure->moveShard must be called without any waits after
// getting the destination team or we could miss failure notifications for the storage
@ -1630,6 +1753,17 @@ ACTOR Future<Void> dataDistributionRelocator(DDQueue* self,
}
} else {
bestTeams.emplace_back(bestTeam.first.get(), bestTeam.second);
if (doBulkLoading) {
TraceEvent(SevInfo, "DDBulkLoadTaskSelectDestTeam", self->distributorId)
.detail("Context", "New")
.detail("SrcIds", describe(rd.src))
.detail("DestIds", bestTeam.first.get()->getServerIDs())
.detail("DestTeam", bestTeam.first.get()->getTeamID())
.detail("BulkLoadTask", rd.bulkLoadTask.get().toString())
.detail("Priority", rd.priority)
.detail("DataMoveId", rd.dataMoveId)
.detail("Primary", tciIndex == 0);
}
}
}
tciIndex++;
@ -1701,6 +1835,7 @@ ACTOR Future<Void> dataDistributionRelocator(DDQueue* self,
}
if (enableShardMove) {
// TODO(BulkLoad): double check if bulk loading can do with physical shard collection feature
if (!rd.isRestore()) {
// when !rd.isRestore(), dataMoveId is just decided as physicalShardIDCandidate
// thus, update the physicalShardIDCandidate to related data structures
@ -1711,7 +1846,7 @@ ACTOR Future<Void> dataDistributionRelocator(DDQueue* self,
self->moveCreateNewPhysicalShard++;
}
rd.dataMoveId = newDataMoveId(
physicalShardIDCandidate, AssignEmptyRange::False, newDataMoveType(), rd.dmReason);
physicalShardIDCandidate, AssignEmptyRange::False, newDataMoveType(doBulkLoading), rd.dmReason);
TraceEvent(SevInfo, "NewDataMoveWithPhysicalShard")
.detail("DataMoveID", rd.dataMoveId.toString())
.detail("Reason", rd.reason.toString())
@ -1752,13 +1887,15 @@ ACTOR Future<Void> dataDistributionRelocator(DDQueue* self,
destinationTeams.push_back(ShardsAffectedByTeamFailure::Team(serverIds, i == 0));
// TODO(psm): Make DataMoveMetaData aware of the two-step data move optimization.
if (allHealthy && anyWithSource && !bestTeams[i].second) {
if (allHealthy && anyWithSource && !bestTeams[i].second && !doBulkLoading) {
// When all servers in bestTeams[i] do not hold the shard (!bestTeams[i].second), it indicates
// the bestTeams[i] is in a new DC where data has not been replicated to.
// To move data (specified in RelocateShard) to bestTeams[i] in the new DC AND reduce data movement
// across DC, we randomly choose a server in bestTeams[i] as the shard's destination, and
// move the shard to the randomly chosen server (in the remote DC), which will later
// propagate its data to the servers in the same team. This saves data movement bandwidth across DC
// Bulk loading data move avoids this optimization since it does not move any data from source
// servers
int idx = deterministicRandom()->randomInt(0, serverIds.size());
destIds.push_back(serverIds[idx]);
healthyIds.push_back(serverIds[idx]);
@ -1777,6 +1914,16 @@ ACTOR Future<Void> dataDistributionRelocator(DDQueue* self,
}
}
// Sanity check for bulk loading data move
if (doBulkLoading) {
validateBulkLoadRelocateData(rd, destIds, self->distributorId);
TraceEvent(SevInfo, "DDBulkLoadTaskGotDestTeam", self->distributorId)
.detail("BulkLoadTask", rd.bulkLoadTask.get().toString())
.detail("DataMoveId", rd.dataMoveId)
.detail("SrcIds", describe(rd.src))
.detail("DestId", describe(destIds));
}
// Sanity check
state int totalIds = 0;
for (auto& destTeam : destinationTeams) {
@ -1823,7 +1970,9 @@ ACTOR Future<Void> dataDistributionRelocator(DDQueue* self,
self->teamCollections.size() > 1,
relocateShardInterval.pairID,
ddEnabledState,
CancelConflictingDataMoves::False);
CancelConflictingDataMoves::False,
rd.bulkLoadTask.present() ? rd.bulkLoadTask.get().coreState
: Optional<BulkLoadState>());
} else {
params = std::make_unique<MoveKeysParams>(rd.dataMoveId,
rd.keys,
@ -1836,7 +1985,9 @@ ACTOR Future<Void> dataDistributionRelocator(DDQueue* self,
self->teamCollections.size() > 1,
relocateShardInterval.pairID,
ddEnabledState,
CancelConflictingDataMoves::False);
CancelConflictingDataMoves::False,
rd.bulkLoadTask.present() ? rd.bulkLoadTask.get().coreState
: Optional<BulkLoadState>());
}
state Future<Void> doMoveKeys = self->txnProcessor->moveKeys(*params);
state Future<Void> pollHealth =
@ -1864,7 +2015,10 @@ ACTOR Future<Void> dataDistributionRelocator(DDQueue* self,
self->teamCollections.size() > 1,
relocateShardInterval.pairID,
ddEnabledState,
CancelConflictingDataMoves::False);
CancelConflictingDataMoves::False,
rd.bulkLoadTask.present()
? rd.bulkLoadTask.get().coreState
: Optional<BulkLoadState>());
} else {
params = std::make_unique<MoveKeysParams>(rd.dataMoveId,
rd.keys,
@ -1877,7 +2031,10 @@ ACTOR Future<Void> dataDistributionRelocator(DDQueue* self,
self->teamCollections.size() > 1,
relocateShardInterval.pairID,
ddEnabledState,
CancelConflictingDataMoves::False);
CancelConflictingDataMoves::False,
rd.bulkLoadTask.present()
? rd.bulkLoadTask.get().coreState
: Optional<BulkLoadState>());
}
doMoveKeys = self->txnProcessor->moveKeys(*params);
} else {
@ -1995,8 +2152,29 @@ ACTOR Future<Void> dataDistributionRelocator(DDQueue* self,
rd.keys, rd.isRestore(), selectedTeams, rd.dataMoveId.first(), metrics, debugID);
}
if (doBulkLoading) {
try {
self->bulkLoadTaskCollection->terminateTask(rd.bulkLoadTask.get().coreState);
TraceEvent(SevInfo, "DDBulkLoadTaskRelocatorComplete", self->distributorId)
.detail("Dests", describe(destIds))
.detail("Task", rd.bulkLoadTask.get().toString());
} catch (Error& bulkLoadError) {
ASSERT_WE_THINK(bulkLoadError.code() == error_code_bulkload_task_outdated);
if (bulkLoadError.code() != error_code_bulkload_task_outdated) {
throw bulkLoadError;
}
TraceEvent(SevInfo, "DDBulkLoadTaskRelocatorCompleteButOutdated", self->distributorId)
.detail("Dests", describe(destIds))
.detail("Task", rd.bulkLoadTask.get().toString());
}
}
return Void();
} else {
if (doBulkLoading) {
TraceEvent(SevInfo, "DDBulkLoadTaskRelocatorError")
.errorUnsuppressed(error)
.detail("Task", rd.bulkLoadTask.get().toString());
}
throw error;
}
} else {
@ -2035,6 +2213,12 @@ ACTOR Future<Void> dataDistributionRelocator(DDQueue* self,
relocationComplete.send(rd);
if (doBulkLoading && e.code() != error_code_actor_cancelled && e.code() != error_code_movekeys_conflict) {
TraceEvent(SevWarnAlways, "DDBulkLoadTaskRelocatorFailed", self->distributorId)
.errorUnsuppressed(e)
.detail("BulkLoadTask", rd.bulkLoadTask.get().toString());
}
if (err.code() == error_code_data_move_dest_team_not_found) {
wait(cancelDataMove(self, rd.keys, ddEnabledState));
TraceEvent(SevWarnAlways, "RelocateShardCancelDataMoveTeamNotFound")

View File

@ -272,6 +272,12 @@ ACTOR Future<Void> trackShardMetrics(DataDistributionTracker::SafeAccessor self,
shardMetrics->get().present() ? shardMetrics->get().get().metrics.bytes : 0);
if (shardMetrics->get().present()) {
DisabledTraceEvent("TrackerChangeSizes")
.detail("Context", "trackShardMetrics")
.detail("Keys", keys)
.detail("TotalSizeEstimate", self()->dbSizeEstimate->get())
.detail("EndSizeOfOldShards", shardMetrics->get().get().metrics.bytes)
.detail("StartingSizeOfNewShards", metrics.first.get().bytes);
self()->dbSizeEstimate->set(self()->dbSizeEstimate->get() + metrics.first.get().bytes -
shardMetrics->get().get().metrics.bytes);
if (SERVER_KNOBS->SHARD_ENCODE_LOCATION_METADATA && SERVER_KNOBS->ENABLE_DD_PHYSICAL_SHARD) {
@ -367,7 +373,10 @@ ACTOR Future<int64_t> getFirstSize(Reference<AsyncVar<Optional<ShardMetrics>>> s
}
}
ACTOR Future<Void> changeSizes(DataDistributionTracker* self, KeyRange keys, int64_t oldShardsEndingSize) {
ACTOR Future<Void> changeSizes(DataDistributionTracker* self,
KeyRange keys,
int64_t oldShardsEndingSize,
std::string context) {
state std::vector<Future<int64_t>> sizes;
state std::vector<Future<int64_t>> systemSizes;
for (auto it : self->shards->intersectingRanges(keys)) {
@ -392,10 +401,12 @@ ACTOR Future<Void> changeSizes(DataDistributionTracker* self, KeyRange keys, int
}
int64_t totalSizeEstimate = self->dbSizeEstimate->get();
/*TraceEvent("TrackerChangeSizes")
DisabledTraceEvent("TrackerChangeSizes")
.detail("Context", "changeSizes when " + context)
.detail("Keys", keys)
.detail("TotalSizeEstimate", totalSizeEstimate)
.detail("EndSizeOfOldShards", oldShardsEndingSize)
.detail("StartingSizeOfNewShards", newShardsStartingSize);*/
.detail("StartingSizeOfNewShards", newShardsStartingSize);
self->dbSizeEstimate->set(totalSizeEstimate + newShardsStartingSize - oldShardsEndingSize);
self->systemSizeEstimate += newSystemShardsStartingSize;
if (keys.begin >= systemKeys.begin) {
@ -502,7 +513,7 @@ void executeShardSplit(DataDistributionTracker* self,
}
}
self->actors.add(changeSizes(self, keys, shardSize->get().get().metrics.bytes));
self->actors.add(changeSizes(self, keys, shardSize->get().get().metrics.bytes, "ShardSplit"));
}
struct RangeToSplit {
@ -918,6 +929,14 @@ static bool shardForwardMergeFeasible(DataDistributionTracker* self, KeyRange co
return false;
}
if (self->bulkLoadEnabled && self->bulkLoadTaskCollection->overlappingTask(nextRange)) {
TraceEvent(SevWarn, "ShardCanForwardMergeButUnderBulkLoading", self->distributorId)
.suppressFor(5.0)
.detail("ShardMerging", keys)
.detail("NextShard", nextRange);
return false;
}
return shardMergeFeasible(self, keys, nextRange);
}
@ -930,9 +949,64 @@ static bool shardBackwardMergeFeasible(DataDistributionTracker* self, KeyRange c
return false;
}
if (self->bulkLoadEnabled && self->bulkLoadTaskCollection->overlappingTask(prevRange)) {
TraceEvent(SevWarn, "ShardCanBackwardMergeButUnderBulkLoading", self->distributorId)
.suppressFor(5.0)
.detail("ShardMerging", keys)
.detail("PrevShard", prevRange);
return false;
}
return shardMergeFeasible(self, keys, prevRange);
}
// Must be atomic
void createShardToBulkLoad(DataDistributionTracker* self, BulkLoadState bulkLoadState) {
KeyRange keys = bulkLoadState.getRange();
ASSERT(!keys.empty());
TraceEvent e(SevInfo, "DDBulkLoadCreateShardToBulkLoad", self->distributorId);
e.detail("TaskId", bulkLoadState.getTaskId());
e.detail("BulkLoadRange", keys);
// Create shards at the two ends and do not data move for those shards
// Create a new shard and trigger data move for bulk loading on the new shard
// Step 1: split left without data move nor updating dbEstimate size (will be rebuilt after DD restarts)
for (auto it : self->shards->intersectingRanges(keys)) {
if (it->range().begin < keys.begin) {
KeyRange leftRange = Standalone(KeyRangeRef(it->range().begin, keys.begin));
restartShardTrackers(self, leftRange);
e.detail("FirstSplitShard", it->range());
}
break;
}
// Step 2: split right without data move nor updating dbEstimate size (will be rebuilt after DD restarts)
for (auto it : self->shards->intersectingRanges(keys)) {
if (it->range().end > keys.end) {
KeyRange rightRange = Standalone(KeyRangeRef(keys.end, it->range().end));
restartShardTrackers(self, rightRange);
e.detail("LastSplitShard", it->range());
break;
}
}
// Step 3: merge with new data move
StorageMetrics oldStats;
int shardCount = 0;
for (auto it : self->shards->intersectingRanges(keys)) {
Reference<AsyncVar<Optional<ShardMetrics>>> stats;
if (it->value().stats->get().present()) {
oldStats = oldStats + it->value().stats->get().get().metrics;
shardCount = shardCount + it->value().stats->get().get().shardCount;
}
}
restartShardTrackers(self, keys, ShardMetrics(oldStats, now(), shardCount));
self->shardsAffectedByTeamFailure->defineShard(keys);
self->output.send(
RelocateShard(keys, DataMovementReason::TEAM_HEALTHY, RelocateReason::OTHER, bulkLoadState.getTaskId()));
e.detail("NewShardToLoad", keys);
return;
}
Future<Void> shardMerger(DataDistributionTracker* self,
KeyRange const& keys,
Reference<AsyncVar<Optional<ShardMetrics>>> shardSize) {
@ -1131,6 +1205,15 @@ ACTOR Future<Void> shardEvaluator(DataDistributionTracker* self,
bool sizeSplit = stats.bytes > shardBounds.max.bytes,
writeSplit = bandwidthStatus == BandwidthStatusHigh && keys.begin < keyServersKeys.begin;
bool shouldSplit = sizeSplit || writeSplit;
bool onBulkLoading = self->bulkLoadEnabled && self->bulkLoadTaskCollection->overlappingTask(keys);
if (onBulkLoading && shouldSplit) {
TraceEvent(SevWarn, "ShardWantToSplitButUnderBulkLoading", self->distributorId)
.suppressFor(5.0)
.detail("KeyRange", keys);
shouldSplit = false;
// Bulk loading will delay shard boundary change until the loading completes
onChange = onChange || delay(SERVER_KNOBS->DD_BULKLOAD_SHARD_BOUNDARY_CHANGE_DELAY_SEC);
}
auto prevIter = self->shards->rangeContaining(keys.begin);
if (keys.begin > allKeys.begin)
@ -1143,6 +1226,14 @@ ACTOR Future<Void> shardEvaluator(DataDistributionTracker* self,
bool shouldMerge = stats.bytes < shardBounds.min.bytes && bandwidthStatus == BandwidthStatusLow &&
(shardForwardMergeFeasible(self, keys, nextIter.range()) ||
shardBackwardMergeFeasible(self, keys, prevIter.range()));
if (onBulkLoading && shouldMerge) {
TraceEvent(SevWarn, "ShardWantToMergeButUnderBulkLoading", self->distributorId)
.suppressFor(5.0)
.detail("KeyRange", keys);
shouldMerge = false;
// Bulk loading will delay shard boundary change until the loading completes
onChange = onChange || delay(SERVER_KNOBS->DD_BULKLOAD_SHARD_BOUNDARY_CHANGE_DELAY_SEC);
}
// Every invocation must set this or clear it
if (shouldMerge && !self->anyZeroHealthyTeams->get()) {
@ -1289,7 +1380,7 @@ ACTOR Future<Void> trackInitialShards(DataDistributionTracker* self, Reference<I
wait(yield(TaskPriority::DataDistribution));
}
Future<Void> initialSize = changeSizes(self, KeyRangeRef(allKeys.begin, allKeys.end), 0);
Future<Void> initialSize = changeSizes(self, KeyRangeRef(allKeys.begin, allKeys.end), 0, "ShardInit");
self->readyToStart.send(Void());
wait(initialSize);
self->maxShardSizeUpdater = updateMaxShardSize(self->dbSizeEstimate, self->maxShardSize);
@ -1511,9 +1602,9 @@ DataDistributionTracker::DataDistributionTracker(DataDistributionTrackerInitPara
: IDDShardTracker(), db(params.db), distributorId(params.distributorId), shards(params.shards), actors(false),
systemSizeEstimate(0), dbSizeEstimate(new AsyncVar<int64_t>()), maxShardSize(new AsyncVar<Optional<int64_t>>()),
output(params.output), shardsAffectedByTeamFailure(params.shardsAffectedByTeamFailure),
physicalShardCollection(params.physicalShardCollection), readyToStart(params.readyToStart),
anyZeroHealthyTeams(params.anyZeroHealthyTeams), trackerCancelled(params.trackerCancelled),
ddTenantCache(params.ddTenantCache) {}
physicalShardCollection(params.physicalShardCollection), bulkLoadTaskCollection(params.bulkLoadTaskCollection),
readyToStart(params.readyToStart), anyZeroHealthyTeams(params.anyZeroHealthyTeams),
trackerCancelled(params.trackerCancelled), ddTenantCache(params.ddTenantCache) {}
DataDistributionTracker::~DataDistributionTracker() {
if (trackerCancelled) {
@ -1564,6 +1655,9 @@ struct DataDistributionTrackerImpl {
when(RebalanceStorageQueueRequest req = waitNext(self->triggerStorageQueueRebalance)) {
triggerStorageQueueRebalance(self, req);
}
when(BulkLoadShardRequest req = waitNext(self->triggerShardBulkLoading)) {
createShardToBulkLoad(self, req.bulkLoadState);
}
when(wait(self->actors.getResult())) {}
when(TenantCacheTenantCreated newTenant = waitNext(tenantCreationSignal.getFuture())) {
self->actors.add(tenantCreationHandling(self, newTenant));
@ -1573,7 +1667,12 @@ struct DataDistributionTrackerImpl {
}
}
} catch (Error& e) {
TraceEvent(SevError, "DataDistributionTrackerError", self->distributorId).error(e);
if (e.code() != error_code_broken_promise) {
TraceEvent(SevError, "DataDistributionTrackerError", self->distributorId)
.error(e); // FIXME: get rid of broken_promise
} else {
TraceEvent(SevWarn, "DataDistributionTrackerError", self->distributorId).error(e);
}
throw e;
}
}
@ -1586,13 +1685,16 @@ Future<Void> DataDistributionTracker::run(
const FutureStream<GetTopKMetricsRequest>& getTopKMetrics,
const FutureStream<GetMetricsListRequest>& getShardMetricsList,
const FutureStream<Promise<int64_t>>& getAverageShardBytes,
const FutureStream<RebalanceStorageQueueRequest>& triggerStorageQueueRebalance) {
const FutureStream<RebalanceStorageQueueRequest>& triggerStorageQueueRebalance,
const FutureStream<BulkLoadShardRequest>& triggerShardBulkLoading) {
self->getShardMetrics = getShardMetrics;
self->getTopKMetrics = getTopKMetrics;
self->getShardMetricsList = getShardMetricsList;
self->averageShardBytes = getAverageShardBytes;
self->triggerStorageQueueRebalance = triggerStorageQueueRebalance;
self->triggerShardBulkLoading = triggerShardBulkLoading;
self->userRangeConfig = initData->userRangeConfig;
self->bulkLoadEnabled = bulkLoadIsEnabled(initData->bulkLoadMode);
return holdWhile(self, DataDistributionTrackerImpl::run(self.getPtr(), initData));
}

View File

@ -195,6 +195,100 @@ public:
req.reply.send(std::make_pair(res, false));
}
// Random selection for load balance
ACTOR static Future<Void> getTeamForBulkLoad(DDTeamCollection* self, GetTeamRequest req) {
try {
TraceEvent(SevInfo, "DDBulkLoadTaskGetTeamReqReceived", self->distributorId)
.detail("TCReady", self->readyToStart.isReady())
.detail("TeamBuilderValid", self->teamBuilder.isValid())
.detail("TeamBuilderReady", self->teamBuilder.isValid() ? self->teamBuilder.isReady() : false)
.detail("SrcIds", describe(req.src))
.detail("Primary", self->isPrimary())
.detail("TeamSize", self->teams.size());
wait(self->checkBuildTeams());
TraceEvent(SevInfo, "DDBulkLoadTaskGetTeamCheckBuildTeamDone", self->distributorId)
.detail("TCReady", self->readyToStart.isReady())
.detail("TeamBuilderValid", self->teamBuilder.isValid())
.detail("TeamBuilderReady", self->teamBuilder.isValid() ? self->teamBuilder.isReady() : false)
.detail("SrcIds", describe(req.src))
.detail("Primary", self->isPrimary())
.detail("TeamSize", self->teams.size());
if (!self->primary && !self->readyToStart.isReady()) {
// When remote DC is not ready, DD shouldn't reply with a new team because
// a data movement to that team can't be completed and such a move
// may block the primary DC from reaching "storage_recovered".
auto team = self->findTeamFromServers(req.completeSources, /*wantHealthy=*/false);
TraceEvent(SevWarn, "DDBulkLoadTaskGetTeamRemoteDCNotReady", self->distributorId)
.suppressFor(1.0)
.detail("Primary", self->primary)
.detail("Team", team.present() ? describe(team.get()->getServerIDs()) : "");
req.reply.send(std::make_pair(team, true));
return Void();
}
self->updateTeamPivotValues();
std::vector<Reference<TCTeamInfo>> candidateTeams;
int unhealthyTeamCount = 0;
int duplicatedCount = 0;
for (const auto& dest : self->teams) {
if (!dest->isHealthy()) {
unhealthyTeamCount++;
continue;
}
bool ok = true;
for (const auto& srcId : req.src) {
std::vector<UID> serverIds = dest->getServerIDs();
for (const auto& serverId : serverIds) {
if (serverId == srcId) {
ok = false; // Do not select a team that has a server owning the bulk loading range
break;
}
}
if (!ok) {
break;
}
}
if (!ok) {
duplicatedCount++;
continue;
}
candidateTeams.push_back(dest);
}
Optional<Reference<IDataDistributionTeam>> res;
if (candidateTeams.size() >= 1) {
res = deterministicRandom()->randomChoice(candidateTeams);
TraceEvent(SevInfo, "DDBulkLoadTaskGetTeamReply", self->distributorId)
.detail("TCReady", self->readyToStart.isReady())
.detail("SrcIds", describe(req.src))
.detail("Primary", self->isPrimary())
.detail("TeamSize", self->teams.size())
.detail("CandidateSize", candidateTeams.size())
.detail("UnhealthyTeamCount", unhealthyTeamCount)
.detail("DuplicatedCount", duplicatedCount)
.detail("DestIds", describe(res.get()->getServerIDs()))
.detail("DestTeam", res.get()->getTeamID());
} else {
TraceEvent(SevWarnAlways, "DDBulkLoadTaskGetTeamFailedToFindValidTeam", self->distributorId)
.detail("TCReady", self->readyToStart.isReady())
.detail("SrcIds", describe(req.src))
.detail("Primary", self->isPrimary())
.detail("TeamSize", self->teams.size())
.detail("CandidateSize", candidateTeams.size())
.detail("UnhealthyTeamCount", unhealthyTeamCount)
.detail("DuplicatedCount", duplicatedCount);
}
req.reply.send(std::make_pair(res, false));
return Void();
} catch (Error& e) {
if (e.code() != error_code_actor_cancelled && req.reply.canBeSet())
req.reply.sendError(e);
throw;
}
}
// Return a threshold of team queue size which guarantees at least DD_LONG_STORAGE_QUEUE_TEAM_MAJORITY_PERCENTILE
// portion of teams that have longer storage queues
// A team storage queue size is defined as the longest storage queue size among all SSes of the team
@ -766,6 +860,7 @@ public:
}
TraceEvent("BuildTeamsBegin", self->distributorId)
.detail("Primary", self->isPrimary())
.detail("TeamsToBuild", teamsToBuild)
.detail("DesiredTeams", desiredTeams)
.detail("MaxTeams", maxTeams)
@ -3022,6 +3117,8 @@ public:
GetTeamRequest req = waitNext(tci.getTeam.getFuture());
if (req.findTeamByServers) {
getTeamByServers(self, req);
} else if (req.findTeamForBulkLoad) {
self->addActor.send(getTeamForBulkLoad(self, req));
} else {
self->addActor.send(self->getTeam(req));
}

View File

@ -299,6 +299,14 @@ class DDTxnProcessorImpl {
return result;
}
result->bulkLoadMode = 0;
Optional<Value> bulkLoadMode = wait(tr.get(bulkLoadModeKey));
if (bulkLoadMode.present()) {
BinaryReader rd(bulkLoadMode.get(), Unversioned());
rd >> result->bulkLoadMode;
}
TraceEvent(SevInfo, "DDBulkLoadInitMode").detail("Mode", result->bulkLoadMode);
state Future<std::vector<ProcessData>> workers = getWorkers(&tr);
state Future<RangeResult> serverList = tr.getRange(serverListKeys, CLIENT_KNOBS->TOO_MANY);
wait(success(workers) && success(serverList));

View File

@ -33,12 +33,14 @@
#include "fdbclient/SystemData.h"
#include "fdbclient/Tenant.h"
#include "fdbrpc/Replication.h"
#include "fdbserver/BulkLoadUtil.actor.h"
#include "fdbserver/DDSharedContext.h"
#include "fdbserver/DDTeamCollection.h"
#include "fdbserver/DataDistribution.actor.h"
#include "fdbserver/FDBExecHelper.actor.h"
#include "fdbserver/IKeyValueStore.h"
#include "fdbserver/Knobs.h"
#include "fdbserver/MoveKeys.actor.h"
#include "fdbserver/QuietDatabase.h"
#include "fdbserver/ServerDBInfo.h"
#include "fdbserver/TLogInterface.h"
@ -360,6 +362,9 @@ ACTOR Future<Void> skipAuditOnRange(Reference<DataDistributor> self,
std::shared_ptr<DDAudit> audit,
KeyRange rangeToSkip);
void runBulkLoadTaskAsync(Reference<DataDistributor> self, KeyRange range, UID taskId, bool restart);
ACTOR Future<Void> scheduleBulkLoadTasks(Reference<DataDistributor> self);
struct DataDistributor : NonCopyable, ReferenceCounted<DataDistributor> {
public:
Reference<AsyncVar<ServerDBInfo> const> dbInfo;
@ -387,7 +392,9 @@ public:
// consumer is a yield stream from producer. The RelocateShard is pushed into relocationProducer and popped from
// relocationConsumer (by DDQueue)
PromiseStream<RelocateShard> relocationProducer, relocationConsumer;
PromiseStream<BulkLoadShardRequest> triggerShardBulkLoading;
Reference<PhysicalShardCollection> physicalShardCollection;
Reference<BulkLoadTaskCollection> bulkLoadTaskCollection;
Promise<Void> initialized;
@ -405,15 +412,19 @@ public:
Promise<Version> configChangeWatching;
Future<Void> onConfigChange;
ActorCollection bulkLoadActors;
bool bulkLoadEnabled;
DataDistributor(Reference<AsyncVar<ServerDBInfo> const> const& db, UID id, Reference<DDSharedContext> context)
: dbInfo(db), context(context), ddId(id), txnProcessor(nullptr), lock(context->lock),
configuration(context->configuration), initialDDEventHolder(makeReference<EventCacheHolder>("InitialDD")),
movingDataEventHolder(makeReference<EventCacheHolder>("MovingData")),
totalDataInFlightEventHolder(makeReference<EventCacheHolder>("TotalDataInFlight")),
totalDataInFlightRemoteEventHolder(makeReference<EventCacheHolder>("TotalDataInFlightRemote")),
teamCollection(nullptr), auditStorageHaLaunchingLock(1), auditStorageReplicaLaunchingLock(1),
auditStorageLocationMetadataLaunchingLock(1), auditStorageSsShardLaunchingLock(1),
auditStorageInitStarted(false) {}
teamCollection(nullptr), bulkLoadTaskCollection(nullptr), auditStorageHaLaunchingLock(1),
auditStorageReplicaLaunchingLock(1), auditStorageLocationMetadataLaunchingLock(1),
auditStorageSsShardLaunchingLock(1), auditStorageInitStarted(false), bulkLoadActors(false),
bulkLoadEnabled(false) {}
// bootstrap steps
@ -682,12 +693,17 @@ public:
}
state std::vector<Key> customBoundaries;
for (auto it : self->initData->userRangeConfig->ranges()) {
auto range = it->range();
customBoundaries.push_back(range.begin);
TraceEvent(SevDebug, "DDInitCustomRangeConfig", self->ddId)
.detail("Range", KeyRangeRef(range.begin, range.end))
.detail("Config", it->value());
if (self->initData->bulkLoadMode == 0) {
for (auto it : self->initData->userRangeConfig->ranges()) {
auto range = it->range();
customBoundaries.push_back(range.begin);
TraceEvent(SevDebug, "DDInitCustomRangeConfig", self->ddId)
.detail("Range", KeyRangeRef(range.begin, range.end))
.detail("Config", it->value());
}
} else {
// Bulk load does not allow boundary change
TraceEvent(SevInfo, "DDInitCustomRangeConfigDisabledByBulkLoadMode", self->ddId);
}
state int shard = 0;
@ -782,7 +798,16 @@ public:
TraceEvent(SevInfo, "EmptyDataMoveRange", self->ddId).detail("DataMoveMetaData", meta.toString());
continue;
}
if (it.value()->isCancelled() || (it.value()->valid && !SERVER_KNOBS->SHARD_ENCODE_LOCATION_METADATA)) {
if (meta.bulkLoadState.present()) {
RelocateShard rs(meta.ranges.front(), DataMovementReason::RECOVER_MOVE, RelocateReason::OTHER);
rs.dataMoveId = meta.id;
rs.cancelled = true;
self->relocationProducer.send(rs);
// Cancel data move for old bulk loading
// Do not assign bulk load to rs so that this is a normal data move cancellation signal
TraceEvent("DDInitScheduledCancelOldBulkLoadDataMove", self->ddId).detail("DataMove", meta.toString());
} else if (it.value()->isCancelled() ||
(it.value()->valid && !SERVER_KNOBS->SHARD_ENCODE_LOCATION_METADATA)) {
RelocateShard rs(meta.ranges.front(), DataMovementReason::RECOVER_MOVE, RelocateReason::OTHER);
rs.dataMoveId = meta.id;
rs.cancelled = true;
@ -994,6 +1019,292 @@ ACTOR Future<Void> serveBlobMigratorRequests(Reference<DataDistributor> self,
}
}
// Trigger a task on range based on the current bulk load task metadata
ACTOR Future<std::pair<BulkLoadState, Version>> triggerBulkLoadTask(Reference<DataDistributor> self,
KeyRange range,
UID taskId,
bool restart) {
loop {
Database cx = self->txnProcessor->context();
state Transaction tr(cx);
state BulkLoadState newBulkLoadState;
try {
tr.setOption(FDBTransactionOptions::LOCK_AWARE);
tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
wait(checkMoveKeysLock(&tr, self->context->lock, self->context->ddEnabledState.get()));
std::vector<BulkLoadPhase> phase;
if (!restart) {
wait(
store(newBulkLoadState,
getBulkLoadTask(&tr, range, taskId, { BulkLoadPhase::Submitted, BulkLoadPhase::Triggered })));
} else {
wait(store(newBulkLoadState,
getBulkLoadTask(&tr, range, taskId, { BulkLoadPhase::Triggered, BulkLoadPhase::Running })));
}
newBulkLoadState.phase = BulkLoadPhase::Triggered;
newBulkLoadState.clearDataMoveId();
newBulkLoadState.restartCount = newBulkLoadState.restartCount + 1;
newBulkLoadState.triggerTime = now();
wait(krmSetRange(&tr, bulkLoadPrefix, newBulkLoadState.getRange(), bulkLoadStateValue(newBulkLoadState)));
wait(tr.commit());
Version commitVersion = tr.getCommittedVersion();
TraceEvent(SevInfo, "DDBulkLoadTaskTriggeredPersist", self->ddId)
.detail("CommitVersion", commitVersion)
.detail("BulkLoadState", newBulkLoadState.toString());
ASSERT(commitVersion != invalidVersion);
return std::make_pair(newBulkLoadState, commitVersion);
} catch (Error& e) {
wait(tr.onError(e));
}
}
}
ACTOR Future<Void> tryStartBulkLoadTaskUntilSucceed(Reference<DataDistributor> self) {
loop {
if (self->bulkLoadTaskCollection->tryStart()) {
break;
}
wait(self->bulkLoadTaskCollection->waitUntilChanged());
}
return Void();
}
ACTOR Future<Void> waitUntilBulkLoadTaskCanStart(Reference<DataDistributor> self) {
loop {
if (self->bulkLoadTaskCollection->canStart()) {
break;
}
wait(self->bulkLoadTaskCollection->waitUntilChanged());
}
return Void();
}
ACTOR Future<Void> doBulkLoadTask(Reference<DataDistributor> self, KeyRange range, UID taskId, bool restart) {
state Promise<Void> completeAck;
state BulkLoadState triggeredBulkLoadTask;
state Version commitVersion = invalidVersion;
TraceEvent(SevInfo, "DDBulkLoadDoBulkLoadBegin", self->ddId)
.detail("Range", range)
.detail("TaskID", taskId)
.detail("Restart", restart);
wait(tryStartBulkLoadTaskUntilSucceed(self)); // increase the task counter when succeed
try {
// Step 1: persist bulk load task phase as triggered
std::pair<BulkLoadState, Version> triggeredBulkLoadTask_ =
wait(triggerBulkLoadTask(self, range, taskId, restart));
triggeredBulkLoadTask = triggeredBulkLoadTask_.first;
commitVersion = triggeredBulkLoadTask_.second;
TraceEvent(SevInfo, "DDBulkLoadDoBulkLoadTaskTriggered", self->ddId)
.setMaxEventLength(-1)
.setMaxFieldLength(-1)
.detail("Task", triggeredBulkLoadTask.toString())
.detail("CommitVersion", commitVersion)
.detail("Restart", restart);
ASSERT(triggeredBulkLoadTask.getRange() == range);
// Step 2: submit the task to in-memory task map, which (1) turns off shard boundary change;
// (2) when starting a data move on the task range, the task will be attached to the data move;
// (3) when the data move completes, the completeAck is satisfied. So, waiting on completeAck
// can get notified when the task is completed by a data move
self->bulkLoadTaskCollection->publishTask(triggeredBulkLoadTask, commitVersion, completeAck);
// Step 3: create bulk load shard and trigger data move and wait for task completion
// The completion of the task relies on the fact that a data move on a range is either
// completed by itself or replaced by a data move on the overlapping range
self->triggerShardBulkLoading.send(BulkLoadShardRequest(triggeredBulkLoadTask));
wait(completeAck.getFuture()); // proceed when a data move completes with this task
TraceEvent(SevInfo, "DDBulkLoadDoBulkLoadTaskComplete", self->ddId)
.setMaxEventLength(-1)
.setMaxFieldLength(-1)
.detail("Task", triggeredBulkLoadTask.toString())
.detail("Restart", restart);
self->bulkLoadTaskCollection->decrementTaskCounter();
} catch (Error& e) {
if (e.code() == error_code_actor_cancelled) {
throw e;
}
TraceEvent(SevWarn, "DDBulkLoadDoBulkLoadTaskFailed", self->ddId)
.errorUnsuppressed(e)
.detail("Range", range)
.detail("TaskID", taskId)
.detail("Restart", restart);
if (e.code() == error_code_bulkload_task_outdated) {
self->bulkLoadTaskCollection->decrementTaskCounter();
// sliently exits
} else if (e.code() == error_code_movekeys_conflict) {
throw e;
} else {
// retry by spawning a new one
runBulkLoadTaskAsync(self, range, taskId, true);
}
}
return Void();
}
ACTOR Future<Void> eraseBulkLoadTask(Reference<DataDistributor> self, KeyRange range, UID taskId) {
loop {
state Database cx = self->txnProcessor->context();
state Transaction tr(cx);
state BulkLoadState bulkLoadTask;
try {
tr.setOption(FDBTransactionOptions::LOCK_AWARE);
tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
wait(store(bulkLoadTask, getBulkLoadTask(&tr, range, taskId, { BulkLoadPhase::Acknowledged })));
wait(krmSetRangeCoalescing(&tr, bulkLoadPrefix, range, normalKeys, StringRef()));
wait(tr.commit());
self->bulkLoadTaskCollection->eraseTask(bulkLoadTask);
break;
} catch (Error& e) {
if (e.code() == error_code_bulkload_task_outdated) {
// Silently exit
break;
}
wait(tr.onError(e));
}
}
return Void();
}
void runBulkLoadTaskAsync(Reference<DataDistributor> self, KeyRange range, UID taskId, bool restart) {
TraceEvent(SevInfo, "DDBulkLoadTaskRunAsync", self->ddId)
.detail("Range", range)
.detail("TaskId", taskId)
.detail("Restart", restart);
self->bulkLoadActors.add(doBulkLoadTask(self, range, taskId, restart));
return;
}
void eraseBulkLoadTaskAsync(Reference<DataDistributor> self, KeyRange range, UID taskId, bool restart) {
TraceEvent(SevInfo, "DDBulkLoadTaskEraseAsync", self->ddId)
.detail("Range", range)
.detail("TaskId", taskId)
.detail("Restart", restart);
self->bulkLoadActors.add(eraseBulkLoadTask(self, range, taskId));
return;
}
ACTOR Future<Void> scheduleBulkLoadTasks(Reference<DataDistributor> self) {
state Key beginKey = normalKeys.begin;
state Key endKey = normalKeys.end;
state KeyRange rangeToRead;
state Database cx = self->txnProcessor->context();
state Transaction tr(cx);
state int i = 0;
state BulkLoadState bulkLoadState;
state RangeResult result;
while (beginKey < endKey) {
try {
rangeToRead = Standalone(KeyRangeRef(beginKey, endKey));
result.clear();
wait(store(
result,
krmGetRanges(&tr, bulkLoadPrefix, rangeToRead, SERVER_KNOBS->DD_BULKLOAD_TASK_METADATA_READ_SIZE)));
i = 0;
for (; i < result.size() - 1; i++) {
if (!result[i].value.empty()) {
KeyRange range = Standalone(KeyRangeRef(result[i].key, result[i + 1].key));
bulkLoadState = decodeBulkLoadState(result[i].value);
if (range != bulkLoadState.getRange()) {
// This task is outdated
continue;
} else if (bulkLoadState.phase == BulkLoadPhase::Submitted) {
wait(waitUntilBulkLoadTaskCanStart(self));
runBulkLoadTaskAsync(
self, bulkLoadState.getRange(), bulkLoadState.getTaskId(), /*restart=*/false);
} else if (bulkLoadState.phase == BulkLoadPhase::Acknowledged) {
eraseBulkLoadTaskAsync(
self, bulkLoadState.getRange(), bulkLoadState.getTaskId(), /*restart=*/false);
} else {
ASSERT(bulkLoadState.phase == BulkLoadPhase::Triggered ||
bulkLoadState.phase == BulkLoadPhase::Running ||
bulkLoadState.phase == BulkLoadPhase::Complete);
}
}
}
beginKey = result.back().key;
} catch (Error& e) {
if (e.code() == error_code_actor_cancelled) {
throw e;
}
wait(tr.onError(e));
}
}
return Void();
}
ACTOR Future<Void> bulkLoadTaskScheduler(Reference<DataDistributor> self) {
loop {
wait(scheduleBulkLoadTasks(self) && delay(SERVER_KNOBS->DD_BULKLOAD_SCHEDULE_MIN_INTERVAL_SEC));
}
}
ACTOR Future<Void> resumeBulkLoadTasks(Reference<DataDistributor> self) {
state Key beginKey = normalKeys.begin;
state Key endKey = normalKeys.end;
state KeyRange rangeToRead;
while (beginKey < endKey) {
Database cx = self->txnProcessor->context();
state Transaction tr(cx);
try {
rangeToRead = Standalone(KeyRangeRef(beginKey, endKey));
RangeResult result =
wait(krmGetRanges(&tr, bulkLoadPrefix, rangeToRead, SERVER_KNOBS->DD_BULKLOAD_TASK_METADATA_READ_SIZE));
for (int i = 0; i < result.size() - 1; i++) {
if (!result[i].value.empty()) {
KeyRange range = Standalone(KeyRangeRef(result[i].key, result[i + 1].key));
BulkLoadState bulkLoadState = decodeBulkLoadState(result[i].value);
if (range != bulkLoadState.getRange()) {
TraceEvent(SevWarn, "DDBulkLoadRestartTriggeredTaskFailed", self->ddId)
.detail("Reason", "Task boundary changed")
.detail("BulkLoadTask", bulkLoadState.toString())
.detail("RangeInSpace", range);
} else if (bulkLoadState.phase == BulkLoadPhase::Triggered) {
runBulkLoadTaskAsync(
self, bulkLoadState.getRange(), bulkLoadState.getTaskId(), /*restart=*/true);
} else if (bulkLoadState.phase == BulkLoadPhase::Running) {
runBulkLoadTaskAsync(
self, bulkLoadState.getRange(), bulkLoadState.getTaskId(), /*restart=*/true);
} else if (bulkLoadState.phase == BulkLoadPhase::Acknowledged) {
eraseBulkLoadTaskAsync(
self, bulkLoadState.getRange(), bulkLoadState.getTaskId(), /*restart=*/true);
} else {
TraceEvent(SevDebug, "DDBulkLoadRestartRangeNoTask", self->ddId).detail("RangeInSpace", range);
}
}
}
beginKey = result.back().key;
} catch (Error& e) {
wait(tr.onError(e));
}
}
TraceEvent(SevInfo, "DDBulkLoadRestartTriggeredTasksComplete", self->ddId);
return Void();
}
ACTOR Future<Void> bulkLoadingCore(Reference<DataDistributor> self, Future<Void> readyToStart) {
wait(readyToStart);
wait(resumeBulkLoadTasks(self));
TraceEvent(SevInfo, "DDBulkLoadCoreResumed", self->ddId);
loop {
try {
self->bulkLoadActors.add(bulkLoadTaskScheduler(self));
wait(self->bulkLoadActors.getResult());
} catch (Error& e) {
if (e.code() == error_code_actor_cancelled) {
throw e;
}
TraceEvent(SevInfo, "DDBulkLoadCoreError", self->ddId).errorUnsuppressed(e);
if (e.code() == error_code_movekeys_conflict) {
throw e;
}
}
self->bulkLoadActors.clear(false);
wait(delay(SERVER_KNOBS->DD_BULKLOAD_SCHEDULE_MIN_INTERVAL_SEC));
}
}
// Runs the data distribution algorithm for FDB, including the DD Queue, DD tracker, and DD team collection
ACTOR Future<Void> dataDistribution(Reference<DataDistributor> self,
PromiseStream<GetMetricsListRequest> getShardMetricsList,
@ -1012,10 +1323,14 @@ ACTOR Future<Void> dataDistribution(Reference<DataDistributor> self,
ASSERT(self->txnProcessor.isValid() && self->txnProcessor->isMocked());
}
TraceEvent(SevInfo, "DataDistributionInitProgress", self->ddId).detail("Phase", "Start");
// Make sure that the watcher has established a baseline before init() below so the watcher will
// see any changes that occur after init() has read the config state.
wait(self->initDDConfigWatch());
TraceEvent(SevInfo, "DataDistributionInitProgress", self->ddId).detail("Phase", "DDConfigWatch Initialized");
loop {
self->context->trackerCancelled = false;
// whether all initial shard are tracked
@ -1029,6 +1344,8 @@ ACTOR Future<Void> dataDistribution(Reference<DataDistributor> self,
wait(DataDistributor::init(self));
TraceEvent(SevInfo, "DataDistributionInitProgress", self->ddId).detail("Phase", "Metadata Initialized");
// When/If this assertion fails, Evan owes Ben a pat on the back for his foresight
ASSERT(self->configuration.storageTeamSize > 0);
@ -1046,8 +1363,12 @@ ACTOR Future<Void> dataDistribution(Reference<DataDistributor> self,
self->shardsAffectedByTeamFailure = makeReference<ShardsAffectedByTeamFailure>();
self->physicalShardCollection = makeReference<PhysicalShardCollection>(self->txnProcessor);
self->bulkLoadTaskCollection =
makeReference<BulkLoadTaskCollection>(self->ddId, SERVER_KNOBS->DD_BULKLOAD_PARALLELISM);
wait(self->resumeRelocations());
TraceEvent(SevInfo, "DataDistributionInitProgress", self->ddId).detail("Phase", "Relocation Resumed");
std::vector<TeamCollectionInterface> tcis; // primary and remote region interface
Reference<AsyncVar<bool>> anyZeroHealthyTeams; // true if primary or remote has zero healthy team
std::vector<Reference<AsyncVar<bool>>> zeroHealthyTeams; // primary and remote
@ -1079,6 +1400,7 @@ ACTOR Future<Void> dataDistribution(Reference<DataDistributor> self,
.output = self->relocationProducer,
.shardsAffectedByTeamFailure = self->shardsAffectedByTeamFailure,
.physicalShardCollection = self->physicalShardCollection,
.bulkLoadTaskCollection = self->bulkLoadTaskCollection,
.anyZeroHealthyTeams = anyZeroHealthyTeams,
.shards = &shards,
.trackerCancelled = &self->context->trackerCancelled,
@ -1089,7 +1411,8 @@ ACTOR Future<Void> dataDistribution(Reference<DataDistributor> self,
getTopKShardMetrics.getFuture(),
getShardMetricsList.getFuture(),
getAverageShardBytes.getFuture(),
triggerStorageQueueRebalance.getFuture()),
triggerStorageQueueRebalance.getFuture(),
self->triggerShardBulkLoading.getFuture()),
"DDTracker",
self->ddId,
&normalDDQueueErrors()));
@ -1101,6 +1424,7 @@ ACTOR Future<Void> dataDistribution(Reference<DataDistributor> self,
.teamCollections = tcis,
.shardsAffectedByTeamFailure = self->shardsAffectedByTeamFailure,
.physicalShardCollection = self->physicalShardCollection,
.bulkLoadTaskCollection = self->bulkLoadTaskCollection,
.getAverageShardBytes = getAverageShardBytes,
.teamSize = replicaSize,
.singleRegionTeamSize = self->configuration.storageTeamSize,
@ -1210,6 +1534,17 @@ ACTOR Future<Void> dataDistribution(Reference<DataDistributor> self,
}
actors.push_back(serveBlobMigratorRequests(self, self->context->tracker, self->context->ddQueue));
if (bulkLoadIsEnabled(self->initData->bulkLoadMode)) {
TraceEvent(SevInfo, "DDBulkLoadModeEnabled", self->ddId)
.detail("UsableRegions", self->configuration.usableRegions);
self->bulkLoadEnabled = true;
if (self->configuration.usableRegions > 1) {
actors.push_back(
bulkLoadingCore(self, self->initialized.getFuture() && remoteRecovered(self->dbInfo)));
} else {
actors.push_back(bulkLoadingCore(self, self->initialized.getFuture()));
}
}
wait(waitForAll(actors));
ASSERT_WE_THINK(false);

View File

@ -3177,6 +3177,8 @@ struct ShardedRocksDBKeyValueStore : IKeyValueStore {
TraceEvent(SevError, "ShardedRocksDBRestoreFailed", logId)
.detail("Reason", "RestoreFilesRangesMismatch")
.detail("Ranges", describe(a.ranges))
.detail("FetchedRanges", describe(fetchedRanges))
.detail("IntendedRanges", describe(intendedRanges))
.setMaxFieldLength(1000)
.detail("FetchedFiles", describe(rkvs));
a.done.sendError(failed_to_restore_checkpoint());

View File

@ -29,6 +29,7 @@
#include "fdbclient/KeyBackedTypes.actor.h"
#include "fdbclient/ManagementAPI.actor.h"
#include "fdbclient/SystemData.h"
#include "fdbserver/BulkLoadUtil.actor.h"
#include "fdbserver/MoveKeys.actor.h"
#include "fdbserver/Knobs.h"
#include "fdbclient/ReadYourWrites.h"
@ -376,7 +377,7 @@ ACTOR static Future<Void> checkPersistentMoveKeysLock(Transaction* tr, MoveKeysL
Future<Void> checkMoveKeysLock(Transaction* tr,
MoveKeysLock const& lock,
const DDEnabledState* ddEnabledState,
bool isWrite = true) {
bool isWrite) {
if (!ddEnabledState->isEnabled()) {
TraceEvent(SevDebug, "DDDisabledByInMemoryCheck").log();
throw movekeys_conflict();
@ -1619,7 +1620,8 @@ ACTOR static Future<Void> startMoveShards(Database occ,
UID relocationIntervalId,
std::map<UID, StorageServerInterface>* tssMapping,
const DDEnabledState* ddEnabledState,
CancelConflictingDataMoves cancelConflictingDataMoves) {
CancelConflictingDataMoves cancelConflictingDataMoves,
Optional<BulkLoadState> bulkLoadState) {
state Future<Void> warningLogger = logWarningAfter("StartMoveShardsTooLong", 600, servers);
wait(startMoveKeysLock->take(TaskPriority::DataDistributionLaunch));
@ -1630,7 +1632,8 @@ ACTOR static Future<Void> startMoveShards(Database occ,
TraceEvent(SevInfo, "StartMoveShardsBegin", relocationIntervalId)
.detail("DataMoveID", dataMoveId)
.detail("TargetRange", describe(ranges));
.detail("TargetRange", describe(ranges))
.detail("BulkLoadState", bulkLoadState.present() ? bulkLoadState.get().toString() : "");
// TODO: make startMoveShards work with multiple ranges.
ASSERT(ranges.size() == 1);
@ -1659,7 +1662,8 @@ ACTOR static Future<Void> startMoveShards(Database occ,
dataMove = dmv;
TraceEvent(sevDm, "StartMoveShardsFoundDataMove", relocationIntervalId)
.detail("DataMoveID", dataMoveId)
.detail("DataMove", dataMove.toString());
.detail("DataMove", dataMove.toString())
.detail("CancelDataMove", cancelDataMove);
if (dataMove.getPhase() == DataMoveMetaData::Deleting) {
TraceEvent(sevDm, "StartMoveShardsDataMoveDeleted", relocationIntervalId)
.detail("DataMove", dataMove.toString())
@ -1748,6 +1752,22 @@ ACTOR static Future<Void> startMoveShards(Database occ,
.detail("DestID", destId)
.detail("ReadVersion", tr.getReadVersion().get());
if (bulkLoadState.present()) {
state std::vector<UID> owners(src.size() + dest.size());
std::merge(src.begin(), src.end(), dest.begin(), dest.end(), owners.begin());
for (const auto& ssid : servers) {
if (std::find(owners.begin(), owners.end(), ssid) != owners.end()) {
TraceEvent(SevWarn, "DDBulkLoadTaskStartMoveShardsMoveInConflict")
.detail("BulkLoadState", bulkLoadState.get().toString())
.detail("DestServerId", ssid)
.detail("OwnerIds", describe(owners))
.detail("DataMove", dataMove.toString());
cancelDataMove = true;
throw retry();
}
}
}
// Pre validate consistency of update of keyServers and serverKeys
if (SERVER_KNOBS->AUDIT_DATAMOVE_PRE_CHECK && runPreCheck) {
std::vector<UID> servers(src.size() + dest.size());
@ -1809,7 +1829,8 @@ ACTOR static Future<Void> startMoveShards(Database occ,
dataMove.src.insert(src.begin(), src.end());
if (shouldCreateCheckpoint(dataMoveId)) {
// If this is a bulk load data move, need not create checkpoint on the source servers
if (shouldCreateCheckpoint(dataMoveId) && !bulkLoadState.present()) {
const UID checkpointId = UID(deterministicRandom()->randomUInt64(), srcId.first());
CheckpointMetaData checkpoint(std::vector<KeyRange>{ rangeIntersectKeys },
DataMoveRocksCF,
@ -1844,11 +1865,38 @@ ACTOR static Future<Void> startMoveShards(Database occ,
}
if (currentKeys.end == keys.end) {
if (bulkLoadState.present()) {
state BulkLoadState newBulkLoadState;
try {
wait(store(newBulkLoadState,
getBulkLoadTask(&tr,
bulkLoadState.get().getRange(),
bulkLoadState.get().getTaskId(),
{ BulkLoadPhase::Triggered, BulkLoadPhase::Running })));
// It is possible that the previous data move is cancelled but has updated the
// task phase as running. In this case, we update the phase from Running to Running
newBulkLoadState.phase = BulkLoadPhase::Running;
} catch (Error& e) {
if (e.code() == error_code_bulkload_task_outdated) {
cancelDataMove = true;
throw retry();
}
throw e;
}
newBulkLoadState.setDataMoveId(dataMoveId);
newBulkLoadState.startTime = now();
wait(krmSetRange(
&tr, bulkLoadPrefix, newBulkLoadState.getRange(), bulkLoadStateValue(newBulkLoadState)));
TraceEvent(SevInfo, "DDBulkLoadTaskRunningPersist", relocationIntervalId)
.detail("BulkLoadState", newBulkLoadState.toString());
dataMove.bulkLoadState = newBulkLoadState;
}
dataMove.setPhase(DataMoveMetaData::Running);
complete = true;
TraceEvent(sevDm, "StartMoveShardsDataMoveComplete", relocationIntervalId)
.detail("DataMoveID", dataMoveId)
.detail("DataMove", dataMove.toString());
.detail("DataMove", dataMove.toString())
.detail("BulkLoadState", bulkLoadState.present() ? bulkLoadState.get().toString() : "");
} else {
dataMove.setPhase(DataMoveMetaData::Prepare);
TraceEvent(sevDm, "StartMoveShardsDataMovePartial", relocationIntervalId)
@ -1870,7 +1918,8 @@ ACTOR static Future<Void> startMoveShards(Database occ,
.detail("CommitVersion", tr.getCommittedVersion())
.detail("DeltaRange", currentKeys.toString())
.detail("Range", describe(dataMove.ranges))
.detail("DataMove", dataMove.toString());
.detail("DataMove", dataMove.toString())
.detail("BulkLoadState", bulkLoadState.present() ? bulkLoadState.get().toString() : "");
// Post validate consistency of update of keyServers and serverKeys
if (SERVER_KNOBS->AUDIT_DATAMOVE_POST_CHECK) {
@ -1989,7 +2038,8 @@ ACTOR static Future<Void> finishMoveShards(Database occ,
bool hasRemote,
UID relocationIntervalId,
std::map<UID, StorageServerInterface> tssMapping,
const DDEnabledState* ddEnabledState) {
const DDEnabledState* ddEnabledState,
Optional<BulkLoadState> bulkLoadState) {
// TODO: make startMoveShards work with multiple ranges.
ASSERT(targetRanges.size() == 1);
state KeyRange keys = targetRanges[0];
@ -2035,7 +2085,8 @@ ACTOR static Future<Void> finishMoveShards(Database occ,
dataMove = decodeDataMoveValue(val.get());
TraceEvent(sevDm, "FinishMoveShardsFoundDataMove", relocationIntervalId)
.detail("DataMoveID", dataMoveId)
.detail("DataMove", dataMove.toString());
.detail("DataMove", dataMove.toString())
.detail("CancelDataMove", cancelDataMove);
if (cancelDataMove) {
dataMove.setPhase(DataMoveMetaData::Deleting);
tr.set(dataMoveKeyFor(dataMoveId), dataMoveValue(dataMove));
@ -2046,7 +2097,8 @@ ACTOR static Future<Void> finishMoveShards(Database occ,
std::sort(destServers.begin(), destServers.end());
if (dataMove.getPhase() == DataMoveMetaData::Deleting) {
TraceEvent(SevWarn, "FinishMoveShardsDataMoveDeleting", relocationIntervalId)
.detail("DataMoveID", dataMoveId);
.detail("DataMoveID", dataMoveId)
.detail("DataMove", dataMove.toString());
throw data_move_cancelled();
}
ASSERT(dataMove.getPhase() == DataMoveMetaData::Running);
@ -2054,7 +2106,8 @@ ACTOR static Future<Void> finishMoveShards(Database occ,
range = dataMove.ranges.front();
} else {
TraceEvent(SevWarn, "FinishMoveShardsDataMoveDeleted", relocationIntervalId)
.detail("DataMoveID", dataMoveId);
.detail("DataMoveID", dataMoveId)
.detail("DataMove", dataMove.toString());
wait(checkDataMoveComplete(occ, dataMoveId, keys, relocationIntervalId));
return Void();
}
@ -2085,13 +2138,15 @@ ACTOR static Future<Void> finishMoveShards(Database occ,
.detail("SrcID", srcId)
.detail("Src", describe(src))
.detail("DestID", destId)
.detail("Dest", describe(dest));
.detail("Dest", describe(dest))
.detail("DataMove", dataMove.toString());
allServers.insert(src.begin(), src.end());
allServers.insert(dest.begin(), dest.end());
if (destId != dataMoveId) {
TraceEvent(SevWarnAlways, "FinishMoveShardsInconsistentIDs", relocationIntervalId)
.detail("DataMoveID", dataMoveId)
.detail("ExistingShardID", destId);
.detail("ExistingShardID", destId)
.detail("DataMove", dataMove.toString());
cancelDataMove = true;
throw retry();
}
@ -2179,7 +2234,8 @@ ACTOR static Future<Void> finishMoveShards(Database occ,
TraceEvent(sevDm, "FinishMoveShardsWaitingServers", relocationIntervalId)
.detail("DataMoveID", dataMoveId)
.detail("NewDestinations", describe(newDestinations));
.detail("NewDestinations", describe(newDestinations))
.detail("DataMove", dataMove.toString());
// Wait for all storage server moves, and explicitly swallow errors for tss ones with
// waitForAllReady If this takes too long the transaction will time out and retry, which is ok
@ -2220,7 +2276,8 @@ ACTOR static Future<Void> finishMoveShards(Database occ,
.detail("DataMoveID", dataMoveId)
.detail("ReadyServers", describe(readyServers))
.detail("NewDestinations", describe(newDestinations))
.detail("ReadyTSS", tssCount);
.detail("ReadyTSS", tssCount)
.detail("DataMove", dataMove.toString());
if (readyServers.size() == newDestinations.size()) {
@ -2240,22 +2297,52 @@ ACTOR static Future<Void> finishMoveShards(Database occ,
TraceEvent(sevDm, "FinishMoveShardsSetServerKeyRange", relocationIntervalId)
.detail("StorageServerID", ssId)
.detail("KeyRange", range)
.detail("ShardID", destHasServer ? dataMoveId : UID());
.detail("ShardID", destHasServer ? dataMoveId : UID())
.detail("DataMove", dataMove.toString());
}
wait(waitForAll(actors));
if (range.end == dataMove.ranges.front().end) {
if (bulkLoadState.present()) {
state BulkLoadState newBulkLoadState;
try {
wait(store(newBulkLoadState,
getBulkLoadTask(&tr,
bulkLoadState.get().getRange(),
bulkLoadState.get().getTaskId(),
{ BulkLoadPhase::Running, BulkLoadPhase::Complete })));
newBulkLoadState.phase = BulkLoadPhase::Complete;
} catch (Error& e) {
if (e.code() == error_code_bulkload_task_outdated) {
cancelDataMove = true;
throw retry();
}
throw e;
}
ASSERT(newBulkLoadState.getDataMoveId().present() &&
newBulkLoadState.getDataMoveId().get() == dataMoveId);
newBulkLoadState.completeTime = now();
wait(krmSetRange(&tr,
bulkLoadPrefix,
newBulkLoadState.getRange(),
bulkLoadStateValue(newBulkLoadState)));
TraceEvent(SevInfo, "DDBulkLoadTaskCompletePersist", relocationIntervalId)
.detail("BulkLoadState", newBulkLoadState.toString());
dataMove.bulkLoadState = newBulkLoadState;
}
wait(deleteCheckpoints(&tr, dataMove.checkpoints, dataMoveId));
tr.clear(dataMoveKeyFor(dataMoveId));
complete = true;
TraceEvent(sevDm, "FinishMoveShardsDeleteMetaData", relocationIntervalId)
.detail("DataMove", dataMove.toString());
} else {
} else if (!bulkLoadState.present()) {
// Bulk Loading data move does not allow partial complete
TraceEvent(SevInfo, "FinishMoveShardsPartialComplete", relocationIntervalId)
.detail("DataMoveID", dataMoveId)
.detail("CurrentRange", range)
.detail("NewDataMoveMetaData", dataMove.toString());
.detail("NewDataMoveMetaData", dataMove.toString())
.detail("DataMove", dataMove.toString());
dataMove.ranges.front() = KeyRangeRef(range.end, dataMove.ranges.front().end);
tr.set(dataMoveKeyFor(dataMoveId), dataMoveValue(dataMove));
}
@ -2306,7 +2393,10 @@ ACTOR static Future<Void> finishMoveShards(Database occ,
throw;
}
TraceEvent(SevInfo, "FinishMoveShardsEnd", relocationIntervalId).detail("DataMoveID", dataMoveId);
TraceEvent(SevInfo, "FinishMoveShardsEnd", relocationIntervalId)
.detail("DataMoveID", dataMoveId)
.detail("BulkLoadState", bulkLoadState.present() ? bulkLoadState.get().toString() : "")
.detail("DataMove", dataMove.toString());
return Void();
}
@ -3137,7 +3227,8 @@ Future<Void> rawStartMovement(Database occ,
params.relocationIntervalId,
&tssMapping,
params.ddEnabledState,
params.cancelConflictingDataMoves);
params.cancelConflictingDataMoves,
params.bulkLoadState);
}
ASSERT(params.keys.present());
return startMoveKeys(std::move(occ),
@ -3187,7 +3278,8 @@ Future<Void> rawFinishMovement(Database occ,
params.hasRemote,
params.relocationIntervalId,
tssMapping,
params.ddEnabledState);
params.ddEnabledState,
params.bulkLoadState);
}
ASSERT(params.keys.present());
return finishMoveKeys(std::move(occ),

View File

@ -771,6 +771,46 @@ bool RocksDBSstFileWriter::finish() {
return true;
}
class RocksDBSstFileReader : public IRocksDBSstFileReader {
public:
RocksDBSstFileReader() : sstReader(std::make_unique<rocksdb::SstFileReader>(rocksdb::Options())){};
~RocksDBSstFileReader() {}
void open(const std::string localFile) override;
KeyValue next() override;
bool hasNext() const override;
private:
std::unique_ptr<rocksdb::SstFileReader> sstReader;
std::unique_ptr<rocksdb::Iterator> iter;
std::string localFile;
};
void RocksDBSstFileReader::open(const std::string localFile) {
this->localFile = abspath(localFile);
rocksdb::Status status = sstReader->Open(this->localFile);
if (status.ok()) {
iter.reset(sstReader->NewIterator(getReadOptions()));
iter->SeekToFirst();
} else {
TraceEvent(SevError, "RocksDBSstFileReaderWrapperOpenFileError")
.detail("LocalFile", this->localFile)
.detail("Status", status.ToString());
}
}
bool RocksDBSstFileReader::hasNext() const {
return iter != nullptr && this->iter->Valid();
}
KeyValue RocksDBSstFileReader::next() {
KeyValue res(KeyValueRef(toStringRef(this->iter->key()), toStringRef(this->iter->value())));
iter->Next();
return res;
}
class RocksDBCheckpointByteSampleReader : public ICheckpointByteSampleReader {
public:
RocksDBCheckpointByteSampleReader(const CheckpointMetaData& checkpoint);
@ -1286,6 +1326,14 @@ std::unique_ptr<IRocksDBSstFileWriter> newRocksDBSstFileWriter() {
return nullptr;
}
std::unique_ptr<IRocksDBSstFileReader> newRocksDBSstFileReader() {
#ifdef WITH_ROCKSDB
std::unique_ptr<IRocksDBSstFileReader> sstReader = std::make_unique<RocksDBSstFileReader>();
return sstReader;
#endif // WITH_ROCKSDB
return nullptr;
}
std::unique_ptr<ICheckpointByteSampleReader> newCheckpointByteSampleReader(const CheckpointMetaData& checkpoint) {
#ifdef WITH_ROCKSDB
return std::make_unique<RocksDBCheckpointByteSampleReader>(checkpoint);

View File

@ -2724,6 +2724,12 @@ void setupSimulatedSystem(std::vector<Future<Void>>* systemActors,
if (possible_ss - simconfig.db.desiredTSSCount / simconfig.db.usableRegions <= simconfig.db.storageTeamSize) {
gradualMigrationPossible = false;
}
TraceEvent("SimulatedClusterAssignMachineToDC")
.detail("DC", dc)
.detail("PossibleSS", possible_ss)
.detail("Machines", machines)
.detail("DcCoordinators", dcCoordinators);
}
g_simulator->desiredCoordinators = coordinatorCount;
@ -2831,11 +2837,12 @@ ACTOR void simulationSetupAndRun(std::string dataFolder,
state bool allowCreatingTenants = testConfig.allowCreatingTenants;
if (!SERVER_KNOBS->SHARD_ENCODE_LOCATION_METADATA &&
// NOTE: PhysicalShardMove is required to have SHARDED_ROCKSDB storage engine working.
// NOTE: PhysicalShardMove and BulkLoading are required to have SHARDED_ROCKSDB storage engine working.
// Inside the TOML file, the SHARD_ENCODE_LOCATION_METADATA is overridden, however, the
// override will not take effect until the test starts. Here, we do an additional check
// for this special simulation test.
std::string_view(testFile).find("PhysicalShardMove") == std::string_view::npos) {
(std::string_view(testFile).find("PhysicalShardMove") == std::string_view::npos &&
std::string_view(testFile).find("BulkLoading") == std::string_view::npos)) {
testConfig.storageEngineExcludeTypes.insert(SimulationStorageEngine::SHARDED_ROCKSDB);
}

View File

@ -0,0 +1,66 @@
/*
* BulkLoadUtil.actor.h
*
* This source file is part of the FoundationDB open source project
*
* Copyright 2013-2024 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_BULKLOADUTIL_ACTOR_G_H)
#define FDBSERVER_BULKLOADUTIL_ACTOR_G_H
#include "fdbserver/BulkLoadUtil.actor.g.h"
#elif !defined(FDBSERVER_BULKLOADUTIL_ACTOR_H)
#define FDBSERVER_BULKLOADUTIL_ACTOR_H
#pragma once
#include "fdbclient/BulkLoading.h"
#include "flow/actorcompiler.h" // has to be last include
struct SSBulkLoadFileSet {
std::unordered_set<std::string> dataFileList;
Optional<std::string> bytesSampleFile;
std::string folder;
SSBulkLoadFileSet() = default;
std::string toString() {
std::string res = "SSBulkLoadFileSet: [DataFiles]: " + describe(dataFileList);
if (bytesSampleFile.present()) {
res = res + ", [BytesSampleFile]: " + bytesSampleFile.get();
}
res = res + ", [Folder]: " + folder;
return res;
}
};
std::string generateRandomBulkLoadDataFileName();
std::string generateRandomBulkLoadBytesSampleFileName();
ACTOR Future<Optional<BulkLoadState>> getBulkLoadStateFromDataMove(Database cx, UID dataMoveId, UID logId);
void bulkLoadFileCopy(std::string fromFile, std::string toFile, size_t fileBytesMax);
ACTOR Future<SSBulkLoadFileSet> bulkLoadTransportCP_impl(std::string dir,
BulkLoadState bulkLoadState,
size_t fileBytesMax,
UID logId);
ACTOR Future<Optional<std::string>> getBytesSamplingFromSSTFiles(std::string folderToGenerate,
std::unordered_set<std::string> dataFiles,
UID logId);
void checkContent(std::unordered_set<std::string> dataFiles, UID logId);
#include "flow/unactorcompiler.h"
#endif

View File

@ -59,6 +59,8 @@ public:
TraceInterval interval;
std::shared_ptr<DataMove> dataMove;
Optional<DDBulkLoadTask> bulkLoadTask;
RelocateData();
explicit RelocateData(RelocateShard const& rs);
@ -109,6 +111,7 @@ struct DDQueueInitParams {
std::vector<TeamCollectionInterface> const& teamCollections;
Reference<ShardsAffectedByTeamFailure> shardsAffectedByTeamFailure;
Reference<PhysicalShardCollection> physicalShardCollection;
Reference<BulkLoadTaskCollection> bulkLoadTaskCollection;
PromiseStream<Promise<int64_t>> const& getAverageShardBytes;
int const& teamSize;
int const& singleRegionTeamSize;
@ -242,6 +245,7 @@ public:
std::vector<TeamCollectionInterface> teamCollections;
Reference<ShardsAffectedByTeamFailure> shardsAffectedByTeamFailure;
Reference<PhysicalShardCollection> physicalShardCollection;
Reference<BulkLoadTaskCollection> bulkLoadTaskCollection;
PromiseStream<Promise<int64_t>> getAverageShardBytes;
FlowLock startMoveKeysParallelismLock;

View File

@ -31,6 +31,7 @@ public:
FutureStream<GetMetricsListRequest> getShardMetricsList;
FutureStream<Promise<int64_t>> averageShardBytes;
FutureStream<RebalanceStorageQueueRequest> triggerStorageQueueRebalance;
FutureStream<BulkLoadShardRequest> triggerShardBulkLoading;
virtual double getAverageShardBytes() = 0;
virtual ~IDDShardTracker() = default;
@ -43,6 +44,7 @@ struct DataDistributionTrackerInitParams {
PromiseStream<RelocateShard> const& output;
Reference<ShardsAffectedByTeamFailure> shardsAffectedByTeamFailure;
Reference<PhysicalShardCollection> physicalShardCollection;
Reference<BulkLoadTaskCollection> bulkLoadTaskCollection;
Reference<AsyncVar<bool>> anyZeroHealthyTeams;
KeyRangeMap<ShardTrackedData>* shards = nullptr;
bool* trackerCancelled = nullptr;
@ -73,6 +75,10 @@ public:
// PhysicalShard Tracker
Reference<PhysicalShardCollection> physicalShardCollection;
// BulkLoadTask Tracker
Reference<BulkLoadTaskCollection> bulkLoadTaskCollection;
bool bulkLoadEnabled = false;
Promise<Void> readyToStart;
Reference<AsyncVar<bool>> anyZeroHealthyTeams;
@ -123,7 +129,8 @@ public:
FutureStream<GetTopKMetricsRequest> const& getTopKMetrics,
FutureStream<GetMetricsListRequest> const& getShardMetricsList,
FutureStream<Promise<int64_t>> const& getAverageShardBytes,
FutureStream<RebalanceStorageQueueRequest> const& triggerStorageQueueRebalance);
FutureStream<RebalanceStorageQueueRequest> const& triggerStorageQueueRebalance,
FutureStream<BulkLoadShardRequest> const& triggerShardBulkLoading);
explicit DataDistributionTracker(DataDistributionTrackerInitParams const& params);
};

View File

@ -24,6 +24,7 @@
#elif !defined(FDBSERVER_DATA_DISTRIBUTION_ACTOR_H)
#define FDBSERVER_DATA_DISTRIBUTION_ACTOR_H
#include "fdbclient/BulkLoading.h"
#include "fdbclient/NativeAPI.actor.h"
#include "fdbserver/MoveKeys.actor.h"
#include "fdbserver/TenantCache.h"
@ -222,6 +223,13 @@ struct GetMetricsListRequest {
GetMetricsListRequest(KeyRange const& keys, const int shardLimit) : keys(keys), shardLimit(shardLimit) {}
};
struct BulkLoadShardRequest {
BulkLoadState bulkLoadState;
BulkLoadShardRequest() {}
BulkLoadShardRequest(BulkLoadState const& bulkLoadState) : bulkLoadState(bulkLoadState) {}
};
// PhysicalShardCollection maintains physical shard concepts in data distribution
// A physical shard contains one or multiple shards (key range)
// PhysicalShardCollection is responsible for creation and maintenance of physical shards (including metrics)
@ -485,6 +493,7 @@ struct InitialDataDistribution : ReferenceCounted<InitialDataDistribution> {
// Read from dataDistributionModeKey. Whether DD is disabled. DD can be disabled persistently (mode = 0). Set mode
// to 1 will enable all disabled parts
int mode;
int bulkLoadMode;
std::vector<std::pair<StorageServerInterface, ProcessClass>> allServers;
std::set<std::vector<UID>> primaryTeams;
std::set<std::vector<UID>> remoteTeams;
@ -520,6 +529,204 @@ bool ddLargeTeamEnabled();
struct TeamCollectionInterface {
PromiseStream<GetTeamRequest> getTeam;
};
struct DDBulkLoadTask {
BulkLoadState coreState;
Version commitVersion;
Promise<Void> completeAck; // satisfied when a data move for this task completes for the first time, where the task
// metadata phase has been complete
DDBulkLoadTask() = default;
DDBulkLoadTask(BulkLoadState coreState, Version commitVersion, Promise<Void> completeAck)
: coreState(coreState), commitVersion(commitVersion), completeAck(completeAck) {}
bool operator==(const DDBulkLoadTask& rhs) const {
return coreState == rhs.coreState && commitVersion == rhs.commitVersion;
}
std::string toString() const {
return coreState.toString() + ", [CommitVersion]: " + std::to_string(commitVersion);
}
};
inline bool bulkLoadIsEnabled(int bulkLoadModeValue) {
return SERVER_KNOBS->SHARD_ENCODE_LOCATION_METADATA && bulkLoadModeValue == 1;
}
class BulkLoadTaskCollection : public ReferenceCounted<BulkLoadTaskCollection> {
public:
BulkLoadTaskCollection(UID ddId, int maxParallelism)
: ddId(ddId), maxParallelism(maxParallelism), numRunningTasks(0) {
bulkLoadTaskMap.insert(allKeys, Optional<DDBulkLoadTask>());
}
// Return true if there exists a bulk load task
bool overlappingTask(KeyRange range) {
for (auto it : bulkLoadTaskMap.intersectingRanges(range)) {
if (!it->value().present()) {
continue;
}
return true;
}
return false;
}
// Return true if there exists a bulk load task since the given commit version
bool overlappingTaskSince(KeyRange range, Version sinceCommitVersion) {
for (auto it : bulkLoadTaskMap.intersectingRanges(range)) {
if (!it->value().present()) {
continue;
}
if (it->value().get().commitVersion > sinceCommitVersion) {
return true;
}
}
return false;
}
// Add a task and this task becomes visible to DDTracker and DDQueue
// DDTracker stops any shard boundary change overlapping the task range
// DDQueue attaches the task to following data moves until the task has been completed
// If there are overlapped old tasks, make it outdated by sending a signal to completeAck
void publishTask(const BulkLoadState& bulkLoadState, Version commitVersion, Promise<Void> completeAck) {
if (overlappingTaskSince(bulkLoadState.getRange(), commitVersion)) {
throw bulkload_task_outdated();
}
DDBulkLoadTask task(bulkLoadState, commitVersion, completeAck);
TraceEvent(SevDebug, "DDBulkLoadCollectionPublishTask", ddId)
.setMaxEventLength(-1)
.setMaxFieldLength(-1)
.detail("Range", bulkLoadState.getRange())
.detail("Task", task.toString());
// For any overlapping task, make it outdated
for (auto it : bulkLoadTaskMap.intersectingRanges(bulkLoadState.getRange())) {
if (!it->value().present()) {
continue;
}
if (it->value().get().coreState.getTaskId() == bulkLoadState.getTaskId()) {
ASSERT(it->value().get().coreState.getRange() == bulkLoadState.getRange());
// In case that the task has been already triggered
// Avoid repeatedly being triggered by throwing the error
// then the current doBulkLoadTask will sliently exit
throw bulkload_task_outdated();
}
if (it->value().get().completeAck.canBeSet()) {
it->value().get().completeAck.sendError(bulkload_task_outdated());
TraceEvent(SevInfo, "DDBulkLoadCollectionPublishTaskOverwriteTask", ddId)
.setMaxEventLength(-1)
.setMaxFieldLength(-1)
.detail("NewRange", bulkLoadState.getRange())
.detail("NewTask", task.toString())
.detail("OldTaskRange", it->range())
.detail("OldTask", it->value().get().toString());
}
}
bulkLoadTaskMap.insert(bulkLoadState.getRange(), task);
return;
}
// This method is called when there is a data move assigned to run the bulk load task
void startTask(const BulkLoadState& bulkLoadState) {
for (auto it : bulkLoadTaskMap.intersectingRanges(bulkLoadState.getRange())) {
if (!it->value().present() || it->value().get().coreState.getTaskId() != bulkLoadState.getTaskId()) {
throw bulkload_task_outdated();
}
TraceEvent(SevDebug, "DDBulkLoadCollectionStartTask", ddId)
.detail("Range", bulkLoadState.getRange())
.detail("TaskRange", it->range())
.detail("Task", it->value().get().toString());
}
return;
}
// Send complete signal to indicate this task has been completed
void terminateTask(const BulkLoadState& bulkLoadState) {
for (auto it : bulkLoadTaskMap.intersectingRanges(bulkLoadState.getRange())) {
if (!it->value().present() || it->value().get().coreState.getTaskId() != bulkLoadState.getTaskId()) {
throw bulkload_task_outdated();
}
// It is possible that the task has been completed by a past data move
if (it->value().get().completeAck.canBeSet()) {
it->value().get().completeAck.send(Void());
TraceEvent(SevDebug, "DDBulkLoadCollectionTerminateTask", ddId)
.detail("Range", bulkLoadState.getRange())
.detail("TaskRange", it->range())
.detail("Task", it->value().get().toString());
}
}
return;
}
// Erase any metadata on the map for the input bulkload task
void eraseTask(const BulkLoadState& bulkLoadState) {
std::vector<KeyRange> rangesToClear;
for (auto it : bulkLoadTaskMap.intersectingRanges(bulkLoadState.getRange())) {
if (!it->value().present() || it->value().get().coreState.getTaskId() != bulkLoadState.getTaskId()) {
continue;
}
TraceEvent(SevDebug, "DDBulkLoadCollectionEraseTaskdata", ddId)
.detail("Range", bulkLoadState.getRange())
.detail("TaskRange", it->range())
.detail("Task", it->value().get().toString());
rangesToClear.push_back(it->range());
}
for (const auto& rangeToClear : rangesToClear) {
bulkLoadTaskMap.insert(rangeToClear, Optional<DDBulkLoadTask>());
}
bulkLoadTaskMap.coalesce(normalKeys);
return;
}
// Get the task which has exactly the same range as the input range
Optional<DDBulkLoadTask> getTaskByRange(KeyRange range) const {
Optional<DDBulkLoadTask> res;
for (auto it : bulkLoadTaskMap.intersectingRanges(range)) {
if (!it->value().present()) {
continue;
}
DDBulkLoadTask bulkLoadTask = it->value().get();
TraceEvent(SevDebug, "DDBulkLoadCollectionGetPublishedTaskEach", ddId)
.detail("Range", range)
.detail("TaskRange", it->range())
.detail("Task", bulkLoadTask.toString());
if (bulkLoadTask.coreState.getRange() == range) {
ASSERT(!res.present());
res = bulkLoadTask;
}
}
TraceEvent(SevDebug, "DDBulkLoadCollectionGetPublishedTask", ddId)
.detail("Range", range)
.detail("Task", res.present() ? describe(res.get()) : "");
return res;
}
inline void decrementTaskCounter() {
ASSERT(numRunningTasks.get() <= maxParallelism);
numRunningTasks.set(numRunningTasks.get() - 1);
ASSERT(numRunningTasks.get() >= 0);
}
// return true if succeed
inline bool tryStart() {
if (numRunningTasks.get() < maxParallelism) {
numRunningTasks.set(numRunningTasks.get() + 1);
return true;
} else {
return false;
}
}
inline bool canStart() const { return numRunningTasks.get() < maxParallelism; }
inline Future<Void> waitUntilChanged() const { return numRunningTasks.onChange(); }
private:
KeyRangeMap<Optional<DDBulkLoadTask>> bulkLoadTaskMap;
UID ddId;
AsyncVar<int> numRunningTasks;
int maxParallelism;
};
#ifndef __INTEL_COMPILER
#pragma endregion
#endif

View File

@ -104,6 +104,7 @@ FDB_BOOLEAN_PARAM(ForReadBalance);
FDB_BOOLEAN_PARAM(PreferLowerReadUtil);
FDB_BOOLEAN_PARAM(FindTeamByServers);
FDB_BOOLEAN_PARAM(PreferWithinShardLimit);
FDB_BOOLEAN_PARAM(FindTeamForBulkLoad);
class TeamSelect {
public:
@ -143,6 +144,7 @@ struct GetTeamRequest {
bool preferWithinShardLimit;
double inflightPenalty;
bool findTeamByServers;
bool findTeamForBulkLoad;
Optional<KeyRange> keys;
bool storageQueueAware;
@ -170,13 +172,15 @@ struct GetTeamRequest {
: teamSelect(teamSelectRequest), storageQueueAware(false), preferLowerDiskUtil(preferLowerDiskUtil),
teamMustHaveShards(teamMustHaveShards), forReadBalance(forReadBalance),
preferLowerReadUtil(preferLowerReadUtil), preferWithinShardLimit(preferWithinShardLimit),
inflightPenalty(inflightPenalty), findTeamByServers(FindTeamByServers::False), keys(keys) {}
inflightPenalty(inflightPenalty), findTeamByServers(FindTeamByServers::False),
findTeamForBulkLoad(FindTeamForBulkLoad::False), keys(keys) {}
GetTeamRequest(std::vector<UID> servers)
: teamSelect(TeamSelect::WANT_COMPLETE_SRCS), storageQueueAware(false),
preferLowerDiskUtil(PreferLowerDiskUtil::False), teamMustHaveShards(TeamMustHaveShards::False),
forReadBalance(ForReadBalance::False), preferLowerReadUtil(PreferLowerReadUtil::False),
preferWithinShardLimit(PreferWithinShardLimit::False), inflightPenalty(1.0),
findTeamByServers(FindTeamByServers::True), src(std::move(servers)) {}
findTeamByServers(FindTeamByServers::True), findTeamForBulkLoad(FindTeamForBulkLoad::False),
src(std::move(servers)) {}
// return true if a.score < b.score
[[nodiscard]] bool lessCompare(TeamRef a, TeamRef b, int64_t aLoadBytes, int64_t bLoadBytes) const {

View File

@ -88,6 +88,8 @@ struct MoveKeysParams {
const DDEnabledState* ddEnabledState = nullptr;
CancelConflictingDataMoves cancelConflictingDataMoves = CancelConflictingDataMoves::False;
Optional<BulkLoadState> bulkLoadState;
MoveKeysParams() {}
MoveKeysParams(UID dataMoveId,
@ -101,13 +103,14 @@ struct MoveKeysParams {
bool hasRemote,
UID relocationIntervalId,
const DDEnabledState* ddEnabledState,
CancelConflictingDataMoves cancelConflictingDataMoves)
CancelConflictingDataMoves cancelConflictingDataMoves,
Optional<BulkLoadState> bulkLoadState)
: dataMoveId(dataMoveId), keys(keys), destinationTeam(destinationTeam), healthyDestinations(healthyDestinations),
lock(lock), dataMovementComplete(dataMovementComplete),
startMoveKeysParallelismLock(startMoveKeysParallelismLock),
finishMoveKeysParallelismLock(finishMoveKeysParallelismLock), hasRemote(hasRemote),
relocationIntervalId(relocationIntervalId), ddEnabledState(ddEnabledState),
cancelConflictingDataMoves(cancelConflictingDataMoves) {}
cancelConflictingDataMoves(cancelConflictingDataMoves), bulkLoadState(bulkLoadState) {}
MoveKeysParams(UID dataMoveId,
const std::vector<KeyRange>& ranges,
@ -120,13 +123,14 @@ struct MoveKeysParams {
bool hasRemote,
UID relocationIntervalId,
const DDEnabledState* ddEnabledState,
CancelConflictingDataMoves cancelConflictingDataMoves)
CancelConflictingDataMoves cancelConflictingDataMoves,
Optional<BulkLoadState> bulkLoadState)
: dataMoveId(dataMoveId), ranges(ranges), destinationTeam(destinationTeam),
healthyDestinations(healthyDestinations), lock(lock), dataMovementComplete(dataMovementComplete),
startMoveKeysParallelismLock(startMoveKeysParallelismLock),
finishMoveKeysParallelismLock(finishMoveKeysParallelismLock), hasRemote(hasRemote),
relocationIntervalId(relocationIntervalId), ddEnabledState(ddEnabledState),
cancelConflictingDataMoves(cancelConflictingDataMoves) {}
cancelConflictingDataMoves(cancelConflictingDataMoves), bulkLoadState(bulkLoadState) {}
};
// read the lock value in system keyspace but do not change anything
@ -204,5 +208,10 @@ ACTOR Future<Void> prepareBlobRestore(Database occ,
UID bmId,
UID reqId = UID());
Future<Void> checkMoveKeysLock(Transaction* tr,
MoveKeysLock const& lock,
const DDEnabledState* ddEnabledState,
bool isWrite = true);
#include "flow/unactorcompiler.h"
#endif

View File

@ -51,6 +51,17 @@ public:
virtual ~IRocksDBSstFileWriter() {}
};
class IRocksDBSstFileReader {
public:
virtual void open(const std::string localFile) = 0;
virtual KeyValue next() = 0;
virtual bool hasNext() const = 0;
virtual ~IRocksDBSstFileReader() {}
};
struct CheckpointFile {
constexpr static FileIdentifier file_identifier = 13804348;
std::string path;
@ -346,6 +357,8 @@ std::unique_ptr<ICheckpointByteSampleReader> newCheckpointByteSampleReader(const
std::unique_ptr<IRocksDBSstFileWriter> newRocksDBSstFileWriter();
std::unique_ptr<IRocksDBSstFileReader> newRocksDBSstFileReader();
RocksDBColumnFamilyCheckpoint getRocksCF(const CheckpointMetaData& checkpoint);
RocksDBCheckpoint getRocksCheckpoint(const CheckpointMetaData& checkpoint);

View File

@ -51,23 +51,33 @@ struct MoveInShardMetaData {
std::vector<CheckpointMetaData> checkpoints; // All related checkpoints, they should cover `ranges`.
Optional<std::string> error;
double startTime;
bool conductBulkLoad;
MoveInShardMetaData() = default;
MoveInShardMetaData(const UID& id,
const UID& dataMoveId,
std::vector<KeyRange> ranges,
const Version version,
MoveInPhase phase)
MoveInPhase phase,
bool conductBulkLoad)
: id(id), dataMoveId(dataMoveId), ranges(ranges), createVersion(version), highWatermark(version),
phase(static_cast<int8_t>(phase)), startTime(now()) {}
MoveInShardMetaData(const UID& id, const UID& dataMoveId, std::vector<KeyRange> ranges, const Version version)
: MoveInShardMetaData(id, dataMoveId, ranges, version, MoveInPhase::Fetching) {}
MoveInShardMetaData(const UID& dataMoveId, std::vector<KeyRange> ranges, const Version version)
phase(static_cast<int8_t>(phase)), startTime(now()), conductBulkLoad(conductBulkLoad) {}
MoveInShardMetaData(const UID& id,
const UID& dataMoveId,
std::vector<KeyRange> ranges,
const Version version,
bool conductBulkLoad)
: MoveInShardMetaData(id, dataMoveId, ranges, version, MoveInPhase::Fetching, conductBulkLoad) {}
MoveInShardMetaData(const UID& dataMoveId,
std::vector<KeyRange> ranges,
const Version version,
bool conductBulkLoad)
: MoveInShardMetaData(deterministicRandom()->randomUniqueID(),
dataMoveId,
ranges,
version,
MoveInPhase::Fetching) {}
MoveInPhase::Fetching,
conductBulkLoad) {}
bool operator<(const MoveInShardMetaData& rhs) const {
return this->ranges.front().begin < rhs.ranges.front().begin;
@ -77,6 +87,8 @@ struct MoveInShardMetaData {
void setPhase(MoveInPhase phase) { this->phase = static_cast<int8_t>(phase); }
bool doBulkLoading() const { return this->conductBulkLoad; }
uint64_t destShardId() const { return this->dataMoveId.first(); }
std::string destShardIdString() const { return format("%016llx", this->dataMoveId.first()); }
@ -85,12 +97,13 @@ struct MoveInShardMetaData {
" [DataMoveID]: " + this->dataMoveId.toString() +
" [ShardCreateVersion]: " + std::to_string(this->createVersion) + " [ID]: " + this->id.toString() +
" [State]: " + std::to_string(static_cast<int>(this->phase)) +
" [HighWatermark]: " + std::to_string(this->highWatermark);
" [HighWatermark]: " + std::to_string(this->highWatermark) +
" [ConductBulkLoad]: " + std::to_string(this->conductBulkLoad);
}
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, id, dataMoveId, ranges, createVersion, highWatermark, phase, checkpoints);
serializer(ar, id, dataMoveId, ranges, createVersion, highWatermark, phase, checkpoints, conductBulkLoad);
}
};

View File

@ -71,6 +71,7 @@
#include "fdbrpc/Smoother.h"
#include "fdbrpc/Stats.h"
#include "fdbserver/AccumulativeChecksumUtil.h"
#include "fdbserver/BulkLoadUtil.actor.h"
#include "fdbserver/DataDistribution.actor.h"
#include "fdbserver/FDBExecHelper.actor.h"
#include "fdbclient/GetEncryptCipherKeys.h"
@ -212,6 +213,7 @@ static const KeyRangeRef persistPendingCheckpointKeys =
static const std::string serverCheckpointFolder = "serverCheckpoints";
static const std::string checkpointBytesSampleTempFolder = "/metadata_temp";
static const std::string fetchedCheckpointFolder = "fetchedCheckpoints";
static const std::string bulkLoadFolder = "bulkLoadFiles";
// Accumulative checksum related prefix
static const KeyRangeRef persistAccumulativeChecksumKeys =
@ -332,6 +334,7 @@ struct MoveInShard {
std::shared_ptr<MoveInUpdates> updates;
bool isRestored;
Version transferredVersion;
bool conductBulkLoad = false;
Future<Void> fetchClient; // holds FetchShard() actor
Promise<Void> fetchComplete;
@ -340,8 +343,17 @@ struct MoveInShard {
Severity logSev = static_cast<Severity>(SERVER_KNOBS->PHYSICAL_SHARD_MOVE_LOG_SEVERITY);
MoveInShard() = default;
MoveInShard(StorageServer* server, const UID& id, const UID& dataMoveId, const Version version, MoveInPhase phase);
MoveInShard(StorageServer* server, const UID& id, const UID& dataMoveId, const Version version);
MoveInShard(StorageServer* server,
const UID& id,
const UID& dataMoveId,
const Version version,
const bool conductBulkLoad,
MoveInPhase phase);
MoveInShard(StorageServer* server,
const UID& id,
const UID& dataMoveId,
const Version version,
const bool conductBulkLoad);
MoveInShard(StorageServer* server, MoveInShardMetaData meta);
~MoveInShard();
@ -1090,7 +1102,9 @@ public:
void checkTenantEntry(Version version, TenantInfo tenant, bool lockAware);
std::vector<StorageServerShard> getStorageServerShards(KeyRangeRef range);
std::shared_ptr<MoveInShard> getMoveInShard(const UID& dataMoveId, const Version version);
std::shared_ptr<MoveInShard> getMoveInShard(const UID& dataMoveId,
const Version version,
const bool conductBulkLoad);
class CurrentRunningFetchKeys {
std::unordered_map<UID, double> startTimeMap;
@ -2417,7 +2431,9 @@ std::vector<StorageServerShard> StorageServer::getStorageServerShards(KeyRangeRe
return res;
}
std::shared_ptr<MoveInShard> StorageServer::getMoveInShard(const UID& dataMoveId, const Version version) {
std::shared_ptr<MoveInShard> StorageServer::getMoveInShard(const UID& dataMoveId,
const Version version,
const bool conductBulkLoad) {
for (auto& [id, moveInShard] : this->moveInShards) {
if (moveInShard->dataMoveId() == dataMoveId && moveInShard->meta->createVersion == version) {
return moveInShard;
@ -2425,10 +2441,12 @@ std::shared_ptr<MoveInShard> StorageServer::getMoveInShard(const UID& dataMoveId
}
const UID id = deterministicRandom()->randomUniqueID();
std::shared_ptr<MoveInShard> shard = std::make_shared<MoveInShard>(this, id, dataMoveId, version);
std::shared_ptr<MoveInShard> shard = std::make_shared<MoveInShard>(this, id, dataMoveId, version, conductBulkLoad);
auto [it, inserted] = this->moveInShards.emplace(id, shard);
ASSERT(inserted);
TraceEvent(SevDebug, "SSNewMoveInShard", this->thisServerID).detail("MoveInShard", shard->toString());
TraceEvent(SevDebug, "SSNewMoveInShard", this->thisServerID)
.detail("MoveInShard", shard->toString())
.detail("ConductBulkLoad", conductBulkLoad);
return shard;
}
@ -9085,7 +9103,8 @@ void changeServerKeysWithPhysicalShards(StorageServer* data,
bool nowAssigned,
Version version,
ChangeServerKeysContext context,
EnablePhysicalShardMove enablePSM);
EnablePhysicalShardMove enablePSM,
bool conductBulkLoad);
ACTOR Future<Void> fallBackToAddingShard(StorageServer* data, MoveInShard* moveInShard) {
if (moveInShard->getPhase() != MoveInPhase::Fetching && moveInShard->getPhase() != MoveInPhase::Ingesting) {
@ -9111,7 +9130,8 @@ ACTOR Future<Void> fallBackToAddingShard(StorageServer* data, MoveInShard* moveI
true,
mLV.version - 1,
CSK_FALL_BACK,
EnablePhysicalShardMove::False);
EnablePhysicalShardMove::False,
false);
} else {
TraceEvent(SevWarn, "ShardAlreadyChanged", data->thisServerID)
.detail("ShardRange", currentShard->keys)
@ -9124,6 +9144,100 @@ ACTOR Future<Void> fallBackToAddingShard(StorageServer* data, MoveInShard* moveI
return Void();
}
ACTOR Future<Void> fetchShardFetchBulkLoadSSTFiles(StorageServer* data,
MoveInShard* moveInShard,
std::string dir,
BulkLoadState bulkLoadState) {
TraceEvent(SevInfo, "SSBulkLoadTaskFetchSSTFile", data->thisServerID)
.detail("BulkLoadTask", bulkLoadState.toString())
.detail("MoveInShard", moveInShard->toString())
.detail("Folder", abspath(dir));
state double fetchStartTime = now();
// Step 1: Fetch data to dir
state SSBulkLoadFileSet fileSetToLoad;
ASSERT(bulkLoadState.getTransportMethod() != BulkLoadTransportMethod::Invalid);
if (bulkLoadState.getTransportMethod() == BulkLoadTransportMethod::CP) {
wait(store(
fileSetToLoad,
bulkLoadTransportCP_impl(dir, bulkLoadState, SERVER_KNOBS->BULKLOAD_FILE_BYTES_MAX, data->thisServerID)));
} else {
throw not_implemented();
}
// At this point, all necessary data for bulk loading locate at fileSetToLoad
TraceEvent(SevInfo, "SSBulkLoadTaskFetchSSTFileFetched", data->thisServerID)
.detail("BulkLoadTask", bulkLoadState.toString())
.detail("MoveInShard", moveInShard->toString())
.detail("Dir", dir)
.detail("FileSetToLoad", fileSetToLoad.toString());
// Step 2: Validation
// TODO(BulkLoad): Validate all files specified in fileSetToLoad exist
// TODO(BulkLoad): Check file checksum
// TODO(BulkLoad): Check file data all in the moveInShard range
// TODO(BulkLoad): checkContent(fileSetToLoad.dataFileList, data->thisServerID);
if (!fileSetToLoad.bytesSampleFile.present()) {
TraceEvent(SevWarn, "SSBulkLoadTaskFetchSSTFileByteSampleNotFound", data->thisServerID)
.detail("BulkLoadState", bulkLoadState.toString())
.detail("FileSetToLoad", fileSetToLoad.toString());
Optional<std::string> bytesSampleFile_ =
wait(getBytesSamplingFromSSTFiles(fileSetToLoad.folder, fileSetToLoad.dataFileList, data->thisServerID));
fileSetToLoad.bytesSampleFile = bytesSampleFile_;
}
TraceEvent(SevInfo, "SSBulkLoadTaskFetchSSTFileValidated", data->thisServerID)
.detail("BulkLoadTask", bulkLoadState.toString())
.detail("MoveInShard", moveInShard->toString())
.detail("Folder", abspath(dir))
.detail("FileSetToLoad", fileSetToLoad.toString());
// Step 3: Build LocalRecord (used by ShardedRocksDB KVStore)
state CheckpointMetaData localRecord;
localRecord.checkpointID = UID();
localRecord.dir = abspath(fileSetToLoad.folder);
for (const auto& range : moveInShard->ranges()) {
ASSERT(bulkLoadState.getRange().contains(range));
}
localRecord.ranges = moveInShard->ranges();
RocksDBCheckpointKeyValues rcp({ bulkLoadState.getRange() });
for (const auto& filePath : fileSetToLoad.dataFileList) {
CheckpointFile cpFile;
cpFile.path = abspath(filePath);
std::vector<KeyRange> coalesceRanges = coalesceRangeList(moveInShard->ranges());
if (coalesceRanges.size() != 1) {
TraceEvent(SevError, "SSBulkLoadTaskFetchSSTFileError", data->thisServerID)
.detail("Reason", "MoveInShard ranges unexpected")
.detail("BulkLoadState", bulkLoadState.toString())
.detail("MoveInShard", moveInShard->toString())
.detail("FileSetToLoad", fileSetToLoad.toString());
}
// TODO(BulkLoad): set loading file size --- logging purpose
cpFile.range = coalesceRanges[0];
rcp.fetchedFiles.push_back(cpFile);
}
localRecord.serializedCheckpoint = ObjectWriter::toValue(rcp, IncludeVersion());
localRecord.version = 0;
localRecord.bytesSampleFile = fileSetToLoad.bytesSampleFile;
localRecord.setFormat(CheckpointFormat::RocksDBKeyValues);
localRecord.setState(CheckpointMetaData::Complete);
moveInShard->meta->checkpoints.push_back(localRecord);
const double duration = now() - fetchStartTime;
const int64_t totalBytes = getTotalFetchedBytes(moveInShard->meta->checkpoints);
TraceEvent(SevInfo, "SSBulkLoadTaskFetchSSTFileBuildMetadata", data->thisServerID)
.detail("BulkLoadTask", bulkLoadState.toString())
.detail("MoveInShard", moveInShard->toString())
.detail("Folder", abspath(dir))
.detail("FileSetToLoad", fileSetToLoad.toString())
.detail("Duration", duration)
.detail("TotalBytes", totalBytes)
.detail("Rate", (double)totalBytes / duration);
// Step 4: Update the moveInShard phase
moveInShard->setPhase(MoveInPhase::Ingesting);
return Void();
}
ACTOR Future<Void> fetchShardCheckpoint(StorageServer* data, MoveInShard* moveInShard, std::string dir) {
TraceEvent(SevInfo, "FetchShardCheckpointMetaDataBegin", data->thisServerID)
.detail("MoveInShard", moveInShard->toString());
@ -9217,9 +9331,12 @@ ACTOR Future<Void> fetchShardCheckpoint(StorageServer* data, MoveInShard* moveIn
return Void();
}
ACTOR Future<Void> fetchShardIngestCheckpoint(StorageServer* data, MoveInShard* moveInShard) {
ACTOR Future<Void> fetchShardIngestCheckpoint(StorageServer* data,
MoveInShard* moveInShard,
Optional<BulkLoadState> bulkLoadState) {
TraceEvent(SevInfo, "FetchShardIngestCheckpointBegin", data->thisServerID)
.detail("Checkpoints", describe(moveInShard->checkpoints()));
.detail("Checkpoints", describe(moveInShard->checkpoints()))
.detail("BulkLoadTask", bulkLoadState.present() ? bulkLoadState.get().toString() : "");
ASSERT(moveInShard->getPhase() == MoveInPhase::Ingesting);
state double startTime = now();
@ -9231,7 +9348,8 @@ ACTOR Future<Void> fetchShardIngestCheckpoint(StorageServer* data, MoveInShard*
TraceEvent(SevWarn, "FetchShardIngestedCheckpointError", data->thisServerID)
.errorUnsuppressed(e)
.detail("MoveInShard", moveInShard->toString())
.detail("Checkpoints", describe(moveInShard->checkpoints()));
.detail("Checkpoints", describe(moveInShard->checkpoints()))
.detail("BulkLoadTask", bulkLoadState.present() ? bulkLoadState.get().toString() : "");
if (e.code() == error_code_failed_to_restore_checkpoint && !moveInShard->failed()) {
moveInShard->setPhase(MoveInPhase::Fetching);
updateMoveInShardMetaData(data, moveInShard);
@ -9242,7 +9360,8 @@ ACTOR Future<Void> fetchShardIngestCheckpoint(StorageServer* data, MoveInShard*
TraceEvent(SevInfo, "FetchShardIngestedCheckpoint", data->thisServerID)
.detail("MoveInShard", moveInShard->toString())
.detail("Checkpoints", describe(moveInShard->checkpoints()));
.detail("Checkpoints", describe(moveInShard->checkpoints()))
.detail("BulkLoadTask", bulkLoadState.present() ? bulkLoadState.get().toString() : "");
if (moveInShard->failed()) {
return Void();
@ -9260,22 +9379,28 @@ ACTOR Future<Void> fetchShardIngestCheckpoint(StorageServer* data, MoveInShard*
while (reader->hasNext()) {
KeyValue kv = reader->next();
int64_t size = BinaryReader::fromStringRef<int64_t>(kv.value, Unversioned());
KeyRef key = kv.key.removePrefix(persistByteSampleKeys.begin);
Key key = kv.key;
if (key.startsWith(persistByteSampleKeys.begin)) {
key = key.removePrefix(persistByteSampleKeys.begin);
}
if (!checkpoint.containsKey(key)) {
TraceEvent(moveInShard->logSev, "StorageRestoreCheckpointKeySampleNotInRange", data->thisServerID)
.detail("Checkpoint", checkpoint.toString())
.detail("SampleKey", key)
.detail("Size", size);
.detail("Size", size)
.detail("BulkLoadTask", bulkLoadState.present() ? bulkLoadState.get().toString() : "");
continue;
}
TraceEvent(moveInShard->logSev, "StorageRestoreCheckpointKeySample", data->thisServerID)
.detail("Checkpoint", checkpoint.checkpointID.toString())
.detail("SampleKey", key)
.detail("Size", size);
.detail("Size", size)
.detail("BulkLoadTask", bulkLoadState.present() ? bulkLoadState.get().toString() : "");
data->metrics.byteSample.sample.insert(key, size);
data->metrics.notifyBytes(key, size);
data->addMutationToMutationLogOrStorage(invalidVersion,
MutationRef(MutationRef::SetValue, kv.key, kv.value));
data->addMutationToMutationLogOrStorage(
invalidVersion,
MutationRef(MutationRef::SetValue, key.withPrefix(persistByteSampleKeys.begin), kv.value));
}
}
@ -9290,7 +9415,8 @@ ACTOR Future<Void> fetchShardIngestCheckpoint(StorageServer* data, MoveInShard*
.detail("Checkpoints", describe(moveInShard->checkpoints()))
.detail("Bytes", totalBytes)
.detail("Duration", duration)
.detail("Rate", static_cast<double>(totalBytes) / duration);
.detail("Rate", static_cast<double>(totalBytes) / duration)
.detail("BulkLoadTask", bulkLoadState.present() ? bulkLoadState.get().toString() : "");
return Void();
}
@ -9424,6 +9550,9 @@ ACTOR Future<Void> fetchShardApplyUpdates(StorageServer* data,
StorageServerShard newShard = currentShard->toStorageServerShard();
ASSERT(newShard.range == range);
newShard.setShardState(StorageServerShard::ReadWrite);
TraceEvent(SevInfo, "MoveInShardReadWrite", data->thisServerID)
.detail("Version", data->version.get())
.detail("MoveInShard", moveInShard->toString());
data->addShard(ShardInfo::newShard(data, newShard));
data->newestAvailableVersion.insert(range, latestVersion);
coalescePhysicalShards(data, range);
@ -9498,6 +9627,18 @@ ACTOR Future<Void> fetchShard(StorageServer* data, MoveInShard* moveInShard) {
wait(data->fetchKeysParallelismLock.take(TaskPriority::DefaultYield));
state FlowLock::Releaser holdingFKPL(data->fetchKeysParallelismLock);
state Optional<BulkLoadState> bulkLoadState;
if (moveInShard->meta->conductBulkLoad) {
wait(store(bulkLoadState,
getBulkLoadStateFromDataMove(data->cx, moveInShard->dataMoveId(), data->thisServerID)));
}
if (bulkLoadState.present()) {
ASSERT(bulkLoadState.get().getDataMoveId() == moveInShard->dataMoveId());
TraceEvent(SevInfo, "FetchShardBeginReceivedBulkLoadTask", data->thisServerID)
.detail("MoveInShard", moveInShard->toString())
.detail("BulkLoadTask", bulkLoadState.get().toString());
}
loop {
phase = moveInShard->getPhase();
TraceEvent(moveInShard->logSev, "FetchShardLoop", data->thisServerID)
@ -9505,9 +9646,13 @@ ACTOR Future<Void> fetchShard(StorageServer* data, MoveInShard* moveInShard) {
try {
// Pending = 0, Fetching = 1, Ingesting = 2, ApplyingUpdates = 3, Complete = 4, Deleting = 4, Fail = 6,
if (phase == MoveInPhase::Fetching) {
wait(fetchShardCheckpoint(data, moveInShard, dir));
if (bulkLoadState.present()) {
wait(fetchShardFetchBulkLoadSSTFiles(data, moveInShard, dir, bulkLoadState.get()));
} else {
wait(fetchShardCheckpoint(data, moveInShard, dir));
}
} else if (phase == MoveInPhase::Ingesting) {
wait(fetchShardIngestCheckpoint(data, moveInShard));
wait(fetchShardIngestCheckpoint(data, moveInShard, bulkLoadState));
} else if (phase == MoveInPhase::ApplyingUpdates) {
wait(fetchShardApplyUpdates(data, moveInShard, moveInUpdates));
} else if (phase == MoveInPhase::Complete) {
@ -9641,8 +9786,14 @@ MoveInShard::MoveInShard(StorageServer* server,
const UID& id,
const UID& dataMoveId,
const Version version,
const bool conductBulkLoad,
MoveInPhase phase)
: meta(std::make_shared<MoveInShardMetaData>(id, dataMoveId, std::vector<KeyRange>(), version, phase)),
: meta(std::make_shared<MoveInShardMetaData>(id,
dataMoveId,
std::vector<KeyRange>(),
version,
phase,
conductBulkLoad)),
server(server), updates(std::make_shared<MoveInUpdates>(id,
version,
server,
@ -9656,8 +9807,12 @@ MoveInShard::MoveInShard(StorageServer* server,
}
}
MoveInShard::MoveInShard(StorageServer* server, const UID& id, const UID& dataMoveId, const Version version)
: MoveInShard(server, id, dataMoveId, version, MoveInPhase::Fetching) {}
MoveInShard::MoveInShard(StorageServer* server,
const UID& id,
const UID& dataMoveId,
const Version version,
const bool conductBulkLoad)
: MoveInShard(server, id, dataMoveId, version, conductBulkLoad, MoveInPhase::Fetching) {}
MoveInShard::MoveInShard(StorageServer* server, MoveInShardMetaData meta)
: meta(std::make_shared<MoveInShardMetaData>(meta)), server(server),
@ -10205,7 +10360,8 @@ void changeServerKeysWithPhysicalShards(StorageServer* data,
bool nowAssigned,
Version version,
ChangeServerKeysContext context,
EnablePhysicalShardMove enablePSM) {
EnablePhysicalShardMove enablePSM,
bool conductBulkLoad) {
ASSERT(!keys.empty());
const Severity sevDm = static_cast<Severity>(SERVER_KNOBS->PHYSICAL_SHARD_MOVE_LOG_SEVERITY);
TraceEvent(SevInfo, "ChangeServerKeysWithPhysicalShards", data->thisServerID)
@ -10214,6 +10370,7 @@ void changeServerKeysWithPhysicalShards(StorageServer* data,
.detail("NowAssigned", nowAssigned)
.detail("Version", version)
.detail("PhysicalShardMove", static_cast<bool>(enablePSM))
.detail("BulkLoading", static_cast<bool>(conductBulkLoad))
.detail("IsTSS", data->isTss())
.detail("Context", changeServerKeysContextName(context));
@ -10366,7 +10523,8 @@ void changeServerKeysWithPhysicalShards(StorageServer* data,
auto& shard = data->shards[range.begin];
if (!shard->assigned()) {
if (enablePSM) {
std::shared_ptr<MoveInShard> moveInShard = data->getMoveInShard(dataMoveId, cVer);
std::shared_ptr<MoveInShard> moveInShard =
data->getMoveInShard(dataMoveId, cVer, conductBulkLoad);
moveInShard->addRange(range);
updatedMoveInShards.emplace(moveInShard->id(), moveInShard);
updatedShards.push_back(StorageServerShard(
@ -10382,6 +10540,7 @@ void changeServerKeysWithPhysicalShards(StorageServer* data,
.detail("NowAssigned", nowAssigned)
.detail("Version", cVer)
.detail("TotalAssignedAtVer", ++totalAssignedAtVer)
.detail("ConductBulkLoad", conductBulkLoad)
.detail("NewShard", updatedShards.back().toString());
} else {
ASSERT(shard->adding != nullptr || shard->moveInShard != nullptr);
@ -10611,6 +10770,7 @@ private:
DataMovementReason dataMoveReason = DataMovementReason::INVALID;
UID dataMoveId;
bool processedStartKey;
bool conductBulkLoad = false;
KeyRef cacheStartKey;
bool processedCacheStartKey;
@ -10633,9 +10793,11 @@ private:
TraceEvent(SevDebug, "SSSetAssignedStatus", data->thisServerID)
.detail("Range", keys)
.detail("NowAssigned", nowAssigned)
.detail("Version", ver);
.detail("Version", ver)
.detail("EnablePSM", enablePSM)
.detail("ConductBulkLoad", conductBulkLoad);
changeServerKeysWithPhysicalShards(
data, keys, dataMoveId, nowAssigned, currentVersion - 1, context, enablePSM);
data, keys, dataMoveId, nowAssigned, currentVersion - 1, context, enablePSM, conductBulkLoad);
} else {
// add changes in shard assignment to the mutation log
setAssignedStatus(data, keys, nowAssigned);
@ -10666,7 +10828,12 @@ private:
dataMoveType = DataMoveType::LOGICAL;
}
enablePSM = EnablePhysicalShardMove(dataMoveType == DataMoveType::PHYSICAL ||
(dataMoveType == DataMoveType::PHYSICAL_EXP && data->isTss()));
(dataMoveType == DataMoveType::PHYSICAL_EXP && data->isTss()) ||
dataMoveType == DataMoveType::PHYSICAL_BULKLOAD);
conductBulkLoad =
dataMoveType == DataMoveType::LOGICAL_BULKLOAD || dataMoveType == DataMoveType::PHYSICAL_BULKLOAD;
// TODO(BulkLoad): remove after logical move based bulk loading has been implmented
ASSERT(enablePSM || !conductBulkLoad);
processedStartKey = true;
} else if (m.type == MutationRef::SetValue && m.param1 == lastEpochEndPrivateKey) {
// lastEpochEnd transactions are guaranteed by the master to be alone in their own batch (version)

View File

@ -0,0 +1,675 @@
/*
* BulkLoading.actor.cpp
*
* This source file is part of the FoundationDB open source project
*
* Copyright 2013-2024 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/BulkLoading.h"
#include "fdbclient/ManagementAPI.actor.h"
#include "fdbclient/NativeAPI.actor.h"
#include "fdbserver/BulkLoadUtil.actor.h"
#include "fdbserver/RocksDBCheckpointUtils.actor.h"
#include "fdbserver/StorageMetrics.actor.h"
#include "fdbserver/workloads/workloads.actor.h"
#include "flow/actorcompiler.h" // This must be the last #include.
const std::string simulationBulkLoadFolderForSimpleTest = "bulkLoadSimple";
const std::string simulationBulkLoadFolderForLargeDataProduce = "bulkLoadLargeData";
const std::string simulationBulkLoadFolderForComplexTest = "bulkLoadComplex";
struct BulkLoadTaskTestUnit {
BulkLoadState bulkLoadTask;
std::vector<KeyValue> data;
BulkLoadTaskTestUnit() = default;
};
struct BulkLoading : TestWorkload {
static constexpr auto NAME = "BulkLoadingWorkload";
const bool enabled;
bool pass;
// This workload is not compatible with following workload because they will race in changing the DD mode
void disableFailureInjectionWorkloads(std::set<std::string>& out) const override {
out.insert({ "RandomMoveKeys",
"DataLossRecovery",
"IDDTxnProcessorApiCorrectness",
"PerpetualWiggleStatsWorkload",
"PhysicalShardMove",
"StorageCorruption",
"StorageServerCheckpointRestoreTest",
"ValidateStorage" });
}
BulkLoading(WorkloadContext const& wcx) : TestWorkload(wcx), enabled(true), pass(true) {}
Future<Void> setup(Database const& cx) override { return Void(); }
Future<Void> start(Database const& cx) override { return _start(this, cx); }
Future<bool> check(Database const& cx) override { return true; }
void getMetrics(std::vector<PerfMetric>& m) override {}
ACTOR Future<Void> submitBulkLoadTasks(BulkLoading* self, Database cx, std::vector<BulkLoadState> tasks) {
state int i = 0;
for (; i < tasks.size(); i++) {
loop {
try {
wait(submitBulkLoadTask(cx, tasks[i]));
TraceEvent("BulkLoadingSubmitBulkLoadTask")
.setMaxEventLength(-1)
.setMaxFieldLength(-1)
.detail("BulkLoadStates", describe(tasks));
break;
} catch (Error& e) {
TraceEvent("BulkLoadingSubmitBulkLoadTaskError")
.setMaxEventLength(-1)
.setMaxFieldLength(-1)
.errorUnsuppressed(e)
.detail("BulkLoadStates", describe(tasks));
wait(delay(0.1));
}
}
}
return Void();
}
ACTOR Future<Void> acknowledgeBulkLoadTasks(BulkLoading* self, Database cx, std::vector<BulkLoadState> tasks) {
state int i = 0;
for (; i < tasks.size(); i++) {
loop {
try {
wait(acknowledgeBulkLoadTask(cx, tasks[i].getRange(), tasks[i].getTaskId()));
TraceEvent("BulkLoadingAcknowledgeBulkLoadTask")
.setMaxEventLength(-1)
.setMaxFieldLength(-1)
.detail("BulkLoadStates", describe(tasks[i]));
break;
} catch (Error& e) {
TraceEvent("BulkLoadingAcknowledgeBulkLoadTaskError")
.setMaxEventLength(-1)
.setMaxFieldLength(-1)
.errorUnsuppressed(e)
.detail("BulkLoadStates", describe(tasks));
wait(delay(0.1));
}
}
}
return Void();
}
Key getRandomKey(const std::vector<Key>& keyCharList, size_t keySizeMin, size_t keySizeMax) {
Key key = ""_sr;
int keyLength = deterministicRandom()->randomInt(keySizeMin, keySizeMax);
for (int j = 0; j < keyLength; j++) {
Key appendedItem = deterministicRandom()->randomChoice(keyCharList);
key = key.withSuffix(appendedItem);
}
return key;
}
std::vector<KeyValue> generateRandomData(KeyRange range, size_t count, const std::vector<Key>& keyCharList) {
std::set<Key> keys;
while (keys.size() < count) {
Key key = getRandomKey(keyCharList, 1, 1000);
if (!range.contains(key)) {
continue;
}
keys.insert(key);
}
std::vector<KeyValue> res;
for (const auto& key : keys) {
UID randomId = deterministicRandom()->randomUniqueID();
Value val = Standalone(StringRef(randomId.toString()));
res.push_back(Standalone(KeyValueRef(key, val)));
}
ASSERT(res.size() == count);
return res;
}
void produceFilesToLoad(BulkLoadTaskTestUnit task) {
std::string folder = task.bulkLoadTask.getFolder();
platform::eraseDirectoryRecursive(folder);
ASSERT(platform::createDirectory(folder));
std::string bytesSampleFile = task.bulkLoadTask.getBytesSampleFile().get();
std::string dataFile = *(task.bulkLoadTask.getDataFiles().begin());
std::unique_ptr<IRocksDBSstFileWriter> sstWriter = newRocksDBSstFileWriter();
sstWriter->open(abspath(dataFile));
std::vector<KeyValue> bytesSample;
for (const auto& kv : task.data) {
ByteSampleInfo sampleInfo = isKeyValueInSample(kv);
if (sampleInfo.inSample) {
Key sampleKey = kv.key;
Value sampleValue = BinaryWriter::toValue(sampleInfo.sampledSize, Unversioned());
bytesSample.push_back(Standalone(KeyValueRef(sampleKey, sampleValue)));
}
sstWriter->write(kv.key, kv.value);
}
TraceEvent("BulkLoadingDataProduced")
.detail("LoadKeyCount", task.data.size())
.detail("BytesSampleSize", bytesSample.size())
.detail("Folder", folder)
.detail("DataFile", dataFile)
.detail("BytesSampleFile", bytesSampleFile);
ASSERT(sstWriter->finish());
if (bytesSample.size() > 0) {
sstWriter->open(abspath(bytesSampleFile));
for (const auto& kv : bytesSample) {
sstWriter->write(kv.key, kv.value);
}
TraceEvent("BulkLoadingByteSampleProduced")
.detail("LoadKeyCount", task.data.size())
.detail("BytesSampleSize", bytesSample.size())
.detail("Folder", folder)
.detail("DataFile", dataFile)
.detail("BytesSampleFile", bytesSampleFile);
ASSERT(sstWriter->finish());
}
TraceEvent("BulkLoadingProduceDataToLoad").detail("Folder", folder).detail("LoadKeyCount", task.data.size());
return;
}
ACTOR Future<bool> checkDDEnabled(Database cx) {
loop {
state Transaction tr(cx);
tr.setOption(FDBTransactionOptions::READ_SYSTEM_KEYS);
try {
state int ddMode = 1;
Optional<Value> mode = wait(tr.get(dataDistributionModeKey));
if (mode.present()) {
BinaryReader rd(mode.get(), Unversioned());
rd >> ddMode;
}
return ddMode == 1;
} catch (Error& e) {
wait(tr.onError(e));
}
}
}
ACTOR Future<bool> allComplete(Database cx) {
state Transaction tr(cx);
state Key beginKey = allKeys.begin;
state Key endKey = allKeys.end;
while (beginKey < endKey) {
try {
tr.setOption(FDBTransactionOptions::READ_SYSTEM_KEYS);
RangeResult res = wait(krmGetRanges(&tr,
bulkLoadPrefix,
Standalone(KeyRangeRef(beginKey, endKey)),
CLIENT_KNOBS->KRM_GET_RANGE_LIMIT,
CLIENT_KNOBS->KRM_GET_RANGE_LIMIT_BYTES));
for (int i = 0; i < res.size() - 1; i++) {
if (!res[i].value.empty()) {
BulkLoadState bulkLoadState = decodeBulkLoadState(res[i].value);
ASSERT(bulkLoadState.isValid());
if (bulkLoadState.getRange() != KeyRangeRef(res[i].key, res[i + 1].key)) {
continue; // Ignore outdated task
}
if (bulkLoadState.phase != BulkLoadPhase::Complete) {
TraceEvent("BulkLoadingWorkLoadIncompleteTasks")
.setMaxEventLength(-1)
.setMaxFieldLength(-1)
.detail("Task", bulkLoadState.toString());
return false;
}
}
}
beginKey = res[res.size() - 1].key;
} catch (Error& e) {
wait(tr.onError(e));
}
}
return true;
}
ACTOR Future<Void> waitUntilAllComplete(BulkLoading* self, Database cx) {
loop {
bool complete = wait(self->allComplete(cx));
if (complete) {
break;
}
wait(delay(10.0));
}
return Void();
}
bool checkData(std::vector<KeyValue> kvs, std::vector<KeyValue> kvsdb) {
if (kvs.size() != kvsdb.size()) {
TraceEvent(SevError, "BulkLoadingWorkLoadDataWrong")
.detail("Reason", "KeyValue count wrong")
.detail("KVS", kvs.size())
.detail("DB", kvsdb.size());
return false;
}
std::sort(kvs.begin(), kvs.end(), [](KeyValue a, KeyValue b) { return a.key < b.key; });
std::sort(kvsdb.begin(), kvsdb.end(), [](KeyValue a, KeyValue b) { return a.key < b.key; });
for (int i = 0; i < kvs.size(); i++) {
if (kvs[i].key != kvsdb[i].key) {
TraceEvent(SevError, "BulkLoadingWorkLoadDataWrong")
.detail("Reason", "Key mismatch")
.detail("KVS", kvs[i])
.detail("DB", kvsdb[i]);
return false;
} else if (kvs[i].value != kvsdb[i].value) {
TraceEvent(SevError, "BulkLoadingWorkLoadDataWrong")
.detail("Reason", "Value mismatch")
.detail("KVS", kvs[i])
.detail("DB", kvsdb[i]);
return false;
}
}
return true;
}
bool keyIsIgnored(Key key, const std::vector<KeyRange>& ignoreRanges) {
for (const auto& range : ignoreRanges) {
if (range.contains(key)) {
return true;
}
}
return false;
}
ACTOR Future<std::vector<KeyValue>> getKvsFromDB(BulkLoading* self,
Database cx,
std::vector<KeyRange> ignoreRanges) {
state std::vector<KeyValue> res;
state Transaction tr(cx);
TraceEvent("BulkLoadingWorkLoadGetKVSFromDBStart");
loop {
try {
RangeResult result = wait(tr.getRange(normalKeys, CLIENT_KNOBS->TOO_MANY));
ASSERT(!result.more);
for (int i = 0; i < result.size(); i++) {
if (!self->keyIsIgnored(result[i].key, ignoreRanges)) {
res.push_back(Standalone(KeyValueRef(result[i].key, result[i].value)));
}
}
break;
} catch (Error& e) {
wait(tr.onError(e));
}
}
TraceEvent("BulkLoadingWorkLoadGetKVSFromDBDone");
return res;
}
BulkLoadTaskTestUnit produceBulkLoadTaskUnit(BulkLoading* self,
const std::vector<Key>& keyCharList,
KeyRange range,
std::string folderName) {
std::string dataFileName = generateRandomBulkLoadDataFileName();
std::string bytesSampleFileName = generateRandomBulkLoadBytesSampleFileName();
std::string folder = joinPath(simulationBulkLoadFolderForSimpleTest, folderName);
BulkLoadTaskTestUnit taskUnit;
taskUnit.bulkLoadTask = newBulkLoadTaskLocalSST(
range, folder, joinPath(folder, dataFileName), joinPath(folder, bytesSampleFileName));
size_t dataSize = deterministicRandom()->randomInt(10, 100);
taskUnit.data = self->generateRandomData(range, dataSize, keyCharList);
self->produceFilesToLoad(taskUnit);
return taskUnit;
}
std::vector<KeyValue> generateSortedKVS(StringRef prefix, size_t count) {
std::vector<KeyValue> res;
for (int i = 0; i < count; i++) {
UID keyId = deterministicRandom()->randomUniqueID();
Value key = Standalone(StringRef(keyId.toString())).withPrefix(prefix);
UID valueId = deterministicRandom()->randomUniqueID();
Value val = Standalone(StringRef(valueId.toString()));
res.push_back(Standalone(KeyValueRef(key, val)));
}
std::sort(res.begin(), res.end(), [](KeyValue a, KeyValue b) { return a.key < b.key; });
return res;
}
ACTOR Future<bool> checkBulkLoadMetadataCleared(BulkLoading* self, Database cx) {
state Key beginKey = allKeys.begin;
state Key endKey = allKeys.end;
state KeyRange rangeToRead;
while (beginKey < endKey) {
state Transaction tr(cx);
try {
tr.setOption(FDBTransactionOptions::READ_SYSTEM_KEYS);
rangeToRead = Standalone(KeyRangeRef(beginKey, endKey));
RangeResult res = wait(krmGetRanges(&tr,
bulkLoadPrefix,
allKeys,
CLIENT_KNOBS->KRM_GET_RANGE_LIMIT,
CLIENT_KNOBS->KRM_GET_RANGE_LIMIT_BYTES));
beginKey = res.back().key;
int emptyCount = 0;
int nonEmptyCount = 0;
for (int i = 0; i < res.size() - 1; i++) {
if (!res[i].value.empty()) {
BulkLoadState bulkLoadState = decodeBulkLoadState(res[i].value);
KeyRange currentRange = Standalone(KeyRangeRef(res[i].key, res[i + 1].key));
if (bulkLoadState.getRange() == currentRange) {
TraceEvent("BulkLoadingWorkLoadMetadataNotCleared")
.setMaxEventLength(-1)
.setMaxFieldLength(-1)
.detail("BulkLoadTask", bulkLoadState.toString());
return false;
} else {
ASSERT(bulkLoadState.getRange().contains(currentRange));
}
nonEmptyCount++;
} else {
emptyCount++;
}
}
ASSERT(emptyCount - 1 - 1 <= nonEmptyCount);
break;
} catch (Error& e) {
wait(tr.onError(e));
}
}
return true;
}
// Issue three non-overlapping tasks and check data consistency and correctness
// Repeat twice
ACTOR Future<Void> simpleTest(BulkLoading* self, Database cx) {
TraceEvent("BulkLoadingWorkLoadSimpleTestBegin");
state std::vector<Key> keyCharList = { "0"_sr, "1"_sr, "2"_sr, "3"_sr, "4"_sr, "5"_sr };
// First round of issuing tasks
state std::vector<BulkLoadState> bulkLoadStates;
state std::vector<std::vector<KeyValue>> bulkLoadDataList;
for (int i = 0; i < 3; i++) {
std::string strIdx = std::to_string(i);
std::string strIdxPlusOne = std::to_string(i + 1);
std::string folderName = strIdx;
Key beginKey = Standalone(StringRef(strIdx));
Key endKey = Standalone(StringRef(strIdxPlusOne));
KeyRange range = Standalone(KeyRangeRef(beginKey, endKey));
BulkLoadTaskTestUnit taskUnit = self->produceBulkLoadTaskUnit(self, keyCharList, range, folderName);
bulkLoadStates.push_back(taskUnit.bulkLoadTask);
bulkLoadDataList.push_back(taskUnit.data);
}
wait(self->submitBulkLoadTasks(self, cx, bulkLoadStates));
TraceEvent("BulkLoadingWorkLoadSimpleTestIssuedTasks");
int old1 = wait(setBulkLoadMode(cx, 1));
TraceEvent("BulkLoadingWorkLoadSimpleTestSetMode").detail("OldMode", old1).detail("NewMode", 1);
wait(self->waitUntilAllComplete(self, cx));
TraceEvent("BulkLoadingWorkLoadSimpleTestAllComplete");
// Second round of issuing tasks
bulkLoadStates.clear();
bulkLoadDataList.clear();
for (int i = 0; i < 3; i++) {
std::string strIdx = std::to_string(i);
std::string strIdxPlusOne = std::to_string(i + 1);
std::string folderName = strIdx;
Key beginKey = Standalone(StringRef(strIdx));
Key endKey = Standalone(StringRef(strIdxPlusOne));
KeyRange range = Standalone(KeyRangeRef(beginKey, endKey));
BulkLoadTaskTestUnit taskUnit = self->produceBulkLoadTaskUnit(self, keyCharList, range, folderName);
bulkLoadStates.push_back(taskUnit.bulkLoadTask);
bulkLoadDataList.push_back(taskUnit.data);
}
wait(self->submitBulkLoadTasks(self, cx, bulkLoadStates));
TraceEvent("BulkLoadingWorkLoadSimpleTestIssuedTasks");
wait(self->waitUntilAllComplete(self, cx));
TraceEvent("BulkLoadingWorkLoadSimpleTestAllComplete");
// Check data
int old2 = wait(setBulkLoadMode(cx, 0));
TraceEvent("BulkLoadingWorkLoadSimpleTestSetMode").detail("OldMode", old2).detail("NewMode", 0);
state std::vector<KeyValue> dbkvs = wait(self->getKvsFromDB(self, cx, std::vector<KeyRange>()));
state std::vector<KeyValue> kvs;
for (int j = 0; j < bulkLoadDataList.size(); j++) {
kvs.insert(kvs.end(), bulkLoadDataList[j].begin(), bulkLoadDataList[j].end());
}
ASSERT(self->checkData(kvs, dbkvs));
// Check bulk load metadata
int old3 = wait(setBulkLoadMode(cx, 1));
TraceEvent("BulkLoadingWorkLoadSimpleTestSetMode").detail("OldMode", old3).detail("NewMode", 1);
wait(self->acknowledgeBulkLoadTasks(self, cx, bulkLoadStates));
loop {
bool cleared = wait(self->checkBulkLoadMetadataCleared(self, cx));
if (cleared) {
break;
}
wait(delay(1.0));
}
TraceEvent("BulkLoadingWorkLoadSimpleTestComplete");
return Void();
}
std::string getStringWithFixedLength(int number, int length) {
std::string numStr = std::to_string(number);
int zeroCount = length - numStr.size();
std::string res;
for (int i = 0; i < zeroCount; i++) {
res = res + "0";
}
res = res + numStr;
return res;
}
BulkLoadTaskTestUnit produceRandomBulkLoadTaskUnit(BulkLoading* self, std::string rootPath, int index) {
std::string dataFileName = generateRandomBulkLoadDataFileName();
std::string bytesSampleFileName = generateRandomBulkLoadBytesSampleFileName();
std::string randomKey1 = deterministicRandom()->randomUniqueID().toString();
std::string randomKey2 = deterministicRandom()->randomUniqueID().toString();
while (randomKey2 == randomKey1) {
randomKey2 = deterministicRandom()->randomUniqueID().toString();
}
StringRef randomKeyRef1 = StringRef(randomKey1);
StringRef randomKeyRef2 = StringRef(randomKey2);
StringRef firstKey = randomKeyRef1 < randomKeyRef2 ? randomKeyRef1 : randomKeyRef2;
StringRef lastKey = randomKeyRef1 < randomKeyRef2 ? randomKeyRef2 : randomKeyRef1;
KeyRange range = Standalone(KeyRangeRef(firstKey, lastKey.withSuffix("\xff"_sr)));
std::string folderName = getStringWithFixedLength(index, 6);
std::string folder = joinPath(rootPath, folderName);
BulkLoadTaskTestUnit taskUnit;
taskUnit.data.push_back(Standalone(KeyValueRef(firstKey, firstKey)));
std::set<std::string> middleKeys;
int keyCount = deterministicRandom()->randomInt(1, 20);
for (int i = 0; i < keyCount; i++) {
middleKeys.insert(deterministicRandom()->randomUniqueID().toString());
}
for (const auto& middleKey : middleKeys) {
Key key = firstKey.withSuffix(middleKey);
taskUnit.data.push_back(Standalone(KeyValueRef(key, key)));
}
taskUnit.data.push_back(Standalone(KeyValueRef(lastKey, lastKey)));
taskUnit.bulkLoadTask = newBulkLoadTaskLocalSST(
range, folder, joinPath(folder, dataFileName), joinPath(folder, bytesSampleFileName));
self->produceFilesToLoad(taskUnit);
return taskUnit;
}
ACTOR Future<Void> complexTest(BulkLoading* self, Database cx) {
int old1 = wait(setBulkLoadMode(cx, 1));
TraceEvent("BulkLoadingWorkLoadComplexTestSetMode").detail("OldMode", old1).detail("NewMode", 1);
// Issue tasks
state KeyRangeMap<Optional<BulkLoadTaskTestUnit>> taskMap;
taskMap.insert(allKeys, Optional<BulkLoadTaskTestUnit>());
state int i = 0;
state int n = deterministicRandom()->randomInt(5, 10);
state int frequencyFactorForWaitAll = std::max(2, (int)(n * deterministicRandom()->random01()));
state int frequencyFactorForSwitchMode = std::max(2, (int)(n * deterministicRandom()->random01()));
for (; i < n; i++) {
state BulkLoadTaskTestUnit taskUnit =
self->produceRandomBulkLoadTaskUnit(self, simulationBulkLoadFolderForComplexTest, i);
taskMap.insert(taskUnit.bulkLoadTask.getRange(), taskUnit);
if (deterministicRandom()->coinflip()) {
wait(delay(deterministicRandom()->random01() * 10));
}
wait(self->submitBulkLoadTasks(self, cx, { taskUnit.bulkLoadTask }));
if (i % frequencyFactorForWaitAll == 0 && deterministicRandom()->coinflip()) {
wait(self->waitUntilAllComplete(self, cx));
}
if (i % frequencyFactorForSwitchMode == 0 && deterministicRandom()->coinflip()) {
int old2 = wait(setBulkLoadMode(cx, 0));
TraceEvent("BulkLoadingWorkLoadComplexTestSetMode").detail("OldMode", old2).detail("NewMode", 0);
wait(delay(deterministicRandom()->random01() * 5));
int old3 = wait(setBulkLoadMode(cx, 1));
TraceEvent("BulkLoadingWorkLoadComplexTestSetMode").detail("OldMode", old3).detail("NewMode", 1);
}
}
// Wait until all tasks have completed
wait(self->waitUntilAllComplete(self, cx));
int old4 = wait(setBulkLoadMode(cx, 0)); // trigger DD restart
TraceEvent("BulkLoadingWorkLoadComplexTestSetMode").detail("OldMode", old4).detail("NewMode", 0);
// Check correctness
state std::vector<KeyValue> kvs;
state std::vector<BulkLoadState> bulkLoadStates;
state std::vector<KeyRange> incompleteRanges;
for (auto& range : taskMap.ranges()) {
if (!range.value().present()) {
continue;
}
if (range.value().get().bulkLoadTask.getRange() != range.range()) {
ASSERT(range.value().get().bulkLoadTask.getRange().contains(range.range()));
incompleteRanges.push_back(range.range());
continue; // outdated
}
std::vector<KeyValue> kvsToCheck = range.value().get().data;
kvs.insert(std::end(kvs), std::begin(kvsToCheck), std::end(kvsToCheck));
bulkLoadStates.push_back(range.value().get().bulkLoadTask);
}
std::vector<KeyValue> dbkvs = wait(self->getKvsFromDB(self, cx, incompleteRanges));
ASSERT(self->checkData(kvs, dbkvs));
// Clear metadata
int old5 = wait(setBulkLoadMode(cx, 1));
TraceEvent("BulkLoadingWorkLoadComplexTestSetMode").detail("OldMode", old5).detail("NewMode", 1);
wait(self->acknowledgeBulkLoadTasks(self, cx, bulkLoadStates));
loop {
bool cleared = wait(self->checkBulkLoadMetadataCleared(self, cx));
if (cleared) {
break;
}
wait(delay(1.0));
}
TraceEvent("BulkLoadingWorkLoadComplexTestComplete");
return Void();
}
void produceLargeDataToLoad(BulkLoadTaskTestUnit task, int count) {
std::string folder = task.bulkLoadTask.getFolder();
platform::eraseDirectoryRecursive(folder);
ASSERT(platform::createDirectory(folder));
std::string bytesSampleFile = task.bulkLoadTask.getBytesSampleFile().get();
std::string dataFile = *(task.bulkLoadTask.getDataFiles().begin());
std::unique_ptr<IRocksDBSstFileWriter> sstWriter = newRocksDBSstFileWriter();
sstWriter->open(abspath(dataFile));
std::vector<KeyValue> bytesSample;
int insertedKeyCount = 0;
for (int i = 0; i < 10; i++) {
std::string idxStr = std::to_string(i);
Key prefix = Standalone(StringRef(idxStr)).withPrefix(task.bulkLoadTask.getRange().begin);
std::vector<KeyValue> kvs = generateSortedKVS(prefix, std::max(count / 10, 1));
for (const auto& kv : kvs) {
ByteSampleInfo sampleInfo = isKeyValueInSample(kv);
if (sampleInfo.inSample) {
Key sampleKey = kv.key;
Value sampleValue = BinaryWriter::toValue(sampleInfo.sampledSize, Unversioned());
bytesSample.push_back(Standalone(KeyValueRef(sampleKey, sampleValue)));
}
sstWriter->write(kv.key, kv.value);
insertedKeyCount++;
}
}
TraceEvent("BulkLoadingDataProduced")
.detail("LoadKeyCount", insertedKeyCount)
.detail("BytesSampleSize", bytesSample.size())
.detail("Folder", folder)
.detail("DataFile", dataFile)
.detail("BytesSampleFile", bytesSampleFile);
ASSERT(sstWriter->finish());
if (bytesSample.size() > 0) {
sstWriter->open(abspath(bytesSampleFile));
for (const auto& kv : bytesSample) {
sstWriter->write(kv.key, kv.value);
}
TraceEvent("BulkLoadingByteSampleProduced")
.detail("LoadKeyCount", task.data.size())
.detail("BytesSampleSize", bytesSample.size())
.detail("Folder", folder)
.detail("DataFile", dataFile)
.detail("BytesSampleFile", bytesSampleFile);
ASSERT(sstWriter->finish());
}
}
void produceDataSet(BulkLoading* self, KeyRange range, std::string folderName) {
std::string dataFileName =
range.begin.toString() + "_" + range.end.toString() + "_" + generateRandomBulkLoadDataFileName();
std::string bytesSampleFileName =
range.begin.toString() + "_" + range.end.toString() + "_" + generateRandomBulkLoadBytesSampleFileName();
std::string folder = joinPath(simulationBulkLoadFolderForLargeDataProduce, folderName);
BulkLoadTaskTestUnit taskUnit;
taskUnit.bulkLoadTask = newBulkLoadTaskLocalSST(
range, folder, joinPath(folder, dataFileName), joinPath(folder, bytesSampleFileName));
self->produceLargeDataToLoad(taskUnit, 5000000);
return;
}
void produceLargeData(BulkLoading* self, Database cx) {
std::string folderName1 = "1";
KeyRange range1 = Standalone(KeyRangeRef("1"_sr, "2"_sr));
self->produceDataSet(self, range1, folderName1);
std::string folderName2 = "2";
KeyRange range2 = Standalone(KeyRangeRef("2"_sr, "3"_sr));
self->produceDataSet(self, range2, folderName2);
std::string folderName3 = "3";
KeyRange range3 = Standalone(KeyRangeRef("3"_sr, "4"_sr));
self->produceDataSet(self, range3, folderName3);
return;
}
ACTOR Future<Void> _start(BulkLoading* self, Database cx) {
if (self->clientId != 0) {
return Void();
}
if (g_network->isSimulated()) {
// Network partition between CC and DD can cause DD no longer existing,
// which results in the bulk loading task cannot complete
// So, this workload disable the network partition
disableConnectionFailures("BulkLoading");
}
if (deterministicRandom()->coinflip()) {
// Inject data to three non-overlapping ranges
wait(self->simpleTest(self, cx));
} else {
// Inject data to many ranges and those ranges can be overlapping
wait(self->complexTest(self, cx));
}
// self->produceLargeData(self, cx); // Produce data set that is used in loop back cluster test
return Void();
}
};
WorkloadFactory<BulkLoading> BulkLoadingFactory;

View File

@ -250,7 +250,8 @@ struct DataLossRecoveryWorkload : TestWorkload {
false,
UID(), // for logging only
&ddEnabledState,
CancelConflictingDataMoves::True);
CancelConflictingDataMoves::True,
Optional<BulkLoadState>());
} else {
UID dataMoveId = newDataMoveId(deterministicRandom()->randomUInt64(),
AssignEmptyRange(false),
@ -268,7 +269,8 @@ struct DataLossRecoveryWorkload : TestWorkload {
false,
UID(), // for logging only
&ddEnabledState,
CancelConflictingDataMoves::True);
CancelConflictingDataMoves::True,
Optional<BulkLoadState>());
}
wait(moveKeys(cx, *params));
break;

View File

@ -367,7 +367,8 @@ struct IDDTxnProcessorApiWorkload : TestWorkload {
false,
UID(),
self->ddContext.ddEnabledState.get(),
CancelConflictingDataMoves::True);
CancelConflictingDataMoves::True,
Optional<BulkLoadState>());
} else {
return MoveKeysParams(dataMoveId,
keys,
@ -380,7 +381,8 @@ struct IDDTxnProcessorApiWorkload : TestWorkload {
false,
UID(),
self->ddContext.ddEnabledState.get(),
CancelConflictingDataMoves::True);
CancelConflictingDataMoves::True,
Optional<BulkLoadState>());
}
}

View File

@ -33,6 +33,7 @@ public:
PromiseStream<GetMetricsListRequest> getShardMetricsList;
PromiseStream<Promise<int64_t>> getAverageShardBytes;
PromiseStream<RebalanceStorageQueueRequest> triggerStorageQueueRebalance;
PromiseStream<BulkLoadShardRequest> triggerShardBulkLoading;
KeyRangeMap<ShardTrackedData> shards;
@ -95,6 +96,8 @@ public:
mock->getInitialDataDistribution(ddcx.id(), ddcx.lock, {}, ddcx.ddEnabledState.get(), SkipDDModeCheck::True)
.get();
Reference<PhysicalShardCollection> physicalShardCollection = makeReference<PhysicalShardCollection>();
Reference<BulkLoadTaskCollection> bulkLoadTaskCollection =
makeReference<BulkLoadTaskCollection>(ddcx.id(), SERVER_KNOBS->DD_BULKLOAD_PARALLELISM);
Reference<AsyncVar<bool>> zeroHealthyTeams = makeReference<AsyncVar<bool>>(false);
shardTracker = makeReference<DataDistributionTracker>(
@ -104,6 +107,7 @@ public:
.output = output,
.shardsAffectedByTeamFailure = ddcx.shardsAffectedByTeamFailure,
.physicalShardCollection = physicalShardCollection,
.bulkLoadTaskCollection = bulkLoadTaskCollection,
.anyZeroHealthyTeams = zeroHealthyTeams,
.shards = &shards,
.trackerCancelled = &ddcx.trackerCancelled,
@ -114,7 +118,8 @@ public:
getTopKMetrics.getFuture(),
getShardMetricsList.getFuture(),
getAverageShardBytes.getFuture(),
triggerStorageQueueRebalance.getFuture()));
triggerStorageQueueRebalance.getFuture(),
triggerShardBulkLoading.getFuture()));
actors.add(relocateShardReporter(this, output.getFuture()));

View File

@ -616,7 +616,8 @@ struct PhysicalShardMoveWorkLoad : TestWorkload {
false,
deterministicRandom()->randomUniqueID(), // for logging only
&ddEnabledState,
CancelConflictingDataMoves::False)));
CancelConflictingDataMoves::False,
Optional<BulkLoadState>())));
break;
} catch (Error& e) {
if (e.code() == error_code_movekeys_conflict) {

View File

@ -174,7 +174,8 @@ struct MoveKeysWorkload : FailureInjectionWorkload {
false,
relocateShardInterval.pairID,
&ddEnabledState,
CancelConflictingDataMoves::True);
CancelConflictingDataMoves::True,
Optional<BulkLoadState>());
} else {
UID dataMoveId = newDataMoveId(deterministicRandom()->randomUInt64(),
AssignEmptyRange(false),
@ -192,7 +193,8 @@ struct MoveKeysWorkload : FailureInjectionWorkload {
false,
relocateShardInterval.pairID,
&ddEnabledState,
CancelConflictingDataMoves::True);
CancelConflictingDataMoves::True,
Optional<BulkLoadState>());
}
wait(moveKeys(cx, *params));
TraceEvent(relocateShardInterval.end()).detail("Result", "Success");

View File

@ -3052,7 +3052,7 @@ size_t readFileBytes(std::string const& filename, uint8_t* buff, size_t len) {
return bytesRead;
}
std::string readFileBytes(std::string const& filename, int maxSize) {
std::string readFileBytes(std::string const& filename, size_t maxSize) {
if (!fileExists(filename)) {
TraceEvent("ReadFileBytes_FileNotFound").detail("Filename", filename);
throw file_not_found();
@ -3060,7 +3060,10 @@ std::string readFileBytes(std::string const& filename, int maxSize) {
size_t size = fileSize(filename);
if (size > maxSize) {
TraceEvent("ReadFileBytes_FileTooLarge").detail("Filename", filename);
TraceEvent("ReadFileBytes_FileTooLarge")
.detail("FileSize", size)
.detail("InputMaxSize", maxSize)
.detail("Filename", filename);
throw file_too_large();
}

View File

@ -343,14 +343,14 @@ void atomicReplace(std::string const& path, std::string const& content, bool tex
// Read a file into memory
// This requires the file to be seekable
std::string readFileBytes(std::string const& filename, int maxSize);
std::string readFileBytes(std::string const& filename, size_t maxSize);
// Read a file into memory supplied by the caller
// If 'len' is greater than file size, then read the filesize bytes.
size_t readFileBytes(std::string const& filename, uint8_t* buff, int64_t len);
size_t readFileBytes(std::string const& filename, uint8_t* buff, size_t len);
// Write data buffer into file
void writeFileBytes(std::string const& filename, const char* data, size_t count);
void writeFileBytes(std::string const& filename, const uint8_t* data, size_t count);
// Write text into file
void writeFile(std::string const& filename, std::string const& content);

View File

@ -154,6 +154,8 @@ ERROR( audit_storage_task_outdated, 1234, "Audit task is scheduled by an outdate
ERROR( transaction_throttled_hot_shard, 1235, "Transaction throttled due to hot shard" )
ERROR( storage_replica_comparison_error, 1236, "Storage replicas not consistent" )
ERROR( unreachable_storage_replica, 1237, "Storage replica cannot be reached" )
ERROR( bulkload_task_failed, 1238, "Bulk loading task failed")
ERROR( bulkload_task_outdated, 1239, "Bulk loading task outdated" )
// 15xx Platform errors
ERROR( platform_error, 1500, "Platform error" )

View File

@ -152,6 +152,7 @@ if(WITH_PYTHON)
add_fdb_test(TEST_FILES fast/BlobRestoreLarge.toml IGNORE)
add_fdb_test(TEST_FILES fast/BlobRestoreToVersion.toml)
add_fdb_test(TEST_FILES fast/BlobRestoreTenantMode.toml)
add_fdb_test(TEST_FILES fast/BulkLoading.toml)
add_fdb_test(TEST_FILES fast/CacheTest.toml)
add_fdb_test(TEST_FILES fast/CloggedSideband.toml)
add_fdb_test(TEST_FILES fast/CompressionUtilsUnit.toml IGNORE)

View File

@ -0,0 +1,26 @@
[configuration]
config = 'triple'
storageEngineType = 5
processesPerMachine = 2
coordinators = 3
machineCount = 40
generateFearless = false # disable remote DC. TODO(BulkLoad): remove this after SimulatedCluster has been fixed
tenantModes = ['disabled'] # BulkLoading test injects SST files with raw bytes. Therefore, disable tenant
[[knobs]]
# This knob is commented out since the knob override is done *after* the simulation system is set up. However,
# this is not going to completely work:
# The purpose of setting the knob to true was to enable the shard rocksdb storage engine
# The shard rocksdb storage engine is set up before this knob is overridden
# The temporary fix is that in SimulatedCluster.cpp:simulationSetupAndRun, we are doing one additional check
# so for this BulkLoading test, the shard RocksDB storage engine is always turned on.
shard_encode_location_metadata = true
dd_physical_shard_move_probability = 1.0
desired_teams_per_server = 10
[[test]]
testTitle = 'BulkLoadingWorkload'
useDB = true
[[test.workload]]
testName = 'BulkLoadingWorkload'