Skip to content
Open
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
321 changes: 321 additions & 0 deletions docs/design/sandbox-pause-resume.md
Original file line number Diff line number Diff line change
@@ -0,0 +1,321 @@
---
title: Sandbox Pod Idle Pause Lifecycle
authors:
- @hzxuzhonghu
reviewers:
- "@volcano-sh/agentcube-approvers"
- TBD
approvers:
- "@volcano-sh/agentcube-approvers"
- TBD
creation-date: 2025-11-21

---

## Sandbox Pod Idle Pause Lifecycle

### Summary

This proposal introduces idle-aware lifecycle management for AgentCube Sandboxes. Instead of deleting the Sandbox CRD and its bound pod when the `last-activity-time` annotation expires, the controller will pause the sandbox pod’s containers via the containerd API and resume them on demand. The change reduces cluster resource consumption, preserves user state, and shortens time-to-first-byte when developers return to their sandboxes. The work covers the pause logic, resume triggers (annotation-based and message-queue driven), observability, and testing strategy.

### Motivation

#### Goals

- Automatically pause sandbox pods whose `last-activity-time` annotation exceeds a configurable idle timeout.
- Resume paused sandbox pods when user activity resumes via annotation or message-queue signals.
- Ensure the sandbox lifecycle remains transparent to users and external systems (status reporting, API responses).
- Provide metrics, events, and audit logs for pause/resume operations.

#### Non-Goals

- Changing the existing Sandbox CRD schema beyond new annotations or status fields required for pause/resume bookkeeping.
- Supporting alternative container runtimes beyond containerd in this iteration.
- Implementing persistent volume snapshotting or memory checkpoint/restore.

### Proposal

#### User Stories (Optional)

##### Story 1

As a developer using AgentCube, when my sandbox is idle overnight it is paused automatically, freeing cluster resources, but when I return the environment resumes within seconds and my previous processes and files are intact.

##### Story 2

As an administrator, I can monitor how many sandboxes are paused, trigger resume operations programmatically, and audit pause/resume events for compliance.

#### Notes/Constraints/Caveats (Optional)

- Pods with multiple containers must pause/resume all workload containers; init and ephemeral containers remain unchanged.
- Containerd pause/resume capability must be enabled on worker nodes; sandbox pods scheduled to runtimes lacking pause support will fall back to deletion or remain running (configurable fallback).
- Pausing terminates network sockets; clients must handle reconnect semantics after resume.
- Sandboxes relying on in-memory timers or background jobs may experience delayed execution until resume.

#### Risks and Mitigations

- **Unsupported container runtime**: Detect runtime class during reconciliation and surface clear events; provide opt-out annotation to disable pausing.
- **Resume signal loss**: Store resume intent in etcd-backed annotations or durable message queues; include retry/backoff logic.
- **Controller restarts mid-operation**: Persist pause/resume state in Sandbox status to ensure idempotent reconciliation.
- **User experience regressions**: Gate rollout behind feature flag and collect metrics before enabling cluster-wide.

### Design Details

#### Pausing and Resuming via containerd

This section describes how the controller talks to containerd to pause and resume the sandbox pod’s containers. We assume Kubernetes is configured with containerd as the CRI implementation (`containerd-shim-runc-v2` or equivalent) and that pause/resume is enabled on the worker nodes.

##### High-level flow

- For each `Sandbox` CRD, there is a bound pod (the sandbox pod) created by the apiserver.
Copy link

Copilot AI Nov 21, 2025

Choose a reason for hiding this comment

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

The reference to "apiserver" on line 70 is inconsistent with the actual component that creates pods. Based on the codebase (pkg/apiserver/k8s_client.go), the apiserver creates Sandbox CRDs, but the Kubernetes controller (likely via the agent-sandbox operator) creates the actual pods from the Sandbox spec. Consider clarifying this to say "created by the Kubernetes controller" or "created from the Sandbox spec".

Suggested change
- For each `Sandbox` CRD, there is a bound pod (the sandbox pod) created by the apiserver.
- For each `Sandbox` CRD, there is a bound pod (the sandbox pod) created by the Kubernetes controller from the Sandbox spec.

Copilot uses AI. Check for mistakes.
- The controller (`AgentdReconciler`) periodically reconciles the `Sandbox` and checks the idle timeout based on `last-activity-time`.
- When the sandbox is idle, the controller pauses all containers in the sandbox pod using containerd and marks the sandbox as `Paused`.
- When a resume signal is detected (annotation or message queue), the controller resumes the containers using containerd and marks the sandbox as `Running`.

##### Locating the pod and containers

