mirror of
https://github.com/apple/foundationdb.git
synced 2025-05-14 01:42:37 +08:00
Bulk Loading Framework (#11369)
This commit is contained in:
parent
92ad7666c3
commit
74990e44bd
186
fdbcli/BulkLoadCommand.actor.cpp
Normal file
186
fdbcli/BulkLoadCommand.actor.cpp
Normal 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
|
@ -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)
|
||||
|
@ -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
36
fdbclient/BulkLoading.cpp
Normal 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);
|
||||
}
|
@ -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);
|
||||
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
|
222
fdbclient/include/fdbclient/BulkLoading.h
Normal file
222
fdbclient/include/fdbclient/BulkLoading.h
Normal 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
|
@ -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);
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -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;
|
||||
|
174
fdbserver/BulkLoadUtil.actor.cpp
Normal file
174
fdbserver/BulkLoadUtil.actor.cpp
Normal 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;
|
||||
}
|
@ -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,
|
||||
|
@ -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")
|
||||
|
@ -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));
|
||||
}
|
||||
|
||||
|
@ -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));
|
||||
}
|
||||
|
@ -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));
|
||||
|
@ -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);
|
||||
|
@ -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());
|
||||
|
@ -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),
|
||||
|
@ -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);
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
|
66
fdbserver/include/fdbserver/BulkLoadUtil.actor.h
Normal file
66
fdbserver/include/fdbserver/BulkLoadUtil.actor.h
Normal 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
|
@ -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;
|
||||
|
@ -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);
|
||||
};
|
||||
|
@ -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
|
||||
|
@ -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 {
|
||||
|
@ -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
|
||||
|
@ -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);
|
||||
|
@ -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);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -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)
|
||||
|
675
fdbserver/workloads/BulkLoading.actor.cpp
Normal file
675
fdbserver/workloads/BulkLoading.actor.cpp
Normal 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;
|
@ -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;
|
||||
|
@ -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>());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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()));
|
||||
|
||||
|
@ -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) {
|
||||
|
@ -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");
|
||||
|
@ -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();
|
||||
}
|
||||
|
||||
|
@ -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);
|
||||
|
@ -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" )
|
||||
|
@ -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)
|
||||
|
26
tests/fast/BulkLoading.toml
Normal file
26
tests/fast/BulkLoading.toml
Normal 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'
|
Loading…
x
Reference in New Issue
Block a user