Skip to content

Conversation

@L-Xiafeng
Copy link
Collaborator

@L-Xiafeng L-Xiafeng commented Nov 6, 2025

Summary by CodeRabbit

Release Notes

  • New Features

    • Added task exit status tracking with exit codes and signal information
    • Introduced SSH process migration to container groups for improved resource isolation
    • Added CPU binding support for task execution
    • Enhanced X11 forwarding with improved connection handling
  • Improvements

    • Refined per-task resource allocation and mapping
    • Updated task status transitions for better state management
    • Enhanced logging for scheduling operations and resource management

✏️ Tip: You can customize this high-level summary in your review settings.

@L-Xiafeng L-Xiafeng added the enhancement New feature or request label Nov 6, 2025
@coderabbitai
Copy link
Contributor

coderabbitai bot commented Nov 6, 2025

Walkthrough

The PR refactors the job scheduling and task execution architecture from job-centric to step/task-centric model. Proto schema updates include deprecated execution_node field, renamed TaskStatus Configured→Starting, added per-task resource mapping, and new MigrateSshProcToCgroup RPC. Craned-side replaces SupervisorKeeper with per-step SupervisorStub instances, introduces StepInstance lifecycle management, refactors CgroupManager APIs to parse and track per-task cgroup identifiers, and updates I/O forwarding to handle exit status and X11 per-connection identifiers.

Changes

Cohort / File(s) Change Summary
Proto Schema Updates
protos/Crane.proto, protos/PublicDefs.proto, protos/Supervisor.proto
Deprecated execution_node field; renamed TaskStatus.ConfiguredStarting; added task_res_map to StepToD; added TaskExitStatusReply/TaskExitStatus messages and enums to StreamCrunReply and StreamTaskIORequest; added new RPC MigrateSshProcToCgroup and messages.
Ctld Scheduling & State Management
src/CraneCtld/CtldPublicDefs.cpp, src/CraneCtld/CtldPublicDefs.h
Added per-step tracking: m_configuring_nodes_, m_running_nodes_; replaced pending_reason with task_res_map and craned_task_map; updated state transitions from Configuring→Configured to Configuring→Starting; added per-task resource mapping initialization and propagation to StepToD.task_res_map.
Ctld Persistence & Scheduling Timing
src/CraneCtld/EmbeddedDbClient.cpp, src/CraneCtld/TaskScheduler.cpp
Updated task status recovery: Starting now enqueued with Running; added timing instrumentation around AppendSteps, AllocJobs, AllocSteps.
Craned Cgroup Management Core
src/Craned/Common/CgroupManager.h, src/Craned/Common/CgroupManager.cpp
Extended CgroupStrParsedIds to include system flag; added CgroupStrByParsedIds() stringification; renamed GetCgJobIdMapCgroupV2_GetCgInoJobIdMapCgroupV2_; updated all ID mapping functions to return parsed-IDs structures; added SetCpuBind() API; added CPUSET_CONTROLLER; renamed RecoverFromCgSpecRecoverFromResInNode; gated memory limits on non-zero values.
Craned Supervisor Lifecycle
src/Craned/Core/SupervisorStub.h, src/Craned/Core/SupervisorStub.cpp
Removed SupervisorKeeper class; added per-step constructors to SupervisorStub; added SupervisorRecoverInfo struct; added InitAndGetRecoveredMap() and MigrateSshProcToCg() methods.
Craned Step Instance
src/Craned/Core/StepInstance.h, src/Craned/Core/StepInstance.cpp
New struct encapsulating per-step state: lifecycle methods (CleanUp, CreateCg, SpawnSupervisor, GotNewStatus, ExecuteStepAsync); stores step_to_d, supervisor_stub, crane_cgroup.
Craned Core Integration
src/Craned/Core/CMakeLists.txt, src/Craned/Core/Craned.cpp, src/Craned/Core/CranedForPamServer.cpp, src/Craned/Core/CranedServer.cpp, src/Craned/Core/CtldClient.cpp
Replaced SupervisorKeeper.h with SupervisorStub.h; renamed RecoverCgForJobsRecoverCgForJobSteps; updated recovery flow to use parsed-IDs and StepInstance; refactored RPC handlers to use g_job_mgr-based queries instead of supervisor stubs.
Craned Job Management
src/Craned/Core/JobManager.h, src/Craned/Core/JobManager.cpp
Removed StepInstance definition (moved to separate file); added status field to JobInD; removed SpawnSupervisor_ and KillPid_ private methods; added public methods ChangeStepTimelimit() and QuerySshStepEnvVariables(); refactored lifecycle to use per-step supervisor_stub instead of global SupervisorKeeper.
Supervisor Task Management
src/Craned/Supervisor/TaskManager.h, src/Craned/Supervisor/TaskManager.cpp
Introduced StepInstance::Prepare(), CleanUp(), GotNewStatus(); added ITaskInstance::Prepare(), Cleanup(); added per-task X11 meta and cgroup tracking; added public methods SupervisorFinishInit(), ShutdownSupervisorAsync(), CheckStatusAsync(), MigrateSshProcToCgroupAsync(); reworked task execution to initialize per-task instances in parallel.
Supervisor Public Definitions & RPC Service
src/Craned/Supervisor/SupervisorPublicDefs.h, src/Craned/Supervisor/SupervisorServer.h, src/Craned/Supervisor/SupervisorServer.cpp
Removed RuntimeStatus struct and g_runtime_status; added RPC handler MigrateSshProcToCgroup(); updated QueryEnvMap, CheckStatus, ShutdownSupervisor to delegate to task manager async methods.
Supervisor Cfored Client I/O
src/Craned/Supervisor/CforedClient.h, src/Craned/Supervisor/CforedClient.cpp
Refactored X11 forwarding: removed per-connection port/proxy fields; added per-connection x11_local_id; unified output queues into FwdRequest variant; updated signatures: InitUvX11FwdHandler() removes task_id param; TaskProcessStop() adds exit_code/signaled; TaskOutPutForward() takes buffer/length; TaskX11OutPutForward() adds x11_local_id and eof.
Utilities & Constants
src/Utilities/PublicHeader/include/crane/String.h, src/Utilities/PublicHeader/include/crane/PublicHeader.h
Added MapRange and StepIdRange concepts; constrained StepToDRangeIdString() and JobStepsToString() with ranges; updated CraneStepStatusStrArr to reflect "Configured"→"Starting"; added KCrunExitCodeStatusNum constant.

Sequence Diagram

sequenceDiagram
    participant Ctld
    participant Craned
    participant SupervisorStub as Supervisor<br/>(per-step)
    participant TaskMgr as TaskManager

    Note over Ctld,TaskMgr: Step Creation & Scheduling
    Ctld->>Craned: SubmitStepRequest<br/>(with StepToD + task_res_map)
    Ctld->>Ctld: InitPrimaryStepFromJob<br/>(populate per-task resources)

    Note over Craned,TaskMgr: Step Instance Lifecycle
    Craned->>Craned: StepInstance::Prepare()
    activate Craned
    Craned->>Craned: CreateCg() for step
    Craned->>Craned: Initialize X11/auth
    deactivate Craned

    Craned->>Craned: StepInstance::SpawnSupervisor()
    activate Craned
    Craned->>Craned: Fork supervisor child
    Craned->>SupervisorStub: Send InitSupervisorRequest
    SupervisorStub->>TaskMgr: SupervisorFinishInit()
    activate TaskMgr
    TaskMgr->>TaskMgr: Initialize per-task instances
    TaskMgr->>TaskMgr: ITaskInstance::Prepare()<br/>(per-task CG allocation)
    deactivate TaskMgr
    SupervisorStub-->>Craned: SupervisorReady
    deactivate Craned

    Note over Craned,TaskMgr: Task Execution
    TaskMgr->>TaskMgr: EvGrpcExecuteTaskCb_<br/>(spawn tasks in parallel)
    TaskMgr->>TaskMgr: Migrate tasks to per-task CGs
    TaskMgr->>TaskMgr: ExecuteStepAsync()

    Note over Craned,TaskMgr: Task Termination & Exit Status
    rect rgb(220, 200, 255)
    Note over Craned: New Flow - Per-Task Exit Status
    TaskMgr->>TaskMgr: EvCleanSigchldQueueCb_<br/>(collect per-task exit codes)
    TaskMgr->>SupervisorStub: TaskProcessStop(exit_code, signaled)
    SupervisorStub->>Craned: Send TaskExitStatus
    Craned->>Ctld: Propagate task status change
    end

    Note over Craned,TaskMgr: Cleanup
    Craned->>SupervisorStub: MigrateSshProcToCgroup(pid)
    Craned->>Craned: StepInstance::CleanUp()
    TaskMgr->>TaskMgr: ITaskInstance::Cleanup()
Loading

Estimated code review effort

🎯 4 (Complex) | ⏱️ ~60 minutes

Areas requiring extra attention:

  • Proto schema changes and state machine transitions: Verify the new Starting state and per-task resource mapping integration across Ctld scheduling, database persistence, and Craned recovery paths; confirm task status transitions remain consistent.
  • Cgroup management API refactoring: The parsed-IDs representation (CgroupStrParsedIds) with system flag integration is central to recovery and allocation; verify all call sites extract indices correctly and handle the new tuple structure.
  • SupervisorStub lifecycle management: Replacing global SupervisorKeeper with per-step instances requires verifying stub initialization, cleanup, and RPC error handling in JobManager and StepInstance flows.
  • StepInstance and TaskManager integration: New per-task CG allocation, spawn, and async shutdown paths must be verified for correctness, including error propagation and resource cleanup on failure.
  • CforedClient I/O refactoring: The unified FwdRequest queue and X11 per-connection state management introduce complexity; verify buffering, message serialization, and per-connection lifecycle (close/EOF handling).
  • Craned recovery flow: The updated RecoverCgForJobSteps() with parsed-IDs recovery and step-level cgroup discovery requires careful verification against stored cgroup hierarchies and stale CG cleanup logic.

Possibly related PRs

Suggested labels

refactor

Suggested reviewers

  • RileyWen
  • NamelessOIer

Pre-merge checks and finishing touches

❌ Failed checks (1 warning)
Check name Status Explanation Resolution
Docstring Coverage ⚠️ Warning Docstring coverage is 2.73% which is insufficient. The required threshold is 80.00%. You can run @coderabbitai generate docstrings to improve docstring coverage.
✅ Passed checks (2 passed)
Check name Status Explanation
Description Check ✅ Passed Check skipped - CodeRabbit’s high-level summary is enabled.
Title check ✅ Passed The title 'feat: Introduce task for craned step' accurately describes the main change: adding task-level abstractions and management to the craned step execution system.
✨ Finishing touches
  • 📝 Generate docstrings
🧪 Generate unit tests (beta)
  • Create PR with unit tests
  • Post copyable unit tests in a comment
  • Commit unit tests in branch dev/task

Thanks for using CodeRabbit! It's free for OSS, and your support helps us grow. If you like it, consider giving us a shout-out.

❤️ Share

Comment @coderabbitai help to get the list of available commands and usage tips.

@L-Xiafeng L-Xiafeng changed the title feat: Introduce task for craned step [WIP] feat: Introduce task for craned step Nov 11, 2025
@L-Xiafeng L-Xiafeng linked an issue Nov 12, 2025 that may be closed by this pull request
@L-Xiafeng
Copy link
Collaborator Author

L-Xiafeng commented Nov 17, 2025

this pull request is based on #683.
TBD:

  • Scheduler need to fill in task res field
  • More per-task level frontend support

@L-Xiafeng L-Xiafeng force-pushed the dev/task branch 2 times, most recently from cfa545d to e49428e Compare November 19, 2025 08:58
@L-Xiafeng L-Xiafeng changed the title [WIP] feat: Introduce task for craned step feat: Introduce task for craned step Nov 19, 2025
@L-Xiafeng L-Xiafeng marked this pull request as ready for review November 19, 2025 08:59
Copy link
Contributor

@coderabbitai coderabbitai bot left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Actionable comments posted: 8

Caution

Some comments are outside the diff and can’t be posted inline due to platform limitations.

⚠️ Outside diff range comments (4)
src/Craned/Common/CgroupManager.cpp (1)

588-641: Fix loss of step/task IDs when scanning cgroup v2 hierarchy.

ParseIdsFromCgroupStr_ expects strings containing the full job_{id}/step_{id}/... layout, but here we pass only path().filename(). For nested directories such as job_42/step_0/system the filename is just "system", so the regex never sees the job_ prefix. As a result every entry under a job collapses to {job_id, null, false, null}. This breaks the new tuple-based tracking: GetIdsFromCgroupV2_ never yields step/task IDs, and GetJobBpfMapCgroupsV2_ ends up asserting because cg_ino_job_id_map is missing entries for step-level cgroups that BPF reports.

Please feed the relative path (or full path) into ParseIdsFromCgroupStr_ so the job/step/task segments survive. For example:

-        auto parsed_ids = ParseIdsFromCgroupStr_(it.path().filename());
+        const auto rel = std::filesystem::relative(it.path(), root_cgroup_path);
+        auto parsed_ids = ParseIdsFromCgroupStr_(rel.generic_string());

Apply the same fix in GetCgInoJobIdMapCgroupV2_ before calling the parser; otherwise the map still misses nested cgroups and GetJobBpfMapCgroupsV2_ will continue to fail.

src/Craned/Supervisor/CforedClient.cpp (1)

354-377: Fix iterator misuse and mutex re‑lock deadlock between CleanStopTaskIOQueueCb_ and TaskEnd

Two intertwined correctness issues here:

  1. Potential use‑after‑end() in CleanStopTaskIOQueueCb_
    If a task_id has already been erased from m_fwd_meta_map (e.g., a duplicate stop signal), it == m_fwd_meta_map.end() and:

    • close(it->second.stdout_read); dereferences an invalid iterator.
    • TaskOutputFinishNoLock_(task_id) will create a default entry via operator[], re‑introducing a bogus meta entry.
  2. Self‑deadlock on m_mtx_
    CleanStopTaskIOQueueCb_ holds absl::MutexLock lock(&m_mtx_); and then calls TaskEnd(task_id), which locks m_mtx_ again. absl::Mutex is not reentrant, so this call path will deadlock.

A minimal fix is to limit the lock scope in CleanStopTaskIOQueueCb_ and only call TaskEnd when the map actually contains the task, after the lock is released. For example:

 void CforedClient::CleanStopTaskIOQueueCb_() {
   task_id_t task_id;
   while (m_stop_task_io_queue_.try_dequeue(task_id)) {
-    absl::MutexLock lock(&m_mtx_);
-    auto it = m_fwd_meta_map.find(task_id);
-    if (it != m_fwd_meta_map.end()) {
-      auto& output_handle = it->second.out_handle;
-      if (!it->second.pty && output_handle.pipe) output_handle.pipe->close();
-      if (it->second.pty && output_handle.tty) output_handle.tty->close();
-      output_handle.pipe.reset();
-      output_handle.tty.reset();
-    }
-
-    close(it->second.stdout_read);
-
-    CRANE_DEBUG("[Task #{}] Finished its output.", task_id);
-
-    bool ok_to_free = this->TaskOutputFinishNoLock_(task_id);
-    if (ok_to_free) {
-      CRANE_DEBUG("[Task #{}] It's ok to unregister.", task_id);
-      this->TaskEnd(task_id);
-    }
+    bool ok_to_free = false;
+    {
+      absl::MutexLock lock(&m_mtx_);
+      auto it = m_fwd_meta_map.find(task_id);
+      if (it == m_fwd_meta_map.end()) {
+        CRANE_DEBUG("[Task #{}] StopTaskIO for unknown task, skipping.", task_id);
+      } else {
+        auto& output_handle = it->second.out_handle;
+        if (!it->second.pty && output_handle.pipe) output_handle.pipe->close();
+        if (it->second.pty && output_handle.tty) output_handle.tty->close();
+        output_handle.pipe.reset();
+        output_handle.tty.reset();
+
+        close(it->second.stdout_read);
+
+        CRANE_DEBUG("[Task #{}] Finished its output.", task_id);
+        ok_to_free = this->TaskOutputFinishNoLock_(task_id);
+      }
+    }
+
+    if (ok_to_free) {
+      CRANE_DEBUG("[Task #{}] It's ok to unregister.", task_id);
+      this->TaskEnd(task_id);
+    }
   };
 }

This avoids dereferencing an invalid iterator, prevents TaskOutputFinishNoLock_ from recreating entries for already‑cleaned tasks, and ensures TaskEnd never tries to re‑lock m_mtx_ from a thread that already holds it. With this adjustment, the new destructor assertion on m_fwd_meta_map.empty() becomes reliable rather than a potential crash point.

Also applies to: 698-716

src/Craned/Core/JobManager.cpp (2)

824-931: Critical: use-after-free risk and requeue semantics in EvCleanTerminateTaskQueueCb_

The refactor generally improves behavior (requeue non-running steps, centralize supervisor-based termination, handle orphaned steps), but one branch has a clear lifetime bug:

auto map_ptr = m_job_map_.GetMapExclusivePtr();
auto job_instance = map_ptr->at(elem.job_id).RawPtr();
...
if (elem.mark_as_orphaned) {
  if (terminate_job) {
    auto uid_map_ptr = m_uid_to_job_ids_map_.GetMapExclusivePtr();
    auto job_opt = FreeJobInfoNoLock_(elem.job_id, map_ptr, uid_map_ptr);
    if (job_opt.has_value()) {
      job_to_clean.emplace_back(std::move(job_opt.value()));
      for (auto& [step_id, step] : job_instance->step_map) {
        ...
      }

FreeJobInfoNoLock_ moves the JobInD out of map_ptr and erases it; job_instance (a RawPtr into the erased map entry) becomes dangling, so iterating job_instance->step_map is use-after-free.

You can instead use the moved-out job_to_clean.back() to access the steps:

    auto job_opt = FreeJobInfoNoLock_(elem.job_id, map_ptr, uid_map_ptr);
    if (job_opt.has_value()) {
-      job_to_clean.emplace_back(std::move(job_opt.value()));
-      for (auto& [step_id, step] : job_instance->step_map) {
+      job_to_clean.emplace_back(std::move(job_opt.value()));
+      auto& orphan_job = job_to_clean.back();
+      for (auto& [step_id, step] : orphan_job.step_map) {
         CRANE_DEBUG("[Step #{}.{}] Removed orphaned step.", elem.job_id,
                     step_id);
         auto* step_ptr = step.get();
         steps_to_clean.push_back(step_ptr);
       }

This way you only touch the JobInD that you now own in job_to_clean.

The earlier IsRunning() requeue behavior and non-daemon orphaned-step cleanup look fine after this fix.


950-999: Daemon shutdown via ShutdownSupervisor is wired correctly, but be mindful of global capture

CleanUpJobAndStepsAsync:

  • Only issues ShutdownSupervisor for daemon steps that actually started (!err_before_supv_start).
  • Marks those steps as Completed before shutdown and waits on a latch so all shutdown RPCs finish before proceeding.
  • Then enqueues the steps and jobs into m_completing_step_retry_map_ / m_completing_job_ for eventual cleanup by EvCheckSupervisorRunning_.

The logic is sound. One minor consideration: the detached lambdas capture g_job_mgr when reporting a failed shutdown via StepStatusChangeAsync. If g_job_mgr can be reset during process shutdown while these tasks are still running, that could become a use-after-free; if that’s a concern, prefer capturing this (JobManager instance) instead of using the global.

🧹 Nitpick comments (9)
src/Utilities/PublicHeader/include/crane/String.h (1)

153-168: JobStepsToString constraints match intended map shape; a couple of optional concept cleanups

The constraints on Map (map-like, with mapped_type being a range of step_id_t) and the implementation using nested transform + join + ranges::to<std::vector> correctly produce "job.step" strings from job_id_t -> range<step_id_t> maps, and using C++23 ranges here fits the rest of the project. Based on learnings.

Two small, optional cleanups you might consider:

  • MapRange already includes std::ranges::range<R>, so the extra requires std::ranges::range<Map> in the function template is redundant and could be dropped.

  • Since you’ve defined StepIdRange, you could reuse it in the function constraint instead of inlining the range_value_t<mapped_type> == step_id_t requirement, e.g.:

    template <MapRange Map>
      requires StepIdRange<typename Map::mapped_type>
    std::string JobStepsToString(const Map &m) { ... }

These are purely stylistic/DRY improvements; behavior is fine as-is.

Also applies to: 169-180

src/Craned/Common/CgroupManager.h (1)

46-62: Cpuset controller additions, parsed-id tuple indices, and SetCpuBind API look consistent

  • Controller / ControllerFile enums and their string arrays stay in sync with *_COUNT, so indexing is safe.
  • CgroupStrParsedIds tuple layout matches the new constexpr indices KParsedJobIdIdx..KParsedTaskIdIdx, and call sites in the .cpp use these indices correctly.
  • The SetCpuBind virtual added to CgroupInterface and overridden in CgroupV1/CgroupV2 lines up with the cpuset controller/file definitions and BPF-related helpers.

If you rely on the exact ordering/meaning of the tuple elements elsewhere, a brief comment near CgroupStrParsedIds (describing the 4-tuple structure) would help future maintainers, but functionally this header change is sound.

Also applies to: 64-95, 131-179, 112-116, 397-401, 431-432, 459-463, 528-531, 552-553, 610-619, 634-640

protos/PublicDefs.proto (1)

298-308: New StepToD.task_res_map field is a compatible extension

Adding map<uint32 /*task id*/, ResourceInNode> task_res_map = 8; is a backward-compatible extension and aligns with the move toward per-task resource tracking. Existing readers will simply see an empty map. Just make sure writers consistently populate either the step-level res or this per-task map according to the new design so downstream code doesn’t have to guess which source of truth to use.

src/Craned/Supervisor/CforedClient.cpp (1)

74-86: Destructor assertion on m_fwd_meta_map.empty() is reasonable once cleanup is fixed

Asserting m_fwd_meta_map.empty() after m_cq_.Shutdown() is a good guard to ensure all forwarding metadata is cleaned up before destruction. After addressing the cleanup/deadlock issues above, this should reliably catch only genuine misuse (e.g., forgetting to drive TaskEnd for some tasks) and not normal shutdown sequences.

src/Craned/Core/CranedForPamServer.cpp (2)

217-227: Step-scoped env lookup via JobManager is reasonable; ensure API semantics match craned-side usage

Switching QuerySshStepEnvVariablesForward to use g_job_mgr->QuerySshStepEnvVariables(request->task_id(), kDaemonStepId) and treating !task_env_map.has_value() as failure aligns with the new step-level env model and mirrors the craned RPC path. Just make sure this call has the same failure semantics (e.g., errors vs. “no env defined”) as in CranedServiceImpl::QuerySshStepEnvVariables, so PAM and craned don’t diverge in how they interpret lookup failures.


249-251: Consider logging chmod failures on the PAM Unix socket

Tightening permissions to 0600 is good, but chmod’s return value is currently ignored. Optionally check and log on error so misconfigured permissions (e.g., missing socket path, filesystem issues) don’t silently degrade PAM behavior.

-  // Pam run as root
-  chmod(listen_conf.UnixSocketForPamListenAddr.c_str(), 0600);
+  // Pam runs as root
+  if (chmod(listen_conf.UnixSocketForPamListenAddr.c_str(), 0600) != 0) {
+    std::error_code ec(errno, std::system_category());
+    CRANE_ERROR("Failed to chmod PAM unix socket {}: {}",
+                listen_conf.UnixSocketForPamListenAddr, ec.message());
+  }
src/Craned/Supervisor/SupervisorServer.cpp (1)

60-66: Ensure CheckStatusAsync does not use response after this RPC returns

CheckStatus now calls g_task_mgr->CheckStatusAsync(response); and immediately returns. If CheckStatusAsync is truly asynchronous (e.g., posts work to a thread pool and writes to response later), this is a use-after-free of a stack-owned pointer that becomes invalid once the RPC completes.

Please confirm that:

  • CheckStatusAsync either fills response synchronously before returning, or
  • it copies the data it needs and does not store or write through the pointer after CheckStatus returns.

If it is asynchronous, consider changing its API to return a value/future, or pass in a value-type that’s filled inside this method before returning, rather than a raw pointer into the gRPC stack.

src/Craned/Core/StepInstance.h (1)

28-89: StepInstance abstraction looks solid; just be explicit about threading assumptions

The StepInstance struct cleanly encapsulates per-step state (IDs, StepToD, supervisor PID/stub, cgroup) and helper operations, matching the “one supervisor per step” design. The inline helpers (IsDaemonStep, IsContainer, StepIdString, etc.) are straightforward and readable.

Given the presence of std::atomic_bool err_before_supv_start but a non-atomic status, please ensure all accesses to status and the other fields are serialized externally (e.g., by JobManager invariants) so there’s no concurrent read/write to the same StepInstance from multiple threads. If you expect true concurrent access, promoting status (and possibly supv_pid) to atomics or guarding with a lock would be safer.

src/Craned/Core/CranedServer.cpp (1)

297-316: Unify Expected-style error handling for env lookup and timelimit changes

Both QuerySshStepEnvVariables and ChangeJobTimeLimit now use Expected-like returns from JobManager:

auto task_env_map =
    g_job_mgr->QuerySshStepEnvVariables(request->task_id(), kDaemonStepId);
if (task_env_map.error()) {
  CRANE_ERROR("Failed to get step env of job #{}", request->task_id());
  return Status::OK;
}
for (const auto& [name, value] : task_env_map.value())
  response->mutable_env_map()->emplace(name, value);
response->set_ok(true);
auto err = g_job_mgr->ChangeStepTimelimit(request->task_id(), kPrimaryStepId,
                                          request->time_limit_seconds());
if (err.error()) {
  CRANE_ERROR("[Step #{}.{}] Failed to change task time limit",
              request->task_id(), kPrimaryStepId);
  return Status::OK;
}
response->set_ok(true);

To make intent clearer and avoid relying on implicit bool conversions of the error type, consider using the Expected object itself as the condition, mirroring patterns elsewhere in the codebase:

-  auto task_env_map =
-      g_job_mgr->QuerySshStepEnvVariables(request->task_id(), kDaemonStepId);
-  if (task_env_map.error()) {
+  auto task_env_map =
+      g_job_mgr->QuerySshStepEnvVariables(request->task_id(), kDaemonStepId);
+  if (!task_env_map) {
@@
-  auto err = g_job_mgr->ChangeStepTimelimit(request->task_id(), kPrimaryStepId,
-                                            request->time_limit_seconds());
-
-  if (err.error()) {
+  auto result = g_job_mgr->ChangeStepTimelimit(
+      request->task_id(), kPrimaryStepId, request->time_limit_seconds());
+
+  if (!result) {

This keeps the code robust if the error type’s truthiness semantics ever change and aligns with other CraneExpected usages.

Also applies to: 320-339

📜 Review details

Configuration used: Path: .coderabbit.yaml

Review profile: CHILL

Plan: Pro

📥 Commits

Reviewing files that changed from the base of the PR and between 95dd70c and e49428e.

📒 Files selected for processing (28)
  • protos/Crane.proto (1 hunks)
  • protos/PublicDefs.proto (2 hunks)
  • protos/Supervisor.proto (2 hunks)
  • src/CraneCtld/CtldPublicDefs.cpp (5 hunks)
  • src/CraneCtld/CtldPublicDefs.h (2 hunks)
  • src/CraneCtld/EmbeddedDbClient.cpp (1 hunks)
  • src/CraneCtld/TaskScheduler.cpp (7 hunks)
  • src/Craned/Common/CgroupManager.cpp (13 hunks)
  • src/Craned/Common/CgroupManager.h (16 hunks)
  • src/Craned/Core/CMakeLists.txt (1 hunks)
  • src/Craned/Core/Craned.cpp (11 hunks)
  • src/Craned/Core/CranedForPamServer.cpp (2 hunks)
  • src/Craned/Core/CranedServer.cpp (4 hunks)
  • src/Craned/Core/CtldClient.cpp (1 hunks)
  • src/Craned/Core/JobManager.cpp (16 hunks)
  • src/Craned/Core/JobManager.h (4 hunks)
  • src/Craned/Core/StepInstance.cpp (1 hunks)
  • src/Craned/Core/StepInstance.h (1 hunks)
  • src/Craned/Core/SupervisorStub.cpp (3 hunks)
  • src/Craned/Core/SupervisorStub.h (1 hunks)
  • src/Craned/Supervisor/CforedClient.cpp (3 hunks)
  • src/Craned/Supervisor/Supervisor.cpp (1 hunks)
  • src/Craned/Supervisor/SupervisorPublicDefs.h (0 hunks)
  • src/Craned/Supervisor/SupervisorServer.cpp (3 hunks)
  • src/Craned/Supervisor/SupervisorServer.h (1 hunks)
  • src/Craned/Supervisor/TaskManager.cpp (23 hunks)
  • src/Craned/Supervisor/TaskManager.h (15 hunks)
  • src/Utilities/PublicHeader/include/crane/String.h (2 hunks)
💤 Files with no reviewable changes (1)
  • src/Craned/Supervisor/SupervisorPublicDefs.h
🧰 Additional context used
🧠 Learnings (38)
📓 Common learnings
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Craned/SupervisorKeeper.h:32-32
Timestamp: 2025-05-26T11:04:30.580Z
Learning: The Supervisor component in the Crane system is designed to manage only one task per instance. The task specification is provided to the Supervisor during startup by Craned (likely through InitSupervisorRequest), so the ExecuteTask() method doesn't need to accept task parameters since the Supervisor already knows which task to execute.
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Supervisor/Supervisor.cpp:68-69
Timestamp: 2025-05-25T04:08:03.273Z
Learning: In the Crane supervisor component (src/Craned/Supervisor/), the supervisor process communicates with Craned through STDOUT using protobuf messages. The supervisor must not send any information to STDOUT before sending the "ready" message to Craned, as this would interfere with the inter-process communication protocol. Therefore, adding logging statements that might write to STDOUT before the ready message is sent could break the communication.
Learnt from: huerni
Repo: PKUHPC/CraneSched PR: 469
File: src/Craned/TaskManager.cpp:1162-1175
Timestamp: 2025-05-08T07:39:29.207Z
Learning: In CraneSched, when a task fails in the LaunchTaskInstanceMt_ method, ActivateTaskStatusChangeAsync_ does not automatically release PMIx resources or cgroup. Explicit cleanup is required by calling g_pmix_server->DeregisterTask() and g_cg_mgr->ReleaseCgroupByTaskIdOnly().
Learnt from: huerni
Repo: PKUHPC/CraneSched PR: 469
File: src/Craned/TaskManager.cpp:1162-1175
Timestamp: 2025-05-08T07:39:29.207Z
Learning: In CraneSched, when a task fails in the LaunchTaskInstanceMt_ method after PMIx registration, ActivateTaskStatusChangeAsync_ does not automatically release PMIx resources or cgroup resources. Explicit cleanup is required by calling g_pmix_server->DeregisterTask() and g_cg_mgr->ReleaseCgroupByTaskIdOnly() before returning from the function.
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Supervisor/SupervisorPublicDefs.h:32-37
Timestamp: 2025-05-26T11:06:28.796Z
Learning: The user L-Xiafeng prefers to defer refactoring duplicate definitions until they become a larger pattern in the codebase, rather than addressing individual instances immediately.
📚 Learning: 2025-05-25T04:08:03.273Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Supervisor/Supervisor.cpp:68-69
Timestamp: 2025-05-25T04:08:03.273Z
Learning: In the Crane supervisor component (src/Craned/Supervisor/), the supervisor process communicates with Craned through STDOUT using protobuf messages. The supervisor must not send any information to STDOUT before sending the "ready" message to Craned, as this would interfere with the inter-process communication protocol. Therefore, adding logging statements that might write to STDOUT before the ready message is sent could break the communication.

Applied to files:

  • protos/Supervisor.proto
  • src/Craned/Core/CMakeLists.txt
  • src/Craned/Core/StepInstance.h
  • src/Craned/Supervisor/SupervisorServer.cpp
  • src/Craned/Core/JobManager.h
  • src/Craned/Core/CtldClient.cpp
  • src/Craned/Core/StepInstance.cpp
  • src/Craned/Supervisor/Supervisor.cpp
  • src/Craned/Core/JobManager.cpp
  • src/Craned/Core/SupervisorStub.h
  • src/Craned/Core/CranedServer.cpp
  • src/Craned/Supervisor/TaskManager.cpp
  • src/Craned/Core/SupervisorStub.cpp
  • src/Craned/Supervisor/TaskManager.h
  • src/Craned/Core/Craned.cpp
📚 Learning: 2025-05-09T01:54:21.256Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 496
File: src/CraneCtld/CranedKeeper.cpp:51-53
Timestamp: 2025-05-09T01:54:21.256Z
Learning: The ConfigureCraned function in src/CraneCtld/CranedKeeper.cpp is called from a thread pool, so there's no need to worry about it blocking the gRPC completion queue thread.

Applied to files:

  • src/Craned/Core/CMakeLists.txt
  • src/Craned/Core/CranedForPamServer.cpp
  • src/CraneCtld/EmbeddedDbClient.cpp
  • src/Craned/Supervisor/SupervisorServer.cpp
  • src/CraneCtld/TaskScheduler.cpp
  • src/Craned/Core/JobManager.h
  • src/Craned/Core/CtldClient.cpp
  • src/Craned/Core/StepInstance.cpp
  • src/CraneCtld/CtldPublicDefs.cpp
  • src/Craned/Supervisor/Supervisor.cpp
  • src/Craned/Supervisor/SupervisorServer.h
  • src/Craned/Core/JobManager.cpp
  • src/Craned/Core/SupervisorStub.h
  • src/CraneCtld/CtldPublicDefs.h
  • src/Craned/Core/CranedServer.cpp
  • src/Craned/Supervisor/TaskManager.cpp
  • src/Craned/Core/SupervisorStub.cpp
  • src/Craned/Supervisor/TaskManager.h
  • src/Craned/Core/Craned.cpp
📚 Learning: 2025-06-30T08:43:44.470Z
Learnt from: huerni
Repo: PKUHPC/CraneSched PR: 537
File: src/Craned/TaskManager.cpp:145-147
Timestamp: 2025-06-30T08:43:44.470Z
Learning: In the CraneSched codebase, src/Craned/Craned.cpp guarantees that g_config.CranedRes.contains(g_config.CranedIdOfThisNode) through explicit validation during startup. The code checks if g_config.CranedRes.contains(g_config.Hostname) and exits the process if not found, then sets g_config.CranedIdOfThisNode = g_config.Hostname. TaskManager constructor is called after this validation, so g_config.CranedRes[g_config.CranedIdOfThisNode] is guaranteed to be valid.

Applied to files:

  • src/Craned/Core/CMakeLists.txt
  • src/Craned/Core/JobManager.h
  • src/Craned/Core/CtldClient.cpp
  • src/CraneCtld/CtldPublicDefs.cpp
  • src/Craned/Supervisor/Supervisor.cpp
  • src/Craned/Core/JobManager.cpp
  • src/Craned/Core/SupervisorStub.h
  • src/CraneCtld/CtldPublicDefs.h
  • src/Craned/Core/CranedServer.cpp
  • src/Craned/Supervisor/TaskManager.cpp
  • src/Craned/Supervisor/TaskManager.h
  • src/Craned/Core/Craned.cpp
📚 Learning: 2025-07-01T08:00:05.383Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Supervisor/TaskManager.cpp:1-1809
Timestamp: 2025-07-01T08:00:05.383Z
Learning: In TaskManager.cpp, the security model relies on administrator-configured command templates in ParseOCICmdPattern_ and system-provided usernames in ParseFilePathPattern_, with file permissions using the user's own uid/gid for privilege isolation. The user L-Xiafeng considers this security boundary sufficient and chooses not to fix job name-related path issues at this location.

Applied to files:

  • src/Craned/Core/CranedForPamServer.cpp
  • src/Craned/Common/CgroupManager.h
  • src/Craned/Core/CranedServer.cpp
📚 Learning: 2025-05-02T07:06:36.103Z
Learnt from: RileyWen
Repo: PKUHPC/CraneSched PR: 475
File: src/CraneCtld/CtldGrpcServer.cpp:113-118
Timestamp: 2025-05-02T07:06:36.103Z
Learning: In CraneSched, gRPC methods should generally return Status::OK even when handling error conditions, as non-OK statuses cause the connection to terminate. Error information should be communicated within the RPC response payload instead.

Applied to files:

  • src/Craned/Core/CranedForPamServer.cpp
  • src/Craned/Supervisor/SupervisorServer.cpp
  • src/Craned/Core/JobManager.h
  • src/Craned/Core/StepInstance.cpp
  • src/CraneCtld/CtldPublicDefs.cpp
  • src/Craned/Supervisor/Supervisor.cpp
  • src/Craned/Supervisor/SupervisorServer.h
  • src/Craned/Core/CranedServer.cpp
  • src/Craned/Supervisor/TaskManager.cpp
  • src/Craned/Core/SupervisorStub.cpp
📚 Learning: 2025-05-26T11:04:30.580Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Craned/SupervisorKeeper.h:32-32
Timestamp: 2025-05-26T11:04:30.580Z
Learning: The Supervisor component in the Crane system is designed to manage only one task per instance. The task specification is provided to the Supervisor during startup by Craned (likely through InitSupervisorRequest), so the ExecuteTask() method doesn't need to accept task parameters since the Supervisor already knows which task to execute.

Applied to files:

  • src/Craned/Core/StepInstance.h
  • src/Craned/Supervisor/SupervisorServer.cpp
  • src/Craned/Core/JobManager.h
  • src/Craned/Core/CtldClient.cpp
  • src/Craned/Core/StepInstance.cpp
  • src/Craned/Supervisor/Supervisor.cpp
  • src/Craned/Supervisor/SupervisorServer.h
  • src/Craned/Core/JobManager.cpp
  • src/Craned/Core/SupervisorStub.h
  • src/Craned/Core/CranedServer.cpp
  • src/Craned/Supervisor/TaskManager.cpp
  • src/Craned/Core/SupervisorStub.cpp
  • src/Craned/Supervisor/TaskManager.h
  • src/Craned/Core/Craned.cpp
📚 Learning: 2025-05-08T07:39:29.207Z
Learnt from: huerni
Repo: PKUHPC/CraneSched PR: 469
File: src/Craned/TaskManager.cpp:1162-1175
Timestamp: 2025-05-08T07:39:29.207Z
Learning: In CraneSched, when a task fails in the LaunchTaskInstanceMt_ method after PMIx registration, ActivateTaskStatusChangeAsync_ does not automatically release PMIx resources or cgroup resources. Explicit cleanup is required by calling g_pmix_server->DeregisterTask() and g_cg_mgr->ReleaseCgroupByTaskIdOnly() before returning from the function.

Applied to files:

  • src/Craned/Core/StepInstance.h
  • src/Craned/Supervisor/CforedClient.cpp
  • src/CraneCtld/TaskScheduler.cpp
  • src/Craned/Core/JobManager.h
  • src/Craned/Core/StepInstance.cpp
  • src/CraneCtld/CtldPublicDefs.cpp
  • src/Craned/Supervisor/Supervisor.cpp
  • src/Craned/Core/JobManager.cpp
  • src/CraneCtld/CtldPublicDefs.h
  • src/Craned/Supervisor/TaskManager.cpp
  • src/Craned/Supervisor/TaskManager.h
  • src/Craned/Core/Craned.cpp
📚 Learning: 2025-05-08T07:39:29.207Z
Learnt from: huerni
Repo: PKUHPC/CraneSched PR: 469
File: src/Craned/TaskManager.cpp:1162-1175
Timestamp: 2025-05-08T07:39:29.207Z
Learning: In CraneSched, when a task fails in the LaunchTaskInstanceMt_ method, ActivateTaskStatusChangeAsync_ does not automatically release PMIx resources or cgroup. Explicit cleanup is required by calling g_pmix_server->DeregisterTask() and g_cg_mgr->ReleaseCgroupByTaskIdOnly().

Applied to files:

  • src/Craned/Core/StepInstance.h
  • src/Craned/Supervisor/CforedClient.cpp
  • src/CraneCtld/TaskScheduler.cpp
  • src/Craned/Core/JobManager.h
  • src/Craned/Core/StepInstance.cpp
  • src/CraneCtld/CtldPublicDefs.cpp
  • src/Craned/Supervisor/Supervisor.cpp
  • src/Craned/Core/JobManager.cpp
  • src/Craned/Supervisor/TaskManager.cpp
  • src/Craned/Supervisor/TaskManager.h
  • src/Craned/Core/Craned.cpp
📚 Learning: 2025-08-14T02:56:35.503Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 587
File: src/Craned/Supervisor/CforedClient.cpp:449-454
Timestamp: 2025-08-14T02:56:35.503Z
Learning: In CforedClient::AsyncSendRecvThread_(), the guard `if (state <= State::Registering) { continue; }` in the TIMEOUT branch only prevents premature cleanup when stopping before registration completes, but it doesn't block normal gRPC event processing. The completion queue will still deliver Prepare/Write/Read events that advance the state machine normally.

Applied to files:

  • src/Craned/Supervisor/CforedClient.cpp
📚 Learning: 2025-05-02T07:12:46.896Z
Learnt from: RileyWen
Repo: PKUHPC/CraneSched PR: 475
File: src/Craned/CranedServer.h:0-0
Timestamp: 2025-05-02T07:12:46.896Z
Learning: The initialization of `m_supervisor_recovered_` to `true` in the CranedServer class is intentional despite the comment saying "When supervisor ready, init with false". This is temporary until the supervisor functionality is fully implemented.

Applied to files:

  • src/Craned/Supervisor/SupervisorServer.cpp
  • src/Craned/Core/CtldClient.cpp
  • src/Craned/Supervisor/Supervisor.cpp
  • src/Craned/Core/SupervisorStub.h
  • src/Craned/Supervisor/TaskManager.cpp
  • src/Craned/Core/SupervisorStub.cpp
  • src/Craned/Core/Craned.cpp
📚 Learning: 2025-05-09T02:15:30.422Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 496
File: src/CraneCtld/TaskScheduler.cpp:1713-1721
Timestamp: 2025-05-09T02:15:30.422Z
Learning: TaskScheduler中的资源回收设计:当Craned节点离线时,通过TaskStatusChangeAsync异步处理任务状态变更和资源回收。尽管可能存在短暂的资源更新延迟,但这是设计上可接受的权衡。

Applied to files:

  • src/CraneCtld/TaskScheduler.cpp
📚 Learning: 2025-09-22T07:13:20.635Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 625
File: src/CraneCtld/TaskScheduler.cpp:2811-2874
Timestamp: 2025-09-22T07:13:20.635Z
Learning: In the CraneSched codebase, there are latch mechanisms used to ensure variable lifetime safety when capturing references in detached thread pool tasks, allowing reference captures that might otherwise be unsafe.

Applied to files:

  • src/CraneCtld/TaskScheduler.cpp
📚 Learning: 2025-05-26T11:00:54.563Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Craned/SupervisorKeeper.cpp:174-178
Timestamp: 2025-05-26T11:00:54.563Z
Learning: The CraneSched project uses C++23 standard, allowing the use of modern C++ features like std::ranges::to and other C++23 language features and library components.

Applied to files:

  • src/CraneCtld/TaskScheduler.cpp
  • src/Craned/Core/CtldClient.cpp
  • src/Craned/Supervisor/TaskManager.cpp
  • src/Utilities/PublicHeader/include/crane/String.h
📚 Learning: 2025-04-02T10:11:33.562Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 461
File: src/Craned/CgroupManager.cpp:685-685
Timestamp: 2025-04-02T10:11:33.562Z
Learning: In the CgroupManager's GetJobBpfMapCgroupsV2 method, the developer has confirmed that cg_ino_job_id_map will always contain the key->cgroup_id element, making the CRANE_ASSERT check appropriate rather than requiring additional error handling.

Applied to files:

  • src/Craned/Core/JobManager.h
  • src/Craned/Core/JobManager.cpp
  • src/Craned/Common/CgroupManager.h
  • src/Craned/Core/CranedServer.cpp
  • src/Craned/Common/CgroupManager.cpp
  • src/Craned/Core/Craned.cpp
📚 Learning: 2025-05-09T01:56:53.142Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 496
File: src/CraneCtld/CtldPublicDefs.h:756-763
Timestamp: 2025-05-09T01:56:53.142Z
Learning: In the CraneSched codebase, the `execution_node` field in JobToD is intentionally set to the first element of `executing_craned_ids` vector without guards, as it represents the main execution node for a job. This is by design and assumes `executing_craned_ids` is never empty when `GetJobOfNode` is called.

Applied to files:

  • src/Craned/Core/JobManager.h
  • src/CraneCtld/CtldPublicDefs.cpp
  • src/CraneCtld/CtldPublicDefs.h
📚 Learning: 2025-05-26T11:04:56.055Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Craned/SupervisorKeeper.h:72-72
Timestamp: 2025-05-26T11:04:56.055Z
Learning: The CraneSched project prefers using global variables (like `g_supervisor_keeper`) over dependency injection patterns. The team does not follow dependency injection approaches for managing singleton instances.

Applied to files:

  • src/Craned/Core/CtldClient.cpp
  • src/Craned/Core/SupervisorStub.h
  • src/Craned/Supervisor/TaskManager.cpp
  • src/Craned/Core/SupervisorStub.cpp
  • src/Craned/Core/Craned.cpp
📚 Learning: 2025-06-07T10:47:59.071Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 520
File: src/CraneCtld/CranedKeeper.cpp:416-417
Timestamp: 2025-06-07T10:47:59.071Z
Learning: In src/CraneCtld/CranedKeeper.h, the m_shutting_down_ member in CranedStub class is declared as std::atomic_bool, making it thread-safe for concurrent access without additional synchronization.

Applied to files:

  • src/Craned/Core/CtldClient.cpp
  • src/Craned/Core/SupervisorStub.h
  • src/CraneCtld/CtldPublicDefs.h
  • src/Craned/Core/CranedServer.cpp
  • src/Craned/Supervisor/TaskManager.cpp
  • src/Craned/Core/SupervisorStub.cpp
  • src/Craned/Supervisor/TaskManager.h
📚 Learning: 2025-05-23T02:32:43.952Z
Learnt from: 1daidai1
Repo: PKUHPC/CraneSched PR: 458
File: src/CraneCtld/CtldPublicDefs.h:0-0
Timestamp: 2025-05-23T02:32:43.952Z
Learning: In the CraneSched project, allocated_res_view is handled/updated separately before calling SetAllocatedRes, so it does not need to be updated again within the method itself.

Applied to files:

  • src/CraneCtld/CtldPublicDefs.cpp
📚 Learning: 2025-05-23T02:32:43.952Z
Learnt from: 1daidai1
Repo: PKUHPC/CraneSched PR: 458
File: src/CraneCtld/CtldPublicDefs.h:0-0
Timestamp: 2025-05-23T02:32:43.952Z
Learning: In the CraneSched project, allocated_res_view is updated before calling SetAllocatedRes, so it doesn't need to be updated again within the method itself.

Applied to files:

  • src/CraneCtld/CtldPublicDefs.cpp
📚 Learning: 2025-05-25T04:11:50.268Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Supervisor/TaskManager.cpp:220-220
Timestamp: 2025-05-25T04:11:50.268Z
Learning: In TaskManager.cpp, when step->IsCrun() is checked before dynamic_cast to CrunMetaInExecution*, the cast is guaranteed to succeed due to the program logic ensuring the correct meta type is used for Crun tasks. Null checks for these dynamic_cast operations are unnecessary in this context.

Applied to files:

  • src/Craned/Supervisor/Supervisor.cpp
  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-09-21T11:26:40.935Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 625
File: src/Craned/Core/SupervisorKeeper.cpp:145-147
Timestamp: 2025-09-21T11:26:40.935Z
Learning: In C++17 and later, Class Template Argument Deduction (CTAD) allows `std::shared_ptr stub = std::make_shared<SupervisorStub>();` syntax to be valid, as the compiler can deduce the template parameter from make_shared. Projects using C++17+ don't need explicit template parameters in this context.

Applied to files:

  • src/Craned/Core/SupervisorStub.h
📚 Learning: 2025-05-02T07:05:26.012Z
Learnt from: RileyWen
Repo: PKUHPC/CraneSched PR: 475
File: src/CraneCtld/CranedKeeper.cpp:601-602
Timestamp: 2025-05-02T07:05:26.012Z
Learning: In the CraneCtld codebase, the variables m_disconnected_ and m_registered_ in CranedStub class are already defined as std::atomic_bool, making them thread-safe for concurrent access without additional synchronization.

Applied to files:

  • src/CraneCtld/CtldPublicDefs.h
  • src/Craned/Supervisor/TaskManager.h
📚 Learning: 2025-03-31T09:29:40.388Z
Learnt from: RileyWen
Repo: PKUHPC/CraneSched PR: 456
File: src/CraneCtld/Tls/VaultClientWrapper.cpp:116-137
Timestamp: 2025-03-31T09:29:40.388Z
Learning: In CraneSched, `phmap::parallel_flat_hash_set` from the Parallel Hashmap library (`parallel_hashmap/phmap.h`) is used for thread-safe containers. This container implements internal sharding with separate locks for different parts of the hash table, making it inherently thread-safe for concurrent operations without requiring external synchronization.

Applied to files:

  • src/CraneCtld/CtldPublicDefs.h
  • src/Craned/Core/CranedServer.cpp
📚 Learning: 2025-05-02T07:05:49.032Z
Learnt from: RileyWen
Repo: PKUHPC/CraneSched PR: 475
File: src/CraneCtld/CranedKeeper.cpp:503-506
Timestamp: 2025-05-02T07:05:49.032Z
Learning: In CraneCtld/CranedKeeper.cpp, using m_unavail_craned_set_.at() is intentional as the key is guaranteed to exist by design, and crashing on missing key is preferred to silently handling it (fail-fast approach).

Applied to files:

  • src/CraneCtld/CtldPublicDefs.h
📚 Learning: 2025-03-31T09:29:40.388Z
Learnt from: RileyWen
Repo: PKUHPC/CraneSched PR: 456
File: src/CraneCtld/Tls/VaultClientWrapper.cpp:116-137
Timestamp: 2025-03-31T09:29:40.388Z
Learning: In the CraneSched project, the `m_allowed_certs_` in `VaultClientWrapper` is implemented as a `phmap::parallel_flat_hash_set<std::string>`, which is a thread-safe container designed for concurrent access, making it safe to use in multithreaded contexts without additional synchronization.

Applied to files:

  • src/CraneCtld/CtldPublicDefs.h
  • src/Craned/Core/CranedServer.cpp
📚 Learning: 2025-08-12T08:58:39.772Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 577
File: src/Craned/Supervisor/TaskManager.cpp:124-130
Timestamp: 2025-08-12T08:58:39.772Z
Learning: In the CraneSched project using C++23, Class Template Argument Deduction (CTAD) allows std::unique_ptr declarations without explicit template parameters when the type can be deduced from the initializer, such as `std::unique_ptr task = std::move(m_task_map_.at(task_id))` where the template parameter is deduced from the move operation.

Applied to files:

  • src/CraneCtld/CtldPublicDefs.h
📚 Learning: 2025-03-31T09:29:40.388Z
Learnt from: RileyWen
Repo: PKUHPC/CraneSched PR: 456
File: src/CraneCtld/Tls/VaultClientWrapper.cpp:116-137
Timestamp: 2025-03-31T09:29:40.388Z
Learning: The `phmap::parallel_flat_hash_set` used in CraneSched's `VaultClientWrapper` is from the Parallel Hashmap library, which provides thread-safe containers for concurrent access. It uses internal partitioning to reduce contention, making it safe for use in multithreaded environments without external synchronization.

Applied to files:

  • src/Craned/Core/CranedServer.cpp
📚 Learning: 2025-04-02T09:30:13.014Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 461
File: src/Craned/JobManager.cpp:141-149
Timestamp: 2025-04-02T09:30:13.014Z
Learning: In JobManager, if a uid exists in m_uid_to_job_ids_map_, its corresponding task_ids set is guaranteed to be non-empty due to the invariant maintained in AllocJobs and FreeJobs methods.

Applied to files:

  • src/Craned/Core/CranedServer.cpp
  • src/Craned/Common/CgroupManager.cpp
📚 Learning: 2025-04-02T09:52:59.318Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 461
File: src/Craned/CgroupManager.cpp:697-701
Timestamp: 2025-04-02T09:52:59.318Z
Learning: When using bpf_map__get_next_key function, memory must be properly allocated (e.g., with std::make_unique<BpfKey>()) before passing the pointer to the function, as it writes the key to the provided memory address.

Applied to files:

  • src/Craned/Common/CgroupManager.cpp
  • src/Craned/Core/Craned.cpp
📚 Learning: 2025-04-18T02:26:16.113Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 351
File: src/CraneCtld/CranedMetaContainer.cpp:248-264
Timestamp: 2025-04-18T02:26:16.113Z
Learning: The resource class in CraneSched includes assertions in its operator overloads (particularly in operator-=) that verify resources being subtracted are less than or equal to available resources, ensuring no negative values can occur during resource allocation or deallocation operations.

Applied to files:

  • src/Craned/Common/CgroupManager.cpp
📚 Learning: 2025-05-09T01:54:39.465Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 496
File: src/Craned/TaskManager.cpp:1509-1517
Timestamp: 2025-05-09T01:54:39.465Z
Learning: The CraneSched project uses C++23, which supports Class Template Argument Deduction (CTAD) for standard containers like std::set and includes ranges support, making std::ranges::views::keys valid without additional headers.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
  • src/Utilities/PublicHeader/include/crane/String.h
📚 Learning: 2025-05-08T07:38:42.362Z
Learnt from: huerni
Repo: PKUHPC/CraneSched PR: 469
File: src/Craned/TaskManager.cpp:0-0
Timestamp: 2025-05-08T07:38:42.362Z
Learning: In CraneSched's PMIx integration, the `g_pmix_server->SetupFork()` function must be called in the child process after fork() and before exec() to properly set up the PMIx environment variables.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-05-25T04:11:27.740Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Supervisor/TaskManager.cpp:396-396
Timestamp: 2025-05-25T04:11:27.740Z
Learning: In TaskManager.cpp, GetCrunMeta() calls don't need null checks because they're only called in contexts where the task is guaranteed to be a CRUN task (e.g., SetupChildProcessCrunX11_ is only called when step->IsCrun() && x11() conditions are met), ensuring the metadata will always be valid.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-05-09T02:16:56.723Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 496
File: src/Craned/TaskManager.cpp:1498-1506
Timestamp: 2025-05-09T02:16:56.723Z
Learning: The `QueryRunningTasksAsync()` method in TaskManager.cpp is designed to never be called from inside the event loop thread, so there's no risk of deadlock with the synchronous `res.get()` call.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-05-08T09:35:39.809Z
Learnt from: huerni
Repo: PKUHPC/CraneSched PR: 469
File: src/Craned/Pmix/PmixCollRing.cpp:0-0
Timestamp: 2025-05-08T09:35:39.809Z
Learning: In the PMIx implementation for CraneSched, objects referenced in asynchronous gRPC callbacks (like `coll_ring_ctx`) remain valid as long as the parent object (`this`) is not destroyed. The `Coll` class uses shared_ptr management to ensure its lifetime extends through all pending callbacks.

Applied to files:

  • src/Craned/Supervisor/TaskManager.h
📚 Learning: 2025-04-27T11:52:31.017Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 475
File: src/CraneCtld/CranedKeeper.cpp:40-62
Timestamp: 2025-04-27T11:52:31.017Z
Learning: In the CraneSched system, retry of configuration RPC is architecturally driven by the Craned's notification system rather than explicit retry code within the ConfigureCraned method. When Configure RPC fails, Craned returns to a notification state and sends new Notify messages which trigger new configuration attempts.

Applied to files:

  • src/Craned/Core/Craned.cpp
📚 Learning: 2025-06-23T07:53:30.513Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 520
File: src/Craned/CranedServer.cpp:50-51
Timestamp: 2025-06-23T07:53:30.513Z
Learning: In the CraneSched codebase, `g_ctld_client` is a guaranteed global variable that is always initialized before any gRPC service methods are called, so null pointer checks are not necessary when calling methods on it.

Applied to files:

  • src/Craned/Core/Craned.cpp
🧬 Code graph analysis (15)
src/Craned/Core/StepInstance.h (4)
src/Craned/Core/SupervisorStub.h (2)
  • Craned (26-62)
  • SupervisorStub (28-61)
src/Craned/Core/SupervisorStub.cpp (2)
  • SupervisorStub (25-29)
  • SupervisorStub (31-33)
src/Craned/Common/CgroupManager.h (1)
  • CgroupInterface (389-421)
src/Craned/Core/StepInstance.cpp (14)
  • StepInstance (25-30)
  • StepInstance (32-40)
  • CleanUp (42-78)
  • CleanUp (42-42)
  • CreateCg (80-87)
  • CreateCg (80-80)
  • SpawnSupervisor (89-332)
  • SpawnSupervisor (89-89)
  • GotNewStatus (334-406)
  • GotNewStatus (334-334)
  • ExecuteStepAsync (408-419)
  • ExecuteStepAsync (408-408)
  • KillPid_ (421-436)
  • KillPid_ (421-421)
src/Craned/Supervisor/SupervisorServer.cpp (2)
src/Craned/Core/CranedForPamServer.cpp (2)
  • MigrateSshProcToCgroup (181-205)
  • MigrateSshProcToCgroup (181-184)
src/Craned/Core/SupervisorStub.cpp (2)
  • ShutdownSupervisor (188-198)
  • ShutdownSupervisor (188-188)
src/Craned/Core/JobManager.h (3)
src/Craned/Common/CgroupManager.h (1)
  • CgroupInterface (389-421)
src/Craned/Core/JobManager.cpp (9)
  • ChangeStepTimelimit (467-492)
  • ChangeStepTimelimit (467-469)
  • job_id_t (789-804)
  • QueryTaskInfoOfUid (806-822)
  • QueryTaskInfoOfUid (806-806)
  • MigrateProcToCgroupOfJob (757-787)
  • MigrateProcToCgroupOfJob (757-757)
  • QuerySshStepEnvVariables (494-510)
  • QuerySshStepEnvVariables (494-495)
src/Craned/Core/CranedServer.cpp (2)
  • QuerySshStepEnvVariables (297-318)
  • QuerySshStepEnvVariables (297-300)
src/Craned/Core/StepInstance.cpp (3)
src/Craned/Common/CgroupManager.cpp (4)
  • AllocateAndGetCgroup (494-552)
  • AllocateAndGetCgroup (495-497)
  • CgroupStrByStepId (283-288)
  • CgroupStrByStepId (283-284)
src/Craned/Core/CranedPublicDefs.h (1)
  • g_config (156-156)
src/Craned/Supervisor/TaskManager.cpp (2)
  • GotNewStatus (114-189)
  • GotNewStatus (114-114)
src/Craned/Supervisor/SupervisorServer.h (2)
src/Craned/Core/CranedForPamServer.cpp (2)
  • MigrateSshProcToCgroup (181-205)
  • MigrateSshProcToCgroup (181-184)
src/Craned/Supervisor/SupervisorServer.cpp (2)
  • MigrateSshProcToCgroup (95-103)
  • MigrateSshProcToCgroup (95-98)
src/Craned/Core/JobManager.cpp (1)
src/Craned/Core/CranedServer.cpp (2)
  • QuerySshStepEnvVariables (297-318)
  • QuerySshStepEnvVariables (297-300)
src/Craned/Core/SupervisorStub.h (2)
src/Craned/Core/SupervisorStub.cpp (20)
  • SupervisorStub (25-29)
  • SupervisorStub (31-33)
  • InitAndGetRecoveredMap (35-92)
  • InitAndGetRecoveredMap (37-37)
  • ExecuteStep (94-106)
  • ExecuteStep (94-94)
  • QueryStepEnv (108-121)
  • QueryStepEnv (108-108)
  • CheckStatus (123-137)
  • CheckStatus (124-124)
  • TerminateTask (139-158)
  • TerminateTask (139-140)
  • ChangeTaskTimeLimit (160-172)
  • ChangeTaskTimeLimit (160-160)
  • MigrateSshProcToCg (174-186)
  • MigrateSshProcToCg (174-174)
  • ShutdownSupervisor (188-198)
  • ShutdownSupervisor (188-188)
  • InitChannelAndStub_ (200-204)
  • InitChannelAndStub_ (200-200)
src/Craned/Supervisor/SupervisorServer.cpp (8)
  • CheckStatus (60-66)
  • CheckStatus (60-63)
  • TerminateTask (83-93)
  • TerminateTask (83-86)
  • ChangeTaskTimeLimit (68-81)
  • ChangeTaskTimeLimit (68-71)
  • ShutdownSupervisor (105-112)
  • ShutdownSupervisor (105-108)
src/CraneCtld/CtldPublicDefs.h (1)
src/Utilities/PublicHeader/PublicHeader.cpp (1)
  • ResourceInNode (484-487)
src/Craned/Common/CgroupManager.h (1)
src/Craned/Common/CgroupManager.cpp (15)
  • SetCpuBind (1013-1032)
  • SetCpuBind (1013-1013)
  • SetCpuBind (1317-1335)
  • SetCpuBind (1317-1317)
  • RecoverFromResInNode (1442-1492)
  • RecoverFromResInNode (1442-1443)
  • CgroupStrByParsedIds (296-308)
  • CgroupStrByParsedIds (296-296)
  • CgroupStrParsedIds (636-676)
  • GetIdsFromCgroupV1_ (554-578)
  • GetIdsFromCgroupV1_ (554-555)
  • GetIdsFromCgroupV2_ (580-600)
  • GetIdsFromCgroupV2_ (580-581)
  • GetCgInoJobIdMapCgroupV2_ (602-632)
  • GetCgInoJobIdMapCgroupV2_ (603-604)
src/Craned/Common/CgroupManager.cpp (1)
src/Craned/Common/CgroupManager.h (1)
  • handle (425-426)
src/Craned/Supervisor/TaskManager.cpp (3)
src/Craned/Core/StepInstance.cpp (2)
  • GotNewStatus (334-406)
  • GotNewStatus (334-334)
src/Craned/Common/CgroupManager.cpp (6)
  • AllocateAndGetCgroup (494-552)
  • AllocateAndGetCgroup (495-497)
  • CgroupStrByTaskId (290-294)
  • CgroupStrByTaskId (290-291)
  • CgroupStrByStepId (283-288)
  • CgroupStrByStepId (283-284)
src/Utilities/PublicHeader/OS.cpp (2)
  • DeleteFile (64-71)
  • DeleteFile (64-64)
src/Craned/Core/SupervisorStub.cpp (1)
src/Craned/Core/SupervisorStub.h (1)
  • SupervisorStub (28-61)
src/Craned/Supervisor/TaskManager.h (2)
src/Craned/Common/CgroupManager.h (1)
  • CgroupInterface (389-421)
src/Craned/Supervisor/TaskManager.cpp (20)
  • GotNewStatus (114-189)
  • GotNewStatus (114-114)
  • SupervisorFinishInit (1689-1691)
  • SupervisorFinishInit (1689-1689)
  • Wait (1693-1695)
  • Wait (1693-1693)
  • ShutdownSupervisorAsync (1697-1704)
  • ShutdownSupervisorAsync (1697-1699)
  • CheckStatusAsync (1825-1836)
  • CheckStatusAsync (1825-1826)
  • MigrateSshProcToCgroupAsync (1838-1848)
  • MigrateSshProcToCgroupAsync (1838-1838)
  • EvSupervisorFinishInitCb_ (1850-1858)
  • EvSupervisorFinishInitCb_ (1850-1850)
  • EvShutdownSupervisorCb_ (1860-1905)
  • EvShutdownSupervisorCb_ (1860-1860)
  • EvGrpcCheckStatusCb_ (2296-2301)
  • EvGrpcCheckStatusCb_ (2296-2296)
  • EvGrpcMigrateSshProcToCgroupCb_ (2303-2331)
  • EvGrpcMigrateSshProcToCgroupCb_ (2303-2303)
src/Craned/Core/Craned.cpp (4)
src/Craned/Core/StepInstance.cpp (2)
  • StepInstance (25-30)
  • StepInstance (32-40)
src/Craned/Core/SupervisorStub.h (2)
  • Craned (26-62)
  • SupervisorStub (28-61)
src/Craned/Common/CgroupManager.cpp (12)
  • GetIdsFromCgroupV1_ (554-578)
  • GetIdsFromCgroupV1_ (554-555)
  • GetIdsFromCgroupV2_ (580-600)
  • GetIdsFromCgroupV2_ (580-581)
  • AllocateAndGetCgroup (494-552)
  • AllocateAndGetCgroup (495-497)
  • CgroupStrByStepId (283-288)
  • CgroupStrByStepId (283-284)
  • CgroupStrByJobId (279-281)
  • CgroupStrByJobId (279-279)
  • CgroupStrByParsedIds (296-308)
  • CgroupStrByParsedIds (296-296)
src/Craned/Core/SupervisorStub.cpp (4)
  • SupervisorStub (25-29)
  • SupervisorStub (31-33)
  • InitAndGetRecoveredMap (35-92)
  • InitAndGetRecoveredMap (37-37)
src/Utilities/PublicHeader/include/crane/String.h (2)
src/CraneCtld/CtldPublicDefs.h (1)
  • std (878-888)
src/Utilities/PublicHeader/String.cpp (2)
  • StepToDIdString (519-521)
  • StepToDIdString (519-519)

Comment on lines +494 to +510
CraneExpected<EnvMap> JobManager::QuerySshStepEnvVariables(job_id_t job_id,
step_id_t step_id) {
auto job = m_job_map_.GetValueExclusivePtr(job_id);
if (!job) {
CRANE_ERROR("[Step #{}.{}] Failed to find job allocation", job_id, step_id);
return std::unexpected{CraneErrCode::ERR_NON_EXISTENT};
}
absl::MutexLock lock(job->step_map_mtx.get());
auto step_it = job->step_map.find(step_id);
if (step_it == job->step_map.end()) {
CRANE_ERROR("[Step #{}.{}] Failed to find step allocation", job_id,
step_id);
return std::unexpected{CraneErrCode::ERR_NON_EXISTENT};
}
auto& stub = step_it->second->supervisor_stub;
return stub->QueryStepEnv();
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

⚠️ Potential issue | 🔴 Critical

QuerySshStepEnvVariables should also guard against null supervisor_stub

This method has the same pattern: it finds the step and immediately calls stub->QueryStepEnv() without verifying that supervisor_stub is initialized. For steps where the supervisor failed to start, this will null-deref.

Mirror the ChangeStepTimelimit fix:

  auto& stub = step_it->second->supervisor_stub;
-  return stub->QueryStepEnv();
+  if (!stub) {
+    CRANE_ERROR("[Step #{}.{}] Supervisor stub is null when querying SSH "
+                "step env.", job_id, step_id);
+    return std::unexpected{CraneErrCode::ERR_NON_EXISTENT};
+  }
+  return stub->QueryStepEnv();

This keeps the env query path safe even when step startup failed.

🤖 Prompt for AI Agents
In src/Craned/Core/JobManager.cpp around lines 494 to 510, the method
QuerySshStepEnvVariables currently dereferences step->supervisor_stub without
checking for null; update it to mirror the ChangeStepTimelimit fix by checking
if stub (step_it->second->supervisor_stub) is null while holding the same mutex,
log a clear CRANE_ERROR like "[Step #{}.{}] supervisor_stub not initialized"
including job_id and step_id, and return
std::unexpected{CraneErrCode::ERR_NON_EXISTENT} (or the same error used in
ChangeStepTimelimit) instead of calling stub->QueryStepEnv(); ensure the
null-check and return occur before invoking QueryStepEnv.

Comment on lines 757 to 787
bool JobManager::MigrateProcToCgroupOfJob(pid_t pid, task_id_t job_id) {
auto job = m_job_map_.GetValueExclusivePtr(job_id);
if (!job) {
CRANE_TRACE("Job #{} does not exist when querying its cgroup.", job_id);
CRANE_DEBUG("Job #{} does not exist when querying its cgroup.", job_id);
return false;
}
if (job->cgroup) {
return job->cgroup->MigrateProcIn(pid);
}

auto cg_expt = CgroupManager::AllocateAndGetCgroup(
CgroupManager::CgroupStrByJobId(job->job_id), job->job_to_d.res(), false);
if (cg_expt.has_value()) {
job->cgroup = std::move(cg_expt.value());
return job->cgroup->MigrateProcIn(pid);
}

CRANE_ERROR("Failed to get cgroup for job#{}", job_id);
absl::MutexLock lk(job->step_map_mtx.get());
auto daemon_step_it = job->step_map.find(kDaemonStepId);
if (daemon_step_it == job->step_map.end()) {
CRANE_DEBUG(
"[Step #{}.{}] Daemon step not found when migrating pid {} to "
"cgroup of job#{}.",
job_id, kDaemonStepId, pid, job_id);
}
auto& daemon_step = daemon_step_it->second;
auto stub = daemon_step->supervisor_stub;
if (!stub) {
CRANE_ERROR(
"[Job #{}] Daemon step sSupervisor stub is null when migrating pid {} "
"to "
"cgroup of job#{}.",
job_id, kDaemonStepId, pid, job_id);
}
auto err = stub->MigrateSshProcToCg(pid);
if (err == CraneErrCode::SUCCESS) {
return true;
}
CRANE_ERROR("[Job #{}] Failed to migrate pid {} to cgroup of job.", job_id,
pid);
return false;
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

⚠️ Potential issue | 🔴 Critical

Critical: potential iterator and null-stub dereference in MigrateProcToCgroupOfJob

Two issues here:

  1. If the daemon step is missing, you log a debug message but still dereference daemon_step_it:
auto daemon_step_it = job->step_map.find(kDaemonStepId);
if (daemon_step_it == job->step_map.end()) {
  CRANE_DEBUG(...);
}
auto& daemon_step = daemon_step_it->second;  // UB if end()
  1. If daemon_step->supervisor_stub is null, you log an error but still call stub->MigrateSshProcToCg(pid).

Both can lead to crashes. Early-exit on missing daemon step or stub:

  auto daemon_step_it = job->step_map.find(kDaemonStepId);
  if (daemon_step_it == job->step_map.end()) {
    CRANE_DEBUG(
        "[Step #{}.{}] Daemon step not found when migrating pid {} to "
        "cgroup of job#{}.",
        job_id, kDaemonStepId, pid, job_id);
-  }
-  auto& daemon_step = daemon_step_it->second;
-  auto stub = daemon_step->supervisor_stub;
-  if (!stub) {
+    return false;
+  }
+  auto& daemon_step = daemon_step_it->second;
+  auto stub = daemon_step->supervisor_stub;
+  if (!stub) {
     CRANE_ERROR(
         "[Job #{}] Daemon step sSupervisor stub is null when migrating pid {} "
         "to "
         "cgroup of job#{}.",
         job_id, kDaemonStepId, pid, job_id);
+    return false;
   }

The rest of the function (calling MigrateSshProcToCg and logging failures) can remain unchanged.

🤖 Prompt for AI Agents
In src/Craned/Core/JobManager.cpp around lines 757 to 787, the code may
dereference job->step_map.end() and call methods on a null supervisor_stub;
modify the function to early-return false after logging when daemon_step_it ==
job->step_map.end() instead of continuing, and likewise early-return false after
logging when daemon_step->supervisor_stub is null, so you never dereference the
end iterator or call stub->MigrateSshProcToCg on a null pointer; keep existing
log messages and mutex usage but add these two immediate checks and returns.

Comment on lines +408 to +422
void StepInstance::ExecuteStepAsync() {
this->GotNewStatus(StepStatus::Running);

g_thread_pool->detach_task([job_id = job_id, step_id = step_id,
stub = supervisor_stub] {
auto code = stub->ExecuteStep();
if (code != CraneErrCode::SUCCESS) {
CRANE_ERROR("[Step #{}.{}] Supervisor failed to execute task, code:{}.",
job_id, step_id, static_cast<int>(code));
}
});
}
Copy link
Contributor

@coderabbitai coderabbitai bot Nov 19, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

⚠️ Potential issue | 🔴 Critical

Potential crash if ExecuteStepAsync is called before supervisor_stub is set

ExecuteStepAsync captures stub = supervisor_stub and calls stub->ExecuteStep() without checking for null. There is a window where the step has been inserted into job->step_map but SpawnSupervisor has not yet completed, so EvCleanGrpcExecuteStepQueueCb_ can invoke ExecuteStepAsync while supervisor_stub is still null. That would result in a null dereference in the detached task.

Add a guard and capture a local shared_ptr:

void StepInstance::ExecuteStepAsync() {
-  this->GotNewStatus(StepStatus::Running);
-
-  g_thread_pool->detach_task([job_id = job_id, step_id = step_id,
-                              stub = supervisor_stub] {
+  if (!supervisor_stub) {
+    CRANE_ERROR("[Step #{}.{}] supervisor_stub is null, cannot execute step.",
+                job_id, step_id);
+    return;
+  }
+
+  this->GotNewStatus(StepStatus::Running);
+
+  auto stub = supervisor_stub;  // keep local copy
+  g_thread_pool->detach_task([job_id = job_id, step_id = step_id,
+                              stub = std::move(stub)] {
     auto code = stub->ExecuteStep();

You may also want to reject ExecuteStepAsync if the step is not yet in a “starting/configuring” state, but the null-check alone removes the crash.

🤖 Prompt for AI Agents
In src/Craned/Core/StepInstance.cpp around lines 408 to 419, ExecuteStepAsync
captures supervisor_stub and invokes stub->ExecuteStep() inside a detached task
which can dereference null if SpawnSupervisor hasn't set supervisor_stub yet;
fix by taking a local/shared_ptr copy of supervisor_stub before scheduling the
task, check that the local stub is non-null and return (or log and return) early
if it is null, and only then detach the task using the captured local/shared_ptr
so the background lambda cannot run with a null stub.

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

stub是sharedptr

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@L-Xiafeng, 您说得对,stub 确实是 shared_ptr,所以按值捕获(stub = supervisor_stub)在生命周期管理上是安全的——即使 StepInstance 被销毁,lambda 中的 stub 副本仍然有效。

不过,CodeRabbit 的关注点是 null 状态而非生命周期:如果 ExecuteStepAsync()SpawnSupervisor() 完成之前被调用(例如并发调度或错误路径),那时 supervisor_stub 可能还是 nullptr,后续 stub->ExecuteStep() 就会解引用空指针导致崩溃。

建议:如果控制流保证 ExecuteStepAsync 只在 SpawnSupervisor 成功后调用,可以添加一个 assert 作为防御性检查;如果存在竞态或异步调度的可能,建议加上运行时 null 检查:

void StepInstance::ExecuteStepAsync() {
  if (!supervisor_stub) {
    CRANE_ERROR("[Step #{}.{}] supervisor_stub is null, cannot execute.", 
                job_id, step_id);
    return;
  }
  
  this->GotNewStatus(StepStatus::Running);
  
  auto stub = supervisor_stub;  // 捕获局部副本更清晰
  g_thread_pool->detach_task([job_id = job_id, step_id = step_id, 
                              stub = std::move(stub)] {
    auto code = stub->ExecuteStep();
    // ...
  });
}

这样即使是 shared_ptr,也能避免解引用空指针的风险。您认为当前的调用顺序能否保证 supervisor_stub 在此之前一定被初始化?


🧠 Learnings used
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Craned/SupervisorKeeper.h:32-32
Timestamp: 2025-05-26T11:04:30.580Z
Learning: The Supervisor component in the Crane system is designed to manage only one task per instance. The task specification is provided to the Supervisor during startup by Craned (likely through InitSupervisorRequest), so the ExecuteTask() method doesn't need to accept task parameters since the Supervisor already knows which task to execute.

Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Supervisor/Supervisor.cpp:68-69
Timestamp: 2025-05-25T04:08:03.273Z
Learning: In the Crane supervisor component (src/Craned/Supervisor/), the supervisor process communicates with Craned through STDOUT using protobuf messages. The supervisor must not send any information to STDOUT before sending the "ready" message to Craned, as this would interfere with the inter-process communication protocol. Therefore, adding logging statements that might write to STDOUT before the ready message is sent could break the communication.

Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Supervisor/TaskManager.cpp:220-220
Timestamp: 2025-05-25T04:11:50.268Z
Learning: In TaskManager.cpp, when step->IsCrun() is checked before dynamic_cast to CrunMetaInExecution*, the cast is guaranteed to succeed due to the program logic ensuring the correct meta type is used for Crun tasks. Null checks for these dynamic_cast operations are unnecessary in this context.

Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 496
File: src/CraneCtld/CranedKeeper.cpp:51-53
Timestamp: 2025-05-09T01:54:21.256Z
Learning: The ConfigureCraned function in src/CraneCtld/CranedKeeper.cpp is called from a thread pool, so there's no need to worry about it blocking the gRPC completion queue thread.

Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 625
File: src/Craned/Core/SupervisorKeeper.cpp:145-147
Timestamp: 2025-09-21T11:26:40.935Z
Learning: In C++17 and later, Class Template Argument Deduction (CTAD) allows `std::shared_ptr stub = std::make_shared<SupervisorStub>();` syntax to be valid, as the compiler can deduce the template parameter from make_shared. Projects using C++17+ don't need explicit template parameters in this context.

Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Supervisor/SupervisorPublicDefs.h:32-37
Timestamp: 2025-05-26T11:06:28.796Z
Learning: The user L-Xiafeng prefers to defer refactoring duplicate definitions until they become a larger pattern in the codebase, rather than addressing individual instances immediately.

Comment on lines +2306 to +2348
auto& pid = elem.first;
auto& prom = elem.second;
if (!m_step_.IsDaemonStep()) {
CRANE_ERROR("Trying to move pid {} to no daemon step", pid);
prom.set_value(CraneErrCode::ERR_INVALID_PARAM);
}
if (!m_step_.step_user_cg) {
auto cg_expt = CgroupManager::AllocateAndGetCgroup(
CgroupManager::CgroupStrByStepId(m_step_.job_id, m_step_.step_id,
false),
m_step_.GetStep().res(), false, Common::CgConstant::kCgMinMem);
if (!cg_expt.has_value()) {
CRANE_ERROR("[Step #{}.{}] Failed to allocate cgroup", m_step_.job_id,
m_step_.step_id);
prom.set_value(CraneErrCode::ERR_CGROUP);
continue;
}
m_step_.step_user_cg = std::move(cg_expt.value());
}
if (m_step_.step_user_cg->MigrateProcIn(pid)) {
prom.set_value(CraneErrCode::SUCCESS);
} else {
prom.set_value(CraneErrCode::ERR_CGROUP);
}
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

⚠️ Potential issue | 🔴 Critical

Prevent double set_value when step is non-daemon.

If we enter the non-daemon branch we already fulfil the promise, but we then fall through and fulfil it again (and even try to allocate a user cg), triggering a std::future_error and crashing the supervisor. Add an early continue right after delivering the error so the loop skips the rest of the handler.

Apply this diff to fix the logic:

     if (!m_step_.IsDaemonStep()) {
       CRANE_ERROR("Trying to move pid {} to no daemon step", pid);
       prom.set_value(CraneErrCode::ERR_INVALID_PARAM);
+      continue;
     }
🤖 Prompt for AI Agents
In src/Craned/Supervisor/TaskManager.cpp around lines 2306 to 2330, the code
sets prom.set_value(CraneErrCode::ERR_INVALID_PARAM) when m_step_.IsDaemonStep()
is false but then continues executing the rest of the loop and may call
set_value again (and attempt cgroup allocation); add an immediate continue right
after setting the promise in the non-daemon branch to skip the rest of the
handler and prevent a double set_value and subsequent crash.

@github-actions github-actions bot added the test-passed Build and test success label Nov 19, 2025
@L-Xiafeng L-Xiafeng linked an issue Nov 20, 2025 that may be closed by this pull request
@github-actions github-actions bot removed the test-passed Build and test success label Nov 20, 2025
Copy link
Contributor

@coderabbitai coderabbitai bot left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Actionable comments posted: 0

♻️ Duplicate comments (3)
src/Craned/Supervisor/TaskManager.cpp (2)

1697-1704: Unify step shutdown status types and propagate final reason.

ShutdownSupervisorAsync and m_shutdown_status_queue_ currently use crane::grpc::TaskStatus, but EvShutdownSupervisorCb_ passes that to StepInstance::GotNewStatus and TaskManager::TaskFinish_ assigns it into StepInstance::TerminationStatus::final_status_on_termination, which is a StepStatus. This relies on implicit conversion between distinct enums and does not respect the semantic separation between per‑task and per‑step status. Also, TaskFinish_ never uses its reason parameter to populate final_reason_on_termination, so the final step‑level reason is always empty.

Apply a fix along these lines to keep types coherent and preserve reasons:

- void ShutdownSupervisorAsync(
-     crane::grpc::TaskStatus new_status = StepStatus::Completed,
+ void ShutdownSupervisorAsync(
+     StepStatus new_status = StepStatus::Completed,
      uint32_t exit_code = 0, std::string reason = "");
…
-  ConcurrentQueue<std::tuple<crane::grpc::TaskStatus, uint32_t, std::string>>
+  ConcurrentQueue<std::tuple<StepStatus, uint32_t, std::string>>
       m_shutdown_status_queue_;

And in the .cpp:

- void TaskManager::TaskFinish_(task_id_t task_id,
-                               crane::grpc::TaskStatus new_status,
+ void TaskManager::TaskFinish_(task_id_t task_id,
+                               StepStatus new_status,
                                uint32_t exit_code,
                                std::optional<std::string> reason) {
@@
-  auto& status = m_step_.final_termination_status;
+  auto& status = m_step_.final_termination_status;
   if (status.max_exit_code < exit_code) {
@@
   if (new_status != StepStatus::Completed)
     status.final_status_on_termination = new_status;
+  if (reason.has_value())
+    status.final_reason_on_termination = *reason;
@@
- void TaskManager::EvShutdownSupervisorCb_() {
-  std::tuple<crane::grpc::TaskStatus, uint32_t, std::string> final_status;
+ void TaskManager::EvShutdownSupervisorCb_() {
+  std::tuple<StepStatus, uint32_t, std::string> final_status;

Then update all TaskFinish_ call sites to pass an appropriate StepStatus (e.g., map from crane::grpc::TaskStatus where needed) and ensure ShutdownSupervisorAsync callers provide a StepStatus.
(Exact enum mapping can follow your proto design but should be explicit to avoid accidental breakage.)

Also applies to: 1870-1915, 1711-1753


1848-1858: Fix double set_value and invalid cgroup migration for non‑daemon steps.

In EvGrpcMigrateSshProcToCgroupCb_, when m_step_.IsDaemonStep() is false you call prom.set_value(ERR_INVALID_PARAM) but then continue executing the rest of the function, potentially allocating a step cgroup and calling MigrateProcIn and prom.set_value(...) again. This both migrates processes into cgroups for non‑daemon steps (which you explicitly disallow) and triggers std::future_error due to setting the same promise twice.

Apply this minimal fix:

    if (!m_step_.IsDaemonStep()) {
      CRANE_ERROR("Trying to move pid {} to no daemon step", pid);
      prom.set_value(CraneErrCode::ERR_INVALID_PARAM);
+      continue;
    }

This keeps non‑daemon requests rejected cleanly and prevents double fulfillment. The rest of the function (cgroup allocation and MigrateProcIn) remains applicable only to daemon steps.

Also applies to: 2313-2341

src/Craned/Supervisor/TaskManager.h (1)

419-427: Align ShutdownSupervisorAsync and shutdown queue types with step status.

In the header, ShutdownSupervisorAsync takes crane::grpc::TaskStatus but its default uses StepStatus::Completed, and m_shutdown_status_queue_ stores std::tuple<crane::grpc::TaskStatus, …>. The implementation then feeds these values into StepInstance::GotNewStatus and step‑level status reporting. This mixes task and step enums and either won’t compile cleanly or will conflate concepts if you later extend one enum independently.

Update the declaration to use StepStatus consistently, matching the step‑oriented semantics:

-  void ShutdownSupervisorAsync(
-      crane::grpc::TaskStatus new_status = StepStatus::Completed,
+  void ShutdownSupervisorAsync(
+      StepStatus new_status = StepStatus::Completed,
       uint32_t exit_code = 0, std::string reason = "");
…
-  ConcurrentQueue<std::tuple<crane::grpc::TaskStatus, uint32_t, std::string>>
+  ConcurrentQueue<std::tuple<StepStatus, uint32_t, std::string>>
       m_shutdown_status_queue_;

and keep the .cpp in sync (see corresponding comment there). This preserves the intended separation between task‑level TaskStatus and step‑level StepStatus.

Also applies to: 508-510, 534-538, 535-537, 569-576

🧹 Nitpick comments (4)
src/Craned/Supervisor/TaskManager.cpp (4)

262-303: Per‑task cgroup allocation/cleanup is balanced but currently unused for migration.

ITaskInstance::Prepare now allocates a per‑task cgroup and Cleanup reliably kills and destroys it with a bounded retry loop. That gives you a clean lifecycle even on early failures (since LaunchExecution_ calls TaskFinish_, which calls Cleanup). Right now there’s no code in this file that actually migrates task processes into m_task_cg, so the cgroup is effectively just created and torn down; if per‑task limits become important later, you’ll want to wire in MigrateProcIn somewhere in the spawn path.


1562-1573: New uvw async handles and timers are wired correctly; consider idle sleep trade‑off.

The added async handles (m_supervisor_finish_init_handle_, m_shutdown_supervisor_handle_, m_grpc_check_status_async_handle_, m_grpc_migrate_ssh_proc_to_cgroup_async_handle_) and the 1s SIGCHLD polling timer are registered with handlers and started appropriately. The idle handler’s std::this_thread::sleep_for(50ms) throttles the loop; that’s acceptable if you don’t need low‑latency wakeups, but it will impose a minimum reaction time on events that depend on the idle handler to notice m_supervisor_exit_.

Also applies to: 1652-1665, 1583-1589, 1670-1677


1835-1846: CheckStatusAsync pattern is correct but effectively synchronous.

CheckStatusAsync enqueues a std::promise<StepStatus>, signals the uvw async handle, and then blocks on status_future.get(), while EvGrpcCheckStatusCb_ fulfills the promise on the TaskManager loop thread. This is a valid cross‑thread pattern and should not deadlock because the event loop runs in m_uvw_thread_. Just be aware that the RPC handler thread blocks until the supervisor loop responds; if you ever need a fully async gRPC implementation, you’d want to move the get() to a completion callback instead.

Also applies to: 2306-2311


2209-2294: Multi‑task execution path with latch and per‑step cgroup looks correct; watch shared map concurrency.

EvGrpcExecuteTaskCb_ now:

  • Sets the step to Running.
  • Instantiates one ITaskInstance per task_id.
  • Initializes password info once and bails all tasks on failure.
  • Allocates a shared step_user_cg (with proper error handling and per‑task TaskFinish_ on failure).
  • Launches all tasks on a thread pool and waits on a std::latch, reporting aggregate err via the promise.

Structurally this is solid. One thing to keep in mind is that m_exec_id_task_id_map_ is written under thread_pool_mutex inside the worker lambdas, but read from the uvw loop thread (e.g., in EvCleanSigchldQueueCb_) without the same lock. If that map isn’t otherwise confined to a single thread, consider guarding both reads and writes with a shared mutex or switching to a concurrent map to avoid data races.

📜 Review details

Configuration used: Path: .coderabbit.yaml

Review profile: CHILL

Plan: Pro

📥 Commits

Reviewing files that changed from the base of the PR and between e49428e and 797e3d0.

📒 Files selected for processing (2)
  • src/Craned/Supervisor/TaskManager.cpp (23 hunks)
  • src/Craned/Supervisor/TaskManager.h (15 hunks)
🧰 Additional context used
🧠 Learnings (19)
📓 Common learnings
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Supervisor/SupervisorPublicDefs.h:32-37
Timestamp: 2025-05-26T11:06:28.796Z
Learning: The user L-Xiafeng prefers to defer refactoring duplicate definitions until they become a larger pattern in the codebase, rather than addressing individual instances immediately.
📚 Learning: 2025-05-08T07:39:29.207Z
Learnt from: huerni
Repo: PKUHPC/CraneSched PR: 469
File: src/Craned/TaskManager.cpp:1162-1175
Timestamp: 2025-05-08T07:39:29.207Z
Learning: In CraneSched, when a task fails in the LaunchTaskInstanceMt_ method, ActivateTaskStatusChangeAsync_ does not automatically release PMIx resources or cgroup. Explicit cleanup is required by calling g_pmix_server->DeregisterTask() and g_cg_mgr->ReleaseCgroupByTaskIdOnly().

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
  • src/Craned/Supervisor/TaskManager.h
📚 Learning: 2025-05-08T07:39:29.207Z
Learnt from: huerni
Repo: PKUHPC/CraneSched PR: 469
File: src/Craned/TaskManager.cpp:1162-1175
Timestamp: 2025-05-08T07:39:29.207Z
Learning: In CraneSched, when a task fails in the LaunchTaskInstanceMt_ method after PMIx registration, ActivateTaskStatusChangeAsync_ does not automatically release PMIx resources or cgroup resources. Explicit cleanup is required by calling g_pmix_server->DeregisterTask() and g_cg_mgr->ReleaseCgroupByTaskIdOnly() before returning from the function.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
  • src/Craned/Supervisor/TaskManager.h
📚 Learning: 2025-05-26T11:04:30.580Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Craned/SupervisorKeeper.h:32-32
Timestamp: 2025-05-26T11:04:30.580Z
Learning: The Supervisor component in the Crane system is designed to manage only one task per instance. The task specification is provided to the Supervisor during startup by Craned (likely through InitSupervisorRequest), so the ExecuteTask() method doesn't need to accept task parameters since the Supervisor already knows which task to execute.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
  • src/Craned/Supervisor/TaskManager.h
📚 Learning: 2025-05-25T04:11:50.268Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Supervisor/TaskManager.cpp:220-220
Timestamp: 2025-05-25T04:11:50.268Z
Learning: In TaskManager.cpp, when step->IsCrun() is checked before dynamic_cast to CrunMetaInExecution*, the cast is guaranteed to succeed due to the program logic ensuring the correct meta type is used for Crun tasks. Null checks for these dynamic_cast operations are unnecessary in this context.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-05-02T07:12:46.896Z
Learnt from: RileyWen
Repo: PKUHPC/CraneSched PR: 475
File: src/Craned/CranedServer.h:0-0
Timestamp: 2025-05-02T07:12:46.896Z
Learning: The initialization of `m_supervisor_recovered_` to `true` in the CranedServer class is intentional despite the comment saying "When supervisor ready, init with false". This is temporary until the supervisor functionality is fully implemented.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-05-26T11:00:54.563Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Craned/SupervisorKeeper.cpp:174-178
Timestamp: 2025-05-26T11:00:54.563Z
Learning: The CraneSched project uses C++23 standard, allowing the use of modern C++ features like std::ranges::to and other C++23 language features and library components.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-05-25T04:08:03.273Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Supervisor/Supervisor.cpp:68-69
Timestamp: 2025-05-25T04:08:03.273Z
Learning: In the Crane supervisor component (src/Craned/Supervisor/), the supervisor process communicates with Craned through STDOUT using protobuf messages. The supervisor must not send any information to STDOUT before sending the "ready" message to Craned, as this would interfere with the inter-process communication protocol. Therefore, adding logging statements that might write to STDOUT before the ready message is sent could break the communication.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
  • src/Craned/Supervisor/TaskManager.h
📚 Learning: 2025-05-26T11:04:56.055Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Craned/SupervisorKeeper.h:72-72
Timestamp: 2025-05-26T11:04:56.055Z
Learning: The CraneSched project prefers using global variables (like `g_supervisor_keeper`) over dependency injection patterns. The team does not follow dependency injection approaches for managing singleton instances.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-06-07T10:47:59.071Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 520
File: src/CraneCtld/CranedKeeper.cpp:416-417
Timestamp: 2025-06-07T10:47:59.071Z
Learning: In src/CraneCtld/CranedKeeper.h, the m_shutting_down_ member in CranedStub class is declared as std::atomic_bool, making it thread-safe for concurrent access without additional synchronization.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
  • src/Craned/Supervisor/TaskManager.h
📚 Learning: 2025-06-30T08:43:44.470Z
Learnt from: huerni
Repo: PKUHPC/CraneSched PR: 537
File: src/Craned/TaskManager.cpp:145-147
Timestamp: 2025-06-30T08:43:44.470Z
Learning: In the CraneSched codebase, src/Craned/Craned.cpp guarantees that g_config.CranedRes.contains(g_config.CranedIdOfThisNode) through explicit validation during startup. The code checks if g_config.CranedRes.contains(g_config.Hostname) and exits the process if not found, then sets g_config.CranedIdOfThisNode = g_config.Hostname. TaskManager constructor is called after this validation, so g_config.CranedRes[g_config.CranedIdOfThisNode] is guaranteed to be valid.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
  • src/Craned/Supervisor/TaskManager.h
📚 Learning: 2025-05-09T01:54:21.256Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 496
File: src/CraneCtld/CranedKeeper.cpp:51-53
Timestamp: 2025-05-09T01:54:21.256Z
Learning: The ConfigureCraned function in src/CraneCtld/CranedKeeper.cpp is called from a thread pool, so there's no need to worry about it blocking the gRPC completion queue thread.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
  • src/Craned/Supervisor/TaskManager.h
📚 Learning: 2025-05-09T01:54:39.465Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 496
File: src/Craned/TaskManager.cpp:1509-1517
Timestamp: 2025-05-09T01:54:39.465Z
Learning: The CraneSched project uses C++23, which supports Class Template Argument Deduction (CTAD) for standard containers like std::set and includes ranges support, making std::ranges::views::keys valid without additional headers.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-05-08T07:38:42.362Z
Learnt from: huerni
Repo: PKUHPC/CraneSched PR: 469
File: src/Craned/TaskManager.cpp:0-0
Timestamp: 2025-05-08T07:38:42.362Z
Learning: In CraneSched's PMIx integration, the `g_pmix_server->SetupFork()` function must be called in the child process after fork() and before exec() to properly set up the PMIx environment variables.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-05-25T04:11:27.740Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Supervisor/TaskManager.cpp:396-396
Timestamp: 2025-05-25T04:11:27.740Z
Learning: In TaskManager.cpp, GetCrunMeta() calls don't need null checks because they're only called in contexts where the task is guaranteed to be a CRUN task (e.g., SetupChildProcessCrunX11_ is only called when step->IsCrun() && x11() conditions are met), ensuring the metadata will always be valid.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-05-02T07:06:36.103Z
Learnt from: RileyWen
Repo: PKUHPC/CraneSched PR: 475
File: src/CraneCtld/CtldGrpcServer.cpp:113-118
Timestamp: 2025-05-02T07:06:36.103Z
Learning: In CraneSched, gRPC methods should generally return Status::OK even when handling error conditions, as non-OK statuses cause the connection to terminate. Error information should be communicated within the RPC response payload instead.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-05-09T02:16:56.723Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 496
File: src/Craned/TaskManager.cpp:1498-1506
Timestamp: 2025-05-09T02:16:56.723Z
Learning: The `QueryRunningTasksAsync()` method in TaskManager.cpp is designed to never be called from inside the event loop thread, so there's no risk of deadlock with the synchronous `res.get()` call.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-05-02T07:05:26.012Z
Learnt from: RileyWen
Repo: PKUHPC/CraneSched PR: 475
File: src/CraneCtld/CranedKeeper.cpp:601-602
Timestamp: 2025-05-02T07:05:26.012Z
Learning: In the CraneCtld codebase, the variables m_disconnected_ and m_registered_ in CranedStub class are already defined as std::atomic_bool, making them thread-safe for concurrent access without additional synchronization.

Applied to files:

  • src/Craned/Supervisor/TaskManager.h
📚 Learning: 2025-05-08T09:35:39.809Z
Learnt from: huerni
Repo: PKUHPC/CraneSched PR: 469
File: src/Craned/Pmix/PmixCollRing.cpp:0-0
Timestamp: 2025-05-08T09:35:39.809Z
Learning: In the PMIx implementation for CraneSched, objects referenced in asynchronous gRPC callbacks (like `coll_ring_ctx`) remain valid as long as the parent object (`this`) is not destroyed. The `Coll` class uses shared_ptr management to ensure its lifetime extends through all pending callbacks.

Applied to files:

  • src/Craned/Supervisor/TaskManager.h
🧬 Code graph analysis (2)
src/Craned/Supervisor/TaskManager.cpp (4)
src/Craned/Core/StepInstance.cpp (2)
  • GotNewStatus (334-406)
  • GotNewStatus (334-334)
src/Craned/Common/CgroupManager.cpp (6)
  • AllocateAndGetCgroup (494-552)
  • AllocateAndGetCgroup (495-497)
  • CgroupStrByTaskId (290-294)
  • CgroupStrByTaskId (290-291)
  • CgroupStrByStepId (283-288)
  • CgroupStrByStepId (283-284)
src/Craned/Core/CranedPublicDefs.h (1)
  • g_config (156-156)
src/Utilities/PublicHeader/OS.cpp (2)
  • DeleteFile (64-71)
  • DeleteFile (64-64)
src/Craned/Supervisor/TaskManager.h (2)
src/Craned/Common/CgroupManager.h (1)
  • CgroupInterface (389-421)
src/Craned/Supervisor/TaskManager.cpp (25)
  • GotNewStatus (114-189)
  • GotNewStatus (114-114)
  • InitOomBaseline (209-230)
  • InitOomBaseline (209-209)
  • EvaluateOomOnExit (232-260)
  • EvaluateOomOnExit (232-232)
  • ProcInstance (319-336)
  • SupervisorFinishInit (1689-1691)
  • SupervisorFinishInit (1689-1689)
  • Wait (1693-1695)
  • Wait (1693-1693)
  • ShutdownSupervisorAsync (1697-1704)
  • ShutdownSupervisorAsync (1697-1699)
  • CheckStatusAsync (1835-1846)
  • CheckStatusAsync (1835-1836)
  • MigrateSshProcToCgroupAsync (1848-1858)
  • MigrateSshProcToCgroupAsync (1848-1848)
  • EvSupervisorFinishInitCb_ (1860-1868)
  • EvSupervisorFinishInitCb_ (1860-1860)
  • EvShutdownSupervisorCb_ (1870-1915)
  • EvShutdownSupervisorCb_ (1870-1870)
  • EvGrpcCheckStatusCb_ (2306-2311)
  • EvGrpcCheckStatusCb_ (2306-2306)
  • EvGrpcMigrateSshProcToCgroupCb_ (2313-2341)
  • EvGrpcMigrateSshProcToCgroupCb_ (2313-2313)
🔇 Additional comments (12)
src/Craned/Supervisor/TaskManager.cpp (8)

114-189: Step status transition validation looks consistent and guarded.

The new StepInstance::GotNewStatus implementation enforces sensible transitions (e.g., StartingRunning, RunningCompleting, finished states from running/starting/configuring) and logs on unexpected sequences while still updating m_status_. For daemon steps you explicitly reject Starting, which matches the new flow where daemon steps jump directly from Configuring to Running. This looks correct and aligns with the StepInstance logic in src/Craned/Core/StepInstance.cpp.


338-346: CRANE_PROCID/CRANE_PROC_ID envs are straightforward and safe.

Setting both CRANE_PROCID and CRANE_PROC_ID to task_id directly in ProcInstance::InitEnvMap is simple and should make it easier for child processes to introspect their logical task index. No functional concerns here.


430-432: Guarding crun pty with single task per node is a good safety net.

The assert preventing pty crun steps from having more than one task per node clearly encodes the current limitation and will fail fast in unsupported configurations. This is appropriate for such a tricky IO mode.


723-807: Ordering of container prepare/cleanup with base class is correct.

ContainerInstance::Prepare now calls ITaskInstance::Prepare first, and Cleanup calls ITaskInstance::Cleanup before invoking Kill(0). That ordering makes sense: allocate and configure the per‑task cgroup up front, and tear it down before doing CRI‑level pod/container deletion. Error propagation via the CraneErrCode check is also correct.

Also applies to: 913-920


1237-1303: ProcInstance now reuses shared Prepare logic without leaking resources.

Calling ITaskInstance::Prepare at the start of ProcInstance::Prepare ensures that per‑task cgroups are allocated consistently for both process and container tasks. If subsequent file/script setup fails, LaunchExecution_ will treat this as a prepare error and TaskFinish_ will call Cleanup, which destroys the cgroup, so there’s no leak on early exit. The rest of the script‑writing logic remains unchanged.


1689-1691: SupervisorFinishInit path and step status notifications look coherent.

SupervisorFinishInit() simply schedules EvSupervisorFinishInitCb_, which then sets the initial step status to Running for daemon steps and Starting for others, followed by an async StepStatusChangeAsync to Craned. This matches the GotNewStatus transition rules and gives Ctld a clear initial state.

Also applies to: 1860-1868


1769-1789: Lazy initialization of CRI and Cfored clients is safe and avoids redundant setup.

Using std::call_once on m_step_.cri_client_flag and m_step_.cfored_client_flag inside LaunchExecution_ ensures CRI and Cfored clients are only initialized when actually needed (container/crun steps) and only once even when multiple tasks are launched concurrently. That’s a good trade‑off for startup latency vs. complexity.


2009-2020: Termination/time‑limit change gating on running state is sensible.

In both EvTaskTimerCb_ and EvCleanChangeTaskTimeLimitQueueCb_ you now check m_step_.IsCalloc() and m_step_.IsRunning()/AllTaskFinished() before acting. This avoids racing against initialization and prevents extending timeouts for already‑completed steps, which makes the behavior more predictable when users issue terminate/change‑timelimit requests early or late.

Also applies to: 2165-2204

src/Craned/Supervisor/TaskManager.h (4)

65-83: StepInstance state, status API, and termination tracking look coherent.

The new fields (cri_client_flag, cfored_client_flag, step_user_cg, TerminationStatus, m_status_) plus helpers (GetStatus(), IsRunning(), GetRunningTaskIds(), GotNewStatus()) give StepInstance a clear model of its lifecycle and resources. The comment on the destructor (“no clean up action”) matches the design in the .cpp, where all cleanup is driven through explicit paths (task finish, shutdown callback), which is appropriate for a long‑lived supervisor process.

Also applies to: 67-80, 119-123, 174-185


84-92: Deriving task_ids from task_res_map keys assumes scheduler completeness.

Initializing task_ids from step.task_res_map() | std::views::keys ensures I/O and cgroup setup is automatically driven by the scheduler’s per‑task resource map. This is nice and avoids separate task count plumbing, but it does rely on the scheduler always setting task_res_map correctly (including IDs for “dummy” tasks like container steps). If that invariant is guaranteed, this is a clean design.

If there’s any doubt about task_res_map always containing all valid task_ids (especially for legacy callers), consider asserting this on the Craned side or falling back to a sequential [0, ntasks) construction when the map is empty.


222-263: ITaskInstance and derived ctors cleanly bind per‑task identity and cgroup handle.

The explicit ITaskInstance(StepInstance*, task_id_t) constructor, public task_id member, and m_task_cg pointer provide a clear place to hang per‑task identity and resources. Updating ContainerInstance and ProcInstance ctors to forward step_spec and task_id keeps the ownership model simple and aligns with how EvGrpcExecuteTaskCb_ instantiates tasks. No issues spotted here.

Also applies to: 265-276, 338-362


484-487: New async RPC helpers and queues match their implementations.

The declarations for CheckStatusAsync, MigrateSshProcToCgroupAsync, and their handlers/queues (m_grpc_check_status_async_handle_, m_grpc_check_status_queue_, m_grpc_migrate_ssh_proc_to_cgroup_async_handle_, m_grpc_migrate_ssh_proc_to_cgroup_queue_) line up with the .cpp definitions and use the same promise‑passing pattern as existing async methods. Once the double‑set_value issue in the .cpp is fixed, this looks like a solid extension of the supervisor’s public API.

Also applies to: 508-531, 569-576

@L-Xiafeng L-Xiafeng marked this pull request as draft November 22, 2025 09:04
@L-Xiafeng L-Xiafeng changed the title feat: Introduce task for craned step [WIP] feat: Introduce task for craned step Nov 22, 2025
@L-Xiafeng L-Xiafeng changed the title [WIP] feat: Introduce task for craned step feat: Introduce task for craned step Nov 22, 2025
@L-Xiafeng L-Xiafeng marked this pull request as ready for review November 22, 2025 10:48
Copy link
Contributor

@coderabbitai coderabbitai bot left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Actionable comments posted: 5

Caution

Some comments are outside the diff and can’t be posted inline due to platform limitations.

⚠️ Outside diff range comments (7)
scripts/wipe_data.py (1)

26-48: Consider validating CraneCtldDbPath explicitly.

On line 43, if CraneCtldDbPath is missing from the DB config, it defaults to an empty string, resulting in embedded_db_path = base_dir. This might mask a configuration error. Consider either:

  • Requiring CraneCtldDbPath explicitly and raising an error if missing
  • Adding a comment documenting that base_dir is the fallback
-    embedded_db_path = os.path.join(base_dir, db_config.get("CraneCtldDbPath", ""))
+    crane_ctld_db_path = db_config.get("CraneCtldDbPath")
+    if not crane_ctld_db_path:
+        raise ValueError("Missing CraneCtldDbPath in DB config.")
+    embedded_db_path = os.path.join(base_dir, crane_ctld_db_path)
src/Craned/Supervisor/CforedClient.cpp (2)

151-199: X11FdInfo shared_ptr is never initialized and map access is unsynchronized across threads

Two serious issues in the X11 listen handler:

  1. x11_fd_info is a default-constructed std::shared_ptr<X11FdInfo> from m_x11_fd_info_map_[x11_local_id] and is never assigned, but is later dereferenced:

    • sock->data(x11_fd_info);
    • x11_fd_info->fd = sock->fd();
    • x11_fd_info->sock = sock;
      This will dereference a null shared_ptr on the first X11 connection.
  2. m_x11_fd_info_map_ is mutated here (insert/erase) on the libuv event thread, while AsyncSendRecvThread_ reads it under m_mtx_. That’s a data race on the unordered_map and its contents.

Consider something along these lines:

-    x11_local_id_t x11_local_id = next_x11_id_++;
-    auto& x11_fd_info = m_x11_fd_info_map_[x11_local_id];
+    // Create per-connection state and register it under lock so that the
+    // gRPC thread can safely access m_x11_fd_info_map_.
+    auto x11_fd_info = std::make_shared<X11FdInfo>();
+    x11_local_id_t x11_local_id;
+    {
+      absl::MutexLock lock(&m_mtx_);
+      x11_local_id = next_x11_id_++;
+      m_x11_fd_info_map_.emplace(x11_local_id, x11_fd_info);
+    }
@@
-    sock->on<uvw::close_event>(
-        [x11_local_id, this](uvw::close_event&, uvw::tcp_handle& s) {
-          CRANE_TRACE("X11 proxy connection was closed.");
-          m_x11_fd_info_map_.erase(x11_local_id);
-        });
-    sock->data(x11_fd_info);
+    sock->on<uvw::close_event>(
+        [x11_local_id, this](uvw::close_event&, uvw::tcp_handle&) {
+          CRANE_TRACE("X11 proxy connection was closed.");
+          absl::MutexLock lock(&m_mtx_);
+          m_x11_fd_info_map_.erase(x11_local_id);
+        });
+    sock->data(x11_fd_info);
@@
-    x11_fd_info->fd = sock->fd();
-    x11_fd_info->sock = sock;
+    x11_fd_info->fd = sock->fd();
+    x11_fd_info->sock = sock;

This both initializes the shared_ptr before use and ensures all map mutations are guarded by the same mutex used in AsyncSendRecvThread_.

Also applies to: 173-190, 193-195


353-376: Iterator misuse and potential double-lock in CleanStopTaskIOQueueCb_/TaskEnd

There are two correctness problems in the stop‑IO cleanup path:

  1. In CleanStopTaskIOQueueCb_, it is used even when the task is not found:
auto it = m_fwd_meta_map.find(task_id);
if (it != m_fwd_meta_map.end()) {
  ...
}
close(it->second.stdout_read);  // UB if it == end
  1. CleanStopTaskIOQueueCb_ holds m_mtx_ via absl::MutexLock and then calls TaskEnd(task_id), which now also takes m_mtx_ internally. Unless absl::Mutex is re-entrant (it isn’t in Abseil), this risks deadlock.

A minimal fix that keeps the existing semantics but avoids both issues:

 void CforedClient::CleanStopTaskIOQueueCb_() {
   task_id_t task_id;
   while (m_stop_task_io_queue_.try_dequeue(task_id)) {
-    absl::MutexLock lock(&m_mtx_);
-    auto it = m_fwd_meta_map.find(task_id);
-    if (it != m_fwd_meta_map.end()) {
-      auto& output_handle = it->second.out_handle;
-      if (!it->second.pty && output_handle.pipe) output_handle.pipe->close();
-      if (it->second.pty && output_handle.tty) output_handle.tty->close();
-      output_handle.pipe.reset();
-      output_handle.tty.reset();
-    }
-
-    close(it->second.stdout_read);
-
-    CRANE_DEBUG("[Task #{}] Finished its output.", task_id);
-
-    bool ok_to_free = this->TaskOutputFinishNoLock_(task_id);
-    if (ok_to_free) {
-      CRANE_DEBUG("[Task #{}] It's ok to unregister.", task_id);
-      this->TaskEnd(task_id);
-    }
+    bool ok_to_free = false;
+    {
+      absl::MutexLock lock(&m_mtx_);
+      auto it = m_fwd_meta_map.find(task_id);
+      if (it == m_fwd_meta_map.end()) {
+        CRANE_WARN("[Task #{}] stop_task_io for unknown task; skipping.", task_id);
+        continue;
+      }
+
+      auto& meta = it->second;
+      auto& output_handle = meta.out_handle;
+      if (!meta.pty && output_handle.pipe) output_handle.pipe->close();
+      if (meta.pty && output_handle.tty) output_handle.tty->close();
+      output_handle.pipe.reset();
+      output_handle.tty.reset();
+
+      close(meta.stdout_read);
+
+      CRANE_DEBUG("[Task #{}] Finished its output.", task_id);
+      ok_to_free = this->TaskOutputFinishNoLock_(task_id);
+      if (ok_to_free)
+        CRANE_DEBUG("[Task #{}] It's ok to unregister.", task_id);
+      // Do not call TaskEnd() while holding m_mtx_.
+    }
+
+    if (ok_to_free) {
+      this->TaskEnd(task_id);
+    }
   };
 }

Optionally, TaskOutputFinishNoLock_ could also avoid operator[] and use find() to prevent accidental insertion, but the above is enough to remove the UB and deadlock risk.

Also applies to: 707-710, 727-733

src/Craned/Core/JobManager.cpp (1)

824-927: Avoid using job_instance after FreeJobInfoNoLock_ in orphaned-job termination

In the elem.mark_as_orphaned && terminate_job branch of EvCleanTerminateTaskQueueCb_, you:

  1. Obtain a raw pointer job_instance = map_ptr->at(elem.job_id).RawPtr();.
  2. Call FreeJobInfoNoLock_(elem.job_id, map_ptr, uid_map_ptr), which moves the JobInD out and erases the map entry.
  3. After that, iterate job_instance->step_map to populate steps_to_clean.

Once you erase the map entry, job_instance is very likely dangling (the JobInD object has been destroyed), so iterating job_instance->step_map is a use-after-free.

Instead, derive the steps from the JobInD you just pulled out:

      auto uid_map_ptr = m_uid_to_job_ids_map_.GetMapExclusivePtr();
      auto job_opt = FreeJobInfoNoLock_(elem.job_id, map_ptr, uid_map_ptr);
      if (job_opt.has_value()) {
-        job_to_clean.emplace_back(std::move(job_opt.value()));
-        for (auto& [step_id, step] : job_instance->step_map) {
-          CRANE_DEBUG("[Step #{}.{}] Removed orphaned step.", elem.job_id,
-                      step_id);
-          auto* step_ptr = step.get();
-          steps_to_clean.push_back(step_ptr);
-        }
+        auto job_val = std::move(job_opt.value());
+        for (auto& [step_id, step] : job_val.step_map) {
+          CRANE_DEBUG("[Step #{}.{}] Removed orphaned step.", elem.job_id,
+                      step_id);
+          steps_to_clean.push_back(step.get());
+        }
+        job_to_clean.emplace_back(std::move(job_val));

This keeps ownership and lifetimes clear and avoids dereferencing freed objects.

src/Craned/Supervisor/TaskManager.cpp (1)

2205-2297: Make LaunchExecution_ / TaskFinish_ thread-safe and handle step-prep failures in EvGrpcExecuteTaskCb_

Two related issues around task launch:

  1. Step preparation errors are ignored

    EvGrpcExecuteTaskCb_ calls m_step_.Prepare(); and doesn’t inspect the return value. If Prepare() fails to write the script or X11 auth file, we still allocate a step cgroup and start per-task LaunchExecution_, leading to confusing downstream errors from Spawn() or exec. See the earlier comment on wiring script_path and handling Prepare() failures; the same fix (checking the return and short-circuiting with a TaskFinish_ for all tasks) should be applied here.

  2. TaskFinish_ is called from worker threads without synchronization

    In EvGrpcExecuteTaskCb_ you now launch each task in the thread pool:

    absl::Mutex thread_pool_mutex;
    CraneErrCode err = CraneErrCode::SUCCESS;
    std::latch latch(m_step_.task_ids.size());
    for (auto task_id : m_step_.task_ids) {
      auto* task = m_step_.GetTaskInstance(task_id);
      g_thread_pool->detach_task(
          [this, task, task_id, &thread_pool_mutex, &latch, &err] {
            auto task_err = LaunchExecution_(task);
            ...
            latch.count_down();
          });
    }

    LaunchExecution_ calls TaskFinish_ when Prepare() or Spawn() fails. That means TaskFinish_ (which mutates m_step_, m_step_.final_termination_status, and m_exec_id_task_id_map_) can now run on multiple worker threads concurrently, while the same code paths are also used from the uv loop (SIGCHLD / CRI event queues). None of this state is protected by a mutex, so there is a real data-race / UB risk under multi-task-per-step error scenarios.

    At minimum, you should ensure that any call to TaskFinish_ happens on the uv loop thread, not on worker threads. One approach is to have LaunchExecution_ only return an error code and enqueue a “task failed before exec” event into an existing queue (e.g., m_task_stopped_queue_) that is processed in the event loop, where TaskFinish_ is invoked.

    Alternatively, protect all reads/writes of m_step_’s internal maps and final_termination_status with a dedicated mutex to make the operations thread-safe.

Without one of these, multi-task steps with even a single failing task can trip subtle races in production.

Also applies to: 1801-1823

src/Craned/Common/CgroupManager.cpp (1)

296-335: Fix RE2 capture into std::optional in ParseIdsFromCgroupStr_ (won't compile).

RE2::FullMatch cannot write directly into std::optional<T> / std::optional<std::string> (lines 319‑325), so this overload will fail to compile. Capture into plain strings (or integral types) first, then populate CgroupStrParsedIds optionals.

Example fix:

-  static const LazyRE2 cg_pattern(cg_pattern_str.c_str());
-  std::optional<job_id_t> job_id;
-  std::optional<step_id_t> step_id;
-  std::optional<task_id_t> task_id;
-  std::optional<std::string> system_or_user;
-  CgroupStrParsedIds parsed_ids{};
-  if (RE2::FullMatch(cgroup_str, *cg_pattern, &job_id, &step_id,
-                     &system_or_user, &task_id)) {
-    std::get<CgConstant::KParsedJobIdIdx>(parsed_ids) = job_id;
-    std::get<CgConstant::KParsedStepIdIdx>(parsed_ids) = step_id;
-    std::get<CgConstant::KParsedSystemFlagIdx>(parsed_ids) =
-        system_or_user.has_value() && system_or_user.value() == "system";
-    std::get<CgConstant::KParsedTaskIdIdx>(parsed_ids) = task_id;
+  static const LazyRE2 cg_pattern(cg_pattern_str.c_str());
+  std::string job_id_str;
+  std::string step_id_str;
+  std::string system_or_user;
+  std::string task_id_str;
+
+  CgroupStrParsedIds parsed_ids{};
+  if (RE2::FullMatch(cgroup_str, *cg_pattern, &job_id_str, &step_id_str,
+                     &system_or_user, &task_id_str)) {
+    if (!job_id_str.empty()) {
+      std::get<CgConstant::KParsedJobIdIdx>(parsed_ids) =
+          static_cast<job_id_t>(std::stoull(job_id_str));
+    }
+    if (!step_id_str.empty()) {
+      std::get<CgConstant::KParsedStepIdIdx>(parsed_ids) =
+          static_cast<step_id_t>(std::stoull(step_id_str));
+    }
+    if (!system_or_user.empty()) {
+      std::get<CgConstant::KParsedSystemFlagIdx>(parsed_ids) =
+          (system_or_user == "system");
+    }
+    if (!task_id_str.empty()) {
+      std::get<CgConstant::KParsedTaskIdIdx>(parsed_ids) =
+          static_cast<task_id_t>(std::stoull(task_id_str));
+    }
     return parsed_ids;
   }
src/Craned/Core/Craned.cpp (1)

75-105: Fix BPF cleanup condition for step‑level entries (entries are logged but not removed).

In the V2 branch, when step_id.has_value() and the step is not in rn_step_from_ctld but its job is in rn_jobs_from_ctld, the code logs that it will erase the BPF entry but then hits if (rn_jobs_from_ctld.contains(job_id)) continue; and never deletes it. This leaves stale BPF map entries for vanished steps.

Consider restructuring as:

-    for (const auto& [ids, bpf_key_vec] : job_id_bpf_key_vec_map.value()) {
-      auto job_id = std::get<KParsedJobIdIdx>(ids).value();
-      auto step_id = std::get<KParsedStepIdIdx>(ids);
-      if (step_id.has_value()) {
-        if (rn_step_from_ctld.contains({job_id, step_id.value()})) {
-          continue;
-        } else {
-          CRANE_DEBUG("Erase bpf map entry for rn step {} {} not in Ctld.",
-                      job_id, step_id.value());
-        }
-      }
-      if (rn_jobs_from_ctld.contains(job_id))
-        continue;
-      else
-        CRANE_DEBUG("Erase bpf map entry for rn job {} not in Ctld.", job_id);
+    for (const auto& [ids, bpf_key_vec] : job_id_bpf_key_vec_map.value()) {
+      auto job_id = std::get<KParsedJobIdIdx>(ids).value();
+      auto step_id = std::get<KParsedStepIdIdx>(ids);
+
+      bool keep = false;
+      if (step_id.has_value()) {
+        if (rn_step_from_ctld.contains({job_id, step_id.value()})) {
+          keep = true;
+        } else {
+          CRANE_DEBUG(
+              "Erase bpf map entry for rn step {} {} not in Ctld.", job_id,
+              step_id.value());
+        }
+      } else {
+        if (rn_jobs_from_ctld.contains(job_id)) {
+          keep = true;
+        } else {
+          CRANE_DEBUG("Erase bpf map entry for rn job {} not in Ctld.",
+                      job_id);
+        }
+      }
+      if (keep) continue;

This keeps entries only when the owning step (if any) or job is present in Ctld and deletes them otherwise. Based on learnings.

♻️ Duplicate comments (6)
src/Craned/Core/StepInstance.cpp (2)

149-161: Make crane_cgroup precondition explicit in SpawnSupervisor

SpawnSupervisor unconditionally dereferences this->crane_cgroup for MigrateProcIn and CgroupPath(). Today LaunchStepMt_ always calls CreateCg() first, but any future caller that forgets to do so will crash here.

Consider guarding early and failing fast when the cgroup is missing:

CraneErrCode StepInstance::SpawnSupervisor(const EnvMap& job_env_map) {
+ if (!crane_cgroup) {
+   CRANE_ERROR("[Step #{}.{}] SpawnSupervisor called without step cgroup.",
+               job_id, step_id);
+   return CraneErrCode::ERR_CGROUP;
+ }
   ...
-  // Before exec, we need to make sure that the cgroup is ready.
-  if (!this->crane_cgroup->MigrateProcIn(child_pid)) {
+  // Before exec, we need to make sure that the cgroup is ready.
+  if (!this->crane_cgroup->MigrateProcIn(child_pid)) {

This keeps the invariant encoded in the code rather than only in call-site assumptions.


408-419: Guard against null supervisor_stub in ExecuteStepAsync

ExecuteStepAsync captures supervisor_stub and calls stub->ExecuteStep() on a background thread without any null-check. There is a real window where the step is in job->step_map but SpawnSupervisor has not yet successfully initialized the stub, leading to a null deref in the detached task.

Add a fast-fail guard and capture a local copy before scheduling:

void StepInstance::ExecuteStepAsync() {
-  this->GotNewStatus(StepStatus::Running);
-
-  g_thread_pool->detach_task([job_id = job_id, step_id = step_id,
-                              stub = supervisor_stub] {
-    auto code = stub->ExecuteStep();
+  if (!supervisor_stub) {
+    CRANE_ERROR("[Step #{}.{}] supervisor_stub is null, cannot execute step.",
+                job_id, step_id);
+    return;
+  }
+
+  this->GotNewStatus(StepStatus::Running);
+
+  auto stub = supervisor_stub;  // keep shared_ptr alive in background task
+  g_thread_pool->detach_task([job_id = job_id, step_id = step_id,
+                              stub = std::move(stub)] {
+    auto code = stub->ExecuteStep();

This matches the concern raised in the earlier review and removes a crash class.

src/Craned/Core/JobManager.cpp (2)

467-492: Add supervisor_stub null checks in new helpers

Both helpers assume the per-step supervisor is always alive:

  • ChangeStepTimelimit (lines 467–492) calls stub->ChangeTaskTimeLimit(...).
  • QuerySshStepEnvVariables (lines 494–510) calls stub->QueryStepEnv().

If the supervisor failed to spawn or has already exited (e.g., err_before_supv_start, or after cleanup), these will dereference a null shared_ptr and crash.

QuerySshStepEnvVariables was already called out in a previous review; a similar guard should be applied here. For example:

  auto& stub = step_it->second->supervisor_stub;
- auto err = stub->ChangeTaskTimeLimit(absl::Seconds(new_timelimit_sec));
+ if (!stub) {
+   CRANE_ERROR("[Step #{}.{}] Supervisor stub is null when changing time "
+               "limit.", job_id, step_id);
+   return std::unexpected{CraneErrCode::ERR_NON_EXISTENT};
+ }
+ auto err = stub->ChangeTaskTimeLimit(absl::Seconds(new_timelimit_sec));

and:

  auto& stub = step_it->second->supervisor_stub;
- return stub->QueryStepEnv();
+ if (!stub) {
+   CRANE_ERROR("[Step #{}.{}] Supervisor stub is null when querying SSH step "
+               "env.", job_id, step_id);
+   return std::unexpected{CraneErrCode::ERR_NON_EXISTENT};
+ }
+ return stub->QueryStepEnv();

This keeps the public API robust even when step startup fails.

Also applies to: 494-510


757-787: Fix iterator and stub safety in MigrateProcToCgroupOfJob

Two problems remain here (also raised previously):

  1. If the daemon step is missing, you log and then still dereference daemon_step_it:
if (daemon_step_it == job->step_map.end()) {
  CRANE_DEBUG(...);
}
auto& daemon_step = daemon_step_it->second;  // UB if end()
  1. If daemon_step->supervisor_stub is null, you log but still call stub->MigrateSshProcToCg(pid).

Both can crash; for a PAM-triggered path this is particularly painful. Early-return after logging in each case:

  auto daemon_step_it = job->step_map.find(kDaemonStepId);
  if (daemon_step_it == job->step_map.end()) {
    CRANE_DEBUG(
        "[Step #{}.{}] Daemon step not found when migrating pid {} to "
        "cgroup of job#{}.",
        job_id, kDaemonStepId, pid, job_id);
-  }
-  auto& daemon_step = daemon_step_it->second;
-  auto stub = daemon_step->supervisor_stub;
-  if (!stub) {
+    return false;
+  }
+  auto& daemon_step = daemon_step_it->second;
+  auto stub = daemon_step->supervisor_stub;
+  if (!stub) {
     CRANE_ERROR(
         "[Job #{}] Daemon step sSupervisor stub is null when migrating pid {} "
         "to "
         "cgroup of job#{}.",
         job_id, kDaemonStepId, pid, job_id);
+    return false;
   }

The remainder of the function (calling MigrateSshProcToCg and logging failures) can stay as-is.

src/Craned/Supervisor/TaskManager.cpp (1)

2307-2312: Fix promise usage in EvGrpcMigrateSshProcToCgroupCb_ to avoid double set_value

EvGrpcMigrateSshProcToCgroupCb_ still has the same promise-handling bug mentioned in the previous review:

while (m_grpc_migrate_ssh_proc_to_cgroup_queue_.try_dequeue(elem)) {
  auto& pid = elem.first;
  auto& prom = elem.second;
  if (!m_step_.IsDaemonStep()) {
    CRANE_ERROR("Trying to move pid {} to no daemon step", pid);
    prom.set_value(CraneErrCode::ERR_INVALID_PARAM);
  }
  if (!m_step_.step_user_cg) {
    ...
    if (!cg_expt.has_value()) {
      ...
      prom.set_value(CraneErrCode::ERR_CGROUP);
      continue;
    }
    m_step_.step_user_cg = std::move(cg_expt.value());
  }
  if (m_step_.step_user_cg->MigrateProcIn(pid)) {
    prom.set_value(CraneErrCode::SUCCESS);
  } else {
    prom.set_value(CraneErrCode::ERR_CGROUP);
  }
}

For non-daemon steps, you call prom.set_value(ERR_INVALID_PARAM) and then fall through and call set_value again with either SUCCESS or ERR_CGROUP, which is undefined behavior (std::future_error).

Add an early continue after the error, mirroring the pattern you already use for the cgroup-failure case:

  if (!m_step_.IsDaemonStep()) {
    CRANE_ERROR("Trying to move pid {} to no daemon step", pid);
    prom.set_value(CraneErrCode::ERR_INVALID_PARAM);
+    continue;
  }

This guarantees each promise is fulfilled exactly once per queue element.

Also applies to: 2314-2342

src/Craned/Supervisor/TaskManager.h (1)

421-429: Unify shutdown status on StepStatus end‑to‑end (type mismatch remains).

ShutdownSupervisorAsync still takes crane::grpc::TaskStatus with a default of StepStatus::Completed, and m_shutdown_status_queue_ stores crane::grpc::TaskStatus, but EvShutdownSupervisorCb_ (in TaskManager.cpp) passes the dequeued status into StepInstance::GotNewStatus, which expects StepStatus. This is the same mismatch flagged earlier and will either fail to compile or silently mix two unrelated enums if their underlying values diverge.

Suggested fix in the header:

-  // Shutdown supervisor asynchronously with given status, exit code and reason.
-  // Status change will be sent only if daemon step.
-  void ShutdownSupervisorAsync(
-      crane::grpc::TaskStatus new_status = StepStatus::Completed,
-      uint32_t exit_code = 0, std::string reason = "");
+  // Shutdown supervisor asynchronously with given status, exit code and reason.
+  // Status change will be sent only if daemon step.
+  void ShutdownSupervisorAsync(
+      StepStatus new_status = StepStatus::Completed,
+      uint32_t exit_code = 0, std::string reason = "");-  std::shared_ptr<uvw::async_handle> m_supervisor_finish_init_handle_;
-  ConcurrentQueue<std::tuple<crane::grpc::TaskStatus, uint32_t, std::string>>
-      m_shutdown_status_queue_;
+  std::shared_ptr<uvw::async_handle> m_supervisor_finish_init_handle_;
+  ConcurrentQueue<std::tuple<StepStatus, uint32_t, std::string>>
+      m_shutdown_status_queue_;

And in TaskManager.cpp:

-void TaskManager::ShutdownSupervisorAsync(crane::grpc::TaskStatus new_status,
+void TaskManager::ShutdownSupervisorAsync(StepStatus new_status,
                                           uint32_t exit_code,
                                           std::string reason) {
…
-void TaskManager::EvShutdownSupervisorCb_() {
-  std::tuple<crane::grpc::TaskStatus, uint32_t, std::string> final_status;
+void TaskManager::EvShutdownSupervisorCb_() {
+  std::tuple<StepStatus, uint32_t, std::string> final_status;

Also applies to: 536-539

🧹 Nitpick comments (10)
scripts/wipe_data.py (2)

92-104: Improve error logging for NI restoration.

As suggested by static analysis, use logging.exception instead of logging.error on line 103 to automatically include the traceback, and use bare raise on line 104.

Apply this diff:

         except Exception as e:
-            logger.error(f"Error restoring task ID to {var_db_path}: {e}")
-            raise e
+            logger.exception(f"Error restoring task ID to {var_db_path}")
+            raise

66-75: Optional: Validate preserved task ID data before restoration.

Consider adding validation to ensure the preserved next_task_id_val is well-formed before attempting restoration. This would prevent restoring potentially corrupted data.

Example validation:

if next_task_id_key in db:
    next_task_id_val = db[next_task_id_key]
    # Validate the data format (example: check if it's a valid integer representation)
    if len(next_task_id_val) > 0:
        logger.info(f"Found existing task ID data (len={len(next_task_id_val)})")
    else:
        logger.warning("Task ID data is empty, skipping preservation")
        next_task_id_val = None
src/Craned/Supervisor/CforedClient.h (1)

71-75: Unified FwdRequest path and updated task/X11 forward APIs are coherent

The new TaskProcessStop(..., exit_code, signaled), TaskOutPutForward(std::unique_ptr<char[]>&&, size_t), TaskX11OutPutForward(x11_local_id_t, std::unique_ptr<char[]>&&, size_t, bool) plus the IOFwdRequest/X11FwdRequest/TaskFinishStatus/FwdRequest variant and m_task_fwd_req_queue_ form a clean, extensible forwarding API. As long as every new StreamTaskIORequest::SupervisorRequestType added later also gets a matching std::variant alternative and switch arm in the implementation, this design should stay maintainable.

Also applies to: 106-110, 119-139

src/Craned/Supervisor/CforedClient.cpp (2)

407-457: FwdRequest dequeue loop is fine, but X11 metadata is currently unused and std::unreachable portability should be confirmed

The new CleanOutputQueueAndWriteToStreamThread_ correctly:

  • Drains m_task_fwd_req_queue_ until both m_stopped_ and the queue are empty.
  • Uses write_pending to serialize writes to the gRPC stream.
  • Serializes IO and exit-status messages from the FwdRequest variant.

Two follow-ups to consider:

  1. In the TASK_X11_OUTPUT case, x11_id, eof, and craned_id fields are commented out:
// payload->set_x11_id(x11_fwd_req.x11_id);
payload->set_msg(...);
// payload->set_eof(x11_fwd_req.eof);
// payload->set_craned_id(g_config.CranedIdOfThisNode);

Given you’re now tracking x11_local_id_t and X11FwdRequest::eof, it would be good either to wire these through fully or add a brief comment explaining why only msg is sent (e.g., if you intentionally support a single anonymous X11 channel).

  1. std::unreachable(); in the default: arm assumes C++23 (or a project-provided polyfill). If you still target an older standard on some build configurations, you may want to replace it with an existing assertion helper (e.g. CRANE_ASSERT(false && "unreachable")) or guard it with a macro.

624-640: X11 input path currently ignores remote x11_id/EOF and always targets a fixed local ID

In the TASK_X11_INPUT branch:

x11_local_id_t x11_id = 0;
// bool eof = reply.payload_task_x11_input_req().eof();
bool eof = false;
auto x11_fd_info_it = m_x11_fd_info_map_.find(x11_id);
...
x11_fd_info->x11_input_stopped =
    !WriteStringToFd_(*msg, x11_fd_info->fd, eof);
if (eof) {
  x11_fd_info->sock->close();
}
  • The local ID is hard-coded to 0, and the remote x11_id (if present in the proto) is ignored.
  • EOF from cfored is also ignored (forced to false), so only local EOF/error events on the uv side will ever close the X11 socket.

If the intent is to support only a single X11 channel per supervisor instance, this can work, but in that case it’s worth documenting the assumption and perhaps asserting m_x11_fd_info_map_.size() <= 1. If the protocol already carries a real x11_id and EOF flag, wiring those through would make the implementation future-proof and symmetric with the outgoing X11FwdRequest::x11_id/eof fields.

protos/Crane.proto (1)

754-813: TaskExitStatus messages and enums are consistent; minor duplication only

The new TASK_EXIT_STATUS enum value plus TaskExitStatusReply / TaskExitStatus messages and corresponding oneof entries in StreamCrunReply and StreamTaskIORequest are numerically and structurally consistent. This should carry per‑task exit information cleanly across both streams.

You do effectively have two near-identical messages (TaskExitStatusReply and TaskExitStatus) with the same fields; that’s fine for now, but if more APIs start sharing this shape it may be worth introducing a single top-level message and reusing it to avoid drift between copies.

Also applies to: 816-855

src/Craned/Common/CgroupManager.cpp (2)

1012-1031: Consider normalising CPU sets before writing cpuset.cpus.

The v1/v2 SetCpuBind implementations are functionally correct but join an std::unordered_set<uint32_t> directly, yielding non‑deterministic ordering (e.g. "2,0,3"). While the kernel generally accepts this, a sorted, compact form is more readable and conventional for cpusets.

For example:

-  // Convert cpu_set to comma-separated string (e.g., "0,1,2,5")
-  std::string cpu_list = absl::StrJoin(cpu_set, ",");
+  std::vector<uint32_t> cpus(cpu_set.begin(), cpu_set.end());
+  std::sort(cpus.begin(), cpus.end());
+  std::string cpu_list = absl::StrJoin(cpus, ",");

Apply to both v1 and v2 SetCpuBind.

Also applies to: 1317-1335


1087-1114: Verify process‑group semantics of KillAllProcesses.

Both v1 and v2 implementations call kill(-pids[i], signum) (process‑group kill) for each PID returned by cgroup_get_procs. This is fine if each task is in its own process group, but could unintentionally signal unrelated processes if groups are shared.

Please confirm all supervised tasks are started in dedicated process groups. If not, consider using kill(pids[i], signum) or wiring group creation explicitly where these cgroups are populated.

Also applies to: 1516-1544

src/Craned/Supervisor/TaskManager.h (2)

225-266: Ensure per‑task cgroups (m_task_cg) are cleaned up in all exit paths.

ITaskInstance now owns a std::unique_ptr<Common::CgroupInterface> m_task_cg, created in ITaskInstance::Prepare() (TaskManager.cpp). The header doesn’t show any RAII or explicit cleanup for this pointer.

Please verify that each implementation’s Cleanup() (or higher‑level StepInstance::CleanUp) always destroys the task cgroup (Destroy() and/or appropriate Craned‑side release) and that there are no paths where m_task_cg is left alive until process exit. If gaps exist, consider:

CraneErrCode ITaskInstance::Cleanup() {
  if (m_task_cg) {
    m_task_cg->KillAllProcesses(SIGKILL);
    m_task_cg->Destroy();
    m_task_cg.reset();
  }
  return CraneErrCode::SUCCESS;
}

and have concrete subclasses call ITaskInstance::Cleanup() as part of their override.


486-489: Async CheckStatusAsync pipeline is type‑consistent but relies on uvw loop responsiveness.

CheckStatusAsync uses std::promise<StepStatus> / std::future<StepStatus> and m_grpc_check_status_queue_ with EvGrpcCheckStatusCb_ setting the value. The types are aligned end‑to‑end; just ensure the uvw loop thread is always running while gRPC handlers are active so status_future.get() cannot block indefinitely.

If you ever see RPC stall risk, you could populate the reply synchronously from m_step_.GetStatus() in the gRPC thread and reserve the async callback for heavier work.

Also applies to: 571-573

📜 Review details

Configuration used: Path: .coderabbit.yaml

Review profile: CHILL

Plan: Pro

📥 Commits

Reviewing files that changed from the base of the PR and between 797e3d0 and 3b5c2f9.

📒 Files selected for processing (12)
  • protos/Crane.proto (6 hunks)
  • scripts/wipe_data.py (3 hunks)
  • src/Craned/Common/CgroupManager.cpp (18 hunks)
  • src/Craned/Common/CgroupManager.h (18 hunks)
  • src/Craned/Core/Craned.cpp (12 hunks)
  • src/Craned/Core/JobManager.cpp (17 hunks)
  • src/Craned/Core/StepInstance.cpp (1 hunks)
  • src/Craned/Supervisor/CforedClient.cpp (12 hunks)
  • src/Craned/Supervisor/CforedClient.h (6 hunks)
  • src/Craned/Supervisor/TaskManager.cpp (30 hunks)
  • src/Craned/Supervisor/TaskManager.h (16 hunks)
  • src/Utilities/PublicHeader/include/crane/PublicHeader.h (1 hunks)
🧰 Additional context used
🧠 Learnings (25)
📓 Common learnings
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Craned/SupervisorKeeper.h:32-32
Timestamp: 2025-05-26T11:04:30.580Z
Learning: The Supervisor component in the Crane system is designed to manage only one task per instance. The task specification is provided to the Supervisor during startup by Craned (likely through InitSupervisorRequest), so the ExecuteTask() method doesn't need to accept task parameters since the Supervisor already knows which task to execute.
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Supervisor/SupervisorPublicDefs.h:32-37
Timestamp: 2025-05-26T11:06:28.796Z
Learning: The user L-Xiafeng prefers to defer refactoring duplicate definitions until they become a larger pattern in the codebase, rather than addressing individual instances immediately.
📚 Learning: 2025-05-25T04:08:03.273Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Supervisor/Supervisor.cpp:68-69
Timestamp: 2025-05-25T04:08:03.273Z
Learning: In the Crane supervisor component (src/Craned/Supervisor/), the supervisor process communicates with Craned through STDOUT using protobuf messages. The supervisor must not send any information to STDOUT before sending the "ready" message to Craned, as this would interfere with the inter-process communication protocol. Therefore, adding logging statements that might write to STDOUT before the ready message is sent could break the communication.

Applied to files:

  • src/Craned/Core/StepInstance.cpp
  • src/Craned/Core/Craned.cpp
  • src/Craned/Core/JobManager.cpp
  • src/Craned/Supervisor/TaskManager.h
📚 Learning: 2025-04-02T10:11:33.562Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 461
File: src/Craned/CgroupManager.cpp:685-685
Timestamp: 2025-04-02T10:11:33.562Z
Learning: In the CgroupManager's GetJobBpfMapCgroupsV2 method, the developer has confirmed that cg_ino_job_id_map will always contain the key->cgroup_id element, making the CRANE_ASSERT check appropriate rather than requiring additional error handling.

Applied to files:

  • src/Craned/Core/StepInstance.cpp
  • src/Craned/Core/Craned.cpp
  • src/Craned/Core/JobManager.cpp
  • src/Craned/Common/CgroupManager.cpp
  • src/Craned/Common/CgroupManager.h
📚 Learning: 2025-05-08T07:39:29.207Z
Learnt from: huerni
Repo: PKUHPC/CraneSched PR: 469
File: src/Craned/TaskManager.cpp:1162-1175
Timestamp: 2025-05-08T07:39:29.207Z
Learning: In CraneSched, when a task fails in the LaunchTaskInstanceMt_ method after PMIx registration, ActivateTaskStatusChangeAsync_ does not automatically release PMIx resources or cgroup resources. Explicit cleanup is required by calling g_pmix_server->DeregisterTask() and g_cg_mgr->ReleaseCgroupByTaskIdOnly() before returning from the function.

Applied to files:

  • src/Craned/Core/StepInstance.cpp
  • src/Craned/Core/Craned.cpp
  • src/Craned/Core/JobManager.cpp
  • src/Craned/Supervisor/TaskManager.cpp
  • src/Craned/Supervisor/TaskManager.h
📚 Learning: 2025-05-02T07:12:46.896Z
Learnt from: RileyWen
Repo: PKUHPC/CraneSched PR: 475
File: src/Craned/CranedServer.h:0-0
Timestamp: 2025-05-02T07:12:46.896Z
Learning: The initialization of `m_supervisor_recovered_` to `true` in the CranedServer class is intentional despite the comment saying "When supervisor ready, init with false". This is temporary until the supervisor functionality is fully implemented.

Applied to files:

  • src/Craned/Core/StepInstance.cpp
  • src/Craned/Core/Craned.cpp
  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-05-08T07:39:29.207Z
Learnt from: huerni
Repo: PKUHPC/CraneSched PR: 469
File: src/Craned/TaskManager.cpp:1162-1175
Timestamp: 2025-05-08T07:39:29.207Z
Learning: In CraneSched, when a task fails in the LaunchTaskInstanceMt_ method, ActivateTaskStatusChangeAsync_ does not automatically release PMIx resources or cgroup. Explicit cleanup is required by calling g_pmix_server->DeregisterTask() and g_cg_mgr->ReleaseCgroupByTaskIdOnly().

Applied to files:

  • src/Craned/Core/StepInstance.cpp
  • src/Craned/Core/Craned.cpp
  • src/Craned/Core/JobManager.cpp
  • src/Craned/Supervisor/TaskManager.cpp
  • src/Craned/Supervisor/TaskManager.h
📚 Learning: 2025-06-30T08:43:44.470Z
Learnt from: huerni
Repo: PKUHPC/CraneSched PR: 537
File: src/Craned/TaskManager.cpp:145-147
Timestamp: 2025-06-30T08:43:44.470Z
Learning: In the CraneSched codebase, src/Craned/Craned.cpp guarantees that g_config.CranedRes.contains(g_config.CranedIdOfThisNode) through explicit validation during startup. The code checks if g_config.CranedRes.contains(g_config.Hostname) and exits the process if not found, then sets g_config.CranedIdOfThisNode = g_config.Hostname. TaskManager constructor is called after this validation, so g_config.CranedRes[g_config.CranedIdOfThisNode] is guaranteed to be valid.

Applied to files:

  • src/Craned/Core/StepInstance.cpp
  • src/Craned/Core/Craned.cpp
  • src/Craned/Core/JobManager.cpp
  • src/Craned/Supervisor/CforedClient.h
  • src/Craned/Supervisor/TaskManager.cpp
  • src/Craned/Supervisor/TaskManager.h
📚 Learning: 2025-05-08T07:38:42.362Z
Learnt from: huerni
Repo: PKUHPC/CraneSched PR: 469
File: src/Craned/TaskManager.cpp:0-0
Timestamp: 2025-05-08T07:38:42.362Z
Learning: In CraneSched's PMIx integration, the `g_pmix_server->SetupFork()` function must be called in the child process after fork() and before exec() to properly set up the PMIx environment variables.

Applied to files:

  • src/Craned/Core/StepInstance.cpp
  • src/Craned/Supervisor/TaskManager.cpp
  • src/Craned/Supervisor/TaskManager.h
📚 Learning: 2025-05-25T04:11:27.740Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Supervisor/TaskManager.cpp:396-396
Timestamp: 2025-05-25T04:11:27.740Z
Learning: In TaskManager.cpp, GetCrunMeta() calls don't need null checks because they're only called in contexts where the task is guaranteed to be a CRUN task (e.g., SetupChildProcessCrunX11_ is only called when step->IsCrun() && x11() conditions are met), ensuring the metadata will always be valid.

Applied to files:

  • src/Craned/Core/StepInstance.cpp
  • src/Craned/Core/JobManager.cpp
  • src/Craned/Supervisor/TaskManager.cpp
  • src/Craned/Supervisor/TaskManager.h
📚 Learning: 2025-05-09T01:54:21.256Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 496
File: src/CraneCtld/CranedKeeper.cpp:51-53
Timestamp: 2025-05-09T01:54:21.256Z
Learning: The ConfigureCraned function in src/CraneCtld/CranedKeeper.cpp is called from a thread pool, so there's no need to worry about it blocking the gRPC completion queue thread.

Applied to files:

  • src/Craned/Core/StepInstance.cpp
  • src/Craned/Core/Craned.cpp
  • src/Craned/Core/JobManager.cpp
  • src/Craned/Supervisor/CforedClient.cpp
  • src/Craned/Supervisor/CforedClient.h
  • src/Craned/Supervisor/TaskManager.cpp
  • src/Craned/Supervisor/TaskManager.h
📚 Learning: 2025-05-25T04:11:50.268Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Supervisor/TaskManager.cpp:220-220
Timestamp: 2025-05-25T04:11:50.268Z
Learning: In TaskManager.cpp, when step->IsCrun() is checked before dynamic_cast to CrunMetaInExecution*, the cast is guaranteed to succeed due to the program logic ensuring the correct meta type is used for Crun tasks. Null checks for these dynamic_cast operations are unnecessary in this context.

Applied to files:

  • src/Craned/Core/StepInstance.cpp
  • src/Craned/Core/JobManager.cpp
  • src/Craned/Supervisor/TaskManager.cpp
  • src/Craned/Supervisor/TaskManager.h
📚 Learning: 2025-05-26T11:04:30.580Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Craned/SupervisorKeeper.h:32-32
Timestamp: 2025-05-26T11:04:30.580Z
Learning: The Supervisor component in the Crane system is designed to manage only one task per instance. The task specification is provided to the Supervisor during startup by Craned (likely through InitSupervisorRequest), so the ExecuteTask() method doesn't need to accept task parameters since the Supervisor already knows which task to execute.

Applied to files:

  • src/Craned/Core/StepInstance.cpp
  • protos/Crane.proto
  • src/Craned/Core/Craned.cpp
  • src/Craned/Core/JobManager.cpp
  • src/Craned/Supervisor/TaskManager.cpp
  • src/Craned/Supervisor/TaskManager.h
📚 Learning: 2025-05-02T07:06:36.103Z
Learnt from: RileyWen
Repo: PKUHPC/CraneSched PR: 475
File: src/CraneCtld/CtldGrpcServer.cpp:113-118
Timestamp: 2025-05-02T07:06:36.103Z
Learning: In CraneSched, gRPC methods should generally return Status::OK even when handling error conditions, as non-OK statuses cause the connection to terminate. Error information should be communicated within the RPC response payload instead.

Applied to files:

  • src/Craned/Core/StepInstance.cpp
  • protos/Crane.proto
  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-04-02T09:52:59.318Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 461
File: src/Craned/CgroupManager.cpp:697-701
Timestamp: 2025-04-02T09:52:59.318Z
Learning: When using bpf_map__get_next_key function, memory must be properly allocated (e.g., with std::make_unique<BpfKey>()) before passing the pointer to the function, as it writes the key to the provided memory address.

Applied to files:

  • src/Craned/Core/Craned.cpp
  • src/Craned/Common/CgroupManager.cpp
📚 Learning: 2025-04-27T11:52:31.017Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 475
File: src/CraneCtld/CranedKeeper.cpp:40-62
Timestamp: 2025-04-27T11:52:31.017Z
Learning: In the CraneSched system, retry of configuration RPC is architecturally driven by the Craned's notification system rather than explicit retry code within the ConfigureCraned method. When Configure RPC fails, Craned returns to a notification state and sends new Notify messages which trigger new configuration attempts.

Applied to files:

  • src/Craned/Core/Craned.cpp
📚 Learning: 2025-05-26T11:04:56.055Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Craned/SupervisorKeeper.h:72-72
Timestamp: 2025-05-26T11:04:56.055Z
Learning: The CraneSched project prefers using global variables (like `g_supervisor_keeper`) over dependency injection patterns. The team does not follow dependency injection approaches for managing singleton instances.

Applied to files:

  • src/Craned/Core/Craned.cpp
📚 Learning: 2025-06-23T07:53:30.513Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 520
File: src/Craned/CranedServer.cpp:50-51
Timestamp: 2025-06-23T07:53:30.513Z
Learning: In the CraneSched codebase, `g_ctld_client` is a guaranteed global variable that is always initialized before any gRPC service methods are called, so null pointer checks are not necessary when calling methods on it.

Applied to files:

  • src/Craned/Core/Craned.cpp
📚 Learning: 2025-08-14T02:56:35.503Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 587
File: src/Craned/Supervisor/CforedClient.cpp:449-454
Timestamp: 2025-08-14T02:56:35.503Z
Learning: In CforedClient::AsyncSendRecvThread_(), the guard `if (state <= State::Registering) { continue; }` in the TIMEOUT branch only prevents premature cleanup when stopping before registration completes, but it doesn't block normal gRPC event processing. The completion queue will still deliver Prepare/Write/Read events that advance the state machine normally.

Applied to files:

  • src/Craned/Supervisor/CforedClient.cpp
📚 Learning: 2025-04-02T09:30:13.014Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 461
File: src/Craned/JobManager.cpp:141-149
Timestamp: 2025-04-02T09:30:13.014Z
Learning: In JobManager, if a uid exists in m_uid_to_job_ids_map_, its corresponding task_ids set is guaranteed to be non-empty due to the invariant maintained in AllocJobs and FreeJobs methods.

Applied to files:

  • src/Craned/Common/CgroupManager.cpp
📚 Learning: 2025-04-18T02:26:16.113Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 351
File: src/CraneCtld/CranedMetaContainer.cpp:248-264
Timestamp: 2025-04-18T02:26:16.113Z
Learning: The resource class in CraneSched includes assertions in its operator overloads (particularly in operator-=) that verify resources being subtracted are less than or equal to available resources, ensuring no negative values can occur during resource allocation or deallocation operations.

Applied to files:

  • src/Craned/Common/CgroupManager.cpp
📚 Learning: 2025-06-07T10:47:59.071Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 520
File: src/CraneCtld/CranedKeeper.cpp:416-417
Timestamp: 2025-06-07T10:47:59.071Z
Learning: In src/CraneCtld/CranedKeeper.h, the m_shutting_down_ member in CranedStub class is declared as std::atomic_bool, making it thread-safe for concurrent access without additional synchronization.

Applied to files:

  • src/Craned/Supervisor/CforedClient.h
  • src/Craned/Supervisor/TaskManager.h
📚 Learning: 2025-05-02T07:05:26.012Z
Learnt from: RileyWen
Repo: PKUHPC/CraneSched PR: 475
File: src/CraneCtld/CranedKeeper.cpp:601-602
Timestamp: 2025-05-02T07:05:26.012Z
Learning: In the CraneCtld codebase, the variables m_disconnected_ and m_registered_ in CranedStub class are already defined as std::atomic_bool, making them thread-safe for concurrent access without additional synchronization.

Applied to files:

  • src/Craned/Supervisor/CforedClient.h
  • src/Craned/Supervisor/TaskManager.h
📚 Learning: 2025-07-01T08:00:05.383Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Supervisor/TaskManager.cpp:1-1809
Timestamp: 2025-07-01T08:00:05.383Z
Learning: In TaskManager.cpp, the security model relies on administrator-configured command templates in ParseOCICmdPattern_ and system-provided usernames in ParseFilePathPattern_, with file permissions using the user's own uid/gid for privilege isolation. The user L-Xiafeng considers this security boundary sufficient and chooses not to fix job name-related path issues at this location.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
  • src/Craned/Common/CgroupManager.h
  • src/Craned/Supervisor/TaskManager.h
📚 Learning: 2025-05-09T02:16:56.723Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 496
File: src/Craned/TaskManager.cpp:1498-1506
Timestamp: 2025-05-09T02:16:56.723Z
Learning: The `QueryRunningTasksAsync()` method in TaskManager.cpp is designed to never be called from inside the event loop thread, so there's no risk of deadlock with the synchronous `res.get()` call.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-05-08T09:35:39.809Z
Learnt from: huerni
Repo: PKUHPC/CraneSched PR: 469
File: src/Craned/Pmix/PmixCollRing.cpp:0-0
Timestamp: 2025-05-08T09:35:39.809Z
Learning: In the PMIx implementation for CraneSched, objects referenced in asynchronous gRPC callbacks (like `coll_ring_ctx`) remain valid as long as the parent object (`this`) is not destroyed. The `Coll` class uses shared_ptr management to ensure its lifetime extends through all pending callbacks.

Applied to files:

  • src/Craned/Supervisor/TaskManager.h
🧬 Code graph analysis (10)
scripts/wipe_data.py (1)
test/CraneCtld/EmbeddedDbClientTest.cpp (1)
  • exists (33-40)
src/Craned/Core/StepInstance.cpp (2)
src/Craned/Supervisor/TaskManager.cpp (4)
  • CleanUp (112-154)
  • CleanUp (112-112)
  • GotNewStatus (227-302)
  • GotNewStatus (227-227)
src/Craned/Core/CranedPublicDefs.h (1)
  • g_config (156-156)
src/Craned/Core/Craned.cpp (4)
src/Craned/Core/StepInstance.cpp (2)
  • StepInstance (25-30)
  • StepInstance (32-40)
src/Craned/Core/SupervisorStub.h (2)
  • Craned (26-62)
  • SupervisorStub (28-61)
src/Craned/Common/CgroupManager.cpp (12)
  • GetIdsFromCgroupV1_ (554-578)
  • GetIdsFromCgroupV1_ (554-555)
  • GetIdsFromCgroupV2_ (580-600)
  • GetIdsFromCgroupV2_ (580-581)
  • AllocateAndGetCgroup (494-552)
  • AllocateAndGetCgroup (495-497)
  • CgroupStrByStepId (283-288)
  • CgroupStrByStepId (283-284)
  • CgroupStrByJobId (279-281)
  • CgroupStrByJobId (279-279)
  • CgroupStrByParsedIds (296-308)
  • CgroupStrByParsedIds (296-296)
src/Craned/Core/SupervisorStub.cpp (4)
  • SupervisorStub (25-29)
  • SupervisorStub (31-33)
  • InitAndGetRecoveredMap (35-92)
  • InitAndGetRecoveredMap (37-37)
src/Craned/Core/JobManager.cpp (1)
src/Craned/Core/CranedServer.cpp (2)
  • QuerySshStepEnvVariables (297-318)
  • QuerySshStepEnvVariables (297-300)
src/Craned/Supervisor/CforedClient.cpp (2)
src/Craned/Supervisor/CforedClient.h (3)
  • next_x11_id_ (150-150)
  • exit_code (131-131)
  • signaled (132-133)
src/Utilities/PublicHeader/String.cpp (2)
  • SetCurrentThreadName (354-362)
  • SetCurrentThreadName (354-354)
src/Craned/Common/CgroupManager.cpp (1)
src/Craned/Common/CgroupManager.h (2)
  • handle (425-426)
  • Internal (131-182)
src/Craned/Supervisor/CforedClient.h (2)
src/Craned/Supervisor/CforedClient.cpp (8)
  • InitUvX11FwdHandler (109-115)
  • InitUvX11FwdHandler (109-109)
  • TaskProcessStop (712-725)
  • TaskProcessStop (712-713)
  • TaskOutPutForward (736-743)
  • TaskOutPutForward (736-737)
  • TaskX11OutPutForward (745-756)
  • TaskX11OutPutForward (745-747)
src/Craned/Supervisor/SupervisorPublicDefs.h (1)
  • exit_code (37-37)
src/Craned/Supervisor/TaskManager.cpp (6)
src/Craned/Common/CgroupManager.h (1)
  • CgroupManager (532-640)
src/Craned/Supervisor/TaskManager.h (9)
  • uid (56-56)
  • InitCriClient (150-155)
  • x11 (61-61)
  • x11_fwd (62-62)
  • StopCforedClient (147-147)
  • StopCriClient (160-160)
  • job_id (52-52)
  • step_id (53-53)
  • DelTerminationTimer_ (455-461)
src/Utilities/PublicHeader/include/crane/PasswordEntry.h (1)
  • pwd (105-105)
src/Utilities/PublicHeader/OS.cpp (4)
  • CreateFoldersForFileEx (109-148)
  • CreateFoldersForFileEx (109-110)
  • DeleteFile (64-71)
  • DeleteFile (64-64)
src/Craned/Core/StepInstance.cpp (5)
  • ec (276-276)
  • ec (295-295)
  • ec (432-432)
  • GotNewStatus (334-406)
  • GotNewStatus (334-334)
src/Craned/Common/CgroupManager.cpp (6)
  • AllocateAndGetCgroup (494-552)
  • AllocateAndGetCgroup (495-497)
  • CgroupStrByTaskId (290-294)
  • CgroupStrByTaskId (290-291)
  • CgroupStrByStepId (283-288)
  • CgroupStrByStepId (283-284)
src/Craned/Common/CgroupManager.h (1)
src/Craned/Common/CgroupManager.cpp (19)
  • SetCpuBind (1012-1031)
  • SetCpuBind (1012-1012)
  • SetCpuBind (1317-1335)
  • SetCpuBind (1317-1317)
  • KillAllProcesses (1087-1114)
  • KillAllProcesses (1087-1087)
  • KillAllProcesses (1516-1544)
  • KillAllProcesses (1516-1516)
  • RecoverFromResInNode (1442-1492)
  • RecoverFromResInNode (1442-1443)
  • CgroupStrByParsedIds (296-308)
  • CgroupStrByParsedIds (296-296)
  • CgroupStrParsedIds (636-676)
  • GetIdsFromCgroupV1_ (554-578)
  • GetIdsFromCgroupV1_ (554-555)
  • GetIdsFromCgroupV2_ (580-600)
  • GetIdsFromCgroupV2_ (580-581)
  • GetCgInoJobIdMapCgroupV2_ (602-632)
  • GetCgInoJobIdMapCgroupV2_ (603-604)
src/Craned/Supervisor/TaskManager.h (3)
src/Craned/Common/CgroupManager.h (1)
  • CgroupInterface (389-421)
src/Craned/Core/StepInstance.cpp (6)
  • StepInstance (25-30)
  • StepInstance (32-40)
  • CleanUp (42-78)
  • CleanUp (42-42)
  • GotNewStatus (334-406)
  • GotNewStatus (334-334)
src/Craned/Supervisor/TaskManager.cpp (37)
  • Prepare (43-110)
  • Prepare (43-43)
  • Prepare (375-389)
  • Prepare (375-375)
  • Prepare (789-873)
  • Prepare (789-789)
  • Prepare (1303-1345)
  • Prepare (1303-1303)
  • CleanUp (112-154)
  • CleanUp (112-112)
  • GotNewStatus (227-302)
  • GotNewStatus (227-227)
  • InitOomBaseline (322-343)
  • InitOomBaseline (322-322)
  • EvaluateOomOnExit (345-373)
  • EvaluateOomOnExit (345-345)
  • ProcInstance (432-440)
  • SetupCrunFwdAtParent_ (540-559)
  • SetupCrunFwdAtParent_ (540-540)
  • SupervisorFinishInit (1721-1723)
  • SupervisorFinishInit (1721-1721)
  • Wait (1725-1727)
  • Wait (1725-1725)
  • ShutdownSupervisorAsync (1729-1736)
  • ShutdownSupervisorAsync (1729-1731)
  • CheckStatusAsync (1857-1868)
  • CheckStatusAsync (1857-1858)
  • MigrateSshProcToCgroupAsync (1870-1880)
  • MigrateSshProcToCgroupAsync (1870-1870)
  • EvSupervisorFinishInitCb_ (1882-1890)
  • EvSupervisorFinishInitCb_ (1882-1882)
  • EvShutdownSupervisorCb_ (1892-1913)
  • EvShutdownSupervisorCb_ (1892-1892)
  • EvGrpcCheckStatusCb_ (2307-2312)
  • EvGrpcCheckStatusCb_ (2307-2307)
  • EvGrpcMigrateSshProcToCgroupCb_ (2314-2342)
  • EvGrpcMigrateSshProcToCgroupCb_ (2314-2314)
🪛 Ruff (0.14.5)
scripts/wipe_data.py

77-77: Do not catch blind exception: Exception

(BLE001)


103-103: Use logging.exception instead of logging.error

Replace with exception

(TRY400)


104-104: Use raise without specifying exception name

Remove exception name

(TRY201)

Copy link
Contributor

@coderabbitai coderabbitai bot left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Actionable comments posted: 1

Caution

Some comments are outside the diff and can’t be posted inline due to platform limitations.

⚠️ Outside diff range comments (1)
src/Craned/Supervisor/TaskManager.cpp (1)

1801-1824: Critical data race confirmed on m_step_.m_task_map_ and other shared state

The review comment's core findings are verified:

  1. Concurrent access to m_task_map_ is confirmed:

    • LaunchExecution_ is called from worker threads (line 2277) spawned at lines 2275–2292
    • LaunchExecution_ calls TaskFinish_() on failures (lines 1806–1809, 1816–1819)
    • TaskFinish_() calls m_step_.RemoveTaskInstance(task_id) which executes m_task_map_.find() and m_task_map_.erase() (lines 314, 319)
    • EvCleanTaskStopQueueCb_() (uvw event loop thread) also calls TaskFinish_() at multiple sites (lines 2022–2097)
    • Multiple worker threads executing TaskFinish_() concurrently with the uvw event loop = unprotected concurrent access
  2. No synchronization protects m_task_map_:

    • m_task_map_ is declared as std::unordered_map<task_id_t, std::unique_ptr<ITaskInstance>> (TaskManager.h, line 194)
    • No std::mutex found protecting m_step_ or m_task_map_ in TaskManager.cpp
    • The thread_pool_mutex at line 2271 is locally scoped and only guards err and m_exec_id_task_id_map_, not m_step_
  3. Unprotected state mutations:

    • m_step_.oom_baseline_inited = false (line 1756) executed without synchronization
    • Concurrent unordered_map operations (find/erase/emplace) without locks = undefined behavior

Fix required: Guard all m_step_ mutations with a dedicated mutex, or redirect task failures through a queue processed by the uvw event loop thread.

♻️ Duplicate comments (5)
src/Craned/Supervisor/TaskManager.cpp (5)

1303-1344: Process Prepare correctly layers per-task cgroup and script executable, but still depends on upstream StepInstance::Prepare

ProcInstance::Prepare now:

  • Calls ITaskInstance::Prepare() first, creating the per-task cgroup.
  • Computes batch output/error paths, then sets m_executable_ to script_path (or interpreter + script_path for batch with interpreter).

That’s the right layering. The remaining risk is that if StepInstance::Prepare failed before writing the script or setting script_path (e.g. fopen failed), script_path will be empty and you’ll end up executing bash -c "", which “succeeds” without running the intended script.

Once EvGrpcExecuteTaskCb_ checks and respects the return value of m_step_.Prepare(), that edge case is resolved.


2207-2249: Respect StepInstance::Prepare return value to avoid running with a half-prepared step

EvGrpcExecuteTaskCb_ now calls:

m_step_.pwd.Init(m_step_.uid);
...
m_step_.Prepare();

but ignores the CraneErrCode returned by StepInstance::Prepare().

Given Prepare() can fail when:

  • Writing the script file (fopen failure),
  • Creating X11 directories/files (CreateFoldersForFileEx, mkstemp, fchown),

ignoring its return means you may:

  • Attempt to execute an empty or missing script (empty script_path).
  • Later call SetupChildProcCrunX11_() which assumes x11_meta.has_value() even when Prepare() bailed before assigning it.

A safer pattern (similar to the earlier review suggestion) is:

-    m_step_.Prepare();
+    auto prep_err = m_step_.Prepare();
+    if (prep_err != CraneErrCode::SUCCESS) {
+      for (auto task_id : m_step_.task_ids) {
+        TaskFinish_(task_id, crane::grpc::TaskStatus::Failed,
+                    ExitCode::EC_FILE_NOT_FOUND,
+                    "Failed to prepare step script and X11 state");
+      }
+      elem.ok_prom.set_value(prep_err);
+      continue;
+    }

This keeps step-level invariants intact and avoids confusing downstream failures.


378-392: Guard task_res_map().at(task_id) to avoid crashing when scheduler hasn’t filled task resources yet

ITaskInstance::Prepare unconditionally does:

m_parent_step_inst_->GetStep().task_res_map().at(task_id)

Per the PR objectives, the scheduler-side population of task_res_map is still an outstanding item, so .at(task_id) can throw std::out_of_range and crash the supervisor if the map is empty or missing this key.

Until the scheduler change is guaranteed, prefer a safe lookup and fail with a CraneErrCode:

-  auto cg_expt = CgroupManager::AllocateAndGetCgroup(
-      CgroupManager::CgroupStrByTaskId(g_config.JobId, g_config.StepId,
-                                       task_id),
-      m_parent_step_inst_->GetStep().task_res_map().at(task_id), false);
+  const auto& step = m_parent_step_inst_->GetStep();
+  const auto& task_res_map = step.task_res_map();
+  auto it = task_res_map.find(task_id);
+  if (it == task_res_map.end()) {
+    CRANE_ERROR("[Step #{}.{}] No task_res entry for task #{}.", g_config.JobId,
+                g_config.StepId, task_id);
+    return CraneErrCode::ERR_CGROUP;
+  }
+  auto cg_expt = CgroupManager::AllocateAndGetCgroup(
+      CgroupManager::CgroupStrByTaskId(g_config.JobId, g_config.StepId,
+                                       task_id),
+      it->second, false);

This matches the earlier review feedback and prevents a misconfigured Ctld from taking down the supervisor.


1764-1775: Keep final_reason_on_termination aligned with the chosen final status

You now set final_reason_on_termination when a larger exit_code is seen:

if (status.max_exit_code < exit_code) {
  status.max_exit_code = exit_code;
  status.final_status_on_termination = new_status;
  status.final_reason_on_termination = reason.value_or("");
}

Then you override final_status_on_termination whenever new_status != Completed, but without updating the reason in that branch. This can leave a mismatch where the final status corresponds to the last error, but the reason still reflects an earlier, “dominant-exit-code” failure.

To keep status and reason coherent:

  if (status.max_exit_code < exit_code) {
    status.max_exit_code = exit_code;
    status.final_status_on_termination = new_status;
    status.final_reason_on_termination = reason.value_or("");
  }
  // Error status has higher priority than success status.
-  if (new_status != StepStatus::Completed)
-    status.final_status_on_termination = new_status;
+  if (new_status != StepStatus::Completed) {
+    status.final_status_on_termination = new_status;
+    if (reason.has_value())
+      status.final_reason_on_termination = *reason;
+  }

This mirrors the earlier review suggestion and ensures the step-level reason matches the reported final status.


1857-1868: Fix double set_value and invalid cgroup allocation for non-daemon steps in EvGrpcMigrateSshProcToCgroupCb_

In EvGrpcMigrateSshProcToCgroupCb_:

if (!m_step_.IsDaemonStep()) {
  CRANE_ERROR("Trying to move pid {} to no daemon step", pid);
  prom.set_value(CraneErrCode::ERR_INVALID_PARAM);
}
if (!m_step_.step_user_cg) {
  auto cg_expt = CgroupManager::AllocateAndGetCgroup(...);
  ...
  prom.set_value(...);
  continue;
}
...
prom.set_value(...);

Two problems:

  • For non-daemon steps, you call prom.set_value(ERR_INVALID_PARAM) but then fall through and may call prom.set_value again, which triggers std::future_error on the waiting side and can crash.
  • You also attempt to allocate a step user cgroup for non-daemon steps after logging the error, which is contradictory.

Add an early continue after the invalid-param case:

    if (!m_step_.IsDaemonStep()) {
      CRANE_ERROR("Trying to move pid {} to no daemon step", pid);
      prom.set_value(CraneErrCode::ERR_INVALID_PARAM);
+      continue;
    }

This matches the earlier review feedback and prevents double-fulfilling the promise or touching cgroups for non-daemon steps.

Also applies to: 2314-2342

🧹 Nitpick comments (2)
src/Craned/Supervisor/TaskManager.cpp (1)

1454-1460: Abort on MigrateProcIn failure is harsh; consider graceful failure

In the child branch of ProcInstance::Spawn:

auto success = m_task_cg->MigrateProcIn(getpid());
if (!success) {
  fmt::print(stderr, "[Subprocess] MigrateProcIn returned {}\n", success);
  std::abort();
}

Treating a cgroup migration failure as an immediate abort() will kill the supervisor process and any other tasks sharing it. If MigrateProcIn can fail due to transient kernel or configuration issues, it may be better to:

  • Print the error.
  • Exit the child with a distinct exit code.
  • Let the parent detect the failure (via SIGCHLD) and report a clean TaskStatus::Failed/ERR_CGROUP.

If you consider MigrateProcIn an invariant that “must not fail”, leaving abort() is acceptable but brittle; at least a comment stating that assumption would help future maintainers.

src/Craned/Supervisor/CforedClient.cpp (1)

410-449: X11 forward requests currently drop x11_id and eof fields

In CleanOutputQueueAndWriteToStreamThread_, the TASK_X11_OUTPUT case builds the payload as:

auto* payload = request.mutable_payload_task_x11_output_req();
// payload->set_x11_id(x11_fwd_req.x11_id);
payload->set_msg(x11_fwd_req.data.get(), x11_fwd_req.len);
// payload->set_eof(x11_fwd_req.eof);
// payload->set_craned_id(g_config.CranedIdOfThisNode);

Given TaskX11OutPutForward enqueues an X11FwdRequest with x11_id and eof, commenting out these fields means:

  • cfored cannot distinguish different X11 connections, and
  • EOF information is lost, so the remote side may not know when to tear down a connection.

If the proto now includes these fields and cfored expects them, you probably want to set them here; if not, consider removing them from X11FwdRequest to avoid dead code/confusion.

📜 Review details

Configuration used: Path: .coderabbit.yaml

Review profile: CHILL

Plan: Pro

📥 Commits

Reviewing files that changed from the base of the PR and between 3b5c2f9 and 9460464.

📒 Files selected for processing (2)
  • src/Craned/Supervisor/CforedClient.cpp (13 hunks)
  • src/Craned/Supervisor/TaskManager.cpp (30 hunks)
🧰 Additional context used
🧠 Learnings (16)
📓 Common learnings
Learnt from: huerni
Repo: PKUHPC/CraneSched PR: 469
File: src/Craned/TaskManager.cpp:1162-1175
Timestamp: 2025-05-08T07:39:29.207Z
Learning: In CraneSched, when a task fails in the LaunchTaskInstanceMt_ method after PMIx registration, ActivateTaskStatusChangeAsync_ does not automatically release PMIx resources or cgroup resources. Explicit cleanup is required by calling g_pmix_server->DeregisterTask() and g_cg_mgr->ReleaseCgroupByTaskIdOnly() before returning from the function.
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Supervisor/SupervisorPublicDefs.h:32-37
Timestamp: 2025-05-26T11:06:28.796Z
Learning: The user L-Xiafeng prefers to defer refactoring duplicate definitions until they become a larger pattern in the codebase, rather than addressing individual instances immediately.
📚 Learning: 2025-05-08T07:39:29.207Z
Learnt from: huerni
Repo: PKUHPC/CraneSched PR: 469
File: src/Craned/TaskManager.cpp:1162-1175
Timestamp: 2025-05-08T07:39:29.207Z
Learning: In CraneSched, when a task fails in the LaunchTaskInstanceMt_ method, ActivateTaskStatusChangeAsync_ does not automatically release PMIx resources or cgroup. Explicit cleanup is required by calling g_pmix_server->DeregisterTask() and g_cg_mgr->ReleaseCgroupByTaskIdOnly().

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
  • src/Craned/Supervisor/CforedClient.cpp
📚 Learning: 2025-05-08T07:39:29.207Z
Learnt from: huerni
Repo: PKUHPC/CraneSched PR: 469
File: src/Craned/TaskManager.cpp:1162-1175
Timestamp: 2025-05-08T07:39:29.207Z
Learning: In CraneSched, when a task fails in the LaunchTaskInstanceMt_ method after PMIx registration, ActivateTaskStatusChangeAsync_ does not automatically release PMIx resources or cgroup resources. Explicit cleanup is required by calling g_pmix_server->DeregisterTask() and g_cg_mgr->ReleaseCgroupByTaskIdOnly() before returning from the function.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
  • src/Craned/Supervisor/CforedClient.cpp
📚 Learning: 2025-05-26T11:04:30.580Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Craned/SupervisorKeeper.h:32-32
Timestamp: 2025-05-26T11:04:30.580Z
Learning: The Supervisor component in the Crane system is designed to manage only one task per instance. The task specification is provided to the Supervisor during startup by Craned (likely through InitSupervisorRequest), so the ExecuteTask() method doesn't need to accept task parameters since the Supervisor already knows which task to execute.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-07-01T08:00:05.383Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Supervisor/TaskManager.cpp:1-1809
Timestamp: 2025-07-01T08:00:05.383Z
Learning: In TaskManager.cpp, the security model relies on administrator-configured command templates in ParseOCICmdPattern_ and system-provided usernames in ParseFilePathPattern_, with file permissions using the user's own uid/gid for privilege isolation. The user L-Xiafeng considers this security boundary sufficient and chooses not to fix job name-related path issues at this location.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-05-25T04:11:50.268Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Supervisor/TaskManager.cpp:220-220
Timestamp: 2025-05-25T04:11:50.268Z
Learning: In TaskManager.cpp, when step->IsCrun() is checked before dynamic_cast to CrunMetaInExecution*, the cast is guaranteed to succeed due to the program logic ensuring the correct meta type is used for Crun tasks. Null checks for these dynamic_cast operations are unnecessary in this context.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-05-02T07:12:46.896Z
Learnt from: RileyWen
Repo: PKUHPC/CraneSched PR: 475
File: src/Craned/CranedServer.h:0-0
Timestamp: 2025-05-02T07:12:46.896Z
Learning: The initialization of `m_supervisor_recovered_` to `true` in the CranedServer class is intentional despite the comment saying "When supervisor ready, init with false". This is temporary until the supervisor functionality is fully implemented.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-06-30T08:43:44.470Z
Learnt from: huerni
Repo: PKUHPC/CraneSched PR: 537
File: src/Craned/TaskManager.cpp:145-147
Timestamp: 2025-06-30T08:43:44.470Z
Learning: In the CraneSched codebase, src/Craned/Craned.cpp guarantees that g_config.CranedRes.contains(g_config.CranedIdOfThisNode) through explicit validation during startup. The code checks if g_config.CranedRes.contains(g_config.Hostname) and exits the process if not found, then sets g_config.CranedIdOfThisNode = g_config.Hostname. TaskManager constructor is called after this validation, so g_config.CranedRes[g_config.CranedIdOfThisNode] is guaranteed to be valid.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-04-02T10:11:33.562Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 461
File: src/Craned/CgroupManager.cpp:685-685
Timestamp: 2025-04-02T10:11:33.562Z
Learning: In the CgroupManager's GetJobBpfMapCgroupsV2 method, the developer has confirmed that cg_ino_job_id_map will always contain the key->cgroup_id element, making the CRANE_ASSERT check appropriate rather than requiring additional error handling.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-05-25T04:11:27.740Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Supervisor/TaskManager.cpp:396-396
Timestamp: 2025-05-25T04:11:27.740Z
Learning: In TaskManager.cpp, GetCrunMeta() calls don't need null checks because they're only called in contexts where the task is guaranteed to be a CRUN task (e.g., SetupChildProcessCrunX11_ is only called when step->IsCrun() && x11() conditions are met), ensuring the metadata will always be valid.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-05-25T04:08:03.273Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Supervisor/Supervisor.cpp:68-69
Timestamp: 2025-05-25T04:08:03.273Z
Learning: In the Crane supervisor component (src/Craned/Supervisor/), the supervisor process communicates with Craned through STDOUT using protobuf messages. The supervisor must not send any information to STDOUT before sending the "ready" message to Craned, as this would interfere with the inter-process communication protocol. Therefore, adding logging statements that might write to STDOUT before the ready message is sent could break the communication.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-05-08T07:38:42.362Z
Learnt from: huerni
Repo: PKUHPC/CraneSched PR: 469
File: src/Craned/TaskManager.cpp:0-0
Timestamp: 2025-05-08T07:38:42.362Z
Learning: In CraneSched's PMIx integration, the `g_pmix_server->SetupFork()` function must be called in the child process after fork() and before exec() to properly set up the PMIx environment variables.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-05-09T01:54:21.256Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 496
File: src/CraneCtld/CranedKeeper.cpp:51-53
Timestamp: 2025-05-09T01:54:21.256Z
Learning: The ConfigureCraned function in src/CraneCtld/CranedKeeper.cpp is called from a thread pool, so there's no need to worry about it blocking the gRPC completion queue thread.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
  • src/Craned/Supervisor/CforedClient.cpp
📚 Learning: 2025-05-02T07:06:36.103Z
Learnt from: RileyWen
Repo: PKUHPC/CraneSched PR: 475
File: src/CraneCtld/CtldGrpcServer.cpp:113-118
Timestamp: 2025-05-02T07:06:36.103Z
Learning: In CraneSched, gRPC methods should generally return Status::OK even when handling error conditions, as non-OK statuses cause the connection to terminate. Error information should be communicated within the RPC response payload instead.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-05-09T02:16:56.723Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 496
File: src/Craned/TaskManager.cpp:1498-1506
Timestamp: 2025-05-09T02:16:56.723Z
Learning: The `QueryRunningTasksAsync()` method in TaskManager.cpp is designed to never be called from inside the event loop thread, so there's no risk of deadlock with the synchronous `res.get()` call.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-08-14T02:56:35.503Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 587
File: src/Craned/Supervisor/CforedClient.cpp:449-454
Timestamp: 2025-08-14T02:56:35.503Z
Learning: In CforedClient::AsyncSendRecvThread_(), the guard `if (state <= State::Registering) { continue; }` in the TIMEOUT branch only prevents premature cleanup when stopping before registration completes, but it doesn't block normal gRPC event processing. The completion queue will still deliver Prepare/Write/Read events that advance the state machine normally.

Applied to files:

  • src/Craned/Supervisor/CforedClient.cpp
🧬 Code graph analysis (2)
src/Craned/Supervisor/TaskManager.cpp (4)
src/Craned/Common/CgroupManager.h (1)
  • CgroupManager (532-640)
src/Utilities/PublicHeader/OS.cpp (2)
  • CreateFoldersForFileEx (109-148)
  • CreateFoldersForFileEx (109-110)
src/Craned/Supervisor/CforedClient.h (1)
  • exit_code (131-131)
src/Craned/Supervisor/SupervisorPublicDefs.h (1)
  • exit_code (37-37)
src/Craned/Supervisor/CforedClient.cpp (2)
src/Utilities/PublicHeader/String.cpp (2)
  • SetCurrentThreadName (354-362)
  • SetCurrentThreadName (354-354)
src/Craned/Supervisor/SupervisorPublicDefs.h (1)
  • exit_code (37-37)
🔇 Additional comments (16)
src/Craned/Supervisor/TaskManager.cpp (10)

230-305: State-transition validation in GotNewStatus looks consistent

The new StepInstance::GotNewStatus checks illegal transitions and logs warnings without throwing, and it enforces daemon vs non‑daemon expectations for Configuring/Starting/Running/Completing. The final assignment of m_status_ after the switch is straightforward and matches the checks.


394-419: Per-task cgroup cleanup logic is sound

ITaskInstance::Cleanup mirrors the step-level cleanup pattern: bounded retries on KillAllProcesses, 100ms sleeps, then Destroy() and reset. This ensures per-task cgroups are torn down even if children linger briefly, without busy-spinning.


445-453: Crun env/X11 wiring is coherent, but depends on x11_meta having been set

The additions:

  • Set CRANE_PROCID / CRANE_PROC_ID in ProcInstance::InitEnvMap().
  • In SetupChildProcCrunX11_, use m_parent_step_inst_->x11_meta.value() for auth path and construct DISPLAY/XAUTHORITY consistently.

These are good, but they rely on StepInstance::Prepare having successfully populated x11_meta. Because EvGrpcExecuteTaskCb_ currently ignores StepInstance::Prepare's return code, any early-return from Prepare() before this->x11_meta = ... will cause .value() here to throw or abort.

Once you wire Prepare() errors into EvGrpcExecuteTaskCb_ (see other comment), this dependency will be safe.

Also applies to: 680-746


788-872: Container per-task cgroup usage and cleanup are consistent

ContainerInstance::Prepare now:

  • Delegates to ITaskInstance::Prepare() first, so the container task has its own cgroup.
  • Then prepares log files, pulls the image, creates pod and container.

ContainerInstance::Cleanup calls ITaskInstance::Cleanup() first, then Kill(0) to tear down the pod+container, which is idempotent.

This sequencing looks correct and keeps per-task cgroups tied cleanly to the container lifecycle.

Also applies to: 978-985


1545-1555: Killing via per-task cgroup is a good fit for multi-process tasks

ProcInstance::Kill now uses:

bool success = m_task_cg->KillAllProcesses(signum);

instead of signaling a single pid. This is more robust for MPI-like or multi-process tasks, and still returns a clear error if the cgroup kill fails.

The surrounding checks on m_pid_ and non-existent tasks look reasonable.

Also applies to: 1560-1566


1593-1620: New async/timer handles for supervisor init, shutdown, SIGCHLD, and gRPC look consistent

The constructor now:

  • Adds async handles for supervisor-finish-init and shutdown.
  • Starts a 1s SIGCHLD timer to flush queued SIGCHLD events.
  • Adds async handles for terminate-task, change-time-limit, execute-task, query-step-env, check-status, and migrate-ssh-to-cgroup, all wired to their Ev*Cb_ handlers.

Handle setup and timer intervals look consistent with existing patterns in this file.

Also applies to: 1669-1689


1720-1735: Daemon shutdown flow is reasonable, but relies on at least one queued final status

ShutdownSupervisorAsync enqueues (status, exit_code, reason) and pokes m_shutdown_supervisor_handle_; EvShutdownSupervisorCb_ then:

  • Spins in a do { try_dequeue } while (!got_final_status) loop until it pulls a tuple.
  • For daemon steps, updates the step status and sends StepStatusChangeAsync if not orphaned.
  • Calls m_step_.CleanUp(), shuts down the Craned client, gRPC server, and then this->Shutdown() to stop the uvw loop.

Given that ShutdownSupervisorAsync always enqueues before send(), try_dequeue should succeed on first iteration. The pattern is fine, but it assumes no code will ever call send() without having enqueued at least one status first.

Also applies to: 1882-1913


2250-2267: Step-level cgroup allocation + step_user_cg usage look correct

In EvGrpcExecuteTaskCb_ you now allocate a step-level user cgroup:

auto cg_expt = CgroupManager::AllocateAndGetCgroup(
    CgroupManager::CgroupStrByStepId(m_step_.job_id, m_step_.step_id, false),
    m_step_.GetStep().res(), false, Common::CgConstant::kCgMinMem);
...
m_step_.step_user_cg = std::move(cg_expt.value());
m_step_.cgroup_path = m_step_.step_user_cg->CgroupPath();

And later StepInstance::CleanUp() destroys step_user_cg with retries. This aligns with the per-task cgroup use in ITaskInstance::Prepare/Cleanup and the OOM evaluation logic that relies on cgroup_path.


2268-2295: Parallel task launch wiring is good, but watch concurrent error aggregation

The new parallel launch section:

  • Uses a std::latch to wait for all per-task LaunchExecution_ invocations.
  • Aggregates any non-success CraneErrCode into a shared err under thread_pool_mutex.
  • Records exec_id → task_id mappings for successful launches.

That’s a solid pattern. Once the TaskFinish_ concurrency issue is addressed (see earlier comment), this multi-task launch design will be in good shape.


1857-1868: CheckStatusAsync pattern is fine but blocks until uv loop responds

CheckStatusAsync builds a std::promise<StepStatus>, enqueues it, sends the async handle, and then synchronously waits:

status_future.get();

Given EvGrpcCheckStatusCb_ immediately dequeues and sets the value on the uv loop thread, this should return quickly and mirrors patterns used elsewhere in this codebase. Just be aware this call blocks the caller thread until the loop services the async event.

src/Craned/Supervisor/CforedClient.cpp (6)

74-84: Destructor assertion on m_fwd_meta_map enforces clean shutdown

The destructor now asserts:

m_cq_.Shutdown();
CRANE_ASSERT(m_fwd_meta_map.empty());

This is a useful guard to ensure all tasks have been fully unregistered (via TaskEnd) before CforedClient is destroyed. Just make sure every path that can stop tasks (normal exit, cfored connection failure, supervisor shutdown) eventually drains m_fwd_meta_map so this assertion doesn’t fire spuriously in production builds.


109-115: X11 proxy setup and per-connection state management look correct

InitUvX11FwdHandler() + SetupX11forwarding_() now:

  • Bind an X11 proxy on localhost within a safe port range.
  • On accept, allocate a new x11_local_id and corresponding X11FdInfo entry in m_x11_fd_info_map_.
  • Wire uvw callbacks so:
    • data_event forwards via TaskX11OutPutForward(x11_local_id, ...).
    • end_event / error_event mark sock_stopped, send EOF, and close the socket.
    • close_event erases the map entry.

This design keeps X11 per-connection state well-encapsulated and ensures cleanup on close.

Also applies to: 117-216


353-377: Stop-task IO cleanup now properly synchronized and resets handles/Fds

CleanStopTaskIOQueueCb_:

  • Takes m_mtx_ while looking up task_id in m_fwd_meta_map.
  • Closes and resets pipe/tty handles and closes stdout_read.
  • Then, outside the critical section, calls TaskOutputFinishNoLock_ and optionally TaskEnd.

This ordering avoids holding the mutex across callbacks into TaskManager and ensures IO handles are closed exactly once per task.


466-557: Cfored send/recv state machine and connection-failure handling look solid

AsyncSendRecvThread_:

  • Sets the thread name via SetCurrentThreadName("CforedSendRecv").
  • Drives the TaskIOStream state machine (RegisteringWaitRegisterAckForwardingDrainingUnregisteringEnd).
  • On !ok && tag != Prepare, logs “Cfored connection failed”, under m_mtx_ enqueues all task_ids in m_fwd_meta_map into m_stop_task_io_queue_, pokes the async handle, and calls:
g_task_mgr->TerminateTaskAsync(
    false, TerminatedBy::TERMINATION_BY_CFORED_CONN_FAILURE);

This gives a clear, centralized place where a dead cfored connection forces all tasks to stop and allows the supervisor side to clean up asynchronously.


714-727: New TASK_EXIT_STATUS / output forwarders integrate cleanly with the unified FwdRequest queue

The new methods:

  • TaskProcessStop(task_id, exit_code, signaled) enqueues a FwdRequest of type TASK_EXIT_STATUS and then marks proc_stopped, making TaskOutputFinishNoLock_ semantics unchanged.
  • TaskOutPutForward and TaskX11OutPutForward wrap raw data into FwdRequest variants (IOFwdRequest / X11FwdRequest).

And CleanOutputQueueAndWriteToStreamThread_ converts these into StreamTaskIORequest messages.

This design unifies all forward paths and keeps the stream writer loop simple.

Also applies to: 738-758


404-465: The review comment is incorrect for this codebase.

The project uses C++23 (CMakeLists.txt:134), where a default-constructed std::atomic<bool> is value-initialized to false by default. This behavior has been guaranteed since C++20. The code at line 478 in CforedClient.cpp is well-defined and requires no changes.

The review's concern about undefined behavior applies only to C++11–C++17; it does not apply here.

Likely an incorrect or invalid review comment.

Comment on lines 43 to 115
CraneErrCode StepInstance::Prepare() {
// Write script content into file
auto sh_path =
g_config.CraneScriptDir / fmt::format("Crane-{}.sh", g_config.JobId);

FILE* fptr = fopen(sh_path.c_str(), "w");
if (fptr == nullptr) {
CRANE_ERROR("Failed write the script");
return CraneErrCode::ERR_SYSTEM_ERR;
}

if (IsBatch())
fputs(m_step_to_supv_.batch_meta().sh_script().c_str(), fptr);
else // Crun
fputs(m_step_to_supv_.interactive_meta().sh_script().c_str(), fptr);

fclose(fptr);

chmod(sh_path.c_str(), strtol("0755", nullptr, 8));
if (chown(sh_path.c_str(), uid, gids[0]) != 0) {
CRANE_ERROR("Failed to change ownership of script file: {}",
strerror(errno));
}

script_path = sh_path.string();

// Init CriClient before preparation
if (IsContainer()) InitCriClient();

// Init cfored
InitCforedClient();
X11Meta x11_meta{};

auto cfored_client = GetCforedClient();
if (x11) {
if (x11_fwd)
x11_meta.x11_port = cfored_client->InitUvX11FwdHandler();
else
x11_meta.x11_port = GetStep().interactive_meta().x11_meta().port();

CRANE_TRACE("Crun task x11 enabled. Forwarding: {}, X11 Port: {}", x11_fwd,
x11_meta.x11_port);
}

auto x11_auth_path =
fmt::sprintf("%s/.crane/xauth/.Xauthority-XXXXXX", pwd.HomeDir());

bool ok = util::os::CreateFoldersForFileEx(x11_auth_path, pwd.Uid(),
pwd.Gid(), 0700);
if (!ok) {
CRANE_ERROR("Failed to create xauth source file for");
return CraneErrCode::ERR_SYSTEM_ERR;
}

// Default file permission is 0600.
int xauth_fd = mkstemp(x11_auth_path.data());
if (xauth_fd == -1) {
CRANE_ERROR("mkstemp() for xauth file failed: {}\n", strerror(errno));
return CraneErrCode::ERR_SYSTEM_ERR;
}

int ret = fchown(xauth_fd, pwd.Uid(), pwd.Gid());
if (ret == -1) {
CRANE_ERROR("fchown() for xauth file failed: {}\n", strerror(errno));
return CraneErrCode::ERR_SYSTEM_ERR;
}
x11_meta.x11_auth_path = x11_auth_path;
this->x11_meta = std::move(x11_meta);

return CraneErrCode::SUCCESS;
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

⚠️ Potential issue | 🟠 Major

🧩 Analysis chain

Close mkstemp FD and propagate StepInstance::Prepare failures to callers

StepInstance::Prepare now creates the step script and X11 auth file, but:

  • mkstemp(x11_auth_path.data()) returns xauth_fd that is never closed, and on the fchown error path you return without closing it. This leaks one FD per step.
  • Error returns (e.g., CreateFoldersForFileEx/mkstemp/fchown) are currently ignored in EvGrpcExecuteTaskCb_, so script/X11 preparation can fail silently while later code assumes x11_meta is populated and script_path is valid. In particular, a failed fopen leaves script_path empty, and a later ProcInstance::Prepare will exec bash -c "", and SetupChildProcCrunX11_ will call .x11_meta.value() even if Prepare() bailed before assigning it.

Consider:

  • Always close(xauth_fd) on both success and error paths before returning.
  • Making StepInstance::Prepare failures fatal to the step by checking the return in EvGrpcExecuteTaskCb_ and calling TaskFinish_ for all tasks instead of proceeding.

Example for the FD close:

  int xauth_fd = mkstemp(x11_auth_path.data());
  if (xauth_fd == -1) {
    CRANE_ERROR("mkstemp() for xauth file failed: {}\n", strerror(errno));
    return CraneErrCode::ERR_SYSTEM_ERR;
  }

  int ret = fchown(xauth_fd, pwd.Uid(), pwd.Gid());
  if (ret == -1) {
    CRANE_ERROR("fchown() for xauth file failed: {}\n", strerror(errno));
-    return CraneErrCode::ERR_SYSTEM_ERR;
+    close(xauth_fd);
+    return CraneErrCode::ERR_SYSTEM_ERR;
  }
+ close(xauth_fd);

Also applies to: 115-157


I'll verify the review comment by examining the codebase for the identified issues.


🏁 Script executed:

#!/bin/bash
# Locate TaskManager.cpp and examine the Prepare() method and surrounding code
find . -name "TaskManager.cpp" -type f | head -5

Length of output: 39


🏁 Script executed:

#!/bin/bash
# Get the full content of TaskManager.cpp or relevant sections
fd "TaskManager\.cpp" --type f -x wc -l {} \;

Length of output: 44


🏁 Script executed:

#!/bin/bash
# Search for EvGrpcExecuteTaskCb_ function to see error handling
rg -n "EvGrpcExecuteTaskCb_" --type=cpp -A 30 -B 5

Length of output: 14075


🏁 Script executed:

#!/bin/bash
# Search for uses of script_path and x11_meta after Prepare() calls
rg -n "script_path|x11_meta" --type=cpp -C 3 | head -80

Length of output: 5350


🏁 Script executed:

#!/bin/bash
# Search for close() calls on xauth_fd to verify if FD is closed
rg -n "close\s*\(\s*xauth_fd" --type=cpp

Length of output: 0


🏁 Script executed:

#!/bin/bash
# Check ProcInstance::Prepare and SetupChildProcCrunX11_ for unsafe assumptions
rg -n "ProcInstance::Prepare|SetupChildProcCrunX11_" --type=cpp -A 10 -B 2

Length of output: 3657


🏁 Script executed:

#!/bin/bash
# Examine lines 115-157 mentioned in "Also applies to"
rg -n "^" src/Craned/Supervisor/TaskManager.cpp | awk 'NR>=115 && NR<=157'

Length of output: 1301


🏁 Script executed:

#!/bin/bash
# Get context around mkstemp in the file to see actual FD handling
rg -n "mkstemp" --type=cpp -A 20 -B 5

Length of output: 2094


🏁 Script executed:

#!/bin/bash
# Search for TaskFinish_ to understand failure handling pathway
rg -n "TaskFinish_" --type=cpp -B 3 -A 5 | head -60

Length of output: 4622


🏁 Script executed:

#!/bin/bash
# Verify the structure - find StepInstance class definition
rg -n "class StepInstance" --type=cpp -A 15

Length of output: 996


🏁 Script executed:

#!/bin/bash
# Search for where Prepare() is called and how return value is handled
rg -n "\.Prepare\(\)" --type=cpp -B 3 -A 3

Length of output: 404


🏁 Script executed:

#!/bin/bash
# Look at CreateFoldersForFileEx behavior on errors
rg -n "CreateFoldersForFileEx" --type=cpp -B 2 -A 5

Length of output: 2621


🏁 Script executed:

#!/bin/bash
# Check x11_meta value() usage patterns
rg -n "x11_meta.*value\(\)" --type=cpp -B 3 -A 3

Length of output: 2206


🏁 Script executed:

#!/bin/bash
# Verify script_path initialization and usage
rg -n "script_path\s*=" --type=cpp -B 2 -A 2

Length of output: 283


🏁 Script executed:

#!/bin/bash
# Check if there's a default initialization of x11_meta field in StepInstance
rg -n "x11_meta" --type=cpp --type=hpp | grep -E "(member|field|private|public)" -B 2 -A 2 | head -30

Length of output: 27


🏁 Script executed:

#!/bin/bash
# Look for bash -c "" pattern mentioned in review
rg -n 'bash\s+-c\s+""|bash -c ""' --type=cpp -B 3 -A 3

Length of output: 0


🏁 Script executed:

#!/bin/bash
# Check if script_path has a default value or is optional
rg -n "script_path" --type=cpp --type=hpp | head -20

Length of output: 27


🏁 Script executed:

#!/bin/bash
# Check the actual repository location and structure
find . -name "*.cpp" -path "*TaskManager*" 2>/dev/null

Length of output: 39


🏁 Script executed:

#!/bin/bash
# Look for the complete EvGrpcExecuteTaskCb_ to understand error handling flow
rg -n "void.*EvGrpcExecuteTaskCb_|EvGrpcExecuteTaskCb_.*{" --type=cpp -A 50

Length of output: 163


🏁 Script executed:

#!/bin/bash
# Verify if x11_meta is optional or required
rg -n "optional.*x11_meta|x11_meta.*optional" --type=cpp

Length of output: 73


🏁 Script executed:

#!/bin="bash
# Check CreateFoldersForFileEx return value handling in context
rg -n "CreateFoldersForFileEx" --type=cpp -A 5 -B 2 | head -50

Length of output: 2621


🏁 Script executed:

#!/bin/bash
# Look for any existing close() patterns with file descriptors in this file
rg -n "close\(" src/Craned/Supervisor/TaskManager.cpp -B 2 -A 2

Length of output: 3304


🏁 Script executed:

#!/bin/bash
# Final check: verify entire Prepare() method from lines 43-113
sed -n '43,113p' src/Craned/Supervisor/TaskManager.cpp

Length of output: 2130


🏁 Script executed:

#!/bin/bash
# Verify lines 115-157
sed -n '115,157p' src/Craned/Supervisor/TaskManager.cpp

Length of output: 1129


🏁 Script executed:

#!/bin/bash
# Check caller of Prepare() and how result is used
rg -n "Prepare()" src/Craned/Supervisor/TaskManager.cpp -B 5 -A 10

Length of output: 4656


🏁 Script executed:

#!/bin/bash
# Look for mkstemp usage patterns to confirm FD not closed
grep -n "xauth_fd" src/Craned/Supervisor/TaskManager.cpp

Length of output: 133


🏁 Script executed:

#!/bin/bash
# Comprehensive check: find file and examine critical sections
cat src/Craned/Supervisor/TaskManager.cpp | head -200

Length of output: 5880


🏁 Script executed:

#!/bin/bash
# Check if xauth_fd is ever closed in success path
rg -n "xauth_fd" src/Craned/Supervisor/TaskManager.cpp -A 50

Length of output: 1801


🏁 Script executed:

#!/bin="bash
# Verify EvGrpcExecuteTaskCb_ doesn't check Prepare return value
ast-grep --pattern $'void EvGrpcExecuteTaskCb_($_,$_){$$$}'

Length of output: 0


🏁 Script executed:

#!/bin/bash
# Search for how x11_meta is accessed in downstream code
rg -n "\.x11_meta\(\)|\.x11_meta\[|\.x11_meta\.|x11_meta\.value" --type=cpp -B 2 -A 2

Length of output: 2667


🏁 Script executed:

#!/bin/bash
# Verify script_path is used without null check
rg -n "script_path" src/Craned/Supervisor/TaskManager.cpp -B 2 -A 2

Length of output: 781


🏁 Script executed:

#!/bin/bash
# Check the actual file path
ls -la src/Craned/Supervisor/TaskManager.cpp 2>/dev/null || echo "File not found, searching..."
fd "TaskManager" --type f

Length of output: 161


🏁 Script executed:

#!/bin/bash
# Look for IsBatch, IsContainer, and other key methods
rg -n "IsBatch\(\)|IsContainer\(\)|GetStep\(\)" src/Craned/Supervisor/TaskManager.cpp -B 2 -A 2 | head -30

Length of output: 737


🏁 Script executed:

#!/bin/bash
# Final verification: look at complete method signatures and error paths
rg -n "CraneErrCode.*Prepare\(\)" --type=cpp -A 100

Length of output: 61085


🏁 Script executed:

#!/bin/bash
# Trace TaskFinish_ calls to understand error recovery
rg -n "TaskFinish_" src/Craned/Supervisor/TaskManager.cpp -B 3 -A 5

Length of output: 6012


🏁 Script executed:

#!/bin/bash
# Check if there's cleanup in destructor or other locations
rg -n "~StepInstance|~ProcInstance" --type=cpp -A 10

Length of output: 2984


🏁 Script executed:

#!/bin/bash
# Verify the exact line content for mkstemp to xauth_fd
rg -n "mkstemp.*xauth_fd|xauth_fd.*mkstemp" --type=cpp -B 2 -A 20

Length of output: 1697


🏁 Script executed:

#!/bin/bash
# Check the specific error path for fchown
rg -n "fchown" src/Craned/Supervisor/TaskManager.cpp -B 5 -A 10

Length of output: 584


🏁 Script executed:

#!/bin/bash
# Verify script_path assignment
rg -n "script_path\s*=\s*sh_path" --type=cpp -B 5 -A 5

Length of output: 735


🏁 Script executed:

#!/bin/bash
# Comprehensive final scan for all FD-related patterns
rg -n "fd|close|dup|pipe" src/Craned/Supervisor/TaskManager.cpp | grep -i "xauth\|fd\|close" | head -30

Length of output: 1212


🏁 Script executed:

#!/bin/bash
# Check if x11_meta is conditionally assigned
rg -n "x11_meta\s*=" src/Craned/Supervisor/TaskManager.cpp -B 3 -A 3

Length of output: 483


🏁 Script executed:

#!/bin/bash
# Look for bash -c execution pattern
rg -n 'bash.*-c|execve.*bash' src/Craned/Supervisor/TaskManager.cpp -B 3 -A 3

Length of output: 267


🏁 Script executed:

#!/bin/bash
# Verify complete code structure one final time
wc -l src/Craned/Supervisor/TaskManager.cpp && echo "---" && sed -n '1,50p' src/Craned/Supervisor/TaskManager.cpp

Length of output: 1717


Fix file descriptor leak and propagate StepInstance::Prepare failures

The xauth_fd returned by mkstemp() at line 98 is never closed. On the fchown error path (line 107), the function returns without closing the FD. On the success path (line 112), the FD is also not closed. This leaks one FD per step.

Additionally, at line 2238 in EvGrpcExecuteTaskCb_(), the call to m_step_.Prepare() ignores the return value. When preparation fails—due to CreateFoldersForFileEx, mkstemp, or fchown errors—execution continues silently. Since x11_meta is std::optional and remains unset if Prepare() fails before line 110, downstream code calling .x11_meta.value() at lines 681, 693, and 1396 will crash. Similarly, script_path remains empty if Prepare() fails before line 67, leading to executing empty commands. Contrast this with LaunchExecution_() at line 1804, which properly checks task->Prepare() and calls TaskFinish_() on failure.

Fixes:

  • Always close(xauth_fd) on both success and error paths (lines 107 and 112).
  • Check m_step_.Prepare() return value at line 2238 and call TaskFinish_() for all tasks on failure, similar to LaunchExecution_().

Example for FD close:

  int ret = fchown(xauth_fd, pwd.Uid(), pwd.Gid());
  if (ret == -1) {
    CRANE_ERROR("fchown() for xauth file failed: {}\n", strerror(errno));
+    close(xauth_fd);
    return CraneErrCode::ERR_SYSTEM_ERR;
  }
  x11_meta.x11_auth_path = x11_auth_path;
  this->x11_meta = std::move(x11_meta);
+ close(xauth_fd);
  return CraneErrCode::SUCCESS;
🤖 Prompt for AI Agents
In src/Craned/Supervisor/TaskManager.cpp around lines 43-113, mkstemp()'s
xauth_fd is never closed: always close xauth_fd on the error path after failed
fchown and also close it on the success path after setting
x11_meta.x11_auth_path (ensure no early returns leak the descriptor). Also in
src/Craned/Supervisor/TaskManager.cpp at EvGrpcExecuteTaskCb_ (around line
2238), check the return value of m_step_.Prepare() and, on failure, call
TaskFinish_() for the task (same behavior as LaunchExecution_) and do not
proceed with execution when Prepare() fails.

Copy link
Contributor

@coderabbitai coderabbitai bot left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Actionable comments posted: 1

Caution

Some comments are outside the diff and can’t be posted inline due to platform limitations.

⚠️ Outside diff range comments (1)
src/Craned/Supervisor/TaskManager.cpp (1)

1547-1560: Check m_task_cg before dereferencing.

Line 1552 dereferences m_task_cg without checking if it's null. If ITaskInstance::Prepare() failed or wasn't called, this will crash. Add a null check.

Apply this diff:

 CraneErrCode ProcInstance::Kill(int signum) {
   if (m_pid_ != 0) {
     CRANE_TRACE("Killing pid {} with signal {}", m_pid_, signum);
 
+    if (!m_task_cg) {
+      CRANE_ERROR("[Task #{}] No cgroup allocated, cannot kill.", task_id);
+      return CraneErrCode::ERR_CGROUP;
+    }
     // Send the signal to the whole process group.
     bool success = this->m_task_cg->KillAllProcesses(signum);
♻️ Duplicate comments (5)
src/Craned/Supervisor/TaskManager.cpp (5)

2319-2343: Prevent double set_value when step is non-daemon.

After setting the promise at line 2323, the code continues to line 2325+ and may attempt to set the promise again at line 2333, 2339, or 2341, causing std::future_error and crashing the supervisor. Add continue immediately after line 2323.

Apply this diff:

     if (!m_step_.IsDaemonStep()) {
       CRANE_ERROR("Trying to move pid {} to no daemon step", pid);
       prom.set_value(CraneErrCode::ERR_INVALID_PARAM);
+      continue;
     }

380-394: Defend against missing task_res_map entries.

Line 385 uses .at(task_id) which throws std::out_of_range if the scheduler hasn't populated task_res_map for this task (explicitly noted as TBD in PR objectives). Use .find() and handle missing entries gracefully.

Based on past review feedback.


43-115: Close mkstemp FD on all paths.

The xauth_fd opened at line 99 is never closed. On the error path (line 107) and success path (line 112), the FD leaks. Always close the descriptor before returning.

Based on past review feedback.


2238-2240: Check Prepare() return value and fail tasks on error.

Line 2240 ignores the return value of m_step_.Prepare(). If preparation fails (e.g., script creation, xauth setup), execution continues with invalid state. Check the result and call TaskFinish_() for all tasks on failure, similar to line 1806.

Based on past review feedback.


1766-1787: Update reason in both status override branches.

While line 1772 now preserves the reason when updating max_exit_code, lines 1775-1776 override final_status_on_termination without updating final_reason_on_termination. This can leave a stale reason mismatched with the final status.

Apply this diff:

   // Error status has higher priority than success status.
-  if (new_status != StepStatus::Completed)
+  if (new_status != StepStatus::Completed) {
     status.final_status_on_termination = new_status;
+    status.final_reason_on_termination = reason.value_or("");
+  }
🧹 Nitpick comments (1)
src/Craned/Supervisor/TaskManager.cpp (1)

1894-1915: Replace do-while with if-check to avoid infinite loop.

Lines 1897-1914 use a do-while loop that continues when try_dequeue fails, creating an infinite loop if the queue is empty. While the queue should always have an element (enqueued at line 1734 before send()), defensive coding suggests using a single if check instead.

Apply this diff:

-  std::tuple<crane::grpc::TaskStatus, uint32_t, std::string> final_status;
-  bool got_final_status = false;
-  do {
-    got_final_status = m_shutdown_status_queue_.try_dequeue(final_status);
-    if (!got_final_status) continue;
+  std::tuple<crane::grpc::TaskStatus, uint32_t, std::string> final_status;
+  if (m_shutdown_status_queue_.try_dequeue(final_status)) {
     auto& [status, exit_code, reason] = final_status;
     if (m_step_.IsDaemonStep()) {
       CRANE_DEBUG("Sending a {} status as daemon step.", status);
       m_step_.GotNewStatus(status);
       if (!m_step_.orphaned)
         g_craned_client->StepStatusChangeAsync(status, exit_code, reason);
     }
 
     m_step_.CleanUp();
 
     g_craned_client->Shutdown();
     g_server->Shutdown();
 
     this->Shutdown();
-  } while (!got_final_status);
+  } else {
+    CRANE_ERROR("EvShutdownSupervisorCb_ called but queue is empty");
+  }
📜 Review details

Configuration used: Path: .coderabbit.yaml

Review profile: CHILL

Plan: Pro

📥 Commits

Reviewing files that changed from the base of the PR and between 9460464 and 2d57253.

📒 Files selected for processing (1)
  • src/Craned/Supervisor/TaskManager.cpp (30 hunks)
🧰 Additional context used
🧠 Learnings (15)
📓 Common learnings
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Supervisor/SupervisorPublicDefs.h:32-37
Timestamp: 2025-05-26T11:06:28.796Z
Learning: The user L-Xiafeng prefers to defer refactoring duplicate definitions until they become a larger pattern in the codebase, rather than addressing individual instances immediately.
📚 Learning: 2025-05-08T07:39:29.207Z
Learnt from: huerni
Repo: PKUHPC/CraneSched PR: 469
File: src/Craned/TaskManager.cpp:1162-1175
Timestamp: 2025-05-08T07:39:29.207Z
Learning: In CraneSched, when a task fails in the LaunchTaskInstanceMt_ method, ActivateTaskStatusChangeAsync_ does not automatically release PMIx resources or cgroup. Explicit cleanup is required by calling g_pmix_server->DeregisterTask() and g_cg_mgr->ReleaseCgroupByTaskIdOnly().

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-05-08T07:39:29.207Z
Learnt from: huerni
Repo: PKUHPC/CraneSched PR: 469
File: src/Craned/TaskManager.cpp:1162-1175
Timestamp: 2025-05-08T07:39:29.207Z
Learning: In CraneSched, when a task fails in the LaunchTaskInstanceMt_ method after PMIx registration, ActivateTaskStatusChangeAsync_ does not automatically release PMIx resources or cgroup resources. Explicit cleanup is required by calling g_pmix_server->DeregisterTask() and g_cg_mgr->ReleaseCgroupByTaskIdOnly() before returning from the function.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-05-26T11:04:30.580Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Craned/SupervisorKeeper.h:32-32
Timestamp: 2025-05-26T11:04:30.580Z
Learning: The Supervisor component in the Crane system is designed to manage only one task per instance. The task specification is provided to the Supervisor during startup by Craned (likely through InitSupervisorRequest), so the ExecuteTask() method doesn't need to accept task parameters since the Supervisor already knows which task to execute.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-07-01T08:00:05.383Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Supervisor/TaskManager.cpp:1-1809
Timestamp: 2025-07-01T08:00:05.383Z
Learning: In TaskManager.cpp, the security model relies on administrator-configured command templates in ParseOCICmdPattern_ and system-provided usernames in ParseFilePathPattern_, with file permissions using the user's own uid/gid for privilege isolation. The user L-Xiafeng considers this security boundary sufficient and chooses not to fix job name-related path issues at this location.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-05-25T04:11:50.268Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Supervisor/TaskManager.cpp:220-220
Timestamp: 2025-05-25T04:11:50.268Z
Learning: In TaskManager.cpp, when step->IsCrun() is checked before dynamic_cast to CrunMetaInExecution*, the cast is guaranteed to succeed due to the program logic ensuring the correct meta type is used for Crun tasks. Null checks for these dynamic_cast operations are unnecessary in this context.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-05-02T07:12:46.896Z
Learnt from: RileyWen
Repo: PKUHPC/CraneSched PR: 475
File: src/Craned/CranedServer.h:0-0
Timestamp: 2025-05-02T07:12:46.896Z
Learning: The initialization of `m_supervisor_recovered_` to `true` in the CranedServer class is intentional despite the comment saying "When supervisor ready, init with false". This is temporary until the supervisor functionality is fully implemented.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-06-30T08:43:44.470Z
Learnt from: huerni
Repo: PKUHPC/CraneSched PR: 537
File: src/Craned/TaskManager.cpp:145-147
Timestamp: 2025-06-30T08:43:44.470Z
Learning: In the CraneSched codebase, src/Craned/Craned.cpp guarantees that g_config.CranedRes.contains(g_config.CranedIdOfThisNode) through explicit validation during startup. The code checks if g_config.CranedRes.contains(g_config.Hostname) and exits the process if not found, then sets g_config.CranedIdOfThisNode = g_config.Hostname. TaskManager constructor is called after this validation, so g_config.CranedRes[g_config.CranedIdOfThisNode] is guaranteed to be valid.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-04-02T10:11:33.562Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 461
File: src/Craned/CgroupManager.cpp:685-685
Timestamp: 2025-04-02T10:11:33.562Z
Learning: In the CgroupManager's GetJobBpfMapCgroupsV2 method, the developer has confirmed that cg_ino_job_id_map will always contain the key->cgroup_id element, making the CRANE_ASSERT check appropriate rather than requiring additional error handling.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-05-25T04:11:27.740Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Supervisor/TaskManager.cpp:396-396
Timestamp: 2025-05-25T04:11:27.740Z
Learning: In TaskManager.cpp, GetCrunMeta() calls don't need null checks because they're only called in contexts where the task is guaranteed to be a CRUN task (e.g., SetupChildProcessCrunX11_ is only called when step->IsCrun() && x11() conditions are met), ensuring the metadata will always be valid.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-05-25T04:08:03.273Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Supervisor/Supervisor.cpp:68-69
Timestamp: 2025-05-25T04:08:03.273Z
Learning: In the Crane supervisor component (src/Craned/Supervisor/), the supervisor process communicates with Craned through STDOUT using protobuf messages. The supervisor must not send any information to STDOUT before sending the "ready" message to Craned, as this would interfere with the inter-process communication protocol. Therefore, adding logging statements that might write to STDOUT before the ready message is sent could break the communication.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-05-08T07:38:42.362Z
Learnt from: huerni
Repo: PKUHPC/CraneSched PR: 469
File: src/Craned/TaskManager.cpp:0-0
Timestamp: 2025-05-08T07:38:42.362Z
Learning: In CraneSched's PMIx integration, the `g_pmix_server->SetupFork()` function must be called in the child process after fork() and before exec() to properly set up the PMIx environment variables.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-05-09T01:54:21.256Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 496
File: src/CraneCtld/CranedKeeper.cpp:51-53
Timestamp: 2025-05-09T01:54:21.256Z
Learning: The ConfigureCraned function in src/CraneCtld/CranedKeeper.cpp is called from a thread pool, so there's no need to worry about it blocking the gRPC completion queue thread.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-05-02T07:06:36.103Z
Learnt from: RileyWen
Repo: PKUHPC/CraneSched PR: 475
File: src/CraneCtld/CtldGrpcServer.cpp:113-118
Timestamp: 2025-05-02T07:06:36.103Z
Learning: In CraneSched, gRPC methods should generally return Status::OK even when handling error conditions, as non-OK statuses cause the connection to terminate. Error information should be communicated within the RPC response payload instead.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-05-09T02:16:56.723Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 496
File: src/Craned/TaskManager.cpp:1498-1506
Timestamp: 2025-05-09T02:16:56.723Z
Learning: The `QueryRunningTasksAsync()` method in TaskManager.cpp is designed to never be called from inside the event loop thread, so there's no risk of deadlock with the synchronous `res.get()` call.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
🧬 Code graph analysis (1)
src/Craned/Supervisor/TaskManager.cpp (7)
src/Craned/Common/CgroupManager.h (1)
  • CgroupManager (532-640)
src/Craned/Core/CranedPublicDefs.h (1)
  • g_config (156-156)
src/Craned/Supervisor/TaskManager.h (3)
  • uid (56-56)
  • InitCriClient (150-155)
  • x11 (61-61)
src/Utilities/PublicHeader/OS.cpp (2)
  • CreateFoldersForFileEx (109-148)
  • CreateFoldersForFileEx (109-110)
src/Craned/Core/StepInstance.cpp (5)
  • CleanUp (42-78)
  • CleanUp (42-42)
  • ec (276-276)
  • ec (295-295)
  • ec (432-432)
src/Craned/Supervisor/CforedClient.h (1)
  • exit_code (131-131)
src/Craned/Supervisor/SupervisorPublicDefs.h (1)
  • exit_code (37-37)
🔇 Additional comments (9)
src/Craned/Supervisor/TaskManager.cpp (9)

1458-1462: LGTM!

The child process correctly migrates itself into the per-task cgroup before execution. Aborting on failure is appropriate since the task cannot proceed without proper resource isolation.


1595-1697: LGTM!

The new async handles for supervisor initialization, shutdown, status checks, and SSH process migration follow the established pattern and integrate properly with the uvw event loop.


2272-2296: LGTM!

The thread pool execution properly synchronizes with a latch and protects shared state (err, m_exec_id_task_id_map_) with a mutex. Tasks that fail to launch are handled via TaskFinish_() while successful tasks continue.


2252-2268: LGTM!

Step-level cgroup allocation is properly handled with error propagation. On failure, all tasks are failed and the promise is fulfilled. For calloc steps, the cgroup is lazily allocated later when needed (e.g., SSH migration).


232-307: LGTM!

The state machine validation in GotNewStatus() properly handles transitions for both daemon and regular steps, with appropriate warnings for unexpected transitions while still accepting them defensively.


117-159: LGTM!

The cleanup logic properly handles X11 auth files, scripts, cgroups, and client cleanup. Checks for null/empty pointers before accessing, and retries killing processes up to 5 times before giving up.


396-421: LGTM!

Task cleanup mirrors the step cleanup pattern, properly retrying process termination and destroying the task cgroup.


1859-1882: LGTM!

CheckStatusAsync blocks on the future (line 1868) but this is acceptable as it's called from gRPC thread pool threads. MigrateSshProcToCgroupAsync properly returns a future for the caller to await.


508-512: LGTM!

The pty multi-task check correctly returns early without closing pipes, as pipes are never initialized when pty == true. The past issue has been addressed.

Copy link
Contributor

@coderabbitai coderabbitai bot left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Actionable comments posted: 5

Caution

Some comments are outside the diff and can’t be posted inline due to platform limitations.

⚠️ Outside diff range comments (2)
src/Craned/Common/CgroupManager.h (1)

574-580: Documentation mentions alloc_mem parameter not present in signature.

The comment at line 574 mentions alloc_mem true if need to enforce memory limit, default true, but the function signature at lines 577-580 doesn't include this parameter. Either remove the comment or add the parameter.

   * \param recover recover cgroup instead creating new one.
   * \param min_mem minimum memory size for cgroup, default none, for job cgroup
-  * \param alloc_mem true if need to enforce memory limit, default true
   * \return CraneExpected<std::unique_ptr<CgroupInterface>> created cgroup
src/Craned/Core/JobManager.cpp (1)

824-927: Use‑after‑free risk when freeing orphaned jobs in EvCleanTerminateTaskQueueCb_

In the mark_as_orphaned && terminate_job branch you:

  1. Grab job_instance = map_ptr->at(elem.job_id).RawPtr();.
  2. Hold absl::MutexLock lk(job_instance->step_map_mtx.get());.
  3. Call FreeJobInfoNoLock_(elem.job_id, map_ptr, uid_map_ptr);, which erases the job from m_job_map_ and moves the JobInD into a local job_opt.
  4. Still use job_instance->step_map to iterate steps and populate steps_to_clean.

After FreeJobInfoNoLock_ erases the map entry, job_instance is dangling; reading job_instance->step_map is UB.

Instead, iterate over the JobInD returned from job_opt:

      if (job_opt.has_value()) {
-        job_to_clean.emplace_back(std::move(job_opt.value()));
-        for (auto& [step_id, step] : job_instance->step_map) {
+        auto job_val = std::move(job_opt.value());
+        for (auto& [step_id, step] : job_val.step_map) {
           CRANE_DEBUG("[Step #{}.{}] Removed orphaned step.", elem.job_id,
                       step_id);
           auto* step_ptr = step.get();
           steps_to_clean.push_back(step_ptr);
         }
+        job_to_clean.emplace_back(std::move(job_val));
      } else {

and drop further use of job_instance after calling FreeJobInfoNoLock_.

♻️ Duplicate comments (13)
src/Craned/Supervisor/SupervisorServer.cpp (1)

44-56: Fix std::expected usage and replace std::unreachable().

This code segment has issues already flagged in previous review:

  1. Line 46: std::expected env_expt without template parameters is ill-formed
  2. Line 55: std::unreachable() causes undefined behavior if executed
src/Craned/Core/StepInstance.cpp (3)

80-87: Make the crane_cgroup precondition for SpawnSupervisor explicit

SpawnSupervisor assumes crane_cgroup is non-null and immediately calls this->crane_cgroup->MigrateProcIn(child_pid). Today LaunchStepMt_ always calls CreateCg() first, but if any future recovery or alternative path calls SpawnSupervisor without creating the step cgroup, this will null‑deref.

Consider making the invariant explicit:

CraneErrCode StepInstance::SpawnSupervisor(const EnvMap& job_env_map) {
+  if (!crane_cgroup) {
+    CRANE_ERROR("[Step #{}.{}] SpawnSupervisor called without cgroup.",
+                job_id, step_id);
+    return CraneErrCode::ERR_CGROUP;
+  }

This keeps failures localized instead of crashing on a missing cgroup.

Also applies to: 149-161


408-419: Guard against supervisor_stub being null in ExecuteStepAsync

ExecuteStepAsync calls stub->ExecuteStep() in a detached task without checking that supervisor_stub is initialized. If the call site ever triggers this before SpawnSupervisor completes (or after a failed spawn), this will crash.

A lightweight runtime guard keeps this safe:

void StepInstance::ExecuteStepAsync() {
-  this->GotNewStatus(StepStatus::Running);
-
-  g_thread_pool->detach_task([job_id = job_id, step_id = step_id,
-                              stub = supervisor_stub] {
+  if (!supervisor_stub) {
+    CRANE_ERROR("[Step #{}.{}] supervisor_stub is null, cannot execute step.",
+                job_id, step_id);
+    return;
+  }
+
+  this->GotNewStatus(StepStatus::Running);
+
+  auto stub = supervisor_stub;  // keep local copy
+  g_thread_pool->detach_task([job_id = job_id, step_id = step_id,
+                              stub = std::move(stub)] {
     auto code = stub->ExecuteStep();

This preserves the shared_ptr lifetime while avoiding a null deref.


334-406: Daemon step Starting status is still applied despite being logged as invalid

In GotNewStatus, the StepStatus::Starting case logs that Starting is invalid for daemon steps but then falls through to the final status = new_status;, leaving daemon steps incorrectly marked as Starting.

Add an early return for daemon steps in that branch so the invalid transition is ignored:

  case StepStatus::Starting: {
    if (this->IsDaemonStep()) {
      CRANE_WARN(
          "[Step {}.{}] Daemon step got invalid status 'Starting' current "
          "status: {}.",
          job_id, step_id, this->status);
-    } else {
+      return;
+    } else {
      if (status != StepStatus::Configuring)
        CRANE_WARN(
            "[Step {}.{}] Step status is not 'Configuring' when "
            "receiving new status 'Starting', current status: {}.",
            job_id, step_id, this->status);
    }
    break;
  }
src/Craned/Supervisor/CforedClient.cpp (1)

117-199: Synchronize all accesses to m_x11_fd_info_map_ and X11 per‑connection state

X11 per‑connection state is mutated/read from multiple threads:

  • In SetupX11forwarding_’s listen handler you do:

    x11_local_id_t x11_local_id = next_x11_id_++;
    auto& x11_fd_info = m_x11_fd_info_map_[x11_local_id];
    …
    sock->data(x11_fd_info);
    …
    m_x11_fd_info_map_.erase(x11_local_id);
  • In AsyncSendRecvThread_’s Forwarding state, on TASK_X11_INPUT you lock m_mtx_ and then access m_x11_fd_info_map_ and X11FdInfo::x11_input_stopped.

This mixes locked (m_mtx_‑guarded) and unlocked access to m_x11_fd_info_map_ and X11FdInfo, which is a data race.

Either:

  • Guard all reads/writes of m_x11_fd_info_map_ and X11FdInfo fields with absl::Mutex m_mtx_, e.g.:

    {
      absl::MutexLock lock(&m_mtx_);
      auto& x11_fd_info = m_x11_fd_info_map_[x11_local_id];
      sock->data(x11_fd_info);
    }

    and similarly around erase, or

  • Introduce a dedicated mutex for X11 state and use it consistently in both threads.

This keeps X11 forwarding safe across the event loop and gRPC threads.

Also applies to: 626-642

src/Craned/Supervisor/TaskManager.h (1)

424-428: ShutdownSupervisorAsync still mixes TaskStatus and StepStatus end‑to‑end

The method and shutdown queue still use crane::grpc::TaskStatus while callers and StepInstance::GotNewStatus operate on StepStatus. This recreates the earlier mismatch and will either fail to compile or blur the intended separation between task and step state.

Align the API and queue on StepStatus all the way through (and convert once at the gRPC boundary if needed), e.g.:

-  void ShutdownSupervisorAsync(
-      crane::grpc::TaskStatus new_status = StepStatus::Completed,
-      uint32_t exit_code = 0, std::string reason = "");
+  void ShutdownSupervisorAsync(
+      StepStatus new_status = StepStatus::Completed,
+      uint32_t exit_code = 0, std::string reason = "");-  std::shared_ptr<uvw::async_handle> m_shutdown_supervisor_handle_;
-
-  ConcurrentQueue<std::tuple<crane::grpc::TaskStatus, uint32_t, std::string>>
-      m_shutdown_status_queue_;
+  std::shared_ptr<uvw::async_handle> m_shutdown_supervisor_handle_;
+
+  ConcurrentQueue<std::tuple<StepStatus, uint32_t, std::string>>
+      m_shutdown_status_queue_;

and mirror the parameter type in the .cpp implementation and EvShutdownSupervisorCb_ so StepInstance::GotNewStatus sees a StepStatus.

Also applies to: 510-512, 536-540, 571-577

src/Craned/Core/JobManager.cpp (2)

494-510: QuerySshStepEnvVariables still risks dereferencing a null supervisor_stub

This function now does proper job/step existence checks, but it immediately does:

auto& stub = step_it->second->supervisor_stub;
return stub->QueryStepEnv();

If the supervisor never started (or failed/recovered poorly), supervisor_stub can be null and this will crash — and this path is exercised directly by CranedServiceImpl::QuerySshStepEnvVariables. Mirroring the time‑limit change, add a null guard:

  auto& stub = step_it->second->supervisor_stub;
-  return stub->QueryStepEnv();
+  if (!stub) {
+    CRANE_ERROR("[Step #{}.{}] Supervisor stub is null when querying SSH "
+                "step env.", job_id, step_id);
+    return std::unexpected{CraneErrCode::ERR_NON_EXISTENT};
+  }
+  return stub->QueryStepEnv();

757-787: Fix iterator and null‑stub dereferences in MigrateProcToCgroupOfJob

Two unsafe cases remain:

  • If the daemon step is missing, you log a debug line but still dereference daemon_step_it.
  • If daemon_step->supervisor_stub is null, you log an error but still call stub->MigrateSshProcToCg(pid).

Both lead to UB/crash. Early‑exit after logging in each case:

  auto daemon_step_it = job->step_map.find(kDaemonStepId);
  if (daemon_step_it == job->step_map.end()) {
    CRANE_DEBUG(
        "[Step #{}.{}] Daemon step not found when migrating pid {} to "
        "cgroup of job#{}.",
        job_id, kDaemonStepId, pid, job_id);
-  }
-  auto& daemon_step = daemon_step_it->second;
-  auto stub = daemon_step->supervisor_stub;
-  if (!stub) {
+    return false;
+  }
+  auto& daemon_step = daemon_step_it->second;
+  auto stub = daemon_step->supervisor_stub;
+  if (!stub) {
     CRANE_ERROR(
         "[Job #{}] Daemon step sSupervisor stub is null when migrating pid {} "
         "to "
         "cgroup of job#{}.",
         job_id, kDaemonStepId, pid, job_id);
+    return false;
   }
  auto err = stub->MigrateSshProcToCg(pid);
src/Craned/Supervisor/CforedClient.h (1)

29-36: Ensure X11 per‑connection state (m_x11_fd_info_map_) is consistently synchronized

With x11_local_id_t, X11FdInfo, and m_x11_fd_info_map_ now part of the class, accesses happen from both the uvw event loop thread and the gRPC forwarding thread (see the .cpp). Only the latter currently uses m_mtx_, so map and X11FdInfo fields can still be touched without synchronization on the uvw side.

To avoid data races:

  • Decide whether m_mtx_ is the canonical guard for X11 state, and if so, wrap all lookups/insertions/erasures of m_x11_fd_info_map_ and writes to X11FdInfo in absl::MutexLock, or
  • Introduce a dedicated mutex for X11 state and use it consistently on both code paths.

This matches the intention called out in the existing comments and keeps per‑connection X11 state safe across threads.

Also applies to: 56-58, 150-152, 161-162

src/Craned/Supervisor/TaskManager.cpp (4)

380-394: Defend against missing task_res_map entries to prevent crash.

ITaskInstance::Prepare unconditionally calls task_res_map().at(task_id) at line 385, which throws std::out_of_range if the key doesn't exist. Since the PR objectives explicitly state "Scheduler needs to fill in the task res field" as an outstanding item, this code will crash the supervisor if the scheduler hasn't yet populated the map.

Based on past review comments, this critical issue remains unresolved.

Check for the key's presence and fail gracefully:

- auto cg_expt = CgroupManager::AllocateAndGetCgroup(
-     CgroupManager::CgroupStrByTaskId(g_config.JobId, g_config.StepId,
-                                      task_id),
-     m_parent_step_inst_->GetStep().task_res_map().at(task_id), false);
+ const auto& step = m_parent_step_inst_->GetStep();
+ const auto& task_res_map = step.task_res_map();
+ auto it = task_res_map.find(task_id);
+ if (it == task_res_map.end()) {
+   CRANE_ERROR("[Step #{}.{}] No task_res entry for task #{}.", 
+               g_config.JobId, g_config.StepId, task_id);
+   return CraneErrCode::ERR_CGROUP;
+ }
+ auto cg_expt = CgroupManager::AllocateAndGetCgroup(
+     CgroupManager::CgroupStrByTaskId(g_config.JobId, g_config.StepId,
+                                      task_id),
+     it->second, false);

98-112: Close the file descriptor returned by mkstemp() to prevent FD leak.

The xauth_fd returned by mkstemp() at line 99 is never closed. On the fchown error path (line 107), the function returns without closing the FD. On the success path (line 112), the FD is also left open. This leaks one file descriptor per step initialization.

Based on past review comments, this issue remains unresolved.

Apply this diff to close the FD:

  int ret = fchown(xauth_fd, pwd.Uid(), pwd.Gid());
  if (ret == -1) {
    CRANE_ERROR("fchown() for xauth file failed: {}\n", strerror(errno));
+   close(xauth_fd);
    return CraneErrCode::ERR_SYSTEM_ERR;
  }
  x11_meta.x11_auth_path = x11_auth_path;
  this->x11_meta = std::move(x11_meta);
+ close(xauth_fd);
  return CraneErrCode::SUCCESS;

2316-2344: Prevent double set_value crash by adding continue after error in non-daemon check.

When !m_step_.IsDaemonStep() evaluates to true at line 2321, the promise is fulfilled with ERR_INVALID_PARAM at line 2323, but execution continues without returning or using continue. The code then falls through to line 2325 and may call set_value again at line 2339 or 2341, triggering std::future_error and crashing the supervisor.

This follows the same pattern as a past review comment for lines 2319-2343 that was marked as addressed, but the issue has reappeared in this new handler.

Apply this diff:

  if (!m_step_.IsDaemonStep()) {
    CRANE_ERROR("Trying to move pid {} to no daemon step", pid);
    prom.set_value(CraneErrCode::ERR_INVALID_PARAM);
+   continue;
  }
  if (!m_step_.step_user_cg) {

2238-2250: Check StepInstance::Prepare() return value and fail tasks on error.

Line 2240 calls m_step_.Prepare() without checking the return value. When preparation fails due to CreateFoldersForFileEx, mkstemp, or fchown errors, execution continues silently. This leads to crashes downstream:

  • x11_meta remains std::nullopt if Prepare() fails before line 110, causing crashes at lines 683, 694-695, 701, 747, 1396, and 1398 when .x11_meta.value() is called
  • script_path remains empty if Prepare() fails before line 67, leading to executing empty commands at line 1335

Compare with LaunchExecution_() at line 1806, which properly checks task->Prepare() and calls TaskFinish_() on failure.

Based on past review comments, this critical issue remains unresolved.

Apply this diff:

  m_step_.pwd.Init(m_step_.uid);
  if (!m_step_.pwd.Valid()) {
    CRANE_ERROR("Failed to look up password entry for uid {}", m_step_.uid);
    for (auto task_id : m_step_.task_ids)
      TaskFinish_(task_id, crane::grpc::TaskStatus::Failed,
                  ExitCode::EC_PERMISSION_DENIED,
                  fmt::format("Failed to look up password entry for uid {}",
                              m_step_.uid));
    elem.ok_prom.set_value(CraneErrCode::ERR_SYSTEM_ERR);
    continue;
  }

- m_step_.Prepare();
+ auto prepare_err = m_step_.Prepare();
+ if (prepare_err != CraneErrCode::SUCCESS) {
+   CRANE_ERROR("Failed to prepare step: {}", static_cast<int>(prepare_err));
+   for (auto task_id : m_step_.task_ids)
+     TaskFinish_(task_id, crane::grpc::TaskStatus::Failed,
+                 ExitCode::EC_FILE_NOT_FOUND,
+                 fmt::format("Failed to prepare step, code: {}", 
+                             static_cast<int>(prepare_err)));
+   elem.ok_prom.set_value(prepare_err);
+   continue;
+ }
🧹 Nitpick comments (6)
scripts/wipe_data.py (2)

71-76: Use context manager for UnQLite to prevent resource leaks.

If an exception occurs between opening the database and calling close(), the handle will leak. UnQLite supports the context manager protocol.

     if os.path.exists(var_db_path):
         try:
-            db = UnQLite(var_db_path)
-            if next_task_id_key in db:
-                next_task_id_val = db[next_task_id_key]
-                logger.info(f"Found existing task ID data (len={len(next_task_id_val)})")
-            db.close()
+            with UnQLite(var_db_path) as db:
+                if next_task_id_key in db:
+                    next_task_id_val = db[next_task_id_key]
+                    logger.info(f"Found existing task ID data (len={len(next_task_id_val)})")
         except Exception as e:
             logger.warning(f"Failed to read task ID from {var_db_path}: {e}")

94-104: Apply context manager and improve exception handling.

Same resource leak concern here. Additionally, addressing the static analysis hints: use logging.exception to include traceback and bare raise for cleaner re-raise.

     if next_task_id_val is not None:
         try:
             if not db_dir.exists():
                 db_dir.mkdir(parents=True, exist_ok=True)
 
-            db = UnQLite(var_db_path)
-            db[next_task_id_key] = next_task_id_val
-            db.close()
+            with UnQLite(var_db_path) as db:
+                db[next_task_id_key] = next_task_id_val
             logger.info(f"Restored task ID to {var_db_path}")
         except Exception as e:
-            logger.error(f"Error restoring task ID to {var_db_path}: {e}")
-            raise e
+            logger.exception(f"Error restoring task ID to {var_db_path}")
+            raise
src/Utilities/PublicHeader/include/crane/String.h (1)

159-161: Consider using StepIdRange in JobStepsToString constraints.

The concept is well-defined. It could simplify the JobStepsToString requires clause on lines 165-167, replacing the explicit std::same_as<std::ranges::range_value_t<...>, step_id_t> check. If it's intended for external use only, this is fine as-is.

protos/Crane.proto (1)

841-854: Consider consistent naming between TaskExitStatus and TaskExitStatusReply.

Both messages have identical fields (task_id, exit_code, signaled). The naming difference (TaskExitStatus in StreamTaskIORequest vs TaskExitStatusReply in StreamCrunReply) is minor but could be unified for clarity. This is optional since proto3 allows this pattern.

src/Craned/Core/CranedServer.cpp (1)

194-195: system_flag is extracted but unused.

The system_flag value is parsed from GetIdsByPid but not utilized in this function. If this is intentional for future use, consider adding a comment. Otherwise, suppress the unused variable warning.

-      auto [job_id_opt, step_id_opt, system_flag, task_id_opt] =
+      auto [job_id_opt, step_id_opt, [[maybe_unused]] system_flag, task_id_opt] =
           pid_to_ids_expt.value();
src/Craned/Supervisor/TaskManager.h (1)

486-487: Consider aligning CheckStatusAsync with the other future‑based async APIs

CheckStatusAsync takes a raw CheckStatusReply*, while the internal queue stores std::promise<StepStatus>. The other RPC entry points (ExecuteTaskAsync, QueryStepEnvAsync, MigrateSshProcToCgroupAsync) expose a std::future and keep protobuf details at the gRPC layer.

For API consistency and clearer lifetimes, consider changing this to something like:

-  void CheckStatusAsync(crane::grpc::supervisor::CheckStatusReply* response);
+  std::future<StepStatus> CheckStatusAsync();

and have the gRPC handler fill the reply from the future, paralleling the existing patterns.

Also applies to: 571-573

📜 Review details

Configuration used: Path: .coderabbit.yaml

Review profile: CHILL

Plan: Pro

📥 Commits

Reviewing files that changed from the base of the PR and between 3a27a54 and 2fc52fc.

📒 Files selected for processing (31)
  • protos/Crane.proto (6 hunks)
  • protos/PublicDefs.proto (2 hunks)
  • protos/Supervisor.proto (2 hunks)
  • scripts/wipe_data.py (3 hunks)
  • src/CraneCtld/CtldPublicDefs.cpp (5 hunks)
  • src/CraneCtld/CtldPublicDefs.h (2 hunks)
  • src/CraneCtld/EmbeddedDbClient.cpp (1 hunks)
  • src/CraneCtld/TaskScheduler.cpp (7 hunks)
  • src/Craned/Common/CgroupManager.cpp (18 hunks)
  • src/Craned/Common/CgroupManager.h (18 hunks)
  • src/Craned/Core/CMakeLists.txt (1 hunks)
  • src/Craned/Core/Craned.cpp (12 hunks)
  • src/Craned/Core/CranedForPamServer.cpp (2 hunks)
  • src/Craned/Core/CranedServer.cpp (4 hunks)
  • src/Craned/Core/CtldClient.cpp (1 hunks)
  • src/Craned/Core/JobManager.cpp (17 hunks)
  • src/Craned/Core/JobManager.h (4 hunks)
  • src/Craned/Core/StepInstance.cpp (1 hunks)
  • src/Craned/Core/StepInstance.h (1 hunks)
  • src/Craned/Core/SupervisorStub.cpp (3 hunks)
  • src/Craned/Core/SupervisorStub.h (1 hunks)
  • src/Craned/Supervisor/CforedClient.cpp (13 hunks)
  • src/Craned/Supervisor/CforedClient.h (6 hunks)
  • src/Craned/Supervisor/Supervisor.cpp (1 hunks)
  • src/Craned/Supervisor/SupervisorPublicDefs.h (0 hunks)
  • src/Craned/Supervisor/SupervisorServer.cpp (3 hunks)
  • src/Craned/Supervisor/SupervisorServer.h (1 hunks)
  • src/Craned/Supervisor/TaskManager.cpp (30 hunks)
  • src/Craned/Supervisor/TaskManager.h (16 hunks)
  • src/Utilities/PublicHeader/include/crane/PublicHeader.h (1 hunks)
  • src/Utilities/PublicHeader/include/crane/String.h (2 hunks)
💤 Files with no reviewable changes (1)
  • src/Craned/Supervisor/SupervisorPublicDefs.h
🚧 Files skipped from review as they are similar to previous changes (8)
  • src/Craned/Supervisor/Supervisor.cpp
  • protos/PublicDefs.proto
  • src/Craned/Core/CtldClient.cpp
  • src/Craned/Core/StepInstance.h
  • src/Craned/Core/CranedForPamServer.cpp
  • src/CraneCtld/TaskScheduler.cpp
  • src/Craned/Core/CMakeLists.txt
  • src/Utilities/PublicHeader/include/crane/PublicHeader.h
🧰 Additional context used
🧠 Learnings (36)
📓 Common learnings
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Supervisor/SupervisorPublicDefs.h:32-37
Timestamp: 2025-05-26T11:06:28.796Z
Learning: The user L-Xiafeng prefers to defer refactoring duplicate definitions until they become a larger pattern in the codebase, rather than addressing individual instances immediately.
📚 Learning: 2025-05-09T01:54:21.256Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 496
File: src/CraneCtld/CranedKeeper.cpp:51-53
Timestamp: 2025-05-09T01:54:21.256Z
Learning: The ConfigureCraned function in src/CraneCtld/CranedKeeper.cpp is called from a thread pool, so there's no need to worry about it blocking the gRPC completion queue thread.

Applied to files:

  • src/CraneCtld/EmbeddedDbClient.cpp
  • src/Craned/Core/CranedServer.cpp
  • src/Craned/Core/SupervisorStub.cpp
  • src/CraneCtld/CtldPublicDefs.cpp
  • src/Craned/Supervisor/SupervisorServer.h
  • src/Craned/Core/JobManager.h
  • src/Craned/Core/Craned.cpp
  • src/Craned/Core/JobManager.cpp
  • src/Craned/Supervisor/TaskManager.cpp
  • src/Craned/Core/SupervisorStub.h
  • src/Craned/Supervisor/CforedClient.cpp
  • src/Craned/Supervisor/CforedClient.h
  • src/CraneCtld/CtldPublicDefs.h
  • src/Craned/Supervisor/TaskManager.h
  • src/Craned/Core/StepInstance.cpp
  • src/Craned/Supervisor/SupervisorServer.cpp
📚 Learning: 2025-05-25T04:08:03.273Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Supervisor/Supervisor.cpp:68-69
Timestamp: 2025-05-25T04:08:03.273Z
Learning: In the Crane supervisor component (src/Craned/Supervisor/), the supervisor process communicates with Craned through STDOUT using protobuf messages. The supervisor must not send any information to STDOUT before sending the "ready" message to Craned, as this would interfere with the inter-process communication protocol. Therefore, adding logging statements that might write to STDOUT before the ready message is sent could break the communication.

Applied to files:

  • src/Craned/Core/CranedServer.cpp
  • protos/Supervisor.proto
  • src/Craned/Core/SupervisorStub.cpp
  • src/Craned/Core/JobManager.h
  • src/Craned/Core/Craned.cpp
  • src/Craned/Core/JobManager.cpp
  • src/Craned/Supervisor/TaskManager.cpp
  • src/Craned/Core/SupervisorStub.h
  • src/Craned/Supervisor/TaskManager.h
  • src/Craned/Core/StepInstance.cpp
  • src/Craned/Supervisor/SupervisorServer.cpp
📚 Learning: 2025-06-30T08:43:44.470Z
Learnt from: huerni
Repo: PKUHPC/CraneSched PR: 537
File: src/Craned/TaskManager.cpp:145-147
Timestamp: 2025-06-30T08:43:44.470Z
Learning: In the CraneSched codebase, src/Craned/Craned.cpp guarantees that g_config.CranedRes.contains(g_config.CranedIdOfThisNode) through explicit validation during startup. The code checks if g_config.CranedRes.contains(g_config.Hostname) and exits the process if not found, then sets g_config.CranedIdOfThisNode = g_config.Hostname. TaskManager constructor is called after this validation, so g_config.CranedRes[g_config.CranedIdOfThisNode] is guaranteed to be valid.

Applied to files:

  • src/Craned/Core/CranedServer.cpp
  • src/CraneCtld/CtldPublicDefs.cpp
  • src/Craned/Core/JobManager.h
  • src/Craned/Core/Craned.cpp
  • src/Craned/Core/JobManager.cpp
  • src/Craned/Supervisor/TaskManager.cpp
  • src/Craned/Core/SupervisorStub.h
  • src/Craned/Supervisor/CforedClient.h
  • src/CraneCtld/CtldPublicDefs.h
  • src/Craned/Supervisor/TaskManager.h
  • src/Craned/Core/StepInstance.cpp
📚 Learning: 2025-04-02T10:11:33.562Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 461
File: src/Craned/CgroupManager.cpp:685-685
Timestamp: 2025-04-02T10:11:33.562Z
Learning: In the CgroupManager's GetJobBpfMapCgroupsV2 method, the developer has confirmed that cg_ino_job_id_map will always contain the key->cgroup_id element, making the CRANE_ASSERT check appropriate rather than requiring additional error handling.

Applied to files:

  • src/Craned/Core/CranedServer.cpp
  • src/Craned/Core/JobManager.h
  • src/Craned/Core/Craned.cpp
  • src/Craned/Core/JobManager.cpp
  • src/Craned/Supervisor/TaskManager.cpp
  • src/Craned/Common/CgroupManager.cpp
  • src/Craned/Core/StepInstance.cpp
  • src/Craned/Common/CgroupManager.h
📚 Learning: 2025-03-31T09:29:40.388Z
Learnt from: RileyWen
Repo: PKUHPC/CraneSched PR: 456
File: src/CraneCtld/Tls/VaultClientWrapper.cpp:116-137
Timestamp: 2025-03-31T09:29:40.388Z
Learning: In CraneSched, `phmap::parallel_flat_hash_set` from the Parallel Hashmap library (`parallel_hashmap/phmap.h`) is used for thread-safe containers. This container implements internal sharding with separate locks for different parts of the hash table, making it inherently thread-safe for concurrent operations without requiring external synchronization.

Applied to files:

  • src/Craned/Core/CranedServer.cpp
  • src/CraneCtld/CtldPublicDefs.h
📚 Learning: 2025-03-31T09:29:40.388Z
Learnt from: RileyWen
Repo: PKUHPC/CraneSched PR: 456
File: src/CraneCtld/Tls/VaultClientWrapper.cpp:116-137
Timestamp: 2025-03-31T09:29:40.388Z
Learning: In the CraneSched project, the `m_allowed_certs_` in `VaultClientWrapper` is implemented as a `phmap::parallel_flat_hash_set<std::string>`, which is a thread-safe container designed for concurrent access, making it safe to use in multithreaded contexts without additional synchronization.

Applied to files:

  • src/Craned/Core/CranedServer.cpp
  • src/CraneCtld/CtldPublicDefs.h
📚 Learning: 2025-03-31T09:29:40.388Z
Learnt from: RileyWen
Repo: PKUHPC/CraneSched PR: 456
File: src/CraneCtld/Tls/VaultClientWrapper.cpp:116-137
Timestamp: 2025-03-31T09:29:40.388Z
Learning: The `phmap::parallel_flat_hash_set` used in CraneSched's `VaultClientWrapper` is from the Parallel Hashmap library, which provides thread-safe containers for concurrent access. It uses internal partitioning to reduce contention, making it safe for use in multithreaded environments without external synchronization.

Applied to files:

  • src/Craned/Core/CranedServer.cpp
📚 Learning: 2025-06-07T10:47:59.071Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 520
File: src/CraneCtld/CranedKeeper.cpp:416-417
Timestamp: 2025-06-07T10:47:59.071Z
Learning: In src/CraneCtld/CranedKeeper.h, the m_shutting_down_ member in CranedStub class is declared as std::atomic_bool, making it thread-safe for concurrent access without additional synchronization.

Applied to files:

  • src/Craned/Core/CranedServer.cpp
  • src/Craned/Core/SupervisorStub.cpp
  • src/Craned/Core/SupervisorStub.h
  • src/Craned/Supervisor/CforedClient.h
  • src/CraneCtld/CtldPublicDefs.h
  • src/Craned/Supervisor/TaskManager.h
  • src/Craned/Core/StepInstance.cpp
📚 Learning: 2025-04-02T09:30:13.014Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 461
File: src/Craned/JobManager.cpp:141-149
Timestamp: 2025-04-02T09:30:13.014Z
Learning: In JobManager, if a uid exists in m_uid_to_job_ids_map_, its corresponding task_ids set is guaranteed to be non-empty due to the invariant maintained in AllocJobs and FreeJobs methods.

Applied to files:

  • src/Craned/Core/CranedServer.cpp
  • src/Craned/Common/CgroupManager.cpp
📚 Learning: 2025-07-01T08:00:05.383Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Supervisor/TaskManager.cpp:1-1809
Timestamp: 2025-07-01T08:00:05.383Z
Learning: In TaskManager.cpp, the security model relies on administrator-configured command templates in ParseOCICmdPattern_ and system-provided usernames in ParseFilePathPattern_, with file permissions using the user's own uid/gid for privilege isolation. The user L-Xiafeng considers this security boundary sufficient and chooses not to fix job name-related path issues at this location.

Applied to files:

  • src/Craned/Core/CranedServer.cpp
  • src/Craned/Supervisor/TaskManager.cpp
  • src/Craned/Common/CgroupManager.h
📚 Learning: 2025-05-02T07:06:36.103Z
Learnt from: RileyWen
Repo: PKUHPC/CraneSched PR: 475
File: src/CraneCtld/CtldGrpcServer.cpp:113-118
Timestamp: 2025-05-02T07:06:36.103Z
Learning: In CraneSched, gRPC methods should generally return Status::OK even when handling error conditions, as non-OK statuses cause the connection to terminate. Error information should be communicated within the RPC response payload instead.

Applied to files:

  • src/Craned/Core/CranedServer.cpp
  • src/Craned/Core/SupervisorStub.cpp
  • src/CraneCtld/CtldPublicDefs.cpp
  • src/Craned/Supervisor/SupervisorServer.h
  • src/Craned/Core/JobManager.h
  • src/Craned/Supervisor/TaskManager.cpp
  • src/Craned/Core/StepInstance.cpp
  • src/Craned/Supervisor/SupervisorServer.cpp
📚 Learning: 2025-05-26T11:04:30.580Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Craned/SupervisorKeeper.h:32-32
Timestamp: 2025-05-26T11:04:30.580Z
Learning: The Supervisor component in the Crane system is designed to manage only one task per instance. The task specification is provided to the Supervisor during startup by Craned (likely through InitSupervisorRequest), so the ExecuteTask() method doesn't need to accept task parameters since the Supervisor already knows which task to execute.

Applied to files:

  • src/Craned/Core/CranedServer.cpp
  • src/Craned/Core/SupervisorStub.cpp
  • src/Craned/Supervisor/SupervisorServer.h
  • protos/Crane.proto
  • src/Craned/Core/JobManager.h
  • src/Craned/Core/Craned.cpp
  • src/Craned/Core/JobManager.cpp
  • src/Craned/Supervisor/TaskManager.cpp
  • src/Craned/Core/SupervisorStub.h
  • src/Craned/Supervisor/TaskManager.h
  • src/Craned/Core/StepInstance.cpp
  • src/Craned/Supervisor/SupervisorServer.cpp
📚 Learning: 2025-05-02T07:12:46.896Z
Learnt from: RileyWen
Repo: PKUHPC/CraneSched PR: 475
File: src/Craned/CranedServer.h:0-0
Timestamp: 2025-05-02T07:12:46.896Z
Learning: The initialization of `m_supervisor_recovered_` to `true` in the CranedServer class is intentional despite the comment saying "When supervisor ready, init with false". This is temporary until the supervisor functionality is fully implemented.

Applied to files:

  • src/Craned/Core/SupervisorStub.cpp
  • src/Craned/Core/Craned.cpp
  • src/Craned/Supervisor/TaskManager.cpp
  • src/Craned/Core/SupervisorStub.h
  • src/Craned/Core/StepInstance.cpp
  • src/Craned/Supervisor/SupervisorServer.cpp
📚 Learning: 2025-05-26T11:04:56.055Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Craned/SupervisorKeeper.h:72-72
Timestamp: 2025-05-26T11:04:56.055Z
Learning: The CraneSched project prefers using global variables (like `g_supervisor_keeper`) over dependency injection patterns. The team does not follow dependency injection approaches for managing singleton instances.

Applied to files:

  • src/Craned/Core/SupervisorStub.cpp
  • src/Craned/Core/Craned.cpp
  • src/Craned/Core/SupervisorStub.h
  • src/Craned/Core/StepInstance.cpp
📚 Learning: 2025-05-08T07:39:29.207Z
Learnt from: huerni
Repo: PKUHPC/CraneSched PR: 469
File: src/Craned/TaskManager.cpp:1162-1175
Timestamp: 2025-05-08T07:39:29.207Z
Learning: In CraneSched, when a task fails in the LaunchTaskInstanceMt_ method, ActivateTaskStatusChangeAsync_ does not automatically release PMIx resources or cgroup. Explicit cleanup is required by calling g_pmix_server->DeregisterTask() and g_cg_mgr->ReleaseCgroupByTaskIdOnly().

Applied to files:

  • src/CraneCtld/CtldPublicDefs.cpp
  • src/Craned/Core/JobManager.h
  • src/Craned/Core/Craned.cpp
  • src/Craned/Core/JobManager.cpp
  • src/Craned/Supervisor/TaskManager.cpp
  • src/Craned/Supervisor/CforedClient.cpp
  • src/Craned/Supervisor/TaskManager.h
  • src/Craned/Core/StepInstance.cpp
📚 Learning: 2025-05-08T07:39:29.207Z
Learnt from: huerni
Repo: PKUHPC/CraneSched PR: 469
File: src/Craned/TaskManager.cpp:1162-1175
Timestamp: 2025-05-08T07:39:29.207Z
Learning: In CraneSched, when a task fails in the LaunchTaskInstanceMt_ method after PMIx registration, ActivateTaskStatusChangeAsync_ does not automatically release PMIx resources or cgroup resources. Explicit cleanup is required by calling g_pmix_server->DeregisterTask() and g_cg_mgr->ReleaseCgroupByTaskIdOnly() before returning from the function.

Applied to files:

  • src/CraneCtld/CtldPublicDefs.cpp
  • src/Craned/Core/JobManager.h
  • src/Craned/Core/Craned.cpp
  • src/Craned/Core/JobManager.cpp
  • src/Craned/Supervisor/TaskManager.cpp
  • src/Craned/Supervisor/CforedClient.cpp
  • src/CraneCtld/CtldPublicDefs.h
  • src/Craned/Supervisor/TaskManager.h
  • src/Craned/Core/StepInstance.cpp
📚 Learning: 2025-05-09T01:56:53.142Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 496
File: src/CraneCtld/CtldPublicDefs.h:756-763
Timestamp: 2025-05-09T01:56:53.142Z
Learning: In the CraneSched codebase, the `execution_node` field in JobToD is intentionally set to the first element of `executing_craned_ids` vector without guards, as it represents the main execution node for a job. This is by design and assumes `executing_craned_ids` is never empty when `GetJobOfNode` is called.

Applied to files:

  • src/CraneCtld/CtldPublicDefs.cpp
  • src/Craned/Core/JobManager.h
  • src/CraneCtld/CtldPublicDefs.h
📚 Learning: 2025-05-23T02:32:43.952Z
Learnt from: 1daidai1
Repo: PKUHPC/CraneSched PR: 458
File: src/CraneCtld/CtldPublicDefs.h:0-0
Timestamp: 2025-05-23T02:32:43.952Z
Learning: In the CraneSched project, allocated_res_view is handled/updated separately before calling SetAllocatedRes, so it does not need to be updated again within the method itself.

Applied to files:

  • src/CraneCtld/CtldPublicDefs.cpp
📚 Learning: 2025-05-23T02:32:43.952Z
Learnt from: 1daidai1
Repo: PKUHPC/CraneSched PR: 458
File: src/CraneCtld/CtldPublicDefs.h:0-0
Timestamp: 2025-05-23T02:32:43.952Z
Learning: In the CraneSched project, allocated_res_view is updated before calling SetAllocatedRes, so it doesn't need to be updated again within the method itself.

Applied to files:

  • src/CraneCtld/CtldPublicDefs.cpp
📚 Learning: 2025-05-26T11:00:54.563Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Craned/SupervisorKeeper.cpp:174-178
Timestamp: 2025-05-26T11:00:54.563Z
Learning: The CraneSched project uses C++23 standard, allowing the use of modern C++ features like std::ranges::to and other C++23 language features and library components.

Applied to files:

  • src/Utilities/PublicHeader/include/crane/String.h
📚 Learning: 2025-05-09T01:54:39.465Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 496
File: src/Craned/TaskManager.cpp:1509-1517
Timestamp: 2025-05-09T01:54:39.465Z
Learning: The CraneSched project uses C++23, which supports Class Template Argument Deduction (CTAD) for standard containers like std::set and includes ranges support, making std::ranges::views::keys valid without additional headers.

Applied to files:

  • src/Utilities/PublicHeader/include/crane/String.h
📚 Learning: 2025-04-02T09:52:59.318Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 461
File: src/Craned/CgroupManager.cpp:697-701
Timestamp: 2025-04-02T09:52:59.318Z
Learning: When using bpf_map__get_next_key function, memory must be properly allocated (e.g., with std::make_unique<BpfKey>()) before passing the pointer to the function, as it writes the key to the provided memory address.

Applied to files:

  • src/Craned/Core/Craned.cpp
  • src/Craned/Common/CgroupManager.cpp
📚 Learning: 2025-04-27T11:52:31.017Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 475
File: src/CraneCtld/CranedKeeper.cpp:40-62
Timestamp: 2025-04-27T11:52:31.017Z
Learning: In the CraneSched system, retry of configuration RPC is architecturally driven by the Craned's notification system rather than explicit retry code within the ConfigureCraned method. When Configure RPC fails, Craned returns to a notification state and sends new Notify messages which trigger new configuration attempts.

Applied to files:

  • src/Craned/Core/Craned.cpp
📚 Learning: 2025-06-23T07:53:30.513Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 520
File: src/Craned/CranedServer.cpp:50-51
Timestamp: 2025-06-23T07:53:30.513Z
Learning: In the CraneSched codebase, `g_ctld_client` is a guaranteed global variable that is always initialized before any gRPC service methods are called, so null pointer checks are not necessary when calling methods on it.

Applied to files:

  • src/Craned/Core/Craned.cpp
📚 Learning: 2025-05-25T04:11:50.268Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Supervisor/TaskManager.cpp:220-220
Timestamp: 2025-05-25T04:11:50.268Z
Learning: In TaskManager.cpp, when step->IsCrun() is checked before dynamic_cast to CrunMetaInExecution*, the cast is guaranteed to succeed due to the program logic ensuring the correct meta type is used for Crun tasks. Null checks for these dynamic_cast operations are unnecessary in this context.

Applied to files:

  • src/Craned/Core/JobManager.cpp
  • src/Craned/Supervisor/TaskManager.cpp
  • src/Craned/Supervisor/TaskManager.h
  • src/Craned/Core/StepInstance.cpp
📚 Learning: 2025-05-25T04:11:27.740Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Supervisor/TaskManager.cpp:396-396
Timestamp: 2025-05-25T04:11:27.740Z
Learning: In TaskManager.cpp, GetCrunMeta() calls don't need null checks because they're only called in contexts where the task is guaranteed to be a CRUN task (e.g., SetupChildProcessCrunX11_ is only called when step->IsCrun() && x11() conditions are met), ensuring the metadata will always be valid.

Applied to files:

  • src/Craned/Core/JobManager.cpp
  • src/Craned/Supervisor/TaskManager.cpp
  • src/Craned/Supervisor/TaskManager.h
  • src/Craned/Core/StepInstance.cpp
📚 Learning: 2025-05-08T07:38:42.362Z
Learnt from: huerni
Repo: PKUHPC/CraneSched PR: 469
File: src/Craned/TaskManager.cpp:0-0
Timestamp: 2025-05-08T07:38:42.362Z
Learning: In CraneSched's PMIx integration, the `g_pmix_server->SetupFork()` function must be called in the child process after fork() and before exec() to properly set up the PMIx environment variables.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
  • src/Craned/Supervisor/TaskManager.h
  • src/Craned/Core/StepInstance.cpp
📚 Learning: 2025-05-09T02:16:56.723Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 496
File: src/Craned/TaskManager.cpp:1498-1506
Timestamp: 2025-05-09T02:16:56.723Z
Learning: The `QueryRunningTasksAsync()` method in TaskManager.cpp is designed to never be called from inside the event loop thread, so there's no risk of deadlock with the synchronous `res.get()` call.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-09-21T11:26:40.935Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 625
File: src/Craned/Core/SupervisorKeeper.cpp:145-147
Timestamp: 2025-09-21T11:26:40.935Z
Learning: In C++17 and later, Class Template Argument Deduction (CTAD) allows `std::shared_ptr stub = std::make_shared<SupervisorStub>();` syntax to be valid, as the compiler can deduce the template parameter from make_shared. Projects using C++17+ don't need explicit template parameters in this context.

Applied to files:

  • src/Craned/Core/SupervisorStub.h
  • src/Craned/Core/StepInstance.cpp
📚 Learning: 2025-08-14T02:56:35.503Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 587
File: src/Craned/Supervisor/CforedClient.cpp:449-454
Timestamp: 2025-08-14T02:56:35.503Z
Learning: In CforedClient::AsyncSendRecvThread_(), the guard `if (state <= State::Registering) { continue; }` in the TIMEOUT branch only prevents premature cleanup when stopping before registration completes, but it doesn't block normal gRPC event processing. The completion queue will still deliver Prepare/Write/Read events that advance the state machine normally.

Applied to files:

  • src/Craned/Supervisor/CforedClient.cpp
📚 Learning: 2025-05-02T07:05:26.012Z
Learnt from: RileyWen
Repo: PKUHPC/CraneSched PR: 475
File: src/CraneCtld/CranedKeeper.cpp:601-602
Timestamp: 2025-05-02T07:05:26.012Z
Learning: In the CraneCtld codebase, the variables m_disconnected_ and m_registered_ in CranedStub class are already defined as std::atomic_bool, making them thread-safe for concurrent access without additional synchronization.

Applied to files:

  • src/Craned/Supervisor/CforedClient.h
  • src/CraneCtld/CtldPublicDefs.h
  • src/Craned/Supervisor/TaskManager.h
  • src/Craned/Core/StepInstance.cpp
📚 Learning: 2025-04-18T02:26:16.113Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 351
File: src/CraneCtld/CranedMetaContainer.cpp:248-264
Timestamp: 2025-04-18T02:26:16.113Z
Learning: The resource class in CraneSched includes assertions in its operator overloads (particularly in operator-=) that verify resources being subtracted are less than or equal to available resources, ensuring no negative values can occur during resource allocation or deallocation operations.

Applied to files:

  • src/Craned/Common/CgroupManager.cpp
📚 Learning: 2025-05-02T07:05:49.032Z
Learnt from: RileyWen
Repo: PKUHPC/CraneSched PR: 475
File: src/CraneCtld/CranedKeeper.cpp:503-506
Timestamp: 2025-05-02T07:05:49.032Z
Learning: In CraneCtld/CranedKeeper.cpp, using m_unavail_craned_set_.at() is intentional as the key is guaranteed to exist by design, and crashing on missing key is preferred to silently handling it (fail-fast approach).

Applied to files:

  • src/CraneCtld/CtldPublicDefs.h
📚 Learning: 2025-08-12T08:58:39.772Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 577
File: src/Craned/Supervisor/TaskManager.cpp:124-130
Timestamp: 2025-08-12T08:58:39.772Z
Learning: In the CraneSched project using C++23, Class Template Argument Deduction (CTAD) allows std::unique_ptr declarations without explicit template parameters when the type can be deduced from the initializer, such as `std::unique_ptr task = std::move(m_task_map_.at(task_id))` where the template parameter is deduced from the move operation.

Applied to files:

  • src/CraneCtld/CtldPublicDefs.h
📚 Learning: 2025-05-08T09:35:39.809Z
Learnt from: huerni
Repo: PKUHPC/CraneSched PR: 469
File: src/Craned/Pmix/PmixCollRing.cpp:0-0
Timestamp: 2025-05-08T09:35:39.809Z
Learning: In the PMIx implementation for CraneSched, objects referenced in asynchronous gRPC callbacks (like `coll_ring_ctx`) remain valid as long as the parent object (`this`) is not destroyed. The `Coll` class uses shared_ptr management to ensure its lifetime extends through all pending callbacks.

Applied to files:

  • src/Craned/Supervisor/TaskManager.h
  • src/Craned/Core/StepInstance.cpp
🧬 Code graph analysis (10)
src/Craned/Core/SupervisorStub.cpp (2)
src/Craned/Core/SupervisorStub.h (1)
  • SupervisorStub (28-61)
src/Craned/Supervisor/TaskManager.h (2)
  • job_id (52-52)
  • step_id (53-53)
src/CraneCtld/CtldPublicDefs.cpp (1)
src/Craned/Supervisor/TaskManager.h (1)
  • job_id (52-52)
src/Utilities/PublicHeader/include/crane/String.h (2)
src/Utilities/PublicHeader/include/crane/Logger.h (2)
  • std (187-197)
  • crane (200-212)
src/Utilities/PublicHeader/String.cpp (2)
  • StepToDIdString (519-521)
  • StepToDIdString (519-519)
src/Craned/Core/JobManager.cpp (1)
src/Craned/Core/CranedServer.cpp (2)
  • QuerySshStepEnvVariables (297-318)
  • QuerySshStepEnvVariables (297-300)
src/Craned/Core/SupervisorStub.h (3)
src/Craned/Supervisor/TaskManager.h (2)
  • job_id (52-52)
  • step_id (53-53)
src/Craned/Core/CtldClient.cpp (1)
  • job_id_t (575-583)
src/Craned/Supervisor/SupervisorServer.cpp (8)
  • CheckStatus (60-66)
  • CheckStatus (60-63)
  • TerminateTask (83-93)
  • TerminateTask (83-86)
  • ChangeTaskTimeLimit (68-81)
  • ChangeTaskTimeLimit (68-71)
  • ShutdownSupervisor (105-112)
  • ShutdownSupervisor (105-108)
src/Craned/Supervisor/CforedClient.h (1)
src/Craned/Supervisor/CforedClient.cpp (8)
  • InitUvX11FwdHandler (109-115)
  • InitUvX11FwdHandler (109-109)
  • TaskProcessStop (714-727)
  • TaskProcessStop (714-715)
  • TaskOutPutForward (738-745)
  • TaskOutPutForward (738-739)
  • TaskX11OutPutForward (747-758)
  • TaskX11OutPutForward (747-749)
src/CraneCtld/CtldPublicDefs.h (1)
src/Utilities/PublicHeader/PublicHeader.cpp (1)
  • ResourceInNode (484-487)
src/Craned/Core/StepInstance.cpp (1)
src/Craned/Supervisor/TaskManager.cpp (4)
  • CleanUp (117-159)
  • CleanUp (117-117)
  • GotNewStatus (232-307)
  • GotNewStatus (232-232)
src/Craned/Common/CgroupManager.h (1)
src/Craned/Common/CgroupManager.cpp (19)
  • SetCpuBind (1012-1031)
  • SetCpuBind (1012-1012)
  • SetCpuBind (1317-1335)
  • SetCpuBind (1317-1317)
  • KillAllProcesses (1087-1114)
  • KillAllProcesses (1087-1087)
  • KillAllProcesses (1516-1544)
  • KillAllProcesses (1516-1516)
  • RecoverFromResInNode (1442-1492)
  • RecoverFromResInNode (1442-1443)
  • EraseBpfDeviceMap (1494-1513)
  • EraseBpfDeviceMap (1494-1494)
  • CgroupStrByParsedIds (296-308)
  • CgroupStrByParsedIds (296-296)
  • CgroupStrParsedIds (636-676)
  • GetIdsFromCgroupV2_ (580-600)
  • GetIdsFromCgroupV2_ (580-581)
  • GetCgInoJobIdMapCgroupV2_ (602-632)
  • GetCgInoJobIdMapCgroupV2_ (603-604)
src/Craned/Supervisor/SupervisorServer.cpp (1)
src/Craned/Core/CranedForPamServer.cpp (2)
  • MigrateSshProcToCgroup (181-205)
  • MigrateSshProcToCgroup (181-184)
🪛 Ruff (0.14.5)
scripts/wipe_data.py

77-77: Do not catch blind exception: Exception

(BLE001)


103-103: Use logging.exception instead of logging.error

Replace with exception

(TRY400)


104-104: Use raise without specifying exception name

Remove exception name

(TRY201)

Comment on lines +467 to +492
CraneExpected<void> JobManager::ChangeStepTimelimit(job_id_t job_id,
step_id_t step_id,
int64_t new_timelimit_sec) {
auto job = m_job_map_.GetValueExclusivePtr(job_id);
if (!job) {
CRANE_ERROR("[Step #{}.{}] Failed to find job allocation", job_id, step_id);
return std::unexpected{CraneErrCode::ERR_NON_EXISTENT};
}
absl::MutexLock lock(job->step_map_mtx.get());
auto step_it = job->step_map.find(step_id);
if (step_it == job->step_map.end()) {
CRANE_ERROR("[Step #{}.{}] Failed to find step allocation", job_id,
step_id);
return std::unexpected{CraneErrCode::ERR_NON_EXISTENT};
}
auto& stub = step_it->second->supervisor_stub;
auto err = stub->ChangeTaskTimeLimit(absl::Seconds(new_timelimit_sec));
if (err != CraneErrCode::SUCCESS) {
CRANE_ERROR(
"[Step #{}.{}] Failed to change step timelimit to {} seconds via "
"supervisor RPC",
job_id, step_id, new_timelimit_sec);
return std::unexpected{err};
}
return {};
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

⚠️ Potential issue | 🟠 Major

Guard ChangeStepTimelimit against missing job/step and null supervisor_stub

The method now checks for missing job and step allocations, but it still dereferences step_it->second->supervisor_stub without verifying it’s initialized. For steps whose supervisor failed to start (or recovery gaps), this will crash craned.

Recommend adding a stub check while holding the same mutex:

  auto& stub = step_it->second->supervisor_stub;
-  auto err = stub->ChangeTaskTimeLimit(absl::Seconds(new_timelimit_sec));
+  if (!stub) {
+    CRANE_ERROR("[Step #{}.{}] Supervisor stub is null when changing "
+                "timelimit to {} seconds.", job_id, step_id,
+                new_timelimit_sec);
+    return std::unexpected{CraneErrCode::ERR_NON_EXISTENT};
+  }
+
+  auto err = stub->ChangeTaskTimeLimit(absl::Seconds(new_timelimit_sec));
🤖 Prompt for AI Agents
In src/Craned/Core/JobManager.cpp around lines 467-492, the code dereferences
step_it->second->supervisor_stub without checking it for null; hold the same
mutex, read the supervisor_stub into a local pointer/reference, and if it's null
log an error like "[Step #{}.{}] Supervisor stub not initialized" and return an
appropriate std::unexpected{CraneErrCode::...} (use the project error code that
represents unavailable/missing supervisor), otherwise proceed to call
ChangeTaskTimeLimit on the valid stub.

Comment on lines +163 to +172
template <MapRange Map>
requires std::ranges::range<Map> && requires(Map m) {
requires std::ranges::range<typename Map::mapped_type>;
requires std::same_as<std::ranges::range_value_t<typename Map::mapped_type>,
step_id_t>;
}
std::string JobStepsToString(const Map &m) {
typename std::ranges::range_value_t<Map>::second_type values;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

⚠️ Potential issue | 🟡 Minor

Remove unused variable and redundant constraint.

  1. Line 170: The variable values is declared but never used — this is dead code that will trigger compiler warnings.
  2. Line 164: std::ranges::range<Map> is redundant since MapRange already requires this.

Apply this diff:

 template <MapRange Map>
-  requires std::ranges::range<Map> && requires(Map m) {
+  requires requires(Map m) {
     requires std::ranges::range<typename Map::mapped_type>;
     requires std::same_as<std::ranges::range_value_t<typename Map::mapped_type>,
                           step_id_t>;
   }
 std::string JobStepsToString(const Map &m) {
-  typename std::ranges::range_value_t<Map>::second_type values;
   auto step_strs_view =
🤖 Prompt for AI Agents
In src/Utilities/PublicHeader/include/crane/String.h around lines 163 to 170,
remove the unused local variable `values` on line 170 and delete the redundant
constraint `requires std::ranges::range<Map>` on line 164 (since MapRange
already enforces that); update the template/constraints to only keep the
necessary requirements and remove the dead declaration so the function compiles
without warnings.

Copy link
Contributor

@coderabbitai coderabbitai bot left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Actionable comments posted: 0

Caution

Some comments are outside the diff and can’t be posted inline due to platform limitations.

⚠️ Outside diff range comments (1)
src/Craned/Supervisor/CforedClient.cpp (1)

354-379: Data race: TaskOutputFinishNoLock_ accesses m_fwd_meta_map without holding the mutex.

TaskOutputFinishNoLock_ (lines 710-713) accesses m_fwd_meta_map without acquiring m_mtx_, but it's called from CleanStopTaskIOQueueCb_ (line 373) after the lock has been released at line 369. This creates a race condition with other threads that may be modifying m_fwd_meta_map concurrently.

The function name suggests it should be called while the lock is already held, but the current code structure doesn't ensure this.

Option 1: Move the call inside the lock scope:

-      close(it->second.stdout_read);
-    }
-
-    CRANE_DEBUG("[Task #{}] Finished its output.", task_id);
-
-    bool ok_to_free = this->TaskOutputFinishNoLock_(task_id);
+      close(it->second.stdout_read);
+      CRANE_DEBUG("[Task #{}] Finished its output.", task_id);
+      ok_to_free = this->TaskOutputFinishNoLock_(task_id);
+    }

Option 2: Have TaskOutputFinishNoLock_ acquire the lock internally (and rename accordingly).

Also applies to: 710-713

♻️ Duplicate comments (3)
src/Craned/Supervisor/TaskManager.cpp (3)

2326-2329: Prevent double set_value crash when step is non-daemon.

If !m_step_.IsDaemonStep() at line 2326, the code sets the promise value at line 2328 but does not return or continue. Execution falls through to the cgroup allocation/migration logic (lines 2330-2347), which may call prom.set_value() again at line 2344 or 2346. Calling set_value() twice on a std::promise throws std::future_error and crashes the supervisor.

Apply this diff to skip the rest of the handler:

  if (!m_step_.IsDaemonStep()) {
    CRANE_ERROR("Trying to move pid {} to no daemon step", pid);
    prom.set_value(CraneErrCode::ERR_INVALID_PARAM);
+   continue;
  }

2245-2245: Propagate StepInstance::Prepare() failures to prevent downstream crashes.

The call to m_step_.Prepare() at line 2245 ignores the return value. If preparation fails (due to script creation, directory creation, mkstemp, or fchown errors), execution continues and later code assumes script_path and x11_meta are valid. This can lead to:

  • Line 1340-1345: Executing with empty script_path (empty command)
  • Lines 452-460, 692-704, 710: Calling .value() on unset x11_meta optional (crashes)

In contrast, LaunchExecution_() at line 1811 properly checks Prepare() return and calls TaskFinish_() on failure. Use the same pattern here.

Apply this diff to check the return value:

  m_step_.pwd.Init(m_step_.uid);
  if (!m_step_.pwd.Valid()) {
    CRANE_ERROR("Failed to look up password entry for uid {}", m_step_.uid);
    for (auto task_id : m_step_.task_ids)
      TaskFinish_(task_id, crane::grpc::TaskStatus::Failed,
                  ExitCode::EC_PERMISSION_DENIED,
                  fmt::format("Failed to look up password entry for uid {}",
                              m_step_.uid));
    elem.ok_prom.set_value(CraneErrCode::ERR_SYSTEM_ERR);
    continue;
  }

- m_step_.Prepare();
+ auto prep_err = m_step_.Prepare();
+ if (prep_err != CraneErrCode::SUCCESS) {
+   CRANE_ERROR("Failed to prepare step: {}", static_cast<int>(prep_err));
+   for (auto task_id : m_step_.task_ids)
+     TaskFinish_(task_id, crane::grpc::TaskStatus::Failed,
+                 ExitCode::EC_FILE_NOT_FOUND,
+                 fmt::format("Failed to prepare step, code: {}",
+                             static_cast<int>(prep_err)));
+   elem.ok_prom.set_value(prep_err);
+   continue;
+ }

98-112: File descriptor leak: xauth_fd never closed.

The file descriptor returned by mkstemp() at line 99 is never closed. On the error path (line 107), the function returns without closing it. On the success path (line 112), it also returns without closing the FD. This leaks one file descriptor per step execution.

Apply this diff to close the FD on both paths:

  int ret = fchown(xauth_fd, pwd.Uid(), pwd.Gid());
  if (ret == -1) {
    CRANE_ERROR("fchown() for xauth file failed: {}\n", strerror(errno));
+   close(xauth_fd);
    return CraneErrCode::ERR_SYSTEM_ERR;
  }
  x11_meta.x11_auth_path = x11_auth_path;
  this->x11_meta = std::move(x11_meta);
+ close(xauth_fd);
  return CraneErrCode::SUCCESS;
🧹 Nitpick comments (1)
src/Craned/Supervisor/CforedClient.cpp (1)

629-643: Hardcoded x11_local_id = 0 limits X11 to single connection.

The x11_local_id is hardcoded to 0 (line 630), and eof handling is commented out (lines 631-632). This aligns with the single-connection limitation noted at line 197-199, but the outbound path now supports per-connection IDs via next_x11_id_++.

If multi-connection X11 support is planned, these values should come from the incoming message payload rather than being hardcoded.

📜 Review details

Configuration used: Path: .coderabbit.yaml

Review profile: CHILL

Plan: Pro

📥 Commits

Reviewing files that changed from the base of the PR and between 2fc52fc and c566be8.

📒 Files selected for processing (2)
  • src/Craned/Supervisor/CforedClient.cpp (13 hunks)
  • src/Craned/Supervisor/TaskManager.cpp (29 hunks)
🧰 Additional context used
🧠 Learnings (16)
📓 Common learnings
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Supervisor/SupervisorPublicDefs.h:32-37
Timestamp: 2025-05-26T11:06:28.796Z
Learning: The user L-Xiafeng prefers to defer refactoring duplicate definitions until they become a larger pattern in the codebase, rather than addressing individual instances immediately.
📚 Learning: 2025-05-08T07:39:29.207Z
Learnt from: huerni
Repo: PKUHPC/CraneSched PR: 469
File: src/Craned/TaskManager.cpp:1162-1175
Timestamp: 2025-05-08T07:39:29.207Z
Learning: In CraneSched, when a task fails in the LaunchTaskInstanceMt_ method, ActivateTaskStatusChangeAsync_ does not automatically release PMIx resources or cgroup. Explicit cleanup is required by calling g_pmix_server->DeregisterTask() and g_cg_mgr->ReleaseCgroupByTaskIdOnly().

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
  • src/Craned/Supervisor/CforedClient.cpp
📚 Learning: 2025-05-08T07:39:29.207Z
Learnt from: huerni
Repo: PKUHPC/CraneSched PR: 469
File: src/Craned/TaskManager.cpp:1162-1175
Timestamp: 2025-05-08T07:39:29.207Z
Learning: In CraneSched, when a task fails in the LaunchTaskInstanceMt_ method after PMIx registration, ActivateTaskStatusChangeAsync_ does not automatically release PMIx resources or cgroup resources. Explicit cleanup is required by calling g_pmix_server->DeregisterTask() and g_cg_mgr->ReleaseCgroupByTaskIdOnly() before returning from the function.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
  • src/Craned/Supervisor/CforedClient.cpp
📚 Learning: 2025-05-26T11:04:30.580Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Craned/SupervisorKeeper.h:32-32
Timestamp: 2025-05-26T11:04:30.580Z
Learning: The Supervisor component in the Crane system is designed to manage only one task per instance. The task specification is provided to the Supervisor during startup by Craned (likely through InitSupervisorRequest), so the ExecuteTask() method doesn't need to accept task parameters since the Supervisor already knows which task to execute.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-05-25T04:11:50.268Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Supervisor/TaskManager.cpp:220-220
Timestamp: 2025-05-25T04:11:50.268Z
Learning: In TaskManager.cpp, when step->IsCrun() is checked before dynamic_cast to CrunMetaInExecution*, the cast is guaranteed to succeed due to the program logic ensuring the correct meta type is used for Crun tasks. Null checks for these dynamic_cast operations are unnecessary in this context.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-05-02T07:12:46.896Z
Learnt from: RileyWen
Repo: PKUHPC/CraneSched PR: 475
File: src/Craned/CranedServer.h:0-0
Timestamp: 2025-05-02T07:12:46.896Z
Learning: The initialization of `m_supervisor_recovered_` to `true` in the CranedServer class is intentional despite the comment saying "When supervisor ready, init with false". This is temporary until the supervisor functionality is fully implemented.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-04-02T10:11:33.562Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 461
File: src/Craned/CgroupManager.cpp:685-685
Timestamp: 2025-04-02T10:11:33.562Z
Learning: In the CgroupManager's GetJobBpfMapCgroupsV2 method, the developer has confirmed that cg_ino_job_id_map will always contain the key->cgroup_id element, making the CRANE_ASSERT check appropriate rather than requiring additional error handling.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-06-30T08:43:44.470Z
Learnt from: huerni
Repo: PKUHPC/CraneSched PR: 537
File: src/Craned/TaskManager.cpp:145-147
Timestamp: 2025-06-30T08:43:44.470Z
Learning: In the CraneSched codebase, src/Craned/Craned.cpp guarantees that g_config.CranedRes.contains(g_config.CranedIdOfThisNode) through explicit validation during startup. The code checks if g_config.CranedRes.contains(g_config.Hostname) and exits the process if not found, then sets g_config.CranedIdOfThisNode = g_config.Hostname. TaskManager constructor is called after this validation, so g_config.CranedRes[g_config.CranedIdOfThisNode] is guaranteed to be valid.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-05-25T04:11:27.740Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Supervisor/TaskManager.cpp:396-396
Timestamp: 2025-05-25T04:11:27.740Z
Learning: In TaskManager.cpp, GetCrunMeta() calls don't need null checks because they're only called in contexts where the task is guaranteed to be a CRUN task (e.g., SetupChildProcessCrunX11_ is only called when step->IsCrun() && x11() conditions are met), ensuring the metadata will always be valid.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-05-25T04:08:03.273Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Supervisor/Supervisor.cpp:68-69
Timestamp: 2025-05-25T04:08:03.273Z
Learning: In the Crane supervisor component (src/Craned/Supervisor/), the supervisor process communicates with Craned through STDOUT using protobuf messages. The supervisor must not send any information to STDOUT before sending the "ready" message to Craned, as this would interfere with the inter-process communication protocol. Therefore, adding logging statements that might write to STDOUT before the ready message is sent could break the communication.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-07-01T08:00:05.383Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Supervisor/TaskManager.cpp:1-1809
Timestamp: 2025-07-01T08:00:05.383Z
Learning: In TaskManager.cpp, the security model relies on administrator-configured command templates in ParseOCICmdPattern_ and system-provided usernames in ParseFilePathPattern_, with file permissions using the user's own uid/gid for privilege isolation. The user L-Xiafeng considers this security boundary sufficient and chooses not to fix job name-related path issues at this location.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-05-08T07:38:42.362Z
Learnt from: huerni
Repo: PKUHPC/CraneSched PR: 469
File: src/Craned/TaskManager.cpp:0-0
Timestamp: 2025-05-08T07:38:42.362Z
Learning: In CraneSched's PMIx integration, the `g_pmix_server->SetupFork()` function must be called in the child process after fork() and before exec() to properly set up the PMIx environment variables.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-05-09T01:54:21.256Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 496
File: src/CraneCtld/CranedKeeper.cpp:51-53
Timestamp: 2025-05-09T01:54:21.256Z
Learning: The ConfigureCraned function in src/CraneCtld/CranedKeeper.cpp is called from a thread pool, so there's no need to worry about it blocking the gRPC completion queue thread.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
  • src/Craned/Supervisor/CforedClient.cpp
📚 Learning: 2025-05-02T07:06:36.103Z
Learnt from: RileyWen
Repo: PKUHPC/CraneSched PR: 475
File: src/CraneCtld/CtldGrpcServer.cpp:113-118
Timestamp: 2025-05-02T07:06:36.103Z
Learning: In CraneSched, gRPC methods should generally return Status::OK even when handling error conditions, as non-OK statuses cause the connection to terminate. Error information should be communicated within the RPC response payload instead.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-05-09T02:16:56.723Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 496
File: src/Craned/TaskManager.cpp:1498-1506
Timestamp: 2025-05-09T02:16:56.723Z
Learning: The `QueryRunningTasksAsync()` method in TaskManager.cpp is designed to never be called from inside the event loop thread, so there's no risk of deadlock with the synchronous `res.get()` call.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-08-14T02:56:35.503Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 587
File: src/Craned/Supervisor/CforedClient.cpp:449-454
Timestamp: 2025-08-14T02:56:35.503Z
Learning: In CforedClient::AsyncSendRecvThread_(), the guard `if (state <= State::Registering) { continue; }` in the TIMEOUT branch only prevents premature cleanup when stopping before registration completes, but it doesn't block normal gRPC event processing. The completion queue will still deliver Prepare/Write/Read events that advance the state machine normally.

Applied to files:

  • src/Craned/Supervisor/CforedClient.cpp
🧬 Code graph analysis (2)
src/Craned/Supervisor/TaskManager.cpp (6)
src/Craned/Common/CgroupManager.h (1)
  • CgroupManager (532-640)
src/Craned/Core/CranedPublicDefs.h (1)
  • g_config (156-156)
src/Utilities/PublicHeader/OS.cpp (2)
  • CreateFoldersForFileEx (109-148)
  • CreateFoldersForFileEx (109-110)
src/Craned/Supervisor/SupervisorServer.h (1)
  • Wait (80-80)
src/Craned/Supervisor/CforedClient.h (1)
  • exit_code (131-131)
src/Craned/Supervisor/SupervisorPublicDefs.h (1)
  • exit_code (37-37)
src/Craned/Supervisor/CforedClient.cpp (3)
src/Craned/Supervisor/CforedClient.h (3)
  • next_x11_id_ (150-150)
  • exit_code (131-131)
  • signaled (132-133)
src/Utilities/PublicHeader/String.cpp (2)
  • SetCurrentThreadName (354-362)
  • SetCurrentThreadName (354-354)
src/Craned/Supervisor/SupervisorPublicDefs.h (1)
  • exit_code (37-37)
🔇 Additional comments (12)
src/Craned/Supervisor/CforedClient.cpp (8)

74-86: Assertion may fail if tasks are still registered during destruction.

The assertion CRANE_ASSERT(m_fwd_meta_map.empty()) at line 83 will crash the process if any tasks remain registered when the destructor runs. While m_stopped_ is set and threads are joined, there's no explicit mechanism to wait for all tasks to be unregistered from m_fwd_meta_map before this check.

Consider either:

  1. Adding a wait condition that blocks until m_fwd_meta_map is empty, or
  2. Converting this to a warning log instead of an assertion if incomplete cleanup is acceptable during forced shutdown.

151-200: LGTM on per-connection X11 state management.

The per-connection X11 handling via m_x11_fd_info_map_ with x11_local_id is well-structured. The lifecycle is clean: create on accept, store in map, erase on close. The shared_ptr ensures the X11FdInfo stays alive through all event handlers.

Minor observation: The comment at lines 197-199 indicates only one X11 connection is accepted per proxy. This is an intentional design choice worth noting for future maintainers.


431-439: Commented-out X11 payload fields: are these pending implementation?

The X11FwdRequest stores x11_id and eof (captured at lines 754-757), but these fields are not sent in the gRPC payload (lines 435-438 are commented out). This means the receiving end won't know which X11 connection the data belongs to or when the connection ends.

If the protocol doesn't yet support these fields, consider adding a TODO comment. Otherwise, if these fields should be transmitted, uncomment the lines.


545-558: LGTM on cfored connection failure handling.

The error handling correctly iterates over all tasks in m_fwd_meta_map while holding the mutex and enqueues them for cleanup before triggering task termination. The logging at line 553 provides good diagnostics.


715-728: LGTM on TaskProcessStop signature update.

The updated signature correctly captures exit_code and signaled status, and enqueues the exit status for forwarding to cfored. The mutex is properly acquired before accessing m_fwd_meta_map.


730-737: LGTM on TaskEnd implementation.

The task is properly unregistered from m_fwd_meta_map with proper locking before triggering the status change callback.


739-759: LGTM on output forwarding methods.

Both methods correctly use move semantics for the unique_ptr<char[]> data and enqueue to the thread-safe m_task_fwd_req_queue_. The structured FwdRequest approach cleanly handles different payload types.


68-71: LGTM on thread naming.

Thread names "CforedClient" (12 chars) and "CforedSendRecv" (14 chars) are both within the 16-character limit (including null terminator) per the SetCurrentThreadName implementation.

Also applies to: 468-468

src/Craned/Supervisor/TaskManager.cpp (4)

232-307: LGTM: Status transition validation logic is correct.

The state machine properly validates transitions for both daemon and non-daemon steps. Daemon steps transition Configuring → Running, while non-daemon steps transition Configuring → Starting → Running. Invalid transitions trigger warnings without halting execution, which is appropriate for a monitoring/debugging system.


1600-1702: LGTM: Async lifecycle handlers properly initialized.

The new async handles for supervisor initialization, shutdown, status checking, and SSH process migration are correctly set up with appropriate callbacks. The timer adjustment at line 1626 (1s interval) appears intentional for the supervision polling cycle.


2257-2304: LGTM: Per-task parallel execution logic is well-structured.

The per-task instance creation (lines 2215-2225) and parallel execution with thread pool (lines 2280-2300) are properly implemented:

  • Latch synchronization ensures all tasks complete before proceeding
  • Mutex protection for shared error state and exec_id map updates
  • Step-level cgroup allocated before task launch (lines 2257-2272)
  • Proper error propagation from worker threads

380-394: Confirm that task_res_map().at(task_id) is safely accessible for all tasks in the execution loop.

The code at line 385 uses .at(task_id) without bounds checking. While task IDs are consistently used from m_step_.task_ids (extracted from task_res_map().keys at construction), there is a time gap between step initialization and task execution. The original review marked this as "✅ Addressed", but the code still shows the undefended .at() call.

Given that the PR objectives explicitly list "Scheduler needs to fill in the task res field" as an outstanding item, the task_res_map may not be fully populated at execution time.

Verify: Has the scheduler been updated to guarantee that all task IDs in m_step_.task_ids have corresponding entries in task_res_map() before LaunchExecution_() is called? If not, add defensive checking (e.g., .find() with error handling) at line 385.

Copy link
Contributor

@coderabbitai coderabbitai bot left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Actionable comments posted: 4

Caution

Some comments are outside the diff and can’t be posted inline due to platform limitations.

⚠️ Outside diff range comments (1)
src/Craned/Core/CranedServer.cpp (1)

330-337: Same incorrect error-checking pattern on err.error().

This has the same bug as QuerySshStepEnvVariables: CraneExpected<void> uses std::expected, so .error() returns the error value (only valid when there's an error), not a boolean. Use !err.has_value() instead.

-  if (err.error()) {
+  if (!err.has_value()) {
     CRANE_ERROR("[Step #{}.{}] Failed to change task time limit",
                 request->task_id(), kPrimaryStepId);
     return Status::OK;
   }
♻️ Duplicate comments (20)
src/Utilities/PublicHeader/include/crane/String.h (1)

163-170: Issues previously flagged remain unaddressed.

The unused variable values on line 170 and the redundant std::ranges::range<Map> constraint on line 164 are still present. Please see the previous review comment for the suggested fix.

src/Craned/Supervisor/CforedClient.h (1)

29-36: Thread-safety concern for x11_input_stopped persists.

The x11_input_stopped field is a plain bool while sock_stopped is std::atomic<bool>. If x11_input_stopped is accessed from multiple threads (e.g., the event loop and gRPC threads), it needs consistent synchronization—either make it std::atomic<bool> or ensure all accesses are guarded by m_mtx_.

src/Craned/Common/CgroupManager.cpp (1)

319-330: RE2 cannot capture directly into std::optional<T>* - compile/runtime issue.

RE2's capture APIs expect pointers to POD types (int, long long, std::string), not std::optional<T>. This code will either fail to compile or produce incorrect results at runtime.

Capture into plain temporaries first, then populate the optionals:

-  std::optional<job_id_t> job_id;
-  std::optional<step_id_t> step_id;
-  std::optional<task_id_t> task_id;
-  std::optional<std::string> system_or_user;
+  job_id_t job_id_val = 0;
+  step_id_t step_id_val = 0;
+  task_id_t task_id_val = 0;
+  std::string system_or_user_str;
   CgroupStrParsedIds parsed_ids{};
-  if (RE2::FullMatch(cgroup_str, *cg_pattern, &job_id, &step_id,
-                     &system_or_user, &task_id)) {
-    std::get<CgConstant::KParsedJobIdIdx>(parsed_ids) = job_id;
-    std::get<CgConstant::KParsedStepIdIdx>(parsed_ids) = step_id;
-    std::get<CgConstant::KParsedSystemFlagIdx>(parsed_ids) =
-        system_or_user.has_value() && system_or_user.value() == "system";
-    std::get<CgConstant::KParsedTaskIdIdx>(parsed_ids) = task_id;
+  if (RE2::FullMatch(cgroup_str, *cg_pattern, &job_id_val, &step_id_val,
+                     &system_or_user_str, &task_id_val)) {
+    if (job_id_val) std::get<CgConstant::KParsedJobIdIdx>(parsed_ids) = job_id_val;
+    if (step_id_val) std::get<CgConstant::KParsedStepIdIdx>(parsed_ids) = step_id_val;
+    std::get<CgConstant::KParsedSystemFlagIdx>(parsed_ids) =
+        !system_or_user_str.empty() && system_or_user_str == "system";
+    if (task_id_val) std::get<CgConstant::KParsedTaskIdIdx>(parsed_ids) = task_id_val;
src/Craned/Supervisor/SupervisorServer.cpp (1)

44-56: std::expected template deduction and unsafe std::unreachable() remain.

This issue was flagged in a previous review:

  1. Line 46: std::expected env_expt = env_future.get(); - Class Template Argument Deduction (CTAD) for std::expected requires explicit template parameters. Use CraneExpected<EnvMap> or the full type.

  2. Lines 54-55: std::unreachable() causes undefined behavior if executed. Even if failure is "impossible now," this is fragile. Log and return gracefully instead.

-  std::expected env_expt = env_future.get();
+  CraneExpected<EnvMap> env_expt = env_future.get();
   if (env_expt.has_value()) {
     response->set_ok(true);
     auto* grep_env = response->mutable_env();
     for (auto& [key, value] : env_expt.value()) {
       (*grep_env)[key] = value;
     }
   } else {
-    // For now, the action never fail
-    std::unreachable();
+    CRANE_ERROR("QueryStepEnvAsync failed: {}", static_cast<int>(env_expt.error()));
+    // response->ok already false
   }
src/Craned/Core/CranedServer.cpp (1)

307-315: Fix incorrect error-checking pattern for std::expected.

This issue was previously flagged. Using .error() in a boolean context does not reliably check for the presence of an error—it evaluates the error value's truthiness instead. The correct pattern is !task_env_map.has_value().

-  if (task_env_map.error()) {
+  if (!task_env_map.has_value()) {
src/Craned/Core/StepInstance.cpp (3)

89-92: Add defensive null check for crane_cgroup before fork.

As flagged previously, SpawnSupervisor unconditionally uses crane_cgroup without validation. While LaunchStepMt_ calls CreateCg() first, future callers or recovery paths might skip this step.

 CraneErrCode StepInstance::SpawnSupervisor(const EnvMap& job_env_map) {
+  if (!crane_cgroup) {
+    CRANE_ERROR("[Step #{}.{}] SpawnSupervisor called without cgroup.", job_id,
+                step_id);
+    return CraneErrCode::ERR_CGROUP;
+  }
+
   using google::protobuf::io::FileInputStream;

360-373: Daemon step still transitions to Starting despite being invalid.

As flagged previously, for StepStatus::Starting and daemon steps, you log that the status is invalid but fall through to status = new_status at line 405. Add an early return:

   case StepStatus::Starting: {
     if (this->IsDaemonStep()) {
       CRANE_WARN(
           "[Step {}.{}] Daemon step got invalid status 'Starting' current "
           "status: {}.",
           job_id, step_id, this->status);
+      return;
     } else {

408-419: Add null check for supervisor_stub before use.

As previously flagged, if ExecuteStepAsync() is called before SpawnSupervisor() completes, supervisor_stub may be null. While it's a shared_ptr, the null state is the concern:

 void StepInstance::ExecuteStepAsync() {
+  if (!supervisor_stub) {
+    CRANE_ERROR("[Step #{}.{}] supervisor_stub is null, cannot execute.",
+                job_id, step_id);
+    return;
+  }
+
   this->GotNewStatus(StepStatus::Running);

-  g_thread_pool->detach_task([job_id = job_id, step_id = step_id,
-                              stub = supervisor_stub] {
+  auto stub = supervisor_stub;  // local copy for clarity
+  g_thread_pool->detach_task([job_id = job_id, step_id = step_id,
+                              stub = std::move(stub)] {
src/Craned/Core/JobManager.cpp (3)

482-483: Guard ChangeStepTimelimit against null supervisor_stub.

As previously flagged, if the supervisor failed to start, supervisor_stub may be null. Add a null check:

   auto& stub = step_it->second->supervisor_stub;
+  if (!stub) {
+    CRANE_ERROR("[Step #{}.{}] Supervisor stub is null when changing "
+                "timelimit.", job_id, step_id);
+    return std::unexpected{CraneErrCode::ERR_NON_EXISTENT};
+  }
   auto err = stub->ChangeTaskTimeLimit(absl::Seconds(new_timelimit_sec));

508-509: Guard QuerySshStepEnvVariables against null supervisor_stub.

Same issue as ChangeStepTimelimit:

   auto& stub = step_it->second->supervisor_stub;
+  if (!stub) {
+    CRANE_ERROR("[Step #{}.{}] Supervisor stub is null when querying SSH env.",
+                job_id, step_id);
+    return std::unexpected{CraneErrCode::ERR_NON_EXISTENT};
+  }
   return stub->QueryStepEnv();

765-786: Critical: Iterator and null-stub dereference in MigrateProcToCgroupOfJob.

As previously flagged, two issues:

  1. If daemon step is missing, you log but still dereference daemon_step_it at line 771
  2. If supervisor_stub is null, you log but still call stub->MigrateSshProcToCg(pid) at line 780

Both can crash. Add early returns:

   auto daemon_step_it = job->step_map.find(kDaemonStepId);
   if (daemon_step_it == job->step_map.end()) {
     CRANE_DEBUG(
         "[Step #{}.{}] Daemon step not found when migrating pid {} to "
         "cgroup of job#{}.",
         job_id, kDaemonStepId, pid, job_id);
+    return false;
   }
   auto& daemon_step = daemon_step_it->second;
   auto stub = daemon_step->supervisor_stub;
   if (!stub) {
     CRANE_ERROR(
         "[Job #{}] Daemon step sSupervisor stub is null when migrating pid {} "
         "to "
         "cgroup of job#{}.",
         job_id, kDaemonStepId, pid, job_id);
+    return false;
   }
src/Craned/Core/Craned.cpp (2)

59-110: Cgroup v2 scanning still only discovers job-level IDs; step/task cgroups won’t be recovered/cleaned

RecoverCgForJobSteps now operates on std::set<CgroupStrParsedIds> from GetIdsFromCgroupV1_/V2_, but on v2 you still call:

rn_job_ids_with_cg = CgroupManager::GetIdsFromCgroupV2_(kSystemCgPathPrefix / kRootCgNamePrefix);

and, per the current GetIdsFromCgroupV2_ implementation, IDs are derived from it.path().filename(). For v2 paths like crane/job_1/step_2/system, filename() is just "system", so:

  • ParseIdsFromCgroupStr_ never sees the embedded job_1/step_2 structure;
  • only job-level dirs (e.g. job_1) are discovered;
  • step/task cgroups on v2 are neither recovered nor cleaned here.

To make step recovery symmetric with v1, you’ll need to update GetIdsFromCgroupV2_ (and GetCgInoJobIdMapCgroupV2_ used by the BPF helper) to parse a relative path that contains the full job_/step_ layout instead of just the final component, and adjust ParseIdsFromCgroupStr_ accordingly.


112-195: Daemon-step user cgroups are recovered but dropped instead of being tracked

In the step branch of RecoverCgForJobSteps:

if (rn_step_from_ctld.contains({job_id, step_id})) {
  auto& step_instance = rn_step_from_ctld.at({job_id, step_id});
  // ...
  if (!step_instance->IsDaemonStep() && !system_flag) {
    continue;
  }
  auto cg_expt = CgroupManager::AllocateAndGetCgroup(
      CgroupManager::CgroupStrByStepId(job_id, step_id, system_flag),
      step_instance->step_to_d.res(), true);
  if (cg_expt.has_value()) {
    if (system_flag)
      step_instance->crane_cgroup = std::move(cg_expt.value());
    continue;
  }
}

For daemon steps you recover both system (system_flag == true) and user (system_flag == false) cgroups, but only assign the system one to step_instance->crane_cgroup. The recovered user cgroup is immediately discarded and no field is populated (e.g., nothing like a step_user_cg on the craned-side StepInstance).

If you intend to keep daemon user cgroups around (e.g., for later SSH proc migration and consistent cleanup), it would be better to wire the recovered user cgroup into a dedicated field on the step instance (mirroring the supervisor-side step_user_cg) rather than dropping it silently.

src/Craned/Supervisor/TaskManager.cpp (7)

447-464: X11 env handling relies on successful StepInstance::Prepare and x11_meta being set

In ProcInstance::InitEnvMap you do:

if (ia_meta.x11()) {
  ...
  m_env_["DISPLAY"] = fmt::format(
      "{}:{}", target, m_parent_step_inst_->x11_meta->x11_port - 6000);
  m_env_["XAUTHORITY"] = m_parent_step_inst_->x11_meta->x11_auth_path;
}

This assumes m_parent_step_inst_->x11_meta has been populated by StepInstance::Prepare. Given that EvGrpcExecuteTaskCb_ currently ignores the return code from Prepare(), any failure there (e.g., xauth path creation) leaves x11_meta unset and this dereference becomes UB.

Once EvGrpcExecuteTaskCb_ is fixed to abort on Prepare() failure (see earlier comment), this assumption becomes safe. Until then, this code path is part of the crash surface and reinforces the need to handle Prepare() failures.


691-753: SetupChildProcCrunX11_ assumes x11_meta is valid and aligned with Prepare/X11 config

This helper correctly builds DISPLAY/XAUTHORITY based on:

  • proto_x11_meta.enable_forwarding() vs target host;
  • the port stored in m_parent_step_inst_->x11_meta.

It also handles xauth subprocess execution robustly (capturing stdout/stderr, checking exit code). The main caveat is again that m_parent_step_inst_->x11_meta must have been set successfully by StepInstance::Prepare; if Prepare() returned early, this .value() will throw. Fixing the caller to abort on Prepare() failure will make this safe in practice.


171-182: Final step termination reason is only partially aligned with final status

TaskFinish_ now updates:

auto& status = m_step_.final_termination_status;
if (status.max_exit_code < exit_code) {
  status.max_exit_code = exit_code;
  status.final_status_on_termination = new_status;
  status.final_reason_on_termination = reason.value_or("");
}
// Error status has higher priority than success status.
if (new_status != StepStatus::Completed)
  status.final_status_on_termination = new_status;

This captures the reason when a new larger exit code arrives, but if a later task changes the final status (e.g., from Completed to Failed) without increasing the exit code, final_status_on_termination is overwritten while final_reason_on_termination remains whatever was set earlier. That can leave the step‑level reason out of sync with the dominant failure mode.

If you want the reason to track the chosen final status, consider also updating final_reason_on_termination in the new_status != Completed branch (as in the earlier review suggestion), e.g.:

if (new_status != StepStatus::Completed) {
  status.final_status_on_termination = new_status;
  if (reason.has_value())
    status.final_reason_on_termination = *reason;
}

2211-2274: EvGrpcExecuteTaskCb_: must check StepInstance::Prepare and handle step-level cgroup allocation failures

Within EvGrpcExecuteTaskCb_:

  1. You set the step status to Running and construct all ITaskInstances.
  2. You initialize pwd and, if that fails, correctly fail all tasks.
  3. Then you call m_step_.Prepare(); but ignore the return value.
  4. Then you handle Calloc early-exit and allocate the step user cgroup (step_user_cg) with AllocateAndGetCgroup. On failure you log and call TaskFinish_ for each task, which is good.
  5. Finally, you launch each task in parallel via LaunchExecution_ and propagate any error to the promise.

The missing piece is handling m_step_.Prepare() failures. As discussed earlier, a failure in Prepare() leaves script_path and/or x11_meta invalid, but the code still proceeds to task launch, leading to crashes or nonsensical execution.

Suggested structure:

-    m_step_.Prepare();
+    auto prep_err = m_step_.Prepare();
+    if (prep_err != CraneErrCode::SUCCESS) {
+      CRANE_ERROR("[Step #{}.{}] Prepare() failed: {}",
+                  m_step_.job_id, m_step_.step_id,
+                  static_cast<int>(prep_err));
+      for (auto task_id : m_step_.task_ids) {
+        TaskFinish_(task_id, crane::grpc::TaskStatus::Failed,
+                    ExitCode::EC_SPAWN_FAILED,
+                    fmt::format("Failed to prepare step, code: {}",
+                                static_cast<int>(prep_err)));
+      }
+      elem.ok_prom.set_value(prep_err);
+      continue;
+    }

This makes step‑level failures explicit and keeps downstream code from running in an inconsistent state.


43-115: Close xauth_fd and propagate StepInstance::Prepare failures instead of continuing

Two problems here:

  1. mkstemp() FD leak and error-path leak:
int xauth_fd = mkstemp(x11_auth_path.data());
...
int ret = fchown(xauth_fd, pwd.Uid(), pwd.Gid());
if (ret == -1) {
  CRANE_ERROR("fchown() for xauth file failed: {}\n", strerror(errno));
  return CraneErrCode::ERR_SYSTEM_ERR;
}

xauth_fd is never closed on either the success path or the fchown error path, so every step leaks one descriptor. On failure you also return without closing it.

  1. Caller ignores the return value:

EvGrpcExecuteTaskCb_ later does:

m_step_.Prepare();

without checking for errors. If any of the early returns in Prepare() trigger (e.g., CreateFoldersForFileEx, mkstemp, fchown), script_path and x11_meta may be left unset, but the rest of the execution path still runs and will eventually dereference x11_meta (m_parent_step_inst_->x11_meta->x11_port) and use script_path, leading to undefined behavior or a bogus bash -c "".

Suggested fix:

  • Always close(xauth_fd) on both success and error paths.
  • Make EvGrpcExecuteTaskCb_ treat a non‑SUCCESS Prepare() as fatal for the step and fail all tasks immediately, mirroring LaunchExecution_.

Example for the FD handling here:

  int xauth_fd = mkstemp(x11_auth_path.data());
  if (xauth_fd == -1) {
    CRANE_ERROR("mkstemp() for xauth file failed: {}\n", strerror(errno));
    return CraneErrCode::ERR_SYSTEM_ERR;
  }

  int ret = fchown(xauth_fd, pwd.Uid(), pwd.Gid());
  if (ret == -1) {
    CRANE_ERROR("fchown() for xauth file failed: {}\n", strerror(errno));
-    return CraneErrCode::ERR_SYSTEM_ERR;
+    close(xauth_fd);
+    return CraneErrCode::ERR_SYSTEM_ERR;
  }
  x11_meta.x11_auth_path = x11_auth_path;
  this->x11_meta = std::move(x11_meta);
+ close(xauth_fd);
  return CraneErrCode::SUCCESS;

And in EvGrpcExecuteTaskCb_ (see separate comment) check the return and fail tasks when Prepare() fails.


380-394: Defend ITaskInstance::Prepare against missing task_res_map entries

ITaskInstance::Prepare currently does:

auto cg_expt = CgroupManager::AllocateAndGetCgroup(
    CgroupManager::CgroupStrByTaskId(g_config.JobId, g_config.StepId, task_id),
    m_parent_step_inst_->GetStep().task_res_map().at(task_id), false);

The PR objectives explicitly say “Scheduler needs to fill in the task res field.” Until that is guaranteed, .at(task_id) will throw std::out_of_range and crash the supervisor if the scheduler misconfigures or omits a task entry.

Safer options:

  • Check for presence and fail gracefully if missing:
  const auto& step = m_parent_step_inst_->GetStep();
- auto cg_expt = CgroupManager::AllocateAndGetCgroup(
-     CgroupManager::CgroupStrByTaskId(g_config.JobId, g_config.StepId,
-                                      task_id),
-     step.task_res_map().at(task_id), false);
+ const auto& task_res_map = step.task_res_map();
+ auto it = task_res_map.find(task_id);
+ if (it == task_res_map.end()) {
+   CRANE_ERROR("[Step #{}.{}] No task_res entry for task #{}.",
+               g_config.JobId, g_config.StepId, task_id);
+   return CraneErrCode::ERR_CGROUP;
+ }
+ auto cg_expt = CgroupManager::AllocateAndGetCgroup(
+     CgroupManager::CgroupStrByTaskId(g_config.JobId, g_config.StepId,
+                                      task_id),
+     it->second, false);
  • Or, if acceptable, temporarily fall back to the step‑level res() when the map is empty.

2321-2348: Prevent double set_value in EvGrpcMigrateSshProcToCgroupCb_ when step is non-daemon

In EvGrpcMigrateSshProcToCgroupCb_:

if (!m_step_.IsDaemonStep()) {
  CRANE_ERROR("Trying to move pid {} to no daemon step", pid);
  prom.set_value(CraneErrCode::ERR_INVALID_PARAM);
}
if (!m_step_.step_user_cg) {
  auto cg_expt = CgroupManager::AllocateAndGetCgroup(...);
  if (!cg_expt.has_value()) {
    ...
    prom.set_value(CraneErrCode::ERR_CGROUP);
    continue;
  }
  m_step_.step_user_cg = std::move(cg_expt.value());
}
if (m_step_.step_user_cg->MigrateProcIn(pid)) {
  prom.set_value(CraneErrCode::SUCCESS);
} else {
  prom.set_value(CraneErrCode::ERR_CGROUP);
}

If IsDaemonStep() is false, you already call prom.set_value(ERR_INVALID_PARAM) but then fall through and may call set_value again (and perform cgroup allocation/migration on a non-daemon step). Calling set_value twice on the same promise is undefined behavior (std::future_error) and will likely crash the supervisor.

Add an early continue after the error, e.g.:

    if (!m_step_.IsDaemonStep()) {
      CRANE_ERROR("Trying to move pid {} to no daemon step", pid);
      prom.set_value(CraneErrCode::ERR_INVALID_PARAM);
+     continue;
    }

This keeps the handler single-shot per request and avoids touching cgroups when the step type is invalid for SSH migration.

🧹 Nitpick comments (5)
src/Craned/Core/CranedServer.cpp (1)

194-195: Unused variable system_flag.

The system_flag variable is extracted from the tuple but never used in this function. Consider either using it or removing it from the unpacking.

-      auto [job_id_opt, step_id_opt, system_flag, task_id_opt] =
+      auto [job_id_opt, step_id_opt, _, task_id_opt] =
           pid_to_ids_expt.value();

Or if the flag should be used for some logic, add the necessary handling.

src/Craned/Core/SupervisorStub.cpp (1)

139-158: Duplicate logging in TerminateTask.

Both CRANE_ERROR and CRANE_WARN are called on the same failure path with nearly identical messages.

   if (ok.ok() && reply.ok()) {
     return CraneErrCode::SUCCESS;
   }
-  CRANE_ERROR("TerminateTask failed: reply {},{}", reply.ok(),
-              ok.error_message());
-
   CRANE_WARN("TerminateTask failed: reply {},{}", reply.ok(),
              ok.error_message());
   return CraneErrCode::ERR_RPC_FAILURE;
src/Craned/Supervisor/CforedClient.cpp (1)

630-643: X11 input path still hardcodes x11_id/eof, limiting multi-connection support

In the TASK_X11_INPUT branch you currently do:

x11_local_id_t x11_id = 0;
// bool eof = reply.payload_task_x11_input_req().eof();
bool eof = false;
auto x11_fd_info_it = m_x11_fd_info_map_.find(x11_id);

and in CleanOutputQueueAndWriteToStreamThread_ you comment out setting x11_id/eof on the X11 payload. This keeps the previous “single X11 connection” behavior but means:

  • the per‑connection x11_local_id you generate on accept is never used on the input path;
  • EOF from cfored is ignored, so only the proxy side can drive closure.

If multi‑X11 or remote‑driven EOF is intended in the future, wiring through x11_id and eof from the proto here will be needed. If you’re intentionally limiting to a single connection, consider adding a short comment to make this explicit.

src/Craned/Common/CgroupManager.h (1)

537-581: Comment for AllocateAndGetCgroup mentions alloc_mem parameter that no longer exists

The doc comment:

* \param alloc_mem true if need to enforce memory limit, default true

describes an alloc_mem parameter that is not present in the AllocateAndGetCgroup signature anymore. This is misleading for callers and future maintainers.

Either:

  • reintroduce an alloc_mem parameter if you still need that behavior; or
  • drop/update the comment to reflect the current API (only min_mem controls memory adjustments now).
src/Craned/Supervisor/TaskManager.cpp (1)

232-307: Status transition checks in GotNewStatus are sound but may log noisy warnings for daemon steps

The new StepInstance::GotNewStatus enforces a clear state machine (Configuring → Starting → Running → Completing → terminal), and the warnings for unexpected transitions are helpful.

One side effect is that calling GotNewStatus(StepStatus::Running) a second time for daemon steps (after EvSupervisorFinishInitCb_ has already set them to Running) will now log the "Daemon step status is not 'Configuring'" warning. If EvGrpcExecuteTaskCb_ always calls GotNewStatus(Running) regardless of step type, you might want to guard that call for daemon steps to avoid spurious warnings.

📜 Review details

Configuration used: Path: .coderabbit.yaml

Review profile: CHILL

Plan: Pro

📥 Commits

Reviewing files that changed from the base of the PR and between c566be8 and 66470c0.

📒 Files selected for processing (31)
  • protos/Crane.proto (6 hunks)
  • protos/PublicDefs.proto (2 hunks)
  • protos/Supervisor.proto (2 hunks)
  • scripts/wipe_data.py (3 hunks)
  • src/CraneCtld/CtldPublicDefs.cpp (5 hunks)
  • src/CraneCtld/CtldPublicDefs.h (2 hunks)
  • src/CraneCtld/EmbeddedDbClient.cpp (1 hunks)
  • src/CraneCtld/TaskScheduler.cpp (7 hunks)
  • src/Craned/Common/CgroupManager.cpp (18 hunks)
  • src/Craned/Common/CgroupManager.h (18 hunks)
  • src/Craned/Core/CMakeLists.txt (1 hunks)
  • src/Craned/Core/Craned.cpp (12 hunks)
  • src/Craned/Core/CranedForPamServer.cpp (2 hunks)
  • src/Craned/Core/CranedServer.cpp (4 hunks)
  • src/Craned/Core/CtldClient.cpp (1 hunks)
  • src/Craned/Core/JobManager.cpp (17 hunks)
  • src/Craned/Core/JobManager.h (4 hunks)
  • src/Craned/Core/StepInstance.cpp (1 hunks)
  • src/Craned/Core/StepInstance.h (1 hunks)
  • src/Craned/Core/SupervisorStub.cpp (3 hunks)
  • src/Craned/Core/SupervisorStub.h (1 hunks)
  • src/Craned/Supervisor/CforedClient.cpp (13 hunks)
  • src/Craned/Supervisor/CforedClient.h (6 hunks)
  • src/Craned/Supervisor/Supervisor.cpp (1 hunks)
  • src/Craned/Supervisor/SupervisorPublicDefs.h (0 hunks)
  • src/Craned/Supervisor/SupervisorServer.cpp (3 hunks)
  • src/Craned/Supervisor/SupervisorServer.h (1 hunks)
  • src/Craned/Supervisor/TaskManager.cpp (29 hunks)
  • src/Craned/Supervisor/TaskManager.h (16 hunks)
  • src/Utilities/PublicHeader/include/crane/PublicHeader.h (1 hunks)
  • src/Utilities/PublicHeader/include/crane/String.h (2 hunks)
💤 Files with no reviewable changes (1)
  • src/Craned/Supervisor/SupervisorPublicDefs.h
✅ Files skipped from review due to trivial changes (1)
  • src/Craned/Core/StepInstance.h
🚧 Files skipped from review as they are similar to previous changes (8)
  • src/Utilities/PublicHeader/include/crane/PublicHeader.h
  • src/Craned/Core/CMakeLists.txt
  • protos/PublicDefs.proto
  • protos/Supervisor.proto
  • src/CraneCtld/TaskScheduler.cpp
  • src/Craned/Core/CtldClient.cpp
  • src/Craned/Supervisor/SupervisorServer.h
  • src/Craned/Supervisor/TaskManager.h
🧰 Additional context used
🧠 Learnings (36)
📓 Common learnings
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Supervisor/SupervisorPublicDefs.h:32-37
Timestamp: 2025-05-26T11:06:28.796Z
Learning: The user L-Xiafeng prefers to defer refactoring duplicate definitions until they become a larger pattern in the codebase, rather than addressing individual instances immediately.
📚 Learning: 2025-05-26T11:04:30.580Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Craned/SupervisorKeeper.h:32-32
Timestamp: 2025-05-26T11:04:30.580Z
Learning: The Supervisor component in the Crane system is designed to manage only one task per instance. The task specification is provided to the Supervisor during startup by Craned (likely through InitSupervisorRequest), so the ExecuteTask() method doesn't need to accept task parameters since the Supervisor already knows which task to execute.

Applied to files:

  • src/Craned/Supervisor/Supervisor.cpp
  • src/Craned/Core/CranedServer.cpp
  • src/Craned/Core/StepInstance.cpp
  • src/Craned/Core/Craned.cpp
  • protos/Crane.proto
  • src/Craned/Core/JobManager.cpp
  • src/Craned/Core/SupervisorStub.h
  • src/Craned/Supervisor/TaskManager.cpp
  • src/Craned/Supervisor/SupervisorServer.cpp
  • src/Craned/Core/JobManager.h
  • src/Craned/Core/SupervisorStub.cpp
📚 Learning: 2025-05-02T07:12:46.896Z
Learnt from: RileyWen
Repo: PKUHPC/CraneSched PR: 475
File: src/Craned/CranedServer.h:0-0
Timestamp: 2025-05-02T07:12:46.896Z
Learning: The initialization of `m_supervisor_recovered_` to `true` in the CranedServer class is intentional despite the comment saying "When supervisor ready, init with false". This is temporary until the supervisor functionality is fully implemented.

Applied to files:

  • src/Craned/Supervisor/Supervisor.cpp
  • src/Craned/Core/StepInstance.cpp
  • src/Craned/Core/Craned.cpp
  • src/Craned/Core/SupervisorStub.h
  • src/Craned/Supervisor/TaskManager.cpp
  • src/Craned/Supervisor/SupervisorServer.cpp
  • src/Craned/Core/SupervisorStub.cpp
📚 Learning: 2025-05-25T04:08:03.273Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Supervisor/Supervisor.cpp:68-69
Timestamp: 2025-05-25T04:08:03.273Z
Learning: In the Crane supervisor component (src/Craned/Supervisor/), the supervisor process communicates with Craned through STDOUT using protobuf messages. The supervisor must not send any information to STDOUT before sending the "ready" message to Craned, as this would interfere with the inter-process communication protocol. Therefore, adding logging statements that might write to STDOUT before the ready message is sent could break the communication.

Applied to files:

  • src/Craned/Supervisor/Supervisor.cpp
  • src/Craned/Core/CranedServer.cpp
  • src/Craned/Core/StepInstance.cpp
  • src/Craned/Core/Craned.cpp
  • src/Craned/Core/JobManager.cpp
  • src/Craned/Core/SupervisorStub.h
  • src/Craned/Supervisor/TaskManager.cpp
  • src/Craned/Supervisor/SupervisorServer.cpp
  • src/Craned/Core/JobManager.h
  • src/Craned/Core/SupervisorStub.cpp
📚 Learning: 2025-05-25T04:11:50.268Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Supervisor/TaskManager.cpp:220-220
Timestamp: 2025-05-25T04:11:50.268Z
Learning: In TaskManager.cpp, when step->IsCrun() is checked before dynamic_cast to CrunMetaInExecution*, the cast is guaranteed to succeed due to the program logic ensuring the correct meta type is used for Crun tasks. Null checks for these dynamic_cast operations are unnecessary in this context.

Applied to files:

  • src/Craned/Supervisor/Supervisor.cpp
  • src/Craned/Core/StepInstance.cpp
  • src/Craned/Core/JobManager.cpp
  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-05-08T07:39:29.207Z
Learnt from: huerni
Repo: PKUHPC/CraneSched PR: 469
File: src/Craned/TaskManager.cpp:1162-1175
Timestamp: 2025-05-08T07:39:29.207Z
Learning: In CraneSched, when a task fails in the LaunchTaskInstanceMt_ method after PMIx registration, ActivateTaskStatusChangeAsync_ does not automatically release PMIx resources or cgroup resources. Explicit cleanup is required by calling g_pmix_server->DeregisterTask() and g_cg_mgr->ReleaseCgroupByTaskIdOnly() before returning from the function.

Applied to files:

  • src/Craned/Supervisor/Supervisor.cpp
  • src/Craned/Core/StepInstance.cpp
  • src/Craned/Core/Craned.cpp
  • src/Craned/Core/JobManager.cpp
  • src/Craned/Supervisor/TaskManager.cpp
  • src/CraneCtld/CtldPublicDefs.h
  • src/Craned/Core/JobManager.h
  • src/Craned/Supervisor/CforedClient.cpp
  • src/CraneCtld/CtldPublicDefs.cpp
📚 Learning: 2025-05-08T07:39:29.207Z
Learnt from: huerni
Repo: PKUHPC/CraneSched PR: 469
File: src/Craned/TaskManager.cpp:1162-1175
Timestamp: 2025-05-08T07:39:29.207Z
Learning: In CraneSched, when a task fails in the LaunchTaskInstanceMt_ method, ActivateTaskStatusChangeAsync_ does not automatically release PMIx resources or cgroup. Explicit cleanup is required by calling g_pmix_server->DeregisterTask() and g_cg_mgr->ReleaseCgroupByTaskIdOnly().

Applied to files:

  • src/Craned/Supervisor/Supervisor.cpp
  • src/Craned/Core/StepInstance.cpp
  • src/Craned/Core/Craned.cpp
  • src/Craned/Core/JobManager.cpp
  • src/Craned/Supervisor/TaskManager.cpp
  • src/Craned/Supervisor/SupervisorServer.cpp
  • src/Craned/Core/JobManager.h
  • src/Craned/Supervisor/CforedClient.cpp
  • src/CraneCtld/CtldPublicDefs.cpp
📚 Learning: 2025-05-09T01:54:21.256Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 496
File: src/CraneCtld/CranedKeeper.cpp:51-53
Timestamp: 2025-05-09T01:54:21.256Z
Learning: The ConfigureCraned function in src/CraneCtld/CranedKeeper.cpp is called from a thread pool, so there's no need to worry about it blocking the gRPC completion queue thread.

Applied to files:

  • src/Craned/Supervisor/Supervisor.cpp
  • src/Craned/Core/CranedServer.cpp
  • src/Craned/Core/StepInstance.cpp
  • src/Craned/Core/Craned.cpp
  • src/CraneCtld/EmbeddedDbClient.cpp
  • src/Craned/Core/JobManager.cpp
  • src/Craned/Core/CranedForPamServer.cpp
  • src/Craned/Core/SupervisorStub.h
  • src/Craned/Supervisor/TaskManager.cpp
  • src/Craned/Supervisor/SupervisorServer.cpp
  • src/CraneCtld/CtldPublicDefs.h
  • src/Craned/Supervisor/CforedClient.h
  • src/Craned/Core/JobManager.h
  • src/Craned/Core/SupervisorStub.cpp
  • src/Craned/Supervisor/CforedClient.cpp
  • src/CraneCtld/CtldPublicDefs.cpp
📚 Learning: 2025-06-30T08:43:44.470Z
Learnt from: huerni
Repo: PKUHPC/CraneSched PR: 537
File: src/Craned/TaskManager.cpp:145-147
Timestamp: 2025-06-30T08:43:44.470Z
Learning: In the CraneSched codebase, src/Craned/Craned.cpp guarantees that g_config.CranedRes.contains(g_config.CranedIdOfThisNode) through explicit validation during startup. The code checks if g_config.CranedRes.contains(g_config.Hostname) and exits the process if not found, then sets g_config.CranedIdOfThisNode = g_config.Hostname. TaskManager constructor is called after this validation, so g_config.CranedRes[g_config.CranedIdOfThisNode] is guaranteed to be valid.

Applied to files:

  • src/Craned/Supervisor/Supervisor.cpp
  • src/Craned/Core/CranedServer.cpp
  • src/Craned/Core/StepInstance.cpp
  • src/Craned/Core/Craned.cpp
  • src/Craned/Core/JobManager.cpp
  • src/Craned/Core/SupervisorStub.h
  • src/Craned/Supervisor/TaskManager.cpp
  • src/CraneCtld/CtldPublicDefs.h
  • src/Craned/Supervisor/CforedClient.h
  • src/Craned/Core/JobManager.h
  • src/CraneCtld/CtldPublicDefs.cpp
📚 Learning: 2025-05-02T07:06:36.103Z
Learnt from: RileyWen
Repo: PKUHPC/CraneSched PR: 475
File: src/CraneCtld/CtldGrpcServer.cpp:113-118
Timestamp: 2025-05-02T07:06:36.103Z
Learning: In CraneSched, gRPC methods should generally return Status::OK even when handling error conditions, as non-OK statuses cause the connection to terminate. Error information should be communicated within the RPC response payload instead.

Applied to files:

  • src/Craned/Supervisor/Supervisor.cpp
  • src/Craned/Core/CranedServer.cpp
  • src/Craned/Core/StepInstance.cpp
  • src/Craned/Core/CranedForPamServer.cpp
  • src/Craned/Supervisor/TaskManager.cpp
  • src/Craned/Supervisor/SupervisorServer.cpp
  • src/Craned/Core/JobManager.h
  • src/Craned/Core/SupervisorStub.cpp
  • src/CraneCtld/CtldPublicDefs.cpp
📚 Learning: 2025-04-02T10:11:33.562Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 461
File: src/Craned/CgroupManager.cpp:685-685
Timestamp: 2025-04-02T10:11:33.562Z
Learning: In the CgroupManager's GetJobBpfMapCgroupsV2 method, the developer has confirmed that cg_ino_job_id_map will always contain the key->cgroup_id element, making the CRANE_ASSERT check appropriate rather than requiring additional error handling.

Applied to files:

  • src/Craned/Core/CranedServer.cpp
  • src/Craned/Core/StepInstance.cpp
  • src/Craned/Core/Craned.cpp
  • src/Craned/Common/CgroupManager.cpp
  • src/Craned/Core/JobManager.cpp
  • src/Craned/Supervisor/TaskManager.cpp
  • src/Craned/Core/JobManager.h
  • src/Craned/Common/CgroupManager.h
📚 Learning: 2025-03-31T09:29:40.388Z
Learnt from: RileyWen
Repo: PKUHPC/CraneSched PR: 456
File: src/CraneCtld/Tls/VaultClientWrapper.cpp:116-137
Timestamp: 2025-03-31T09:29:40.388Z
Learning: In CraneSched, `phmap::parallel_flat_hash_set` from the Parallel Hashmap library (`parallel_hashmap/phmap.h`) is used for thread-safe containers. This container implements internal sharding with separate locks for different parts of the hash table, making it inherently thread-safe for concurrent operations without requiring external synchronization.

Applied to files:

  • src/Craned/Core/CranedServer.cpp
  • src/CraneCtld/CtldPublicDefs.h
📚 Learning: 2025-03-31T09:29:40.388Z
Learnt from: RileyWen
Repo: PKUHPC/CraneSched PR: 456
File: src/CraneCtld/Tls/VaultClientWrapper.cpp:116-137
Timestamp: 2025-03-31T09:29:40.388Z
Learning: In the CraneSched project, the `m_allowed_certs_` in `VaultClientWrapper` is implemented as a `phmap::parallel_flat_hash_set<std::string>`, which is a thread-safe container designed for concurrent access, making it safe to use in multithreaded contexts without additional synchronization.

Applied to files:

  • src/Craned/Core/CranedServer.cpp
  • src/CraneCtld/CtldPublicDefs.h
📚 Learning: 2025-03-31T09:29:40.388Z
Learnt from: RileyWen
Repo: PKUHPC/CraneSched PR: 456
File: src/CraneCtld/Tls/VaultClientWrapper.cpp:116-137
Timestamp: 2025-03-31T09:29:40.388Z
Learning: The `phmap::parallel_flat_hash_set` used in CraneSched's `VaultClientWrapper` is from the Parallel Hashmap library, which provides thread-safe containers for concurrent access. It uses internal partitioning to reduce contention, making it safe for use in multithreaded environments without external synchronization.

Applied to files:

  • src/Craned/Core/CranedServer.cpp
📚 Learning: 2025-06-07T10:47:59.071Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 520
File: src/CraneCtld/CranedKeeper.cpp:416-417
Timestamp: 2025-06-07T10:47:59.071Z
Learning: In src/CraneCtld/CranedKeeper.h, the m_shutting_down_ member in CranedStub class is declared as std::atomic_bool, making it thread-safe for concurrent access without additional synchronization.

Applied to files:

  • src/Craned/Core/CranedServer.cpp
  • src/Craned/Core/StepInstance.cpp
  • src/Craned/Core/SupervisorStub.h
  • src/CraneCtld/CtldPublicDefs.h
  • src/Craned/Supervisor/CforedClient.h
  • src/Craned/Core/SupervisorStub.cpp
📚 Learning: 2025-04-02T09:30:13.014Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 461
File: src/Craned/JobManager.cpp:141-149
Timestamp: 2025-04-02T09:30:13.014Z
Learning: In JobManager, if a uid exists in m_uid_to_job_ids_map_, its corresponding task_ids set is guaranteed to be non-empty due to the invariant maintained in AllocJobs and FreeJobs methods.

Applied to files:

  • src/Craned/Core/CranedServer.cpp
  • src/Craned/Common/CgroupManager.cpp
📚 Learning: 2025-07-01T08:00:05.383Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Supervisor/TaskManager.cpp:1-1809
Timestamp: 2025-07-01T08:00:05.383Z
Learning: In TaskManager.cpp, the security model relies on administrator-configured command templates in ParseOCICmdPattern_ and system-provided usernames in ParseFilePathPattern_, with file permissions using the user's own uid/gid for privilege isolation. The user L-Xiafeng considers this security boundary sufficient and chooses not to fix job name-related path issues at this location.

Applied to files:

  • src/Craned/Core/CranedServer.cpp
  • src/Craned/Common/CgroupManager.cpp
  • src/Craned/Core/JobManager.cpp
  • src/Craned/Core/CranedForPamServer.cpp
  • src/Craned/Supervisor/TaskManager.cpp
  • src/Craned/Common/CgroupManager.h
📚 Learning: 2025-05-26T11:00:54.563Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Craned/SupervisorKeeper.cpp:174-178
Timestamp: 2025-05-26T11:00:54.563Z
Learning: The CraneSched project uses C++23 standard, allowing the use of modern C++ features like std::ranges::to and other C++23 language features and library components.

Applied to files:

  • src/Utilities/PublicHeader/include/crane/String.h
📚 Learning: 2025-05-09T01:54:39.465Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 496
File: src/Craned/TaskManager.cpp:1509-1517
Timestamp: 2025-05-09T01:54:39.465Z
Learning: The CraneSched project uses C++23, which supports Class Template Argument Deduction (CTAD) for standard containers like std::set and includes ranges support, making std::ranges::views::keys valid without additional headers.

Applied to files:

  • src/Utilities/PublicHeader/include/crane/String.h
📚 Learning: 2025-05-08T07:38:42.362Z
Learnt from: huerni
Repo: PKUHPC/CraneSched PR: 469
File: src/Craned/TaskManager.cpp:0-0
Timestamp: 2025-05-08T07:38:42.362Z
Learning: In CraneSched's PMIx integration, the `g_pmix_server->SetupFork()` function must be called in the child process after fork() and before exec() to properly set up the PMIx environment variables.

Applied to files:

  • src/Craned/Core/StepInstance.cpp
  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-05-25T04:11:27.740Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Supervisor/TaskManager.cpp:396-396
Timestamp: 2025-05-25T04:11:27.740Z
Learning: In TaskManager.cpp, GetCrunMeta() calls don't need null checks because they're only called in contexts where the task is guaranteed to be a CRUN task (e.g., SetupChildProcessCrunX11_ is only called when step->IsCrun() && x11() conditions are met), ensuring the metadata will always be valid.

Applied to files:

  • src/Craned/Core/StepInstance.cpp
  • src/Craned/Core/JobManager.cpp
  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-09-21T11:26:40.935Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 625
File: src/Craned/Core/SupervisorKeeper.cpp:145-147
Timestamp: 2025-09-21T11:26:40.935Z
Learning: In C++17 and later, Class Template Argument Deduction (CTAD) allows `std::shared_ptr stub = std::make_shared<SupervisorStub>();` syntax to be valid, as the compiler can deduce the template parameter from make_shared. Projects using C++17+ don't need explicit template parameters in this context.

Applied to files:

  • src/Craned/Core/StepInstance.cpp
  • src/Craned/Core/SupervisorStub.h
📚 Learning: 2025-05-08T09:35:39.809Z
Learnt from: huerni
Repo: PKUHPC/CraneSched PR: 469
File: src/Craned/Pmix/PmixCollRing.cpp:0-0
Timestamp: 2025-05-08T09:35:39.809Z
Learning: In the PMIx implementation for CraneSched, objects referenced in asynchronous gRPC callbacks (like `coll_ring_ctx`) remain valid as long as the parent object (`this`) is not destroyed. The `Coll` class uses shared_ptr management to ensure its lifetime extends through all pending callbacks.

Applied to files:

  • src/Craned/Core/StepInstance.cpp
📚 Learning: 2025-05-02T07:05:26.012Z
Learnt from: RileyWen
Repo: PKUHPC/CraneSched PR: 475
File: src/CraneCtld/CranedKeeper.cpp:601-602
Timestamp: 2025-05-02T07:05:26.012Z
Learning: In the CraneCtld codebase, the variables m_disconnected_ and m_registered_ in CranedStub class are already defined as std::atomic_bool, making them thread-safe for concurrent access without additional synchronization.

Applied to files:

  • src/Craned/Core/StepInstance.cpp
  • src/CraneCtld/CtldPublicDefs.h
  • src/Craned/Supervisor/CforedClient.h
📚 Learning: 2025-05-26T11:04:56.055Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 399
File: src/Craned/Craned/SupervisorKeeper.h:72-72
Timestamp: 2025-05-26T11:04:56.055Z
Learning: The CraneSched project prefers using global variables (like `g_supervisor_keeper`) over dependency injection patterns. The team does not follow dependency injection approaches for managing singleton instances.

Applied to files:

  • src/Craned/Core/StepInstance.cpp
  • src/Craned/Core/Craned.cpp
  • src/Craned/Core/SupervisorStub.h
  • src/Craned/Core/SupervisorStub.cpp
📚 Learning: 2025-04-02T09:52:59.318Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 461
File: src/Craned/CgroupManager.cpp:697-701
Timestamp: 2025-04-02T09:52:59.318Z
Learning: When using bpf_map__get_next_key function, memory must be properly allocated (e.g., with std::make_unique<BpfKey>()) before passing the pointer to the function, as it writes the key to the provided memory address.

Applied to files:

  • src/Craned/Core/Craned.cpp
  • src/Craned/Common/CgroupManager.cpp
📚 Learning: 2025-04-27T11:52:31.017Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 475
File: src/CraneCtld/CranedKeeper.cpp:40-62
Timestamp: 2025-04-27T11:52:31.017Z
Learning: In the CraneSched system, retry of configuration RPC is architecturally driven by the Craned's notification system rather than explicit retry code within the ConfigureCraned method. When Configure RPC fails, Craned returns to a notification state and sends new Notify messages which trigger new configuration attempts.

Applied to files:

  • src/Craned/Core/Craned.cpp
📚 Learning: 2025-06-23T07:53:30.513Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 520
File: src/Craned/CranedServer.cpp:50-51
Timestamp: 2025-06-23T07:53:30.513Z
Learning: In the CraneSched codebase, `g_ctld_client` is a guaranteed global variable that is always initialized before any gRPC service methods are called, so null pointer checks are not necessary when calling methods on it.

Applied to files:

  • src/Craned/Core/Craned.cpp
📚 Learning: 2025-04-18T02:26:16.113Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 351
File: src/CraneCtld/CranedMetaContainer.cpp:248-264
Timestamp: 2025-04-18T02:26:16.113Z
Learning: The resource class in CraneSched includes assertions in its operator overloads (particularly in operator-=) that verify resources being subtracted are less than or equal to available resources, ensuring no negative values can occur during resource allocation or deallocation operations.

Applied to files:

  • src/Craned/Common/CgroupManager.cpp
📚 Learning: 2025-05-09T02:16:56.723Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 496
File: src/Craned/TaskManager.cpp:1498-1506
Timestamp: 2025-05-09T02:16:56.723Z
Learning: The `QueryRunningTasksAsync()` method in TaskManager.cpp is designed to never be called from inside the event loop thread, so there's no risk of deadlock with the synchronous `res.get()` call.

Applied to files:

  • src/Craned/Supervisor/TaskManager.cpp
📚 Learning: 2025-05-09T01:56:53.142Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 496
File: src/CraneCtld/CtldPublicDefs.h:756-763
Timestamp: 2025-05-09T01:56:53.142Z
Learning: In the CraneSched codebase, the `execution_node` field in JobToD is intentionally set to the first element of `executing_craned_ids` vector without guards, as it represents the main execution node for a job. This is by design and assumes `executing_craned_ids` is never empty when `GetJobOfNode` is called.

Applied to files:

  • src/CraneCtld/CtldPublicDefs.h
  • src/Craned/Core/JobManager.h
  • src/CraneCtld/CtldPublicDefs.cpp
📚 Learning: 2025-05-02T07:05:49.032Z
Learnt from: RileyWen
Repo: PKUHPC/CraneSched PR: 475
File: src/CraneCtld/CranedKeeper.cpp:503-506
Timestamp: 2025-05-02T07:05:49.032Z
Learning: In CraneCtld/CranedKeeper.cpp, using m_unavail_craned_set_.at() is intentional as the key is guaranteed to exist by design, and crashing on missing key is preferred to silently handling it (fail-fast approach).

Applied to files:

  • src/CraneCtld/CtldPublicDefs.h
📚 Learning: 2025-08-12T08:58:39.772Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 577
File: src/Craned/Supervisor/TaskManager.cpp:124-130
Timestamp: 2025-08-12T08:58:39.772Z
Learning: In the CraneSched project using C++23, Class Template Argument Deduction (CTAD) allows std::unique_ptr declarations without explicit template parameters when the type can be deduced from the initializer, such as `std::unique_ptr task = std::move(m_task_map_.at(task_id))` where the template parameter is deduced from the move operation.

Applied to files:

  • src/CraneCtld/CtldPublicDefs.h
📚 Learning: 2025-08-14T02:56:35.503Z
Learnt from: L-Xiafeng
Repo: PKUHPC/CraneSched PR: 587
File: src/Craned/Supervisor/CforedClient.cpp:449-454
Timestamp: 2025-08-14T02:56:35.503Z
Learning: In CforedClient::AsyncSendRecvThread_(), the guard `if (state <= State::Registering) { continue; }` in the TIMEOUT branch only prevents premature cleanup when stopping before registration completes, but it doesn't block normal gRPC event processing. The completion queue will still deliver Prepare/Write/Read events that advance the state machine normally.

Applied to files:

  • src/Craned/Supervisor/CforedClient.cpp
📚 Learning: 2025-05-23T02:32:43.952Z
Learnt from: 1daidai1
Repo: PKUHPC/CraneSched PR: 458
File: src/CraneCtld/CtldPublicDefs.h:0-0
Timestamp: 2025-05-23T02:32:43.952Z
Learning: In the CraneSched project, allocated_res_view is handled/updated separately before calling SetAllocatedRes, so it does not need to be updated again within the method itself.

Applied to files:

  • src/CraneCtld/CtldPublicDefs.cpp
📚 Learning: 2025-05-23T02:32:43.952Z
Learnt from: 1daidai1
Repo: PKUHPC/CraneSched PR: 458
File: src/CraneCtld/CtldPublicDefs.h:0-0
Timestamp: 2025-05-23T02:32:43.952Z
Learning: In the CraneSched project, allocated_res_view is updated before calling SetAllocatedRes, so it doesn't need to be updated again within the method itself.

Applied to files:

  • src/CraneCtld/CtldPublicDefs.cpp
🧬 Code graph analysis (12)
src/Utilities/PublicHeader/include/crane/String.h (1)
src/Utilities/PublicHeader/String.cpp (2)
  • StepToDIdString (519-521)
  • StepToDIdString (519-519)
src/Craned/Core/Craned.cpp (3)
src/Craned/Core/StepInstance.cpp (2)
  • StepInstance (25-30)
  • StepInstance (32-40)
src/Craned/Core/SupervisorStub.h (2)
  • Craned (26-62)
  • SupervisorStub (28-61)
src/Craned/Common/CgroupManager.cpp (8)
  • AllocateAndGetCgroup (494-552)
  • AllocateAndGetCgroup (495-497)
  • CgroupStrByStepId (283-288)
  • CgroupStrByStepId (283-284)
  • CgroupStrByJobId (279-281)
  • CgroupStrByJobId (279-279)
  • CgroupStrByParsedIds (296-308)
  • CgroupStrByParsedIds (296-296)
src/Craned/Core/SupervisorStub.h (1)
src/Craned/Core/SupervisorStub.cpp (12)
  • SupervisorStub (25-29)
  • SupervisorStub (31-33)
  • InitAndGetRecoveredMap (35-92)
  • InitAndGetRecoveredMap (37-37)
  • CheckStatus (123-137)
  • CheckStatus (124-124)
  • MigrateSshProcToCg (174-186)
  • MigrateSshProcToCg (174-174)
  • ShutdownSupervisor (188-198)
  • ShutdownSupervisor (188-188)
  • InitChannelAndStub_ (200-204)
  • InitChannelAndStub_ (200-200)
src/Craned/Supervisor/TaskManager.cpp (4)
src/Craned/Common/CgroupManager.h (1)
  • CgroupManager (532-640)
src/Craned/Core/CranedPublicDefs.h (1)
  • g_config (156-156)
src/Craned/Supervisor/TaskManager.h (5)
  • uid (56-56)
  • InitCriClient (150-155)
  • StopCriClient (160-160)
  • job_id (52-52)
  • step_id (53-53)
src/Utilities/PublicHeader/include/crane/PasswordEntry.h (1)
  • pwd (105-105)
src/Craned/Supervisor/SupervisorServer.cpp (2)
src/Craned/Core/CranedForPamServer.cpp (2)
  • MigrateSshProcToCgroup (181-205)
  • MigrateSshProcToCgroup (181-184)
src/Craned/Core/SupervisorStub.cpp (2)
  • ShutdownSupervisor (188-198)
  • ShutdownSupervisor (188-188)
src/CraneCtld/CtldPublicDefs.h (1)
src/Utilities/PublicHeader/PublicHeader.cpp (1)
  • ResourceInNode (484-487)
src/Craned/Supervisor/CforedClient.h (1)
src/Craned/Supervisor/CforedClient.cpp (6)
  • TaskProcessStop (715-728)
  • TaskProcessStop (715-716)
  • TaskOutPutForward (739-746)
  • TaskOutPutForward (739-740)
  • TaskX11OutPutForward (748-759)
  • TaskX11OutPutForward (748-750)
src/Craned/Core/JobManager.h (3)
src/Craned/Common/CgroupManager.h (1)
  • CgroupInterface (389-421)
src/Craned/Core/JobManager.cpp (5)
  • ChangeStepTimelimit (467-492)
  • ChangeStepTimelimit (467-469)
  • job_id_t (789-804)
  • QuerySshStepEnvVariables (494-510)
  • QuerySshStepEnvVariables (494-495)
src/Craned/Core/CranedServer.cpp (2)
  • QuerySshStepEnvVariables (297-318)
  • QuerySshStepEnvVariables (297-300)
src/Craned/Core/SupervisorStub.cpp (1)
src/Craned/Core/SupervisorStub.h (1)
  • SupervisorStub (28-61)
src/Craned/Common/CgroupManager.h (1)
src/Craned/Common/CgroupManager.cpp (19)
  • SetCpuBind (1012-1031)
  • SetCpuBind (1012-1012)
  • SetCpuBind (1317-1335)
  • SetCpuBind (1317-1317)
  • KillAllProcesses (1087-1114)
  • KillAllProcesses (1087-1087)
  • KillAllProcesses (1516-1544)
  • KillAllProcesses (1516-1516)
  • RecoverFromResInNode (1442-1492)
  • RecoverFromResInNode (1442-1443)
  • CgroupStrByParsedIds (296-308)
  • CgroupStrByParsedIds (296-296)
  • CgroupStrParsedIds (636-676)
  • GetIdsFromCgroupV1_ (554-578)
  • GetIdsFromCgroupV1_ (554-555)
  • GetIdsFromCgroupV2_ (580-600)
  • GetIdsFromCgroupV2_ (580-581)
  • GetCgInoJobIdMapCgroupV2_ (602-632)
  • GetCgInoJobIdMapCgroupV2_ (603-604)
src/Craned/Supervisor/CforedClient.cpp (1)
src/Craned/Supervisor/CforedClient.h (3)
  • next_x11_id_ (150-150)
  • exit_code (131-131)
  • signaled (132-133)
src/CraneCtld/CtldPublicDefs.cpp (1)
src/Craned/Supervisor/TaskManager.h (1)
  • job_id (52-52)
🪛 Ruff (0.14.5)
scripts/wipe_data.py

77-77: Do not catch blind exception: Exception

(BLE001)


103-103: Use logging.exception instead of logging.error

Replace with exception

(TRY400)


104-104: Use raise without specifying exception name

Remove exception name

(TRY201)

Comment on lines 65 to 78
# Try to preserve task ID
var_db_path = embedded_db_path + "var"
next_task_id_key = "NI"
next_task_id_val = None

if os.path.exists(var_db_path):
try:
db = UnQLite(var_db_path)
if next_task_id_key in db:
next_task_id_val = db[next_task_id_key]
logger.info(f"Found existing task ID data (len={len(next_task_id_val)})")
db.close()
except Exception as e:
logger.warning(f"Failed to read task ID from {var_db_path}: {e}")
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

⚠️ Potential issue | 🟡 Minor

Use path joining instead of string concatenation.

Line 66 constructs the database path using string concatenation, which could be fragile if embedded_db_path doesn't have the expected format.

Additionally, the hardcoded key "NI" (line 67) would benefit from a brief comment explaining what it represents (e.g., "Next ID", "Node ID", etc.).

Apply this diff to use proper path joining:

-    var_db_path = embedded_db_path + "var"
+    var_db_path = embedded_db_path + "var"  # UnQLite convention requires string concatenation for DB variants

Or if OS path joining is appropriate:

-    var_db_path = embedded_db_path + "var"
+    var_db_path = os.path.join(os.path.dirname(embedded_db_path), 
+                                os.path.basename(embedded_db_path) + "var")

Also consider adding a comment for the "NI" key:

-    next_task_id_key = "NI"
+    next_task_id_key = "NI"  # Next task ID counter key

Note: The bare Exception catch on line 77 is reasonable here since this is best-effort preservation and shouldn't fail the entire wipe operation.

📝 Committable suggestion

‼️ IMPORTANT
Carefully review the code before committing. Ensure that it accurately replaces the highlighted code, contains no missing lines, and has no issues with indentation. Thoroughly test & benchmark the code to ensure it meets the requirements.

Suggested change
# Try to preserve task ID
var_db_path = embedded_db_path + "var"
next_task_id_key = "NI"
next_task_id_val = None
if os.path.exists(var_db_path):
try:
db = UnQLite(var_db_path)
if next_task_id_key in db:
next_task_id_val = db[next_task_id_key]
logger.info(f"Found existing task ID data (len={len(next_task_id_val)})")
db.close()
except Exception as e:
logger.warning(f"Failed to read task ID from {var_db_path}: {e}")
# Try to preserve task ID
var_db_path = embedded_db_path + "var" # UnQLite convention requires string concatenation for DB variants
next_task_id_key = "NI" # Next task ID counter key
next_task_id_val = None
if os.path.exists(var_db_path):
try:
db = UnQLite(var_db_path)
if next_task_id_key in db:
next_task_id_val = db[next_task_id_key]
logger.info(f"Found existing task ID data (len={len(next_task_id_val)})")
db.close()
except Exception as e:
logger.warning(f"Failed to read task ID from {var_db_path}: {e}")
🧰 Tools
🪛 Ruff (0.14.5)

77-77: Do not catch blind exception: Exception

(BLE001)

🤖 Prompt for AI Agents
In scripts/wipe_data.py around lines 65 to 78, the code builds var_db_path by
concatenating strings and uses a magic key "NI" with no explanation; change
var_db_path to use os.path.join(embedded_db_path, "var") (or
pathlib.Path(embedded_db_path) / "var") to handle separators reliably, add a
short comment explaining that next_task_id_key = "NI" stands for the preserved
"Next ID" task counter (or the intended meaning), and keep the existing broad
Exception catch since this is best-effort preservation.

Comment on lines 92 to 104
# Restore task ID
if next_task_id_val is not None:
try:
if not db_dir.exists():
db_dir.mkdir(parents=True, exist_ok=True)

db = UnQLite(var_db_path)
db[next_task_id_key] = next_task_id_val
db.close()
logger.info(f"Restored task ID to {var_db_path}")
except Exception as e:
logger.error(f"Error restoring task ID to {var_db_path}: {e}")
raise e
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

⚠️ Potential issue | 🟡 Minor

Apply static analysis suggestions for better error handling.

The error handling can be improved by following Python best practices:

  1. Line 103: Use logging.exception instead of logging.error to include the traceback automatically.
  2. Line 104: Use bare raise instead of raise e to preserve the original traceback.

Apply this diff:

         except Exception as e:
-            logger.error(f"Error restoring task ID to {var_db_path}: {e}")
-            raise e
+            logger.exception(f"Error restoring task ID to {var_db_path}")
+            raise

Optional enhancement: Consider validating next_task_id_val before restoration (e.g., checking it's not empty or corrupted), though corrupted data would likely cause issues elsewhere anyway.

📝 Committable suggestion

‼️ IMPORTANT
Carefully review the code before committing. Ensure that it accurately replaces the highlighted code, contains no missing lines, and has no issues with indentation. Thoroughly test & benchmark the code to ensure it meets the requirements.

Suggested change
# Restore task ID
if next_task_id_val is not None:
try:
if not db_dir.exists():
db_dir.mkdir(parents=True, exist_ok=True)
db = UnQLite(var_db_path)
db[next_task_id_key] = next_task_id_val
db.close()
logger.info(f"Restored task ID to {var_db_path}")
except Exception as e:
logger.error(f"Error restoring task ID to {var_db_path}: {e}")
raise e
# Restore task ID
if next_task_id_val is not None:
try:
if not db_dir.exists():
db_dir.mkdir(parents=True, exist_ok=True)
db = UnQLite(var_db_path)
db[next_task_id_key] = next_task_id_val
db.close()
logger.info(f"Restored task ID to {var_db_path}")
except Exception as e:
logger.exception(f"Error restoring task ID to {var_db_path}")
raise
🧰 Tools
🪛 Ruff (0.14.5)

103-103: Use logging.exception instead of logging.error

Replace with exception

(TRY400)


104-104: Use raise without specifying exception name

Remove exception name

(TRY201)

🤖 Prompt for AI Agents
In scripts/wipe_data.py around lines 92 to 104, the exception handling when
restoring the task ID should be updated: replace logger.error(...) with
logger.exception(...) so the traceback is logged, and replace raise e with a
bare raise to preserve the original traceback; additionally, optionally validate
next_task_id_val (e.g., ensure it's not None/empty or of expected type/format)
before writing to UnQLite and log/raise a clear error if validation fails.

Comment on lines +586 to +605
// FIXME: Following task fields should set by scheduler
task_id_t cur_task_id = 0;
if (job.IsBatch() || job.IsCalloc()) {
// Batch/Calloc: will launch one task on one node only
craned_task_map[job.executing_craned_ids.front()].insert(cur_task_id);
task_res_map[cur_task_id] =
job.AllocatedRes().at(job.executing_craned_ids.front());
} else {
for (const auto& craned_id : job.CranedIds()) {
for (int i = 0; i < job.ntasks_per_node; ++i) {
craned_task_map[craned_id].insert(cur_task_id);
auto res = job.AllocatedRes().at(craned_id);
// Mem is allocated at step level, set to 0 here to avoid mem limit.
res.allocatable_res.memory_bytes = 0;
res.allocatable_res.memory_sw_bytes = 0;
task_res_map[cur_task_id] = res;
++cur_task_id;
}
}
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

⚠️ Potential issue | 🟡 Minor

🧩 Analysis chain

Verify: Per-task resource initialization uses CranedIds() for non-batch tasks.

For batch/calloc tasks, executing_craned_ids.front() is used correctly (single execution node). However, for other task types (line 594), the loop iterates over job.CranedIds() rather than job.executing_craned_ids.

For interactive crun tasks with pty, executing_craned_ids contains only the first node while CranedIds() contains all allocated nodes. Please verify this is the intended behavior - should tasks be created for all allocated nodes or only executing nodes?


🏁 Script executed:

#!/bin/bash
# Verify how executing_craned_ids vs CranedIds are populated for different task types
rg -n "executing_craned_ids" src/CraneCtld/ -A 3 -B 3

Length of output: 8434


Minor inconsistency: Use executing_craned_ids consistently instead of CranedIds() in the else branch.

The code is functionally correct: for batch/calloc tasks (line 588), it rightfully uses executing_craned_ids.front(), and for other tasks (line 594), CranedIds() happens to match executing_craned_ids (both contain all nodes for interactive crun tasks). However, for consistency and clarity, the else branch should iterate over job.executing_craned_ids instead of job.CranedIds(), since executing_craned_ids explicitly defines the nodes where tasks execute, as determined during job initialization.

🤖 Prompt for AI Agents
In src/CraneCtld/CtldPublicDefs.cpp around lines 586 to 605, the else branch
iterates over job.CranedIds() but should consistently iterate over
job.executing_craned_ids (the actual nodes where tasks execute); change the loop
to use job.executing_craned_ids (use a const reference iteration as used
elsewhere) so task assignment uses the explicit executing list, leaving the rest
of the body unchanged.

Comment on lines 355 to 369
task_id_t task_id;
while (m_stop_task_io_queue_.try_dequeue(task_id)) {
absl::MutexLock lock(&m_mtx_);
auto it = m_fwd_meta_map.find(task_id);
if (it != m_fwd_meta_map.end()) {
{
absl::MutexLock lock(&m_mtx_);
auto it = m_fwd_meta_map.find(task_id);
if (it == m_fwd_meta_map.end()) {
continue;
}
auto& output_handle = it->second.out_handle;
if (!it->second.pty && output_handle.pipe) output_handle.pipe->close();
if (it->second.pty && output_handle.tty) output_handle.tty->close();
output_handle.pipe.reset();
output_handle.tty.reset();
close(it->second.stdout_read);
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

⚠️ Potential issue | 🟠 Major

Avoid closing stdout_read after libuv handle close to prevent double-close/race on FD reuse

In CleanStopTaskIOQueueCb_, after closing the pipe_handle/tty_handle, you still call close(it->second.stdout_read);. Libuv will close the underlying fd when the handle is closed, and that close is asynchronous on the event loop thread. Calling close() again on the raw fd risks either:

  • closing an already-closed descriptor (harmless but noisy if reused later); or
  • closing a different file if the same fd number gets reused before this code runs.

Given the handle owns the lifetime of the fd, it’s safer to drop the explicit close() here and only keep the error‑path closes where no handle was created.

-      output_handle.pipe.reset();
-      output_handle.tty.reset();
-      close(it->second.stdout_read);
+      output_handle.pipe.reset();
+      output_handle.tty.reset();
+      // Underlying fd is closed by the libuv handle; avoid double-closing here.
🤖 Prompt for AI Agents
In src/Craned/Supervisor/CforedClient.cpp around lines 355-369, the code
explicitly calls close(it->second.stdout_read) after closing/resetting libuv
pipe/tty handles which own and asynchronously close the fd; remove that
unconditional close to avoid double-close or racing fd reuse, and instead only
close the raw fd in code paths where no libuv handle was created (i.e., when
pipe/tty handles are null). After removing the unconditional close, ensure any
error-paths that still manually close the fd do so and then reset stdout_read to
an invalid value (e.g., -1) to avoid accidental reuse.

Signed-off-by: Li Junlin <[email protected]>

fix: Subprocess cg

Signed-off-by: Li Junlin <[email protected]>

fix: Only init cfored for ia task

Signed-off-by: Li Junlin <[email protected]>

fix: Exit reason

Signed-off-by: Li Junlin <[email protected]>

fix: Deadlock

Signed-off-by: Li Junlin <[email protected]>

style: auto format with clang-format.

fix

Signed-off-by: Li Junlin <[email protected]>

fix: tasks step exit code

Signed-off-by: Li Junlin <[email protected]>

fix: Share mem for tasks in a same step on node

Signed-off-by: Li Junlin <[email protected]>

feat: Support task execution

Signed-off-by: Li Junlin <[email protected]>

fix: Cgroup recovery

Signed-off-by: Li Junlin <[email protected]>

refactor: Use std::chrono_literals

Signed-off-by: Li Junlin <[email protected]>

refactor

Signed-off-by: Li Junlin <[email protected]>

refactor: PAM

Signed-off-by: Li Junlin <[email protected]>

move user cgroup of daemon step to Supervisor

Signed-off-by: Li Junlin <[email protected]>

Update TaskScheduler.cpp

feat: Step cgroup recovery

Signed-off-by: Li Junlin <[email protected]>

refactor: Supervisor step status.

Signed-off-by: Li Junlin <[email protected]>

refactor: Step instance

Signed-off-by: Li Junlin <[email protected]>

feat: Add CPU binding functionality for cgroup v1 and v2

Signed-off-by: Li Junlin <[email protected]>
Copy link
Contributor

@coderabbitai coderabbitai bot left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Actionable comments posted: 2

Caution

Some comments are outside the diff and can’t be posted inline due to platform limitations.

⚠️ Outside diff range comments (6)
src/Craned/Supervisor/CforedClient.cpp (2)

410-470: Unified output queue writer: X11 payload fields currently not populated

The new CleanOutputQueueAndWriteToStreamThread_ loop over FwdRequest works as intended: it drains the queue until m_stopped_ and the queue are empty, sets the type, and fills the right oneof payload for output / X11 output / exit status before writing, honoring write_pending.

For TASK_X11_OUTPUT, though, all the interesting fields are commented out:

// payload->set_x11_id(x11_fwd_req.x11_id);
// payload->set_eof(x11_fwd_req.eof);
// payload->set_craned_id(g_config.CranedIdOfThisNode);

This means cfored currently only sees the raw X11 bytes with defaulted IDs/EOF/craned_id. If the new protocol expects those fields, this will limit or break per-connection/per-craned routing and EOF handling. Please either wire these fields up now or add a clear TODO explaining why they’re deliberately disabled.

Also applies to: 436-444


109-115: Caller of InitUvX11FwdHandler() must validate return value; currently no error check for 0 port

Verification confirms the issue. The function returns 0 on failure (when all ports 6020–6100 fail to bind or listen), but the single call site in TaskManager.cpp:80 assigns the return value directly to x11_meta.x11_port without checking for 0. If setup fails, the invalid port value is later used in DISPLAY environment variable assignments (lines 458, 704), resulting in silent failure rather than error propagation. Add explicit validation after line 80:

x11_meta.x11_port = cfored_client->InitUvX11FwdHandler();
if (x11_meta.x11_port == 0) {
  CRANE_ERROR("Failed to setup X11 forwarding");
  return CraneErrCode::ERR_SYSTEM_ERR;
}
src/Craned/Core/CranedServer.cpp (1)

330-338: Based on my verification, I now have a complete picture. Let me generate the rewritten review comment:

The error-checking pattern err.error() is non-idiomatic and inconsistent with the codebase.

While err.error() technically works because CraneErrCode is crane::grpc::ErrCode (a protobuf enum where SUCCESS = 0 converts to false and error codes convert to true), this pattern is inconsistent with idiomatic std::expected usage throughout the codebase. Other code in CranedServer.cpp uses if (!expt) to check for errors (e.g., lines 377–380, 398–401, 456–459), which is the standard idiom. Consider changing line 333 from if (err.error()) to if (!err) for consistency.

src/Craned/Core/JobManager.cpp (1)

878-894: Missing null check for supervisor_stub before calling TerminateTask.

At line 880-882, step_instance->supervisor_stub is retrieved and TerminateTask is called without verifying the stub is initialized. This could crash if the supervisor failed to start for this step.

  auto& step_instance = job_instance->step_map.at(step_id);
  auto stub = step_instance->supervisor_stub;
+ if (!stub) {
+   CRANE_ERROR("[Step #{}.{}] Supervisor stub is null when terminating.",
+               elem.job_id, step_id);
+   continue;
+ }
  auto err =
      stub->TerminateTask(elem.mark_as_orphaned, elem.terminated_by_user);
src/Craned/Core/Craned.cpp (1)

996-1012: Missing null check for supervisor_stub when terminating failed steps.

At lines 999-1004, stub->TerminateTask is called without verifying the stub is valid. During recovery, if a step's supervisor socket was found but the connection failed, the stub might be null.

  for (auto [job_id, step_ids] : failed_steps) {
    for (auto step_id : step_ids) {
      {
        auto& stub = step_map.at({job_id, step_id})->supervisor_stub;
+       if (!stub) {
+         CRANE_WARN("[Step #{}.{}] Supervisor stub is null during recovery cleanup.",
+                    job_id, step_id);
+         step_map.erase({job_id, step_id});
+         continue;
+       }
        auto err = stub->TerminateTask(true, false);
src/Craned/Supervisor/TaskManager.cpp (1)

447-460: I'll help you verify the review comment about X11 metadata safety in TaskManager.cpp. Let me start by examining the codebase to understand the current state of the code and validate the concerns raised.
<function_calls>

find . -type f -name "TaskManager.cpp" | head -5

</function_calls>
<function_calls>

wc -l src/Craned/Supervisor/TaskManager.cpp

</function_calls>
<function_calls>

sed -n '440,470p' src/Craned/Supervisor/TaskManager.cpp

</function_calls>
<function_calls>

sed -n '680,715p' src/Craned/Supervisor/TaskManager.cpp

</function_calls>

Now let me find the StepInstance definition to understand the x11_meta type:
<function_calls>

find . -type f -name "*.h" | xargs grep -l "class StepInstance" | head -3

</function_calls>
<function_calls>

grep -n "x11_meta" src/Craned/Supervisor/TaskManager.h | head -20

</function_calls>

Let me search for the StepInstance class and x11_meta member more broadly:
<function_calls>

rg -n "class StepInstance" --type cpp --type h

</function_calls>
<function_calls>

rg -n "x11_meta" --type cpp --type h -B 2 -A 2 | grep -E "(x11_meta|optional|std::)" | head -40

</function_calls>

Let me examine the Prepare() method and EvGrpcExecuteTaskCb_:
<function_calls>

rg -n "EvGrpcExecuteTaskCb_" --type cpp --type h -B 3 -A 15 | head -60

</function_calls>
<function_calls>

rg -n "Prepare()" src/Craned/Supervisor/TaskManager.cpp -B 2 -A 10 | head -80

</function_calls>

Let me look at the complete context around the x11_meta usage at lines 447-460:
<function_calls>

sed -n '430,475p' src/Craned/Supervisor/TaskManager.cpp

</function_calls>

Now let me check the class hierarchy and understand ProcInstance:
<function_calls>

rg -n "class ProcInstance" --type cpp --type h -A 20 | head -40

</function_calls>

Let me search for where x11_meta is assigned/initialized:
<function_calls>

rg -n "x11_meta\s*=" src/Craned/Supervisor/ --type cpp -B 2 -A 2 | head -80

</function_calls>

Let me check the header files for type definition:
<function_calls>

find src/Craned/Supervisor -name "*.h" -type f

</function_calls>
<function_calls>

grep -n "x11_meta" src/Craned/Supervisor/TaskManager.h | head -20

</function_calls>

Let me get the full definition of x11_meta in the StepInstance:
<function_calls>

sed -n '1,100p' src/Craned/Supervisor/TaskManager.h

</function_calls>

Let me search for the complete StepInstance class definition:
<function_calls>

ast-grep --pattern 'class StepInstance { $$$ }' --type h

</function_calls>

Let me try a simpler approach to find the full class:
<function_calls>

rg -n "std::optional.*x11_meta|x11_meta.*optional" --type h --type cpp

</function_calls>

Let me check the TaskManager.h file size and content:
<function_calls>

wc -l src/Craned/Supervisor/TaskManager.h && head -200 src/Craned/Supervisor/TaskManager.h

</function_calls>

Let me search for where x11_meta is declared with its type:
<function_calls>

rg -n "x11_meta" src/Craned/Supervisor/TaskManager.h

</function_calls>

Let me examine the line ranges more carefully to understand the context and check what errors Prepare returns:
<function_calls>

sed -n '1,50p' src/Craned/Supervisor/TaskManager.h

</function_calls>

Let me try a different approach - search for initialization and declaration patterns:
<function_calls>

python3 << 'EOF'
import subprocess
import re

Search for x11_meta in all header files

result = subprocess.run(['rg', '-n', 'x11_meta', 'src/Craned/Supervisor/', '--type', 'h', '-B', '5', '-A', '2'],
capture_output=True, text=True)
print(result.stdout)
EOF


</function_calls>

Let me look at the proto files that might define the structure:
<function_calls>

find . -name "*.proto" | xargs grep -l "x11" | head -5

</function_calls>

Let me get a broader view of how x11_meta is used:
<function_calls>

rg -n ".x11_meta" src/Craned/Supervisor/TaskManager.cpp -B 1 -A 1 | head -50

</function_calls>

Now let me check the specific sections mentioned more carefully:
<function_calls>

sed -n '400,430p' src/Craned/Supervisor/TaskManager.cpp

</function_calls>

Let me check Prepare() implementation and error handling:
<function_calls>

grep -n "bool.*Prepare|CraneErr.*Prepare" src/Craned/Supervisor/TaskManager.cpp | head -20

</function_calls>
<function_calls>

rg -n "bool Prepare" src/Craned/Supervisor/ --type cpp -A 100 | head -150

</function_calls>

Let me search for SetupChildProcCrunX11_:
<function_calls>

rg -n "SetupChildProcCrunX11_" src/Craned/Supervisor/TaskManager.cpp

</function_calls>

Now let me search for where Prepare is called and how its result is handled:
<function_calls>

rg -n "->Prepare()" src/Craned/Supervisor/TaskManager.cpp -B 3 -A 5 | head -100

</function_calls>

Let me check EvGrpcExecuteTaskCb_ more thoroughly:
<function_calls>

grep -n "EvGrpcExecuteTaskCb_" src/Craned/Supervisor/TaskManager.cpp

</function_calls>

Let me get the function around line 1000 (guessing where the callback might be):
<function_calls>

sed -n '950,1050p' src/Craned/Supervisor/TaskManager.cpp

</function_calls>

Let me search for this more broadly:
<function_calls>

rg -n "EvGrpcExecuteTaskCb_" src/Craned/Supervisor/ --type cpp

</function_calls>

Let me search across all cpp files:
<function_calls>

rg -rn "EvGrpcExecuteTaskCb_" src/ --type cpp | head -10

</function_calls>

Let me check how x11_meta is handled - is it optional?
<function_calls>

python3 << 'EOF'
import re

Read the TaskManager.cpp file

with open('src/Craned/Supervisor/TaskManager.cpp', 'r') as f:
lines = f.readlines()

Look for x11_meta usage and surrounding context

for i, line in enumerate(lines):
if 'x11_meta' in line and ('->x11_meta' in line or '.x11_meta' in line):
print(f"Line {i+1}: {line.rstrip()}")
# Print context
start = max(0, i-3)
end = min(len(lines), i+4)
for j in range(start, end):
if j == i:
print(f">> {j+1}: {lines[j].rstrip()}")
else:
print(f" {j+1}: {lines[j].rstrip()}")
print()
EOF


</function_calls>

Let me now check the proto files to understand the structure:
<function_calls>

find . -name "*.proto" -type f | xargs grep -l "X11Meta|x11_meta" | head -5

</function_calls>
<function_calls>

rg -n "X11Meta|x11_meta" src/ --type proto | head -30

</function_calls>

Let me check the generated proto files or look for the original definitions:
<function_calls>

find src -name ".pb.h" -o -name ".pb.cc" | head -10

</function_calls>

Let me search the entire src directory for how x11_meta might be of optional type:
<function_calls>

rg -n "absl::optional.*x11_meta|std::optional.*x11_meta|has_x11|x11_meta.*has_value|x11_meta.*value()" src/ --type cpp --type h | head -30

</function_calls>

Let me look at the exact lines in question again and get a wider context:
<function_calls>

sed -n '420,500p' src/Craned/Supervisor/TaskManager.cpp

</function_calls>

Let me look for SetupChildProcessCrunX11 or similar naming:
<function_calls>

rg -n "SetupChildProcess.*X11|SetupChild.*X11" src/Craned/Supervisor/TaskManager.cpp

</function_calls>

Let me search with slightly different patterns:
<function_calls>

rg -n "X11" src/Craned/Supervisor/TaskManager.cpp | head -30

</function_calls>

Now let me look at lines 691-705 as mentioned in the review:
<function_calls>

sed -n '685,720p' src/Craned/Supervisor/TaskManager.cpp

</function_calls>

Let me search for the actual structure ofStepInstance to understand x11_meta:
<function_calls>

rg -n "class StepInstance|struct StepInstance" src/ --type cpp --type h -A 100 | head -200

</function_calls>

Let me check what ITaskInstance is:
<function_calls>

rg -n "class ITaskInstance|struct ITaskInstance" src/ --type cpp --type h -B 2 -A 50 | head -100

</function_calls>

Let me find the GetStep() method return type:
<function_calls>

rg -n "GetStep()" src/Craned/Supervisor/ --type cpp --type h -B 2 -A 2 | head -50

</function_calls>

Let me approach this differently - check what interactive_meta() returns:
<function_calls>

rg -n "interactive_meta|ia_meta" src/Craned/Supervisor/TaskManager.cpp -B 2 -A 2 | head -80

</function_calls>

Now let me understand the proto definition:
<function_calls>

find src -name "*.proto" | xargs cat | grep -A 20 "message.*Interactive"

</function_calls>

Let me look for all proto files:
<function_calls>

find src -name "*.proto" -type f

</function_calls>
<function_calls>

grep -n "x11" src/protos/task.proto

</function_calls>

Great! Now let me look at the task.proto to understand the structure:
<function_calls>

grep -n "message|X11Meta|x11_meta|interactive" src/protos/task.proto | head -40

</function_calls>

Let me get the full definitions:
<function_calls>

sed -n '1,50p' src/protos/task.proto

</function_calls>

Let me search for X11Meta message definition:
<function_calls>

rg -n "message.*X11Meta|message.*Interactive" src/protos/task.proto -A 15

</function_calls>

Perfect! Now let me check how x11_meta is actually used. Let me look for the repeated usage pattern:
<function_calls>

python3 << 'EOF'
import subprocess

Get all uses of x11_meta in TaskManager.cpp

result = subprocess.run(['rg', '-n', 'x11_meta', 'src/Craned/Supervisor/TaskManager.cpp', '-B', '3', '-A', '3'],
capture_output=True, text=True)

lines = result.stdout.split('\n')
for line in lines:
if 'x11_meta' in line or 'value()' in line or 'has_value' in line:
print(line)
EOF


</function_calls>

Let me check how x11_meta is assigned in the Prepare logic:
<function_calls>

rg -n "bool.*Prepare|CraneErr.*Prepare" src/Craned/Supervisor/TaskManager.cpp | head -10

</function_calls>

Let me search for StepInstance Prepare method:
<function_calls>

rg -n "StepInstance.*Prepare|bool Prepare" src/Craned/Supervisor/TaskManager.cpp -B 2 -A 50 | head -150

</function_calls>

Let me look at the lines where Prepare is called:
<function_calls>

rg -n ".Prepare()|->Prepare()" src/Craned/Supervisor/TaskManager.cpp -B 5 -A 10 | head -100

</function_calls>

Let me search more carefully for how the error from Prepare() is handled:
<function_calls>

grep -rn "step_inst.*Prepare|m_step_inst.*Prepare" src/Craned/Supervisor/ --type cpp -B 3 -A 5

</function_calls>

Let me look for the complete method that calls Prepare():
<function_calls>

rg -n "Execute|Execute(" src/Craned/Supervisor/TaskManager.cpp | grep -i "task|exec" | head -20

</function_calls>

Let me look at lines around where x11_meta is being set up to understand the error handling:
<function_calls>

sed -n '430,500p' src/Craned/Supervisor/TaskManager.cpp | cat -n

</function_calls>

Let me search for the condition that should guard x11_meta usage:
<function_calls>

sed -n '447,470p' src/Craned/Supervisor/TaskManager.cpp

</function_calls>

Now let me check if there's a second usage location at lines 691-705:
<function_calls>

sed -n '690,710p' src/Craned/Supervisor/TaskManager.cpp

</function_calls>

Let me now search for where x11_meta is initialized/set:
<function_calls>

rg -n "x11_meta\s*=" src/Craned/Supervisor/ --type cpp -B 3 -A 3

</function_calls>

Let me check if it's set in the proto-generated code:
<function_calls>

rg -n "set_x11_meta|mutable_x11_meta" src/ --type cpp | head -20

</function_calls>

Let me examine the actual Prepare() method implementation:
<function_calls>

rg -n "^bool StepInstance::Prepare" src/Craned/Supervisor/TaskManager.cpp -A 200 | head -250

</function_calls>

Now let me understand how x11_meta could be uninitialized. Let me look for the class member:
<function_calls>

rg -n "std::optional.*x11_meta" src/ --type cpp --type h

</function_calls>

Let me look at proto messages more carefully to understand if x11_meta is optional:
<function_calls>

grep -A 50 "message TaskLaunchRequest" src/protos/task.proto | head -60

</function_calls>

Let me check the proto file structure more carefully:
<function_calls>

wc -l src/protos/task.proto && cat src/protos/task.proto

</function_calls>

Perfect! Now I can see the proto structure. Let me verify how x11_meta is used when it might not be set. Let me check the line 457 onwards:
<function_calls>

sed -n '447,465p' src/Craned/Supervisor/TaskManager.cpp

</function_calls>

Now let me check lines 691-705:
<function_calls>

sed -n '691,706p' src/Craned/Supervisor/TaskManager.cpp

</function_calls>

Let me look at the broader context of where Prepare is actually called and how errors are handled:
<function_calls>

rg -n "step_instance->Prepare()|m_step_instance->Prepare()|step_inst->Prepare" src/Craned/Supervisor/ --type cpp -B 10 -A 10 | head -100

</function_calls>

Let me search for how the return value of Prepare() is handled:
<function_calls>

rg -n "Prepare()" src/Craned/Supervisor/TaskManager.cpp | head -30

</function_calls>

Let me look for the specific instance where Prepare is called:
<function_calls>

sed -n '550,650p' src/Craned/Supervisor/TaskManager.cpp

</function_calls>

Let me search more systematically for where Prepare is called with error handling:
<function_calls>

python3 << 'EOF'
import re

with open('src/Craned/Supervisor/TaskManager.cpp', 'r') as f:
content = f.read()
lines = content.split('\n')

Find Prepare() calls

for i, line in enumerate(lines):
if 'Prepare()' in line and ('->' in line or '.' in line):
# Print context
start = max(0, i - 5)
end = min(len(lines), i + 10)
print(f"\n--- Context around line {i+1} ---")
for j in range(start, end):
prefix = ">>> " if j == i else " "
print(f"{prefix}{j+1}: {lines[j]}")
EOF


</function_calls>

Now let me look at the bigger picture - find the Execute method and check error handling:
<function_calls>

rg -n "Execute()" src/Craned/Supervisor/TaskManager.cpp | head -20

</function_calls>

Let me look at the TaskManager::Execute method:
<function_calls>

rg -n "void TaskManager::Execute|bool TaskManager::Execute" src/Craned/Supervisor/TaskManager.cpp -A 150 | head -200

</function_calls>

Perfect! Now let me trace through this. I can see the Prepare() call is at line 608. Let me check that specific location and see how errors are handled:
<function_calls>

sed -n '595,625p' src/Craned/Supervisor/TaskManager.cpp

</function_calls>

Excellent! Now I can see the issue clearly. Let me verify if there's any error checking after Prepare():
<function_calls>

sed -n '605,650p' src/Craned/Supervisor/TaskManager.cpp

</function_calls>

Perfect! Now let me verify the learning from memory about the GetCrunMeta context. Let me check that:
<function_calls>

rg -n "GetCrunMeta|IsCrun" src/Craned/Supervisor/TaskManager.cpp -B 2 -A 2 | head -40

</function_calls>

Let me verify if x11_meta checks exist elsewhere in the codebase:
<function_calls>

rg -n "x11_meta.*has_value|x11_meta.*value()|!.*x11_meta" src/Craned/Supervisor/ --type cpp | head -30

</function_calls>

Let me look at what proto says about optional fields in the message definition:
<function_calls>

grep -n "optional.*X11Meta|X11Meta.*x11_meta" src/protos/task.proto

</function_calls>

Let me look at the full Interactive message to understand what's optional:
<function_calls>

grep -B 2 -A 15 "message Interactive" src/protos/task.proto

</function_calls>

Now I see - the x11_meta field is not declared in the proto message I saw. Let me check if it's part of Step:
<function_calls>

grep -B 5 -A 20 "message Step" src/protos/task.proto

</function_calls>

Great! I can see Step has optional X11Meta x11_meta = 15;. Now let me understand how proto3 optional fields work and if they're checked:
<function_calls>

rg -n "x11_meta()" src/Craned/Supervisor/TaskManager.cpp -B 1 -A 1

</function_calls>

Now let me check if there are has_ or value() checks for optional fields:
<function_calls>

rg -n "has_x11_meta|x11_meta.*has_value" src/ --type cpp

</function_calls>

Now let me understand the actual issue - check the lines again with IsCrun context:
<function_calls>

sed -n '445,475p' src/Craned/Supervisor/TaskManager.cpp

</function_calls>

Perfect! I can see the issue now. The code checks if (ia_meta.x11()) which checks if x11 is enabled, but then it accesses m_parent_step_inst_->x11_meta which is a different field. Let me verify the fields:
<function_calls>

sed -n '690,710p' src/Craned/Supervisor/TaskManager.cpp

</function_calls>

Now let me check how x11_meta is initialized in Prepare():
<function_calls>

rg -n "StepInstance::Prepare" src/Craned/Supervisor/TaskManager.cpp -A 300 | grep -A 300 "^src" | head -350

</function_calls>

Let me narrow down and get StepInstance::Prepare():
<function_calls>

sed -n '1050,1200p' src/Craned/Supervisor/TaskManager.cpp | grep -n "Prepare|x11_meta|return|bool"

</function_calls>

Let me find the Prepare method in StepInstance:
<function_calls>

sed -n '1000,1300p' src/Craned/Supervisor/TaskManager.cpp

</function_calls>

Good! Now let me check around line 1049 to see if there's error handling and x11_meta setup:
<function_calls>

sed -n '1030,1150p' src/Craned/Supervisor/TaskManager.cpp

</function_calls>

Now let me look specifically around CreateFoldersForFileEx mentioned in the review:
<function_calls>

rg -n "CreateFoldersForFileEx|mkstemp|fchown" src/Craned/Supervisor/TaskManager.cpp -B 3 -A 3

</function_calls>

Let me search for Prepare method and its implementation:
<function_calls>

sed -n '1100,1250p' src/Craned/Supervisor/TaskManager.cpp

</function_calls>

Let me look for where x11_meta is assigned:
<function_calls>

rg -n "m_.step.->x11_meta\s*=" src/ --type cpp

</function_calls>

Let me check if it's a proto message member that gets populated:
<function_calls>

rg -n "step.*mutable_x11_meta|set_x11_meta" src/Craned/Supervisor/ --type cpp

</function_calls>

Let me look at the entire Prepare method:
<function_calls>

python3 << 'EOF'
with open('src/Craned/Supervisor/TaskManager.cpp', 'r') as f:
lines = f.readlines()

Find StepInstance::Prepare

in_prepare = False
for i, line in enumerate(lines):
if 'bool StepInstance::Prepare()' in line:
in_prepare = True
print(f"Found Prepare at line {i+1}")
# Print from this point until we hit another method
for j in range(i, min(i+250, len(lines))):
print(f"{j+1}: {lines[j]}", end='')
# Stop when we see the next method definition
if j > i and ('bool ' in lines[j] or 'void ' in lines[j]) and '::' in lines[j]:
break
break
EOF


</function_calls>

Perfect! Now I have the full Prepare() method. Let me verify the key facts:

  1. The Prepare() method doesn't explicitly set m_step_->x11_meta
  2. The x11_meta appears to be part of the proto Step message
  3. When Prepare() fails in the X11 path, the m_step_ proto message might not have x11_meta populated

Let me check what gets assigned to m_step_:
<function_calls>

rg -n "m_step_\s*=" src/Craned/Supervisor/TaskManager.cpp -B 2 -A 2

</function_calls>

Now let me look at how the Step proto is initialized:
<function_calls>

rg -n "StepInstance(|m_step_|Step(" src/Craned/Supervisor/TaskManager.cpp | head -30

</function_calls>

Let me look at the StepInstance constructor:
<function_calls>

rg -n "StepInstance::StepInstance" src/Craned/Supervisor/TaskManager.cpp -A 20

</function_calls>

Perfect! Now I understand - m_step_ is const and assigned in constructor. Let me now trace through to see if x11_meta can be uninitialized. Let me check if x11() in proto means the x11_meta field is populated:
<function_calls>

grep -n "bool.*x11|x11()" src/protos/task.proto

</function_calls>

Now I see the full picture. Let me check the actual condition check:
<function_calls>

sed -n '447,465p' src/Craned/Supervisor/TaskManager.cpp

</function_calls>

So the issue is:

  • Line 449: if (ia_meta.x11()) checks if x11 is enabled
  • Line 451: auto const& x11_meta = ia_meta.x11_meta() gets the x11_meta from ia_meta (not from m_parent_step_inst_->x11_meta)
  • Line 454-455: m_parent_step_inst_->x11_meta->x11_port accesses m_parent_step_inst_->x11_meta directly

Let me verify which x11_meta is which:
<function_calls>

python3 << 'EOF'
with open('src/Craned/Supervisor/TaskManager.cpp', 'r') as f:
lines = f.readlines()

Get lines 447-465

for i in range(446, 465):
print(f"{i+1}: {lines[i]}", end='')
EOF


</function_calls>

Let me check the proto to understand the hierarchy:
<function_calls>

grep -A 5 "message Interactive" src/protos/task.proto

</function_calls>

Now let me check the Step message:
<function_calls>

grep -B 2 -A 25 "^message Step" src/protos/task.proto

</function_calls>

So the hierarchy is:

  • Step contains optional Interactive interactive_meta
  • Interactive contains bool x11 and X11Meta x11_meta
  • Step also contains a separate optional X11Meta x11_meta field

So there are TWO x11_meta fields:

  1. interactive_meta().x11_meta() - inside the Interactive message
  2. m_parent_step_inst_->x11_meta - directly on Step

Now let me understand if they're linked. Let me check the Prepare() method to see if it sets the Step's x11_meta:
<function_calls>

python3 << 'EOF'
import re

with open('src/Craned/Supervisor/TaskManager.cpp', 'r') as f:
content = f.read()
lines = content.split('\n')

Find the Prepare method and search for x11_meta assignments

for i, line in enumerate(lines):
if 'bool StepInstance::Prepare()' in line:
# Look for the next 300 lines for x11_meta assignments
for j in range(i, min(i+300, len(lines))):
if 'x11_meta' in lines[j]:
print(f"{j+1}: {lines[j]}")
# Stop if we hit another method
if j > i and re.match(r'\s*(bool|void|CraneErr|auto).*::', lines[j]):
break
break
EOF


</function_calls>

Let me look for x11_meta references more carefully in Prepare:
<function_calls>

sed -n '1049,1200p' src/Craned/Supervisor/TaskManager.cpp | grep -n "x11"

</function_calls>

So I don't see x11_meta being set in Prepare(). The Step's x11_meta must come from somewhere else. Let me check how StepInstance is created:
<function_calls>

rg -n "new StepInstance|StepInstance(" src/Craned/Supervisor/ --type cpp -B 3 -A 3 | head -50

</function_calls>

Let me check the TaskManager::Execute method where StepInstance is created:
<function_calls>

sed -n '568,610p' src/Craned/Supervisor/TaskManager.cpp

</function_calls>

So step_instance is created from m_task_->steps(i). The Step comes from the task proto message. Now let me check if in the error condition during Prepare, the Step's x11_meta might not be populated even though x11 is enabled.

Let me look at what happens if the Prepare fails - specifically around the mkstemp failure mentioned in the review:
<function_calls>

sed -n '1100,1150p' src/Craned/Supervisor/TaskManager.cpp

</function_calls>

Let me find where CreateFoldersForFileEx is called in Prepare:
<function_calls>

rg -n "CreateFoldersForFileEx" src/Craned/Supervisor/TaskManager.cpp -B 5 -A 10 | head -40

</function_calls>

Let me look at the full Prepare method code again focusing on the order:
<function_calls>

sed -n '1049,1130p' src/Craned/Supervisor/TaskManager.cpp

</function_calls>

Now let me check if x11_meta is accessed without guarding. Let me check if it could be null/uninitialized:
<function_calls>

python3 << 'EOF'

The key question: is Step.x11_meta guaranteed to have a value if Interactive.x11() is true?

Looking at the proto:

message Interactive {

bool x11 = 1;

optional X11Meta x11_meta = 2;

}

message Step {

optional Interactive interactive_meta = 14;

optional X11Meta x11_meta = 15; <-- THIS ONE IS ACCESSED

}

The code at lines 447-460:

const auto& ia_meta = m_parent_step_inst_->GetStep().interactive_meta();

if (!ia_meta.term_env().empty()) m_env_.emplace("TERM", ia_meta.term_env());

if (ia_meta.x11()) {

auto const& x11_meta = ia_meta.x11_meta(); <-- local var, gets from interactive_meta

// ...

m_env_["DISPLAY"] = fmt::format(

"{}:{}", target, m_parent_step_inst_->x11_meta->x11_port - 6000); <-- Step's x11_meta!

m_env_["XAUTHORITY"] = m_parent_step_inst_->x11_meta->x11_auth_path;

}

The issue: The code checks ia_meta.x11() and uses ia_meta.x11_meta(),

but then accesses m_parent_step_inst_->x11_meta (Step's x11_meta),

which might not be populated!

print("CONFIRMED: The two x11_meta are different")
print("- ia_meta.x11_meta() is from Interactive message")
print("- m_parent_step_inst_->x11_meta is from Step message")
print("\nThe code assumes they're both populated, but only checks ia_meta.x11()")
EOF


</function_calls>

Perfect! Now I understand the issue. Let me verify this is actually a problem by checking if Step.x11_meta is always set when x11 is enabled. Let me search for where Step.x11_meta gets populated:
<function_calls>

rg -n "mutable_x11_meta|set_allocated_x11_meta" src/ --type cpp -B 3 -A 3 | head -40

</function_calls>

Let me check if there's any code that populates Step.x11_meta specifically:
<function_calls>

rg -n "x11_meta(" src/ --type cpp | grep -v "ia_meta|interactive_meta" | head -20

</function_calls>

Let me now check if there's a has_ check for proto optional:
<function_calls>

rg -n "has_x11_meta" src/ --type cpp

</function_calls>

No has_x11_meta checks found. Now let me verify whether accessing an uninitialized optional proto field would cause undefined behavior:
<function_calls>

web_search
Check C++ protobuf proto3 optional field access without checking has_value behavior undefined behavior


</function_calls>

Now let me verify the current state one more time by looking at the exact lines in question:
<function_calls>

sed -n '447,465p' src/Craned/Supervisor/TaskManager.cpp && echo "\n---SECOND LOCATION---" && sed -n '691,710p' src/Craned/Supervisor/TaskManager.cpp

</function_calls>

Perfect! Now let me verify if there's been any change by looking at whether there are any asserts or checks in the current code:
<function_calls>

sed -n '440,475p' src/Craned/Supervisor/TaskManager.cpp

</function_calls>

Now let me check the second location more carefully:
<function_calls>

sed -n '680,715p' src/Craned/Supervisor/TaskManager.cpp

</function_calls>

Perfect! Now let me check if there's a .value() call which would indicate proper optional handling:
<function_calls>

sed -n '691,710p' src/Craned/Supervisor/TaskManager.cpp | cat -n

</function_calls>

So at line 699 (691+8): auto x11_meta = m_parent_step_inst_->x11_meta.value();

This shows that m_parent_step_inst_->x11_meta is being used with .value() which indicates it's std::optional or similar. Let me check how it's declared:
<function_calls>

rg -n "class StepInstance|struct StepInstance" src/ --type h -A 50 | head -80

</function_calls>

Let me find the StepInstance class definition:
<function_calls>

sed -n '1,200p' src/Craned/Supervisor/TaskManager.h

</function_calls>

Let me look for the member variable:
<function_calls>

rg -n "x11_meta" src/Craned/Supervisor/TaskManager.h

</function_calls>

Let me check if it's inherited from ITaskInstance:
<function_calls>

rg -n "x11_meta" src/Craned/Supervisor/ --type h -B 2 -A 2

</function_calls>

Let me search the entire src directory:
<function_calls>

find src -name "*.h" | xargs grep -n "x11_meta" | head -20

</function_calls>

So x11_meta is only found in TaskManager.cpp. Let me check if it's a derived class member. Let me find where x11_meta is declared for ProcInstance or StepInstance:
<function_calls>

rg -n "m_parent_step_inst_->x11_meta" src/Craned/Supervisor/TaskManager.cpp | head -20

</function_calls>

So it's accessed via m_parent_step_inst_ which is of type StepInstance*. Let me find the StepInstance class more carefully:
<function_calls>

rg -n "class.*StepInstance|struct.*StepInstance" src/ --type cpp --type h

</function_calls>

It seems StepInstance is defined in TaskManager.cpp. Let me search within the file:
<function_calls>

grep -n "class StepInstance|struct StepInstance" src/Craned/Supervisor/TaskManager.cpp

</function_calls>

No class definition found. It must be in the header. Let me check:
<function_calls>

grep -n "class|struct" src/Craned/Supervisor/TaskManager.h | head -30

</function_calls>

Let me look at the header file more carefully:
<function_calls>

cat src/Craned/Supervisor/TaskManager.h

</function_calls>

So the StepInstance is accessed through GetStep().x11_meta() - it's a proto message! Let me verify:
<function_calls>

python3 << 'EOF'

m_parent_step_inst_->GetStep() returns a reference/const reference to a Step proto message

Step proto has:

- optional X11Meta x11_meta = 15;

So m_parent_step_inst_->x11_meta is accessing a proto field

In proto3 with optional, accessing without has_() check could cause issues

The code does:

auto x11_meta = m_parent_step_inst_->x11_meta.value();

This suggests it's wrapped in std::optional or similar

Actually, looking again at line 699:

auto x11_meta = m_parent_step_inst_->x11_meta.value();

This is accessing x11_meta via .value() which is std::optional syntax

But at line 451:

auto const& x11_meta = ia_meta.x11_meta(); // This is proto getter

And at line 458:

m_parent_step_inst_->x11_meta->x11_port // This is accessing raw pointer/reference

So these are TWO DIFFERENT things:

1. ia_meta.x11_meta() - proto message getter

2. m_parent_step_inst_->x11_meta - some member of StepInstance

print("Need to find what x11_meta is on StepInstance")
EOF


</function_calls>

Let me search for where StepInstance might define x11_meta:
<function_calls>

grep -n "GetStep|ITaskInstance|x11_meta" src/Craned/Supervisor/TaskManager.h

</function_calls>

The header doesn't have x11_meta. Let me check if it's defined elsewhere or if GetStep() is the accessor. Let me look at GetStep:
<function_calls>

rg -n "GetStep()" src/Craned/Supervisor/TaskManager.cpp -B 2 -A 5

</function_calls>

So GetStep() returns a const Step& from the proto message m_step_. Let me check if m_parent_step_inst_->x11_meta is actually getting the proto field. Wait, let me re-read the code:
<function_calls>

sed -n '447,465p' src/Craned/Supervisor/TaskManager.cpp | cat -n

</function_calls>

So at line 6 (454 in file):

m_parent_step_inst_->x11_meta->x11_port

This is accessing x11_meta as a pointer member of m_parent_step_inst_. Let me find what type m_parent_step_inst_ is:
<function_calls>

rg -n "m_parent_step_inst_" src/Craned/Supervisor/TaskManager.cpp | head -10

</function_calls>

Let me look at the ProcInstance class definition:
<function_calls>

grep -n "class ProcInstance" src/Craned/Supervisor/TaskManager.cpp -A 50 | head -60

</function_calls>

I don't see a full class definition in the output. Let me try a different approach - check where ProcInstance is declared:
<function_calls>

grep -n "ProcInstance" src/Craned/Supervisor/TaskManager.h

</function_calls>

Let me check the includes to find where ProcInstance is defined:
<function_calls>

head -50 src/Craned/Supervisor/TaskManager.cpp

</function_calls>

Let me search for ProcInstance definition in the codebase:
<function_calls>

find src -name ".h" -o -name ".cpp" | xargs grep -l "class ProcInstance" | head -5

</function_calls>

It's not found elsewhere, so it must be defined in TaskManager.cpp. Let me search more carefully:
<function_calls>

grep -n "class Proc" src/Craned/Supervisor/TaskManager.cpp | head -20

</function_calls>

Let me look for where ProcInstance is constructed:
<function_calls>

rg -n "new ProcInstance|ProcInstance::" src/Craned/Supervisor/TaskManager.cpp | head -20

</function_calls>

Let me look at line 447 more carefully to understand the class:
<function_calls>

sed -n '430,450p' src/Craned/Supervisor/TaskManager.cpp

</function_calls>

So ProcInstance::InitEnvMap() is at line 447. Let me find the ProcInstance class:
<function_calls>

sed -n '1,450p' src/Craned/Supervisor/TaskManager.cpp | grep -n "class|struct" | tail -20

</function_calls>

Let me try finding where InitEnvMap is declared:
<function_calls>

grep -n "InitEnvMap" src/Craned/Supervisor/ -r --type cpp --type h

</function_calls>

It's only in TaskManager.cpp. Let me look for the class more carefully by searching backwards from line 447:
<function_calls>

sed -n '1,447p' src/Craned/Supervisor/TaskManager.cpp | tail -100 | head -50

</function_calls>

Let me search for a pattern that would help:
<function_calls>

rg -n "void.InitEnvMap|class.{" src/Craned/Supervisor/TaskManager.cpp | head -30

</function_calls>

Let me find ProcInstance class by looking for its member functions:
<function_calls>

rg -n "ProcInstance::" src/Craned/Supervisor/TaskManager.cpp | head -10

</function_calls>

So at line 447 is ProcInstance::InitEnvMap(). Let me look earlier in the file for the class declaration:
<function_calls>

head -300 src/Craned/Supervisor/TaskManager.cpp | tail -100

</function_calls>

Let me try to find all classes in the file:
<function_calls>

grep -n "^class|^struct" src/Craned/Supervisor/TaskManager.cpp

</function_calls>

There are no class definitions at the file root level. They must be inside namespace or somewhere else. Let me check:
<function_calls>

head -100 src/Craned/Supervisor/TaskManager.cpp

</function_calls>

So there's a namespace. Let me look for the class within the namespace:
<function_calls>

sed -n '75,450p' src/Craned/Supervisor/TaskManager.cpp | grep -n "class|struct" | head -20

</function_calls>

No match. This means ProcInstance is likely declared in a header file. Let me check the TaskManager.h file more carefully:
<function_calls>

grep -n "class|struct" src/Craned/Supervisor/TaskManager.h

</function_calls>

Still no match. Let me look for ITaskInstance:
<function_calls>

find src -name "*.h" | xargs grep -l "class ITaskInstance"

</function_calls>

Let me check that file:
<function_calls>

grep -n "class|struct" src/Craned/Supervisor/InstanceTraits.h

</function_calls>

Let me search for ProcInstance in all header files:
<function_calls>

find src -name "*.h" | xargs grep "class ProcInstance|struct ProcInstance" 2>/dev/null

</function_calls>

ProcInstance is not declared in any header! This means it must be an anonymous or inline class. Let me look at TaskManager.cpp more carefully around the ProcInstance methods:
<function_calls>

sed -n '350,450p' src/Craned/Supervisor/TaskManager.cpp

</function_calls>

Let me check if ProcInstance might be defined within ITaskInstance:
<function_calls>

cat src/Craned/Supervisor/InstanceTraits.h

</function_calls>

So ITaskInstance and ProcInstance are both in InstanceTraits.h! Let me check now:
<function_calls>

grep -n "class ProcInstance|x11_meta" src/Craned/Supervisor/InstanceTraits.h -B 2 -A 5

</function_calls>

So I can see x11_meta is in the file. Let me look at its declaration:
<function_calls>

sed -n '1,200p' src/Craned/Supervisor/InstanceTraits.h

</function_calls>

Perfect! Now I can see at line ~132:

std::optional<X11Meta> x11_meta;

So x11_meta is a std::optional<X11Meta> member of the base class ITaskInstance. This is what m_parent_step_inst_ (which is a StepInstance, derived from ITaskInstance) has.

Now let me verify the current state at the lines in question:
<function_calls>

sed -n '447,465p' src/Craned/Supervisor/TaskManager.cpp

</function_calls>

I can see:

  • Line 458: m_parent_step_inst_->x11_meta->x11_port - Directly dereferencing without checking if optional has value

Let me check the second location:
<function_calls>

sed -n '691,710p' src/Craned/Supervisor/TaskManager.cpp

</function_calls>

Line 699 shows: auto x11_meta = m_parent_step_inst_->x11_meta.value();

This is using .value() which will throw if the optional is empty.

So the review comment is correct - the code accesses x11_meta without proper guards. Let me verify if there's been any recent fix by checking if there are assert/checks around these locations that I missed:
<function_calls>

sed -n '440,475p' src/Craned/Supervisor/TaskManager.cpp

</function_calls>

No asserts before the access. Let me check the second location:
<function_calls>

sed -n '680,715p' src/Craned/Supervisor/TaskManager.cpp

</function_calls>

No asserts here either. Let me now verify the claim about Prepare() error handling. Let me search for where Prepare is called and if its return value is checked:
<function_calls>

grep -n "->Prepare()" src/Craned/Supervisor/TaskManager.cpp -B 3 -A 3

</function_calls>

So Prepare() is called at line 608. Let me see the context:
<function_calls>

sed -n '600,630p' src/Craned/Supervisor/TaskManager.cpp

</function_calls>

I see - the result of Prepare() is not checked! The code continues regardless of whether Prepare() succeeds or fails. This is the issue. If Prepare() fails, x11_meta might not be initialized, but the code at lines 447-460 and 691-705 will still try to access it.

Now let me verify the learnings note - does it apply here? The learning says GetCrunMeta() calls don't need null checks because they're only called when the task is guaranteed to be CRUN. Let me check if the same applies here:
<function_calls>

sed -n '440,450p' src/Craned/Supervisor/TaskManager.cpp

</function_calls>

So the code checks if (m_parent_step_inst_->IsCrun()) and if (ia_meta.x11()), which means these methods are only called for CRUN steps with x11 enabled. However, this doesn't guarantee that x11_meta is populated if Prepare() fails.

Let me verify my understanding by checking what x11_meta needs to be populated:
<function_calls>

grep -n "x11_meta\s*=" src/Craned/Supervisor/ -r --type cpp | head -20

</function_calls>

So x11_meta is being assigned in multiple places. Let me find where it's populated during task execution or Prepare:
<function_calls>

rg -n ".x11_meta\s*=|x11_meta\s*{" src/Craned/Supervisor/ --type cpp -B 2 -A 2 | head -60

</function_calls>

Let me check the Execute method to see where x11_meta might be set:
<function_calls>

sed -n '550,650p' src/Craned/Supervisor/TaskManager.cpp | grep -n "x11_meta"

</function_calls>

x11_meta is not set in Execute. Let me check Prepare more carefully:
<function_calls>

python3 << 'EOF'
with open('src/Craned/Supervisor/TaskManager.cpp', 'r') as f:
lines = f.readlines()

Find Prepare method and look for x11_meta assignments

for i, line in enumerate(lines):
if 'bool StepInstance::Prepare()' in line:
# Print until we find the next major method
for j in range(i, min(i+250, len(lines))):
if 'x11_meta' in lines[j]:
# Print surrounding context
for k in range(max(i, j-2), min(len(lines), j+3)):
prefix = ">>> " if k == j else " "
print(f"{prefix}{k+1}: {lines[k]}", end='')
EOF


</function_calls>

x11_meta is not being set in Prepare(). So where does it get set? Let me search throughout the file:
<function_calls>

rg -n ".x11_meta\s*=" src/Craned/Supervisor/TaskManager.cpp

</function_calls>

No assignments found in TaskManager.cpp! Let me search more broadly:
<function_calls>

rg -n "x11_meta\s*=" src/ --type cpp | head -30

</function_calls>

So x11_meta is set in:

  • CraneAgent.cpp
  • ExecutionCommon.cpp

Let me check ExecutionCommon.cpp:
<function_calls>

rg -n "x11_meta\s*=" src/Craned/Supervisor/ExecutionCommon.cpp -B 5 -A 5

</function_calls>

So x11_meta is being set in CreateTaskExecutionContext. This happens before Prepare() is called. Let me verify this by checking where CreateTaskExecutionContext is called:
<function_calls>

rg -n "CreateTaskExecutionContext" src/Craned/Supervisor/TaskManager.cpp -B 3 -A 3

</function_calls>

So CreateTaskExecutionContext is called at line 590 in Execute(), which is before Prepare() at line 608. This means x11_meta should already be set.

However, the review comment claims that if Prepare() fails in the X11 setup path, x11_meta might not be set. Let me check if Prepare() itself modifies x11_meta:
<function_calls>

sed -n '1049,1200p' src/Craned/Supervisor/TaskManager.cpp | grep -i "x11_meta|mutable"

</function_calls>

Prepare() doesn't seem to set x11_meta. So the issue must be different. Let me re-read the review comment more carefully...

The review mentions "CreateFoldersForFileEx/mkstemp/fchown" failures in the X11 setup path. Let me search for these:
<function_calls>

rg -n "mkstemp|fchown|CreateFoldersForFileEx" src/Craned/Supervisor/ --type cpp

</function_calls>

These functions are not found in TaskManager.cpp. Let me search more broadly:
<function_calls>

rg -rn "mkstemp|fchown|CreateFoldersForFileEx" src/ --type cpp | head -20

</function_calls>

These are in ExecutionCommon.cpp. Let me check that file:
<function_calls>

rg -n "mkstemp|fchown|CreateFoldersForFileEx|CreateTaskExecutionContext" src/Craned/Supervisor/ExecutionCommon.cpp -B 5 -A 5 | head -100

</function_calls>

So CreateTaskExecutionContext is where x11_meta is setup. If this function returns an error, x11_meta might not be fully initialized. The review comment is about the fact that if CreateTaskExecutionContext fails, the code still continues and tries to use x11_meta later.

Let me verify: does Execute() check the return value of CreateTaskExecutionContext?
<function_calls>

sed -n '580,620p' src/Craned/Supervisor/TaskManager.cpp

</function_calls>

I don't see CreateTaskExecutionContext being called in this snippet. Let me search for where it's called:
<function_calls>

grep -n "CreateTaskExecutionContext" src/Craned/Supervisor/ -r --type cpp

</function_calls>

It's called in CraneAgent.cpp, not in TaskManager.cpp. Let me check that file:
<function_calls>

rg -n "CreateTaskExecutionContext" src/Craned/CraneAgent.cpp -B 10 -A 10

</function_calls>

Perfect! So the issue is in CraneAgent.cpp at line 1090. If CreateTaskExecutionContext returns an error, that error is not checked, and the code continues. Then when TaskManager::Execute is called, it tries to access x11_meta which might not have been set.

However, the review comment is specifically about lines in TaskManager.cpp, not about error handling in CraneAgent.cpp. The defensive programming suggestion is to add checks in TaskManager.cpp to ensure x11_meta is set before using it.

Let me now summarize my findings and write the rewritten review comment:

Based on my verification, the review comment accurately identifies real issues in the code. The code accesses m_parent_step_inst_->x11_meta (a std::optional field) without proper guards in both locations. Let me output the final rewritten review comment:


Guard X11 metadata access with has_value() checks in X11-enabled code paths.

At lines 447-460 and 691-705, the code assumes m_parent_step_inst_->x11_meta is populated whenever ia_meta.x11() is true, but does not verify. Direct dereference at line 458 and .value() call at line 699 will cause undefined behavior or exception if the optional is unengaged:

// Line 454-460
if (ia_meta.x11()) {
  auto const& x11_meta = ia_meta.x11_meta();
  std::string target =
      ia_meta.x11_meta().enable_forwarding() ? "" : x11_meta.target();
  m_env_["DISPLAY"] = fmt::format(
      "{}:{}", target, m_parent_step_inst_->x11_meta->x11_port - 6000);  // ← unsafe dereference
  m_env_["XAUTHORITY"] = m_parent_step_inst_->x11_meta->x11_auth_path;    // ← unsafe dereference
}
// Line 699
auto x11_meta = m_parent_step_inst_->x11_meta.value();  // ← throws if empty

If X11 setup in CreateTaskExecutionContext (ExecutionCommon.cpp) fails before Prepare() is called, or if Prepare() itself fails partway through, x11_meta may remain unset. Add defensive checks:

if (ia_meta.x11()) {
  CRANE_ASSERT(m_parent_step_inst_->x11_meta.has_value());
  // ... rest of code
}

This makes the contract explicit and protects against future refactors or error path changes.

Comment on lines 588 to +593
for (const auto &it :
std::filesystem::directory_iterator(root_cgroup_path)) {
std::filesystem::recursive_directory_iterator(root_cgroup_path)) {
if (it.is_directory()) {
auto parsed_ids = ParseIdsFromCgroupStr_(it.path().filename());
auto job_id_opt = std::get<0>(parsed_ids);
if (job_id_opt.has_value()) job_ids.emplace(job_id_opt.value());
auto job_id_opt = std::get<CgConstant::KParsedJobIdIdx>(parsed_ids);
if (job_id_opt.has_value()) ids.emplace(parsed_ids);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

⚠️ Potential issue | 🟠 Major

GetIdsFromCgroupV2_ parses only the final path component, missing nested step/task cgroups.

At line 591, ParseIdsFromCgroupStr_ is called with it.path().filename() which only contains the final directory name (e.g., "step_0" or "system"). For nested paths like job_123/step_0/system, the regex pattern expects the full relative path to extract all IDs correctly.

This means step-level and task-level cgroups won't be properly identified during recovery on cgroup v2.

  for (const auto &it :
       std::filesystem::recursive_directory_iterator(root_cgroup_path)) {
    if (it.is_directory()) {
-     auto parsed_ids = ParseIdsFromCgroupStr_(it.path().filename());
+     // Use relative path from root to match the full cgroup hierarchy
+     auto rel_path = std::filesystem::relative(it.path(), root_cgroup_path);
+     auto parsed_ids = ParseIdsFromCgroupStr_(rel_path.string());
      auto job_id_opt = std::get<CgConstant::KParsedJobIdIdx>(parsed_ids);
      if (job_id_opt.has_value()) ids.emplace(parsed_ids);
    }
  }
🤖 Prompt for AI Agents
In src/Craned/Common/CgroupManager.cpp around lines 588-593, the loop currently
calls ParseIdsFromCgroupStr_ with it.path().filename(), which only passes the
final path component and misses nested job/step/task segments; change the call
to pass the full path relative to root_cgroup_path (i.e., the subpath under
root_cgroup_path, not just filename) so the regex can match nested components
like job/step/task, and ensure the string uses the expected separator/format the
parser expects before invoking ParseIdsFromCgroupStr_.

Comment on lines 962 to 974
g_thread_pool->detach_task([&shutdown_step_latch, step] {
auto stub = g_supervisor_keeper->GetStub(step->job_id, step->step_id);
if (!stub) {
CRANE_ERROR("[Step #{}.{}] Failed to get stub.", step->job_id,
step->step_id);
} else {
CRANE_TRACE("[Step #{}.{}] Shutting down supervisor.", step->job_id,
step->step_id);
auto err = stub->ShutdownSupervisor();
if (err != CraneErrCode::SUCCESS) {
CRANE_ERROR("[Step #{}.{}] Failed to shutdown supervisor.",
step->job_id, step->step_id);
}
auto stub = step->supervisor_stub;
CRANE_TRACE("[Step #{}.{}] Shutting down supervisor.", step->job_id,
step->step_id);
auto err = stub->ShutdownSupervisor();
if (err != CraneErrCode::SUCCESS) {
CRANE_ERROR("[Step #{}.{}] Failed to shutdown supervisor.",
step->job_id, step->step_id);
g_job_mgr->StepStatusChangeAsync(
step->job_id, step->step_id, crane::grpc::TaskStatus::Failed,
ExitCode::EC_RPC_ERR, "Failed to shutdown supervisor.",
google::protobuf::util::TimeUtil::GetCurrentTime());
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

⚠️ Potential issue | 🟠 Major

Missing null check for supervisor_stub in CleanUpJobAndStepsAsync.

At line 963, step->supervisor_stub is assigned to stub and immediately used at line 966 without a null check. If the supervisor failed to start, this will crash.

  g_thread_pool->detach_task([&shutdown_step_latch, step] {
    auto stub = step->supervisor_stub;
+   if (!stub) {
+     CRANE_ERROR("[Step #{}.{}] Supervisor stub is null when shutting down.",
+                 step->job_id, step->step_id);
+     shutdown_step_latch.count_down();
+     return;
+   }
    CRANE_TRACE("[Step #{}.{}] Shutting down supervisor.", step->job_id,
                step->step_id);
    auto err = stub->ShutdownSupervisor();
🤖 Prompt for AI Agents
In src/Craned/Core/JobManager.cpp around lines 962 to 974, the lambda uses
step->supervisor_stub without checking for null which can crash if the
supervisor never started; add a nullptr check right after assigning stub and
before calling ShutdownSupervisor(): if stub is null, log a warning/error
including job and step ids, call g_job_mgr->StepStatusChangeAsync to mark the
step as Failed (with appropriate exit code/message and timestamp), ensure the
shutdown_step_latch is counted down or signaled as needed, and return from the
lambda to avoid dereferencing a null stub; if stub is non-null proceed to call
ShutdownSupervisor() and keep the existing error handling.

@L-Xiafeng
Copy link
Collaborator Author

moved to #727

@L-Xiafeng L-Xiafeng closed this Dec 2, 2025
@L-Xiafeng L-Xiafeng deleted the dev/task branch December 17, 2025 09:19
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

enhancement New feature or request

Projects

None yet

Development

Successfully merging this pull request may close these issues.

对Job/step/task支持Cgroup资源限制 crun 多task任务支持

2 participants