1. Given a `Sandbox` object, the controller finds the bound pod by label or name (existing binding logic in apiserver).
2. For the pod, the controller lists all **workload containers**:
- `spec.containers[*]` are candidates for pause/resume.
- Init containers and ephemeral containers are **not** paused or resumed; they have already completed or are temporary.
3. For each workload container, the controller resolves the container runtime ID from the pod status (`status.containerStatuses[*].containerID`). This ID is of the form `containerd://<container-id>`.

##### Using containerd to pause

The controller runs on the node or has access to containerd via its Unix socket (e.g. `/run/containerd/containerd.sock`). We rely on the official Go client for containerd.

1. Establish a containerd client:
- `client, err := containerd.New("/run/containerd/containerd.sock")`.
- Use a short timeout context (e.g. 5–10 seconds) to avoid blocking reconciles.
Copy link

Copilot AI Nov 21, 2025

Choose a reason for hiding this comment

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

The timeout value of "5–10 seconds" for containerd operations may be insufficient for pause/resume operations under heavy node load. When a node is under memory pressure or high I/O load, containerd operations can take longer than 10 seconds. Additionally, if pausing/resuming multiple containers in sequence (as described in line 90 "For each workload container"), the total time could exceed this timeout.

Consider:

  1. Using a per-operation timeout (e.g., 30 seconds per container) rather than a global timeout for all containers
  2. Making the timeout configurable
  3. Documenting the expected time for pause/resume operations under normal and stressed conditions
  4. Implementing proper cancellation handling if a context timeout occurs mid-operation

Without adequate timeouts, the reconciliation loop could block, preventing other sandboxes from being processed.

Suggested change
- Use a short timeout context (e.g. 5–10 seconds) to avoid blocking reconciles.
- For each container operation, use a per-operation timeout context (e.g., 30 seconds per container). The timeout should be configurable to accommodate different node loads and operational environments.
- Document the expected time for pause/resume operations under both normal and stressed conditions, and implement proper cancellation handling if a context timeout occurs mid-operation.

Copilot uses AI. Check for mistakes.
2. For each workload container:
- Strip the `containerd://` prefix from `containerID`.
- Look up the corresponding task with `client.LoadTask(ctx, containerID)`.
Copy link

Copilot AI Nov 21, 2025

Choose a reason for hiding this comment

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

The API call client.LoadTask(ctx, containerID) is incorrect. The containerd Go client API requires loading a container first and then getting its task. The correct pattern is: container, err := client.LoadContainer(ctx, containerID) followed by task, err := container.Task(ctx, nil). The LoadTask method does not exist on the containerd client in this way.

Suggested change
- Look up the corresponding task with `client.LoadTask(ctx, containerID)`.
- Look up the corresponding task:
- `container, err := client.LoadContainer(ctx, containerID)`
- `task, err := container.Task(ctx, nil)`

Copilot uses AI. Check for mistakes.
- If the task state is already `Paused`, skip.
- Call `task.Pause(ctx)`.
3. If any container fails to pause:
- Record a `Paused` condition with `status=False` and a reason (e.g. `ContainerdPauseError`).
- Emit a Kubernetes event on the `Sandbox` and the pod.
- Optionally retry with backoff; give up after a small number of attempts to keep reconciliation bounded.
Comment on lines +95 to +98
Copy link

Copilot AI Nov 21, 2025

Choose a reason for hiding this comment

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

The error handling description states that the controller should "give up after a small number of attempts to keep reconciliation bounded," but this conflicts with the idempotency design mentioned in lines 125-133. If the reconciler is supposed to be idempotent and converge to desired state, giving up on pause operations could leave the sandbox in an inconsistent state. Consider clarifying whether failed pause operations should transition to an error state and wait for manual intervention, or if they should continue retrying on subsequent reconciliation loops.

Copilot uses AI. Check for mistakes.
4. Once all containers are paused:
- Set `sandbox.lifecycle.volcano.sh/state=Paused`.
- Set `sandbox.lifecycle.volcano.sh/last-paused-at=<now RFC3339>`.
- Expose the paused state via apiserver and SDKs.

##### Using containerd to resume

Resuming is symmetrical to pausing.

1. Establish the same containerd client as above.
2. For each workload container in the sandbox pod:
- Resolve and strip the containerd container ID.
- Load the container task.
- If the task state is `Running`, skip.
- Call `task.Resume(ctx)`.
3. Handle errors similarly to pause:
- Record a `ResumePending` or `Paused` condition with appropriate reason.
- Emit events and logs.
4. After successful resume of all containers:
- Set `sandbox.lifecycle.volcano.sh/state=Running`.
- Set `sandbox.lifecycle.volcano.sh/last-resumed-at=<now>`.
- Refresh `last-activity-time` to `now` to avoid immediate re-pause.
Copy link

Copilot AI Nov 21, 2025

Choose a reason for hiding this comment

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

