Checkpointing work
This commit is contained in:
@@ -102,12 +102,12 @@ namespace daggy {
|
||||
while (t.has_value()) {
|
||||
// Schedule the task to run
|
||||
auto &taskName = t.value().first;
|
||||
auto &task = t.value().second;
|
||||
taskAttemptCounts_[taskName] = 1;
|
||||
|
||||
logger_.updateTaskState(runID_, taskName, RunState::RUNNING);
|
||||
try {
|
||||
auto fut = executor_.execute(runID_, taskName, task);
|
||||
auto &task = t.value().second;
|
||||
auto fut = executor_.execute(runID_, taskName, task);
|
||||
runningTasks_.emplace(taskName, std::move(fut));
|
||||
}
|
||||
catch (std::exception &e) {
|
||||
@@ -125,8 +125,8 @@ namespace daggy {
|
||||
void DAGRunner::collectFinished()
|
||||
{
|
||||
for (auto &[taskName, fut] : runningTasks_) {
|
||||
if (fut.valid() and fut.wait_for(1ms) == std::future_status::ready) {
|
||||
auto attempt = fut.get();
|
||||
if (fut->ready()) {
|
||||
auto attempt = fut->get();
|
||||
logger_.logTaskAttempt(runID_, taskName, attempt);
|
||||
|
||||
// Not a reference, since adding tasks will invalidate references
|
||||
|
||||
@@ -234,7 +234,7 @@ namespace daggy {
|
||||
curl_easy_setopt(curl, CURLOPT_URL, url.c_str());
|
||||
curl_easy_setopt(curl, CURLOPT_WRITEFUNCTION, curlWriter);
|
||||
curl_easy_setopt(curl, CURLOPT_WRITEDATA, &buffer);
|
||||
curl_easy_setopt(curl, CURLOPT_TIMEOUT, 2);
|
||||
// curl_easy_setopt(curl, CURLOPT_TIMEOUT, 30);
|
||||
|
||||
if (trace) {
|
||||
curl_easy_setopt(curl, CURLOPT_DEBUGFUNCTION, http_trace);
|
||||
|
||||
@@ -145,8 +145,9 @@ std::vector<ConfigValues> DaggyRunnerTaskExecutor::expandTaskParameters(
|
||||
}
|
||||
|
||||
// Runs the task
|
||||
std::future<AttemptRecord> DaggyRunnerTaskExecutor::execute(
|
||||
DAGRunID runID, const std::string &taskName, const Task &task)
|
||||
TaskFuture DaggyRunnerTaskExecutor::execute(DAGRunID runID,
|
||||
const std::string &taskName,
|
||||
const Task &task)
|
||||
{
|
||||
auto taskUsed = capacityFromTask(task);
|
||||
|
||||
@@ -183,11 +184,9 @@ std::future<AttemptRecord> DaggyRunnerTaskExecutor::execute(
|
||||
}
|
||||
|
||||
if (impacts.empty()) {
|
||||
std::promise<AttemptRecord> prom;
|
||||
auto fut = prom.get_future();
|
||||
AttemptRecord record{.rc = -1,
|
||||
.executorLog = "No runners available for execution"};
|
||||
prom.set_value(std::move(record));
|
||||
auto fut = std::make_shared<Future<AttemptRecord>>();
|
||||
fut->set(AttemptRecord{
|
||||
.rc = -1, .executorLog = "No runners available for execution"});
|
||||
return fut;
|
||||
}
|
||||
}
|
||||
@@ -217,22 +216,20 @@ std::future<AttemptRecord> DaggyRunnerTaskExecutor::execute(
|
||||
}
|
||||
|
||||
if (submitted_runner.empty()) {
|
||||
std::promise<AttemptRecord> prom;
|
||||
auto fut = prom.get_future();
|
||||
AttemptRecord record{.rc = -1,
|
||||
.executorLog = "No runners available for execution"};
|
||||
prom.set_value(std::move(record));
|
||||
auto fut = std::make_shared<Future<AttemptRecord>>();
|
||||
fut->set(AttemptRecord{
|
||||
.rc = -1, .executorLog = "No runners available for execution"});
|
||||
return fut;
|
||||
}
|
||||
|
||||
RunningTask rt{.prom{},
|
||||
RunningTask rt{.fut = std::make_shared<Future<AttemptRecord>>(),
|
||||
.runID = runID,
|
||||
.taskName = taskName,
|
||||
.runnerURL = submitted_runner,
|
||||
.retries = 3,
|
||||
.resources = taskUsed};
|
||||
|
||||
auto fut = rt.prom.get_future();
|
||||
TaskFuture fut = rt.fut;
|
||||
|
||||
std::lock_guard<std::mutex> lock(rtGuard_);
|
||||
runningTasks_.emplace(std::make_pair(runID, taskName), std::move(rt));
|
||||
@@ -293,34 +290,34 @@ void DaggyRunnerTaskExecutor::monitor()
|
||||
rj::Document doc;
|
||||
try {
|
||||
auto [code, json] = JSON_HTTP_REQUEST(runnerURL + "/v1/poll");
|
||||
if (code != HTTPCode::Ok)
|
||||
if (code != HTTPCode::Ok) {
|
||||
std::cout << "Unable to poll: " << code << ": " << dumpJSON(json)
|
||||
<< std::endl;
|
||||
continue;
|
||||
}
|
||||
|
||||
doc.Swap(json);
|
||||
}
|
||||
catch (std::exception &e) {
|
||||
std::cout << "Unable to poll: " << e.what() << std::endl;
|
||||
continue;
|
||||
}
|
||||
std::cout << "Doc is now: " << doc.Size() << std::endl;
|
||||
|
||||
const auto tasks = doc.GetArray();
|
||||
for (size_t idx = 0; idx < tasks.Size(); ++idx) {
|
||||
const auto &task = tasks[idx];
|
||||
auto tid = std::make_pair(task["runID"].GetInt64(),
|
||||
task["taskName"].GetString());
|
||||
|
||||
if (task["state"] == "PENDING") {
|
||||
resolvedJobs.emplace(tid, std::nullopt);
|
||||
auto it = taskResources.find(tid);
|
||||
if (it != taskResources.end()) {
|
||||
const auto &res = taskResources.at(tid);
|
||||
caps.current.cores += res.cores;
|
||||
caps.current.memoryMB += res.memoryMB;
|
||||
}
|
||||
else {
|
||||
auto it = taskResources.find(tid);
|
||||
if (it != taskResources.end()) {
|
||||
const auto &res = taskResources.at(tid);
|
||||
caps.current.cores += res.cores;
|
||||
caps.current.memoryMB += res.memoryMB;
|
||||
}
|
||||
|
||||
auto attempt = attemptRecordFromJSON(task["attempt"]);
|
||||
resolvedJobs.emplace(tid, attemptRecordFromJSON(task["attempt"]));
|
||||
}
|
||||
auto attempt = attemptRecordFromJSON(task["attempt"]);
|
||||
resolvedJobs.emplace(tid, attemptRecordFromJSON(task["attempt"]));
|
||||
}
|
||||
}
|
||||
|
||||
@@ -329,20 +326,11 @@ void DaggyRunnerTaskExecutor::monitor()
|
||||
std::lock_guard<std::mutex> lock(rtGuard_);
|
||||
for (auto &[taskID, task] : runningTasks_) {
|
||||
auto it = resolvedJobs.find(taskID);
|
||||
if (it == resolvedJobs.end()) {
|
||||
--task.retries;
|
||||
|
||||
if (task.retries == 0) {
|
||||
AttemptRecord record{
|
||||
.rc = -1, .executorLog = "Unable to query runner for progress"};
|
||||
task.prom.set_value(std::move(record));
|
||||
completedTasks.emplace_back(taskID);
|
||||
}
|
||||
if (it == resolvedJobs.end())
|
||||
continue;
|
||||
}
|
||||
else if (it->second.has_value()) {
|
||||
if (it->second.has_value()) {
|
||||
// Task has completed
|
||||
task.prom.set_value(it->second.value());
|
||||
task.fut->set(std::move(it->second.value()));
|
||||
completedTasks.emplace_back(taskID);
|
||||
}
|
||||
}
|
||||
@@ -351,6 +339,6 @@ void DaggyRunnerTaskExecutor::monitor()
|
||||
}
|
||||
}
|
||||
|
||||
std::this_thread::sleep_for(std::chrono::seconds(1));
|
||||
std::this_thread::sleep_for(std::chrono::seconds(10));
|
||||
}
|
||||
}
|
||||
|
||||
@@ -90,8 +90,9 @@ bool ForkingTaskExecutor::stop(DAGRunID runID, const std::string &taskName)
|
||||
return true;
|
||||
}
|
||||
|
||||
std::future<daggy::AttemptRecord> ForkingTaskExecutor::execute(
|
||||
DAGRunID runID, const std::string &taskName, const Task &task)
|
||||
TaskFuture ForkingTaskExecutor::execute(DAGRunID runID,
|
||||
const std::string &taskName,
|
||||
const Task &task)
|
||||
{
|
||||
std::string key = std::to_string(runID) + "_" + taskName;
|
||||
std::lock_guard<std::mutex> lock(taskControlsGuard_);
|
||||
|
||||
@@ -8,18 +8,20 @@ namespace daggy::executors::task {
|
||||
return "NoopTaskExecutor";
|
||||
}
|
||||
|
||||
std::future<daggy::AttemptRecord> NoopTaskExecutor::execute(
|
||||
DAGRunID runID, const std::string &taskName, const Task &task)
|
||||
TaskFuture NoopTaskExecutor::execute(DAGRunID runID,
|
||||
const std::string &taskName,
|
||||
const Task &task)
|
||||
{
|
||||
std::promise<daggy::AttemptRecord> promise;
|
||||
auto ts = Clock::now();
|
||||
promise.set_value(AttemptRecord{.startTime = ts,
|
||||
.stopTime = ts,
|
||||
.rc = 0,
|
||||
.executorLog = taskName,
|
||||
.outputLog = taskName,
|
||||
.errorLog = taskName});
|
||||
return promise.get_future();
|
||||
auto ts = Clock::now();
|
||||
auto fut = std::make_shared<Future<AttemptRecord>>();
|
||||
fut->set(AttemptRecord{.startTime = ts,
|
||||
.stopTime = ts,
|
||||
.rc = 0,
|
||||
.executorLog = taskName,
|
||||
.outputLog = taskName,
|
||||
.errorLog = taskName});
|
||||
return fut;
|
||||
}
|
||||
|
||||
bool NoopTaskExecutor::validateTaskParameters(const ConfigValues &job)
|
||||
|
||||
@@ -87,8 +87,7 @@ namespace daggy::executors::task {
|
||||
// Resolve the remaining futures
|
||||
std::lock_guard<std::mutex> lock(promiseGuard_);
|
||||
for (auto &[jobID, job] : runningJobs_) {
|
||||
job.prom.set_value(
|
||||
AttemptRecord{.rc = -1, .executorLog = "executor killed"});
|
||||
job.fut->set(AttemptRecord{.rc = -1, .executorLog = "executor killed"});
|
||||
}
|
||||
runningJobs_.clear();
|
||||
}
|
||||
@@ -153,8 +152,9 @@ namespace daggy::executors::task {
|
||||
return newValues;
|
||||
}
|
||||
|
||||
std::future<AttemptRecord> SlurmTaskExecutor::execute(
|
||||
DAGRunID runID, const std::string &taskName, const Task &task)
|
||||
TaskFuture SlurmTaskExecutor::execute(DAGRunID runID,
|
||||
const std::string &taskName,
|
||||
const Task &task)
|
||||
{
|
||||
std::stringstream executorLog;
|
||||
|
||||
@@ -247,12 +247,12 @@ namespace daggy::executors::task {
|
||||
slurm_free_submit_response_response_msg(resp_msg);
|
||||
|
||||
std::lock_guard<std::mutex> lock(promiseGuard_);
|
||||
Job newJob{.prom{},
|
||||
Job newJob{.fut = std::make_shared<Future<AttemptRecord>>(),
|
||||
.stdoutFile = stdoutFile,
|
||||
.stderrFile = stderrFile,
|
||||
.runID = runID,
|
||||
.taskName = taskName};
|
||||
auto fut = newJob.prom.get_future();
|
||||
auto fut = newJob.fut;
|
||||
runningJobs_.emplace(jobID, std::move(newJob));
|
||||
|
||||
return fut;
|
||||
@@ -348,7 +348,7 @@ namespace daggy::executors::task {
|
||||
readAndClean(job.stdoutFile, record.outputLog);
|
||||
readAndClean(job.stderrFile, record.errorLog);
|
||||
|
||||
job.prom.set_value(std::move(record));
|
||||
job.fut->set(std::move(record));
|
||||
resolvedJobs.insert(jobID);
|
||||
}
|
||||
|
||||
|
||||
Reference in New Issue
Block a user