Allow One BulkloadTask Do Multiple Manifests (#12036)
This commit is contained in:
parent
d5946157f0
commit
0e736c68e7
|
@ -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,
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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.
|
||||
};
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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())
|
||||
|
|
|
@ -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());
|
||||
|
|
Loading…
Reference in New Issue