The proposal mentions "Refresh last-activity-time to now to avoid immediate re-pause" but doesn't discuss the race condition where:

  1. A sandbox is paused due to idle timeout
  2. User resumes it via annotation
  3. Controller resumes the sandbox and refreshes last-activity-time
  4. But no actual user activity occurs

This results in the sandbox staying active for another full timeout period even though the user might have only briefly checked it. Consider whether the resume operation should:

  1. Set a shorter grace period after resume before checking for idle timeout again
  2. Require actual user activity (not just resume request) to reset the timeout
  3. Use a different annotation like last-resume-time separate from last-activity-time

The current design could lead to resource waste if users frequently trigger resumes without actual activity.

Suggested change
- Refresh `last-activity-time` to `now` to avoid immediate re-pause.
- **Do not refresh `last-activity-time` on resume.** Only update `last-activity-time` when actual user activity is detected within the sandbox. This avoids extending the idle timeout period unnecessarily if the user resumes the sandbox but does not interact with it.

Copilot uses AI. Check for mistakes.
- Clear any outstanding resume triggers (annotations, MQ markers).

##### Idempotency and failure handling

- The pause and resume operations must be idempotent:
- Calling pause on an already paused container must be a no-op.
- Calling resume on a running container must be a no-op.
- The controller reconciler drives desired state from annotations and status, not from one-shot RPC calls.
- If the controller restarts in the middle of pause or resume, the next reconcile evaluates:
- The current `state` annotation (`Running|Paused|Resuming|Error`).
- The actual containerd task states.
- Any outstanding resume triggers.
- The controller then converges to the desired state (e.g. `Paused` or `Running`) without double-pausing or double-resuming.

Copy link

Copilot AI Nov 21, 2025

Choose a reason for hiding this comment

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

The design doesn't address potential race conditions when multiple reconciliation loops or resume triggers occur simultaneously. For example, if a pause operation is in progress and a resume annotation is added, or if two resume signals (annotation and MQ) arrive concurrently. Consider documenting concurrency control mechanisms such as using optimistic locking (resourceVersion checks), adding an operation-in-progress indicator, or ensuring the reconciler uses proper locking/queuing to serialize operations on the same sandbox.

Suggested change
##### Concurrency control and race condition prevention
- To prevent race conditions when multiple reconciliation loops or resume triggers occur simultaneously (e.g., overlapping pause and resume requests, or concurrent triggers from annotations and MQ), the controller implements the following concurrency control mechanisms:
- **Optimistic locking:** All updates to the sandbox resource use Kubernetes `resourceVersion` checks to ensure that only the latest version is modified. If a concurrent update occurs, the reconcile is retried with the new state.
- **Operation-in-progress indicator:** The controller sets an explicit status field or annotation (e.g., `sandbox.lifecycle.volcano.sh/operation-in-progress`) to indicate when a pause or resume operation is underway. New operations are not started until the current one completes, ensuring serialization of state transitions.
- **Serialized reconciliation:** The reconciler ensures that only one operation (pause or resume) is performed at a time for a given sandbox, either by relying on the controller-runtime's per-resource queueing or by explicit locking if needed.
- These mechanisms ensure that the controller converges to the correct desired state without double-pausing, double-resuming, or inconsistent state transitions, even under high event rates or controller restarts.

Copilot uses AI. Check for mistakes.
##### Interaction with Kubernetes primitives

- We keep the pod **Running** from Kubernetes’ perspective; we do not delete the pod or set its phase to `Succeeded` or `Failed`.
- Liveness and readiness probes may fail while the container is paused; operators can:
- Disable liveness probes for sandbox workloads, or
- Use readiness-only probes and accept that a paused sandbox is `NotReady`.
- Network connections are dropped when a container is paused; clients are expected to reconnect after resume.
Copy link

Copilot AI Nov 21, 2025

Choose a reason for hiding this comment

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

The proposal states that pausing "terminates network sockets" but doesn't address the implications for persistent connections or services that depend on the sandbox. Specifically:

  1. What happens to Services/Endpoints pointing to the paused pod? They may continue routing traffic to it, causing connection failures
  2. Should the pod be removed from Service endpoints when paused? If so, how does it get re-added on resume?
  3. What about WebSocket connections, long-polling, or other persistent connection patterns?
  4. How do clients discover that a sandbox was paused vs experiencing a network failure?

Consider adding guidance on updating Service/Endpoint configurations during pause/resume, or documenting that paused sandboxes should not be exposed via Services.

