Allow One BulkloadTask Do Multiple Manifests (#12036)

This commit is contained in:
Zhe Wang 2025-03-17 11:45:15 -07:00 committed by GitHub
parent d5946157f0
commit 0e736c68e7
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
13 changed files with 546 additions and 238 deletions

View File

@ -179,7 +179,9 @@ BulkLoadTaskState createBulkLoadTask(const UID& jobId,
const BulkLoadTransportMethod& transportMethod) {
BulkLoadManifest manifest(
fileSet, range.begin, range.end, snapshotVersion, bytes, keyCount, byteSampleSetting, type, transportMethod);
return BulkLoadTaskState(jobId, manifest);
BulkLoadManifestSet manifests(1);
manifests.addManifest(manifest);
return BulkLoadTaskState(jobId, manifests);
}
BulkLoadJobState createBulkLoadJob(const UID& dumpJobIdToLoad,

View File

@ -387,6 +387,7 @@ void ServerKnobs::initialize(Randomize randomize, ClientKnobs* clientKnobs, IsSi
init( DISABLE_AUDIT_STORAGE_FINAL_REPLICA_CHECK_IN_SIM, false );
init( SS_BULKLOAD_GETRANGE_BATCH_SIZE, 10000 ); if (isSimulated) SS_BULKLOAD_GETRANGE_BATCH_SIZE = deterministicRandom()->randomInt(1, 10);
init( BULKLOAD_ASYNC_READ_WRITE_BLOCK_SIZE, 1024*1024 ); if (isSimulated) BULKLOAD_ASYNC_READ_WRITE_BLOCK_SIZE = deterministicRandom()->randomInt(1024, 10240);
init( MANIFEST_COUNT_MAX_PER_BULKLOAD_TASK, 10 ); if (isSimulated) MANIFEST_COUNT_MAX_PER_BULKLOAD_TASK = deterministicRandom()->randomInt(1, 11);
// BulkDumping
init( DD_BULKDUMP_SCHEDULE_MIN_INTERVAL_SEC, 5.0 ); if( randomize && BUGGIFY ) DD_BULKDUMP_SCHEDULE_MIN_INTERVAL_SEC = deterministicRandom()->random01() * 10 + 1;

View File

@ -30,7 +30,6 @@
#include "fdbclient/FDBTypes.h"
#include "fdbclient/Knobs.h"
#include "fdbrpc/fdbrpc.h"
// For all trace events for bulkload/dump operations
inline Severity bulkLoadVerboseEventSev() {
@ -199,16 +198,13 @@ public:
bool isValid() const {
if (rootPath.empty()) {
ASSERT(false);
return false;
}
if (!hasManifestFile()) {
ASSERT(false);
return false;
}
if (!hasDataFile() && hasByteSampleFile()) {
// If bytes sample file exists, the data file must exist.
ASSERT(false);
return false;
}
return true;
@ -320,6 +316,8 @@ private:
BulkLoadChecksum checksum;
};
using BulkLoadFileSetKeyMap = std::vector<std::pair<KeyRange, BulkLoadFileSet>>;
// Define the metadata of bulkload manifest file.
// The manifest file stores the ground true of metadata of dumped data file, such as range and version.
// The manifest file is uploaded along with the data file.
@ -541,6 +539,129 @@ enum class BulkLoadPhase : uint8_t {
Error = 6, // Updated by DD when this task has unretriable error
};
struct BulkLoadManifestSet {
public:
constexpr static FileIdentifier file_identifier = 1384493;
BulkLoadManifestSet() = default;
BulkLoadManifestSet(int inputMaxCount) { maxCount = inputMaxCount; }
bool isValid() const {
if (maxCount == 0) {
return false;
}
if (manifests.empty()) {
return false;
}
if (manifests.size() > maxCount) {
return false;
}
if (!minBeginKey.present()) {
return false;
}
if (!maxEndKey.present()) {
return false;
}
for (const auto& manifest : manifests) {
if (!manifest.isValid()) {
return false;
}
}
return true;
}
// Return true if succeed
bool addManifest(const BulkLoadManifest& manifest) {
if (manifests.size() > maxCount) {
return false;
}
manifests.push_back(manifest);
if (transportMethod == BulkLoadTransportMethod::Invalid) {
transportMethod = manifest.getTransportMethod();
} else {
ASSERT(transportMethod == manifest.getTransportMethod());
}
if (loadType == BulkLoadType::Invalid) {
loadType = manifest.getLoadType();
} else {
ASSERT(loadType == manifest.getLoadType());
}
if (!byteSampleSetting.isValid()) {
byteSampleSetting = manifest.getByteSampleSetting();
} else {
ASSERT(byteSampleSetting == manifest.getByteSampleSetting());
}
if (!minBeginKey.present() || minBeginKey.get() > manifest.getBeginKey()) {
minBeginKey = manifest.getBeginKey();
}
if (!maxEndKey.present() || maxEndKey.get() < manifest.getEndKey()) {
maxEndKey = manifest.getEndKey();
}
return true;
}
bool isFull() const { return manifests.size() >= maxCount; }
bool hasEmptyData() const {
for (const auto& manifest : manifests) {
if (!manifest.hasEmptyData()) {
return false;
}
}
return true;
}
int64_t getTotalBytes() const {
int64_t res = 0;
for (const auto& manifest : manifests) {
res = res + manifest.getTotalBytes();
}
return res;
}
const std::vector<BulkLoadManifest>& getManifests() const { return manifests; }
BulkLoadTransportMethod getTransportMethod() const { return transportMethod; }
BulkLoadByteSampleSetting getByteSampleSetting() const { return byteSampleSetting; }
BulkLoadType getLoadType() const { return loadType; }
size_t size() const { return manifests.size(); }
bool empty() const { return manifests.empty(); }
Key getMinBeginKey() const {
ASSERT(minBeginKey.present());
return minBeginKey.get();
}
Key getMaxEndKey() const {
ASSERT(maxEndKey.present());
return maxEndKey.get();
}
std::string toString() const { return describe(manifests); }
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, maxCount, transportMethod, loadType, byteSampleSetting, minBeginKey, maxEndKey, manifests);
}
private:
int maxCount = 0;
BulkLoadTransportMethod transportMethod = BulkLoadTransportMethod::Invalid;
BulkLoadType loadType = BulkLoadType::Invalid;
BulkLoadByteSampleSetting byteSampleSetting;
Optional<Key> minBeginKey;
Optional<Key> maxEndKey;
std::vector<BulkLoadManifest> manifests;
};
struct BulkLoadTaskState {
public:
constexpr static FileIdentifier file_identifier = 1384499;
@ -548,23 +669,23 @@ public:
BulkLoadTaskState() = default;
// For submitting a task by a job
BulkLoadTaskState(const UID& jobId, const BulkLoadManifest& manifest)
: jobId(jobId), taskId(deterministicRandom()->randomUniqueID()), manifest(manifest) {
if (manifest.hasEmptyData()) {
BulkLoadTaskState(const UID& jobId, const BulkLoadManifestSet& manifests)
: jobId(jobId), taskId(deterministicRandom()->randomUniqueID()), manifests(manifests) {
ASSERT(!manifests.empty());
if (manifests.hasEmptyData()) {
phase = BulkLoadPhase::Complete; // If no data to load, the task is complete.
} else {
phase = BulkLoadPhase::Submitted;
}
// We define the task range is the range of the min begin key and the max end key among all manifests
taskRange = Standalone(KeyRangeRef(manifests.getMinBeginKey(), manifests.getMaxEndKey()));
}
bool operator==(const BulkLoadTaskState& rhs) const {
return jobId == rhs.jobId && taskId == rhs.taskId && getRange() == rhs.getRange() &&
getDataFileFullPath() == rhs.getDataFileFullPath();
}
bool operator==(const BulkLoadTaskState& rhs) const { return jobId == rhs.jobId && taskId == rhs.taskId; }
std::string toString() const {
std::string res = "BulkLoadTaskState: [JobId]: " + jobId.toString() + ", [TaskId]: " + taskId.toString() +
", [Manifest]: " + manifest.toString();
", [Manifests]: " + manifests.toString();
if (dataMoveId.present()) {
res = res + ", [DataMoveId]: " + dataMoveId.get().toString();
}
@ -572,33 +693,23 @@ public:
return res;
}
KeyRange getRange() const { return manifest.getRange(); }
KeyRange getRange() const { return taskRange; }
UID getTaskId() const { return taskId; }
UID getJobId() const { return jobId; }
bool hasEmptyData() const { return manifest.hasEmptyData(); }
bool hasEmptyData() const { return manifests.hasEmptyData(); }
std::string getRootPath() const { return manifest.getRootPath(); }
BulkLoadTransportMethod getTransportMethod() const { return manifests.getTransportMethod(); }
BulkLoadTransportMethod getTransportMethod() const { return manifest.getTransportMethod(); }
int64_t getTotalBytes() const { return manifests.getTotalBytes(); }
std::string getDataFileFullPath() const { return manifest.getDataFileFullPath(); }
const std::vector<BulkLoadManifest>& getManifests() const { return manifests.getManifests(); }
std::string getBytesSampleFileFullPath() const { return manifest.getBytesSampleFileFullPath(); }
BulkLoadByteSampleSetting getByteSampleSetting() const { return manifests.getByteSampleSetting(); }
std::string getFolder() const { return manifest.getFolder(); }
int64_t getTotalBytes() const { return manifest.getTotalBytes(); }
int64_t getKeyCount() const { return manifest.getKeyCount(); }
BulkLoadFileSet getFileSet() const { return manifest.getFileSet(); }
BulkLoadByteSampleSetting getByteSampleSetting() const { return manifest.getByteSampleSetting(); }
BulkLoadType getLoadType() const { return manifest.getLoadType(); }
BulkLoadType getLoadType() const { return manifests.getLoadType(); }
void setCancelledDataMovePriority(int priority) { cancelledDataMovePriority = priority; }
@ -633,7 +744,7 @@ public:
if (!taskId.isValid()) {
return false;
}
if (checkManifest && !manifest.isValid()) {
if (checkManifest && !manifests.isValid()) {
return false;
}
return true;
@ -645,13 +756,14 @@ public:
jobId,
taskId,
dataMoveId,
manifest,
manifests,
phase,
submitTime,
triggerTime,
startTime,
completeTime,
restartCount,
taskRange,
cancelledDataMovePriority);
}
@ -670,7 +782,8 @@ private:
// Set by DD
Optional<UID> dataMoveId;
// Set by DD or users
BulkLoadManifest manifest;
BulkLoadManifestSet manifests;
KeyRange taskRange;
Optional<int> cancelledDataMovePriority; // Set when the task is failed for unretrievable error.
// In this case, we want to re-issue data move on the task range if the data move is team unhealthy related.
};

View File

@ -411,6 +411,7 @@ public:
// manifest file
int SS_BULKDUMP_BATCH_COUNT_MAX_PER_REQUEST; // the max number of batch count per bulkdump request to a SS
int BULKLOAD_ASYNC_READ_WRITE_BLOCK_SIZE; // the block size when performing async read/write for bulkload
int MANIFEST_COUNT_MAX_PER_BULKLOAD_TASK; // the max number of manifest that a bulkload task can process
bool CC_ENFORCE_USE_UNFIT_DD_IN_SIM; // Set for CC to enforce to use an unfit DD in the simulation. This knob takes
// effect only in the simulation.
bool DISABLE_AUDIT_STORAGE_FINAL_REPLICA_CHECK_IN_SIM; // Set to disable audit storage replica check in the

View File

@ -312,8 +312,6 @@ ACTOR Future<BulkLoadFileSet> bulkLoadDownloadTaskFileSet(BulkLoadTransportMetho
UNREACHABLE();
}
TraceEvent(bulkLoadVerboseEventSev(), "SSBulkLoadTaskDownloadFileSet", logId)
.setMaxEventLength(-1)
.setMaxFieldLength(-1)
.detail("FromRemoteFileSet", fromRemoteFileSet.toString())
.detail("ToLocalRoot", toLocalRoot)
.detail("Duration", now() - startTime)
@ -326,8 +324,6 @@ ACTOR Future<BulkLoadFileSet> bulkLoadDownloadTaskFileSet(BulkLoadTransportMetho
throw e;
}
TraceEvent(SevWarn, "SSBulkLoadTaskDownloadFileSetError", logId)
.setMaxEventLength(-1)
.setMaxFieldLength(-1)
.errorUnsuppressed(e)
.detail("FromRemoteFileSet", fromRemoteFileSet.toString())
.detail("ToLocalRoot", toLocalRoot)
@ -339,6 +335,26 @@ ACTOR Future<BulkLoadFileSet> bulkLoadDownloadTaskFileSet(BulkLoadTransportMetho
}
}
ACTOR Future<Void> bulkLoadDownloadTaskFileSets(BulkLoadTransportMethod transportMethod,
std::shared_ptr<BulkLoadFileSetKeyMap> fromRemoteFileSets,
std::shared_ptr<BulkLoadFileSetKeyMap> localFileSets,
std::string toLocalRoot,
UID logId) {
state BulkLoadFileSetKeyMap::iterator iter = fromRemoteFileSets->begin();
state KeyRange keys;
for (; iter != fromRemoteFileSets->end(); iter++) {
keys = iter->first;
if (!iter->second.hasDataFile()) {
// Ignore the remote fileSet if it does not have data file
continue;
}
BulkLoadFileSet localFileSet =
wait(bulkLoadDownloadTaskFileSet(transportMethod, iter->second, toLocalRoot, logId));
localFileSets->push_back(std::make_pair(keys, localFileSet));
}
return Void();
}
ACTOR Future<Void> downloadManifestFile(BulkLoadTransportMethod transportMethod,
std::string fromRemotePath,
std::string toLocalPath,
@ -501,18 +517,26 @@ ACTOR Future<Void> getBulkLoadJobFileManifestEntryFromJobManifestFile(
return Void();
}
ACTOR Future<BulkLoadManifest> getBulkLoadManifestMetadataFromEntry(BulkLoadJobFileManifestEntry manifestEntry,
std::string manifestLocalTempFolder,
BulkLoadTransportMethod transportMethod,
std::string jobRoot,
UID logId) {
state std::string remoteManifestFilePath = appendToPath(jobRoot, manifestEntry.getManifestRelativePath());
state std::string localManifestFilePath =
joinPath(manifestLocalTempFolder,
deterministicRandom()->randomUniqueID().toString() + "-" + basename(getPath(remoteManifestFilePath)));
wait(downloadManifestFile(transportMethod, remoteManifestFilePath, localManifestFilePath, logId));
state std::shared_ptr<std::string> manifestRawString = std::make_shared<std::string>();
wait(readBulkFileBytes(abspath(localManifestFilePath), SERVER_KNOBS->BULKLOAD_FILE_BYTES_MAX, manifestRawString));
ASSERT(!manifestRawString->empty());
return BulkLoadManifest(*manifestRawString);
ACTOR Future<BulkLoadManifestSet> getBulkLoadManifestMetadataFromEntry(
std::vector<BulkLoadJobFileManifestEntry> manifestEntries,
std::string manifestLocalTempFolder,
BulkLoadTransportMethod transportMethod,
std::string jobRoot,
UID logId) {
state BulkLoadManifestSet manifests(SERVER_KNOBS->MANIFEST_COUNT_MAX_PER_BULKLOAD_TASK);
state int i = 0;
for (; i < manifestEntries.size(); i++) {
state std::string remoteManifestFilePath = appendToPath(jobRoot, manifestEntries[i].getManifestRelativePath());
state std::string localManifestFilePath = joinPath(manifestLocalTempFolder,
deterministicRandom()->randomUniqueID().toString() + "-" +
basename(getPath(remoteManifestFilePath)));
wait(downloadManifestFile(transportMethod, remoteManifestFilePath, localManifestFilePath, logId));
state std::shared_ptr<std::string> manifestRawString = std::make_shared<std::string>();
wait(readBulkFileBytes(
abspath(localManifestFilePath), SERVER_KNOBS->BULKLOAD_FILE_BYTES_MAX, manifestRawString));
ASSERT(!manifestRawString->empty());
BulkLoadManifest manifest(*manifestRawString);
ASSERT(manifests.addManifest(manifest));
}
return manifests;
}

View File

@ -1054,12 +1054,11 @@ void DDQueue::launchQueuedWork(std::set<RelocateData, std::greater<RelocateData>
bool doBulkLoading = runPendingBulkLoadTaskWithRelocateData(this, rd);
if (doBulkLoading) {
TraceEvent(bulkLoadVerboseEventSev(), "DDBulkLoadTaskLaunchingDataMove", this->distributorId)
.setMaxEventLength(-1)
.setMaxFieldLength(-1)
.detail("NewDataMoveId", rd.dataMoveId)
.detail("NewDataMovePriority", rd.priority)
.detail("NewDataMoveRange", rd.keys)
.detail("BulkLoadTask", rd.bulkLoadTask.get().toString());
.detail("TaskID", rd.bulkLoadTask.get().coreState.getTaskId().toString())
.detail("JobID", rd.bulkLoadTask.get().coreState.getJobId().toString());
}
// Check if there is an inflight shard that is overlapped with the queued relocateShard (rd)
@ -1524,10 +1523,14 @@ ACTOR Future<Void> dataDistributionRelocator(DDQueue* self,
rd.bulkLoadTask.get().coreState.getTaskId(),
{ BulkLoadPhase::Triggered, BulkLoadPhase::Running }));
TraceEvent(bulkLoadVerboseEventSev(), "DDBulkLoadTaskDataMoveLaunched", self->distributorId)
.setMaxEventLength(-1)
.setMaxFieldLength(-1)
.detail("BulkLoadTask", rd.bulkLoadTask.get().coreState.toString())
.detail("ExistState", currentBulkLoadTaskState.toString());
.detail("TrackID", rd.randomId)
.detail("DataMovePriority", rd.priority)
.detail("JobID", rd.bulkLoadTask.get().coreState.getJobId().toString())
.detail("TaskID", rd.bulkLoadTask.get().coreState.getTaskId().toString())
.detail("TaskRange", rd.bulkLoadTask.get().coreState.getRange().toString())
.detail("ExistingJobID", currentBulkLoadTaskState.getJobId().toString())
.detail("ExistingTaskID", currentBulkLoadTaskState.getTaskId().toString())
.detail("ExistingTaskRange", currentBulkLoadTaskState.getRange().toString());
break;
} catch (Error& e) {
if (e.code() == error_code_bulkload_task_outdated) {
@ -1545,9 +1548,11 @@ ACTOR Future<Void> dataDistributionRelocator(DDQueue* self,
// TODO(BulkLoad): reset rd.bulkLoadTask here for the risk of overloading the source
// servers.
TraceEvent(SevWarn, "DDBulkLoadTaskFallbackToNormalDataMove", self->distributorId)
.setMaxEventLength(-1)
.setMaxFieldLength(-1)
.detail("BulkLoadTask", rd.bulkLoadTask.get().coreState.toString());
.detail("TrackID", rd.randomId)
.detail("DataMovePriority", rd.priority)
.detail("JobID", rd.bulkLoadTask.get().coreState.getJobId().toString())
.detail("TaskID", rd.bulkLoadTask.get().coreState.getTaskId().toString())
.detail("TaskRange", rd.bulkLoadTask.get().coreState.getRange().toString());
break;
}
wait(tr.onError(e));
@ -1561,7 +1566,6 @@ ACTOR Future<Void> dataDistributionRelocator(DDQueue* self,
.detail("TrackID", rd.randomId)
.detail("Range", rd.keys)
.detail("Priority", rd.priority)
.detail("Reason", rd.reason.toString())
.detail("DataMoveType", dataMoveType)
.detail("DoBulkLoading", doBulkLoading)
.detail("DataMoveReason", static_cast<int>(rd.dmReason));
@ -1654,8 +1658,6 @@ ACTOR Future<Void> dataDistributionRelocator(DDQueue* self,
bestTeams.emplace_back(bestTeam.first.get(), bestTeam.second);
if (doBulkLoading) {
TraceEvent(bulkLoadVerboseEventSev(), "DDBulkLoadTaskSelectDestTeam", self->distributorId)
.setMaxEventLength(-1)
.setMaxFieldLength(-1)
.detail("Context", "Restore")
.detail("SrcIds", describe(rd.src))
.detail("DestIds", bestTeam.first.get()->getServerIDs())
@ -1735,7 +1737,8 @@ ACTOR Future<Void> dataDistributionRelocator(DDQueue* self,
"DDBulkLoadTaskRelocatorBestTeamReceived",
self->distributorId)
.detail("DataMoveID", rd.dataMoveId)
.detail("BulkLoadTask", rd.bulkLoadTask.get().toString())
.detail("JobID", rd.bulkLoadTask.get().coreState.getJobId().toString())
.detail("TaskID", rd.bulkLoadTask.get().coreState.getTaskId().toString())
.detail("BestTeamReady", bestTeamReady);
}
if (tciIndex > 0 && !bestTeamReady) {
@ -1841,7 +1844,8 @@ ACTOR Future<Void> dataDistributionRelocator(DDQueue* self,
.detail("SrcIds", describe(rd.src))
.detail("DestIds", bestTeam.first.get()->getServerIDs())
.detail("DestTeam", bestTeam.first.get()->getTeamID())
.detail("BulkLoadTask", rd.bulkLoadTask.get().toString())
.detail("JobID", rd.bulkLoadTask.get().coreState.getJobId().toString())
.detail("TaskID", rd.bulkLoadTask.get().coreState.getTaskId().toString())
.detail("Priority", rd.priority)
.detail("DataMoveId", rd.dataMoveId)
.detail("Primary", tciIndex == 0);
@ -2270,10 +2274,9 @@ ACTOR Future<Void> dataDistributionRelocator(DDQueue* self,
self->bulkLoadTaskCollection->terminateTask(rd.bulkLoadTask.get().coreState);
TraceEvent(
bulkLoadVerboseEventSev(), "DDBulkLoadTaskRelocatorComplete", self->distributorId)
.setMaxEventLength(-1)
.setMaxFieldLength(-1)
.detail("Dests", describe(destIds))
.detail("Task", rd.bulkLoadTask.get().toString());
.detail("JobID", rd.bulkLoadTask.get().coreState.getJobId().toString())
.detail("TaskID", rd.bulkLoadTask.get().coreState.getTaskId().toString());
} catch (Error& bulkLoadError) {
ASSERT_WE_THINK(bulkLoadError.code() == error_code_bulkload_task_outdated);
if (bulkLoadError.code() != error_code_bulkload_task_outdated) {
@ -2282,20 +2285,18 @@ ACTOR Future<Void> dataDistributionRelocator(DDQueue* self,
TraceEvent(bulkLoadVerboseEventSev(),
"DDBulkLoadTaskRelocatorCompleteButOutdated",
self->distributorId)
.setMaxEventLength(-1)
.setMaxFieldLength(-1)
.detail("Dests", describe(destIds))
.detail("Task", rd.bulkLoadTask.get().toString());
.detail("JobID", rd.bulkLoadTask.get().coreState.getJobId().toString())
.detail("TaskID", rd.bulkLoadTask.get().coreState.getTaskId().toString());
}
}
return Void();
} else {
if (doBulkLoading) {
TraceEvent(bulkLoadVerboseEventSev(), "DDBulkLoadTaskRelocatorError")
.setMaxEventLength(-1)
.setMaxFieldLength(-1)
.errorUnsuppressed(error)
.detail("Task", rd.bulkLoadTask.get().toString());
.detail("JobID", rd.bulkLoadTask.get().coreState.getJobId().toString())
.detail("TaskID", rd.bulkLoadTask.get().coreState.getTaskId().toString());
}
throw error;
}

View File

@ -952,7 +952,7 @@ static bool shardForwardMergeFeasible(DataDistributionTracker* self, KeyRange co
return false;
}
if (self->bulkLoadEnabled && self->bulkLoadTaskCollection->onBulkLoading(nextRange)) {
if (self->bulkLoadEnabled && self->bulkLoadTaskCollection->bulkLoading(nextRange)) {
TraceEvent(SevWarn, "ShardCanForwardMergeButUnderBulkLoading", self->distributorId)
.suppressFor(5.0)
.detail("ShardMerging", keys)
@ -972,7 +972,7 @@ static bool shardBackwardMergeFeasible(DataDistributionTracker* self, KeyRange c
return false;
}
if (self->bulkLoadEnabled && self->bulkLoadTaskCollection->onBulkLoading(prevRange)) {
if (self->bulkLoadEnabled && self->bulkLoadTaskCollection->bulkLoading(prevRange)) {
TraceEvent(SevWarn, "ShardCanBackwardMergeButUnderBulkLoading", self->distributorId)
.suppressFor(5.0)
.detail("ShardMerging", keys)
@ -996,7 +996,7 @@ void createShardToBulkLoad(DataDistributionTracker* self,
TraceEvent e(issueDataMoveForCancel ? SevWarnAlways : bulkLoadVerboseEventSev(),
"DDBulkLoadEngineCreateShardToBulkLoad",
self->distributorId);
e.detail("TaskId", bulkLoadTaskState.getTaskId());
e.detail("TaskID", bulkLoadTaskState.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
@ -1333,7 +1333,7 @@ ACTOR Future<Void> shardTracker(DataDistributionTracker::SafeAccessor self,
try {
loop {
while (self()->bulkLoadEnabled && self()->bulkLoadTaskCollection->onBulkLoading(keys)) {
while (self()->bulkLoadEnabled && self()->bulkLoadTaskCollection->bulkLoading(keys)) {
TraceEvent(SevWarn, "ShardBoundaryChangeDisabledForBulkLoad", self()->distributorId)
.suppressFor(60.0)
.detail("KeyRange", keys);

View File

@ -1124,20 +1124,18 @@ ACTOR Future<std::pair<BulkLoadTaskState, Version>> triggerBulkLoadTask(Referenc
wait(tr.commit());
Version commitVersion = tr.getCommittedVersion();
TraceEvent(bulkLoadVerboseEventSev(), "DDBulkLoadTaskPersistTriggerState", self->ddId)
.setMaxEventLength(-1)
.setMaxFieldLength(-1)
.detail("CommitVersion", commitVersion)
.detail("BulkLoadTaskState", newBulkLoadTaskState.toString());
.detail("TaskID", newBulkLoadTaskState.getTaskId().toString())
.detail("JobID", newBulkLoadTaskState.getJobId().toString());
ASSERT(commitVersion != invalidVersion);
return std::make_pair(newBulkLoadTaskState, commitVersion);
} catch (Error& e) {
if (e.code() != error_code_actor_cancelled) {
TraceEvent(SevWarn, "DDBulkLoadTaskPersistTriggerStateError", self->ddId)
.setMaxEventLength(-1)
.setMaxFieldLength(-1)
.errorUnsuppressed(e)
.detail("BulkLoadTaskState", newBulkLoadTaskState.toString());
.detail("TaskID", newBulkLoadTaskState.getTaskId().toString())
.detail("JobID", newBulkLoadTaskState.getJobId().toString());
}
wait(tr.onError(e));
}
@ -1168,18 +1166,16 @@ ACTOR Future<Void> failBulkLoadTask(Reference<DataDistributor> self,
wait(tr.commit());
Version commitVersion = tr.getCommittedVersion();
TraceEvent(bulkLoadVerboseEventSev(), "DDBulkLoadTaskPersistErrorState", self->ddId)
.setMaxEventLength(-1)
.setMaxFieldLength(-1)
.detail("CommitVersion", commitVersion)
.detail("BulkLoadTaskState", bulkLoadTaskState.toString());
.detail("TaskID", bulkLoadTaskState.getTaskId().toString())
.detail("JobID", bulkLoadTaskState.getJobId().toString());
break;
} catch (Error& e) {
if (e.code() != error_code_actor_cancelled) {
TraceEvent(SevWarn, "DDBulkLoadTaskPersistErrorStateError", self->ddId)
.setMaxEventLength(-1)
.setMaxFieldLength(-1)
.errorUnsuppressed(e)
.detail("BulkLoadTaskState", bulkLoadTaskState.toString());
.detail("TaskID", bulkLoadTaskState.getTaskId().toString())
.detail("JobID", bulkLoadTaskState.getJobId().toString());
}
wait(tr.onError(e));
}
@ -1203,10 +1199,10 @@ ACTOR Future<Void> doBulkLoadTask(Reference<DataDistributor> self, KeyRange rang
triggeredBulkLoadTask = triggeredBulkLoadTask_.first;
commitVersion = triggeredBulkLoadTask_.second;
TraceEvent(bulkLoadVerboseEventSev(), "DDBulkLoadTaskDoTask", self->ddId)
.setMaxEventLength(-1)
.setMaxFieldLength(-1)
.detail("Phase", "Triggered")
.detail("Task", triggeredBulkLoadTask.toString())
.detail("TaskID", triggeredBulkLoadTask.getTaskId().toString())
.detail("TaskRange", triggeredBulkLoadTask.getRange().toString())
.detail("JobID", triggeredBulkLoadTask.getJobId().toString())
.detail("CommitVersion", commitVersion)
.detail("Duration", now() - beginTime);
ASSERT(triggeredBulkLoadTask.getRange() == range);
@ -1297,7 +1293,7 @@ ACTOR Future<Void> eraseBulkLoadTask(Reference<DataDistributor> self, KeyRange t
TraceEvent(bulkLoadVerboseEventSev(), "DDBulkLoadTaskEraseState", self->ddId)
.detail("CommitVersion", commitVersion)
.detail("TaskRange", taskRange)
.detail("TaskId", taskId.toString());
.detail("TaskID", taskId.toString());
self->bulkLoadTaskCollection->eraseTask(bulkLoadTask);
Optional<int> cancelledDataMovePriority = bulkLoadTask.getCancelledDataMovePriority();
if (cancelledDataMovePriority.present() &&
@ -1308,7 +1304,7 @@ ACTOR Future<Void> eraseBulkLoadTask(Reference<DataDistributor> self, KeyRange t
TraceEvent(bulkLoadVerboseEventSev(), "DDBulkLoadTaskTriggerShardDatamove", self->ddId)
.detail("CommitVersion", commitVersion)
.detail("TaskRange", taskRange)
.detail("TaskId", taskId.toString());
.detail("TaskID", taskId.toString());
}
break;
} catch (Error& e) {
@ -1501,13 +1497,14 @@ ACTOR Future<Optional<BulkLoadTaskState>> bulkLoadJobFindTask(Reference<DataDist
return bulkLoadTaskState;
}
// Trigger a bulk load job task
ACTOR Future<BulkLoadTaskState> bulkLoadJobTriggerTask(Reference<DataDistributor> self,
UID jobId,
BulkLoadManifest manifest) {
// Submit a bulkload task for the given jobId
ACTOR Future<BulkLoadTaskState> bulkLoadJobSubmitTask(Reference<DataDistributor> self,
UID jobId,
BulkLoadManifestSet manifests) {
state Database cx = self->txnProcessor->context();
state Transaction tr(cx);
state BulkLoadTaskState bulkLoadTask(jobId, manifest);
// We define the task range is the range of the min begin key and the max end key among all input manifests
state BulkLoadTaskState bulkLoadTask(jobId, manifests);
loop {
try {
// At any time, there must be at most one bulkload job
@ -1517,11 +1514,11 @@ ACTOR Future<BulkLoadTaskState> bulkLoadJobTriggerTask(Reference<DataDistributor
wait(tr.commit());
Version commitVersion = tr.getCommittedVersion();
TraceEvent(bulkLoadVerboseEventSev(), "DDBulkLoadJobExecutorSubmitTask", self->ddId)
.setMaxEventLength(-1)
.setMaxFieldLength(-1)
.detail("JobId", jobId.toString())
.detail("Manifest", manifest.toString())
.detail("BulkLoadTask", bulkLoadTask.toString())
.detail("Manifest", manifests.size())
.detail("TaskID", bulkLoadTask.getTaskId().toString())
.detail("TaskRange", bulkLoadTask.getRange().toString())
.detail("JobID", bulkLoadTask.getJobId().toString())
.detail("CommitVersion", commitVersion);
break;
} catch (Error& e) {
@ -1572,15 +1569,92 @@ ACTOR Future<Void> bulkLoadJobWaitUntilTaskCompleteOrError(Reference<DataDistrib
}
}
// Given manifestEntry, create a bulkload task and wait until the task is complete or error.
// Given a list of manifestEntry, create a bulkload task and wait until the task is complete or error.
// There is an invariant by bulkload engine: if a task metadata is persisted, the task
// is guaranteed to be eventually marked as complete or error.
ACTOR Future<Void> bulkLoadJobExecuteTask(Reference<DataDistributor> self,
ACTOR Future<Void> bulkLoadJobNewTask(Reference<DataDistributor> self,
UID jobId,
std::string jobRoot,
BulkLoadTransportMethod jobTransportMethod,
std::string manifestLocalTempFolder,
std::vector<BulkLoadJobFileManifestEntry> manifestEntries,
Promise<Void> errorOut) {
state Database cx = self->txnProcessor->context();
state BulkLoadTaskState bulkLoadTask;
state BulkLoadManifestSet manifests;
state double beginTime = now();
ASSERT(self->bulkLoadParallelismLimitor.canStart());
self->bulkLoadParallelismLimitor.incrementTaskCounter();
ASSERT(!manifestEntries.empty());
try {
// Step 1: Get manifest metadata by downloading the manifest file
wait(store(manifests,
getBulkLoadManifestMetadataFromEntry(
manifestEntries, manifestLocalTempFolder, jobTransportMethod, jobRoot, self->ddId)));
// Step 2: Check if the task has been created
// We define the task range is the range between the min begin key and the max end key of all manifests
Optional<BulkLoadTaskState> bulkLoadTask_ = wait(bulkLoadJobFindTask(
self, KeyRangeRef(manifests.getMinBeginKey(), manifests.getMaxEndKey()), jobId, self->ddId));
if (bulkLoadTask_.present()) {
// The task was not existing in the metadata but existing now. So, we need not create the task.
self->bulkLoadParallelismLimitor.decrementTaskCounter();
return Void();
}
// Step 3: Trigger bulkload task which is handled by bulkload task engine
// Discussion about what if another newer job has persist some task on the range with a different
// job Id. This case should never happens because before the newer job starts, the old job has been
// completed or cancelled.
wait(store(bulkLoadTask, bulkLoadJobSubmitTask(self, jobId, manifests)));
TraceEvent(bulkLoadVerboseEventSev(), "DDBulkLoadJobExecutorTask", self->ddId)
.detail("Phase", "Task submitted")
.detail("JobID", jobId.toString())
.detail("TaskID", bulkLoadTask.getTaskId().toString())
.detail("TaskRange", bulkLoadTask.getRange().toString())
.detail("Duration", now() - beginTime);
if (g_network->isSimulated() && deterministicRandom()->random01() < 0.1) {
TraceEvent(SevWarnAlways, "DDBulkLoadJobExecutorInjectDDRestart", self->ddId).detail("Context", "New");
throw movekeys_conflict(); // improve code coverage
}
// Step 4: Monitor the bulkload completion
wait(bulkLoadJobWaitUntilTaskCompleteOrError(self, jobId, bulkLoadTask));
TraceEvent(bulkLoadPerfEventSev(), "DDBulkLoadJobExecutorTask", self->ddId)
.detail("Phase", "Task complete")
.detail("JobId", jobId.toString())
.detail("TaskId", bulkLoadTask.getTaskId().toString())
.detail("TaskRange", bulkLoadTask.getRange().toString())
.detail("Duration", now() - beginTime);
self->bulkLoadParallelismLimitor.decrementTaskCounter();
} catch (Error& e) {
if (e.code() == error_code_actor_cancelled) {
throw e;
}
TraceEvent(SevWarn, "DDBulkLoadJobExecutorTaskError", self->ddId)
.errorUnsuppressed(e)
.detail("JobId", jobId.toString())
.detail("TaskId", bulkLoadTask.getTaskId().toString())
.detail("TaskRange", bulkLoadTask.getRange().toString())
.detail("Duration", now() - beginTime);
self->bulkLoadParallelismLimitor.decrementTaskCounter();
if (errorOut.canBeSet()) {
errorOut.sendError(e);
}
// Currently, all errors here come from the bulkload job mechanism.
// BulkLoad task is guaranteed to be completed by the engine given a task metadata is persisted.
}
return Void();
}
// Given a bulkload task range, find the task and wait until the task is complete or error.
// There is an invariant by bulkload engine: if a task metadata is persisted, the task
// is guaranteed to be eventually marked as complete or error.
ACTOR Future<Void> bulkLoadJobMonitorTask(Reference<DataDistributor> self,
UID jobId,
std::string jobRoot,
BulkLoadTransportMethod jobTransportMethod,
std::string manifestLocalTempFolder,
BulkLoadJobFileManifestEntry manifestEntry,
KeyRange taskRange,
Promise<Void> errorOut) {
state Database cx = self->txnProcessor->context();
state BulkLoadTaskState bulkLoadTask;
@ -1589,53 +1663,41 @@ ACTOR Future<Void> bulkLoadJobExecuteTask(Reference<DataDistributor> self,
self->bulkLoadParallelismLimitor.incrementTaskCounter();
try {
// Step 1: Check if the task has been created
Optional<BulkLoadTaskState> bulkLoadTask_ =
wait(bulkLoadJobFindTask(self, manifestEntry.getRange(), jobId, self->ddId));
Optional<BulkLoadTaskState> bulkLoadTask_ = wait(bulkLoadJobFindTask(self, taskRange, jobId, self->ddId));
if (!bulkLoadTask_.present()) {
// Step 2: Get manifest metadata by downloading the manifest file
BulkLoadManifest manifest = wait(getBulkLoadManifestMetadataFromEntry(
manifestEntry, manifestLocalTempFolder, jobTransportMethod, jobRoot, self->ddId));
// Step 3: Trigger bulkload task which is handled by bulkload task engine
// Discussion about what if another newer job has persist some task on the range with a different
// job Id. This case should never happens because before the newer job starts, the old job has been
// completed or cancelled.
wait(store(bulkLoadTask, bulkLoadJobTriggerTask(self, jobId, manifest)));
} else {
bulkLoadTask = bulkLoadTask_.get();
TraceEvent(bulkLoadVerboseEventSev(), "DDBulkLoadJobExecutorTaskFound", self->ddId)
.setMaxEventLength(-1)
.setMaxFieldLength(-1)
.detail("JobId", jobId.toString())
.detail("Task", bulkLoadTask.toString())
.detail("Duration", now() - beginTime);
// The task was existing in the metadata but now disappear. So, we need not monitor the task.
self->bulkLoadParallelismLimitor.decrementTaskCounter();
return Void();
}
bulkLoadTask = bulkLoadTask_.get();
TraceEvent(bulkLoadVerboseEventSev(), "DDBulkLoadJobExecutorTask", self->ddId)
.detail("Phase", "Task found")
.detail("JobID", jobId.toString())
.detail("TaskID", bulkLoadTask.getTaskId().toString())
.detail("TaskRange", bulkLoadTask.getRange().toString())
.detail("Duration", now() - beginTime);
if (g_network->isSimulated() && deterministicRandom()->random01() < 0.1) {
TraceEvent(SevWarnAlways, "DDBulkLoadJobExecutorInjectDDRestart", self->ddId);
TraceEvent(SevWarnAlways, "DDBulkLoadJobExecutorInjectDDRestart", self->ddId).detail("Context", "Monitor");
throw movekeys_conflict(); // improve code coverage
}
// Step 4: Monitor the bulkload completion
// Step 2: Monitor the bulkload completion
wait(bulkLoadJobWaitUntilTaskCompleteOrError(self, jobId, bulkLoadTask));
TraceEvent(bulkLoadPerfEventSev(), "DDBulkLoadJobExecutorTaskEnd", self->ddId)
.setMaxEventLength(-1)
.setMaxFieldLength(-1)
.detail("JobId", jobId.toString())
.detail("Task", bulkLoadTask.toString())
TraceEvent(bulkLoadPerfEventSev(), "DDBulkLoadJobExecutorTask", self->ddId)
.detail("Phase", "Found task complete")
.detail("JobID", jobId.toString())
.detail("TaskID", bulkLoadTask.getTaskId().toString())
.detail("Duration", now() - beginTime);
self->bulkLoadParallelismLimitor.decrementTaskCounter();
} catch (Error& e) {
if (e.code() == error_code_actor_cancelled) {
throw e;
}
TraceEvent(SevWarn, "DDBulkLoadJobExecutorTaskError", self->ddId)
.setMaxEventLength(-1)
.setMaxFieldLength(-1)
TraceEvent(SevWarn, "DDBulkLoadJobExecutorTaskMonitorError", self->ddId)
.errorUnsuppressed(e)
.detail("JobId", jobId.toString())
.detail("Task", bulkLoadTask.toString())
.detail("JobID", jobId.toString())
.detail("TaskID", bulkLoadTask.getTaskId().toString())
.detail("Duration", now() - beginTime);
self->bulkLoadParallelismLimitor.decrementTaskCounter();
if (errorOut.canBeSet()) {
@ -1801,40 +1863,73 @@ ACTOR Future<Void> scheduleBulkLoadJob(Reference<DataDistributor> self, Promise<
wait(krmGetRanges(&tr, bulkLoadTaskPrefix, KeyRangeRef(beginKey, jobState.getJobRange().end)));
state int i = 0;
for (; i < res.size() - 1; i++) {
// Proceeding the beginKey for submitted/completed tasks.
// Bypass completed tasks.
// Start monitoring for submitted tasks.
if (!res[i].value.empty()) {
BulkLoadTaskState task = decodeBulkLoadTaskState(res[i].value);
state BulkLoadTaskState task = decodeBulkLoadTaskState(res[i].value);
if (task.isValid()) {
if (task.getJobId() != self->bulkLoadJobManager.get().jobState.getJobId()) {
throw bulkload_task_outdated();
}
// Check key invariant, the task begin key must be in the manifestEntryMap
ASSERT(self->bulkLoadJobManager.get().manifestEntryMap->find(task.getRange().begin) !=
self->bulkLoadJobManager.get().manifestEntryMap->end());
if (task.onAnyPhase(
{ BulkLoadPhase::Complete, BulkLoadPhase::Acknowledged, BulkLoadPhase::Error })) {
auto it = self->bulkLoadJobManager.get().manifestEntryMap->find(beginKey);
ASSERT(it != self->bulkLoadJobManager.get().manifestEntryMap->end());
manifestEntry = it->second;
beginKey = manifestEntry.getEndKey();
continue; // Bypass completed tasks
ASSERT(task.getRange().end == res[i + 1].key);
beginKey = task.getRange().end;
// Bypass completed tasks
continue;
} else if (task.onAnyPhase({ BulkLoadPhase::Submitted,
BulkLoadPhase::Triggered,
BulkLoadPhase::Running })) {
// Limit parallelism
loop {
if (self->bulkLoadParallelismLimitor.canStart()) {
break;
}
wait(self->bulkLoadParallelismLimitor.waitUntilCounterChanged());
}
// Monitor submitted tasks
actors.push_back(bulkLoadJobMonitorTask(self, task.getJobId(), task.getRange(), errorOut));
ASSERT(task.getRange().end == res[i + 1].key);
beginKey = task.getRange().end;
continue;
} else {
UNREACHABLE();
}
}
}
// Schedule new tasks on range between res[i].key and res[i + 1].key
// Limit parallelism
loop {
if (self->bulkLoadParallelismLimitor.canStart()) {
break;
ASSERT(beginKey == res[i].key);
while (beginKey < res[i + 1].key) {
loop {
if (self->bulkLoadParallelismLimitor.canStart()) {
break;
}
wait(self->bulkLoadParallelismLimitor.waitUntilCounterChanged());
}
wait(self->bulkLoadParallelismLimitor.waitUntilCounterChanged());
std::vector<BulkLoadJobFileManifestEntry> manifestEntries;
while (manifestEntries.size() < SERVER_KNOBS->MANIFEST_COUNT_MAX_PER_BULKLOAD_TASK &&
beginKey < res[i + 1].key) {
auto it = self->bulkLoadJobManager.get().manifestEntryMap->find(beginKey);
ASSERT(it != self->bulkLoadJobManager.get().manifestEntryMap->end());
manifestEntry = it->second;
manifestEntries.push_back(manifestEntry);
beginKey = manifestEntry.getEndKey();
}
ASSERT(!manifestEntries.empty());
actors.push_back(bulkLoadJobNewTask(self,
jobState.getJobId(),
jobState.getJobRoot(),
jobState.getTransportMethod(),
self->bulkLoadJobManager.get().manifestLocalTempFolder,
manifestEntries,
errorOut));
}
auto it = self->bulkLoadJobManager.get().manifestEntryMap->find(beginKey);
ASSERT(it != self->bulkLoadJobManager.get().manifestEntryMap->end());
manifestEntry = it->second;
actors.push_back(bulkLoadJobExecuteTask(self,
jobState.getJobId(),
jobState.getJobRoot(),
jobState.getTransportMethod(),
self->bulkLoadJobManager.get().manifestLocalTempFolder,
manifestEntry,
errorOut));
beginKey = manifestEntry.getEndKey();
ASSERT(beginKey == res[i + 1].key);
}
if (beginKey == jobState.getJobRange().end) {
// last round

View File

@ -1897,10 +1897,9 @@ ACTOR static Future<Void> startMoveShards(Database occ,
bulkLoadTaskStateValue(newBulkLoadTaskState)));
TraceEvent(
bulkLoadVerboseEventSev(), "DDBulkLoadTaskSetRunningStateTransaction", relocationIntervalId)
.setMaxEventLength(-1)
.setMaxFieldLength(-1)
.detail("DataMoveID", dataMoveId)
.detail("BulkLoadTaskState", newBulkLoadTaskState.toString());
.detail("JobID", newBulkLoadTaskState.getJobId().toString())
.detail("TaskID", newBulkLoadTaskState.getTaskId().toString());
dataMove.bulkLoadTaskState = newBulkLoadTaskState;
}
dataMove.setPhase(DataMoveMetaData::Running);
@ -1927,10 +1926,10 @@ ACTOR static Future<Void> startMoveShards(Database occ,
if (currentKeys.end == keys.end && bulkLoadTaskState.present()) {
Version commitVersion = tr.getCommittedVersion();
TraceEvent(bulkLoadVerboseEventSev(), "DDBulkLoadTaskPersistRunningState", relocationIntervalId)
.setMaxEventLength(-1)
.setMaxFieldLength(-1)
.detail("DataMoveID", dataMoveId)
.detail("BulkLoadTaskState", bulkLoadTaskState.get().toString())
.detail("JobID", bulkLoadTaskState.get().getJobId().toString())
.detail("DataMoveID", dataMoveId.toString())
.detail("TaskID", bulkLoadTaskState.get().getTaskId().toString())
.detail("TaskRange", bulkLoadTaskState.get().getRange().toString())
.detail("CommitVersion", commitVersion);
}
@ -2349,10 +2348,9 @@ ACTOR static Future<Void> finishMoveShards(Database occ,
bulkLoadTaskStateValue(newBulkLoadTaskState)));
TraceEvent(
bulkLoadVerboseEventSev(), "DDBulkLoadTaskSetCompleteTransaction", relocationIntervalId)
.setMaxEventLength(-1)
.setMaxFieldLength(-1)
.detail("DataMoveID", dataMoveId)
.detail("BulkLoadTaskState", newBulkLoadTaskState.toString());
.detail("JobID", newBulkLoadTaskState.getJobId().toString())
.detail("TaskID", newBulkLoadTaskState.getTaskId().toString());
dataMove.bulkLoadTaskState = newBulkLoadTaskState;
}
wait(deleteCheckpoints(&tr, dataMove.checkpoints, dataMoveId));
@ -2376,10 +2374,10 @@ ACTOR static Future<Void> finishMoveShards(Database occ,
Version commitVersion = tr.getCommittedVersion();
TraceEvent(
bulkLoadVerboseEventSev(), "DDBulkLoadTaskPersistCompleteState", relocationIntervalId)
.setMaxEventLength(-1)
.setMaxFieldLength(-1)
.detail("DataMoveID", dataMoveId)
.detail("BulkLoadTaskState", bulkLoadTaskState.get().toString())
.detail("JobID", bulkLoadTaskState.get().getJobId().toString())
.detail("DataMoveID", dataMoveId.toString())
.detail("TaskID", bulkLoadTaskState.get().getTaskId().toString())
.detail("TaskRange", bulkLoadTaskState.get().getRange().toString())
.detail("CommitVersion", commitVersion);
}

View File

@ -56,6 +56,12 @@ ACTOR Future<BulkLoadFileSet> bulkLoadDownloadTaskFileSet(BulkLoadTransportMetho
std::string toLocalRoot,
UID logId);
ACTOR Future<Void> bulkLoadDownloadTaskFileSets(BulkLoadTransportMethod transportMethod,
std::shared_ptr<BulkLoadFileSetKeyMap> fromRemoteFileSets,
std::shared_ptr<BulkLoadFileSetKeyMap> localFileSets,
std::string toLocalRoot,
UID logId);
ACTOR Future<bool> doBytesSamplingOnDataFile(std::string dataFileFullPath,
std::string byteSampleFileFullPath,
UID logId);
@ -74,11 +80,12 @@ ACTOR Future<Void> getBulkLoadJobFileManifestEntryFromJobManifestFile(
std::shared_ptr<BulkLoadManifestFileMap> manifestMap);
// Get BulkLoad manifest metadata from the entry in the job manifest file
ACTOR Future<BulkLoadManifest> getBulkLoadManifestMetadataFromEntry(BulkLoadJobFileManifestEntry manifestEntry,
std::string manifestLocalTempFolder,
BulkLoadTransportMethod transportMethod,
std::string jobRoot,
UID logId);
ACTOR Future<BulkLoadManifestSet> getBulkLoadManifestMetadataFromEntry(
std::vector<BulkLoadJobFileManifestEntry> manifestEntries,
std::string manifestLocalTempFolder,
BulkLoadTransportMethod transportMethod,
std::string jobRoot,
UID logId);
#include "flow/unactorcompiler.h"
#endif

View File

@ -585,7 +585,7 @@ public:
// Return true if there exists a bulk load job/task or the collection has not been initialized.
// This takes effect only when DDBulkLoad Mode is enabled.
bool onBulkLoading(KeyRange range) {
bool bulkLoading(const KeyRange& range) {
if (!initialized) {
return true;
}
@ -660,12 +660,14 @@ public:
if (it->value().get().completeAck.canBeSet()) {
it->value().get().completeAck.sendError(bulkload_task_outdated());
TraceEvent(bulkLoadVerboseEventSev(), "DDBulkLoadTaskCollectionPublishTaskOverwriteTask", ddId)
.setMaxEventLength(-1)
.setMaxFieldLength(-1)
.detail("NewRange", bulkLoadTaskState.getRange())
.detail("NewTask", task.toString())
.detail("NewTaskRange", bulkLoadTaskState.getRange())
.detail("NewJobId", task.coreState.getJobId().toString())
.detail("NewTaskId", task.coreState.getTaskId().toString())
.detail("NewCommitVersion", task.commitVersion)
.detail("OldTaskRange", it->range())
.detail("OldTask", it->value().get().toString());
.detail("OldJobId", it->value().get().coreState.getJobId().toString())
.detail("OldTaskId", it->value().get().coreState.getTaskId().toString())
.detail("OldCommitVersion", it->value().get().commitVersion);
}
}
bulkLoadTaskMap.insert(bulkLoadTaskState.getRange(), task);

View File

@ -8741,51 +8741,79 @@ bool fetchKeyCanRetry(const Error& e) {
}
}
ACTOR Future<BulkLoadFileSet> bulkLoadFetchKeyValueFileToLoad(StorageServer* data,
std::string dir,
BulkLoadTaskState bulkLoadTaskState) {
ACTOR Future<Void> bulkLoadFetchKeyValueFileToLoad(StorageServer* data,
std::string dir,
BulkLoadTaskState bulkLoadTaskState,
std::shared_ptr<BulkLoadFileSetKeyMap> localFileSets) {
localFileSets->clear();
ASSERT(bulkLoadTaskState.getLoadType() == BulkLoadType::SST);
TraceEvent(bulkLoadVerboseEventSev(), "SSBulkLoadTaskFetchSSTFile", data->thisServerID)
.setMaxEventLength(-1)
.setMaxFieldLength(-1)
.detail("BulkLoadTask", bulkLoadTaskState.toString())
.detail("JobID", bulkLoadTaskState.getJobId().toString())
.detail("TaskID", bulkLoadTaskState.getTaskId().toString())
.detail("Dir", abspath(dir));
state double fetchStartTime = now();
// Download data file from fromRemoteFileSet to toLocalFileSet
state BulkLoadFileSet fromRemoteFileSet = bulkLoadTaskState.getFileSet();
state BulkLoadFileSet toLocalFileSet = wait(bulkLoadDownloadTaskFileSet(
bulkLoadTaskState.getTransportMethod(), fromRemoteFileSet, dir, data->thisServerID));
state std::shared_ptr<BulkLoadFileSetKeyMap> fromRemoteFileSets = std::make_shared<BulkLoadFileSetKeyMap>();
for (const auto& manifest : bulkLoadTaskState.getManifests()) {
fromRemoteFileSets->push_back(std::make_pair(manifest.getRange(), manifest.getFileSet()));
}
wait(bulkLoadDownloadTaskFileSets(
bulkLoadTaskState.getTransportMethod(), fromRemoteFileSets, localFileSets, dir, data->thisServerID));
// Do not need byte sampling locally in fetchKeys
const double duration = now() - fetchStartTime;
const int64_t totalBytes = bulkLoadTaskState.getTotalBytes();
std::string localFileSetString;
int count = 0;
for (auto iter = localFileSets->cbegin(); iter < localFileSets->cend(); iter++) {
localFileSetString = localFileSetString + iter->first.toString() + ", " + iter->second.toString();
count++;
if (count < localFileSets->size()) {
localFileSetString = localFileSetString + ", ";
}
}
TraceEvent(bulkLoadVerboseEventSev(), "SSBulkLoadTaskFetchSSTFileFetched", data->thisServerID)
.setMaxEventLength(-1)
.setMaxFieldLength(-1)
.detail("BulkLoadTask", bulkLoadTaskState.toString())
.detail("JobID", bulkLoadTaskState.getJobId().toString())
.detail("TaskID", bulkLoadTaskState.getTaskId().toString())
.detail("Dir", abspath(dir))
.detail("LocalFileSet", toLocalFileSet.toString())
.detail("LocalFileSetMap", localFileSetString)
.detail("Duration", duration)
.detail("TotalBytes", totalBytes)
.detail("Rate", duration == 0 ? -1.0 : (double)totalBytes / duration);
return toLocalFileSet;
return Void();
}
ACTOR Future<Void> tryGetRangeForBulkLoad(PromiseStream<RangeResult> results, KeyRange keys, std::string dataPath) {
ACTOR Future<Void> tryGetRangeForBulkLoadFromSST(PromiseStream<RangeResult> results,
KeyRange keys,
std::string sstFilePath,
bool lastOne) {
state Key beginKey = keys.begin;
state Key endKey = keys.end;
try {
// TODO(BulkLoad): what if the data file is empty but the totalKeyCount is not zero
state Key beginKey = keys.begin;
state Key endKey = keys.end;
state std::unique_ptr<IRocksDBSstFileReader> reader = newRocksDBSstFileReader(
keys, SERVER_KNOBS->SS_BULKLOAD_GETRANGE_BATCH_SIZE, SERVER_KNOBS->FETCH_BLOCK_BYTES);
// TODO(BulkLoad): this can be a slow task. We will make this as async call.
reader->open(abspath(dataPath));
reader->open(abspath(sstFilePath));
loop {
// TODO(BulkLoad): this is a blocking call. We will make this as async call.
RangeResult rep = reader->getRange(KeyRangeRef(beginKey, endKey));
results.send(rep);
if (!rep.more) {
results.sendError(end_of_stream());
return Void();
if (lastOne) {
rep.more = false;
results.send(rep);
results.sendError(end_of_stream());
return Void();
} else {
if (!rep.empty()) {
rep.more = true;
// Avoid breaking readThrough contract
// The reply cannot be empty of the more is true
results.send(rep);
}
return Void();
}
} else {
results.send(rep);
}
beginKey = keyAfter(rep.back().key);
wait(delay(0.1)); // context switch to avoid busy loop
@ -8799,6 +8827,44 @@ ACTOR Future<Void> tryGetRangeForBulkLoad(PromiseStream<RangeResult> results, Ke
}
}
ACTOR Future<Void> tryGetRangeForBulkLoad(PromiseStream<RangeResult> results,
KeyRange keys,
std::shared_ptr<BulkLoadFileSetKeyMap> localFileSets) {
try {
// Build bulkLoadFileSetsToLoad
state std::vector<std::pair<KeyRange, BulkLoadFileSet>> bulkLoadFileSetsToLoad;
KeyRangeMap<BulkLoadFileSet> localFileSetMap;
for (auto it = localFileSets->begin(); it < localFileSets->end(); it++) {
localFileSetMap.insert(it->first, it->second);
}
for (auto range : localFileSetMap.intersectingRanges(keys)) {
if (!range->value().isValid()) {
continue;
}
bulkLoadFileSetsToLoad.push_back(std::make_pair(range->range(), range->value()));
}
// Streaming results given the input keys using bulkLoadFileSetsToLoad
state int i = 0;
for (; i < bulkLoadFileSetsToLoad.size(); i++) {
std::string sstFilePath = bulkLoadFileSetsToLoad[i].second.getDataFileFullPath();
KeyRange rangeToLoad = bulkLoadFileSetsToLoad[i].first & keys;
ASSERT(!rangeToLoad.empty());
wait(tryGetRangeForBulkLoadFromSST(
results, rangeToLoad, sstFilePath, i == bulkLoadFileSetsToLoad.size() - 1));
}
return Void();
} catch (Error& e) {
if (e.code() == error_code_actor_cancelled) {
throw;
}
if (e.code() == error_code_inverted_range) {
TraceEvent(SevError, "Zhe1").detail("Keys", keys);
}
results.sendError(bulkload_task_failed());
throw;
}
}
ACTOR Future<Void> fetchKeys(StorageServer* data, AddingShard* shard) {
state const UID fetchKeysID = deterministicRandom()->randomUniqueID();
state TraceInterval interval("FetchKeys");
@ -8812,6 +8878,7 @@ ACTOR Future<Void> fetchKeys(StorageServer* data, AddingShard* shard) {
state ConductBulkLoad conductBulkLoad = ConductBulkLoad(shard->getSSBulkLoadMetadata().getConductBulkLoad());
state std::string bulkLoadLocalDir =
joinPath(joinPath(data->bulkLoadFolder, dataMoveId.toString()), fetchKeysID.toString());
state std::shared_ptr<BulkLoadFileSetKeyMap> localBulkLoadFileSets;
// Since the fetchKey can split, so multiple fetchzkeys can have the same data move id. We want each fetchkey
// downloads its file without conflict, so we add fetchKeysID to the bulkLoadLocalDir.
state PromiseStream<Key> destroyedFeeds;
@ -9015,10 +9082,6 @@ ACTOR Future<Void> fetchKeys(StorageServer* data, AddingShard* shard) {
rangeEnd = keys.end;
}
} else if (conductBulkLoad) {
TraceEvent(bulkLoadVerboseEventSev(), "SSBulkLoadTaskFetchKey", data->thisServerID)
.detail("DataMoveId", dataMoveId.toString())
.detail("Range", keys)
.detail("Phase", "Read task metadata");
ASSERT(dataMoveIdIsValidForBulkLoad(dataMoveId)); // TODO(BulkLoad): remove dangerous assert
// Get the bulkload task metadata from the data move metadata. Note that a SS can receive a data move
// mutation before the bulkload task metadata is persisted. In this case, the SS will not be able to
@ -9039,9 +9102,14 @@ ACTOR Future<Void> fetchKeys(StorageServer* data, AddingShard* shard) {
// dataMoveId.
ASSERT(bulkLoadTaskState.getDataMoveId() == dataMoveId);
// We download the data file to local disk and pass the data file path to read in the next step.
BulkLoadFileSet localFileSet =
wait(bulkLoadFetchKeyValueFileToLoad(data, bulkLoadLocalDir, bulkLoadTaskState));
hold = tryGetRangeForBulkLoad(results, keys, localFileSet.getDataFileFullPath());
localBulkLoadFileSets = std::make_shared<BulkLoadFileSetKeyMap>();
wait(bulkLoadFetchKeyValueFileToLoad(
data, bulkLoadLocalDir, bulkLoadTaskState, /*output=*/localBulkLoadFileSets));
TraceEvent(bulkLoadVerboseEventSev(), "SSBulkLoadTaskFetchKey", data->thisServerID)
.detail("DataMoveId", dataMoveId.toString())
.detail("Range", keys)
.detail("Phase", "File download");
hold = tryGetRangeForBulkLoad(results, keys, localBulkLoadFileSets);
rangeEnd = keys.end;
} else {
hold = tryGetRange(results, &tr, keys);
@ -9563,16 +9631,18 @@ ACTOR Future<Void> bulkLoadFetchShardFileToLoad(StorageServer* data,
BulkLoadTaskState bulkLoadTaskState) {
ASSERT(bulkLoadTaskState.getLoadType() == BulkLoadType::SST);
TraceEvent(bulkLoadVerboseEventSev(), "SSBulkLoadTaskFetchShardFile", data->thisServerID)
.setMaxEventLength(-1)
.setMaxFieldLength(-1)
.detail("BulkLoadTask", bulkLoadTaskState.toString())
.detail("JobID", bulkLoadTaskState.getJobId().toString())
.detail("TaskID", bulkLoadTaskState.getTaskId().toString())
.detail("MoveInShard", moveInShard->toString())
.detail("LocalRoot", abspath(localRoot));
state double fetchStartTime = now();
// Step 1: Download files to localRoot
state BulkLoadFileSet fromRemoteFileSet = bulkLoadTaskState.getFileSet();
// TODO(BulkLoad): support bulkload task mutiple sst for sharded rocksdb.
ASSERT(SERVER_KNOBS->MANIFEST_COUNT_MAX_PER_BULKLOAD_TASK == 1);
ASSERT(bulkLoadTaskState.getManifests().size() == 1);
state BulkLoadFileSet fromRemoteFileSet = bulkLoadTaskState.getManifests()[0].getFileSet();
BulkLoadByteSampleSetting currentClusterByteSampleSetting(
0,
"hashlittle2", // use function name to represent the method
@ -9589,9 +9659,8 @@ ACTOR Future<Void> bulkLoadFetchShardFileToLoad(StorageServer* data,
state BulkLoadFileSet toLocalFileSet = wait(bulkLoadDownloadTaskFileSet(
bulkLoadTaskState.getTransportMethod(), fromRemoteFileSet, localRoot, data->thisServerID));
TraceEvent(bulkLoadVerboseEventSev(), "SSBulkLoadTaskFetchShardSSTFileFetched", data->thisServerID)
.setMaxEventLength(-1)
.setMaxFieldLength(-1)
.detail("BulkLoadTask", bulkLoadTaskState.toString())
.detail("JobID", bulkLoadTaskState.getJobId().toString())
.detail("TaskID", bulkLoadTaskState.getTaskId().toString())
.detail("MoveInShard", moveInShard->toString())
.detail("RemoteFileSet", fromRemoteFileSet.toString())
.detail("LocalFileSet", toLocalFileSet.toString());
@ -9600,9 +9669,8 @@ ACTOR Future<Void> bulkLoadFetchShardFileToLoad(StorageServer* data,
if (!toLocalFileSet.hasByteSampleFile()) {
TraceEvent(
bulkLoadVerboseEventSev(), "SSBulkLoadTaskFetchShardSSTFileValidByteSampleNotFound", data->thisServerID)
.setMaxEventLength(-1)
.setMaxFieldLength(-1)
.detail("BulkLoadTaskState", bulkLoadTaskState.toString())
.detail("JobID", bulkLoadTaskState.getJobId().toString())
.detail("TaskID", bulkLoadTaskState.getTaskId().toString())
.detail("LocalFileSet", toLocalFileSet.toString());
state std::string byteSampleFileName =
generateBulkLoadBytesSampleFileNameFromDataFileName(toLocalFileSet.getDataFileName());
@ -9614,9 +9682,8 @@ ACTOR Future<Void> bulkLoadFetchShardFileToLoad(StorageServer* data,
}
}
TraceEvent(bulkLoadVerboseEventSev(), "SSBulkLoadTaskFetchShardByteSampled", data->thisServerID)
.setMaxEventLength(-1)
.setMaxFieldLength(-1)
.detail("BulkLoadTask", bulkLoadTaskState.toString())
.detail("JobID", bulkLoadTaskState.getJobId().toString())
.detail("TaskID", bulkLoadTaskState.getTaskId().toString())
.detail("MoveInShard", moveInShard->toString())
.detail("RemoteFileSet", fromRemoteFileSet.toString())
.detail("LocalFileSet", toLocalFileSet.toString());
@ -9655,9 +9722,8 @@ ACTOR Future<Void> bulkLoadFetchShardFileToLoad(StorageServer* data,
const double duration = now() - fetchStartTime;
const int64_t totalBytes = getTotalFetchedBytes(moveInShard->meta->checkpoints);
TraceEvent(bulkLoadVerboseEventSev(), "SSBulkLoadTaskFetchShardSSTFileBuildMetadata", data->thisServerID)
.setMaxEventLength(-1)
.setMaxFieldLength(-1)
.detail("BulkLoadTask", bulkLoadTaskState.toString())
.detail("JobID", bulkLoadTaskState.getJobId().toString())
.detail("TaskID", bulkLoadTaskState.getTaskId().toString())
.detail("MoveInShard", moveInShard->toString())
.detail("LocalRoot", abspath(localRoot))
.detail("LocalFileSet", toLocalFileSet.toString())

View File

@ -109,9 +109,7 @@ struct BulkLoading : TestWorkload {
loop {
try {
wait(submitBulkLoadTask(cx, tasks[i]));
TraceEvent(bulkLoadVerboseEventSev(), "BulkLoadingSubmitBulkLoadTask")
.setMaxEventLength(-1)
.setMaxFieldLength(-1)
TraceEvent(SevDebug, "BulkLoadingSubmitBulkLoadTask")
.detail("BulkLoadTaskState", tasks[i].toString());
break;
} catch (Error& e) {
@ -133,7 +131,7 @@ struct BulkLoading : TestWorkload {
loop {
try {
wait(finalizeBulkLoadTask(cx, tasks[i].getRange(), tasks[i].getTaskId()));
TraceEvent(bulkLoadVerboseEventSev(), "BulkLoadingAcknowledgeBulkLoadTask")
TraceEvent(SevDebug, "BulkLoadingAcknowledgeBulkLoadTask")
.setMaxEventLength(-1)
.setMaxFieldLength(-1)
.detail("BulkLoadTaskState", tasks[i].toString());