Suggested change
- Network connections are dropped when a container is paused; clients are expected to reconnect after resume.
- **Service/Endpoint management:** When a sandbox pod is paused, it should be marked as `NotReady` via readiness probes. This ensures that Kubernetes removes the pod from Service endpoints, preventing new traffic from being routed to the paused pod and avoiding connection failures for new requests. Upon resuming, the pod should become `Ready` again and be re-added to Service endpoints automatically.
- **Persistent connections:** All network connections, including persistent connections such as WebSockets and long-polling, are dropped when a container is paused. Clients must be prepared to handle connection loss and reconnect after the pod resumes.
- **Client experience:** Clients cannot distinguish between a paused sandbox and a generic network failure unless additional signaling is implemented at the application layer. Operators should document this behavior for users and consider implementing custom signaling if needed.
- **Guidance:** It is recommended that paused sandboxes are not exposed via Services. Ensure readiness probes are configured so that paused pods are marked NotReady, and avoid exposing paused sandboxes to external traffic.

Copilot uses AI. Check for mistakes.

Copy link

Copilot AI Nov 21, 2025

Choose a reason for hiding this comment

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

The proposal doesn't address what happens to resource requests and limits while a container is paused. Kubernetes scheduler decisions are based on resource requests, but a paused container consumes significantly less CPU (near zero) while still holding memory. This could lead to:

  1. Inefficient bin-packing - nodes may appear full but have substantial unused CPU
  2. Potential issues with cluster autoscaler decisions
  3. Misleading resource utilization metrics

The proposal should discuss whether resource requests should be adjusted for paused containers, or if this is an accepted limitation that should be documented for operators.

Suggested change
##### Resource Requests and Limits While Paused
- When a container is paused, it consumes significantly less CPU (near zero), but continues to hold its requested CPU and memory resources from the perspective of the Kubernetes scheduler.
- This can lead to:
- **Inefficient bin-packing:** Nodes may appear full due to reserved CPU, even though paused containers are not actively using it.
- **Potential issues with cluster autoscaler:** The autoscaler may not scale down nodes with paused containers, as their resource requests are still considered in use.
- **Misleading resource utilization metrics:** Metrics may show high resource reservation but low actual usage.
- At present, Kubernetes does not support dynamically adjusting resource requests for paused containers. This is an accepted limitation of the current design.
- **Operator guidance:** Operators should be aware of this behavior when configuring scheduling and autoscaling policies. Monitoring actual resource usage versus requested resources is recommended to avoid confusion.

Copilot uses AI. Check for mistakes.
##### Security and configuration

- The controller needs sufficient permissions to talk to the containerd socket. There are two deployment options:
- Run the controller as a DaemonSet on each node with hostPath mount of `/run/containerd/containerd.sock`.
Copy link

Copilot AI Nov 21, 2025

Choose a reason for hiding this comment

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

The implementation mentions "Run the controller as a DaemonSet on each node with hostPath mount" (line 146), but the existing AgentdReconciler appears to be a cluster-scoped controller, not a DaemonSet. This architectural change has significant implications for deployment, RBAC, and high availability that aren't fully addressed. Consider adding a dedicated section on deployment architecture changes and migration strategy, or clarify if this refers to a separate component from the existing AgentdReconciler.

Copilot uses AI. Check for mistakes.
- Or, delegate pause/resume to a lightweight node-side agent that exposes a narrow gRPC API to the cluster-wide controller.
- This proposal focuses on the direct containerd socket approach for simplicity; a follow-up can introduce the node agent if needed.

#### Implementation Plan

1. Extend `AgentdReconciler` to replace delete-on-expire with a `PauseSandbox(sandbox)` workflow built on containerd APIs.
2. Introduce a `SandboxLifecycle` helper that:
- Verifies pod binding and locates container IDs from pod status.
- Uses the containerd Go client to pause/resume containers.
- Updates annotations/status fields (`sandbox.lifecycle.volcano.sh/state`, timestamps, error messages).
3. Implement resume triggers (see next section):
- **Annotation path**: Users (or apiserver) set `sandbox.lifecycle.volcano.sh/resume-requested-at`; reconciler resumes and clears annotation.
- **Message queue path**: Integrate with a concrete MQ to emit durable resume requests into the controller queue.
4. Add feature flags and configuration knobs: idle timeout, pause enablement, containerd socket path, per-namespace opt-out, fallback behavior (delete vs keep running).
5. Update apiserver and SDKs to surface paused state and expose a `/v1/sandboxes/{sandboxId}/pause` API.
Copy link

Copilot AI Nov 21, 2025

Choose a reason for hiding this comment

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

The API endpoint path mentioned here (/v1/sandboxes/{sandboxId}/pause) doesn't match the resume endpoint mentioned in line 173 (/v1/sandboxes/{sandboxId}/resume). If this proposal is introducing both pause and resume operations, both endpoints should be documented consistently. Additionally, the text only mentions exposing a pause API, but the context suggests resume should also be exposed. Consider documenting both endpoints or clarifying if only one is being added.

Suggested change
5. Update apiserver and SDKs to surface paused state and expose a `/v1/sandboxes/{sandboxId}/pause` API.
5. Update apiserver and SDKs to surface paused state and expose `/v1/sandboxes/{sandboxId}/pause` and `/v1/sandboxes/{sandboxId}/resume` APIs.

Copilot uses AI. Check for mistakes.
6. Ship metrics/events/logging and document operational playbooks.
7. Add end-to-end tests that validate containerd pause/resume against a real or fake runtime.

#### API / CRD Changes

- New annotations:
- `sandbox.lifecycle.volcano.sh/state`: `Running|Paused|Resuming|Error`.
- `sandbox.lifecycle.volcano.sh/resume-requested-at`: RFC3339 timestamp.
- `sandbox.lifecycle.volcano.sh/last-paused-at`, `sandbox.lifecycle.volcano.sh/last-resumed-at` (optional audit).
Comment on lines +167 to +170
Copy link

Copilot AI Nov 21, 2025

Choose a reason for hiding this comment

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

The proposed annotation keys use domain-qualified names (e.g., sandbox.lifecycle.volcano.sh/state, sandbox.lifecycle.volcano.sh/resume-requested-at) which is inconsistent with existing annotation conventions in the codebase. The existing annotations use simple keys without domain prefixes: last-activity-time and creator-service-account (see pkg/apiserver/k8s_client.go:26,28).

For consistency, consider either:

  1. Using simple annotation keys like sandbox-state, resume-requested-at, last-paused-at, last-resumed-at
  2. Or, if domain-qualified names are preferred for these new lifecycle annotations, document why this departure from the existing pattern is necessary and consider migrating existing annotations to the same pattern

Mixing annotation styles can lead to confusion about which convention to follow for future additions.

Copilot uses AI. Check for mistakes.
Comment on lines +167 to +170
Copy link

Copilot AI Nov 21, 2025

Choose a reason for hiding this comment

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

The annotation sandbox.lifecycle.volcano.sh/state storing values like "Running|Paused|Resuming|Error" should be clarified. Based on Kubernetes conventions, runtime state should typically be stored in status fields rather than annotations. Annotations are intended for metadata and non-authoritative information. Consider:

  1. Moving the state to a proper status.phase or status.lifecycleState field
  2. If keeping as annotation, explain why this design choice was made over using status fields
  3. Document how conflicts are resolved if both annotation and status contain state information

This is especially important given that line 172 mentions status.conditions entries, suggesting status is already being used for related information.

Copilot uses AI. Check for mistakes.
- Sandbox status additions:
- `status.conditions` entries for `Paused` and `ResumePending`.
- Optional: extend apiserver REST/SDK to POST `/v1/sandboxes/{sandboxId}/resume`.
Comment on lines +161 to +173
Copy link

Copilot AI Nov 21, 2025

Choose a reason for hiding this comment

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

The proposal mentions a /v1/sandboxes/{sandboxId}/pause API (line 161) but later only describes /v1/sandboxes/{sandboxId}/resume (line 173 and 193). The document should clarify whether both pause and resume APIs are being added, or if pause is only automatic via timeout. If both are added, document the use cases for manual pause. If only resume is exposed, remove the reference to the pause API endpoint.

Copilot uses AI. Check for mistakes.

#### Resume signaling via annotations and message queue

There are two ways to request that a paused sandbox be resumed: via annotations on the `Sandbox` resource, and via messages sent to a message queue.

##### Annotation-based resume

The annotation path is the simplest and is always enabled.

- To request a resume, a client (user, SDK, or apiserver) sets:
- `sandbox.lifecycle.volcano.sh/resume-requested-at=<now RFC3339>`.
- On reconciliation:
- If the sandbox `state=Paused` and `resume-requested-at` is set, the controller transitions the state to `Resuming` and calls the containerd resume workflow.
- After successful resume, the controller:
- Clears `resume-requested-at`.
- Sets `state=Running`.
- Updates `last-resumed-at` and `last-activity-time`.
- If resume fails, the controller sets `state=Error` and records the reason.

This path is ideal for synchronous APIs (e.g. HTTP `POST /v1/sandboxes/{sandboxId}/resume`).

##### Message-queue-based resume

For asynchronous and cross-cluster resume triggers (e.g. from SaaS control planes or CI systems), we introduce a message queue consumer that translates messages into resume intents in the Kubernetes cluster.

###### Message queue choice

We propose using **plain NATS** (core NATS server with durable subscriptions) as the concrete message queue for the first implementation.

Rationale:

- **Simplicity and footprint**: NATS is a single small binary and easy to run as a Kubernetes StatefulSet. It has fewer moving parts than Kafka or RabbitMQ.
- **Cloud-native and Kubernetes-friendly**: There are mature Helm charts and operators; it integrates well with Kubernetes RBAC and multi-tenant setups.
- **At-least-once delivery via durable subscriptions**: While we are not using JetStream, NATS durable subscriptions and manual acknowledgements give us at-least-once behavior for consumers that stay connected. Combined with idempotent handling on the controller side, this is sufficient for resume signals.
Copy link

Copilot AI Nov 21, 2025

Choose a reason for hiding this comment

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

The description mentions "durable subscriptions" for plain NATS core, but this could be misleading. In NATS terminology, "durable subscriptions" typically refer to queue groups with durable names, which provide at-most-once delivery per consumer group but don't persist messages when all consumers are offline. The document later acknowledges this limitation (line 260-263), but line 207 might give a false impression of stronger durability guarantees than plain NATS core actually provides. Consider clarifying that this refers to queue groups with durable consumer names, not JetStream's persistent message storage.

Suggested change
- **At-least-once delivery via durable subscriptions**: While we are not using JetStream, NATS durable subscriptions and manual acknowledgements give us at-least-once behavior for consumers that stay connected. Combined with idempotent handling on the controller side, this is sufficient for resume signals.
- **At-least-once delivery via queue groups with durable names (not JetStream; messages are not persisted if all consumers are offline)**: While we are not using JetStream, NATS queue groups with durable names and manual acknowledgements give us at-least-once behavior for consumers that stay connected. Combined with idempotent handling on the controller side, this is sufficient for resume signals.

Copilot uses AI. Check for mistakes.
- **Low latency and high fanout**: Fits interactive developer workflows where resume should feel instantaneous.
- **Mature Go client**: AgentCube is predominantly Go; NATS has a first-class Go client, reducing integration friction.

The MQ layer is kept pluggable so that other backends (e.g. Kafka, RabbitMQ, or a different NATS deployment model) can be added later.

###### Message schema

We define a simple JSON payload for resume messages:

- Subject: `agentcube.sandbox.resume` (NATS subject)
- Payload:
- `sandboxNamespace` (string, required)
- `sandboxName` (string, required)
- `requestedBy` (string, optional, user or system ID)
- `reason` (string, optional, free-form)
- `requestedAt` (string, RFC3339, optional; default: server time)
- `traceID` (string, optional, for correlating logs and traces)

Example JSON:

```json
{
"sandboxNamespace": "user-a",
"sandboxName": "sandbox-123",
"requestedBy": "web-frontend",
"reason": "user-opened-ide-tab",
"requestedAt": "2025-11-21T09:15:00Z",
"traceID": "abc123"
}
```

###### MQ consumer and controller integration

1. A dedicated `ResumeConsumer` process (can be part of the agentd binary) connects to the NATS server.
2. It subscribes to the `agentcube.sandbox.resume` subject using a **durable subscription** (queue group or durable name) so that resumes are processed even if there are multiple replicas.
Copy link

Copilot AI Nov 21, 2025

Choose a reason for hiding this comment

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

Typo: "ensure sandboxes are processed" should be "ensures that resume messages are processed" or similar. The phrase "so that resumes are processed" is grammatically unclear - it should specify what "resumes" refers to (resume messages or resume operations).

Suggested change
2. It subscribes to the `agentcube.sandbox.resume` subject using a **durable subscription** (queue group or durable name) so that resumes are processed even if there are multiple replicas.
2. It subscribes to the `agentcube.sandbox.resume` subject using a **durable subscription** (queue group or durable name) so that resume messages are processed even if there are multiple replicas.

Copilot uses AI. Check for mistakes.
3. For each message:
- Validate and parse the payload.
- Use the Kubernetes client to fetch the referenced `Sandbox`.
- If the sandbox is already `Running`, ACK and ignore.
- If the sandbox is `Paused` (or `Error` but resumable), set or update:
- `sandbox.lifecycle.volcano.sh/resume-requested-at=<requestedAt or now>`.
- Optionally, `sandbox.lifecycle.volcano.sh/resume-requested-by=<requestedBy>`.
- Persist the annotation update; on success, ACK the NATS message.
- On transient errors (e.g. API server throttling), do not ACK so that NATS redelivers, and/or explicitly resubscribe with backoff.
4. The normal Kubernetes reconciliation loop picks up the changed annotation and executes the containerd resume workflow described earlier.

This design deliberately keeps the MQ consumer stateless and idempotent; the source of truth for desired state remains the `Sandbox` object in etcd.

###### Ordering, duplicates, and failure modes

- **Duplicates**: At-least-once delivery (via durable subscription and reconnect behavior) means the same resume intent may be delivered multiple times. We handle this by making annotations idempotent: setting `resume-requested-at` to the latest timestamp is safe even if resume is already in progress or completed.
- **Ordering**: If multiple resume requests arrive, we only care about the most recent one. NATS preserves ordering per subject, but the controller logic does not rely on strict ordering.
- **Lost messages**: For long consumer outages, plain NATS does not guarantee persistence. To mitigate this, we:
Copy link

Copilot AI Nov 21, 2025

Choose a reason for hiding this comment

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

The statement "For long consumer outages, plain NATS does not guarantee persistence" (line 260) contradicts the earlier claim (line 207) that "NATS durable subscriptions and manual acknowledgements give us at-least-once behavior." This inconsistency should be resolved. The document should be clear upfront that core NATS without JetStream does not persist messages when no consumers are connected, which is a significant limitation for a resume signaling mechanism where reliability is important.

Copilot uses AI. Check for mistakes.
- Treat the MQ as a **hint** and store the true desired state in the `Sandbox` object.
- Allow callers to fall back to the annotation-based resume path if a message might have been lost.
- Document that environments requiring strong durability can swap in a persistent MQ (e.g. Kafka) behind the same interface.
- **Back-pressure**: If the Kubernetes API becomes slow, the consumer can slow its processing rate and rely on NATS’ internal buffering and flow control.
- **Security**: NATS can be configured with TLS and token-based or NKey authentication. The consumer uses a dedicated NATS account/creds with access only to the required subjects.

Copy link

Copilot AI Nov 21, 2025

Choose a reason for hiding this comment

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

The security section mentions "token-based or NKey authentication" for NATS (line 265), but doesn't address authentication/authorization for the annotation-based resume path. Any client with permission to update Sandbox annotations can trigger a resume, potentially bypassing intended access controls. Consider documenting the required RBAC permissions and whether additional authorization checks are needed beyond Kubernetes RBAC.

Suggested change
- **Annotation-based resume path**: The annotation-based resume mechanism relies on Kubernetes RBAC for access control. Only trusted controllers or users should be granted permission to update the `Sandbox` resource's annotations (specifically, the `sandbox.lifecycle.volcano.sh/resume-requested-at` and related keys). Granting broader update access could allow unauthorized clients to trigger resume operations, potentially bypassing intended access controls. It is recommended to define fine-grained RBAC roles that restrict annotation updates to only those principals that are authorized to trigger resume actions. If additional authorization checks are required beyond Kubernetes RBAC, they should be implemented in the controller logic and documented here.

Copilot uses AI. Check for mistakes.
###### Alternatives considered for MQ

- **Kafka**: Excellent for high-throughput event streams, but heavier to operate, with more complex configuration and higher resource usage. For our relatively low-volume, latency-sensitive resume events, this is overkill.
- **RabbitMQ**: Feature-rich AMQP broker, but operationally more complex and less cloud-native than NATS; clustering and observability are more involved.
- **Cloud provider queues (SQS, Pub/Sub, etc.)**: Great for managed environments but would tie AgentCube to specific cloud vendors and complicate on-premise deployments.

Given AgentCube’s focus on Kubernetes-first, multi-environment deployment, plain NATS offers a good balance of simplicity, performance, and operational cost for this relatively low-volume control-plane signal.

#### Control Flow

```mermaid
flowchart TD
Start["Reconcile Sandbox"] --> CheckIdle{Idle timeout exceeded?}
CheckIdle -- "No" --> Exit["Requeue/Exit"]
CheckIdle -- "Yes" --> VerifyPod["Verify bound pod exists"]
VerifyPod -- "Missing" --> Warn["Log warning & requeue"]
VerifyPod -- "Found" --> PauseContainers["Pause pod containers via containerd"]
PauseContainers --> UpdateState["Update annotations/status: state=Paused"]
UpdateState --> EmitPaused["Emit SandboxPaused event"]
EmitPaused --> Exit
Start["Reconcile Sandbox"] --> WaitResume{Resume signal?}
WaitResume -- "Annotation" --> HandleAnnotation["Process resume annotation"]
WaitResume -- "Message queue" --> HandleMQ["Process MQ resume message"]
HandleAnnotation --> ResumeContainers
HandleMQ --> ResumeContainers
ResumeContainers["Resume containers via containerd"] --> RefreshActivity["Refresh last-activity timestamp"]
RefreshActivity --> UpdateRunning["Set state=Running & clear triggers"]
UpdateRunning --> EmitResumed["Emit SandboxResumed event"]
EmitResumed --> Exit
Comment on lines +278 to +295
Copy link
Contributor

Choose a reason for hiding this comment

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

medium

The control flow diagram is a bit confusing as it shows two separate Start nodes for what should be a single reconciliation loop. A reconciler typically has one entry point and then branches based on the state of the resource. I suggest restructuring the diagram to reflect a single, unified flow, which will improve the clarity of the proposed design. The suggested diagram below first checks for a resume signal and then for idleness, which is a more common pattern in controller logic.

Suggested change
flowchart TD
Start["Reconcile Sandbox"] --> CheckIdle{Idle timeout exceeded?}
CheckIdle -- "No" --> Exit["Requeue/Exit"]
CheckIdle -- "Yes" --> VerifyPod["Verify bound pod exists"]
VerifyPod -- "Missing" --> Warn["Log warning & requeue"]
VerifyPod -- "Found" --> PauseContainers["Pause pod containers via containerd"]
PauseContainers --> UpdateState["Update annotations/status: state=Paused"]
UpdateState --> EmitPaused["Emit SandboxPaused event"]
EmitPaused --> Exit
Start["Reconcile Sandbox"] --> WaitResume{Resume signal?}
WaitResume -- "Annotation" --> HandleAnnotation["Process resume annotation"]
WaitResume -- "Message queue" --> HandleMQ["Process MQ resume message"]
HandleAnnotation --> ResumeContainers
HandleMQ --> ResumeContainers
ResumeContainers["Resume containers via containerd"] --> RefreshActivity["Refresh last-activity timestamp"]
RefreshActivity --> UpdateRunning["Set state=Running & clear triggers"]
UpdateRunning --> EmitResumed["Emit SandboxResumed event"]
EmitResumed --> Exit
flowchart TD
Start["Reconcile Sandbox"] --> WaitResume{Resume signal?}
WaitResume -- "Yes (Annotation or MQ)" --> ResumeContainers["Resume containers via containerd"]
ResumeContainers --> RefreshActivity["Refresh last-activity timestamp"]
RefreshActivity --> UpdateRunning["Set state=Running & clear triggers"]
UpdateRunning --> EmitResumed["Emit SandboxResumed event"]
EmitResumed --> Exit["Requeue/Exit"]
WaitResume -- "No" --> CheckIdle{Idle timeout exceeded?}
CheckIdle -- "No" --> Exit
CheckIdle -- "Yes" --> VerifyPod["Verify bound pod exists"]
VerifyPod -- "Missing" --> Warn["Log warning & requeue"]
VerifyPod -- "Found" --> PauseContainers["Pause pod containers via containerd"]
PauseContainers --> UpdateState["Update annotations/status: state=Paused"]
UpdateState --> EmitPaused["Emit SandboxPaused event"]
EmitPaused --> Exit

```
Comment on lines +277 to +296
Copy link

Copilot AI Nov 21, 2025

Choose a reason for hiding this comment

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

The control flow diagram has two separate starting points labeled "Start["Reconcile Sandbox"]" (lines 276 and 284). This creates an ambiguous and potentially confusing flowchart structure. The pause and resume flows should be part of a single unified reconciliation flow where the controller checks both idle timeout and resume signals in one reconciliation cycle. Consider restructuring this into a single flowchart that starts with one "Reconcile Sandbox" node and then branches based on the sandbox's current state and signals.

Copilot uses AI. Check for mistakes.
Comment on lines +277 to +296
Copy link

Copilot AI Nov 21, 2025

Choose a reason for hiding this comment

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

The mermaid diagram has a logic issue: it shows two separate flows both starting with Start["Reconcile Sandbox"], which creates duplicate starting nodes. The pause and resume flows should be part of a single flowchart with conditional branches, or they should be clearly separated as different scenarios. Currently, both CheckIdle and WaitResume appear to be evaluated simultaneously from the same start node, which doesn't make logical sense.

Copilot uses AI. Check for mistakes.

#### Observability

- Metrics (Prometheus):
- `agentcube_sandbox_pause_operations_total{status="success|failure", reason="..."}`
- `agentcube_sandbox_resume_operations_total{trigger="annotation|mq", status="success|failure", reason="..."}`
- `agentcube_sandbox_pause_duration_seconds` (histogram)
- `agentcube_sandbox_resume_duration_seconds` (histogram)
- `agentcube_sandbox_paused` (gauge: number of sandboxes currently paused)
- Events on Sandbox and Pod resources for pause/resume.
- Structured logs with sandbox ID, namespace, and reason.
- Optional tracing span around containerd operations.

#### Test Plan

- **Unit tests**: Fake containerd client verifying pause/resume sequencing and error handling in reconciler.
- **Integration tests**: envtest-based controller suite ensuring annotations/status transitions.
- **e2e tests**: Deploy sandbox, simulate activity heartbeat, wait for pause, ensure resume via both annotation and MQ trigger.
- **Stress tests**: Batch pause/resume to gauge controller throughput and node impact.

### Alternatives

- **Keep deletion-based reclaim**: Simpler but forces cold boot and data loss; fails goal of quick resume.
- **Scale pod replicas to zero**: Works with workloads managed by Deployments, but Sandbox pods are singleton objects bound to CRD; scaling is not applicable.
- **Rely on runtime class features (e.g., Kata VM suspend)**: Adds dependencies on specific runtime configurations and hardware, reducing portability.
Loading