task = scope.trackTask(new Task());
+ task.close();
+} // β scope automatically closed
+```
\ No newline at end of file
diff --git a/internal-apis/src/main/java/io/github/jbellis/jvector/status/StatusContext.java b/internal-apis/src/main/java/io/github/jbellis/jvector/status/StatusContext.java
new file mode 100644
index 000000000..242cb3ec4
--- /dev/null
+++ b/internal-apis/src/main/java/io/github/jbellis/jvector/status/StatusContext.java
@@ -0,0 +1,548 @@
+/*
+ * Copyright DataStax, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package io.github.jbellis.jvector.status;
+
+import io.github.jbellis.jvector.status.eventing.StatusSink;
+import io.github.jbellis.jvector.status.eventing.StatusSource;
+import io.github.jbellis.jvector.status.eventing.StatusUpdate;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+/**
+ * Central coordinator for task tracking that manages scopes, trackers, sinks, and the monitoring infrastructure.
+ * A context represents a cohesive tracking scope (such as a batch operation or application subsystem)
+ * and owns exactly one {@link StatusMonitor} instance plus a collection of {@link StatusSink}s.
+ *
+ * Architectural Model:
+ * The API enforces a clear separation between organizational structure and work execution:
+ *
+ * - {@link StatusScope}: Organizational containers with no progress/state
+ *
+ * - Created via {@link #createScope(String)}
+ * - Can contain child scopes (nested organization)
+ * - Can contain task trackers (actual work)
+ * - Completion checked via {@link StatusScope#isComplete()}
+ *
+ *
+ * - {@link StatusTracker}: Leaf nodes representing actual work
+ *
+ * - Created via {@link StatusScope#trackTask}
+ * - Have progress and state (PENDING, RUNNING, SUCCESS, etc.)
+ * - Cannot have children - purely leaf nodes
+ * - Report status via {@link StatusSource#getTaskStatus()}
+ *
+ *
+ *
+ *
+ * Key Responsibilities:
+ *
+ * - Scope Creation: Factory for {@link StatusScope} via {@link #createScope}
+ * - Tracker Creation: Delegates to {@link StatusScope#trackTask} to create {@link StatusTracker}s
+ * - Monitor Ownership: Owns a single {@link StatusMonitor} that polls all trackers
+ * - Sink Management: Maintains a collection of sinks that receive status updates
+ * - Status Routing: Routes status updates from monitor to all registered sinks
+ * - Lifecycle Management: Coordinates cleanup of scopes, trackers, monitor, and sinks
+ *
+ *
+ * Data Flow:
+ *
+ * - User creates scope via {@code createScope()}
+ * - Scope creates trackers via {@code trackTask()}
+ * - Context registers tracker with its {@link StatusMonitor}
+ * - Monitor periodically polls tracker via {@link StatusTracker#refreshAndGetStatus()}
+ * - Tracker observes its object and caches the status
+ * - Monitor forwards status to context via {@link #pushStatus}
+ * - Context routes status to all registered {@link StatusSink}s
+ * - Status flows unidirectionally: Task β Tracker β Monitor β Context β Sinks
+ *
+ *
+ * Usage Example:
+ * {@code
+ * try (StatusContext context = new StatusContext("data-pipeline")) {
+ * context.addSink(new ConsolePanelSink.builder().build());
+ * context.addSink(new MetricsStatusSink());
+ *
+ * // Create organizational scope
+ * try (StatusScope ingestionScope = context.createScope("Ingestion");
+ * StatusScope processingScope = context.createScope("Processing")) {
+ *
+ * // Add actual work as leaf tasks
+ * StatusTracker loader = ingestionScope.trackTask(new LoadTask());
+ * StatusTracker transformer = processingScope.trackTask(new TransformTask());
+ *
+ * // Execute tasks...
+ * loader.getTracked().execute();
+ * transformer.getTracked().execute();
+ *
+ * // Check scope completion
+ * boolean ingestionDone = ingestionScope.isComplete();
+ * }
+ * }
+ * }
+ *
+ * Thread Safety:
+ *
+ * - All public methods are thread-safe and can be called concurrently from multiple threads
+ * - Scope creation: Multiple threads can create scopes and child scopes concurrently
+ * - Tracker creation: Multiple threads can create trackers concurrently
+ * - Sink management: Sinks can be added/removed while trackers are active
+ * - Sink notifications: Delivered on the {@link StatusMonitor} background thread, not the caller's thread
+ * - Internal collections: Uses {@link CopyOnWriteArrayList} for thread-safe iteration during concurrent modifications
+ * - Closed state: Checked via volatile boolean for visibility across threads
+ *
+ *
+ * This class implements {@link StatusSink} to receive events from the monitor and forward them
+ * to registered sinks. It also implements {@link AutoCloseable} to ensure proper cleanup of resources.
+ *
+ * @see StatusScope
+ * @see StatusTracker
+ * @see StatusMonitor
+ * @see StatusSink
+ * @since 4.0.0
+ */
+public final class StatusContext implements AutoCloseable, StatusSink {
+
+ private static final Logger logger = LogManager.getLogger(StatusContext.class);
+
+ private final String name;
+ private final Duration defaultPollInterval;
+ private final CopyOnWriteArrayList sinks;
+ private final CopyOnWriteArrayList> activeTrackers;
+ private final CopyOnWriteArrayList activeScopes;
+ private final StatusMonitor monitor;
+ private volatile boolean closed = false;
+
+ /**
+ * Creates a new context with the specified name and default configuration
+ * (100ms poll interval, no sinks).
+ *
+ * @param name the name of this context for identification purposes
+ */
+ public StatusContext(String name) {
+ this(name, Duration.ofMillis(1000), List.of());
+ }
+
+ /**
+ * Creates a new context with the specified name and poll interval, with no sinks.
+ *
+ * @param name the name of this context for identification purposes
+ * @param defaultPollInterval the default interval between status observations
+ */
+ public StatusContext(String name, Duration defaultPollInterval) {
+ this(name, defaultPollInterval, List.of());
+ }
+
+ /**
+ * Creates a new context with the specified name and initial sinks, using the
+ * default poll interval of 100ms.
+ *
+ * @param name the name of this context for identification purposes
+ * @param sinks initial collection of sinks to register
+ */
+ public StatusContext(String name, List sinks) {
+ this(name, Duration.ofMillis(100), sinks);
+ }
+
+ /**
+ * Creates a new context with full configuration.
+ *
+ * @param name the name of this context for identification purposes
+ * @param defaultPollInterval the default interval between status observations (minimum 100ms enforced)
+ * @param sinks initial collection of sinks to register
+ */
+ public StatusContext(String name, Duration defaultPollInterval, List sinks) {
+ this.name = Objects.requireNonNull(name, "name");
+
+ // Enforce minimum poll interval of 100ms
+ Duration requestedInterval = Objects.requireNonNullElse(defaultPollInterval, Duration.ofMillis(100));
+ if (requestedInterval.toMillis() < 100) {
+ logger.warn("Poll interval of {}ms is below minimum 100ms. Using 100ms instead. " +
+ "Faster polling significantly increases CPU usage with minimal benefit. " +
+ "The monitoring thread wakes up on this interval to poll all tasks. " +
+ "For context '{}', requested {}ms, using 100ms.",
+ requestedInterval.toMillis(), name, requestedInterval.toMillis());
+ this.defaultPollInterval = Duration.ofMillis(100);
+ } else {
+ this.defaultPollInterval = requestedInterval;
+ }
+
+ this.sinks = new CopyOnWriteArrayList<>(Objects.requireNonNullElse(sinks, List.of()));
+ this.activeTrackers = new CopyOnWriteArrayList<>();
+ this.activeScopes = new CopyOnWriteArrayList<>();
+ this.monitor = new StatusMonitor(this);
+ }
+
+ /**
+ * Creates a root-level organizational scope for grouping related tasks.
+ * Scopes provide hierarchical organization without having their own progress or state.
+ *
+ * @param name the name of the scope
+ * @return a new StatusScope registered with this context
+ */
+ public StatusScope createScope(String name) {
+ checkNotClosed();
+ StatusScope scope = new StatusScope(this, null, name);
+ activeScopes.add(scope);
+ scopeStarted(scope);
+ return scope;
+ }
+
+ StatusScope createChildScope(StatusScope parent, String name) {
+ checkNotClosed();
+ StatusScope scope = new StatusScope(this, parent, name);
+ activeScopes.add(scope);
+ scopeStarted(scope);
+ return scope;
+ }
+
+ /**
+ * Registers an existing scope with this context. This is used internally
+ * by StatusScope's default constructor to register itself.
+ *
+ * @param scope the scope to register
+ */
+ void registerScope(StatusScope scope) {
+ activeScopes.add(scope);
+ scopeStarted(scope);
+ }
+
+ /**
+ * Registers an existing tracker with this context. This is used internally
+ * by StatusTracker's public constructor to register itself.
+ *
+ * @param tracker the tracker to register
+ * @param initialStatus the initial status of the tracker
+ */
+ void registerTracker(StatusTracker tracker, StatusUpdate initialStatus) {
+ activeTrackers.add(tracker);
+ monitor.register(tracker, defaultPollInterval, initialStatus);
+ taskStarted(tracker);
+ }
+
+ @Override
+ public void scopeStarted(StatusScope scope) {
+ notifySinks(sink -> sink.scopeStarted(scope), "notifying sink of scope start");
+ }
+
+ /**
+ * Creates a tracker for a task implementing {@link StatusSource} without requiring
+ * an explicit scope. A scope will be automatically created for the tracker.
+ *
+ * @param tracked the object to track
+ * @param the type of object being tracked
+ * @return a new StatusTracker with an auto-created scope
+ */
+ public > StatusTracker track(U tracked) {
+ return track(tracked, StatusSource::getTaskStatus);
+ }
+
+ /**
+ * Creates a tracker with a custom status function without requiring an explicit scope.
+ * A scope will be automatically created for the tracker.
+ *
+ * @param tracked the object to track
+ * @param statusFunction function to extract status from the tracked object
+ * @param the type of object being tracked
+ * @return a new StatusTracker with an auto-created scope
+ */
+ public StatusTracker track(T tracked, Function> statusFunction) {
+ return createTracker(null, tracked, statusFunction);
+ }
+
+ // Package-private methods for StatusScope to create tasks
+ > StatusTracker trackInScope(StatusScope scope, U tracked) {
+ return trackInScope(scope, tracked, StatusSource::getTaskStatus);
+ }
+
+ StatusTracker trackInScope(StatusScope scope,
+ T tracked,
+ Function> statusFunction) {
+ return createTracker(scope, tracked, statusFunction);
+ }
+
+ private StatusTracker createTracker(StatusScope scope,
+ T tracked,
+ Function> statusFunction) {
+ checkNotClosed();
+ // scope can be null - StatusTracker will create one automatically if needed
+
+ if (scope != null && scope.getContext() != this) {
+ throw new IllegalArgumentException("Scope belongs to a different StatusContext");
+ }
+
+ StatusTracker tracker = new StatusTracker<>(this, scope, tracked, statusFunction);
+ StatusUpdate initial = tracker.refreshAndGetStatus();
+ activeTrackers.add(tracker);
+ monitor.register(tracker, defaultPollInterval, initial);
+ taskStarted(tracker);
+ return tracker;
+ }
+
+ /**
+ * Adds a sink to receive status updates for all trackers in this context.
+ * The sink will immediately begin receiving events for existing trackers
+ * and all future trackers.
+ *
+ * @param sink the sink to add, ignored if null or already registered
+ * @throws IllegalStateException if this context has been closed
+ */
+ public void addSink(StatusSink sink) {
+ checkNotClosed();
+ if (sink != null && !sinks.contains(sink)) {
+ sinks.add(sink);
+ }
+ }
+
+ /**
+ * Removes a sink from this context. After removal, the sink will no longer
+ * receive status updates.
+ *
+ * @param sink the sink to remove
+ * @throws IllegalStateException if this context has been closed
+ */
+ public void removeSink(StatusSink sink) {
+ checkNotClosed();
+ sinks.remove(sink);
+ }
+
+ /**
+ * Returns a snapshot of all registered sinks. The returned list is a defensive
+ * copy and will not reflect subsequent additions or removals.
+ *
+ * @return an immutable snapshot of registered sinks
+ */
+ public List getSinks() {
+ return new ArrayList<>(sinks);
+ }
+
+ /**
+ * Returns the number of currently active (not closed) trackers.
+ *
+ * @return the count of active trackers
+ */
+ public int getActiveTrackerCount() {
+ return activeTrackers.size();
+ }
+
+ /**
+ * Returns a snapshot of all active trackers. The returned list is a defensive
+ * copy and will not reflect subsequent tracker additions or closures.
+ *
+ * @return an immutable snapshot of active trackers
+ */
+ public List> getActiveTrackers() {
+ return new ArrayList<>(activeTrackers);
+ }
+
+ /**
+ * Returns the name of this context.
+ *
+ * @return the context name
+ */
+ public String getName() {
+ return name;
+ }
+
+ /**
+ * Returns the default poll interval used when creating trackers without
+ * an explicit interval.
+ *
+ * @return the default poll interval
+ */
+ public Duration getDefaultPollInterval() {
+ return defaultPollInterval;
+ }
+
+ /**
+ * Callback invoked when a tracker is closed. Unregisters the tracker from
+ * monitoring and removes it from the active tracker list.
+ *
+ * @param tracker the tracker that was closed
+ */
+ void onTrackerClosed(StatusTracker> tracker) {
+ monitor.unregister(tracker);
+ activeTrackers.remove(tracker);
+ taskFinished(tracker);
+
+ // Remove from parent scope if it has one
+ StatusScope parentScope = tracker.getParentScope();
+ if (parentScope != null) {
+ parentScope.removeChildTask(tracker);
+ }
+ }
+
+ /**
+ * Callback invoked when a scope is closed. Removes the scope from
+ * the active scope list.
+ *
+ * @param scope the scope that was closed
+ */
+ void onScopeClosed(StatusScope scope) {
+ activeScopes.remove(scope);
+ scopeFinished(scope);
+ }
+
+ @Override
+ public void scopeFinished(StatusScope scope) {
+ notifySinks(sink -> sink.scopeFinished(scope), "notifying sink of scope finish");
+ }
+
+ private List snapshotSinks() {
+ return new ArrayList<>(sinks);
+ }
+
+ private void checkNotClosed() {
+ if (closed) {
+ throw new IllegalStateException("StatusContext '" + name + "' has been closed");
+ }
+ }
+
+ /**
+ * Routes a status update from the monitor to all registered sinks.
+ * This method implements the unidirectional status flow: the monitor observes
+ * the tracked object, the tracker caches the status, and this method forwards
+ * it to all sinks for processing.
+ *
+ * Called by {@link StatusMonitor} after {@link StatusTracker#refreshAndGetStatus()}.
+ *
+ * @param tracker the tracker reporting the status
+ * @param status the observed status
+ * @param the type of object being tracked
+ */
+ void pushStatus(StatusTracker tracker, StatusUpdate status) {
+ if (tracker == null || status == null) {
+ return;
+ }
+
+ // Status flows unidirectionally: Monitor observes β Context routes β Sinks receive
+ // (Tracker already cached the status in refreshAndGetStatus())
+ taskUpdate(tracker, status);
+ }
+
+ /**
+ * Notifies all registered sinks that a task has started tracking.
+ * This method is part of the {@link StatusSink} interface and is called
+ * when a new tracker is created.
+ *
+ * @param task the tracker that started
+ */
+ @Override
+ public void taskStarted(StatusTracker> task) {
+ notifySinks(sink -> sink.taskStarted(task), "notifying sink of task start");
+ }
+
+ /**
+ * Notifies all registered sinks of a status update.
+ * This method is part of the {@link StatusSink} interface and is called
+ * by {@link #pushStatus} when the monitor provides a new status observation.
+ *
+ * @param task the tracker reporting the update
+ * @param status the new status
+ */
+ @Override
+ public void taskUpdate(StatusTracker> task, StatusUpdate> status) {
+ notifySinks(sink -> sink.taskUpdate(task, status), "notifying sink of status change");
+ }
+
+ /**
+ * Notifies all registered sinks that a task has finished.
+ * This method is part of the {@link StatusSink} interface and is called
+ * when a tracker is closed.
+ *
+ * @param task the tracker that finished
+ */
+ @Override
+ public void taskFinished(StatusTracker> task) {
+ notifySinks(sink -> sink.taskFinished(task), "notifying sink of task finish");
+ }
+
+ /**
+ * Helper method to safely notify all sinks, catching and logging any exceptions
+ * to prevent one failing sink from affecting others.
+ *
+ * @param sinkAction the action to perform on each sink
+ * @param errorContext description of the action for error messages
+ */
+ private void notifySinks(Consumer sinkAction, String errorContext) {
+ for (StatusSink sink : snapshotSinks()) {
+ try {
+ sinkAction.accept(sink);
+ } catch (Exception e) {
+ logger.warn("Error {}: {}", errorContext, e.getMessage(), e);
+ }
+ }
+ }
+
+ /**
+ * Closes this context and all associated resources. This method:
+ *
+ * - Closes all active trackers (which notifies sinks)
+ * - Stops the monitoring thread
+ * - Clears all registered sinks
+ *
+ *
+ * This method is idempotent and safe to call multiple times.
+ * After closing, attempts to create new trackers or modify sinks will throw
+ * {@link IllegalStateException}.
+ */
+ @Override
+ public void close() {
+ if (closed) {
+ return;
+ }
+ closed = true;
+
+ // Close trackers first so they can signal sinks before monitors are torn down.
+ for (StatusTracker> tracker : new ArrayList<>(activeTrackers)) {
+ try {
+ tracker.close();
+ } catch (Exception e) {
+ logger.warn("Error closing tracker: {}", e.getMessage(), e);
+ }
+ }
+ activeTrackers.clear();
+
+ monitor.close();
+ sinks.clear();
+ }
+
+ /**
+ * Returns whether this context has been closed.
+ *
+ * @return true if {@link #close()} has been called, false otherwise
+ */
+ public boolean isClosed() {
+ return closed;
+ }
+
+ /**
+ * Returns the path identifier for this context (name prefixed with slash).
+ *
+ * @return the path identifier
+ */
+ public String getPath() {
+ return "/" + name;
+ }
+}
diff --git a/internal-apis/src/main/java/io/github/jbellis/jvector/status/StatusMonitor.java b/internal-apis/src/main/java/io/github/jbellis/jvector/status/StatusMonitor.java
new file mode 100644
index 000000000..8086a2850
--- /dev/null
+++ b/internal-apis/src/main/java/io/github/jbellis/jvector/status/StatusMonitor.java
@@ -0,0 +1,211 @@
+/*
+ * Copyright DataStax, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package io.github.jbellis.jvector.status;
+
+import io.github.jbellis.jvector.status.eventing.StatusUpdate;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.time.Duration;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * Background polling engine used by {@link StatusContext} to periodically observe tracked objects.
+ * Each context owns exactly one monitor, which runs a single daemon thread that continuously
+ * polls registered {@link StatusTracker}s at their configured intervals.
+ *
+ *
Key Responsibilities:
+ *
+ * - Polling Loop: Maintains a single daemon thread that polls all registered trackers
+ * - Interval Management: Respects per-tracker poll intervals to balance responsiveness and overhead
+ * - Status Observation: Calls {@link StatusTracker#refreshAndGetStatus()} on trackers
+ * to observe their tracked objects
+ * - Event Routing: Forwards observed status to {@link StatusContext#pushStatus} for
+ * distribution to sinks
+ * - Cleanup: Automatically removes closed trackers from the polling loop
+ *
+ *
+ * Architectural Flow:
+ *
+ * - Monitor thread wakes up and checks all registered trackers
+ * - For each tracker whose poll interval has elapsed:
+ *
+ * - Monitor calls {@link StatusTracker#refreshAndGetStatus()}
+ * - Tracker observes its object and caches the status
+ * - Monitor receives the observed status
+ * - Monitor forwards status to {@link StatusContext#pushStatus}
+ * - Context routes status to all registered sinks
+ *
+ *
+ * - Monitor calculates next wake time based on shortest remaining interval
+ * - Monitor sleeps until next wake time
+ *
+ *
+ * Thread Safety:
+ *
+ * - Single monitor thread: All polling occurs on one dedicated daemon thread named "StatusMonitor"
+ * - Concurrent registration: {@link #register} and {@link #unregister} can be called from any thread
+ * due to {@link ConcurrentHashMap} usage for tracker storage
+ * - Status observation: Each tracker is polled exclusively from the monitor thread,
+ * preventing concurrent calls to {@link StatusTracker#refreshAndGetStatus()}
+ * - Shutdown coordination: Uses {@link AtomicBoolean} for thread-safe shutdown signaling
+ * - Volatile timing: {@code nextPollMillis} in {@link MonitoredEntry} is volatile for cross-thread visibility
+ *
+ *
+ * This class is package-private and should only be instantiated by {@link StatusContext}.
+ *
+ * @see StatusContext
+ * @see StatusTracker
+ * @see StatusTracker#refreshAndGetStatus()
+ * @since 4.0.0
+ */
+final class StatusMonitor implements AutoCloseable {
+
+ private static final Logger logger = LogManager.getLogger(StatusMonitor.class);
+ private static final long MIN_SLEEP_MILLIS = 10;
+
+ private final StatusContext context;
+ private final ConcurrentHashMap, MonitoredEntry>> entries = new ConcurrentHashMap<>();
+ private final AtomicBoolean running = new AtomicBoolean(true);
+ private final Thread monitorThread;
+
+ StatusMonitor(StatusContext context) {
+ this.context = context;
+ this.monitorThread = new Thread(this::runLoop, "StatusMonitor");
+ this.monitorThread.setDaemon(true);
+ this.monitorThread.start();
+ }
+
+ /**
+ * Registers a tracker for periodic polling. The initial status is immediately
+ * pushed to the context to notify sinks of the new tracker.
+ *
+ * @param tracker the tracker to monitor
+ * @param pollInterval the interval between status observations
+ * @param initialStatus the initial status to push to sinks
+ * @param the type of object being tracked
+ */
+ void register(StatusTracker tracker,
+ Duration pollInterval,
+ StatusUpdate initialStatus) {
+ entries.put(tracker, new MonitoredEntry<>(tracker, pollInterval));
+ context.pushStatus(tracker, initialStatus);
+ }
+
+ /**
+ * Unregisters a tracker from polling. After this call, the monitor will no
+ * longer observe the tracker or forward its status updates.
+ *
+ * @param tracker the tracker to unregister
+ */
+ void unregister(StatusTracker> tracker) {
+ entries.remove(tracker);
+ }
+
+ /**
+ * Main polling loop that runs continuously until the monitor is closed.
+ * For each registered tracker, checks if its poll interval has elapsed and
+ * if so, calls {@link #pollTracker} to observe and forward status.
+ *
+ * The loop automatically removes closed trackers and calculates optimal
+ * sleep times based on the shortest remaining poll interval.
+ */
+ private void runLoop() {
+ while (running.get()) {
+ long now = System.currentTimeMillis();
+ long nextWake = now + MIN_SLEEP_MILLIS;
+
+ for (Map.Entry, MonitoredEntry>> mapEntry : entries.entrySet()) {
+ StatusTracker> tracker = mapEntry.getKey();
+ MonitoredEntry> entry = mapEntry.getValue();
+
+ if (tracker.isClosed()) {
+ entries.remove(tracker);
+ continue;
+ }
+
+ if (now >= entry.nextPollMillis) {
+ pollTracker(entry);
+ entry.nextPollMillis = now + entry.intervalMillis;
+ }
+
+ nextWake = Math.min(nextWake, entry.nextPollMillis);
+ }
+
+ long sleepMillis = Math.max(MIN_SLEEP_MILLIS, nextWake - System.currentTimeMillis());
+ try {
+ TimeUnit.MILLISECONDS.sleep(sleepMillis);
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ break;
+ }
+ }
+ }
+
+ /**
+ * Polls a single tracker by calling its {@link StatusTracker#refreshAndGetStatus()} method
+ * and forwarding the result to the context. Errors during polling are caught and logged
+ * to prevent one failing tracker from affecting others.
+ *
+ * @param entry the monitored entry containing the tracker to poll
+ * @param the type of object being tracked
+ */
+ @SuppressWarnings("unchecked") // Safe: MonitoredEntry always contains StatusTracker
+ private void pollTracker(MonitoredEntry entry) {
+ StatusTracker tracker = entry.tracker;
+ try {
+ // Tracker observes its own tracked object and caches the result
+ StatusUpdate status = tracker.refreshAndGetStatus();
+ // Context routes the observed status to sinks
+ context.pushStatus(tracker, status);
+ } catch (Throwable t) {
+ logger.warn("Error polling status for tracker: {}", t.getMessage(), t);
+ }
+ }
+
+ /**
+ * Closes the monitor, stopping the polling thread and clearing all registered trackers.
+ * This method is idempotent and safe to call multiple times.
+ */
+ @Override
+ public void close() {
+ if (running.compareAndSet(true, false)) {
+ monitorThread.interrupt();
+ try {
+ monitorThread.join(500);
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ }
+ entries.clear();
+ }
+ }
+
+ private static final class MonitoredEntry {
+ final StatusTracker tracker;
+ final long intervalMillis;
+ volatile long nextPollMillis;
+
+ MonitoredEntry(StatusTracker tracker, Duration pollInterval) {
+ this.tracker = tracker;
+ this.intervalMillis = Math.max(pollInterval.toMillis(), MIN_SLEEP_MILLIS);
+ this.nextPollMillis = System.currentTimeMillis();
+ }
+ }
+}
diff --git a/internal-apis/src/main/java/io/github/jbellis/jvector/status/StatusScope.java b/internal-apis/src/main/java/io/github/jbellis/jvector/status/StatusScope.java
new file mode 100644
index 000000000..a3f190488
--- /dev/null
+++ b/internal-apis/src/main/java/io/github/jbellis/jvector/status/StatusScope.java
@@ -0,0 +1,344 @@
+/*
+ * Copyright DataStax, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package io.github.jbellis.jvector.status;
+
+import io.github.jbellis.jvector.status.eventing.StatusSource;
+import io.github.jbellis.jvector.status.eventing.StatusUpdate;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.function.Function;
+
+/**
+ * An organizational container for grouping related tasks in a hierarchical structure.
+ * Unlike {@link StatusTracker}, a StatusScope has no progress or state of its own -
+ * it serves purely as an umbrella for organizing child tasks and nested scopes.
+ *
+ * Architectural Role:
+ *
+ * - Scopes are organizational nodes that can contain:
+ *
+ * - Child scopes (for nested organization)
+ * - Task trackers (actual work units)
+ *
+ *
+ * - Task Trackers are leaf nodes representing actual work:
+ *
+ * - Have progress and state (PENDING, RUNNING, SUCCESS, etc.)
+ * - Cannot have children (enforced by design)
+ * - Report status updates via {@link StatusSource}
+ *
+ *
+ *
+ *
+ * Hierarchy Example:
+ *
+ * DataPipeline (Scope)
+ * ββ Ingestion (Scope)
+ * β ββ LoadCSV (Task: 45% complete)
+ * β ββ ValidateSchema (Task: 100% complete)
+ * ββ Processing (Scope)
+ * ββ Transform (Task: 30% complete)
+ * ββ Index (Task: PENDING)
+ *
+ *
+ * Usage Example with StatusContext:
+ * {@code
+ * try (StatusContext context = new StatusContext("pipeline");
+ * StatusScope dataScope = context.createScope("DataPipeline")) {
+ *
+ * // Create nested organizational scopes
+ * StatusScope ingestionScope = dataScope.createChildScope("Ingestion");
+ * StatusScope processingScope = dataScope.createChildScope("Processing");
+ *
+ * // Add actual tasks as leaf nodes
+ * StatusTracker loadTracker = ingestionScope.trackTask(new LoadTask());
+ * StatusTracker validateTracker = ingestionScope.trackTask(new ValidateTask());
+ *
+ * // Execute tasks...
+ * }
+ * }
+ *
+ * Usage Example with Default Context:
+ * {@code
+ * // Create a standalone scope with its own default context
+ * try (StatusScope scope = new StatusScope("my-work")) {
+ * // Access the auto-created context if needed
+ * StatusContext context = scope.getContext();
+ * context.addSink(new ConsoleLoggerSink());
+ *
+ * // Create tasks directly
+ * StatusTracker tracker = scope.trackTask(new Task());
+ * tracker.getTracked().execute();
+ * }
+ * }
+ *
+ * Completion Semantics:
+ * A scope is considered "complete" when all its children (both nested scopes and tasks) are complete.
+ * This provides a natural aggregation of completion state without needing the scope itself to track progress.
+ *
+ * Thread Safety:
+ *
+ * - All public methods are thread-safe and can be called concurrently
+ * - Child management: Uses {@link CopyOnWriteArrayList} for thread-safe child collections
+ * - Concurrent scope creation: Multiple threads can create child scopes simultaneously
+ * - Concurrent tracker creation: Multiple threads can create task trackers simultaneously
+ * - Completion checking: {@link #isComplete()} is safe to call while children are being added/closed
+ * - Close operation: Idempotent and safe to call from any thread
+ * - Parent-child relationships: Maintained atomically via CopyOnWriteArrayList operations
+ *
+ *
+ * @see StatusTracker
+ * @see StatusContext
+ * @since 4.0.0
+ */
+public final class StatusScope implements AutoCloseable {
+
+ private final StatusContext context;
+ private final StatusScope parent;
+ private final String name;
+ private final List childScopes = new CopyOnWriteArrayList<>();
+ private final List> childTasks = new CopyOnWriteArrayList<>();
+ private volatile boolean closed = false;
+ private final boolean ownsContext;
+
+ /**
+ * Creates a root StatusScope with its own default StatusContext.
+ * The context will be automatically closed when this scope is closed.
+ * This is useful for simple cases where you don't need to configure
+ * the context explicitly.
+ *
+ * @param name the name of the scope (also used as the context name)
+ */
+ public StatusScope(String name) {
+ this.context = new StatusContext(name);
+ this.parent = null;
+ this.name = name;
+ this.ownsContext = true;
+ context.registerScope(this);
+ }
+
+ /**
+ * Package-private constructor. Use {@link StatusContext#createScope(String)} or
+ * {@link #createChildScope(String)} to create scopes within an existing context.
+ */
+ StatusScope(StatusContext context, StatusScope parent, String name) {
+ this.context = context;
+ this.parent = parent;
+ this.name = name;
+ this.ownsContext = false;
+
+ if (parent != null) {
+ parent.childScopes.add(this);
+ }
+ }
+
+ /**
+ * Creates a nested organizational scope under this scope.
+ *
+ * @param name the name of the child scope
+ * @return a new StatusScope as a child of this scope
+ */
+ public StatusScope createChildScope(String name) {
+ checkNotClosed();
+ return context.createChildScope(this, name);
+ }
+
+ /**
+ * Creates a task tracker for an object implementing {@link StatusSource}.
+ * The task becomes a leaf node under this scope.
+ *
+ * @param tracked the object to track
+ * @param the type of object being tracked
+ * @return a new StatusTracker as a child of this scope
+ */
+ public > StatusTracker trackTask(U tracked) {
+ checkNotClosed();
+ StatusTracker tracker = context.trackInScope(this, tracked);
+ childTasks.add(tracker);
+ return tracker;
+ }
+
+ /**
+ * Creates a task tracker using a custom status function.
+ *
+ * @param tracked the object to track
+ * @param statusFunction function to extract status from the tracked object
+ * @param the type of object being tracked
+ * @return a new StatusTracker as a child of this scope
+ */
+ public StatusTracker trackTask(T tracked, Function> statusFunction) {
+ checkNotClosed();
+ StatusTracker tracker = context.trackInScope(this, tracked, statusFunction);
+ childTasks.add(tracker);
+ return tracker;
+ }
+
+ /**
+ * Returns the name of this scope.
+ *
+ * @return the scope name
+ */
+ public String getName() {
+ return name;
+ }
+
+ /**
+ * Returns the parent scope, or null if this is a root scope.
+ *
+ * @return the parent scope or null
+ */
+ public StatusScope getParent() {
+ return parent;
+ }
+
+ /**
+ * Returns the context that owns this scope.
+ * For scopes created with the default constructor, this returns
+ * the auto-created context.
+ *
+ * @return the owning context
+ */
+ public StatusContext getContext() {
+ return context;
+ }
+
+ /**
+ * Returns a snapshot of all child scopes.
+ *
+ * @return an immutable list of child scopes
+ */
+ public List getChildScopes() {
+ return new ArrayList<>(childScopes);
+ }
+
+ /**
+ * Returns a snapshot of all child tasks.
+ *
+ * @return an immutable list of child task trackers
+ */
+ public List> getChildTasks() {
+ return new ArrayList<>(childTasks);
+ }
+
+ /**
+ * Returns whether all children (scopes and tasks) are complete.
+ * A scope is complete when:
+ *
+ * - All child scopes are complete, AND
+ * - All child tasks are finished (have finishTime > 0)
+ *
+ *
+ * @return true if this scope and all descendants are complete
+ */
+ public boolean isComplete() {
+ // Check all child scopes are complete
+ for (StatusScope childScope : childScopes) {
+ if (!childScope.isComplete()) {
+ return false;
+ }
+ }
+
+ // Check all child tasks are finished
+ for (StatusTracker> task : childTasks) {
+ if (!task.isClosed()) {
+ return false;
+ }
+ }
+
+ return true;
+ }
+
+ /**
+ * Returns whether this scope has been closed.
+ *
+ * @return true if closed, false otherwise
+ */
+ public boolean isClosed() {
+ return closed;
+ }
+
+ /**
+ * Closes this scope, removing it from its parent's child list.
+ * If this scope owns its context (created via the default constructor),
+ * the context will also be closed, which closes all associated resources.
+ * Child scopes and tasks are not automatically closed.
+ */
+ @Override
+ public void close() {
+ if (closed) {
+ return;
+ }
+ closed = true;
+
+ if (parent != null) {
+ parent.childScopes.remove(this);
+ }
+
+ context.onScopeClosed(this);
+
+ // If we own the context, close it as well
+ if (ownsContext) {
+ context.close();
+ }
+ }
+
+ /**
+ * Called by child tracker when it closes to remove itself from this scope's child list.
+ */
+ void removeChildTask(StatusTracker> tracker) {
+ childTasks.remove(tracker);
+ }
+
+ /**
+ * Package-private method to add a tracker to this scope's child list.
+ * Used by StatusTracker's public constructor when creating a standalone tracker.
+ */
+ void addChildTask(StatusTracker> tracker) {
+ childTasks.add(tracker);
+ }
+
+ private void checkNotClosed() {
+ if (closed) {
+ throw new IllegalStateException("StatusScope '" + name + "' has been closed");
+ }
+ }
+
+ /**
+ * Returns a summary string showing the scope name and task counts.
+ * Format: "ScopeName (active: N, done: M)"
+ */
+ @Override
+ public String toString() {
+ long active = childTasks.stream().filter(t -> !t.isClosed()).count();
+ long done = childTasks.stream().filter(StatusTracker::isClosed).count();
+
+ // Recursively count from child scopes
+ for (StatusScope childScope : childScopes) {
+ active += childScope.childTasks.stream().filter(t -> !t.isClosed()).count();
+ done += childScope.childTasks.stream().filter(StatusTracker::isClosed).count();
+ }
+
+ if (active == 0 && done == 0) {
+ return name; // No tasks yet
+ }
+ return String.format("%s (active: %d, done: %d)", name, active, done);
+ }
+}
diff --git a/internal-apis/src/main/java/io/github/jbellis/jvector/status/StatusTracker.java b/internal-apis/src/main/java/io/github/jbellis/jvector/status/StatusTracker.java
new file mode 100644
index 000000000..bb5dd081c
--- /dev/null
+++ b/internal-apis/src/main/java/io/github/jbellis/jvector/status/StatusTracker.java
@@ -0,0 +1,461 @@
+/*
+ * Copyright DataStax, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package io.github.jbellis.jvector.status;
+
+import io.github.jbellis.jvector.status.eventing.StatusSink;
+import io.github.jbellis.jvector.status.eventing.StatusSource;
+import io.github.jbellis.jvector.status.eventing.StatusUpdate;
+
+import java.lang.reflect.Method;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.function.Function;
+
+/**
+ * Represents a leaf node in the task tracking hierarchy managed by a {@link StatusContext}.
+ * Trackers are responsible for observing actual work units and reporting their progress and state.
+ * Unlike {@link StatusScope}, trackers have progress/state and cannot have children.
+ *
+ * Key Responsibilities:
+ *
+ * - Observation: Periodically calls the status function to observe the tracked object
+ * via {@link #refreshAndGetStatus()}, which is invoked by {@link StatusMonitor}
+ * - Caching: Stores the most recent status observation for query and timing purposes
+ * without requiring re-observation of the tracked object
+ * - Timing: Tracks task execution timing including start time, running duration,
+ * and accumulated running time across multiple RUNNING states
+ * - Scope Membership: Optionally belongs to a {@link StatusScope} for
+ * organizational hierarchy
+ *
+ *
+ * Architectural Flow:
+ *
+ * - {@link StatusMonitor} periodically calls {@link #refreshAndGetStatus()} on this tracker
+ * - Tracker invokes its status function to observe the tracked object
+ * - Tracker caches the observed status and updates timing information
+ * - {@link StatusContext} receives the status and routes it to all registered {@link StatusSink}s
+ * - Status flows unidirectionally: Task β Tracker β Monitor β Context β Sinks
+ *
+ *
+ * Usage Pattern:
+ * {@code
+ * try (StatusContext context = new StatusContext("operation");
+ * StatusScope scope = context.createScope("DataProcessing")) {
+ *
+ * // Create trackers as leaf nodes
+ * StatusTracker loader = scope.trackTask(new LoadTask());
+ * StatusTracker processor = scope.trackTask(new ProcessTask());
+ *
+ * // Trackers report progress automatically
+ * // Cannot create children - use scopes for hierarchy
+ * }
+ * }
+ *
+ * Thread Safety:
+ *
+ * - Status observation: {@link #refreshAndGetStatus()} is called only by the {@link StatusMonitor}
+ * thread, ensuring single-threaded access to the status function
+ * - Cached status: Read via volatile field, safe for concurrent access by multiple threads
+ * - Timing data: Protected by internal synchronization lock ({@code timingLock}), which is
+ * guaranteed non-null (final field initialized inline). All timing mutations occur within synchronized blocks,
+ * preventing race conditions between the monitor thread and close operations
+ * - Public methods: {@link #getStatus()}, {@link #getTracked()}, {@link #close()} are thread-safe
+ * - Close operation: Idempotent and safe to call from any thread. Uses {@code timingLock}
+ * synchronization to safely coordinate with concurrent monitor polling
+ *
+ *
+ * This class should not be instantiated directly. Use {@link StatusContext#track} methods
+ * or {@link StatusScope#trackTask} methods to create trackers.
+ *
+ * @param the type of object being tracked
+ * @see StatusScope
+ * @see StatusContext
+ * @see StatusMonitor
+ * @see StatusSink
+ * @see StatusUpdate
+ * @since 4.0.0
+ */
+public final class StatusTracker implements AutoCloseable {
+
+ private final StatusContext context;
+ private final StatusScope parentScope;
+ private final Function> statusFunction;
+ private final T tracked;
+ private final boolean ownsScope;
+
+ private volatile boolean closed = false;
+ private volatile StatusUpdate lastStatus;
+
+ // Synchronization lock for timing data. Guaranteed non-null (final, initialized inline).
+ // Protects timing fields from concurrent access by the StatusMonitor thread and close() calls.
+ private final Object timingLock = new Object();
+ private volatile Long runningStartTime;
+ private volatile Long firstRunningStartTime;
+ private volatile long accumulatedRunTimeMillis;
+
+ /**
+ * Creates a standalone tracker with its own default StatusScope and StatusContext.
+ * This is the simplest way to track a task when you don't need to configure
+ * the context or organize multiple tasks.
+ *
+ * Example usage:
+ *
{@code
+ * try (StatusTracker tracker = new StatusTracker<>(new MyTask())) {
+ * // Add a sink to see progress
+ * tracker.getContext().addSink(new ConsoleLoggerSink());
+ *
+ * // Execute the task
+ * tracker.getTracked().execute();
+ * }
+ * }
+ *
+ * @param tracked the task implementing StatusSource to track
+ */
+ public StatusTracker(T tracked) {
+ this(tracked, null);
+ }
+
+ /**
+ * Creates a standalone tracker with a custom status function, using its own
+ * default StatusScope and StatusContext.
+ *
+ * @param tracked the object to track
+ * @param statusFunction function to extract status from the tracked object (null to use StatusSource interface)
+ */
+ @SuppressWarnings("unchecked")
+ public StatusTracker(T tracked, Function> statusFunction) {
+ this.tracked = Objects.requireNonNull(tracked, "tracked");
+
+ // Determine status function
+ if (statusFunction == null) {
+ if (!(tracked instanceof StatusSource)) {
+ throw new IllegalArgumentException("Task must implement StatusSource or provide a statusFunction");
+ }
+ StatusSource> source = (StatusSource>) tracked;
+ this.statusFunction = t -> (StatusUpdate) source.getTaskStatus();
+ } else {
+ this.statusFunction = statusFunction;
+ }
+
+ // Create our own scope (which creates its own context)
+ String taskName = extractTaskNameFromObject(tracked);
+ this.parentScope = new StatusScope("tracker-" + taskName);
+ this.context = parentScope.getContext();
+ this.ownsScope = true;
+
+ // Register ourselves with the scope
+ parentScope.addChildTask(this);
+
+ // Register with monitor
+ StatusUpdate initial = refreshAndGetStatus();
+ context.registerTracker(this, initial);
+ }
+
+ StatusTracker(StatusContext context,
+ StatusScope parentScope,
+ T tracked,
+ Function> statusFunction) {
+ this.context = Objects.requireNonNull(context, "context");
+ this.tracked = Objects.requireNonNull(tracked, "tracked");
+ this.statusFunction = Objects.requireNonNull(statusFunction, "statusFunction");
+
+ // If no parent scope provided, create a default one
+ if (parentScope == null) {
+ String taskName = extractTaskNameFromObject(tracked);
+ this.parentScope = context.createScope("auto-scope-" + taskName);
+ this.ownsScope = true;
+ } else {
+ this.parentScope = parentScope;
+ this.ownsScope = false;
+ }
+ // Note: parentScope relationship is managed by StatusScope.trackTask() or auto-created above
+ }
+
+ /**
+ * Helper method to extract task name from an object (non-static version for constructor use).
+ */
+ private static String extractTaskNameFromObject(Object tracked) {
+ try {
+ Method getNameMethod = tracked.getClass().getMethod("getName");
+ Object name = getNameMethod.invoke(tracked);
+ if (name instanceof String) {
+ return (String) name;
+ }
+ } catch (Exception ignored) {
+ // Fall back to toString
+ }
+ return tracked.toString();
+ }
+
+ /**
+ * Observes the tracked object by invoking the status function and caches the result.
+ * This method is called by {@link StatusMonitor} at configured poll intervals.
+ * The cached status is used for timing calculations and can be retrieved without
+ * re-observation via {@link #getLastStatus()}.
+ *
+ * @return the newly observed status from the tracked object
+ */
+ StatusUpdate refreshAndGetStatus() {
+ StatusUpdate status = statusFunction.apply(tracked);
+ this.lastStatus = status;
+ updateTiming(status);
+ return status;
+ }
+
+ /**
+ * Returns the last cached status without re-observing the tracked object.
+ * This method is used internally for efficient status retrieval when fresh
+ * observation is not required.
+ *
+ * @return the most recently cached status, or null if no status has been observed yet
+ */
+ StatusUpdate getLastStatus() {
+ return lastStatus;
+ }
+
+
+ /**
+ * Returns whether this tracker has been closed.
+ *
+ * @return true if {@link #close()} has been called, false otherwise
+ */
+ boolean isClosed() {
+ return closed;
+ }
+
+ /**
+ * Returns the object being tracked by this tracker.
+ *
+ * @return the tracked object
+ */
+ public T getTracked() {
+ return tracked;
+ }
+
+ /**
+ * Returns the current status of the tracked object. If no status has been
+ * cached yet, this method will perform an immediate observation by calling
+ * {@link #refreshAndGetStatus()}.
+ *
+ * @return the current status of the tracked object
+ */
+ public StatusUpdate getStatus() {
+ StatusUpdate current = lastStatus;
+ if (current == null) {
+ current = refreshAndGetStatus();
+ }
+ return current;
+ }
+
+ /**
+ * Returns the scope that contains this tracker. For trackers created with an explicit scope,
+ * returns that scope. For trackers created without a scope, returns the automatically
+ * created scope.
+ *
+ * @return the scope containing this tracker
+ */
+ public StatusScope getScope() {
+ return parentScope;
+ }
+
+ /**
+ * Returns the parent scope if this tracker belongs to a scope, or null otherwise.
+ * @deprecated Use {@link #getScope()} instead
+ *
+ * @return the parent scope, or null if this tracker doesn't belong to a scope
+ */
+ @Deprecated
+ public StatusScope getParentScope() {
+ return parentScope;
+ }
+
+ /**
+ * Closes this tracker, unregistering it from monitoring. This method is idempotent
+ * and safe to call multiple times.
+ *
+ * When closed:
+ *
+ * - The tracker performs a final status observation to capture the latest state
+ * - The tracker is unregistered from {@link StatusMonitor} (no more polling)
+ * - Running time is finalized for timing calculations
+ * - The tracker is removed from its parent scope (if any)
+ * - {@link StatusContext#onTrackerClosed} is invoked to complete cleanup and notify sinks
+ * - If the tracker owns its scope (auto-created), the scope is also closed
+ *
+ */
+ @Override
+ public void close() {
+ if (closed) {
+ return;
+ }
+ closed = true;
+
+ // Perform final status observation to capture the latest state before closing
+ refreshAndGetStatus();
+
+ finalizeRunningTime(System.currentTimeMillis());
+ context.onTrackerClosed(this);
+
+ // If we own the scope (it was auto-created), close it as well
+ if (ownsScope && parentScope != null) {
+ parentScope.close();
+ }
+ }
+
+ /**
+ * Returns the context that manages this tracker.
+ *
+ * @return the owning context
+ */
+ public StatusContext getContext() {
+ return context;
+ }
+
+ /**
+ * Returns the total accumulated running time for this tracker in milliseconds.
+ * This includes all time spent in the RUNNING state, even across multiple
+ * transitions to/from RUNNING. For tasks currently running, includes the
+ * time elapsed since the current RUNNING state began.
+ *
+ * @return accumulated running time in milliseconds
+ */
+ public long getElapsedRunningTime() {
+ synchronized (timingLock) {
+ long total = accumulatedRunTimeMillis;
+ if (runningStartTime != null) {
+ total += Math.max(0, System.currentTimeMillis() - runningStartTime);
+ }
+ return total;
+ }
+ }
+
+ /**
+ * Returns the timestamp when this tracker first entered the RUNNING state,
+ * or null if it has never been RUNNING.
+ *
+ * @return the first running start timestamp in milliseconds since epoch, or null
+ */
+ public Long getRunningStartTime() {
+ return firstRunningStartTime;
+ }
+
+ /**
+ * Updates timing information based on the current status. This method is called
+ * automatically by {@link #refreshAndGetStatus()} after observing the tracked object.
+ *
+ * Timing transitions:
+ *
+ * - PENDING β RUNNING: Records first and current running start times
+ * - RUNNING β SUCCESS/FAILED/CANCELLED: Finalizes accumulated running time
+ * - Other transitions: No timing changes
+ *
+ *
+ * @param status the observed status containing runstate information
+ */
+ private void updateTiming(StatusUpdate status) {
+ if (status == null || status.runstate == null) {
+ return;
+ }
+
+ synchronized (timingLock) {
+ switch (status.runstate) {
+ case RUNNING:
+ if (runningStartTime == null) {
+ runningStartTime = status.timestamp;
+ if (firstRunningStartTime == null) {
+ firstRunningStartTime = runningStartTime;
+ }
+ }
+ break;
+ case SUCCESS:
+ case FAILED:
+ case CANCELLED:
+ finalizeRunningTimeLocked(status.timestamp);
+ break;
+ default:
+ // No-op for PENDING and other non-running states
+ break;
+ }
+ }
+ }
+
+ private void finalizeRunningTime(long timestamp) {
+ synchronized (timingLock) {
+ finalizeRunningTimeLocked(timestamp);
+ }
+ }
+
+ private void finalizeRunningTimeLocked(long timestamp) {
+ if (runningStartTime == null) {
+ return;
+ }
+
+ accumulatedRunTimeMillis += Math.max(0, timestamp - runningStartTime);
+ runningStartTime = null;
+ }
+
+ /**
+ * Extracts a human-readable name from the tracked object.
+ * Attempts to call a getName() method via reflection, falling back to toString().
+ *
+ * @param tracker the tracker whose tracked object should be named
+ * @return the extracted name
+ */
+ public static String extractTaskName(StatusTracker> tracker) {
+ Object tracked = tracker.getTracked();
+ try {
+ Method getNameMethod = tracked.getClass().getMethod("getName");
+ Object name = getNameMethod.invoke(tracked);
+ if (name instanceof String) {
+ return (String) name;
+ }
+ } catch (Exception ignored) {
+ // Fall back to toString
+ }
+ return tracked.toString();
+ }
+
+ /**
+ * Returns a one-line summary of the tracker's current state.
+ * Format: "TaskName [progress%] state (elapsed time)"
+ *
+ * Example output:
+ *
+ * - "DataLoader [45.2%] RUNNING (1234ms)"
+ * - "DataProcessor [100.0%] SUCCESS (2567ms)"
+ * - "ValidationTask [0.0%] PENDING (0ms)"
+ *
+ *
+ * @return a formatted string summarizing the tracker's status
+ */
+ @Override
+ public String toString() {
+ StatusUpdate status = getStatus();
+ String taskName = extractTaskName(this);
+ double progressPercent = status.progress * 100.0;
+ long elapsed = getElapsedRunningTime();
+
+ return String.format("%s [%.1f%%] %s (%dms)",
+ taskName,
+ progressPercent,
+ status.runstate,
+ elapsed);
+ }
+}
diff --git a/internal-apis/src/main/java/io/github/jbellis/jvector/status/eventing/RunState.java b/internal-apis/src/main/java/io/github/jbellis/jvector/status/eventing/RunState.java
new file mode 100644
index 000000000..819248ae1
--- /dev/null
+++ b/internal-apis/src/main/java/io/github/jbellis/jvector/status/eventing/RunState.java
@@ -0,0 +1,99 @@
+/*
+ * Copyright DataStax, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package io.github.jbellis.jvector.status.eventing;
+
+/**
+ * Represents the execution state of a tracked task. This enum defines the standard
+ * lifecycle states that tasks transition through during execution.
+ *
+ * State Transitions:
+ *
+ * - PENDING β RUNNING: Task begins execution
+ * - RUNNING β SUCCESS: Task completes successfully
+ * - RUNNING β FAILED: Task encounters an error
+ * - RUNNING β CANCELLED: Task is explicitly cancelled
+ * - PENDING β CANCELLED: Task is cancelled before starting
+ *
+ *
+ * Terminal States: {@link #SUCCESS}, {@link #FAILED}, and {@link #CANCELLED} are
+ * terminal states - tasks should not transition from these states to any other state.
+ *
+ *
Usage in Status Updates:
+ *
{@code
+ * // Task starts
+ * return new StatusUpdate<>(0.0, RunState.PENDING, this);
+ *
+ * // Task begins execution
+ * return new StatusUpdate<>(0.0, RunState.RUNNING, this);
+ *
+ * // Task progresses
+ * return new StatusUpdate<>(0.5, RunState.RUNNING, this);
+ *
+ * // Task completes
+ * return new StatusUpdate<>(1.0, RunState.SUCCESS, this);
+ * }
+ *
+ * @see StatusUpdate
+ * @see StatusSource
+ * @since 4.0.0
+ */
+public enum RunState {
+ /**
+ * Task is queued or waiting to start execution.
+ * Progress is typically 0.0 in this state.
+ */
+ PENDING("β³"),
+
+ /**
+ * Task is actively executing.
+ * Progress typically increases from 0.0 towards 1.0.
+ */
+ RUNNING("π"),
+
+ /**
+ * Task completed successfully (terminal state).
+ * Progress should be 1.0 in this state.
+ */
+ SUCCESS("β
"),
+
+ /**
+ * Task failed due to an error (terminal state).
+ * Progress may be any value depending on when the failure occurred.
+ */
+ FAILED("β"),
+
+ /**
+ * Task was explicitly cancelled (terminal state).
+ * Progress may be any value depending on when cancellation occurred.
+ */
+ CANCELLED("π«");
+
+ private final String glyph;
+
+ RunState(String glyph) {
+ this.glyph = glyph;
+ }
+
+ /**
+ * Returns the Unicode glyph associated with this state for display purposes.
+ *
+ * @return a Unicode emoji character representing this state
+ */
+ public String getGlyph() {
+ return glyph;
+ }
+}
diff --git a/internal-apis/src/main/java/io/github/jbellis/jvector/status/eventing/StatusSink.java b/internal-apis/src/main/java/io/github/jbellis/jvector/status/eventing/StatusSink.java
new file mode 100644
index 000000000..172ba160d
--- /dev/null
+++ b/internal-apis/src/main/java/io/github/jbellis/jvector/status/eventing/StatusSink.java
@@ -0,0 +1,245 @@
+/*
+ * Copyright DataStax, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package io.github.jbellis.jvector.status.eventing;
+
+import io.github.jbellis.jvector.status.StatusContext;
+import io.github.jbellis.jvector.status.StatusTracker;
+import io.github.jbellis.jvector.status.StatusScope;
+import io.github.jbellis.jvector.status.sinks.ConsoleLoggerSink;
+import io.github.jbellis.jvector.status.sinks.LoggerStatusSink;
+import io.github.jbellis.jvector.status.sinks.MetricsStatusSink;
+import io.github.jbellis.jvector.status.sinks.NoopStatusSink;
+
+/**
+ * A contract for objects that receive task lifecycle events and status updates.
+ * TaskSink implementations define how task monitoring information is processed,
+ * stored, or displayed. This interface enables flexible monitoring strategies
+ * by allowing multiple sinks to observe the same tasks.
+ *
+ * The sink pattern allows for:
+ *
+ * - Multiple concurrent monitoring outputs (console, logs, metrics, etc.)
+ * - Pluggable monitoring strategies without changing task code
+ * - Centralized event processing and filtering
+ * - Integration with external monitoring systems
+ *
+ *
+ * Lifecycle Events
+ * Sinks receive three types of events in order:
+ *
+ * - taskStarted: Called once when task tracking begins
+ * - taskUpdate: Called zero or more times as task progresses
+ * - taskFinished: Called once when task tracking ends
+ *
+ *
+ * Implementation Examples:
+ *
+ * Basic Console Sink
+ * {@code
+ * public class SimpleConsoleSink implements TaskSink {
+ * @Override
+ * public void taskStarted(Tracker> task) {
+ * System.out.println("Started: " + getTaskName(task));
+ * }
+ *
+ * @Override
+ * public void taskUpdate(Tracker> task, TaskStatus> status) {
+ * System.out.printf("Update: %s [%.1f%%] %s%n",
+ * getTaskName(task), status.progress * 100, status.runstate);
+ * }
+ *
+ * @Override
+ * public void taskFinished(Tracker> task) {
+ * System.out.println("Finished: " + getTaskName(task));
+ * }
+ *
+ * private String getTaskName(Tracker> task) {
+ * return task.getTracked().toString();
+ * }
+ * }
+ * }
+ *
+ * Filtering Sink
+ * {@code
+ * public class FilteringSink implements TaskSink {
+ * private final TaskSink delegate;
+ * private final Predicate> filter;
+ *
+ * public FilteringSink(TaskSink delegate, Predicate> filter) {
+ * this.delegate = delegate;
+ * this.filter = filter;
+ * }
+ *
+ * @Override
+ * public void taskStarted(Tracker> task) {
+ * if (filter.test(task)) {
+ * delegate.taskStarted(task);
+ * }
+ * }
+ *
+ * @Override
+ * public void taskUpdate(Tracker> task, TaskStatus> status) {
+ * if (filter.test(task) && status.progress > 0.1) { // Only > 10%
+ * delegate.taskUpdate(task, status);
+ * }
+ * }
+ *
+ * @Override
+ * public void taskFinished(Tracker> task) {
+ * if (filter.test(task)) {
+ * delegate.taskFinished(task);
+ * }
+ * }
+ * }
+ * }
+ *
+ * Composite Sink
+ * {@code
+ * public class CompositeSink implements TaskSink {
+ * private final List sinks;
+ *
+ * public CompositeSink(TaskSink... sinks) {
+ * this.sinks = Arrays.asList(sinks);
+ * }
+ *
+ * @Override
+ * public void taskStarted(Tracker> task) {
+ * sinks.forEach(sink -> {
+ * try {
+ * sink.taskStarted(task);
+ * } catch (Exception e) {
+ * handleSinkError(sink, e);
+ * }
+ * });
+ * }
+ *
+ * // Similar for taskUpdate and taskFinished...
+ * }
+ * }
+ *
+ * Built-in Implementations
+ * The framework provides several ready-to-use sink implementations:
+ *
+ * - {@link ConsoleLoggerSink} - Human-readable console output
+ * - {@link LoggerStatusSink} - Java logging integration
+ * - {@link MetricsStatusSink} - Performance metrics collection
+ * - {@link NoopStatusSink} - No-operation for testing
+ *
+ *
+ * Integration Patterns
+ *
+ * StatusContext Configuration
+ * {@code
+ * StatusContext context = new StatusContext("data-processing");
+ * context.addSink(new ConsoleTaskSink());
+ * context.addSink(new MetricsTaskSink());
+ * context.addSink(new LoggerTaskSink("app.tasks"));
+ * }
+ *
+ * Direct Tracker Usage
+ * {@code
+ * List sinks = Arrays.asList(
+ * new ConsoleTaskSink(),
+ * new MetricsTaskSink()
+ * );
+ * try (Tracker tracker = Tracker.withInstrumented(task, sinks)) {
+ * task.execute();
+ * }
+ * }
+ *
+ * Error Handling
+ * Sink implementations should handle errors gracefully:
+ *
+ * - Catch and log exceptions rather than propagating them
+ * - Continue processing even if individual operations fail
+ * - Provide fallback behavior for resource unavailability
+ * - Consider circuit breaker patterns for external integrations
+ *
+ *
+ * Thread Safety
+ * Sink implementations must be thread-safe as they may receive concurrent
+ * calls from multiple tracker threads. Use appropriate synchronization or
+ * concurrent data structures as needed.
+ *
+ * Performance Considerations
+ * Since sink methods are called on the monitoring thread:
+ *
+ * - Keep method implementations fast to avoid blocking task monitoring
+ * - Use asynchronous processing for expensive operations (I/O, network)
+ * - Consider batching updates for high-frequency scenarios
+ * - Implement back-pressure mechanisms if needed
+ *
+ *
+ * @see StatusTracker
+ * @see StatusUpdate
+ * @see StatusContext
+ * @since 4.0.0
+ */
+public interface StatusSink {
+ /**
+ * Called when task tracking begins.
+ * This is the first method called for any tracked task.
+ *
+ * @param task the tracker instance that started monitoring
+ */
+ void taskStarted(StatusTracker> task);
+
+ /**
+ * Called when the task status changes.
+ * This method may be called zero or more times during task execution.
+ *
+ * @param task the tracker instance reporting the update
+ * @param status the current task status with progress and state information
+ */
+ void taskUpdate(StatusTracker> task, StatusUpdate> status);
+
+ /**
+ * Called when task tracking ends.
+ * This is the final method called for any tracked task.
+ *
+ * @param task the tracker instance that finished monitoring
+ */
+ void taskFinished(StatusTracker> task);
+
+ /**
+ * Called when a scope starts. Scopes provide hierarchical organization
+ * of related tasks without having their own progress or state.
+ *
+ * This method has a default no-op implementation to maintain backward
+ * compatibility with sinks that don't need scope notifications.
+ *
+ * @param scope the scope that started
+ * @since 4.0.0
+ */
+ default void scopeStarted(StatusScope scope) {
+ // No-op by default - sinks can override to handle scope events
+ }
+
+ /**
+ * Called when a scope finishes. This is called after all tasks and
+ * child scopes within this scope have completed.
+ *
+ * This method has a default no-op implementation to maintain backward
+ * compatibility with sinks that don't need scope notifications.
+ *
+ * @param scope the scope that finished
+ * @since 4.0.0
+ */
+ default void scopeFinished(StatusScope scope) {
+ // No-op by default - sinks can override to handle scope events
+ }
+}
diff --git a/internal-apis/src/main/java/io/github/jbellis/jvector/status/eventing/StatusSource.java b/internal-apis/src/main/java/io/github/jbellis/jvector/status/eventing/StatusSource.java
new file mode 100644
index 000000000..2da290ab4
--- /dev/null
+++ b/internal-apis/src/main/java/io/github/jbellis/jvector/status/eventing/StatusSource.java
@@ -0,0 +1,87 @@
+/*
+ * Copyright DataStax, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package io.github.jbellis.jvector.status.eventing;
+
+import io.github.jbellis.jvector.status.StatusContext;
+import io.github.jbellis.jvector.status.StatusTracker;
+
+/**
+ * Interface for objects that can provide their own status updates without requiring
+ * a custom status function. Objects implementing this interface can be tracked directly
+ * by passing them to {@link StatusContext#track(StatusSource)}
+ * methods.
+ *
+ *
Usage Example:
+ *
{@code
+ * public class MyTask implements StatusSource {
+ * private volatile double progress = 0.0;
+ * private volatile RunState state = RunState.PENDING;
+ *
+ * @Override
+ * public StatusUpdate getTaskStatus() {
+ * return new StatusUpdate<>(progress, state, this);
+ * }
+ *
+ * public void execute() {
+ * state = RunState.RUNNING;
+ * // ... do work, updating progress ...
+ * progress = 1.0;
+ * state = RunState.SUCCESS;
+ * }
+ * }
+ *
+ * // Track the task
+ * try (StatusContext context = new StatusContext("my-operation");
+ * StatusTracker tracker = context.track(myTask)) {
+ * myTask.execute();
+ * }
+ * }
+ *
+ * Thread Safety: Implementations should ensure that {@link #getTaskStatus()}
+ * can be safely called concurrently from the monitoring thread while the task is executing.
+ * Use appropriate synchronization or volatile fields for progress and state.
+ *
+ * @param the type of the object providing status (must be the implementing class itself)
+ * @see StatusUpdate
+ * @see RunState
+ * @see StatusContext#track(StatusSource)
+ * @since 4.0.0
+ */
+public interface StatusSource> {
+ /**
+ * Returns the current status of this object. This method is called periodically by
+ * the monitoring framework to observe the object's state.
+ *
+ * Implementations should return a new {@link StatusUpdate} containing:
+ *
+ * - Current progress value (0.0 to 1.0)
+ * - Current {@link RunState}
+ * - Reference to this object (typically {@code this})
+ *
+ *
+ * Example implementation:
+ *
{@code
+ * @Override
+ * public StatusUpdate getTaskStatus() {
+ * return new StatusUpdate<>(currentProgress, currentState, this);
+ * }
+ * }
+ *
+ * @return the current status including progress, state, and tracked object reference
+ */
+ StatusUpdate getTaskStatus();
+}
diff --git a/internal-apis/src/main/java/io/github/jbellis/jvector/status/eventing/StatusUpdate.java b/internal-apis/src/main/java/io/github/jbellis/jvector/status/eventing/StatusUpdate.java
new file mode 100644
index 000000000..3274bd1b7
--- /dev/null
+++ b/internal-apis/src/main/java/io/github/jbellis/jvector/status/eventing/StatusUpdate.java
@@ -0,0 +1,173 @@
+/*
+ * Copyright DataStax, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package io.github.jbellis.jvector.status.eventing;
+
+import io.github.jbellis.jvector.status.StatusContext;
+import io.github.jbellis.jvector.status.StatusTracker;
+
+/**
+ * Represents the current status of a tracked task at a specific point in time.
+ * This immutable snapshot contains the task's progress percentage, execution state,
+ * timestamp when the status was created, and optionally a reference to the tracked object.
+ *
+ * StatusUpdate objects are created by monitoring systems to report task progress
+ * and are passed to {@link StatusSink} implementations for processing. Each status
+ * represents a moment-in-time view of task execution and should be treated as immutable.
+ *
+ *
Usage Examples:
+ *
+ * Creating Status Updates
+ * {@code
+ * // For tasks implementing TaskStatus.Provider
+ * public class DataProcessor implements TaskStatus.Provider {
+ * private volatile double progress = 0.0;
+ * private volatile RunState state = RunState.PENDING;
+ *
+ * @Override
+ * public TaskStatus getTaskStatus() {
+ * return new TaskStatus<>(progress, state);
+ * }
+ *
+ * public void processData() {
+ * state = RunState.RUNNING;
+ * for (int i = 0; i < totalItems; i++) {
+ * processItem(i);
+ * progress = (double) i / totalItems;
+ * }
+ * state = RunState.SUCCESS;
+ * progress = 1.0;
+ * }
+ * }
+ * }
+ *
+ * Custom Status Functions
+ * {@code
+ * // For tasks that don't implement the Provider interface
+ * Function> statusFunction = job -> {
+ * RunState state;
+ * if (job.isComplete()) {
+ * state = job.hasErrors() ? RunState.FAILED : RunState.SUCCESS;
+ * } else if (job.isStarted()) {
+ * state = RunState.RUNNING;
+ * } else {
+ * state = RunState.PENDING;
+ * }
+ *
+ * return new TaskStatus<>(job.getCompletionRatio(), state);
+ * };
+ * }
+ *
+ * Status Processing in Sinks
+ * {@code
+ * public class CustomSink implements TaskSink {
+ * @Override
+ * public void taskUpdate(Tracker> task, TaskStatus> status) {
+ * switch (status.runstate) {
+ * case RUNNING:
+ * if (status.progress > 0.5) {
+ * logger.info("Task {} is halfway complete", getTaskName(task));
+ * }
+ * break;
+ * case FAILED:
+ * logger.error("Task {} failed at {}% completion",
+ * getTaskName(task), status.progress * 100);
+ * break;
+ * case SUCCESS:
+ * logger.info("Task {} completed successfully", getTaskName(task));
+ * break;
+ * }
+ * }
+ * }
+ * }
+ *
+ * Progress Values
+ * Progress is represented as a double value between 0.0 and 1.0:
+ *
+ * - 0.0: Task not started or 0% complete
+ * - 0.5: Task is 50% complete
+ * - 1.0: Task is 100% complete
+ *
+ *
+ * Progress values should be monotonically increasing during normal execution,
+ * though this is not enforced by the framework.
+ *
+ * RunState Values
+ * The {@link RunState} enum represents the execution phase:
+ *
+ * - PENDING: Task is queued but not yet started
+ * - RUNNING: Task is actively executing
+ * - SUCCESS: Task completed successfully
+ * - FAILED: Task completed with errors
+ * - CANCELLED: Task was cancelled before completion
+ *
+ *
+ * Provider Interface
+ * Objects that can provide their own status should implement the
+ * {@link StatusSource} interface. This enables automatic status polling
+ * without requiring custom status functions:
+ *
+ * {@code
+ * public class InstrumentedTask implements TaskStatus.Provider {
+ * @Override
+ * public TaskStatus getTaskStatus() {
+ * return new TaskStatus<>(getCurrentProgress(), getCurrentState());
+ * }
+ * }
+ * }
+ *
+ * Thread Safety
+ * TaskStatus objects are immutable after construction and are thread-safe.
+ * However, the underlying task objects that provide status may require their
+ * own synchronization for thread-safe access to progress and state fields.
+ *
+ * @param the type of task being tracked
+ * @see StatusTracker
+ * @see StatusSink
+ * @see RunState
+ * @since 4.0.0
+ */
+public class StatusUpdate {
+ public final double progress;
+ public final RunState runstate;
+ public final long timestamp;
+ public final T tracked;
+
+ /**
+ * Creates a StatusUpdate with the current timestamp and no tracked object reference.
+ *
+ * @param progress the task's progress (0.0 to 1.0)
+ * @param runstate the task's current execution state
+ */
+ public StatusUpdate(double progress, RunState runstate) {
+ this(progress, runstate, null);
+ }
+
+ /**
+ * Creates a StatusUpdate with the current timestamp and a reference to the tracked object.
+ *
+ * @param progress the task's progress (0.0 to 1.0)
+ * @param runstate the task's current execution state
+ * @param tracked the object being tracked (may be null)
+ */
+ public StatusUpdate(double progress, RunState runstate, T tracked) {
+ this.progress = progress;
+ this.runstate = runstate;
+ this.timestamp = System.currentTimeMillis();
+ this.tracked = tracked;
+ }
+
+}
diff --git a/internal-apis/src/main/java/io/github/jbellis/jvector/status/package-info.java b/internal-apis/src/main/java/io/github/jbellis/jvector/status/package-info.java
new file mode 100644
index 000000000..9c2f2f8d4
--- /dev/null
+++ b/internal-apis/src/main/java/io/github/jbellis/jvector/status/package-info.java
@@ -0,0 +1,182 @@
+/*
+ * Copyright DataStax, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * A comprehensive task tracking and monitoring framework designed for concurrent applications.
+ * This package provides live status tracking capabilities with flexible monitoring strategies,
+ * hierarchical task organization, and pluggable output systems.
+ *
+ * Design Goals
+ * The framework is built around several core principles:
+ *
+ * - Real-time Feedback: Enable tactile user feedback on the status of ongoing tasks
+ * through multiple output channels (console, logging, metrics)
+ * - Performance Monitoring: Provide efficient instrumentation for internal JVector use
+ * with minimal overhead and comprehensive task lifecycle tracking
+ * - Hierarchical Organization: Support complex task structures including batch processing,
+ * nested task groups, and DAG-based execution patterns
+ * - Pluggable Architecture: Enable consumer-agnostic monitoring through flexible sink
+ * implementations and configurable scope management
+ *
+ *
+ * Core Architecture
+ *
+ * Primary Components
+ *
+ * - {@link io.github.jbellis.jvector.status.StatusTracker} - The main monitoring wrapper
+ * that provides background polling and sink notification for any task object
+ * - {@link io.github.jbellis.jvector.status.StatusContext} - Configuration and
+ * lifecycle management for creating tracker instances; also owns the monitoring threads
+ * - {@link io.github.jbellis.jvector.status.eventing.StatusSink} - Processing interface for
+ * task lifecycle events, enabling multiple concurrent monitoring strategies
+ * - {@link io.github.jbellis.jvector.status.StatusMonitor} - Background polling engine
+ * that continuously monitors task status and detects changes
+ *
+ *
+ * Supporting Types
+ *
+ * - {@link io.github.jbellis.jvector.status.eventing.StatusUpdate} - Immutable status
+ * snapshots containing progress, state, and timestamp information for monitoring
+ *
+ *
+ * Built-in Implementations
+ *
+ * Monitoring Sinks
+ *
+ * - {@link io.github.jbellis.jvector.status.sinks.ConsoleLoggerSink} - Human-readable
+ * console output with progress bars and configurable formatting
+ * - {@link io.github.jbellis.jvector.status.sinks.LoggerStatusSink} - Java logging
+ * framework integration with configurable levels and hierarchies
+ * - {@link io.github.jbellis.jvector.status.sinks.MetricsStatusSink} - Performance
+ * metrics collection with detailed statistics and reporting
+ * - {@link io.github.jbellis.jvector.status.sinks.NoopStatusSink} - Zero-overhead
+ * no-operation sink for testing and performance-critical scenarios
+ *
+ *
+ * Essential Features
+ *
+ * Lifecycle Consistency
+ * The framework ensures tracking state never goes stale with respect to tracked tasks:
+ *
+ * - Automatic resource cleanup through {@link java.lang.AutoCloseable} integration
+ * - Thread-safe lifecycle management with atomic state transitions
+ * - Proper cleanup on task completion, failure, or cancellation
+ * - Background thread management with daemon thread patterns
+ *
+ *
+ * Hierarchical Task Support
+ * Support for complex task structures beyond single task awareness:
+ *
+ * - Nested task groups with inherited configuration and sink management
+ * - Batch processing patterns with concurrent task execution
+ * - DAG (Directed Acyclic Graph) compatibility for complex workflows
+ * - Scope-based configuration inheritance with override capabilities
+ *
+ *
+ * Consumer-Agnostic Design
+ * Flexible monitoring strategies through indirect observer wiring:
+ *
+ * - Multiple concurrent monitoring outputs (console, logs, metrics, external systems)
+ * - Pluggable sink architecture for custom monitoring implementations
+ * - Configuration-driven monitoring behavior without code changes
+ * - Environment-specific monitoring strategies (dev, staging, production)
+ *
+ *
+ * Advanced Features
+ *
+ * Performance Optimization
+ *
+ * - Configurable polling intervals to balance responsiveness and overhead
+ * - Lock-free data structures for high-concurrency scenarios
+ * - Memory-efficient design with minimal per-task overhead
+ * - Zero-cost abstractions when monitoring is disabled
+ *
+ *
+ * Integration Flexibility
+ *
+ * - Support for both instrumented objects (implementing TaskStatus.Provider)
+ * and custom status extraction functions
+ * - Dynamic sink addition/removal during task execution
+ * - Compatibility with existing task frameworks and execution models
+ * - Minimal intrusion on existing codebases
+ *
+ *
+ * Monitoring Capabilities
+ *
+ * - Real-time progress tracking with customizable update frequencies
+ * - Automatic completion detection and monitoring termination
+ * - Exception handling and error recovery in monitoring code
+ * - Performance metrics collection and analysis
+ *
+ *
+ * Usage Patterns
+ *
+ * Simple Task Monitoring
+ * {@code
+ * // Basic tracking with console output
+ * try (Tracker tracker = Tracker.withInstrumented(task, new ConsoleTaskSink())) {
+ * task.execute();
+ * } // Automatic cleanup
+ * }
+ *
+ * Context-Based Configuration
+ * {@code
+ * // Hierarchical configuration with multiple sinks
+ * StatusContext context = new StatusContext("data-processing");
+ * context.addSink(new ConsoleTaskSink());
+ * context.addSink(new MetricsTaskSink());
+ *
+ * try (StatusTracker tracker = context.track(processor)) {
+ * processor.processLargeDataset();
+ * }
+ * }
+ *
+ * Batch Processing
+ * {@code
+ * // Multiple concurrent tasks with scope management
+ * try (StatusContext context = new StatusContext("batch-job")) {
+ * List> trackers = createTasks(context);
+ *
+ * CompletableFuture.allOf(
+ * trackers.stream()
+ * .map(tracker -> CompletableFuture.runAsync(() -> processTask(tracker)))
+ * .toArray(CompletableFuture[]::new)
+ * ).join();
+ * } // All trackers automatically cleaned up
+ * }
+ *
+ * Thread Safety
+ * All components in this package are designed for thread-safe operation:
+ *
+ * - Concurrent access to task collections and monitoring state
+ * - Atomic operations for lifecycle management
+ * - Proper synchronization in sink implementations
+ * - Safe cleanup during concurrent task execution
+ *
+ *
+ * @since 4.0.0
+ * @see io.github.jbellis.jvector.status.StatusTracker
+ * @see io.github.jbellis.jvector.status.StatusContext
+ * @see io.github.jbellis.jvector.status.eventing.StatusSink
+ */
+package io.github.jbellis.jvector.status;
+
+import io.github.jbellis.jvector.status.eventing.StatusSink;
+import io.github.jbellis.jvector.status.eventing.StatusUpdate;
+import io.github.jbellis.jvector.status.sinks.ConsoleLoggerSink;
+import io.github.jbellis.jvector.status.sinks.LoggerStatusSink;
+import io.github.jbellis.jvector.status.sinks.MetricsStatusSink;
+import io.github.jbellis.jvector.status.sinks.NoopStatusSink;
diff --git a/internal-apis/src/main/java/io/github/jbellis/jvector/status/sinks/ConsoleLoggerSink.java b/internal-apis/src/main/java/io/github/jbellis/jvector/status/sinks/ConsoleLoggerSink.java
new file mode 100644
index 000000000..fa91bc05a
--- /dev/null
+++ b/internal-apis/src/main/java/io/github/jbellis/jvector/status/sinks/ConsoleLoggerSink.java
@@ -0,0 +1,222 @@
+/*
+ * Copyright DataStax, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package io.github.jbellis.jvector.status.sinks;
+
+import io.github.jbellis.jvector.status.*;
+import io.github.jbellis.jvector.status.eventing.StatusSink;
+import io.github.jbellis.jvector.status.eventing.StatusUpdate;
+
+import java.io.PrintStream;
+import java.time.LocalDateTime;
+import java.time.format.DateTimeFormatter;
+
+/**
+ * A human-readable task sink that outputs task progress and lifecycle events to the console
+ * or any PrintStream. This sink provides real-time visual feedback with customizable formatting
+ * including progress bars, timestamps, and status indicators.
+ *
+ * This sink provides:
+ *
+ * - Visual progress bars with Unicode block characters
+ * - Task lifecycle events (started, updates, finished)
+ * - Configurable timestamp display with millisecond precision
+ * - Flexible output destination (console, file, etc.)
+ * - Automatic task name extraction from various object types
+ *
+ *
+ * Usage Examples:
+ *
+ * Basic Console Output
+ * {@code
+ * // Default configuration: System.out, timestamps, progress bars
+ * TaskSink consoleSink = new ConsoleTaskSink();
+ *
+ * try (Tracker tracker = Tracker.withInstrumented(task, consoleSink)) {
+ * task.execute();
+ * // Output:
+ * // [14:32:15.123] βΆ Started: data-processing
+ * // [14:32:15.245] data-processing [ββββββββββββββββββββ] 60.0% - RUNNING
+ * // [14:32:16.891] β Finished: data-processing
+ * }
+ * }
+ *
+ * Custom PrintStream Output
+ * {@code
+ * // Output to a file or custom stream
+ * PrintStream fileOut = new PrintStream(new FileOutputStream("progress.log"));
+ * TaskSink fileSink = new ConsoleTaskSink(fileOut);
+ *
+ * try (Tracker tracker = Tracker.withInstrumented(job, fileSink)) {
+ * job.processBatch();
+ * } finally {
+ * fileOut.close();
+ * }
+ * }
+ *
+ * Minimal Output Format
+ * {@code
+ * // No timestamps, no progress bars - just basic text updates
+ * TaskSink simpleSink = new ConsoleTaskSink(System.out, false, false);
+ *
+ * try (Tracker tracker = Tracker.withInstrumented(task, simpleSink)) {
+ * task.importData();
+ * // Output:
+ * // βΆ Started: csv-import
+ * // csv-import [75.0%] - RUNNING
+ * // β Finished: csv-import
+ * }
+ * }
+ *
+ * Multiple Tasks with a Shared Context
+* {@code
+ * StatusContext context = new StatusContext("batch-operations");
+ * context.addSink(new ConsoleTaskSink());
+*
+ * // All tasks in scope will output to console
+ * try (StatusTracker t1 = context.track(task1);
+ * StatusTracker t2 = context.track(task2)) {
+ *
+ * CompletableFuture.allOf(
+ * CompletableFuture.runAsync(task1::execute),
+ * CompletableFuture.runAsync(task2::execute)
+ * ).join();
+ * }
+ * }
+ *
+ * Debugging and Development
+ * {@code
+ * // Use different streams for different priority levels
+ * TaskSink debugSink = new ConsoleTaskSink(System.err, true, true);
+ *
+ * try (Tracker tracker = Tracker.withInstrumented(debugTask, debugSink)) {
+ * // Debug output goes to stderr with full formatting
+ * }
+ * }
+ *
+ * Output Format
+ * The sink produces formatted output with these elements:
+ *
+ * - Timestamps: [HH:mm:ss.SSS] format when enabled
+ * - Status Icons: βΆ for started, β for finished
+ * - Progress Bars: Unicode block characters (β filled, β empty)
+ * - Progress Text: Percentage with one decimal place
+ * - Run State: Current TaskStatus.RunState value
+ *
+ *
+ * Task Name Resolution
+ * The sink automatically extracts meaningful task names using this priority:
+ *
+ * - getName() method found via reflection
+ * - Any getName() method found via reflection
+ * - Object.toString() as fallback
+ *
+ *
+ * Thread Safety
+ * This sink is thread-safe and can handle concurrent updates from multiple trackers.
+ * Output lines are atomic but may be interleaved if multiple tasks update simultaneously.
+ *
+ * @see StatusSink
+ * @see StatusTracker
+ * @see StatusContext
+ * @since 4.0.0
+ */
+public class ConsoleLoggerSink implements StatusSink {
+
+ private final PrintStream output;
+ private final boolean showTimestamp;
+ private final boolean useProgressBar;
+ private final DateTimeFormatter timeFormatter;
+
+ /**
+ * Creates a new console logger sink with default settings: System.out output,
+ * timestamps enabled, and progress bars enabled.
+ */
+ public ConsoleLoggerSink() {
+ this(System.out, true, true);
+ }
+
+ /**
+ * Creates a new console logger sink with custom output stream and default formatting:
+ * timestamps enabled and progress bars enabled.
+ *
+ * @param output the output stream for console messages
+ */
+ public ConsoleLoggerSink(PrintStream output) {
+ this(output, true, true);
+ }
+
+ /**
+ * Creates a new console logger sink with full customization of output stream and formatting.
+ *
+ * @param output the output stream for console messages
+ * @param showTimestamp whether to include timestamps in output
+ * @param useProgressBar whether to display visual progress bars
+ */
+ public ConsoleLoggerSink(PrintStream output, boolean showTimestamp, boolean useProgressBar) {
+ this.output = output;
+ this.showTimestamp = showTimestamp;
+ this.useProgressBar = useProgressBar;
+ this.timeFormatter = DateTimeFormatter.ofPattern("HH:mm:ss.SSS");
+ }
+
+ @Override
+ public void taskStarted(StatusTracker> task) {
+ String taskName = StatusTracker.extractTaskName(task);
+ String timestamp = showTimestamp ? "[" + LocalDateTime.now().format(timeFormatter) + "] " : "";
+ output.println(timestamp + "βΆ Started: " + taskName);
+ }
+
+ @Override
+ public void taskUpdate(StatusTracker> task, StatusUpdate> status) {
+ String taskName = StatusTracker.extractTaskName(task);
+ double progress = status.progress;
+ String timestamp = showTimestamp ? "[" + LocalDateTime.now().format(timeFormatter) + "] " : "";
+
+ if (useProgressBar) {
+ String progressBar = createProgressBar(progress);
+ String progressText = String.format(" %.1f%%", progress * 100);
+ output.println(timestamp + " " + taskName + " " + progressBar + progressText + " - " + status.runstate);
+ } else {
+ output.println(timestamp + " " + taskName + " [" + String.format("%.1f%%", progress * 100) + "] - " + status.runstate);
+ }
+ }
+
+ @Override
+ public void taskFinished(StatusTracker> task) {
+ String taskName = StatusTracker.extractTaskName(task);
+ String timestamp = showTimestamp ? "[" + LocalDateTime.now().format(timeFormatter) + "] " : "";
+ output.println(timestamp + "β Finished: " + taskName);
+ }
+
+ private String createProgressBar(double progress) {
+ int barLength = 20;
+ int filled = (int) (barLength * progress);
+ int empty = barLength - filled;
+
+ StringBuilder bar = new StringBuilder("[");
+ for (int i = 0; i < filled; i++) {
+ bar.append("β");
+ }
+ for (int i = 0; i < empty; i++) {
+ bar.append("β");
+ }
+ bar.append("]");
+
+ return bar.toString();
+ }
+
+}
diff --git a/internal-apis/src/main/java/io/github/jbellis/jvector/status/sinks/ConsolePanelLogIntercept.java b/internal-apis/src/main/java/io/github/jbellis/jvector/status/sinks/ConsolePanelLogIntercept.java
new file mode 100644
index 000000000..ade26dea9
--- /dev/null
+++ b/internal-apis/src/main/java/io/github/jbellis/jvector/status/sinks/ConsolePanelLogIntercept.java
@@ -0,0 +1,158 @@
+/*
+ * Copyright DataStax, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package io.github.jbellis.jvector.status.sinks;
+
+import org.apache.logging.log4j.Level;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.core.LoggerContext;
+import org.apache.logging.log4j.core.config.Configuration;
+import org.apache.logging.log4j.core.layout.PatternLayout;
+import org.slf4j.bridge.SLF4JBridgeHandler;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * Centralized logging configuration utility for applications using the status tracking framework.
+ * This class configures Log4j 2 to integrate with {@link ConsolePanelSink}
+ * by installing a {@link LogBuffer} appender that captures
+ * log output for display in the interactive console UI.
+ *
+ * When configured for interactive mode, this class:
+ *
+ * - Removes existing console appenders to prevent duplicate output
+ * - Installs a LogBuffer appender that captures all log messages
+ * - Configures log formatting with timestamp, level, logger name, and message
+ * - Bridges java.util.logging to SLF4J for unified log capture
+ *
+ *
+ * Usage with ConsolePanel:
+ *
{@code
+ * // Configure logging before creating ConsolePanelSink
+ * LoggerConfig.configure(OutputMode.INTERACTIVE);
+ *
+ * // Create sink with console panel
+ * ConsolePanelSink sink = ConsolePanelSink.builder().build();
+ *
+ * // Now all logging will appear in the console panel
+ * Logger logger = LogManager.getLogger(MyClass.class);
+ * logger.info("This will appear in the console panel");
+ * }
+ *
+ * This class is designed for application startup configuration and should be
+ * called once before creating any {@link ConsolePanelSink}
+ * instances.
+ *
+ * @see ConsolePanelSink
+ * @see LogBuffer
+ * @see OutputMode
+ * @since 4.0.0
+ */
+public final class ConsolePanelLogIntercept {
+
+ private static final AtomicBoolean CONFIGURING = new AtomicBoolean(false);
+ private static final String APPENDER_NAME = "ConsolePanelLogBuffer";
+
+ private ConsolePanelLogIntercept() {
+ }
+
+ /**
+ * Configures logging for the specified output mode. For {@link OutputMode#INTERACTIVE}
+ * mode, installs a Log4j 2 appender that captures and forwards log output to the console panel.
+ * Other output modes have no effect.
+ *
+ * This method is idempotent and thread-safe. Multiple calls will only configure logging once.
+ *
+ * @param outputMode the desired output mode (only INTERACTIVE triggers configuration)
+ */
+ public static void configure(OutputMode outputMode) {
+ if (outputMode == OutputMode.INTERACTIVE) {
+ configureForConsolePanel();
+ }
+ }
+
+ /**
+ * Convenience method for configuration from static initializers. Detects the appropriate
+ * output mode based on environment and configures logging accordingly.
+ *
+ * This method uses {@link OutputMode#detect()}
+ * to determine the best mode based on terminal capabilities.
+ */
+ public static void configureForStaticInit() {
+ configure(OutputMode.detect());
+ }
+
+ private static void configureForConsolePanel() {
+ if (!CONFIGURING.compareAndSet(false, true)) {
+ return;
+ }
+
+ try {
+ LoggerContext context = (LoggerContext) LogManager.getContext(false);
+ Configuration configuration = context.getConfiguration();
+
+ org.apache.logging.log4j.core.config.LoggerConfig rootConfig = configuration.getRootLogger();
+
+ PatternLayout layout = PatternLayout.newBuilder()
+ .withPattern("[%d{HH:mm:ss}] %-5level %logger{1} - %msg")
+ .withConfiguration(configuration)
+ .build();
+
+ LogBuffer appender = LogBuffer.createAppender(APPENDER_NAME, layout);
+ configuration.addAppender(appender);
+
+ // Remove existing appenders before registering the console panel buffer.
+ List existingAppenders = new ArrayList<>(rootConfig.getAppenders().keySet());
+ for (String appenderName : existingAppenders) {
+ rootConfig.removeAppender(appenderName);
+ }
+
+ // Set appender level to ALL so LogBuffer receives all events
+ // LogBuffer will apply its own display-level filtering
+ rootConfig.addAppender(appender, Level.ALL, null);
+ rootConfig.setLevel(Level.ALL);
+
+ // Ensure child logger configurations also inherit from the root and do not keep stale appenders.
+ for (org.apache.logging.log4j.core.config.LoggerConfig loggerConfig : configuration.getLoggers().values()) {
+ if (loggerConfig != rootConfig) {
+ for (String appenderName : new ArrayList<>(loggerConfig.getAppenders().keySet())) {
+ loggerConfig.removeAppender(appenderName);
+ }
+ loggerConfig.setLevel(null);
+ loggerConfig.setAdditive(true);
+ }
+ }
+
+ context.updateLoggers();
+
+ installJulBridge();
+ } finally {
+ CONFIGURING.set(false);
+ }
+ }
+
+ private static void installJulBridge() {
+ try {
+ java.util.logging.LogManager.getLogManager().reset();
+ SLF4JBridgeHandler.removeHandlersForRootLogger();
+ SLF4JBridgeHandler.install();
+ } catch (Exception ignored) {
+ // Bridge installation is best effort.
+ }
+ }
+}
diff --git a/internal-apis/src/main/java/io/github/jbellis/jvector/status/sinks/ConsolePanelSink.java b/internal-apis/src/main/java/io/github/jbellis/jvector/status/sinks/ConsolePanelSink.java
new file mode 100644
index 000000000..aa01a2de9
--- /dev/null
+++ b/internal-apis/src/main/java/io/github/jbellis/jvector/status/sinks/ConsolePanelSink.java
@@ -0,0 +1,2726 @@
+/*
+ * Copyright DataStax, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package io.github.jbellis.jvector.status.sinks;
+
+import io.github.jbellis.jvector.status.eventing.RunState;
+import io.github.jbellis.jvector.status.eventing.StatusSink;
+import io.github.jbellis.jvector.status.StatusTracker;
+import io.github.jbellis.jvector.status.StatusScope;
+import io.github.jbellis.jvector.status.eventing.StatusUpdate;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+import org.jline.utils.NonBlockingReader;
+import org.jline.terminal.Terminal;
+import org.jline.terminal.TerminalBuilder;
+import org.jline.terminal.Size;
+import org.jline.utils.AttributedString;
+import org.jline.utils.AttributedStringBuilder;
+import org.jline.utils.AttributedStyle;
+import org.jline.utils.Display;
+
+
+import java.io.*;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.StandardOpenOption;
+import java.time.LocalDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.*;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+/**
+ * A sophisticated terminal-based status sink that provides a hierarchical, stateful view
+ * of task progress using JLine3. This enhanced version includes a scrollable logging panel,
+ * captures all console output for integrated display, and supports custom keyboard handlers
+ * for application-specific interactivity.
+ *
+ * Features:
+ *
+ * - Hierarchical task display with parent-child relationships
+ * - Real-time updates without terminal scrolling
+ * - Scrollable logging panel for console output
+ * - Full terminal control with output redirection
+ * - Color-coded status indicators
+ * - Progress bars with percentage display
+ * - Task duration tracking
+ * - Automatic cleanup of completed tasks
+ * - Custom keyboard handlers for application-specific controls
+ * - Interactive search and filtering capabilities
+ *
+ *
+ * Display Layout:
+ *
+ * ββββ Task Status Monitor ββββββββββββββββββββββββββββββββββββββββ
+ * β β
+ * β βΆ [14:32:15] RootTask [ββββββββββββββββββββ] 60% (2.3s) β
+ * β ββ β [14:32:16] SubTask1 [ββββββββββββββββββββ] 100% β β
+ * β ββ βΆ [14:32:17] SubTask2 [ββββββββββββββββββββ] 30% β
+ * β β
+ * β Active: 2 | Completed: 1 | Failed: 0 β
+ * β βββ Console Output βββββββββββββββββββββββββββββββββββββββββββββ£
+ * β [INFO ] Starting data processing... β
+ * β [DEBUG] Loading configuration from file β
+ * β [WARN ] Cache miss for key: user_123 β
+ * β [INFO ] Processing batch 1 of 10 β
+ * β βΌ (β/β to scroll, 4 more lines) β
+ * βββββββββββββββββββββββββββββββββββββββββββββββββββββββββββββββββ
+ *
+ *
+ * Built-in Keyboard Controls:
+ *
+ * - β / β: Scroll through console log output
+ * - [ / ]: Adjust split between task and log panels
+ * - PgUp / PgDn: Quick split adjustment
+ * - Home: Reset scroll positions and split to defaults
+ * - End: Jump to end of logs and tasks
+ * - s: Save current display to file
+ * - q: Quit and shutdown (when auto-exit enabled)
+ * - ?: Show help panel
+ * - /: Enter search mode
+ *
+ *
+ * Custom Keyboard Handlers:
+ * Applications can register custom keyboard handlers using
+ * {@link Builder#withKeyHandler(String, Runnable)} to extend the interactive
+ * capabilities. Currently supported custom key combinations include:
+ *
+ * - shift-left: Shift + Left Arrow
+ * - shift-right: Shift + Right Arrow
+ *
+ *
+ * Example of registering a custom handler:
+ * {@code
+ * ConsolePanelSink sink = ConsolePanelSink.builder()
+ * .withKeyHandler("shift-right", () -> {
+ * clock.speedUp();
+ * sink.addLogMessage("Time speed increased");
+ * })
+ * .withKeyHandler("shift-left", () -> {
+ * clock.slowDown();
+ * sink.addLogMessage("Time speed decreased");
+ * })
+ * .build();
+ * }
+ *
+ * Thread Safety:
+ * This class is thread-safe. Status updates can be received from multiple threads
+ * concurrently. A dedicated render thread handles all terminal I/O and keyboard input
+ * processing. Custom keyboard handlers are executed synchronously on the render thread,
+ * so they should complete quickly to avoid blocking the UI.
+ *
+ * @see StatusSink
+ * @see StatusTracker
+ * @see Builder
+ * @since 4.0.0
+ */
+public class ConsolePanelSink implements StatusSink, AutoCloseable {
+
+ private static final Logger logger = LogManager.getLogger(ConsolePanelSink.class);
+
+ private Terminal terminal;
+ private Display display;
+ private final Thread renderThread;
+ private final Map, TaskNode> taskNodes;
+ private final Map scopeNodes;
+ private final DisplayNode rootNode;
+ private final DateTimeFormatter timeFormatter;
+ private final long refreshRateMs;
+ private final long completedRetentionMs;
+ private final boolean useColors;
+ private final boolean autoExit;
+ private final AtomicBoolean closed;
+ private final AtomicBoolean shouldRender;
+ private final AtomicBoolean introComplete; // Flag to prevent rendering during intro
+
+ // Custom keyboard handlers
+ private final Map customKeyHandlers;
+
+ // Logging panel components
+ private final LinkedList logBuffer; // Simple linked list for efficient head/tail operations
+ private final int maxLogLines;
+ private volatile int logScrollOffset;
+ private volatile int taskScrollOffset;
+ private volatile int splitOffset; // Controls split between task panel and log panel
+ private volatile boolean isUserScrollingLogs = false; // Track if user is manually scrolling
+ private volatile long lastLogDisplayTime = 0;
+ private final ReentrantReadWriteLock logLock;
+ private final PrintStream originalOut;
+ private final PrintStream originalErr;
+ private final LogCapturePrintStream capturedOut;
+ private final LogCapturePrintStream capturedErr;
+ private volatile int lastTaskContentHeight = 10;
+ private volatile int lastLogContentHeight = 5;
+ private volatile List lastRenderSnapshot = Collections.emptyList();
+
+ // Double-tap 'q' to exit tracking
+ private volatile long lastQPressTime = 0;
+ private static final long DOUBLE_TAP_WINDOW_MS = 300; // 300ms window for double-tap
+ private volatile boolean autoExitEnabled = false; // Track current auto-exit state
+
+ // Help panel state
+ private volatile boolean showingHelp = false;
+
+ // Search state
+ private enum SearchMode { NONE, EDITING, NAVIGATING }
+ private volatile SearchMode searchMode = SearchMode.NONE;
+ private volatile String searchPattern = "";
+ private volatile List searchMatches = new ArrayList<>();
+ private volatile int currentSearchIndex = 0;
+ private volatile String searchError = null;
+
+ // Log level filtering
+ private static final org.apache.logging.log4j.Level[] LOG_LEVELS = {
+ org.apache.logging.log4j.Level.ALL,
+ org.apache.logging.log4j.Level.TRACE,
+ org.apache.logging.log4j.Level.DEBUG,
+ org.apache.logging.log4j.Level.INFO,
+ org.apache.logging.log4j.Level.WARN,
+ org.apache.logging.log4j.Level.ERROR,
+ org.apache.logging.log4j.Level.FATAL,
+ org.apache.logging.log4j.Level.OFF
+ };
+
+
+ // ANSI color codes for different states
+ private static final AttributedStyle STYLE_PENDING = AttributedStyle.DEFAULT.foreground(AttributedStyle.WHITE);
+ private static final AttributedStyle STYLE_RUNNING = AttributedStyle.DEFAULT.foreground(AttributedStyle.CYAN);
+ private static final AttributedStyle STYLE_SUCCESS = AttributedStyle.DEFAULT.foreground(AttributedStyle.GREEN);
+ private static final AttributedStyle STYLE_FAILED = AttributedStyle.DEFAULT.foreground(AttributedStyle.RED);
+ private static final AttributedStyle STYLE_HEADER = AttributedStyle.DEFAULT.bold();
+ private static final AttributedStyle STYLE_LOG_INFO = AttributedStyle.DEFAULT.foreground(AttributedStyle.WHITE);
+ private static final AttributedStyle STYLE_LOG_WARN = AttributedStyle.DEFAULT.foreground(AttributedStyle.YELLOW);
+ private static final AttributedStyle STYLE_LOG_ERROR = AttributedStyle.DEFAULT.foreground(AttributedStyle.RED);
+ private static final AttributedStyle STYLE_LOG_DEBUG = AttributedStyle.DEFAULT.foreground(AttributedStyle.BRIGHT | AttributedStyle.BLACK);
+ private static final AttributedStyle STYLE_SECONDARY = AttributedStyle.DEFAULT.foreground(AttributedStyle.BRIGHT | AttributedStyle.CYAN);
+ private static final AttributedStyle STYLE_BORDER = AttributedStyle.DEFAULT.bold().foreground(AttributedStyle.BRIGHT | AttributedStyle.CYAN);
+ private static final AttributedStyle STYLE_BORDER_TITLE = AttributedStyle.DEFAULT.bold().foreground(AttributedStyle.YELLOW);
+ private static final DateTimeFormatter SCREENSHOT_FORMAT = DateTimeFormatter.ofPattern("yyyyMMdd_HHmmss");
+
+ private ConsolePanelSink(Builder builder) {
+ try {
+ this.terminal = TerminalBuilder.builder()
+ .system(true)
+ .jansi(true)
+ .jna(true) // Enable JNA for better terminal support
+ .color(builder.useColors) // Explicitly set color support
+ .build();
+
+ // Enter raw mode to capture single keystrokes without waiting for Enter
+ terminal.enterRawMode();
+
+ // Create display with fullscreen mode enabled for proper rendering
+ this.display = new Display(terminal, true);
+
+ // Resize display to current terminal size
+ Size initialSize = terminal.getSize();
+ if (initialSize == null || initialSize.getRows() <= 0 || initialSize.getColumns() <= 0) {
+ initialSize = new Size(100, 40);
+ }
+ display.resize(initialSize.getRows(), initialSize.getColumns());
+
+ // Initialize the display by clearing and setting up the screen
+ try {
+ terminal.puts(org.jline.utils.InfoCmp.Capability.clear_screen);
+ terminal.flush();
+ } catch (Exception e) {
+ logger.warn("Could not clear screen: {}", e.getMessage());
+ }
+ } catch (IOException e) {
+ throw new RuntimeException("Failed to initialize JLine terminal: " + e.getMessage(), e);
+ }
+
+ this.refreshRateMs = builder.refreshRateMs;
+ this.completedRetentionMs = builder.completedRetentionMs;
+ this.useColors = builder.useColors;
+ this.autoExit = builder.autoExit;
+ this.autoExitEnabled = builder.autoExit; // Initialize from builder
+ this.maxLogLines = builder.maxLogLines;
+ this.customKeyHandlers = new ConcurrentHashMap<>(builder.customKeyHandlers);
+ this.taskNodes = new ConcurrentHashMap<>();
+ this.scopeNodes = new ConcurrentHashMap<>();
+ this.rootNode = new RootNode();
+ this.timeFormatter = DateTimeFormatter.ofPattern("HH:mm:ss");
+ this.closed = new AtomicBoolean(false);
+ this.shouldRender = new AtomicBoolean(true);
+ this.introComplete = new AtomicBoolean(true); // Default to true, set to false by showIntroScreen()
+
+ // Initialize logging components
+ this.logBuffer = new LinkedList<>();
+ this.logScrollOffset = 0;
+ this.splitOffset = 0; // Start with default split
+ this.isUserScrollingLogs = false;
+ this.lastLogDisplayTime = 0;
+ this.logLock = new ReentrantReadWriteLock();
+
+ // Capture System.out and System.err
+ this.originalOut = System.out;
+ this.originalErr = System.err;
+ this.capturedOut = new LogCapturePrintStream("OUT");
+ this.capturedErr = new LogCapturePrintStream("ERR");
+
+ // Redirect console output only if requested
+ if (builder.captureSystemStreams) {
+ System.setOut(capturedOut);
+ System.setErr(capturedErr);
+ }
+
+ // Create and start the dedicated render thread
+ this.renderThread = new Thread(this::renderLoop, "ConsolePanelSink-Renderer");
+ this.renderThread.setDaemon(true);
+ this.renderThread.start();
+
+ // Add shutdown hook to properly clean up terminal on exit
+ Runtime.getRuntime().addShutdownHook(new Thread(new ShutdownCleanupRunnable(this), "ConsolePanelSink-Shutdown"));
+
+ // Register with LogBuffer to receive log messages
+ LogBuffer.setActiveSink(this);
+
+ // Force an immediate full frame render to initialize the layout
+ try {
+ Thread.sleep(50); // Brief pause to let thread start
+ // Do a direct refresh call to trigger immediate render
+ refresh();
+ Thread.sleep(50); // Give time for the initial render to complete
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ }
+ }
+
+ private void renderLoop() {
+ // Render loop with non-blocking input handling
+ NonBlockingReader reader = null;
+ try {
+ // Set up non-blocking reader for keyboard input
+ reader = terminal.reader();
+
+ long lastRenderTime = System.currentTimeMillis();
+ long lastCleanupTime = System.currentTimeMillis();
+
+ // Log that render loop has started
+ logger.debug("Render loop started with non-blocking input");
+
+ while (!closed.get()) {
+ long now = System.currentTimeMillis();
+
+ // Check for keyboard input (non-blocking)
+ try {
+ int c = reader.read(1); // Non-blocking read with 1ms timeout
+ if (c != -2 && c != -1) { // -2 means no input available, -1 means EOF
+ handleInput(reader, c);
+ }
+ } catch (IOException e) {
+ // Ignore read errors to prevent interrupting the render loop
+ }
+
+ // Clean up completed tasks periodically
+ if (now - lastCleanupTime >= 1000) { // Check every second
+ cleanupCompletedTasks(now);
+ lastCleanupTime = now;
+
+ // Check for auto-exit condition
+ if (autoExitEnabled && isEverythingComplete()) {
+ addLogMessage("Auto-exit: All work complete - shutting down...");
+ performExit();
+ break;
+ }
+ }
+
+ // Render at specified refresh rate
+ if (now - lastRenderTime >= refreshRateMs) {
+ refresh();
+ lastRenderTime = now;
+ }
+
+ // Small sleep to prevent CPU spinning
+ if (!closed.get()) {
+ Thread.sleep(10);
+ }
+ }
+ } catch (Exception e) {
+ if (!closed.get()) {
+ logger.error("Render loop error", e);
+ }
+ } finally {
+ // Clean up reader
+ if (reader != null) {
+ try {
+ reader.close();
+ } catch (IOException e) {
+ // Ignore
+ }
+ }
+ logger.debug("Render loop exited");
+ }
+ }
+
+ private boolean isEverythingComplete() {
+ // Check if root node is complete (which recursively checks all scopes and tasks)
+ return rootNode.isComplete();
+ }
+
+ private void cleanupCompletedTasks(long now) {
+ // Remove completed tasks and scopes based on parent completion status
+ List, TaskNode>> tasksToRemove = new ArrayList<>();
+ List> scopesToRemove = new ArrayList<>();
+
+ // Cleanup tasks
+ for (Map.Entry, TaskNode> entry : taskNodes.entrySet()) {
+ TaskNode node = entry.getValue();
+ if (node.finishTime > 0) {
+ // Check if this completed task should be removed
+ if (node.parent == null || node.parent instanceof RootNode) {
+ // Root-level task - use standard retention time
+ if ((now - node.finishTime) > completedRetentionMs) {
+ tasksToRemove.add(entry);
+ }
+ } else if (node.parent.isComplete()) {
+ // Parent (scope or task) is also completed - remove child after brief delay
+ // This ensures the final state is visible before cleanup
+ if ((now - node.finishTime) > 1000) { // 1 second minimum visibility
+ tasksToRemove.add(entry);
+ }
+ }
+ // If parent is still running, keep this completed child visible
+ }
+ }
+
+ // Cleanup scopes
+ for (Map.Entry entry : scopeNodes.entrySet()) {
+ ScopeNode node = entry.getValue();
+ if (node.finishTime > 0) {
+ // Check if this completed scope should be removed
+ if (node.parent == null || node.parent instanceof RootNode) {
+ // Root-level scope - use standard retention time
+ if ((now - node.finishTime) > completedRetentionMs) {
+ scopesToRemove.add(entry);
+ }
+ } else if (node.parent.isComplete()) {
+ // Parent scope is also completed - remove child after brief delay
+ if ((now - node.finishTime) > 1000) { // 1 second minimum visibility
+ scopesToRemove.add(entry);
+ }
+ }
+ }
+ }
+
+ // Execute removals
+ for (Map.Entry, TaskNode> entry : tasksToRemove) {
+ TaskNode node = entry.getValue();
+ taskNodes.remove(entry.getKey());
+ if (node.parent != null) {
+ node.parent.children.remove(node);
+ }
+ }
+
+ for (Map.Entry entry : scopesToRemove) {
+ ScopeNode node = entry.getValue();
+ scopeNodes.remove(entry.getKey());
+ if (node.parent != null) {
+ node.parent.children.remove(node);
+ }
+ }
+ }
+
+
+ /**
+ * Add a log message to the display buffer.
+ * This is called by LogBuffer to add logging framework messages.
+ * Only sink methods should mutate the logBuffer.
+ */
+ /**
+ * Adds a message to the log buffer. The message will be timestamped and displayed
+ * in the console output panel.
+ *
+ * @param message the message to add
+ */
+ public void addLogMessage(String message) {
+ if (message == null || message.trim().isEmpty()) {
+ return;
+ }
+
+ logLock.writeLock().lock();
+ try {
+ // Add timestamp if not present
+ if (!message.matches("^\\[\\d{2}:\\d{2}:\\d{2}\\].*")) {
+ message = "[" + LocalDateTime.now().format(timeFormatter) + "] " + message;
+ }
+
+ logBuffer.addLast(message);
+
+ // Limit buffer size to maxLogLines (default 1000)
+ while (logBuffer.size() > maxLogLines) {
+ logBuffer.removeFirst();
+ // When manually scrolling, we need to adjust offset to maintain the same view
+ // Even if offset is 0, removing from front means we need to "scroll up" to stay in place
+ // However, we can't have negative offset, so content will shift if at the very top
+ if (isUserScrollingLogs && logScrollOffset > 0) {
+ logScrollOffset--;
+ } else if (!isUserScrollingLogs) {
+ // When auto-scrolling, decrement to maintain bottom view
+ if (logScrollOffset > 0) {
+ logScrollOffset--;
+ }
+ }
+ }
+
+ // Auto-scroll to latest if not manually scrolling
+ if (!isUserScrollingLogs) {
+ int maxScroll = Math.max(0, logBuffer.size() - getLogPanelHeight());
+ logScrollOffset = maxScroll;
+ }
+ } finally {
+ logLock.writeLock().unlock();
+ }
+ }
+
+ private void handleSearchInput(NonBlockingReader reader, int c) {
+ if (searchMode == SearchMode.EDITING) {
+ // Enter - switch to navigation mode
+ if (c == '\n' || c == '\r') {
+ if (searchPattern.length() >= 2 && !searchMatches.isEmpty()) {
+ searchMode = SearchMode.NAVIGATING;
+ currentSearchIndex = 0;
+ jumpToMatch(0);
+ } else if (searchPattern.length() < 2) {
+ searchError = "Pattern too short (min 2 chars)";
+ } else {
+ searchError = "No matches found";
+ }
+ return;
+ }
+
+ // Handle [ and ] for log level during search (same as normal mode)
+ if (c == '[') {
+ cycleLogLevel(false);
+ updateSearchPreview();
+ return;
+ }
+
+ if (c == ']') {
+ cycleLogLevel(true);
+ updateSearchPreview();
+ return;
+ }
+
+ // ESC - cancel search
+ if (c == 27) {
+ searchMode = SearchMode.NONE;
+ searchPattern = "";
+ searchMatches.clear();
+ searchError = null;
+ return;
+ }
+
+ // Backspace
+ if (c == 127 || c == '\b') {
+ if (searchPattern.length() > 0) {
+ searchPattern = searchPattern.substring(0, searchPattern.length() - 1);
+ updateSearchPreview();
+ }
+ return;
+ }
+
+ // Add character to search pattern (all printable ASCII allowed)
+ if (c >= 32 && c < 127) { // Printable ASCII
+ searchPattern += (char) c;
+ updateSearchPreview();
+ }
+ } else if (searchMode == SearchMode.NAVIGATING) {
+ // ESC - exit navigation mode
+ if (c == 27) {
+ searchMode = SearchMode.NONE;
+ searchPattern = "";
+ searchMatches.clear();
+ searchError = null;
+ return;
+ }
+
+ // n - next match
+ if (c == 'n') {
+ currentSearchIndex = (currentSearchIndex + 1) % searchMatches.size();
+ jumpToMatch(currentSearchIndex);
+ return;
+ }
+
+ // p or N - previous match
+ if (c == 'p' || c == 'N') {
+ currentSearchIndex = (currentSearchIndex - 1 + searchMatches.size()) % searchMatches.size();
+ jumpToMatch(currentSearchIndex);
+ return;
+ }
+ }
+ }
+
+ private void updateSearchPreview() {
+ searchError = null;
+ searchMatches.clear();
+
+ if (searchPattern.length() < 2) {
+ return;
+ }
+
+ try {
+ java.util.regex.Pattern pattern = java.util.regex.Pattern.compile(searchPattern);
+
+ // Search through log buffer
+ for (int i = 0; i < logBuffer.size(); i++) {
+ String line = logBuffer.get(i);
+ if (pattern.matcher(line).find()) {
+ searchMatches.add(i);
+ }
+ }
+ } catch (java.util.regex.PatternSyntaxException e) {
+ searchError = e.getMessage();
+ searchMatches.clear();
+ }
+ }
+
+ private void jumpToMatch(int matchIndex) {
+ if (matchIndex >= 0 && matchIndex < searchMatches.size()) {
+ int matchLine = searchMatches.get(matchIndex);
+ logScrollOffset = Math.max(0, matchLine - getLogPanelHeight() / 2);
+ isUserScrollingLogs = true;
+ }
+ }
+
+ private void performExit() {
+ // Signal to close first
+ if (!closed.compareAndSet(false, true)) {
+ return; // Already closing
+ }
+
+ // Clear the display and show exit message
+ try {
+ List exitMessage = new ArrayList<>();
+ exitMessage.add(new AttributedStringBuilder()
+ .style(AttributedStyle.DEFAULT.foreground(AttributedStyle.YELLOW))
+ .append("Console panel shutting down...")
+ .toAttributedString());
+ exitMessage.add(new AttributedStringBuilder()
+ .append("Restoring normal terminal...")
+ .toAttributedString());
+ display.update(exitMessage, exitMessage.size());
+ terminal.flush();
+ } catch (Exception e) {
+ // Ignore display errors during shutdown
+ }
+
+ // Perform actual cleanup
+ performCleanup();
+ }
+
+ private void handleInput(NonBlockingReader reader, int c) {
+ logLock.writeLock().lock();
+ try {
+ // Handle [ and ] for log level
+ // [ = decrease verbosity (less verbose, higher level)
+ // ] = increase verbosity (more verbose, lower level)
+ if (c == '[') {
+ cycleLogLevel(false);
+ return;
+ }
+
+ if (c == ']') {
+ cycleLogLevel(true);
+ return;
+ }
+
+ // If showing help, any key closes it
+ if (showingHelp) {
+ showingHelp = false;
+ return;
+ }
+
+ // Handle search mode
+ if (searchMode != SearchMode.NONE) {
+ handleSearchInput(reader, c);
+ return;
+ }
+
+ // Handle help command
+ if (c == '?') {
+ showingHelp = true;
+ return;
+ }
+
+ // Handle search command
+ if (c == '/') {
+ searchMode = SearchMode.EDITING;
+ searchPattern = "";
+ searchMatches.clear();
+ currentSearchIndex = 0;
+ searchError = null;
+ return;
+ }
+
+ // Handle quit command
+ if (c == 'q' || c == 'Q') {
+ handleQuitCommand();
+ return;
+ }
+
+
+ if (c == 's' || c == 'S') {
+ saveScreenshot();
+ return;
+ }
+
+ // Handle arrow keys and special sequences (including numpad +/-)
+ if (c == 27) { // ESC sequence
+ try {
+ int next = reader.read(10);
+
+ // If no follow-up character, treat ESC as standalone quit command (like 'q')
+ if (next == -2) { // timeout, standalone ESC
+ handleQuitCommand();
+ return;
+ }
+
+ if (next == 'O') {
+ // ESC O sequences - just handle special keys, removed numpad +/- confusion
+ int key = reader.read(10);
+ // Fall through to regular handling
+ }
+
+ // For ESC [ sequences
+ if (next == '[') {
+ int key = reader.read(10);
+
+ switch (key) {
+ case 'A': // Up arrow - scroll logs up (show older content)
+ // Enable manual scrolling FIRST to prevent auto-scroll from interfering
+ isUserScrollingLogs = true;
+ int maxScrollRangeUp = Math.max(0, logBuffer.size() - getLogPanelHeight());
+ // Clamp current offset to valid range
+ logScrollOffset = Math.max(0, Math.min(logScrollOffset, maxScrollRangeUp));
+ // Then scroll up if possible
+ if (logScrollOffset > 0) {
+ logScrollOffset--;
+ }
+ break;
+ case 'B': // Down arrow - scroll logs down (show newer content)
+ int maxLogScroll = Math.max(0, logBuffer.size() - getLogPanelHeight());
+ // First clamp current offset to valid range
+ logScrollOffset = Math.max(0, Math.min(logScrollOffset, maxLogScroll));
+ // Then scroll down if possible
+ if (logScrollOffset < maxLogScroll) {
+ logScrollOffset++;
+ isUserScrollingLogs = true;
+ } else {
+ isUserScrollingLogs = false; // At bottom, resume auto-follow
+ }
+ break;
+ case '1': // ESC[1;2A or ESC[1;2B - Shift+arrows
+ int next2 = reader.read(10);
+ if (next2 == ';') {
+ int modifier = reader.read(10);
+ if (modifier == '2') { // Shift modifier
+ int direction = reader.read(10);
+ if (direction == 'A' || direction == 'B') {
+ // Shift+Up/Down - page up/down logs
+ int pageSize = getLogPanelHeight() - 1;
+ int maxScroll = Math.max(0, logBuffer.size() - getLogPanelHeight());
+ // First clamp current offset
+ logScrollOffset = Math.max(0, Math.min(logScrollOffset, maxScroll));
+ if (direction == 'A') { // Shift+Up - page up
+ if (logScrollOffset > 0) {
+ logScrollOffset = Math.max(0, logScrollOffset - pageSize);
+ isUserScrollingLogs = true;
+ }
+ } else if (direction == 'B') { // Shift+Down - page down
+ if (logScrollOffset < maxScroll) {
+ logScrollOffset = Math.min(logScrollOffset + pageSize, maxScroll);
+ isUserScrollingLogs = logScrollOffset < maxScroll;
+ }
+ }
+ } else if (direction == 'C') { // Shift+Right
+ Runnable handler = customKeyHandlers.get("shift-right");
+ if (handler != null) {
+ handler.run();
+ }
+ } else if (direction == 'D') { // Shift+Left
+ Runnable handler = customKeyHandlers.get("shift-left");
+ if (handler != null) {
+ handler.run();
+ }
+ }
+ }
+ }
+ break;
+ case '5': // Page Up (ESC[5~) - Scroll logs up by page
+ if (reader.read(10) == '~') { // consume ~
+ int pageSize = getLogPanelHeight() - 1;
+ int maxScroll = Math.max(0, logBuffer.size() - getLogPanelHeight());
+ logScrollOffset = Math.max(0, Math.min(logScrollOffset, maxScroll));
+ if (logScrollOffset > 0) {
+ logScrollOffset = Math.max(0, logScrollOffset - pageSize);
+ isUserScrollingLogs = true;
+ }
+ }
+ break;
+ case '6': // Page Down (ESC[6~) - Scroll logs down by page
+ if (reader.read(10) == '~') { // consume ~
+ int pageSize = getLogPanelHeight() - 1;
+ int maxScroll = Math.max(0, logBuffer.size() - getLogPanelHeight());
+ logScrollOffset = Math.max(0, Math.min(logScrollOffset, maxScroll));
+ if (logScrollOffset < maxScroll) {
+ logScrollOffset = Math.min(logScrollOffset + pageSize, maxScroll);
+ isUserScrollingLogs = logScrollOffset < maxScroll;
+ }
+ }
+ break;
+ case 'H': // Home
+ logScrollOffset = 0;
+ taskScrollOffset = 0;
+ splitOffset = 0; // Reset split to default
+ break;
+ case 'F': // End
+ logScrollOffset = Math.max(0, logBuffer.size() - getLogPanelHeight());
+ taskScrollOffset = Math.max(0, taskNodes.size() - 10);
+ break;
+ }
+ }
+ } catch (IOException e) {
+ // Ignore
+ }
+ }
+ } finally {
+ logLock.writeLock().unlock();
+ }
+ }
+
+ /**
+ * Called when a scope is created. Adds it to the display hierarchy.
+ */
+ @Override
+ public void scopeStarted(StatusScope scope) {
+ if (closed.get()) return;
+
+ // Find parent scope or use root
+ StatusScope parentScope = scope.getParent();
+ DisplayNode parent = parentScope != null ? scopeNodes.get(parentScope) : rootNode;
+
+ ScopeNode node = new ScopeNode(scope, parent);
+ scopeNodes.put(scope, node);
+
+ if (parent != null) {
+ parent.children.add(node);
+ }
+ }
+
+ /**
+ * Called when a scope is closed. Marks it as finished but keeps it visible for retention period.
+ */
+ @Override
+ public void scopeFinished(StatusScope scope) {
+ if (closed.get()) return;
+
+ ScopeNode node = scopeNodes.get(scope);
+ if (node != null) {
+ node.finishTime = System.currentTimeMillis();
+ // Don't remove immediately - let cleanup handle it based on retention time
+ }
+ }
+
+ @Override
+ public void taskStarted(StatusTracker> task) {
+ if (closed.get()) return;
+
+ // Find parent: either a scope or root
+ StatusScope parentScope = task.getParentScope();
+ DisplayNode parent = parentScope != null ? scopeNodes.get(parentScope) : rootNode;
+
+ TaskNode node = new TaskNode(task, parent);
+ taskNodes.put(task, node);
+
+ if (parent != null) {
+ parent.children.add(node);
+ }
+ // Note: Render thread will pick up this change at next refresh cycle
+ }
+
+ /**
+ * Handles quit command logic - supports both 'q' and ESC keys.
+ * Implements double-tap detection and auto-exit toggle behavior.
+ */
+ private void handleQuitCommand() {
+ long now = System.currentTimeMillis();
+ boolean isDoubleTap = (now - lastQPressTime) < DOUBLE_TAP_WINDOW_MS;
+ lastQPressTime = now;
+
+ // Check if all work is complete (scopes and tasks)
+ boolean allTasksFinished = isEverythingComplete();
+
+ // Double-tap always exits
+ if (isDoubleTap) {
+ addLogMessage("Double-tap detected - shutting down console panel...");
+ performExit();
+ return;
+ }
+
+ // If auto-exit is enabled and all tasks are finished, exit immediately
+ if (autoExitEnabled && allTasksFinished) {
+ addLogMessage("All tasks finished - shutting down console panel...");
+ performExit();
+ return;
+ }
+
+ // If auto-exit is enabled but tasks are still running, just disable it
+ if (autoExitEnabled) {
+ autoExitEnabled = false;
+ addLogMessage("Auto-exit disabled. Press 'q'/ESC again within 300ms to exit.");
+ return;
+ }
+
+ // If auto-exit is disabled, enable it
+ if (!autoExitEnabled) {
+ autoExitEnabled = true;
+ if (allTasksFinished) {
+ addLogMessage("Auto-exit enabled - all work complete, exiting now...");
+ performExit();
+ } else {
+ addLogMessage("Auto-exit enabled. Will exit when all work completes.");
+ }
+ }
+ }
+
+ @Override
+ public void taskUpdate(StatusTracker> task, StatusUpdate> status) {
+ if (closed.get()) return;
+
+ // Buffer the update - no rendering here, just update data structures
+ TaskNode node = taskNodes.get(task);
+ if (node != null) {
+ node.lastStatus = status;
+ node.lastUpdateTime = System.currentTimeMillis();
+ }
+ // Note: Render thread will pick up this change at next refresh cycle
+ }
+
+ @Override
+ public void taskFinished(StatusTracker> task) {
+ if (closed.get()) return;
+
+ // Buffer the update - no rendering here, just update data structures
+ TaskNode node = taskNodes.get(task);
+ if (node != null) {
+ node.finishTime = System.currentTimeMillis();
+ // Get the final status to ensure we show 100% for SUCCESS tasks
+ StatusUpdate> finalStatus = task.getStatus();
+ if (finalStatus != null) {
+ // Ensure completed tasks show 100% progress if they succeeded
+ if (finalStatus.runstate == RunState.SUCCESS) {
+ node.lastStatus = new StatusUpdate<>(1.0, RunState.SUCCESS, finalStatus.tracked);
+ } else {
+ node.lastStatus = finalStatus;
+ }
+ node.lastUpdateTime = System.currentTimeMillis();
+ }
+ // Note: Completed tasks will be cleaned up by the render thread
+ // based on completedRetentionMs
+ }
+ }
+
+
+ private static volatile int refreshCount = 0;
+ private Size lastKnownSize = null;
+
+ private void refresh() {
+ if (closed.get() || !introComplete.get()) return;
+
+ try {
+ refreshCount++;
+
+// // Debug: log refresh attempts
+// if (refreshCount == 1 || refreshCount % 50 == 0) {
+// System.err.println("[ConsolePanelSink] Refresh #" + refreshCount + " starting");
+// }
+
+ // Get terminal size
+ Size size = terminal.getSize();
+
+ // If terminal size is invalid, use a reasonable default
+ if (size == null || size.getRows() <= 0 || size.getColumns() <= 0) {
+ if (refreshCount == 1) { // Only log once on first refresh
+ if (size != null) {
+ logger.info("Terminal size detection failed (got {}x{}). Using default size 100x40. " +
+ "This may happen when running in certain IDEs or piped environments.",
+ size.getColumns(), size.getRows());
+ } else {
+ logger.info("Terminal size is null. Using default size 100x40.");
+ }
+ }
+ size = new Size(100, 40); // More reasonable default for modern terminals
+ }
+
+ // Detect terminal resize
+ boolean terminalResized = false;
+ if (lastKnownSize != null &&
+ (lastKnownSize.getRows() != size.getRows() || lastKnownSize.getColumns() != size.getColumns())) {
+ terminalResized = true;
+
+ // Force a complete redraw on resize
+ try {
+ // Clear the entire screen
+ terminal.puts(org.jline.utils.InfoCmp.Capability.clear_screen);
+ terminal.puts(org.jline.utils.InfoCmp.Capability.cursor_home);
+ terminal.flush();
+
+ // Reset the display to force full redraw
+ display.clear();
+ display.resize(size.getRows(), size.getColumns());
+
+ // Reset display state to force complete refresh
+ display.update(Collections.emptyList(), 0);
+
+ } catch (Exception e) {
+ // If clear fails, try alternative approach
+ try {
+ terminal.writer().print("\033[2J\033[H"); // ANSI clear screen and home
+ terminal.flush();
+ } catch (Exception ignored) {}
+ }
+ }
+ lastKnownSize = size;
+
+ // Only enable debug logging if explicitly requested
+ boolean debugThisRefresh = false; // (refreshCount % 100 == 1); // Uncomment for debugging
+
+ List lines = new ArrayList<>();
+
+ // Calculate layout dimensions - minimize border overhead
+ // Base: 1 top border, 1 middle divider, 1 bottom border, 1 status bar = 4 total overhead lines
+ // Add 2 more lines if search panel is active (border + content, reuses middle divider)
+ int totalOverhead = 4;
+ int searchPanelHeight = 0;
+ if (searchMode != SearchMode.NONE) {
+ searchPanelHeight = 2; // Search panel: border + content line
+ totalOverhead += searchPanelHeight;
+ }
+ int availableContent = Math.max(2, size.getRows() - totalOverhead);
+
+ // Split available content between task and log panels (2/3 for tasks, 1/3 for logs)
+ int baseTaskContent = Math.max(1, (availableContent * 2) / 3);
+ int taskContentHeight = Math.max(1, Math.min(availableContent - 1, baseTaskContent + splitOffset));
+ int logContentHeight = Math.max(1, availableContent - taskContentHeight);
+
+ lastTaskContentHeight = taskContentHeight;
+ lastLogContentHeight = logContentHeight;
+
+ // Build the display with minimal borders
+ if (showingHelp) {
+ renderHelpPanel(lines, size.getColumns(), size.getRows());
+ } else {
+ renderCompactHeader(lines, size.getColumns());
+ renderTaskContent(lines, size.getColumns(), taskContentHeight);
+
+ // Insert search panel if active
+ if (searchMode != SearchMode.NONE) {
+ renderSearchPanel(lines, size.getColumns());
+ }
+
+ renderMiddleDivider(lines, size.getColumns());
+ renderLogContent(lines, size.getColumns(), logContentHeight);
+ renderBottomBar(lines, size.getColumns());
+ }
+
+ // Ensure we don't exceed terminal height
+ while (lines.size() > size.getRows()) {
+ lines.remove(lines.size() - 1);
+ }
+
+ // Pad to fill screen
+ while (lines.size() < size.getRows()) {
+ lines.add(new AttributedString(""));
+ }
+
+ // Debug: Check what we're about to render
+ if (debugThisRefresh) {
+ logger.debug("About to update display with {} lines", lines.size());
+ if (!lines.isEmpty()) {
+ String firstLine = lines.get(0).toAnsi(terminal);
+ logger.debug("First line content (len={}): {}", firstLine.length(),
+ firstLine.length() > 50 ? firstLine.substring(0, 50) + "..." : firstLine);
+ }
+ }
+
+ // Update display - Display class handles differential updates
+ // Position cursor at bottom right to hide it
+ display.update(lines, size.cursorPos(size.getRows() - 1, size.getColumns() - 1));
+ terminal.flush();
+
+ List snapshot = new ArrayList<>(lines.size());
+ for (AttributedString line : lines) {
+ snapshot.add(line.toString());
+ }
+ lastRenderSnapshot = snapshot;
+
+ if (debugThisRefresh) {
+ logger.debug("Display update completed");
+ }
+
+ } catch (Exception e) {
+ // Log error but continue
+ logger.error("Display update error: {}", e.getMessage(), e);
+ }
+ }
+
+
+ // New compact rendering methods
+ private void renderCompactHeader(List lines, int width) {
+ // Single top border line with title
+ lines.add(buildSectionBorder('β', 'β', "Active Tasks", width));
+ }
+
+ private void renderSearchPanel(List lines, int width) {
+ // Top border
+ lines.add(buildSectionBorder('β ', 'β£', "Search", width));
+
+ // Search content line
+ AttributedStringBuilder content = new AttributedStringBuilder();
+
+ if (searchMode == SearchMode.EDITING) {
+ // Editing mode - show pattern input
+ content.style(AttributedStyle.DEFAULT.foreground(AttributedStyle.CYAN))
+ .append("Pattern: ");
+ content.style(AttributedStyle.DEFAULT.bold().foreground(AttributedStyle.YELLOW))
+ .append(searchPattern.isEmpty() ? "_" : searchPattern);
+
+ // Show match count or error
+ if (searchError != null) {
+ content.style(AttributedStyle.DEFAULT.foreground(AttributedStyle.RED))
+ .append(" [ERROR: " + searchError + "]");
+ } else if (searchPattern.length() >= 2) {
+ content.style(AttributedStyle.DEFAULT.foreground(AttributedStyle.GREEN))
+ .append(" [" + searchMatches.size() + " matches]");
+ } else {
+ content.style(AttributedStyle.DEFAULT.foreground(AttributedStyle.BRIGHT | AttributedStyle.BLACK))
+ .append(" [min 2 chars]");
+ }
+
+ content.style(AttributedStyle.DEFAULT.foreground(AttributedStyle.CYAN))
+ .append(" | Enter: navigate | ESC: cancel");
+ } else if (searchMode == SearchMode.NAVIGATING) {
+ // Navigation mode - show current match
+ content.style(AttributedStyle.DEFAULT.foreground(AttributedStyle.CYAN))
+ .append("Searching: ");
+ content.style(AttributedStyle.DEFAULT.bold().foreground(AttributedStyle.YELLOW))
+ .append(searchPattern);
+ content.style(AttributedStyle.DEFAULT.foreground(AttributedStyle.GREEN))
+ .append(String.format(" | Match %d/%d", currentSearchIndex + 1, searchMatches.size()));
+ content.style(AttributedStyle.DEFAULT.foreground(AttributedStyle.CYAN))
+ .append(" | n: next | p: prev | ESC: exit");
+ }
+
+ lines.add(wrapWithSideBorders(content.toAttributedString(), width));
+ // Note: No bottom border - renderMiddleDivider will provide the divider to Console Output
+ }
+
+ private void renderMiddleDivider(List lines, int width) {
+ // Single divider line between panels with title
+ lines.add(buildSectionBorder('β ', 'β£', "Console Output", width));
+ }
+
+ private void renderBottomBar(List lines, int width) {
+ // Bottom border
+ lines.add(buildSectionBorder('β', 'β', null, width));
+
+ // Status bar (kept separate for information display)
+ renderStatusLine(lines, width);
+ }
+
+ private void renderTaskContent(List lines, int width, int contentHeight) {
+ // Render task lines without additional borders
+ List taskLines = collectTaskContentLines(width - 4, contentHeight);
+ for (AttributedString line : taskLines) {
+ lines.add(wrapWithSideBorders(line, width));
+ }
+ }
+
+ private void renderLogContent(List lines, int width, int contentHeight) {
+ // Render log lines without additional borders
+ List logLines = collectLogContentLines(width - 4, contentHeight);
+ for (AttributedString line : logLines) {
+ lines.add(wrapWithSideBorders(line, width));
+ }
+ }
+
+ private AttributedString wrapWithSideBorders(AttributedString content, int width) {
+ AttributedStringBuilder builder = new AttributedStringBuilder();
+ builder.style(STYLE_BORDER).append("β ");
+ builder.append(content);
+
+ // Calculate padding
+ int contentLength = content.columnLength();
+ int paddingNeeded = Math.max(0, width - contentLength - 4); // 4 for "β " and " β"
+ builder.append(" ".repeat(paddingNeeded));
+
+ builder.style(STYLE_BORDER).append(" β");
+ return builder.toAttributedString();
+ }
+
+ private List collectTaskContentLines(int innerWidth, int contentHeight) {
+ List result = new ArrayList<>();
+
+ // Collect task entries
+ List taskEntries = new ArrayList<>();
+ collectTaskLines(rootNode, taskEntries, "", true, innerWidth);
+
+ if (taskEntries.isEmpty()) {
+ result.add(new AttributedString(center("No active tasks", innerWidth)));
+ } else {
+ // Add visible task lines
+ int startIdx = Math.min(taskScrollOffset, Math.max(0, taskEntries.size() - contentHeight));
+ int endIdx = Math.min(taskEntries.size(), startIdx + contentHeight);
+
+ for (int i = startIdx; i < endIdx; i++) {
+ SectionLine line = taskEntries.get(i);
+ if (line.style != null) {
+ result.add(new AttributedStringBuilder().style(line.style).append(line.text).toAttributedString());
+ } else {
+ result.add(new AttributedString(line.text));
+ }
+ }
+ }
+
+ // Pad to fill height
+ while (result.size() < contentHeight) {
+ result.add(new AttributedString(""));
+ }
+
+ return result;
+ }
+
+ private List collectLogContentLines(int innerWidth, int contentHeight) {
+ List result = new ArrayList<>();
+
+ logLock.readLock().lock();
+ try {
+ int totalLogs = logBuffer.size();
+
+ if (totalLogs > 0) {
+ // Calculate starting position for display
+ // When not scrolling, show the bottom (most recent) contentHeight lines
+ // When scrolling, logScrollOffset is the start index (0 = top of buffer)
+ int startIdx;
+ if (isUserScrollingLogs) {
+ // logScrollOffset is the start index - clamp to valid range
+ int maxStartIdx = Math.max(0, totalLogs - contentHeight);
+ startIdx = Math.max(0, Math.min(logScrollOffset, maxStartIdx));
+ } else {
+ // Show the bottom (most recent) contentHeight lines
+ startIdx = Math.max(0, totalLogs - contentHeight);
+ }
+
+ // Prepare search highlighting pattern
+ java.util.regex.Pattern highlightPattern = null;
+ if (!searchMatches.isEmpty() && !searchPattern.isEmpty()) {
+ try {
+ highlightPattern = java.util.regex.Pattern.compile(searchPattern);
+ } catch (Exception e) {
+ // Ignore pattern errors during rendering
+ }
+ }
+
+ // Most common case: showing recent logs (at or near the end)
+ // Use descending iterator and collect the needed lines
+ if (startIdx >= totalLogs - contentHeight * 2) {
+ // We're close to the end, use descending iterator
+ Iterator descIter = logBuffer.descendingIterator();
+ List tempLines = new ArrayList<>();
+ List tempIndices = new ArrayList<>();
+
+ // Skip the newest lines we don't need
+ int toSkip = totalLogs - startIdx - contentHeight;
+ for (int i = 0; i < toSkip && descIter.hasNext(); i++) {
+ descIter.next();
+ }
+
+ // Collect the lines we need (in reverse order)
+ int currentIdx = totalLogs - toSkip - 1;
+ for (int i = 0; i < contentHeight && descIter.hasNext(); i++) {
+ tempLines.add(descIter.next());
+ tempIndices.add(currentIdx--);
+ }
+
+ // Reverse to get correct order and process
+ Collections.reverse(tempLines);
+ Collections.reverse(tempIndices);
+ for (int i = 0; i < tempLines.size(); i++) {
+ String line = tempLines.get(i);
+ int lineIdx = tempIndices.get(i);
+ result.add(formatLogLine(line, innerWidth, lineIdx, highlightPattern));
+ }
+ } else {
+ // We're closer to the start, use forward iterator
+ Iterator iter = logBuffer.iterator();
+
+ // Skip to start position
+ for (int i = 0; i < startIdx && iter.hasNext(); i++) {
+ iter.next();
+ }
+
+ // Collect the lines we need
+ for (int i = 0; i < contentHeight && iter.hasNext(); i++) {
+ String line = iter.next();
+ int lineIdx = startIdx + i;
+ result.add(formatLogLine(line, innerWidth, lineIdx, highlightPattern));
+ }
+ }
+ }
+ } finally {
+ logLock.readLock().unlock();
+ }
+
+ // Pad to fill height
+ while (result.size() < contentHeight) {
+ result.add(new AttributedString(""));
+ }
+
+ return result;
+ }
+
+ private AttributedString formatLogLine(String line, int maxWidth, int lineIndex, java.util.regex.Pattern highlightPattern) {
+ String logLine = fitLine(line, maxWidth);
+ AttributedStyle baseStyle = getLogStyle(logLine);
+
+ // Check if this line is a search match
+ boolean isMatch = searchMatches.contains(lineIndex);
+ boolean isCurrentMatch = (searchMode == SearchMode.NAVIGATING &&
+ !searchMatches.isEmpty() &&
+ searchMatches.get(currentSearchIndex) == lineIndex);
+
+ if (highlightPattern != null && isMatch) {
+ // Highlight matching portions
+ return highlightMatches(logLine, highlightPattern, baseStyle, isCurrentMatch);
+ } else if (isMatch) {
+ // Just show with base style if it's a match but no pattern
+ return new AttributedStringBuilder().style(baseStyle).append(logLine).toAttributedString();
+ } else {
+ return new AttributedStringBuilder().style(baseStyle).append(logLine).toAttributedString();
+ }
+ }
+
+ private AttributedString highlightMatches(String line, java.util.regex.Pattern pattern, AttributedStyle baseStyle, boolean isCurrentMatch) {
+ AttributedStringBuilder builder = new AttributedStringBuilder();
+ java.util.regex.Matcher matcher = pattern.matcher(line);
+
+ int lastEnd = 0;
+ while (matcher.find()) {
+ // Add text before match with base style
+ if (matcher.start() > lastEnd) {
+ builder.style(baseStyle).append(line.substring(lastEnd, matcher.start()));
+ }
+
+ // Add matched text with highlight
+ // Current match gets green background, other matches get yellow background
+ if (isCurrentMatch) {
+ builder.style(AttributedStyle.DEFAULT.bold().background(AttributedStyle.GREEN).foreground(AttributedStyle.BLACK))
+ .append(line.substring(matcher.start(), matcher.end()));
+ } else {
+ builder.style(AttributedStyle.DEFAULT.bold().background(AttributedStyle.YELLOW).foreground(AttributedStyle.BLACK))
+ .append(line.substring(matcher.start(), matcher.end()));
+ }
+ lastEnd = matcher.end();
+ }
+
+ // Add remaining text
+ if (lastEnd < line.length()) {
+ builder.style(baseStyle).append(line.substring(lastEnd));
+ }
+
+ return builder.toAttributedString();
+ }
+
+ private void renderStatusLine(List lines, int width) {
+ AttributedStringBuilder statusBar = new AttributedStringBuilder();
+
+ // Get current log level and total buffered logs
+ String logLevel = getCurrentLogLevel();
+ int totalBuffered = LogBuffer.getAllLogEntries().size();
+ int filtered = logBuffer.size();
+
+ // Build compact status line
+ statusBar.style(AttributedStyle.DEFAULT.foreground(AttributedStyle.CYAN))
+ .append(" Logs: " + filtered + "/" + totalBuffered)
+ .append(" | ")
+ .append("Level: ")
+ .style(AttributedStyle.DEFAULT.bold().foreground(AttributedStyle.YELLOW))
+ .append(logLevel)
+ .style(AttributedStyle.DEFAULT.foreground(AttributedStyle.CYAN))
+ .append(" (+/-)")
+ .append(" | ")
+ .append("Scroll: ")
+ .style(!isUserScrollingLogs ?
+ AttributedStyle.DEFAULT.bold().foreground(AttributedStyle.GREEN) :
+ AttributedStyle.DEFAULT.foreground(AttributedStyle.YELLOW))
+ .append(!isUserScrollingLogs ? "AUTO" : "MANUAL")
+ .style(AttributedStyle.DEFAULT.foreground(AttributedStyle.CYAN))
+ .append(" | ");
+
+ statusBar.append("AutoExit: ")
+ .style(autoExitEnabled ?
+ AttributedStyle.DEFAULT.bold().foreground(AttributedStyle.GREEN) :
+ AttributedStyle.DEFAULT.foreground(AttributedStyle.BRIGHT | AttributedStyle.BLACK))
+ .append(autoExitEnabled ? "ON" : "OFF")
+ .style(AttributedStyle.DEFAULT.foreground(AttributedStyle.CYAN))
+ .append(" | ")
+ .append("?: help");
+
+ // Pad to width
+ int currentLen = statusBar.toAttributedString().columnLength();
+ if (currentLen < width) {
+ statusBar.append(" ".repeat(width - currentLen));
+ }
+
+ lines.add(statusBar.toAttributedString());
+ }
+
+ private void renderHelpPanel(List lines, int width, int height) {
+ AttributedStringBuilder builder = new AttributedStringBuilder();
+
+ // Title
+ lines.add(new AttributedString(""));
+ builder.style(AttributedStyle.DEFAULT.bold().foreground(AttributedStyle.CYAN));
+ builder.append(center("KEYBOARD SHORTCUTS", width));
+ lines.add(builder.toAttributedString());
+ lines.add(new AttributedString(""));
+
+ // Help content
+ addHelpLine(lines, "Navigation", "", width);
+ addHelpLine(lines, " β / β", "Scroll log panel up/down", width);
+ addHelpLine(lines, " PgUp / PgDn", "Page up/down in logs", width);
+ addHelpLine(lines, " Shift+β / Shift+β", "Page up/down in logs (alternate)", width);
+ addHelpLine(lines, " Home", "Jump to top of logs, reset split", width);
+ addHelpLine(lines, " End", "Jump to bottom of logs", width);
+ lines.add(new AttributedString(""));
+
+ addHelpLine(lines, "Panel Controls", "", width);
+ addHelpLine(lines, " [ / ]", "Adjust split ratio between tasks and logs", width);
+ lines.add(new AttributedString(""));
+
+ addHelpLine(lines, "Log Filtering", "", width);
+ addHelpLine(lines, " ] (not in search)", "Increase verbosity (show more logs)", width);
+ addHelpLine(lines, " [ (not in search)", "Decrease verbosity (show fewer logs)", width);
+ lines.add(new AttributedString(""));
+
+ addHelpLine(lines, "Search", "", width);
+ addHelpLine(lines, " /", "Search logs with regex pattern", width);
+ addHelpLine(lines, " n / p", "Next/previous search match", width);
+ addHelpLine(lines, " [ / ]", "Change log level (works during search)", width);
+ addHelpLine(lines, " ESC", "Cancel search", width);
+ lines.add(new AttributedString(""));
+
+ addHelpLine(lines, "Exit Controls", "", width);
+ addHelpLine(lines, " q or ESC (single)", "Toggle auto-exit on/off", width);
+ addHelpLine(lines, " q/ESC (auto-exit ON", "Exits when all tasks finish", width);
+ addHelpLine(lines, " + tasks done)", "", width);
+ addHelpLine(lines, " q+q or ESC+ESC", "Double-tap (<300ms) to force exit anytime", width);
+ lines.add(new AttributedString(""));
+
+ builder = new AttributedStringBuilder();
+ builder.style(AttributedStyle.DEFAULT.foreground(AttributedStyle.CYAN));
+ builder.append(" ");
+ builder.style(AttributedStyle.DEFAULT.italic());
+ builder.append("Note: Auto-exit defaults to OFF. Panel stays open");
+ lines.add(builder.toAttributedString());
+ builder = new AttributedStringBuilder();
+ builder.style(AttributedStyle.DEFAULT.foreground(AttributedStyle.CYAN));
+ builder.append(" ");
+ builder.style(AttributedStyle.DEFAULT.italic());
+ builder.append("after tasks finish so you can review the results.");
+ lines.add(builder.toAttributedString());
+ lines.add(new AttributedString(""));
+
+ addHelpLine(lines, "Other", "", width);
+ addHelpLine(lines, " s", "Save screenshot to file", width);
+ addHelpLine(lines, " ?", "Show/hide this help", width);
+ lines.add(new AttributedString(""));
+
+ // Current status display
+ builder = new AttributedStringBuilder();
+ builder.style(AttributedStyle.DEFAULT.foreground(AttributedStyle.CYAN));
+ builder.append(" Current status: ");
+ builder.style(AttributedStyle.DEFAULT.bold().foreground(AttributedStyle.YELLOW));
+ builder.append("Scroll=");
+ builder.append(!isUserScrollingLogs ? "AUTO" : "MANUAL");
+ builder.append(", AutoExit=");
+ builder.style(autoExitEnabled ?
+ AttributedStyle.DEFAULT.bold().foreground(AttributedStyle.GREEN) :
+ AttributedStyle.DEFAULT.bold().foreground(AttributedStyle.YELLOW));
+ builder.append(autoExitEnabled ? "ON" : "OFF");
+
+ lines.add(builder.toAttributedString());
+ lines.add(new AttributedString(""));
+
+ // Bottom message
+ builder = new AttributedStringBuilder();
+ builder.style(AttributedStyle.DEFAULT.foreground(AttributedStyle.YELLOW));
+ builder.append(center("Press any key to close help", width));
+ lines.add(builder.toAttributedString());
+
+ // Pad to fill screen
+ while (lines.size() < height) {
+ lines.add(new AttributedString(""));
+ }
+ }
+
+ private void addHelpLine(List lines, String key, String description, int width) {
+ AttributedStringBuilder builder = new AttributedStringBuilder();
+ if (description.isEmpty()) {
+ // Section header
+ builder.style(AttributedStyle.DEFAULT.bold().foreground(AttributedStyle.YELLOW));
+ builder.append(" " + key);
+ } else {
+ // Regular help line
+ builder.style(AttributedStyle.DEFAULT.foreground(AttributedStyle.GREEN));
+ builder.append(String.format(" %-20s", key));
+ builder.style(AttributedStyle.DEFAULT.foreground(AttributedStyle.WHITE));
+ builder.append(" " + description);
+ }
+ lines.add(builder.toAttributedString());
+ }
+
+ private String getCurrentLogLevel() {
+ // Return the display level from LogBuffer
+ org.apache.logging.log4j.Level displayLevel = LogBuffer.getDisplayLevel();
+ return displayLevel != null ? displayLevel.name() : "INFO";
+ }
+
+ /**
+ * Cycles the display log level up or down through the standard Log4j2 levels.
+ * This changes the display filtering, not the appender level.
+ * @param increaseVerbosity true to show more logs (lower level), false to show fewer logs (higher level)
+ */
+ private void cycleLogLevel(boolean increaseVerbosity) {
+ try {
+ org.apache.logging.log4j.Level currentLevel = LogBuffer.getDisplayLevel();
+
+ // Find current level index
+ int currentIndex = 3; // Default to INFO
+ for (int i = 0; i < LOG_LEVELS.length; i++) {
+ if (LOG_LEVELS[i].equals(currentLevel)) {
+ currentIndex = i;
+ break;
+ }
+ }
+
+ // Calculate new index
+ int newIndex;
+ if (increaseVerbosity) {
+ newIndex = Math.max(0, currentIndex - 1); // Move towards ALL/TRACE
+ } else {
+ newIndex = Math.min(LOG_LEVELS.length - 1, currentIndex + 1); // Move towards OFF
+ }
+
+ // Set new display level (this will trigger refresh)
+ org.apache.logging.log4j.Level newLevel = LOG_LEVELS[newIndex];
+
+ // Add a pre-change message
+ addLogMessage("[SYSTEM] Changing display level from " + currentLevel.name() + " to " + newLevel.name() + "...");
+
+ // This will trigger refreshDisplayBuffer() via LogBuffer.setDisplayLevel()
+ LogBuffer.setDisplayLevel(newLevel);
+
+ // The refresh adds its own stats message, so we don't need to add another one here
+ } catch (Exception e) {
+ addLogMessage("[ERROR] Failed to change display level: " + e.getMessage());
+ }
+ }
+
+ /**
+ * Refreshes the display buffer by refiltering all log entries from LogBuffer
+ * based on the current display level. Rebuilds from the tail backwards to keep
+ * the most recent matching entries.
+ *
+ * @return statistics message about the refresh for logging
+ */
+ public String refreshDisplayBuffer() {
+ logLock.writeLock().lock();
+ try {
+ // Save current buffer size for debugging
+ int beforeSize = logBuffer.size();
+
+ // Clear current display buffer
+ logBuffer.clear();
+
+ // Collect all matching entries first
+ java.util.List matchingEntries = new java.util.ArrayList<>();
+ org.apache.logging.log4j.Level displayLevel = LogBuffer.getDisplayLevel();
+
+ // Get the backing buffer
+ java.util.Queue backingBuffer = LogBuffer.getAllLogEntries();
+
+ // CRITICAL DEBUG: Check if backing buffer is empty
+ if (backingBuffer == null) {
+ logBuffer.addLast("[ERROR] LogBuffer.getAllLogEntries() returned NULL!");
+ return "ERROR: Backing buffer is null";
+ }
+
+ if (backingBuffer.isEmpty()) {
+ logBuffer.addLast("[WARNING] LogBuffer backing buffer is EMPTY - no events have been captured yet");
+ return "WARNING: Backing buffer is empty (0 entries)";
+ }
+
+ // Count entries by level for debugging
+ int totalEntries = 0;
+ java.util.Map countByLevel = new java.util.LinkedHashMap<>();
+
+ for (LogBuffer.LogEntry entry : backingBuffer) {
+ totalEntries++;
+ String levelName = entry.level.name();
+ countByLevel.put(levelName, countByLevel.getOrDefault(levelName, 0) + 1);
+
+ // Debug first 3 entries to see what we're comparing
+ if (totalEntries <= 3) {
+ String debugMsg = String.format("[DEBUG] Entry %d: level=%s (%d) vs display=%s (%d), match=%b",
+ totalEntries, entry.level.name(), entry.level.intLevel(),
+ displayLevel.name(), displayLevel.intLevel(),
+ entry.level.intLevel() <= displayLevel.intLevel());
+ // Store for later display
+ matchingEntries.add(debugMsg);
+ }
+
+ // Add entry if its level is at or above display level
+ // Note: Lower intLevel = more severe (ERROR=200, WARN=300, INFO=400, DEBUG=500, TRACE=600)
+ // So to show INFO and above, we want intLevel <= INFO.intLevel()
+ if (entry.level.intLevel() <= displayLevel.intLevel()) {
+ matchingEntries.add(entry.formattedMessage);
+ }
+ }
+
+ // Keep only the tail (most recent entries) if we exceed maxLogLines
+ int startIndex = Math.max(0, matchingEntries.size() - maxLogLines);
+ for (int i = startIndex; i < matchingEntries.size(); i++) {
+ logBuffer.addLast(matchingEntries.get(i));
+ }
+
+ // Reset scroll if we were at the bottom
+ if (!isUserScrollingLogs) {
+ logScrollOffset = Math.max(0, logBuffer.size() - getLogPanelHeight());
+ }
+
+ // Build detailed stats message
+ StringBuilder stats = new StringBuilder();
+ stats.append(String.format("Buffer refresh: %dβ%d entries | Backing: %d total (",
+ beforeSize, logBuffer.size(), totalEntries));
+ boolean first = true;
+ for (java.util.Map.Entry e : countByLevel.entrySet()) {
+ if (!first) stats.append(", ");
+ stats.append(e.getKey()).append("=").append(e.getValue());
+ first = false;
+ }
+ stats.append(String.format(") | Level=%s | Matched=%d | Showing=%d",
+ displayLevel.name(), matchingEntries.size(), logBuffer.size()));
+
+ return stats.toString();
+ } finally {
+ logLock.writeLock().unlock();
+ }
+ }
+
+ private void renderTaskPanel(List lines, int width, int contentHeight) {
+ int innerWidth = Math.max(10, width - 4);
+
+ List taskEntries = new ArrayList<>();
+ collectTaskLines(rootNode, taskEntries, "", true, innerWidth);
+
+ if (taskEntries.isEmpty()) {
+ taskEntries.add(new SectionLine(center("No active tasks", innerWidth), STYLE_SECONDARY));
+ }
+
+ long active = 0;
+ long completed = 0;
+ long failed = 0;
+ for (TaskNode node : taskNodes.values()) {
+ if (node.finishTime > 0) {
+ if (node.lastStatus != null && node.lastStatus.runstate == RunState.FAILED) {
+ failed++;
+ } else {
+ completed++;
+ }
+ } else {
+ active++;
+ }
+ }
+
+ SectionLine summaryLine = new SectionLine(
+ String.format("Active: %d Completed: %d Failed: %d", active, completed, failed),
+ STYLE_SECONDARY);
+
+ int bodyLines = Math.max(0, contentHeight - 1);
+ int totalEntries = taskEntries.size();
+ int maxScrollStart = Math.max(0, totalEntries - bodyLines);
+ int startIdx = Math.max(0, Math.min(taskScrollOffset, maxScrollStart));
+ int endIdx = Math.min(totalEntries, startIdx + bodyLines);
+
+ List visibleBody = new ArrayList<>();
+ if (bodyLines > 0) {
+ visibleBody.addAll(taskEntries.subList(startIdx, endIdx));
+
+ if (totalEntries > bodyLines && !visibleBody.isEmpty()) {
+ String indicatorText = String.format("Tasks %d-%d of %d (PgUp/PgDn)",
+ startIdx + 1, endIdx, totalEntries);
+ SectionLine indicator = new SectionLine(indicatorText, STYLE_SECONDARY);
+ visibleBody.set(visibleBody.size() - 1, indicator);
+ }
+ }
+
+ renderBoxedSection(lines, "Active Tasks", visibleBody, summaryLine, width, contentHeight);
+ }
+
+ private void collectTaskLines(DisplayNode node, List lines, String prefix, boolean isLast, int innerWidth) {
+ // Format this node's line (skip for root)
+ if (!(node instanceof RootNode)) {
+ String nodeLine = formatNodeLine(node, prefix, isLast, innerWidth);
+ nodeLine = fitLine(nodeLine, innerWidth);
+ lines.add(new SectionLine(nodeLine, AttributedStyle.DEFAULT));
+ }
+
+ // Recursively add children
+ List children = new ArrayList<>(node.children);
+ for (int i = 0; i < children.size(); i++) {
+ DisplayNode child = children.get(i);
+ boolean childIsLast = (i == children.size() - 1);
+
+ String childPrefix = prefix;
+ if (!(node instanceof RootNode)) {
+ childPrefix += isLast ? " " : "β ";
+ }
+
+ collectTaskLines(child, lines, childPrefix, childIsLast, innerWidth);
+ }
+ }
+
+ private String formatNodeLine(DisplayNode node, String prefix, boolean isLast, int availableWidth) {
+ if (node instanceof TaskNode) {
+ return formatTaskLine((TaskNode) node, prefix, isLast, availableWidth);
+ } else if (node instanceof ScopeNode) {
+ return formatScopeLine((ScopeNode) node, prefix, isLast, availableWidth);
+ }
+ return "";
+ }
+
+ private String formatScopeLine(ScopeNode node, String prefix, boolean isLast, int availableWidth) {
+ StringBuilder line = new StringBuilder();
+
+ // Tree connector and base prefix
+ if (!prefix.isEmpty()) {
+ line.append(prefix);
+ line.append(isLast ? "ββ " : "ββ ");
+ }
+
+ // Scope icon
+ line.append(node.getSymbol()).append(" ");
+ line.append(node.getName());
+
+ // Add completion indicator for closed scopes
+ if (node.isComplete()) {
+ line.append(" β");
+ }
+
+ return line.toString();
+ }
+
+ private String formatTaskLine(TaskNode node, String prefix, boolean isLast, int availableWidth) {
+ StringBuilder rightPortion = new StringBuilder();
+
+ // Build the right-aligned portion (duration, then progress bar with percentage)
+
+ // Duration first - use elapsed running time if task is/was running
+ long duration;
+ if (node.tracker != null && node.tracker.getRunningStartTime() != null) {
+ // Task has started running - use actual running time
+ if (node.finishTime > 0) {
+ duration = node.finishTime - node.tracker.getRunningStartTime();
+ } else {
+ duration = node.tracker.getElapsedRunningTime();
+ }
+ } else if (node.lastStatus != null && node.lastStatus.runstate == RunState.PENDING) {
+ // Task hasn't started yet
+ duration = 0;
+ } else {
+ // Fallback to old calculation
+ duration = (node.finishTime > 0 ? node.finishTime : System.currentTimeMillis()) - node.startTime;
+ }
+ long seconds = Math.max(0, Math.round(duration / 1000.0));
+ rightPortion.append(String.format(" (%ds) ", seconds));
+
+ // Progress bar with percentage centered in it (fixed 22 characters total)
+ if (node.lastStatus != null) {
+ String progressBarWithPercent = createProgressBarWithCenteredPercent(node.lastStatus.progress);
+ rightPortion.append(progressBarWithPercent);
+ } else {
+ rightPortion.append("[ 0% ]");
+ }
+
+ // Completion marker
+ if (node.finishTime > 0 && node.lastStatus != null) {
+ if (node.lastStatus.runstate == RunState.SUCCESS) {
+ rightPortion.append(" β");
+ } else if (node.lastStatus.runstate == RunState.FAILED) {
+ rightPortion.append(" β");
+ }
+ }
+
+ StringBuilder line = new StringBuilder();
+
+ // Tree connector and base prefix
+ if (!prefix.isEmpty()) {
+ line.append(prefix);
+ line.append(isLast ? "ββ " : "ββ ");
+ }
+
+ // Status icon
+ line.append(node.getSymbol()).append(" ");
+
+ // Determine maximum available width for the task name before adding context/spaces
+ int maxNameWidth = Math.max(0, availableWidth - line.length() - rightPortion.length());
+ String taskName = node.getName();
+ if (taskName.length() > maxNameWidth) {
+ taskName = fitTaskName(taskName, maxNameWidth);
+ }
+ line.append(taskName);
+
+ // Calculate space for contextual details
+ int leftLength = line.length();
+ int rightLength = rightPortion.length();
+ int totalUsed = leftLength + rightLength;
+ int spacesNeeded = Math.max(1, availableWidth - totalUsed);
+
+ // Add contextual details in the middle if space allows
+ if (spacesNeeded > 5) {
+ StringBuilder context = new StringBuilder();
+
+ // Add task state if not running
+ if (node.lastStatus != null) {
+ if (node.lastStatus.runstate == RunState.PENDING) {
+ context.append(" [pending]");
+ } else if (node.lastStatus.runstate == RunState.RUNNING) {
+ // Add any additional context from the task if available
+ Object tracked = node.lastStatus.tracked;
+ if (tracked != null && tracked.toString().contains(":")) {
+ // Extract detail after colon if present
+ String detail = tracked.toString();
+ int colonIdx = detail.indexOf(":");
+ if (colonIdx >= 0 && colonIdx < detail.length() - 1) {
+ context.append(" -").append(detail.substring(colonIdx + 1).trim());
+ }
+ }
+ }
+ }
+
+ line.append(context);
+ spacesNeeded = Math.max(1, availableWidth - leftLength - context.length() - rightLength);
+ }
+
+ // Fill with spaces
+ for (int i = 0; i < spacesNeeded; i++) {
+ line.append(" ");
+ }
+
+ // Add right-aligned portion
+ line.append(rightPortion);
+
+ return line.toString();
+ }
+
+ private void renderLogPanel(List lines, Size size) {
+ renderLogPanel(lines, size.getColumns(), 10);
+ }
+
+ private void renderLogPanel(List lines, int width, int contentHeight) {
+ int innerWidth = Math.max(10, width - 4);
+ List logLines = new ArrayList<>();
+ SectionLine footerLine;
+
+ logLock.readLock().lock();
+ try {
+ int bodyLines = Math.max(0, contentHeight - 1);
+ int totalLogs = logBuffer.size();
+
+ int startIdx;
+ if (isUserScrollingLogs) {
+ int maxScrollStart = Math.max(0, totalLogs - bodyLines);
+ startIdx = Math.max(0, Math.min(logScrollOffset, maxScrollStart));
+ logScrollOffset = startIdx;
+ } else {
+ startIdx = Math.max(0, totalLogs - bodyLines);
+ logScrollOffset = startIdx;
+ }
+ int endIdx = Math.min(totalLogs, startIdx + bodyLines);
+
+ for (int i = startIdx; i < endIdx; i++) {
+ String logLine = logBuffer.get(i);
+ logLines.add(new SectionLine(fitLine(logLine, innerWidth), getLogStyle(logLine)));
+ }
+
+ String footerText;
+ if (totalLogs == 0) {
+ footerText = "Waiting for log outputβ¦";
+ } else if (bodyLines == 0) {
+ footerText = String.format("%d log lines (expand panel to view)", totalLogs);
+ } else if (totalLogs > bodyLines) {
+ if (isUserScrollingLogs) {
+ footerText = String.format("Logs %d-%d of %d (β/β to scroll)",
+ startIdx + 1, endIdx, totalLogs);
+ } else {
+ footerText = String.format("LIVE showing last %d of %d lines (β to scroll)",
+ Math.max(0, endIdx - startIdx), totalLogs);
+ }
+ } else {
+ footerText = String.format("Showing all %d log lines", totalLogs);
+ }
+
+ footerLine = new SectionLine(fitLine(footerText, innerWidth), STYLE_SECONDARY);
+ lastLogDisplayTime = System.currentTimeMillis();
+ } finally {
+ logLock.readLock().unlock();
+ }
+
+ renderBoxedSection(lines, "Console Output", logLines, footerLine, width, contentHeight);
+ }
+
+ private void renderBoxedSection(List target, String title, List body,
+ SectionLine footer, int width, int contentHeight) {
+ int adjustedHeight = Math.max(1, contentHeight);
+ int innerWidth = Math.max(10, width - 4);
+ target.add(buildSectionBorder('β', 'β', title, width));
+
+ int rowsRendered = 0;
+ int bodyLines = adjustedHeight - (footer != null ? 1 : 0);
+ if (bodyLines < 0) {
+ bodyLines = 0;
+ }
+
+ for (int i = 0; i < bodyLines; i++) {
+ SectionLine line = (i < body.size()) ? body.get(i) : null;
+ target.add(renderBoxLine(line, innerWidth));
+ rowsRendered++;
+ }
+
+ if (footer != null) {
+ target.add(renderBoxLine(footer, innerWidth));
+ rowsRendered++;
+ }
+
+ while (rowsRendered < adjustedHeight) {
+ target.add(renderBoxLine(null, innerWidth));
+ rowsRendered++;
+ }
+
+ target.add(buildSectionBorder('β', 'β', null, width));
+ }
+
+ private AttributedString renderBoxLine(SectionLine line, int innerWidth) {
+ String text = line != null ? line.text : "";
+ String fitted = fitLine(text, innerWidth);
+ int padding = Math.max(0, innerWidth - fitted.length());
+
+ AttributedStringBuilder builder = new AttributedStringBuilder();
+ builder.style(STYLE_BORDER).append("β ");
+ if (line != null && line.style != null) {
+ builder.style(line.style);
+ } else {
+ builder.style(AttributedStyle.DEFAULT);
+ }
+ builder.append(fitted);
+ builder.style(STYLE_BORDER).append(" ".repeat(padding)).append(" β");
+ return builder.toAttributedString();
+ }
+
+ private AttributedString buildSectionBorder(char left, char right, String title, int width) {
+ int innerWidth = Math.max(0, width - 2);
+ AttributedStringBuilder builder = new AttributedStringBuilder();
+
+ // Apply bright cyan bold style for borders
+ builder.style(STYLE_BORDER);
+ builder.append(String.valueOf(left));
+
+ if (title != null && !title.isEmpty()) {
+ String trimmed = title.trim();
+ if (trimmed.length() > innerWidth - 4) {
+ trimmed = fitLine(trimmed, innerWidth - 4);
+ }
+
+ int titleLen = trimmed.length() + 4; // Account for spaces and equals
+ int remaining = Math.max(0, innerWidth - titleLen);
+ int leftPad = remaining / 2;
+ int rightPad = remaining - leftPad;
+
+ // Left padding with double lines
+ for (int i = 0; i < leftPad; i++) {
+ builder.append("β");
+ }
+
+ // Title with yellow highlight
+ builder.append("β");
+ builder.style(STYLE_BORDER_TITLE).append(" " + trimmed + " ");
+ builder.style(STYLE_BORDER).append("β");
+
+ // Right padding with double lines
+ for (int i = 0; i < rightPad; i++) {
+ builder.append("β");
+ }
+ } else {
+ // Fill with double lines
+ for (int i = 0; i < innerWidth; i++) {
+ builder.append("β");
+ }
+ }
+
+ builder.append(String.valueOf(right));
+ return builder.toAttributedString();
+ }
+
+ private String fitLine(String text, int maxWidth) {
+ if (text == null) {
+ return "";
+ }
+ if (text.length() <= maxWidth) {
+ return text;
+ }
+ if (maxWidth <= 1) {
+ return text.substring(0, Math.max(0, maxWidth));
+ }
+ return text.substring(0, Math.max(0, maxWidth - 1)) + "β¦";
+ }
+
+ private int getLogPanelHeight() {
+ Size size = terminal.getSize();
+ int taskLines = countTaskLines(rootNode);
+ int headerFooterLines = 6; // Headers and footers
+ int remaining = size.getRows() - taskLines - headerFooterLines;
+ return Math.max(5, Math.min(remaining, 10));
+ }
+
+ private static final class SectionLine {
+ final String text;
+ final AttributedStyle style;
+
+ SectionLine(String text, AttributedStyle style) {
+ this.text = text == null ? "" : text;
+ this.style = style;
+ }
+ }
+
+ private int countTaskLines(DisplayNode node) {
+ int count = (node instanceof RootNode) ? 0 : 1;
+ for (DisplayNode child : node.children) {
+ count += countTaskLines(child);
+ }
+ return count;
+ }
+
+ private AttributedStyle getLogStyle(String logLine) {
+ String upper = logLine.toUpperCase();
+ if (upper.contains("[ERROR]") || upper.contains("ERROR") || upper.contains("SEVERE")) {
+ return STYLE_LOG_ERROR;
+ } else if (upper.contains("[WARN]") || upper.contains("WARNING")) {
+ return STYLE_LOG_WARN;
+ } else if (upper.contains("[DEBUG]") || upper.contains("TRACE")) {
+ return STYLE_LOG_DEBUG;
+ } else {
+ return STYLE_LOG_INFO;
+ }
+ }
+
+ private String center(String text, int width) {
+ int padding = (width - text.length()) / 2;
+ return " ".repeat(Math.max(0, padding)) + text + " ".repeat(Math.max(0, width - text.length() - padding));
+ }
+
+ private void renderStatusBar(List lines, int width) {
+ // Add bottom border first
+ lines.add(buildSectionBorder('β', 'β', null, width));
+
+ // Create bottom status bar
+ AttributedStringBuilder statusBar = new AttributedStringBuilder();
+
+ // Get current time
+ String timeStr = LocalDateTime.now().format(DateTimeFormatter.ofPattern("HH:mm:ss"));
+
+ // Count active tasks
+ long activeTasks = taskNodes.values().stream()
+ .filter(n -> n.lastStatus != null && n.lastStatus.runstate == RunState.RUNNING)
+ .count();
+ long completedTasks = taskNodes.values().stream()
+ .filter(n -> n.lastStatus != null && n.lastStatus.runstate == RunState.SUCCESS)
+ .count();
+
+ // Build status line
+ statusBar.style(AttributedStyle.DEFAULT.background(AttributedStyle.BLUE).foreground(AttributedStyle.WHITE))
+ .append(" ")
+ .append(timeStr)
+ .append(" β ");
+
+ statusBar.style(AttributedStyle.DEFAULT.background(AttributedStyle.BLUE).foreground(AttributedStyle.YELLOW))
+ .append("Active: ").append(String.valueOf(activeTasks))
+ .append(" ");
+
+ statusBar.style(AttributedStyle.DEFAULT.background(AttributedStyle.BLUE).foreground(AttributedStyle.GREEN))
+ .append("Complete: ").append(String.valueOf(completedTasks))
+ .append(" ");
+
+ statusBar.style(AttributedStyle.DEFAULT.background(AttributedStyle.BLUE).foreground(AttributedStyle.WHITE))
+ .append("β ")
+ .append(isUserScrollingLogs ? "ββ: Scroll Logs β" : "β: Scroll Logs β");
+ statusBar.append(" PgUp/PgDn: Adjust Split β q: Quit");
+
+ // Pad to full width
+ int currentLen = statusBar.toAttributedString().columnLength();
+ if (currentLen < width) {
+ statusBar.append(" ".repeat(width - currentLen));
+ }
+
+ lines.add(statusBar.toAttributedString());
+ }
+
+
+ private String createProgressBar(double progress) {
+ int barLength = 20;
+
+ // Braille patterns for 1/8 increments
+ char[] brailleProgress = {
+ ' ', // 0/8 - empty
+ 'β‘', // 1/8
+ 'β‘', // 2/8
+ 'β‘', // 3/8
+ 'β‘', // 4/8
+ 'β£', // 5/8
+ 'β£§', // 6/8
+ 'β£·', // 7/8
+ };
+ char fullBlock = 'β£Ώ'; // 8/8 - full
+
+ // Calculate progress in terms of 1/8 increments
+ double totalEighths = barLength * 8.0 * progress;
+ int fullChars = (int) (totalEighths / 8);
+ int remainder = (int) (totalEighths % 8);
+
+ StringBuilder bar = new StringBuilder("[");
+
+ for (int i = 0; i < barLength; i++) {
+ if (i < fullChars) {
+ bar.append(fullBlock);
+ } else if (i == fullChars && remainder > 0) {
+ bar.append(brailleProgress[remainder]);
+ } else {
+ bar.append(' ');
+ }
+ }
+
+ bar.append("]");
+ return bar.toString();
+ }
+
+ private String createProgressBarWithCenteredPercent(double progress) {
+ int barLength = 20; // Total bar length
+ String percentStr = String.format("%3.0f%%", progress * 100);
+ int percentLen = percentStr.length();
+
+ // Braille patterns for 1/8 increments (0/8 to 7/8 filled)
+ // Using vertical Braille patterns that fill from left to right
+ char[] brailleProgress = {
+ ' ', // 0/8 - empty
+ 'β‘', // 1/8
+ 'β‘', // 2/8
+ 'β‘', // 3/8
+ 'β‘', // 4/8
+ 'β£', // 5/8
+ 'β£§', // 6/8
+ 'β£·', // 7/8
+ };
+ char fullBlock = 'β£Ώ'; // 8/8 - full
+
+ // Calculate progress in terms of 1/8 increments
+ double totalEighths = barLength * 8.0 * progress;
+ int fullChars = (int) (totalEighths / 8);
+ int remainder = (int) (totalEighths % 8);
+
+ // Calculate where to place the percentage (centered)
+ int percentStart = (barLength - percentLen) / 2;
+
+ StringBuilder bar = new StringBuilder("[");
+
+ for (int i = 0; i < barLength; i++) {
+ // Check if we should insert percentage text here
+ if (i >= percentStart && i < percentStart + percentLen) {
+ bar.append(percentStr.charAt(i - percentStart));
+ } else if (i < fullChars) {
+ bar.append(fullBlock);
+ } else if (i == fullChars && remainder > 0) {
+ bar.append(brailleProgress[remainder]);
+ } else {
+ bar.append(' ');
+ }
+ }
+
+ bar.append("]");
+ return bar.toString();
+ }
+
+ private String getTaskName(StatusTracker> tracker, int maxWidth) {
+ String fullName = StatusTracker.extractTaskName(tracker);
+ return fitTaskName(fullName, maxWidth);
+ }
+
+ private String fitTaskName(String name, int maxWidth) {
+ if (maxWidth <= 0) {
+ return "";
+ }
+
+ if (name.length() <= maxWidth) {
+ return name;
+ }
+
+ if (maxWidth <= 3) {
+ return name.substring(0, maxWidth);
+ }
+
+ return name.substring(0, maxWidth - 3) + "...";
+ }
+
+ public boolean isClosed() {
+ return closed.get();
+ }
+
+ @Override
+ public void close() {
+ if (closed.compareAndSet(false, true)) {
+ performCleanup();
+ }
+ }
+
+ private void performCleanup() {
+ // Clear the active sink
+ LogBuffer.clearActiveSink();
+
+ // Capture current log buffer state before closing
+ List logsSnapshot;
+ logLock.readLock().lock();
+ try {
+ logsSnapshot = new ArrayList<>(logBuffer);
+ } finally {
+ logLock.readLock().unlock();
+ }
+
+ // Stop the render thread first
+ try {
+ renderThread.join(2000); // Wait up to 2 seconds for clean shutdown
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ }
+
+ try {
+ // Clear the display completely and reset terminal
+ display.update(Collections.emptyList(), 0);
+
+ // Clear screen and reset cursor
+ terminal.puts(org.jline.utils.InfoCmp.Capability.clear_screen);
+ terminal.puts(org.jline.utils.InfoCmp.Capability.cursor_home);
+
+ // Reset all terminal attributes
+ terminal.writer().print("\033[0m"); // Reset colors
+ terminal.writer().print("\033[?25h"); // Show cursor
+ terminal.writer().print("\033[?1049l"); // Exit alternate screen (if used)
+ terminal.writer().flush();
+
+ // Exit raw mode to restore normal terminal behavior
+ org.jline.terminal.Attributes attrs = terminal.getAttributes();
+ attrs.setLocalFlag(org.jline.terminal.Attributes.LocalFlag.ICANON, true);
+ attrs.setLocalFlag(org.jline.terminal.Attributes.LocalFlag.ECHO, true);
+ terminal.setAttributes(attrs);
+
+ terminal.flush();
+ terminal.close();
+ } catch (Exception e) {
+ logger.error("Error during terminal cleanup", e);
+ // Force terminal restoration even if normal cleanup failed
+ try {
+ System.out.print("\033[0m"); // Reset colors
+ System.out.print("\033[?25h"); // Show cursor
+ System.out.flush();
+ } catch (Exception ignored) {}
+ } finally {
+ // Always restore original streams, even if terminal cleanup fails
+ System.setOut(originalOut);
+ System.setErr(originalErr);
+
+ // Force terminal reset using direct ANSI codes to stdout
+ try {
+ originalOut.print("\033[0m"); // Reset all attributes
+ originalOut.print("\033[?25h"); // Show cursor
+ originalOut.print("\033[?1049l"); // Exit alternate screen
+ originalOut.print("\033c"); // Reset terminal (RIS)
+ originalOut.flush();
+ } catch (Exception ignored) {}
+
+ // Print a newline to ensure clean prompt
+ originalOut.println();
+
+ // Dump the log buffer to stdout for user context
+ if (!logsSnapshot.isEmpty()) {
+ originalOut.println("=== Console Log History ===");
+ for (String log : logsSnapshot) {
+ originalOut.println(log);
+ }
+ originalOut.println("=== End Console Log History ===");
+ }
+
+ originalOut.flush();
+ }
+ }
+
+
+ /**
+ * Custom PrintStream that captures output and adds it to the log buffer
+ */
+ private class LogCapturePrintStream extends PrintStream {
+ private final String prefix;
+ private final ByteArrayOutputStream pendingBytes;
+
+ LogCapturePrintStream(String prefix) {
+ super(new ByteArrayOutputStream());
+ this.prefix = prefix;
+ this.pendingBytes = new ByteArrayOutputStream();
+ }
+
+ @Override
+ public synchronized void println(String x) {
+ writeByteArray((x == null ? "null" : x).getBytes(StandardCharsets.UTF_8));
+ write('\n');
+ }
+
+ @Override
+ public synchronized void println() {
+ write('\n');
+ }
+
+ @Override
+ public synchronized void print(String s) {
+ writeByteArray((s == null ? "null" : s).getBytes(StandardCharsets.UTF_8));
+ }
+
+ @Override
+ public synchronized void print(char[] s) {
+ if (s == null) {
+ writeByteArray("null".getBytes(StandardCharsets.UTF_8));
+ } else {
+ writeByteArray(new String(s).getBytes(StandardCharsets.UTF_8));
+ }
+ }
+
+ @Override
+ public synchronized void println(char[] s) {
+ print(s);
+ write('\n');
+ }
+
+ @Override
+ public synchronized void write(byte[] buf, int off, int len) {
+ if (buf == null || len <= 0) {
+ return;
+ }
+
+ int end = off + len;
+ for (int i = off; i < end; i++) {
+ byte b = buf[i];
+ if (b == '\n' || b == '\r') {
+ flushPendingBytes();
+ } else {
+ pendingBytes.write(b);
+ }
+ }
+ }
+
+ @Override
+ public synchronized void write(int b) {
+ byte value = (byte) b;
+ if (value == '\n' || value == '\r') {
+ flushPendingBytes();
+ } else {
+ pendingBytes.write(value);
+ }
+ }
+
+ @Override
+ public synchronized void flush() {
+ flushPendingBytes();
+ }
+
+ private void writeByteArray(byte[] data) {
+ if (data == null || data.length == 0) {
+ return;
+ }
+ write(data, 0, data.length);
+ }
+
+ private void flushPendingBytes() {
+ if (pendingBytes.size() == 0) {
+ return;
+ }
+
+ String line = new String(pendingBytes.toByteArray(), StandardCharsets.UTF_8);
+ pendingBytes.reset();
+ emitLine(line);
+ }
+
+ private void emitLine(String line) {
+ if (line == null || line.trim().isEmpty()) {
+ return;
+ }
+
+ logLock.writeLock().lock();
+ try {
+ String decorated = line;
+ if (!decorated.matches("^\\[\\d{2}:\\d{2}:\\d{2}\\].*")) {
+ decorated = "[" + LocalDateTime.now().format(timeFormatter) + "] " + decorated;
+ }
+
+ if (prefix != null && !prefix.isEmpty()) {
+ if (decorated.startsWith("[") && decorated.indexOf(']') != -1) {
+ int closing = decorated.indexOf(']');
+ decorated = decorated.substring(0, closing + 1) + " [" + prefix + "]" + decorated.substring(closing + 1);
+ } else {
+ decorated = "[" + prefix + "] " + decorated;
+ }
+ }
+
+ logBuffer.addLast(decorated);
+
+ while (logBuffer.size() > maxLogLines) {
+ logBuffer.removeFirst();
+ // When manually scrolling, we need to adjust offset to maintain the same view
+ // Even if offset is 0, removing from front means we need to "scroll up" to stay in place
+ // However, we can't have negative offset, so content will shift if at the very top
+ if (isUserScrollingLogs && logScrollOffset > 0) {
+ logScrollOffset--;
+ } else if (!isUserScrollingLogs) {
+ // When auto-scrolling, decrement to maintain bottom view
+ if (logScrollOffset > 0) {
+ logScrollOffset--;
+ }
+ }
+ }
+
+ if (!isUserScrollingLogs) {
+ int maxScroll = Math.max(0, logBuffer.size() - getLogPanelHeight());
+ logScrollOffset = maxScroll;
+ }
+ } finally {
+ logLock.writeLock().unlock();
+ }
+ }
+ }
+
+ private void saveScreenshot() {
+ List snapshot = lastRenderSnapshot;
+ if (snapshot == null || snapshot.isEmpty()) {
+ addLogMessage("No screen content available to save.");
+ return;
+ }
+
+ String timestamp = LocalDateTime.now().format(SCREENSHOT_FORMAT);
+ Path path = Paths.get(String.format("console-panel-%s.txt", timestamp));
+ try {
+ Files.write(path, snapshot, StandardCharsets.UTF_8, StandardOpenOption.CREATE_NEW);
+ addLogMessage("Saved console snapshot to " + path.toAbsolutePath());
+ } catch (IOException e) {
+ addLogMessage("Failed to save console snapshot: " + e.getMessage());
+ }
+ }
+
+ /**
+ * Base class for display hierarchy nodes (scopes and tasks)
+ */
+ private static abstract class DisplayNode {
+ final DisplayNode parent;
+ final List children;
+ final long startTime;
+
+ DisplayNode(DisplayNode parent) {
+ this.parent = parent;
+ this.children = Collections.synchronizedList(new ArrayList<>());
+ this.startTime = System.currentTimeMillis();
+ }
+
+ abstract String getName();
+ abstract String getSymbol();
+ abstract boolean isComplete();
+ }
+
+ /**
+ * Root node containing all top-level scopes and trackers
+ */
+ private static class RootNode extends DisplayNode {
+ RootNode() {
+ super(null);
+ }
+
+ @Override
+ String getName() {
+ return "Root";
+ }
+
+ @Override
+ String getSymbol() {
+ return "";
+ }
+
+ @Override
+ boolean isComplete() {
+ return children.stream().allMatch(DisplayNode::isComplete);
+ }
+ }
+
+ /**
+ * Node representing an organizational scope
+ */
+ private static class ScopeNode extends DisplayNode {
+ final StatusScope scope;
+ long finishTime;
+
+ ScopeNode(StatusScope scope, DisplayNode parent) {
+ super(parent);
+ this.scope = scope;
+ this.finishTime = 0;
+ }
+
+ @Override
+ String getName() {
+ return scope.toString(); // Uses scope's toString with task counts
+ }
+
+ @Override
+ String getSymbol() {
+ return "π"; // Folder icon for scopes
+ }
+
+ @Override
+ boolean isComplete() {
+ return finishTime > 0 || scope.isClosed() || scope.isComplete();
+ }
+ }
+
+ /**
+ * Node representing an actual task being tracked
+ */
+ private static class TaskNode extends DisplayNode {
+ final StatusTracker> tracker;
+ StatusUpdate> lastStatus;
+ long lastUpdateTime;
+ long finishTime;
+
+ TaskNode(StatusTracker> tracker, DisplayNode parent) {
+ super(parent);
+ this.tracker = tracker;
+ this.lastUpdateTime = startTime;
+ this.finishTime = 0;
+ }
+
+ @Override
+ String getName() {
+ return StatusTracker.extractTaskName(tracker);
+ }
+
+ @Override
+ String getSymbol() {
+ if (lastStatus == null) {
+ return "β"; // Pending
+ }
+ switch (lastStatus.runstate) {
+ case PENDING: return "β";
+ case RUNNING: return "βΆ";
+ case SUCCESS: return "β";
+ case FAILED: return "β";
+ case CANCELLED: return "βΌ";
+ default: return "?";
+ }
+ }
+
+ @Override
+ boolean isComplete() {
+ return finishTime > 0;
+ }
+ }
+
+ /**
+ * Runnable implementation for JVM shutdown hook that ensures proper terminal cleanup.
+ * This named type improves stack trace clarity during shutdown sequences and makes
+ * debugging shutdown-related issues easier to diagnose.
+ */
+ private static final class ShutdownCleanupRunnable implements Runnable {
+ private final ConsolePanelSink sink;
+
+ ShutdownCleanupRunnable(ConsolePanelSink sink) {
+ this.sink = sink;
+ }
+
+ @Override
+ public void run() {
+ if (!sink.closed.get()) {
+ sink.close();
+ }
+ }
+ }
+
+ /**
+ * Create a new builder for configuring ConsolePanelSink.
+ * @return a new Builder instance
+ */
+ public static Builder builder() {
+ return new Builder();
+ }
+
+ /**
+ * Builder for configuring {@link ConsolePanelSink} instances with fluent method chaining.
+ * Provides control over refresh rates, task retention, logging behavior, keyboard handlers,
+ * and visual presentation.
+ *
+ * Default Configuration:
+ *
+ * - Refresh rate: 250ms
+ * - Completed task retention: 5000ms (5 seconds)
+ * - Color output: enabled
+ * - Max log lines: 1000
+ * - System stream capture: disabled
+ * - Auto-exit on 'q': disabled
+ * - Custom keyboard handlers: none
+ *
+ *
+ * Example:
+ * {@code
+ * ConsolePanelSink sink = ConsolePanelSink.builder()
+ * .withRefreshRateMs(100)
+ * .withCompletedTaskRetention(3, TimeUnit.SECONDS)
+ * .withColorOutput(true)
+ * .withMaxLogLines(500)
+ * .withCaptureSystemStreams(true)
+ * .withAutoExit(true)
+ * .withKeyHandler("shift-right", () -> handleSpeedUp())
+ * .build();
+ * }
+ *
+ * @see ConsolePanelSink
+ */
+ public static class Builder {
+ private long refreshRateMs = 250;
+ private long completedRetentionMs = 5000;
+ private boolean useColors = true;
+ private int maxLogLines = 1000;
+ private boolean captureSystemStreams = false;
+ private boolean autoExit = false; // Default to false - user must enable
+ private Map customKeyHandlers = new HashMap<>();
+
+ public Builder withRefreshRate(long duration, TimeUnit unit) {
+ this.refreshRateMs = unit.toMillis(duration);
+ return this;
+ }
+
+ public Builder withCompletedTaskRetention(long duration, TimeUnit unit) {
+ this.completedRetentionMs = unit.toMillis(duration);
+ return this;
+ }
+
+ public Builder withColorOutput(boolean useColors) {
+ this.useColors = useColors;
+ return this;
+ }
+
+ public Builder withMaxLogLines(int maxLogLines) {
+ this.maxLogLines = maxLogLines;
+ return this;
+ }
+
+ public Builder withCaptureSystemStreams(boolean capture) {
+ this.captureSystemStreams = capture;
+ return this;
+ }
+
+ public Builder withRefreshRateMs(long refreshRateMs) {
+ this.refreshRateMs = refreshRateMs;
+ return this;
+ }
+
+ /**
+ * Configures whether the panel should exit automatically when 'q' is pressed.
+ * When false, the panel will remain open even when 'q' is pressed, requiring
+ * external shutdown (e.g., via close() or application exit).
+ *
+ * @param autoExit true (default) to allow 'q' to exit, false to disable auto-exit
+ * @return this builder
+ */
+ public Builder withAutoExit(boolean autoExit) {
+ this.autoExit = autoExit;
+ return this;
+ }
+
+ /**
+ * Registers a custom keyboard handler for the specified key combination.
+ * This allows applications to extend the interactive capabilities of the
+ * console panel with application-specific controls.
+ *
+ * The handler is invoked synchronously on the render thread when the
+ * corresponding key combination is detected. Handlers should execute quickly
+ * to avoid blocking the UI. For longer operations, consider spawning a
+ * separate thread from within the handler.
+ *
+ *
Supported Key Combinations:
+ *
+ * "shift-left"
- Shift + Left Arrow key
+ * "shift-right"
- Shift + Right Arrow key
+ *
+ *
+ * Example Usage:
+ * {@code
+ * SimulatedClock clock = new SimulatedClock();
+ * ConsolePanelSink sink = ConsolePanelSink.builder()
+ * .withKeyHandler("shift-right", () -> {
+ * clock.speedUp();
+ * sink.addLogMessage("Simulation speed: " + clock.getSpeedDescription());
+ * })
+ * .withKeyHandler("shift-left", () -> {
+ * clock.slowDown();
+ * sink.addLogMessage("Simulation speed: " + clock.getSpeedDescription());
+ * })
+ * .build();
+ * }
+ *
+ * Thread Safety: Handlers are executed on the render thread,
+ * so any shared state accessed by the handler should be properly synchronized.
+ *
+ * @param key the key combination identifier (case-sensitive)
+ * @param handler the runnable to execute when the key is pressed; must not be null
+ * @return this builder for method chaining
+ * @throws NullPointerException if handler is null
+ */
+ public Builder withKeyHandler(String key, Runnable handler) {
+ this.customKeyHandlers.put(key, handler);
+ return this;
+ }
+
+ public ConsolePanelSink build() {
+ return new ConsolePanelSink(this);
+ }
+ }
+}
diff --git a/internal-apis/src/main/java/io/github/jbellis/jvector/status/sinks/LogBuffer.java b/internal-apis/src/main/java/io/github/jbellis/jvector/status/sinks/LogBuffer.java
new file mode 100644
index 000000000..0822303a7
--- /dev/null
+++ b/internal-apis/src/main/java/io/github/jbellis/jvector/status/sinks/LogBuffer.java
@@ -0,0 +1,279 @@
+/*
+ * Copyright DataStax, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package io.github.jbellis.jvector.status.sinks;
+
+import org.apache.logging.log4j.Level;
+import org.apache.logging.log4j.core.LogEvent;
+import org.apache.logging.log4j.core.Layout;
+import org.apache.logging.log4j.core.appender.AbstractAppender;
+import org.apache.logging.log4j.core.config.plugins.Plugin;
+import org.apache.logging.log4j.core.config.plugins.PluginAttribute;
+import org.apache.logging.log4j.core.config.plugins.PluginElement;
+import org.apache.logging.log4j.core.config.plugins.PluginFactory;
+import org.apache.logging.log4j.core.layout.PatternLayout;
+
+import java.io.Serializable;
+import java.util.Objects;
+import java.util.Queue;
+import java.util.concurrent.ConcurrentLinkedQueue;
+
+/**
+ * Log4j 2 appender that captures and forwards log events to the active {@link ConsolePanelSink}.
+ * This appender integrates the logging framework with the interactive console display, allowing
+ * log messages from any logger to appear in the console panel's scrollable log section.
+ *
+ * Key Responsibilities:
+ *
+ * - Event Capture: Intercepts all log events routed through Log4j 2
+ * - Message Formatting: Formats log events with level, logger name, and message
+ * - Buffering: Queues messages when no active sink is available (up to 1000 messages)
+ * - Forwarding: Delivers formatted messages to the active {@link ConsolePanelSink}
+ *
+ *
+ * Integration Pattern
+ * This appender is installed by {@link ConsolePanelLogIntercept} when
+ * configuring for interactive mode:
+ * {@code
+ * // Automatic installation
+ * LoggerConfig.configure(OutputMode.INTERACTIVE);
+ *
+ * // Create ConsolePanelSink - it registers itself as the active sink
+ * ConsolePanelSink sink = ConsolePanelSink.builder().build();
+ *
+ * // All logging now flows to the console panel
+ * Logger logger = LogManager.getLogger(MyClass.class);
+ * logger.info("This appears in the console panel");
+ * }
+ *
+ * Lifecycle Management
+ * The appender coordinates with {@link ConsolePanelSink} through static methods:
+ *
+ * - {@link #setActiveSink(ConsolePanelSink)} - Called when sink is created; flushes buffer
+ * - {@link #clearActiveSink()} - Called when sink is closed; resumes buffering
+ *
+ *
+ * Message Buffering
+ * When no sink is active, messages are buffered in memory (max 1000 entries). When a sink
+ * becomes active, the buffer is flushed to the sink. This ensures log messages generated during
+ * application startup are not lost.
+ *
+ * Message Format
+ * Log events are formatted as:
+ * [LEVEL] LoggerName - Message
+ * For example:
+ * [INFO ] DemoTask - Starting task: DataLoad
+ *
+ * Thread Safety
+ * This appender is thread-safe through the use of {@link java.util.concurrent.ConcurrentLinkedQueue}
+ * for buffering and volatile references for the active sink.
+ *
+ * Performance Considerations
+ * The appender uses a lock-free queue for high-throughput logging scenarios. However, when
+ * forwarding to {@link ConsolePanelSink}, the sink's thread-safe methods are called, which may
+ * introduce synchronization overhead.
+ *
+ * @see ConsolePanelSink
+ * @see ConsolePanelLogIntercept
+ * @see OutputMode
+ * @since 4.0.0
+ */
+@Plugin(name = "LogBuffer", category = "Core", elementType = "appender", printObject = true)
+public class LogBuffer extends AbstractAppender {
+
+ /**
+ * Represents a captured log entry with its level and formatted message.
+ */
+ public static class LogEntry {
+ public final Level level;
+ public final String formattedMessage;
+
+ public LogEntry(Level level, String formattedMessage) {
+ this.level = level;
+ this.formattedMessage = formattedMessage;
+ }
+ }
+
+ private static volatile ConsolePanelSink activeSink;
+ private static final Queue allLogEntries = new ConcurrentLinkedQueue<>();
+ private static final Queue bufferedMessages = new ConcurrentLinkedQueue<>();
+ private static final int MAX_BUFFER_SIZE = 10000;
+ private static volatile Level displayLevel = Level.INFO;
+ private static final long START_TIME = System.currentTimeMillis();
+
+ protected LogBuffer(String name, Layout extends Serializable> layout) {
+ super(name,
+ null,
+ Objects.requireNonNullElse(layout, PatternLayout.newBuilder().withPattern("%msg").build()),
+ false,
+ null);
+ }
+
+ /**
+ * Log4j 2 plugin factory method for creating LogBuffer appenders from configuration.
+ * This method is called by Log4j 2 when the appender is configured in log4j2.xml or
+ * programmatically via the Configuration API.
+ *
+ * @param name the name of the appender instance
+ * @param layout the layout for formatting log events (optional, defaults to simple pattern)
+ * @return a started LogBuffer appender instance
+ */
+ @PluginFactory
+ public static LogBuffer createAppender(@PluginAttribute("name") String name,
+ @PluginElement("Layout") Layout extends Serializable> layout) {
+ LogBuffer appender = new LogBuffer(Objects.requireNonNullElse(name, "LogBuffer"), layout);
+ appender.start();
+ return appender;
+ }
+
+ /**
+ * Convenience factory method for creating a LogBuffer with default layout.
+ *
+ * @param name the name of the appender instance
+ * @return a started LogBuffer appender instance with default pattern layout
+ */
+ public static LogBuffer createAppender(String name) {
+ return createAppender(name, null);
+ }
+
+ /**
+ * Registers a {@link ConsolePanelSink} as the active sink for receiving log messages.
+ * When a sink is set, all buffered messages are immediately flushed to the sink.
+ * This method is called automatically by {@link ConsolePanelSink} during initialization.
+ *
+ * @param sink the console panel sink to receive log messages, or null to clear
+ */
+ public static void setActiveSink(ConsolePanelSink sink) {
+ activeSink = sink;
+ if (sink != null) {
+ String msg;
+ while ((msg = bufferedMessages.poll()) != null) {
+ sink.addLogMessage(msg);
+ }
+ }
+ }
+
+ /**
+ * Clears the active sink reference, causing subsequent log messages to be buffered
+ * instead of forwarded. This method is called automatically by {@link ConsolePanelSink}
+ * during cleanup.
+ */
+ public static void clearActiveSink() {
+ activeSink = null;
+ }
+
+ /**
+ * Sets the display level filter. Only log entries at or above this level will be
+ * visible to the sink.
+ *
+ * @param level the minimum level to display
+ */
+ public static void setDisplayLevel(Level level) {
+ displayLevel = level;
+ if (activeSink != null) {
+ String stats = activeSink.refreshDisplayBuffer();
+ // Log the stats as a system message (will go through normal logging)
+ activeSink.addLogMessage("[SYSTEM] " + stats);
+ }
+ }
+
+ /**
+ * Gets the current display level filter.
+ *
+ * @return the current display level
+ */
+ public static Level getDisplayLevel() {
+ return displayLevel;
+ }
+
+ /**
+ * Gets all log entries (unfiltered) for rebuilding the display buffer.
+ *
+ * @return all captured log entries
+ */
+ public static Queue getAllLogEntries() {
+ return allLogEntries;
+ }
+
+ /**
+ * Appends a log event to the buffer or forwards it to the active sink.
+ * This method is called by Log4j 2 for each log event that passes through this appender.
+ *
+ * The event is formatted with level, logger name (simple name only), and message.
+ * If an exception is present, its message is appended on a new line.
+ *
+ * All events are stored in the complete buffer. Events are only forwarded to the
+ * active sink if they meet or exceed the current display level.
+ *
+ * @param event the log event to append
+ */
+ @Override
+ public void append(LogEvent event) {
+ String loggerName = event.getLoggerName();
+ if (loggerName != null) {
+ int lastDot = loggerName.lastIndexOf('.');
+ if (lastDot >= 0 && lastDot < loggerName.length() - 1) {
+ loggerName = loggerName.substring(lastDot + 1);
+ }
+ }
+
+ // Format timestamp as relative time from start (e.g., "+00:02.345" for 2.345 seconds)
+ long timestamp = event.getTimeMillis();
+ long elapsedMs = timestamp - START_TIME;
+ long seconds = elapsedMs / 1000;
+ long millis = elapsedMs % 1000;
+ long minutes = seconds / 60;
+ seconds = seconds % 60;
+ String timeStr = String.format("+%02d:%02d.%03d", minutes, seconds, millis);
+
+ String formattedMessage = String.format("[%s] [%-5s] %s - %s",
+ timeStr,
+ event.getLevel(),
+ Objects.requireNonNullElse(loggerName, "root"),
+ event.getMessage().getFormattedMessage());
+
+ if (event.getThrown() != null) {
+ formattedMessage += "\n" + event.getThrown().getMessage();
+ }
+
+ // Store in complete buffer (up to max size)
+ Level eventLevel = event.getLevel();
+ if (allLogEntries.size() < MAX_BUFFER_SIZE) {
+ allLogEntries.offer(new LogEntry(eventLevel, formattedMessage));
+ } else {
+ // Remove oldest entry to make room
+ allLogEntries.poll();
+ allLogEntries.offer(new LogEntry(eventLevel, formattedMessage));
+ }
+
+ ConsolePanelSink sink = activeSink;
+ if (sink != null) {
+ // Debug: log every 100th entry to show backing buffer is being populated
+ if (allLogEntries.size() % 100 == 0) {
+ sink.addLogMessage("[DEBUG] LogBuffer backing buffer now has " + allLogEntries.size() + " entries");
+ }
+
+ // Only forward if event level is at or above display level
+ // Note: Lower intLevel = more severe (ERROR=200, WARN=300, INFO=400, DEBUG=500, TRACE=600)
+ // So to show INFO and above, we want intLevel <= INFO.intLevel()
+ if (eventLevel.intLevel() <= displayLevel.intLevel()) {
+ sink.addLogMessage(formattedMessage);
+ }
+ } else if (bufferedMessages.size() < MAX_BUFFER_SIZE) {
+ bufferedMessages.offer(formattedMessage);
+ }
+ }
+}
diff --git a/internal-apis/src/main/java/io/github/jbellis/jvector/status/sinks/LoggerStatusSink.java b/internal-apis/src/main/java/io/github/jbellis/jvector/status/sinks/LoggerStatusSink.java
new file mode 100644
index 000000000..950f92a08
--- /dev/null
+++ b/internal-apis/src/main/java/io/github/jbellis/jvector/status/sinks/LoggerStatusSink.java
@@ -0,0 +1,206 @@
+/*
+ * Copyright DataStax, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package io.github.jbellis.jvector.status.sinks;
+
+import io.github.jbellis.jvector.status.StatusContext;
+import io.github.jbellis.jvector.status.StatusTracker;
+import io.github.jbellis.jvector.status.eventing.StatusSink;
+import io.github.jbellis.jvector.status.eventing.StatusUpdate;
+import org.apache.logging.log4j.Level;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.util.Objects;
+
+/**
+ * A task sink that integrates with Log4j 2 to record task progress
+ * and lifecycle events. This sink is ideal for production environments where task
+ * monitoring needs to be integrated with existing logging infrastructure and
+ * centralized log management systems.
+ *
+ *
This sink provides:
+ *
+ * - Integration with the Log4j 2 logging framework
+ * - Configurable log levels for different environments
+ * - Structured log messages with task names and progress
+ * - Support for custom loggers and logger hierarchies
+ * - Automatic task name extraction and formatting
+ *
+ *
+ * Usage Examples:
+ *
+ * Basic Logging with Default Logger
+ * {@code
+ * // Uses the class name as logger name with INFO level
+ * TaskSink loggerSink = new LoggerStatusSink("io.myapp.TaskProcessor");
+ *
+ * try (StatusTracker tracker = StatusTracker.withInstrumented(processor, loggerSink)) {
+ * processor.processData();
+ * // Log output:
+ * // INFO: Task started: data-processing
+ * // INFO: Task update: data-processing [45.0%] - RUNNING
+ * // INFO: Task finished: data-processing
+ * }
+ * }
+ *
+ * Custom Logger and Level
+ * {@code
+ * Logger customLogger = LogManager.getLogger("app.background.tasks");
+ * TaskSink debugSink = new LoggerStatusSink(customLogger, Level.DEBUG);
+ *
+ * try (StatusTracker tracker = StatusTracker.withInstrumented(job, debugSink)) {
+* job.execute(); // Debug level logging
+* }
+* }
+ *
+ * Production Environment Setup
+ * {@code
+ * // Production configuration with WARNING level for critical tasks
+ * TaskSink productionSink = new LoggerStatusSink("production.critical.tasks", Level.WARN);
+ *
+ * StatusContext context = new StatusContext("critical-operations");
+ * context.addSink(productionSink);
+ *
+ * try (StatusTracker tracker = context.track(criticalTask)) {
+ * criticalTask.execute(); // Only logs at WARNING level
+ * }
+ * }
+ *
+ * Multiple Loggers for Different Components
+ * {@code
+ * // Different loggers for different subsystems
+ * TaskSink databaseSink = new LoggerStatusSink("app.database.operations", Level.INFO);
+ * TaskSink networkSink = new LoggerStatusSink("app.network.operations", Level.INFO);
+ * TaskSink fileSystemSink = new LoggerStatusSink("app.filesystem.operations", Level.DEBUG);
+ *
+ * // Use appropriate sink based on task type
+ * try (StatusTracker dbTracker = StatusTracker.withInstrumented(dbTask, databaseSink);
+ * StatusTracker netTracker = StatusTracker.withInstrumented(netTask, networkSink);
+ * StatusTracker fileTracker = StatusTracker.withInstrumented(fileTask, fileSystemSink)) {
+ *
+ * CompletableFuture.allOf(
+ * CompletableFuture.runAsync(dbTask::execute),
+ * CompletableFuture.runAsync(netTask::execute),
+ * CompletableFuture.runAsync(fileTask::execute)
+ * ).join();
+ * }
+ * }
+ *
+ * Integration with Existing Logger Hierarchy
+ * {@code
+ * // Leverage existing logger configuration
+ * Logger rootLogger = LogManager.getLogger("com.mycompany.myapp");
+ * TaskSink appSink = new LoggerStatusSink(rootLogger, Level.INFO);
+ *
+ * // Child logger inherits parent configuration
+ * TaskSink moduleSpecificSink = new LoggerStatusSink("com.mycompany.myapp.processing");
+ *
+ * try (StatusTracker tracker = StatusTracker.withInstrumented(task, appSink)) {
+ * task.run();
+ * }
+ * }
+ *
+ * Log Message Format
+ * The sink produces structured log messages with this format:
+ *
+ * - Task Started: "Task started: [task-name]"
+ * - Task Update: "Task update: [task-name] [XX.X%] - [run-state]"
+ * - Task Finished: "Task finished: [task-name]"
+ *
+ *
+ * Logger Integration Benefits
+ * Using this sink provides several advantages in production environments:
+ *
+ * - Centralized log management through existing logging infrastructure
+ * - Configurable output through Log4j 2 configuration or programmatic setup
+ * - Integration with log aggregation systems (ELK, Splunk, etc.)
+ * - Level-based filtering for different environments (dev, staging, prod)
+ * - Thread safety provided by Log4j 2
+ *
+ *
+ * Best Practices
+ *
+ * - Use hierarchical logger names for better organization (e.g., "app.module.component")
+ * - Choose appropriate log levels (INFO for normal operations, DEBUG for detailed tracing)
+ * - Configure appenders and layouts to match your logging infrastructure
+ * - Consider using different loggers for different types of tasks
+ * - Test log output in different environments to ensure proper configuration
+ *
+ *
+ * Thread Safety
+ * This sink is thread-safe through Log4j 2, which handles concurrent access to
+ * loggers and their appenders.
+ *
+ * @see StatusSink
+ * @see StatusTracker
+ * @see StatusContext
+ * @see Logger
+ * @since 4.0.0
+ */
+public class LoggerStatusSink implements StatusSink {
+
+ private final Logger logger;
+ private final Level level;
+
+ public LoggerStatusSink() {
+ this(LogManager.getLogger(LoggerStatusSink.class));
+ }
+
+ public LoggerStatusSink(Logger logger) {
+ this(logger, Level.INFO);
+ }
+
+ public LoggerStatusSink(Logger logger, Level level) {
+ this.logger = Objects.requireNonNull(logger, "logger");
+ this.level = Objects.requireNonNullElse(level, Level.INFO);
+ }
+
+ public LoggerStatusSink(String loggerName) {
+ this(LogManager.getLogger(loggerName));
+ }
+
+ public LoggerStatusSink(String loggerName, Level level) {
+ this(LogManager.getLogger(loggerName), level);
+ }
+
+ @Override
+ public void taskStarted(StatusTracker> task) {
+ String taskName = StatusTracker.extractTaskName(task);
+ log("Task started: " + taskName);
+ }
+
+ @Override
+ public void taskUpdate(StatusTracker> task, StatusUpdate> status) {
+ String taskName = StatusTracker.extractTaskName(task);
+ double progress = status.progress * 100;
+
+ log(String.format("Task update: %s [%.1f%%] - %s", taskName, progress, status.runstate));
+ }
+
+ @Override
+ public void taskFinished(StatusTracker> task) {
+ String taskName = StatusTracker.extractTaskName(task);
+ log("Task finished: " + taskName);
+ }
+
+ private void log(String message) {
+ Level effectiveLevel = Objects.requireNonNullElse(level, Level.INFO);
+ if (logger.isEnabled(effectiveLevel)) {
+ logger.log(effectiveLevel, message);
+ }
+ }
+}
diff --git a/internal-apis/src/main/java/io/github/jbellis/jvector/status/sinks/MetricsStatusSink.java b/internal-apis/src/main/java/io/github/jbellis/jvector/status/sinks/MetricsStatusSink.java
new file mode 100644
index 000000000..96ebd74e4
--- /dev/null
+++ b/internal-apis/src/main/java/io/github/jbellis/jvector/status/sinks/MetricsStatusSink.java
@@ -0,0 +1,345 @@
+/*
+ * Copyright DataStax, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package io.github.jbellis.jvector.status.sinks;
+
+import io.github.jbellis.jvector.status.eventing.StatusSink;
+import io.github.jbellis.jvector.status.eventing.StatusUpdate;
+import io.github.jbellis.jvector.status.StatusTracker;
+import io.github.jbellis.jvector.status.StatusContext;
+
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.DoubleAdder;
+
+/**
+ * A metrics-collecting task sink that captures detailed performance and progress statistics
+ * for task tracking and analysis. This sink is designed for production environments where
+ * task performance monitoring, SLA tracking, and operational metrics are important.
+ *
+ * This sink provides comprehensive metrics including:
+ *
+ * - Task execution timing (start, end, duration)
+ * - Progress statistics (updates, averages, current values)
+ * - Aggregate counts (total tasks, finished tasks, active tasks)
+ * - Performance analysis (average durations, update frequencies)
+ * - Detailed reporting with task breakdowns
+ *
+ *
+ * All metrics are thread-safe and can be safely accessed during concurrent task execution.
+ * The sink uses atomic operations and concurrent data structures to ensure accuracy under load.
+ *
+ *
Usage Examples:
+ *
+ * Basic Metrics Collection
+ * {@code
+ * MetricsTaskSink metrics = new MetricsTaskSink();
+ * StatusContext context = new StatusContext("batch-processing");
+ * context.addSink(metrics);
+ *
+ * // Process multiple tasks
+ * try (StatusTracker t1 = context.track(task1);
+ * StatusTracker t2 = context.track(task2)) {
+ *
+ * CompletableFuture.allOf(
+ * CompletableFuture.runAsync(task1::execute),
+ * CompletableFuture.runAsync(task2::execute)
+ * ).join();
+ * }
+ *
+ * // Analyze results
+ * System.out.println("Tasks completed: " + metrics.getTotalTasksFinished());
+ * System.out.println("Average duration: " + metrics.getAverageTaskDuration() + "ms");
+ * }
+ *
+ * Individual Task Metrics
+ * {@code
+ * MetricsTaskSink metrics = new MetricsTaskSink();
+ *
+ * try (Tracker tracker = Tracker.withInstrumented(processor, metrics)) {
+ * processor.processLargeDataset();
+ *
+ * // Get detailed metrics for this specific task
+ * MetricsTaskSink.TaskMetrics taskMetrics = metrics.getMetrics(tracker);
+ * System.out.println("Task duration: " + taskMetrics.getDuration() + "ms");
+ * System.out.println("Progress updates: " + taskMetrics.getUpdateCount());
+ * System.out.println("Final progress: " + taskMetrics.getLastProgress() * 100 + "%");
+ * }
+ * }
+ *
+ * Performance Monitoring and Alerting
+ * {@code
+ * MetricsTaskSink metrics = new MetricsTaskSink();
+ * // ... run tasks
+ *
+ * // Check for performance issues
+ * double avgDuration = metrics.getAverageTaskDuration();
+ * if (avgDuration > 30000) { // 30 seconds
+ * logger.warn("Tasks running slower than expected: " + avgDuration + "ms average");
+ * }
+ *
+ * // Monitor active task count
+ * long activeTasks = metrics.getActiveTaskCount();
+ * if (activeTasks > 100) {
+ * logger.warn("High number of active tasks: " + activeTasks);
+ * }
+ * }
+ *
+ * Detailed Reporting
+ * {@code
+ * MetricsTaskSink metrics = new MetricsTaskSink();
+ * // ... run tasks
+ *
+ * // Generate comprehensive report
+ * String report = metrics.generateReport();
+ * System.out.println(report);
+ *
+ * // Output:
+ * // === Task Metrics Report ===
+ * // Total tasks started: 15
+ * // Total tasks finished: 12
+ * // Active tasks: 3
+ * // Total updates: 847
+ * // Average task duration: 2547.33 ms
+ * //
+ * // Task Details:
+ * // - data-processing:
+ * // Duration: 3200 ms
+ * // Updates: 64
+ * // Progress: 100.0%
+ * // Status: Finished
+ * }
+ *
+ * Integration with Monitoring Systems
+ * {@code
+ * MetricsTaskSink metrics = new MetricsTaskSink();
+ * // ... configure and run tasks
+ *
+ * // Periodically export metrics to external systems
+ * ScheduledExecutorService scheduler = Executors.newSingleThreadScheduledExecutor();
+ * scheduler.scheduleAtFixedRate(() -> {
+ * // Export to Prometheus, StatsD, CloudWatch, etc.
+ * exportToMonitoringSystem(
+ * "tasks.started", metrics.getTotalTasksStarted(),
+ * "tasks.finished", metrics.getTotalTasksFinished(),
+ * "tasks.active", metrics.getActiveTaskCount(),
+ * "tasks.avg_duration", metrics.getAverageTaskDuration()
+ * );
+ * }, 0, 60, TimeUnit.SECONDS);
+ * }
+ *
+ * Memory Management
+ * {@code
+ * MetricsTaskSink metrics = new MetricsTaskSink();
+ * // ... run tasks
+ *
+ * // Clean up finished task metrics to prevent memory leaks
+ * Map, MetricsTaskSink.TaskMetrics> allMetrics = metrics.getAllMetrics();
+ * allMetrics.entrySet().removeIf(entry -> {
+ * MetricsTaskSink.TaskMetrics taskMetrics = entry.getValue();
+ * return taskMetrics.isFinished() &&
+ * (System.currentTimeMillis() - taskMetrics.getEndTime()) > Duration.ofHours(1).toMillis();
+ * });
+ * }
+ *
+ * TaskMetrics Class
+ * The {@link TaskMetrics} inner class provides detailed statistics for individual tasks:
+ *
+ * - Timing: Start time, end time, duration (including running tasks)
+ * - Progress: Update count, average progress, last progress value
+ * - Status: Task name, completion status
+ *
+ *
+ * Thread Safety and Performance
+ * This sink is designed for high-throughput environments with multiple concurrent tasks:
+ *
+ * - Thread-safe using {@link AtomicLong} and {@link DoubleAdder} for counters
+ * - {@link ConcurrentHashMap} for metrics storage with minimal lock contention
+ * - Low overhead per task update (O(1) operations)
+ * - Safe for concurrent read/write access from multiple threads
+ *
+ *
+ * Memory Considerations
+ * The sink retains metrics for all tracked tasks until explicitly cleared:
+ *
+ * - Use {@link #removeMetrics(StatusTracker)} to clean up individual tasks
+ * - Use {@link #clearMetrics()} to reset all metrics
+ * - Implement periodic cleanup for long-running applications
+ * - Monitor memory usage in high-volume scenarios
+ *
+ *
+ * @see StatusSink
+ * @see StatusTracker
+ * @see StatusContext
+ * @since 4.0.0
+ */
+public class MetricsStatusSink implements StatusSink {
+
+ public static class TaskMetrics {
+ private final AtomicLong startTime = new AtomicLong();
+ private final AtomicLong endTime = new AtomicLong();
+ private final AtomicLong updateCount = new AtomicLong();
+ private final DoubleAdder totalProgress = new DoubleAdder();
+ private volatile double lastProgress = 0.0;
+ private volatile String taskName;
+
+ public long getStartTime() {
+ return startTime.get();
+ }
+
+ public long getEndTime() {
+ return endTime.get();
+ }
+
+ public long getDuration() {
+ long end = endTime.get();
+ if (end == 0) {
+ return System.currentTimeMillis() - startTime.get();
+ }
+ return end - startTime.get();
+ }
+
+ public long getUpdateCount() {
+ return updateCount.get();
+ }
+
+ public double getAverageProgress() {
+ long count = updateCount.get();
+ return count > 0 ? totalProgress.sum() / count : 0.0;
+ }
+
+ public double getLastProgress() {
+ return lastProgress;
+ }
+
+ public String getTaskName() {
+ return taskName;
+ }
+
+ public boolean isFinished() {
+ return endTime.get() > 0;
+ }
+ }
+
+ private final Map, TaskMetrics> metricsMap = new ConcurrentHashMap<>();
+ private final AtomicLong totalTasksStarted = new AtomicLong();
+ private final AtomicLong totalTasksFinished = new AtomicLong();
+ private final AtomicLong totalUpdates = new AtomicLong();
+
+ @Override
+ public void taskStarted(StatusTracker> task) {
+ TaskMetrics metrics = new TaskMetrics();
+ metrics.startTime.set(System.currentTimeMillis());
+ metrics.taskName = StatusTracker.extractTaskName(task);
+ metricsMap.put(task, metrics);
+ totalTasksStarted.incrementAndGet();
+ }
+
+ @Override
+ public void taskUpdate(StatusTracker> task, StatusUpdate> status) {
+ TaskMetrics metrics = metricsMap.get(task);
+ if (metrics != null) {
+ metrics.updateCount.incrementAndGet();
+ metrics.totalProgress.add(status.progress);
+ metrics.lastProgress = status.progress;
+ }
+ totalUpdates.incrementAndGet();
+ }
+
+ @Override
+ public void taskFinished(StatusTracker> task) {
+ TaskMetrics metrics = metricsMap.get(task);
+ if (metrics != null) {
+ metrics.endTime.set(System.currentTimeMillis());
+ }
+ totalTasksFinished.incrementAndGet();
+ }
+
+ public TaskMetrics getMetrics(StatusTracker> task) {
+ if (task == null) {
+ return null;
+ }
+ return metricsMap.get(task);
+ }
+
+ public Map, TaskMetrics> getAllMetrics() {
+ return new ConcurrentHashMap<>(metricsMap);
+ }
+
+ public void clearMetrics() {
+ metricsMap.clear();
+ }
+
+ public void removeMetrics(StatusTracker> task) {
+ metricsMap.remove(task);
+ }
+
+ public long getTotalTasksStarted() {
+ return totalTasksStarted.get();
+ }
+
+ public long getTotalTasksFinished() {
+ return totalTasksFinished.get();
+ }
+
+ public long getTotalUpdates() {
+ return totalUpdates.get();
+ }
+
+ public long getActiveTaskCount() {
+ return metricsMap.values().stream()
+ .filter(m -> !m.isFinished())
+ .count();
+ }
+
+ public double getAverageTaskDuration() {
+ long finishedCount = 0;
+ long totalDuration = 0;
+
+ for (TaskMetrics metrics : metricsMap.values()) {
+ if (metrics.isFinished()) {
+ finishedCount++;
+ totalDuration += metrics.getDuration();
+ }
+ }
+
+ return finishedCount > 0 ? (double) totalDuration / finishedCount : 0.0;
+ }
+
+ public String generateReport() {
+ StringBuilder report = new StringBuilder();
+ report.append("=== Task Metrics Report ===\n");
+ report.append("Total tasks started: ").append(totalTasksStarted.get()).append("\n");
+ report.append("Total tasks finished: ").append(totalTasksFinished.get()).append("\n");
+ report.append("Active tasks: ").append(getActiveTaskCount()).append("\n");
+ report.append("Total updates: ").append(totalUpdates.get()).append("\n");
+ report.append("Average task duration: ").append(String.format("%.2f ms", getAverageTaskDuration())).append("\n");
+
+ report.append("\nTask Details:\n");
+ for (Map.Entry, TaskMetrics> entry : metricsMap.entrySet()) {
+ TaskMetrics metrics = entry.getValue();
+ report.append(" - ").append(metrics.getTaskName()).append(":\n");
+ report.append(" Duration: ").append(metrics.getDuration()).append(" ms\n");
+ report.append(" Updates: ").append(metrics.getUpdateCount()).append("\n");
+ report.append(" Progress: ").append(String.format("%.1f%%", metrics.getLastProgress() * 100)).append("\n");
+ report.append(" Status: ").append(metrics.isFinished() ? "Finished" : "Running").append("\n");
+ }
+
+ return report.toString();
+ }
+
+}
diff --git a/internal-apis/src/main/java/io/github/jbellis/jvector/status/sinks/NoopStatusSink.java b/internal-apis/src/main/java/io/github/jbellis/jvector/status/sinks/NoopStatusSink.java
new file mode 100644
index 000000000..5af10bcd8
--- /dev/null
+++ b/internal-apis/src/main/java/io/github/jbellis/jvector/status/sinks/NoopStatusSink.java
@@ -0,0 +1,160 @@
+/*
+ * Copyright DataStax, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package io.github.jbellis.jvector.status.sinks;
+
+import io.github.jbellis.jvector.status.eventing.StatusSink;
+import io.github.jbellis.jvector.status.eventing.StatusUpdate;
+import io.github.jbellis.jvector.status.StatusTracker;
+import io.github.jbellis.jvector.status.StatusContext;
+
+/**
+ * A no-operation task sink that discards all task events and produces no output.
+ * This sink is useful for disabling task monitoring without changing tracking code,
+ * testing scenarios where output interferes with assertions, and performance-sensitive
+ * environments where monitoring overhead must be minimized.
+ *
+ * This sink provides:
+ *
+ * - Zero overhead - all methods are empty and inline
+ * - Singleton pattern to minimize memory usage
+ * - Thread-safe operation with no state
+ * - Drop-in replacement for other sinks
+ *
+ *
+ * Usage Examples:
+ *
+ * Conditional Monitoring
+ * {@code
+ * // Enable monitoring only in development
+ * TaskSink sink = isDevelopment()
+ * ? new ConsoleTaskSink()
+ * : NoopTaskSink.getInstance();
+ *
+ * try (Tracker tracker = Tracker.withInstrumented(processor, sink)) {
+ * processor.processData(); // Monitoring enabled/disabled based on environment
+ * }
+ * }
+ *
+ * Testing Without Output
+ * {@code
+ * // Prevent console output during unit tests
+ * @Test
+ * public void testTaskExecution() {
+ * StatusContext context = new StatusContext("test-scope");
+ * context.addSink(NoopTaskSink.getInstance());
+ *
+ * try (StatusTracker tracker = context.track(testTask)) {
+ * testTask.execute();
+ * // No console output, test runs cleanly
+ * assertEquals(TestTask.State.COMPLETED, testTask.getState());
+ * }
+ * }
+ * }
+ *
+ * Performance Benchmarking
+ * {@code
+ * // Benchmark task execution without monitoring overhead
+ * TaskSink noopSink = NoopTaskSink.getInstance();
+ * long startTime = System.nanoTime();
+ *
+ * try (Tracker tracker = Tracker.withInstrumented(task, noopSink)) {
+ * task.executeBenchmark();
+ * }
+ *
+ * long duration = System.nanoTime() - startTime;
+ * System.out.println("Pure execution time: " + duration / 1_000_000 + "ms");
+ * }
+ *
+ * Configurable Sink Selection
+ * {@code
+ * public class TaskRunner {
+ * private final TaskSink sink;
+ *
+ * public TaskRunner(boolean enableLogging) {
+ * this.sink = enableLogging
+ * ? new LoggerTaskSink("app.tasks")
+ * : NoopTaskSink.getInstance();
+ * }
+ *
+ * public void runTask(MyTask task) {
+ * try (Tracker tracker = Tracker.withInstrumented(task, sink)) {
+ * task.execute();
+ * }
+ * }
+ * }
+ * }
+ *
+ * Default Sink in Context Configuration
+ * {@code
+ * public StatusContext createContext(String name, boolean enableMonitoring) {
+ * StatusContext context = new StatusContext(name);
+ * if (enableMonitoring) {
+ * context.addSink(new ConsoleTaskSink());
+ * context.addSink(new MetricsTaskSink());
+ * } else {
+ * context.addSink(NoopTaskSink.getInstance());
+ * }
+ * return context;
+ * }
+ * }
+ *
+ * Design Pattern
+ * This class implements the Null Object pattern, providing a valid sink implementation
+ * that performs no operations. This eliminates the need for null checks and conditional
+ * logic throughout the tracking code.
+ *
+ * Singleton Pattern
+ * The class uses a singleton pattern since all instances behave identically. Use
+ * {@link #getInstance()} to obtain the shared instance rather than creating new objects.
+ *
+ * Performance
+ * This sink has zero overhead:
+ *
+ * - All methods are empty and will be inlined by the JIT compiler
+ * - No memory allocations or I/O operations
+ * - No synchronization overhead
+ * - Minimal object memory footprint due to singleton pattern
+ *
+ *
+ * @see StatusSink
+ * @see StatusTracker
+ * @see StatusContext
+ * @since 4.0.0
+ */
+public class NoopStatusSink implements StatusSink {
+
+ private static final NoopStatusSink INSTANCE = new NoopStatusSink();
+
+ private NoopStatusSink() {
+ }
+
+ public static NoopStatusSink getInstance() {
+ return INSTANCE;
+ }
+
+ @Override
+ public void taskStarted(StatusTracker> task) {
+ }
+
+ @Override
+ public void taskUpdate(StatusTracker> task, StatusUpdate> status) {
+ }
+
+ @Override
+ public void taskFinished(StatusTracker> task) {
+ }
+}
diff --git a/internal-apis/src/main/java/io/github/jbellis/jvector/status/sinks/OutputMode.java b/internal-apis/src/main/java/io/github/jbellis/jvector/status/sinks/OutputMode.java
new file mode 100644
index 000000000..b99f00b99
--- /dev/null
+++ b/internal-apis/src/main/java/io/github/jbellis/jvector/status/sinks/OutputMode.java
@@ -0,0 +1,157 @@
+/*
+ * Copyright DataStax, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package io.github.jbellis.jvector.status.sinks;
+
+/**
+ * Defines output modes for status display and logging configuration. The mode determines
+ * how status information is rendered to the console and which terminal features are used.
+ *
+ * Mode Selection:
+ *
+ * - INTERACTIVE: Use for applications running in real terminals with full
+ * JLine support. Provides hierarchical display, keyboard controls, and log panel.
+ * - ENHANCED: Use for terminals with ANSI color support but limited control
+ * (e.g., some CI/CD environments, piped output with TERM set).
+ * - BASIC: Use for dumb terminals or when output is captured/piped
+ * (e.g., logs, basic shells).
+ * - AUTO: Let the framework detect the best mode based on environment.
+ *
+ *
+ * Environment Detection:
+ * The {@link #detect()} method determines the appropriate mode using:
+ *
+ * - {@code TERM} environment variable (null or "dumb" β BASIC)
+ * - {@code System.console()} availability (null β ENHANCED or BASIC)
+ * - Full terminal capabilities (present β INTERACTIVE)
+ *
+ *
+ * @see ConsolePanelLogIntercept
+ * @see ConsolePanelSink
+ * @since 4.0.0
+ */
+public enum OutputMode {
+ /**
+ * Interactive mode with full JLine terminal control, hierarchical display, and keyboard input.
+ * Best for development and interactive use in real terminals.
+ */
+ INTERACTIVE("interactive", "Full terminal control with hierarchical display and keyboard interaction"),
+
+ /**
+ * Enhanced mode with colors and ANSI formatting but no terminal control.
+ * Suitable for CI/CD environments and piped output where colors are supported.
+ */
+ ENHANCED("enhanced", "Color-enabled output with ANSI formatting"),
+
+ /**
+ * Basic mode with plain text output, no colors or special formatting.
+ * Use for dumb terminals, log files, and environments without ANSI support.
+ */
+ BASIC("basic", "Plain text output without colors or special formatting"),
+
+ /**
+ * Auto-detect the best mode based on environment variables and terminal capabilities.
+ * Uses {@link #detect()} to choose INTERACTIVE, ENHANCED, or BASIC.
+ */
+ AUTO("auto", "Automatically detect the best output mode");
+
+ private final String name;
+ private final String description;
+
+ OutputMode(String name, String description) {
+ this.name = name;
+ this.description = description;
+ }
+
+ /**
+ * Returns the lowercase string name of this mode.
+ *
+ * @return the mode name (e.g., "interactive", "enhanced", "basic", "auto")
+ */
+ public String getName() {
+ return name;
+ }
+
+ /**
+ * Returns a human-readable description of this mode.
+ *
+ * @return the mode description
+ */
+ public String getDescription() {
+ return description;
+ }
+
+ /**
+ * Parses a string value to an OutputMode enum value. Accepts both lowercase names
+ * (e.g., "interactive") and uppercase enum names (e.g., "INTERACTIVE").
+ *
+ * @param value the string to parse, case-insensitive
+ * @return the corresponding OutputMode, or AUTO if the value is null or unrecognized
+ */
+ public static OutputMode fromString(String value) {
+ if (value == null) {
+ return AUTO;
+ }
+
+ String lower = value.toLowerCase().trim();
+ for (OutputMode mode : values()) {
+ if (mode.name.equals(lower)) {
+ return mode;
+ }
+ }
+
+ // Try to match by enum name as well
+ try {
+ return OutputMode.valueOf(value.toUpperCase());
+ } catch (IllegalArgumentException e) {
+ // Caller should handle unknown values if logging is needed
+ return AUTO;
+ }
+ }
+
+ /**
+ * Automatically detects the best output mode based on the runtime environment.
+ * Uses terminal capabilities and environment variables to determine which mode
+ * will work best.
+ *
+ * Detection Logic:
+ *
+ * - If {@code TERM} is null or "dumb" β returns BASIC
+ * - If {@code System.console()} is null (piped/redirected) β returns ENHANCED
+ * - If real terminal detected β returns INTERACTIVE
+ *
+ *
+ * @return the detected output mode (never returns AUTO)
+ */
+ public static OutputMode detect() {
+ // Check TERM environment variable
+ String term = System.getenv("TERM");
+
+ // If TERM is not set or is "dumb", use basic mode
+ if (term == null || term.equals("dumb")) {
+ return BASIC;
+ }
+
+ // Check if output is being piped (System.console() returns null when piped)
+ if (System.console() == null) {
+ // Output is piped, but TERM is set - use enhanced mode for colors
+ return ENHANCED;
+ }
+
+ // We have a real terminal - use interactive mode
+ return INTERACTIVE;
+ }
+}
\ No newline at end of file
diff --git a/internal-apis/src/main/java/io/github/jbellis/jvector/status/sinks/package-info.java b/internal-apis/src/main/java/io/github/jbellis/jvector/status/sinks/package-info.java
new file mode 100644
index 000000000..41581eb89
--- /dev/null
+++ b/internal-apis/src/main/java/io/github/jbellis/jvector/status/sinks/package-info.java
@@ -0,0 +1,84 @@
+/*
+ * Copyright DataStax, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Provides implementations of {@link io.github.jbellis.jvector.status.eventing.StatusSink}
+ * for various output formats and destinations. Sinks receive status events from the monitoring
+ * framework and process them according to their specific purpose (console display, logging,
+ * metrics collection, etc.).
+ *
+ * Available Sink Implementations
+ *
+ * Display Sinks
+ *
+ * - {@link ConsoleLoggerSink} - Simple console output
+ * with progress bars and timestamps for standard terminal usage
+ * - {@link ConsolePanelSink} - Advanced interactive
+ * terminal UI using JLine3 with hierarchical display, scrollable logs, and keyboard controls
+ *
+ *
+ * Integration Sinks
+ *
+ * - {@link LoggerStatusSink} - Integration with Log4j 2
+ * for routing status updates through the logging framework
+ * - {@link MetricsStatusSink} - Collects performance
+ * metrics and statistics for monitoring and analysis
+ *
+ *
+ * Utility Sinks
+ *
+ * - {@link NoopStatusSink} - No-operation sink for
+ * disabling output or testing
+ *
+ *
+ * Supporting Classes
+ *
+ * - {@link LogBuffer} - Log4j 2 appender that bridges
+ * logging output to ConsolePanelSink
+ * - {@link OutputMode} - Enum for configuring output
+ * mode based on terminal capabilities
+ *
+ *
+ * Usage Patterns
+ *
+ * Single Sink
+ * {@code
+ * StatusContext context = new StatusContext("my-operation");
+ * context.addSink(new ConsoleLoggerSink());
+ * }
+ *
+ * Multiple Sinks
+ * {@code
+ * StatusContext context = new StatusContext("my-operation");
+ * context.addSink(new ConsoleLoggerSink());
+ * context.addSink(new LoggerStatusSink("app.tasks"));
+ * context.addSink(new MetricsStatusSink());
+ * }
+ *
+ * Conditional Sink Selection
+ * {@code
+ * OutputMode mode = OutputMode.detect();
+ * StatusSink sink = mode == OutputMode.INTERACTIVE
+ * ? ConsolePanelSink.builder().build()
+ * : new ConsoleLoggerSink();
+ * context.addSink(sink);
+ * }
+ *
+ * @see io.github.jbellis.jvector.status.eventing.StatusSink
+ * @see io.github.jbellis.jvector.status.StatusContext
+ * @since 4.0.0
+ */
+package io.github.jbellis.jvector.status.sinks;
diff --git a/internal-apis/src/test/java/io/github/jbellis/jvector/status/SimpleStatusUpdateTest.java b/internal-apis/src/test/java/io/github/jbellis/jvector/status/SimpleStatusUpdateTest.java
new file mode 100644
index 000000000..a6a833b04
--- /dev/null
+++ b/internal-apis/src/test/java/io/github/jbellis/jvector/status/SimpleStatusUpdateTest.java
@@ -0,0 +1,34 @@
+/*
+ * Copyright DataStax, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package io.github.jbellis.jvector.status;
+
+import io.github.jbellis.jvector.status.eventing.RunState;
+import io.github.jbellis.jvector.status.eventing.StatusUpdate;
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.*;
+
+public class SimpleStatusUpdateTest {
+
+ @Test
+ public void testTaskStatusCreation() {
+ StatusUpdate status = new StatusUpdate<>(0.5, RunState.RUNNING);
+ assertEquals(0.5, status.progress, 0.001);
+ assertEquals(RunState.RUNNING, status.runstate);
+ assertTrue(status.timestamp > 0);
+ }
+}
diff --git a/internal-apis/src/test/java/io/github/jbellis/jvector/status/SimulatedClock.java b/internal-apis/src/test/java/io/github/jbellis/jvector/status/SimulatedClock.java
new file mode 100644
index 000000000..37d8178d2
--- /dev/null
+++ b/internal-apis/src/test/java/io/github/jbellis/jvector/status/SimulatedClock.java
@@ -0,0 +1,287 @@
+/*
+ * Copyright DataStax, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package io.github.jbellis.jvector.status;
+
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * A simulated clock that can run faster or slower than real time, useful for
+ * demonstrations, testing, and controlled time-dependent behavior. The clock
+ * maintains virtual time that advances based on a configurable speed multiplier.
+ *
+ * The clock runs in a dedicated background thread that continuously advances
+ * virtual time based on elapsed real time and the current speed multiplier.
+ * This allows tasks to sleep in "virtual time" while actual wall-clock time
+ * passes more quickly or slowly.
+ *
+ *
Speed Control
+ * The clock supports preset speed multipliers ranging from 0.0001x (extremely slow)
+ * to 50x (very fast). Common presets include:
+ *
+ * - 0.0001x, 0.001x, 0.01x, 0.1x - Slower than real time
+ * - 1.0x - Real-time speed (default)
+ * - 2.0x, 5.0x, 10.0x, 20.0x, 50.0x - Faster than real time
+ *
+ *
+ * Speed can be adjusted dynamically using {@link #speedUp()} and {@link #slowDown()},
+ * which cycle through the preset values. Current speed can be queried via
+ * {@link #getSpeedMultiplier()}, {@link #getSpeedDescription()}, or
+ * {@link #getSpeedIndicator()}.
+ *
+ * Example Usage
+ * {@code
+ * try (SimulatedClock clock = new SimulatedClock()) {
+ * // Start at normal speed (1.0x)
+ * long startTime = clock.currentTimeMillis();
+ *
+ * // Speed up simulation
+ * clock.speedUp(); // Now at 2.0x
+ * clock.speedUp(); // Now at 5.0x
+ *
+ * // Sleep for 1000ms virtual time (takes ~200ms real time at 5.0x)
+ * clock.sleep(1000);
+ *
+ * long elapsed = clock.currentTimeMillis() - startTime;
+ * System.out.println("Virtual time elapsed: " + elapsed + "ms");
+ * }
+ * }
+ *
+ * Integration with ConsolePanelSink
+ * This clock is commonly used with {@link io.github.jbellis.jvector.status.sinks.ConsolePanelSink}
+ * custom keyboard handlers to provide interactive time control in demonstrations:
+ * {@code
+ * SimulatedClock clock = new SimulatedClock();
+ * ConsolePanelSink sink = ConsolePanelSink.builder()
+ * .withKeyHandler("shift-right", () -> {
+ * clock.speedUp();
+ * sink.addLogMessage("Speed: " + clock.getSpeedDescription());
+ * })
+ * .withKeyHandler("shift-left", () -> {
+ * clock.slowDown();
+ * sink.addLogMessage("Speed: " + clock.getSpeedDescription());
+ * })
+ * .build();
+ * }
+ *
+ * Thread Safety
+ * This class is thread-safe. The {@link #sleep(long)} method can be called
+ * from multiple threads concurrently, and speed adjustments are safely visible
+ * across all threads. The background clock thread is marked as a daemon and will
+ * not prevent JVM shutdown.
+ *
+ * @see AutoCloseable
+ * @since 4.0.0
+ */
+public class SimulatedClock implements AutoCloseable {
+ private static final double[] SPEED_PRESETS = {0.0001, 0.001, 0.01, 0.1, 0.25, 0.5, 1.0, 2.0, 5.0, 10.0, 20.0, 50.0};
+ private static final int DEFAULT_SPEED_INDEX = 6; // 1.0x
+
+ private final AtomicLong virtualTimeMillis;
+ private final long realStartTime;
+ private volatile double speedMultiplier;
+ private volatile int speedIndex;
+ private volatile boolean running;
+ private final Thread clockThread;
+
+ /**
+ * Creates a new simulated clock starting at normal speed (1.0x).
+ * The clock immediately begins advancing virtual time in a background thread.
+ * Virtual time starts at 0ms when the clock is created.
+ */
+ public SimulatedClock() {
+ this.realStartTime = System.currentTimeMillis();
+ this.virtualTimeMillis = new AtomicLong(0);
+ this.speedIndex = DEFAULT_SPEED_INDEX;
+ this.speedMultiplier = SPEED_PRESETS[speedIndex];
+ this.running = true;
+
+ this.clockThread = new Thread(this::runClock, "SimulatedClock");
+ this.clockThread.setDaemon(true);
+ this.clockThread.start();
+ }
+
+ private void runClock() {
+ long lastUpdateTime = System.currentTimeMillis();
+
+ while (running) {
+ try {
+ Thread.sleep(10); // Update every 10ms
+
+ long now = System.currentTimeMillis();
+ long realElapsed = now - lastUpdateTime;
+ long virtualElapsed = (long) (realElapsed * speedMultiplier);
+
+ virtualTimeMillis.addAndGet(virtualElapsed);
+ lastUpdateTime = now;
+
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ break;
+ }
+ }
+ }
+
+ /**
+ * Returns the current virtual time in milliseconds since clock creation.
+ * Virtual time starts at 0 and advances based on the current speed multiplier.
+ *
+ * @return the current virtual time in milliseconds
+ */
+ public long currentTimeMillis() {
+ return virtualTimeMillis.get();
+ }
+
+ /**
+ * Sleeps for the specified virtual duration. The actual real-world sleep time
+ * is adjusted based on the current speed multiplier. This method is responsive
+ * to speed changes during the sleep by checking the speed multiplier periodically.
+ *
+ * For example, if the speed is 5.0x, sleeping for 1000ms virtual time will
+ * take approximately 200ms of real time. If the speed changes during the sleep,
+ * the remaining sleep duration will be adjusted accordingly.
+ *
+ * If {@code virtualMillis} is 0 or negative, this method returns immediately
+ * without sleeping.
+ *
+ * @param virtualMillis the virtual time to sleep in milliseconds
+ * @throws InterruptedException if the current thread is interrupted while sleeping
+ */
+ public void sleep(long virtualMillis) throws InterruptedException {
+ if (virtualMillis <= 0) {
+ return;
+ }
+
+ long targetVirtualTime = virtualTimeMillis.get() + virtualMillis;
+
+ // Sleep in small chunks to be responsive to speed changes
+ // Use 100ms real-time chunks for responsiveness
+ while (virtualTimeMillis.get() < targetVirtualTime) {
+ long remaining = targetVirtualTime - virtualTimeMillis.get();
+ long realSleepMillis = (long) Math.min(100, remaining / speedMultiplier);
+
+ if (realSleepMillis > 0) {
+ Thread.sleep(realSleepMillis);
+ } else {
+ // Speed is very fast, just yield to let clock thread advance
+ Thread.yield();
+ // Add a tiny sleep to prevent busy-waiting
+ Thread.sleep(1);
+ }
+ }
+ }
+
+ /**
+ * Increases the clock speed to the next preset value.
+ * If already at the maximum speed (50.0x), this method has no effect.
+ * Speed changes take effect immediately and apply to all ongoing sleep operations.
+ */
+ public void speedUp() {
+ if (speedIndex < SPEED_PRESETS.length - 1) {
+ speedIndex++;
+ speedMultiplier = SPEED_PRESETS[speedIndex];
+ }
+ }
+
+ /**
+ * Decreases the clock speed to the previous preset value.
+ * If already at the minimum speed (0.0001x), this method has no effect.
+ * Speed changes take effect immediately and apply to all ongoing sleep operations.
+ */
+ public void slowDown() {
+ if (speedIndex > 0) {
+ speedIndex--;
+ speedMultiplier = SPEED_PRESETS[speedIndex];
+ }
+ }
+
+ /**
+ * Returns the current speed multiplier as a numeric value.
+ * For example, 1.0 means real-time, 2.0 means twice as fast, 0.5 means half speed.
+ *
+ * @return the current speed multiplier
+ */
+ public double getSpeedMultiplier() {
+ return speedMultiplier;
+ }
+
+ /**
+ * Returns a human-readable description of the current speed.
+ * Examples: "x1 (realtime)", "x10 (faster)", "x1/100 (slower)".
+ *
+ * @return a formatted string describing the current speed setting
+ */
+ public String getSpeedDescription() {
+ if (speedMultiplier == 1.0) {
+ return "x1 (realtime)";
+ } else if (speedMultiplier >= 1.0) {
+ // Faster: x2, x5, x10, x20, x50
+ int speedInt = (int) speedMultiplier;
+ if (speedInt == speedMultiplier) {
+ return "x" + speedInt + " (faster)";
+ } else {
+ return String.format("x%.1f (faster)", speedMultiplier);
+ }
+ } else {
+ // Slower: x1/10, x1/100, x1/1000, x1/10000
+ double inverse = 1.0 / speedMultiplier;
+ int inverseInt = (int) Math.round(inverse);
+ return "x1/" + inverseInt + " (slower)";
+ }
+ }
+
+ /**
+ * Returns a compact speed indicator suitable for inline status display.
+ * Examples: "x1", "x10", "x1/10", "x1/100", "x1/10000".
+ * This is a shorter version of {@link #getSpeedDescription()} without the descriptive suffix.
+ *
+ * @return a compact string indicating the current speed
+ */
+ public String getSpeedIndicator() {
+ if (speedMultiplier == 1.0) {
+ return "x1";
+ } else if (speedMultiplier >= 1.0) {
+ // Faster: x2, x5, x10, x20, x50
+ int speedInt = (int) speedMultiplier;
+ if (speedInt == speedMultiplier) {
+ return "x" + speedInt;
+ } else {
+ return String.format("x%.1f", speedMultiplier);
+ }
+ } else {
+ // Slower: x1/10, x1/100, x1/1000, x1/10000
+ double inverse = 1.0 / speedMultiplier;
+ int inverseInt = (int) Math.round(inverse);
+ return "x1/" + inverseInt;
+ }
+ }
+
+ /**
+ * Stops the clock's background thread and releases resources.
+ * This method blocks for up to 1 second waiting for the background thread
+ * to terminate. After calling close, the clock should not be used further.
+ */
+ @Override
+ public void close() {
+ running = false;
+ clockThread.interrupt();
+ try {
+ clockThread.join(1000);
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ }
+ }
+}
diff --git a/internal-apis/src/test/java/io/github/jbellis/jvector/status/SimulatedTask.java b/internal-apis/src/test/java/io/github/jbellis/jvector/status/SimulatedTask.java
new file mode 100644
index 000000000..cb7bdb7cc
--- /dev/null
+++ b/internal-apis/src/test/java/io/github/jbellis/jvector/status/SimulatedTask.java
@@ -0,0 +1,54 @@
+/*
+ * Copyright DataStax, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package io.github.jbellis.jvector.status;
+
+import java.time.Duration;
+import java.time.Instant;
+import java.util.concurrent.Callable;
+
+public class SimulatedTask implements Callable {
+
+ public final String name;
+ public final int count;
+ public final Duration stepTime;
+ private final StringBuilder buffer;
+
+ public SimulatedTask(String name, StringBuilder buffer, int count, Duration stepTime) {
+ this.name = name;
+ this.count = count;
+ this.stepTime = stepTime;
+ this.buffer = buffer;
+ }
+
+ @Override
+ public String call() throws Exception {
+ int divisor = count / 100;
+ var startAt = Instant.now();
+
+ for (int micro = 0; micro < count; micro+=divisor) {
+ int end = Math.min(micro + divisor, count);
+ buffer.append(name).append(" micro:").append(micro).append(" count:").append(count).append("\n");
+ for (int i = micro; i < end; i++) {
+ Thread.sleep(stepTime.toMillis());
+ }
+ }
+ var endAt = Instant.now();
+ String summary= name + " completed in " + Duration.between(startAt, endAt).toString() + " \n";
+ return buffer.toString();
+
+ }
+}
diff --git a/internal-apis/src/test/java/io/github/jbellis/jvector/status/StatusScopeTest.java b/internal-apis/src/test/java/io/github/jbellis/jvector/status/StatusScopeTest.java
new file mode 100644
index 000000000..6a696dd6d
--- /dev/null
+++ b/internal-apis/src/test/java/io/github/jbellis/jvector/status/StatusScopeTest.java
@@ -0,0 +1,181 @@
+/*
+ * Copyright DataStax, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package io.github.jbellis.jvector.status;
+
+import io.github.jbellis.jvector.status.eventing.RunState;
+import io.github.jbellis.jvector.status.eventing.StatusSink;
+import io.github.jbellis.jvector.status.eventing.StatusSource;
+import io.github.jbellis.jvector.status.eventing.StatusUpdate;
+import io.github.jbellis.jvector.status.sinks.NoopStatusSink;
+import io.github.jbellis.jvector.status.StatusScope;
+import org.junit.jupiter.api.Test;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.junit.jupiter.api.Assertions.*;
+
+/**
+ * Verifies that {@link StatusContext} manages track hierarchies and monitoring resources.
+ */
+public class StatusScopeTest {
+
+ private static final class InstrumentedTask implements StatusSource {
+ private final String name;
+ private volatile double progress;
+ private volatile RunState state = RunState.PENDING;
+
+ private InstrumentedTask(String name) {
+ this.name = name;
+ }
+
+ void start() {
+ state = RunState.RUNNING;
+ progress = 0.0;
+ }
+
+ void advance(double increment) {
+ progress = Math.min(1.0, progress + increment);
+ if (progress >= 1.0) {
+ state = RunState.SUCCESS;
+ }
+ }
+
+ @Override
+ public StatusUpdate getTaskStatus() {
+ return new StatusUpdate<>(progress, state);
+ }
+
+ @Override
+ public String toString() {
+ return name;
+ }
+ }
+
+ @Test
+ public void tracksRootTaskInSingleContext() {
+ try (StatusContext context = new StatusContext("root");
+ StatusScope scope = context.createScope("root-scope")) {
+ InstrumentedTask task = new InstrumentedTask("root-task");
+ try (StatusTracker tracker = scope.trackTask(task)) {
+ assertEquals("root", context.getName());
+ assertEquals(Duration.ofMillis(100), context.getDefaultPollInterval());
+ assertEquals(1, context.getActiveTrackerCount());
+ assertEquals(scope, tracker.getParentScope());
+ }
+ assertEquals(0, context.getActiveTrackerCount());
+ }
+ }
+
+ @Test
+ public void scopedTasksShareContextAndHierarchy() {
+ try (StatusContext context = new StatusContext("root")) {
+ InstrumentedTask task1 = new InstrumentedTask("task1");
+ InstrumentedTask task2 = new InstrumentedTask("task2");
+
+ try (StatusScope scope = context.createScope("TestScope")) {
+ try (StatusTracker tracker1 = scope.trackTask(task1);
+ StatusTracker tracker2 = scope.trackTask(task2)) {
+ assertSame(scope, tracker1.getParentScope());
+ assertSame(scope, tracker2.getParentScope());
+ assertEquals(2, scope.getChildTasks().size());
+ assertEquals(2, context.getActiveTrackerCount());
+ }
+ assertTrue(scope.getChildTasks().isEmpty());
+ }
+ }
+ }
+
+ @Test
+ public void contextOwnsMonitorLifecycle() {
+ try (StatusContext context = new StatusContext("root");
+ StatusScope scope = context.createScope("test")) {
+ InstrumentedTask task = new InstrumentedTask("monitored");
+ try (StatusTracker tracker = scope.trackTask(task)) {
+ task.start();
+ task.advance(0.5);
+ assertEquals(RunState.RUNNING, tracker.getStatus().runstate);
+ }
+ assertEquals(0, context.getActiveTrackerCount());
+ }
+ }
+
+ @Test
+ public void additionalSinksApplyToNewTrackers() {
+ RecordingSink sink = new RecordingSink();
+ try (StatusContext context = new StatusContext("root")) {
+ context.addSink(sink);
+ try (var scope = context.createScope("test-scope");
+ StatusTracker tracker = scope.trackTask(new InstrumentedTask("task"))) {
+ // Sinks are managed by context, verify notifications reach the sink
+ tracker.getStatus();
+ }
+ assertTrue(sink.events.contains(RecordingSink.Event.START));
+
+ sink.events.clear();
+ context.removeSink(sink);
+
+ try (var scope = context.createScope("test-scope2");
+ StatusTracker tracker = scope.trackTask(new InstrumentedTask("task2"))) {
+ tracker.getStatus();
+ }
+ // After removal, sink should not receive events
+ assertFalse(sink.events.contains(RecordingSink.Event.START));
+ }
+ }
+
+ @Test
+ public void closingContextStopsAllTrackersAndScopes() {
+ StatusContext context = new StatusContext("root");
+ InstrumentedTask task1 = new InstrumentedTask("task1");
+ InstrumentedTask task2 = new InstrumentedTask("task2");
+
+ StatusScope scope = context.createScope("TestScope");
+ StatusTracker tracker1 = scope.trackTask(task1);
+ StatusTracker tracker2 = scope.trackTask(task2);
+
+ context.close();
+
+ assertTrue(context.isClosed());
+ assertEquals(0, context.getActiveTrackerCount());
+ assertTrue(scope.getChildTasks().isEmpty());
+ assertNotNull(tracker1.getStatus());
+ assertNotNull(tracker2.getStatus());
+ }
+
+ private static final class RecordingSink implements StatusSink {
+ enum Event { START, UPDATE, FINISH }
+
+ final List events = new ArrayList<>();
+
+ @Override
+ public void taskStarted(StatusTracker> task) {
+ events.add(Event.START);
+ }
+
+ @Override
+ public void taskUpdate(StatusTracker> task, StatusUpdate> status) {
+ events.add(Event.UPDATE);
+ }
+
+ @Override
+ public void taskFinished(StatusTracker> task) {
+ events.add(Event.FINISH);
+ }
+ }
+}
diff --git a/internal-apis/src/test/java/io/github/jbellis/jvector/status/StatusTrackerTest.java b/internal-apis/src/test/java/io/github/jbellis/jvector/status/StatusTrackerTest.java
new file mode 100644
index 000000000..400302a71
--- /dev/null
+++ b/internal-apis/src/test/java/io/github/jbellis/jvector/status/StatusTrackerTest.java
@@ -0,0 +1,253 @@
+/*
+ * Copyright DataStax, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package io.github.jbellis.jvector.status;
+
+import io.github.jbellis.jvector.status.eventing.RunState;
+import io.github.jbellis.jvector.status.eventing.StatusSink;
+import io.github.jbellis.jvector.status.eventing.StatusSource;
+import io.github.jbellis.jvector.status.eventing.StatusUpdate;
+import org.junit.jupiter.api.Test;
+import io.github.jbellis.jvector.status.StatusScope;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.junit.jupiter.api.Assertions.*;
+
+public class StatusTrackerTest {
+
+ private static final class InstrumentedTask implements StatusSource {
+ private final String name;
+ private volatile double progress;
+ private volatile RunState state = RunState.PENDING;
+
+ private InstrumentedTask(String name) {
+ this.name = name;
+ }
+
+ void setProgress(double value) {
+ progress = value;
+ if (value >= 1.0) {
+ state = RunState.SUCCESS;
+ } else if (value > 0) {
+ state = RunState.RUNNING;
+ }
+ }
+
+ @Override
+ public StatusUpdate getTaskStatus() {
+ return new StatusUpdate<>(progress, state);
+ }
+
+ @Override
+ public String toString() {
+ return name;
+ }
+ }
+
+ @Test
+ public void instrumentedTaskProgressIsReported() {
+ try (StatusContext context = new StatusContext("tracker-test");
+ StatusScope scope = context.createScope("test")) {
+ InstrumentedTask task = new InstrumentedTask("task");
+ try (StatusTracker tracker = scope.trackTask(task)) {
+ task.setProgress(0.5);
+ StatusUpdate update = tracker.getStatus();
+ assertEquals(0.5, update.progress, 1e-6);
+ assertEquals(RunState.RUNNING, update.runstate);
+ }
+ }
+ }
+
+ @Test
+ public void functorBasedTrackingUsesCustomFunction() {
+ try (StatusContext context = new StatusContext("functor", Duration.ofMillis(25));
+ StatusScope scope = context.createScope("test")) {
+ List samples = new ArrayList<>();
+ FunctionTask task = new FunctionTask("functor");
+ try (StatusTracker tracker = scope.trackTask(task, t -> {
+ samples.add(t.progress);
+ return new StatusUpdate<>(t.progress, t.state);
+ })) {
+ task.advance(0.25);
+ tracker.getStatus();
+ task.advance(0.5);
+ tracker.getStatus();
+ }
+ assertTrue(samples.stream().anyMatch(v -> v >= 0.5));
+ }
+ }
+
+ private static final class FunctionTask {
+ private final String name;
+ private double progress;
+ private RunState state = RunState.PENDING;
+
+ private FunctionTask(String name) {
+ this.name = name;
+ }
+
+ void advance(double value) {
+ progress = Math.min(1.0, progress + value);
+ state = progress >= 1.0 ? RunState.SUCCESS : RunState.RUNNING;
+ }
+
+ @Override
+ public String toString() {
+ return name;
+ }
+ }
+
+ @Test
+ public void scopedTasksInheritContext() {
+ RecordingSink sink = new RecordingSink();
+ try (StatusContext context = new StatusContext("hierarchy", List.of(sink))) {
+ InstrumentedTask task1 = new InstrumentedTask("task1");
+ InstrumentedTask task2 = new InstrumentedTask("task2");
+
+ try (StatusScope scope = context.createScope("TestScope")) {
+ try (StatusTracker tracker1 = scope.trackTask(task1);
+ StatusTracker tracker2 = scope.trackTask(task2)) {
+ // Verify scope relationship and context
+ assertEquals(scope, tracker1.getParentScope());
+ assertEquals(scope, tracker2.getParentScope());
+ assertEquals(context, tracker1.getContext());
+ assertEquals(context, tracker2.getContext());
+ task2.setProgress(1.0);
+ tracker2.getStatus();
+ }
+ }
+
+ assertEquals(RecordingSink.Event.FINISH, sink.events.get(sink.events.size() - 1));
+ }
+ }
+
+ @Test
+ public void addAndRemoveSinksDynamically() {
+ RecordingSink sink = new RecordingSink();
+ try (StatusContext context = new StatusContext("dynamics");
+ StatusScope scope = context.createScope("test")) {
+ InstrumentedTask task = new InstrumentedTask("task");
+ try (StatusTracker tracker = scope.trackTask(task)) {
+ // Sinks are managed at context level
+ context.addSink(sink);
+ task.setProgress(1.0);
+ tracker.getStatus();
+ context.removeSink(sink);
+ }
+ assertTrue(sink.events.contains(RecordingSink.Event.START));
+ assertTrue(sink.events.contains(RecordingSink.Event.FINISH));
+ }
+ }
+
+ @Test
+ public void closingTrackerIdempotent() {
+ StatusContext context = new StatusContext("idempotent");
+ StatusScope scope = context.createScope("test");
+ InstrumentedTask task = new InstrumentedTask("task");
+ StatusTracker tracker = scope.trackTask(task);
+ tracker.close();
+ tracker.close();
+ assertEquals(0, context.getActiveTrackerCount());
+ context.close();
+ }
+
+ @Test
+ public void contextClosesAllScopesAndTasks() {
+ StatusContext context = new StatusContext("parent");
+ StatusScope scope = context.createScope("TestScope");
+ InstrumentedTask task1 = new InstrumentedTask("task1");
+ InstrumentedTask task2 = new InstrumentedTask("task2");
+ StatusTracker tracker1 = scope.trackTask(task1);
+ StatusTracker tracker2 = scope.trackTask(task2);
+
+ context.close();
+
+ assertTrue(scope.getChildTasks().isEmpty());
+ assertTrue(context.getActiveTrackers().isEmpty());
+ assertNotNull(tracker1.getStatus());
+ assertNotNull(tracker2.getStatus());
+ }
+
+ @Test
+ public void scopeMustShareSameContext() {
+ try (StatusContext contextA = new StatusContext("A");
+ StatusContext contextB = new StatusContext("B")) {
+ StatusScope scopeA = contextA.createScope("ScopeA");
+ StatusScope scopeB = contextB.createScope("ScopeB");
+
+ // Attempting to track a task in scopeB using contextA should fail
+ IllegalArgumentException ex = assertThrows(
+ IllegalArgumentException.class,
+ () -> contextA.trackInScope(scopeB, new InstrumentedTask("task"), StatusSource::getTaskStatus));
+
+ assertTrue(ex.getMessage().contains("different StatusContext"));
+
+ scopeA.close();
+ scopeB.close();
+ }
+ }
+
+ @Test
+ public void elapsedRunningTimeTracksExecution() throws InterruptedException {
+ try (StatusContext context = new StatusContext("timing");
+ StatusScope scope = context.createScope("test")) {
+ InstrumentedTask task = new InstrumentedTask("task");
+ try (StatusTracker tracker = scope.trackTask(task)) {
+ assertNull(tracker.getRunningStartTime());
+ assertEquals(0, tracker.getElapsedRunningTime());
+
+ task.setProgress(0.1);
+ tracker.refreshAndGetStatus();
+ assertNotNull(tracker.getRunningStartTime());
+
+ Thread.sleep(5);
+ long runningElapsed = tracker.getElapsedRunningTime();
+ assertTrue(runningElapsed >= 5);
+
+ task.setProgress(1.0);
+ tracker.refreshAndGetStatus();
+
+ long finalElapsed = tracker.getElapsedRunningTime();
+ assertTrue(finalElapsed >= runningElapsed);
+ assertNotNull(tracker.getRunningStartTime());
+ }
+ }
+ }
+
+ private static final class RecordingSink implements StatusSink {
+ enum Event { START, UPDATE, FINISH }
+
+ final List events = new ArrayList<>();
+
+ @Override
+ public void taskStarted(StatusTracker> task) {
+ events.add(Event.START);
+ }
+
+ @Override
+ public void taskUpdate(StatusTracker> task, StatusUpdate> status) {
+ events.add(Event.UPDATE);
+ }
+
+ @Override
+ public void taskFinished(StatusTracker> task) {
+ events.add(Event.FINISH);
+ }
+ }
+}
diff --git a/internal-apis/src/test/java/io/github/jbellis/jvector/status/StatusTrackingIntegrationTest.java b/internal-apis/src/test/java/io/github/jbellis/jvector/status/StatusTrackingIntegrationTest.java
new file mode 100644
index 000000000..477147f61
--- /dev/null
+++ b/internal-apis/src/test/java/io/github/jbellis/jvector/status/StatusTrackingIntegrationTest.java
@@ -0,0 +1,93 @@
+/*
+ * Copyright DataStax, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package io.github.jbellis.jvector.status;
+
+import io.github.jbellis.jvector.status.eventing.RunState;
+import io.github.jbellis.jvector.status.eventing.StatusSink;
+import io.github.jbellis.jvector.status.eventing.StatusSource;
+import io.github.jbellis.jvector.status.eventing.StatusUpdate;
+import io.github.jbellis.jvector.status.sinks.MetricsStatusSink;
+import io.github.jbellis.jvector.status.StatusScope;
+import org.junit.jupiter.api.Test;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.junit.jupiter.api.Assertions.*;
+
+class StatusTrackingIntegrationTest {
+
+ private static final class RecordingSink implements StatusSink {
+ final List states = new ArrayList<>();
+
+ @Override
+ public void taskStarted(StatusTracker> task) {
+ states.add(RunState.PENDING);
+ }
+
+ @Override
+ public void taskUpdate(StatusTracker> task, StatusUpdate> status) {
+ states.add(status.runstate);
+ }
+
+ @Override
+ public void taskFinished(StatusTracker> task) {
+ states.add(RunState.SUCCESS);
+ }
+ }
+
+ private static final class WorkTask implements StatusSource {
+ private volatile double progress;
+ private volatile RunState state = RunState.PENDING;
+
+ @Override
+ public StatusUpdate getTaskStatus() {
+ return new StatusUpdate<>(progress, state, this);
+ }
+
+ void advance(double delta) {
+ progress = Math.min(1.0, progress + delta);
+ state = progress >= 1.0 ? RunState.SUCCESS : RunState.RUNNING;
+ }
+ }
+
+ @Test
+ void trackersShareContextSinks() throws InterruptedException {
+ RecordingSink sink = new RecordingSink();
+ MetricsStatusSink metrics = new MetricsStatusSink();
+
+ try (StatusContext context = new StatusContext("integration", Duration.ofMillis(15), List.of(sink, metrics))) {
+ try (StatusScope scope = context.createScope("TestWorkload")) {
+ WorkTask task1 = new WorkTask();
+ WorkTask task2 = new WorkTask();
+ try (StatusTracker tracker1 = scope.trackTask(task1);
+ StatusTracker tracker2 = scope.trackTask(task2)) {
+ task1.advance(0.5);
+ task2.advance(1.0);
+ Thread.sleep(50);
+ assertEquals(RunState.RUNNING, tracker1.getStatus().runstate);
+ assertEquals(RunState.SUCCESS, tracker2.getStatus().runstate);
+ }
+ }
+ }
+
+ assertTrue(metrics.getTotalTasksStarted() >= 2);
+ assertTrue(metrics.getTotalTasksFinished() >= 2);
+ assertTrue(sink.states.contains(RunState.RUNNING));
+ }
+}
diff --git a/internal-apis/src/test/java/io/github/jbellis/jvector/status/StatusTrackingOverheadBenchmark.java b/internal-apis/src/test/java/io/github/jbellis/jvector/status/StatusTrackingOverheadBenchmark.java
new file mode 100644
index 000000000..677e845a1
--- /dev/null
+++ b/internal-apis/src/test/java/io/github/jbellis/jvector/status/StatusTrackingOverheadBenchmark.java
@@ -0,0 +1,401 @@
+/*
+ * Copyright DataStax, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package io.github.jbellis.jvector.status;
+
+import io.github.jbellis.jvector.status.eventing.RunState;
+import io.github.jbellis.jvector.status.eventing.StatusSource;
+import io.github.jbellis.jvector.status.eventing.StatusUpdate;
+import org.openjdk.jmh.annotations.*;
+import org.openjdk.jmh.infra.Blackhole;
+
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * JMH benchmarks characterizing the per-iteration overhead of status tracking.
+ *
+ * Uses @Setup/@TearDown to amortize context lifecycle costs and measure
+ * only the actual progress tracking overhead during task execution.
+ *
+ *
Benchmarks compare:
+ *
+ * - Baseline: No tracking
+ * - Efficient pattern: Volatile long increment
+ * - Inefficient pattern: Calculate fraction every iteration
+ * - Batched: Update every N iterations
+ * - Atomic: AtomicLong for parallel safety
+ * - Different poll intervals: 100ms, 500ms, 1000ms
+ *
+ */
+@BenchmarkMode(Mode.AverageTime)
+@OutputTimeUnit(TimeUnit.MILLISECONDS)
+@Warmup(iterations = 3, time = 1)
+@Measurement(iterations = 5, time = 2)
+@Fork(1)
+public class StatusTrackingOverheadBenchmark {
+
+ // Baseline: no tracking
+ @State(Scope.Thread)
+ public static class BaselineState {
+ @Param({"100000", "1000000", "10000000"})
+ int iterations;
+
+ long itemsProcessed = 0;
+
+ public void process(Blackhole bh) {
+ for (int i = 0; i < iterations; i++) {
+ bh.consume(i * 2);
+ itemsProcessed++;
+ }
+ }
+ }
+
+ // Efficient pattern with volatile long
+ @State(Scope.Thread)
+ public static class EfficientState {
+ @Param({"100000", "1000000", "10000000"})
+ int iterations;
+
+ @Param({"100", "500", "1000"})
+ int pollIntervalMs;
+
+ StatusContext context;
+ StatusTracker tracker;
+ Task task;
+
+ static class Task implements StatusSource {
+ private volatile long itemsProcessed = 0;
+ private volatile RunState state = RunState.PENDING;
+ private final long total;
+
+ Task(long total) {
+ this.total = total;
+ }
+
+ @Override
+ public StatusUpdate getTaskStatus() {
+ double progress = (double) itemsProcessed / total;
+ return new StatusUpdate<>(progress, state, this);
+ }
+
+ public void process(Blackhole bh, int count) {
+ state = RunState.RUNNING;
+ for (int i = 0; i < count; i++) {
+ bh.consume(i * 2);
+ itemsProcessed++; // Just increment
+ }
+ state = RunState.SUCCESS;
+ }
+
+ public void reset() {
+ itemsProcessed = 0;
+ state = RunState.PENDING;
+ }
+ }
+
+ @Setup(Level.Trial)
+ public void setup() {
+ context = new StatusContext("bench", Duration.ofMillis(pollIntervalMs));
+ task = new Task(iterations);
+ tracker = context.track(task);
+ }
+
+ @TearDown(Level.Trial)
+ public void teardown() {
+ tracker.close();
+ context.close();
+ }
+
+ @Setup(Level.Invocation)
+ public void resetTask() {
+ task.reset();
+ }
+ }
+
+ // Inefficient pattern: division every iteration
+ @State(Scope.Thread)
+ public static class InefficientState {
+ @Param({"100000", "1000000", "10000000"})
+ int iterations;
+
+ StatusContext context;
+ StatusTracker tracker;
+ Task task;
+
+ static class Task implements StatusSource {
+ private volatile double progress = 0.0;
+ private volatile RunState state = RunState.PENDING;
+ private final long total;
+
+ Task(long total) {
+ this.total = total;
+ }
+
+ @Override
+ public StatusUpdate getTaskStatus() {
+ return new StatusUpdate<>(progress, state, this);
+ }
+
+ public void process(Blackhole bh, int count) {
+ state = RunState.RUNNING;
+ for (int i = 0; i < count; i++) {
+ bh.consume(i * 2);
+ progress = (double) (i + 1) / total; // Division every iteration
+ }
+ state = RunState.SUCCESS;
+ }
+
+ public void reset() {
+ progress = 0.0;
+ state = RunState.PENDING;
+ }
+ }
+
+ @Setup(Level.Trial)
+ public void setup() {
+ context = new StatusContext("bench", Duration.ofMillis(100));
+ task = new Task(iterations);
+ tracker = context.track(task);
+ }
+
+ @TearDown(Level.Trial)
+ public void teardown() {
+ tracker.close();
+ context.close();
+ }
+
+ @Setup(Level.Invocation)
+ public void resetTask() {
+ task.reset();
+ }
+ }
+
+ // Batched updates
+ @State(Scope.Thread)
+ public static class BatchedState {
+ @Param({"100000", "1000000", "10000000"})
+ int iterations;
+
+ @Param({"100", "10"}) // 1% and 10% batching
+ int batchesPerTask;
+
+ StatusContext context;
+ StatusTracker tracker;
+ Task task;
+ long batchSize;
+
+ static class Task implements StatusSource {
+ private volatile long itemsProcessed = 0;
+ private volatile RunState state = RunState.PENDING;
+ private final long total;
+
+ Task(long total) {
+ this.total = total;
+ }
+
+ @Override
+ public StatusUpdate getTaskStatus() {
+ double progress = (double) itemsProcessed / total;
+ return new StatusUpdate<>(progress, state, this);
+ }
+
+ public void process(Blackhole bh, int count, long batchSize) {
+ state = RunState.RUNNING;
+ long batchCount = 0;
+
+ for (int i = 0; i < count; i++) {
+ bh.consume(i * 2);
+ batchCount++;
+
+ if (batchCount >= batchSize) {
+ itemsProcessed += batchCount;
+ batchCount = 0;
+ }
+ }
+
+ itemsProcessed += batchCount;
+ state = RunState.SUCCESS;
+ }
+
+ public void reset() {
+ itemsProcessed = 0;
+ state = RunState.PENDING;
+ }
+ }
+
+ @Setup(Level.Trial)
+ public void setup() {
+ context = new StatusContext("bench", Duration.ofMillis(100));
+ task = new Task(iterations);
+ tracker = context.track(task);
+ batchSize = Math.max(1, iterations / batchesPerTask);
+ }
+
+ @TearDown(Level.Trial)
+ public void teardown() {
+ tracker.close();
+ context.close();
+ }
+
+ @Setup(Level.Invocation)
+ public void resetTask() {
+ task.reset();
+ }
+ }
+
+ // AtomicLong for parallel safety
+ @State(Scope.Thread)
+ public static class AtomicState {
+ @Param({"100000", "1000000", "10000000"})
+ int iterations;
+
+ StatusContext context;
+ StatusTracker tracker;
+ Task task;
+
+ static class Task implements StatusSource {
+ private final AtomicLong itemsProcessed = new AtomicLong(0);
+ private volatile RunState state = RunState.PENDING;
+ private final long total;
+
+ Task(long total) {
+ this.total = total;
+ }
+
+ @Override
+ public StatusUpdate getTaskStatus() {
+ double progress = (double) itemsProcessed.get() / total;
+ return new StatusUpdate<>(progress, state, this);
+ }
+
+ public void process(Blackhole bh, int count) {
+ state = RunState.RUNNING;
+ for (int i = 0; i < count; i++) {
+ bh.consume(i * 2);
+ itemsProcessed.incrementAndGet();
+ }
+ state = RunState.SUCCESS;
+ }
+
+ public void reset() {
+ itemsProcessed.set(0);
+ state = RunState.PENDING;
+ }
+ }
+
+ @Setup(Level.Trial)
+ public void setup() {
+ context = new StatusContext("bench", Duration.ofMillis(100));
+ task = new Task(iterations);
+ tracker = context.track(task);
+ }
+
+ @TearDown(Level.Trial)
+ public void teardown() {
+ tracker.close();
+ context.close();
+ }
+
+ @Setup(Level.Invocation)
+ public void resetTask() {
+ task.reset();
+ }
+ }
+
+ // ==================== BASELINE BENCHMARKS ====================
+
+ /**
+ * Baseline: No tracking overhead - pure task execution cost.
+ * Use this to calculate relative slowdown of tracked versions.
+ */
+ @Benchmark
+ public void baseline_noTracking(BaselineState state, Blackhole bh) {
+ state.process(bh);
+ }
+
+ // ==================== TRACKED BENCHMARKS ====================
+
+ /**
+ * Efficient pattern with volatile long increment.
+ * Compare against baseline_noTracking to see volatile write overhead.
+ */
+ @Benchmark
+ public void tracked_efficient(EfficientState state, Blackhole bh) {
+ state.task.process(bh, state.iterations);
+ }
+
+ /**
+ * Inefficient pattern: division every iteration.
+ * Compare against baseline_noTracking to see division overhead.
+ */
+ @Benchmark
+ public void tracked_inefficient(InefficientState state, Blackhole bh) {
+ state.task.process(bh, state.iterations);
+ }
+
+ /**
+ * Batched updates: Only update every Nth iteration.
+ * Compare against baseline_noTracking to see batching overhead.
+ * This should show the LOWEST overhead of all tracked variants.
+ */
+ @Benchmark
+ public void tracked_batched(BatchedState state, Blackhole bh) {
+ state.task.process(bh, state.iterations, state.batchSize);
+ }
+
+ /**
+ * AtomicLong for thread-safe operations.
+ * Compare against baseline_noTracking to see atomic operation overhead.
+ */
+ @Benchmark
+ public void tracked_atomic(AtomicState state, Blackhole bh) {
+ state.task.process(bh, state.iterations);
+ }
+
+ // ==================== RELATIVE SLOWDOWN COMPARISONS ====================
+
+ /**
+ * Calculate relative slowdown by comparing results:
+ *
+ * Relative Slowdown = (tracked_time / baseline_time) - 1
+ *
+ * Example from previous run @ 10M iterations:
+ * - Baseline: 0.766ms
+ * - Batched (1%): 2.975ms β (2.975 / 0.766) - 1 = 2.88x = 288% slowdown
+ * - Batched (10%): 3.026ms β (3.026 / 0.766) - 1 = 2.95x = 295% slowdown
+ * - Efficient: 44.353ms β (44.353 / 0.766) - 1 = 56.9x = 5690% slowdown
+ * - Inefficient: 3.911ms β (3.911 / 0.766) - 1 = 4.10x = 410% slowdown
+ * - Atomic: 19.963ms β (19.963 / 0.766) - 1 = 25.0x = 2500% slowdown
+ *
+ * Key Findings:
+ * 1. Batching (1% or 10%): ~3-4x slowdown - RECOMMENDED
+ * 2. Inefficient (division): ~4-5x slowdown - Acceptable for <1M iterations
+ * 3. Atomic: ~20-25x slowdown - Only use when thread safety required
+ * 4. Efficient (volatile): ~50-60x slowdown - AVOID for hot loops!
+ *
+ * Conclusion: Use batching for minimal overhead (3-4x vs baseline)
+ */
+
+ /**
+ * Main method to run benchmarks from IDE or command line.
+ */
+ public static void main(String[] args) throws Exception {
+ org.openjdk.jmh.Main.main(args);
+ }
+}
+
diff --git a/internal-apis/src/test/java/io/github/jbellis/jvector/status/StatusUpdateTest.java b/internal-apis/src/test/java/io/github/jbellis/jvector/status/StatusUpdateTest.java
new file mode 100644
index 000000000..acdac72c1
--- /dev/null
+++ b/internal-apis/src/test/java/io/github/jbellis/jvector/status/StatusUpdateTest.java
@@ -0,0 +1,438 @@
+/*
+ * Copyright DataStax, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package io.github.jbellis.jvector.status;
+
+import io.github.jbellis.jvector.status.eventing.RunState;
+import io.github.jbellis.jvector.status.eventing.StatusUpdate;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.*;
+
+/**
+ * Core functionality tests for TaskStatus and related status tracking components.
+ *
+ * This test suite covers the core task status data structures and their behavior:
+ *
+ * - TaskStatus: Progress tracking, state management, timestamp handling
+ * - TestableTask: Task implementation for testing scenarios
+ * - TaskStatus.Provider: Interface implementation patterns
+ * - Integration: Usage patterns with tracking framework
+ *
+ *
+ * Test Coverage Areas:
+ *
+ * - TaskStatus creation and immutability
+ * - Progress value validation and constraints
+ * - RunState transitions and behavior
+ * - Timestamp accuracy and ordering
+ * - TaskStatus.Provider interface compliance
+ * - TestableTask functionality and edge cases
+ *
+ *
+ * Related Test Classes:
+ *
+ * - {@link StatusTrackerTest} - Higher-level Tracker wrapper functionality
+ * - {@link TaskMonitorTest} - Background monitoring and polling
+ * - {@link StatusTrackingIntegrationTest} - End-to-end integration scenarios
+ * - StatusSinkTest - TaskSink implementations
+ *
+ *
+ * @see StatusUpdate
+ * @see TestableTask
+ * @since 4.0.0
+ */
+@Tag("Core")
+public class StatusUpdateTest {
+
+ /**
+ * Tests basic TaskStatus creation and field access.
+ *
+ * Purpose: Validates that TaskStatus correctly captures and stores
+ * progress, run state, and timestamp information during construction.
+ *
+ * Why Important: TaskStatus is the fundamental data structure
+ * for all status reporting in the framework.
+ *
+ * Coverage: Basic object creation, field access, immutability
+ */
+ @Test
+ public void testTaskStatusBasicCreation() {
+ long beforeTime = System.currentTimeMillis();
+ StatusUpdate status = new StatusUpdate<>(0.75, RunState.RUNNING);
+ long afterTime = System.currentTimeMillis();
+
+ assertEquals(0.75, status.progress, 0.0001);
+ assertEquals(RunState.RUNNING, status.runstate);
+ assertTrue(status.timestamp >= beforeTime && status.timestamp <= afterTime,
+ "Timestamp should be between before and after times");
+ }
+
+ /**
+ * Tests TaskStatus timestamp ordering and accuracy.
+ *
+ * Purpose: Validates that TaskStatus instances capture
+ * accurate timestamps that reflect their creation order.
+ *
+ * Why Important: Timestamp ordering is crucial for
+ * monitoring systems that track progress over time.
+ *
+ * Coverage: Timestamp accuracy, temporal ordering
+ */
+ @Test
+ public void testTaskStatusTimestampOrdering() throws InterruptedException {
+ StatusUpdate status1 = new StatusUpdate<>(0.25, RunState.RUNNING);
+
+ // Small delay to ensure different timestamps
+ Thread.sleep(1);
+
+ StatusUpdate status2 = new StatusUpdate<>(0.50, RunState.RUNNING);
+
+ Thread.sleep(1);
+
+ StatusUpdate status3 = new StatusUpdate<>(0.75, RunState.SUCCESS);
+
+ assertTrue(status1.timestamp < status2.timestamp,
+ "First status should have earlier timestamp");
+ assertTrue(status2.timestamp < status3.timestamp,
+ "Second status should have earlier timestamp than third");
+ assertTrue(status1.timestamp > 0 && status2.timestamp > 0 && status3.timestamp > 0,
+ "All timestamps should be reasonable");
+ }
+
+ /**
+ * Tests RunState enum values and glyph representations.
+ *
+ * Purpose: Validates that all RunState enum values
+ * exist and have appropriate string representations.
+ *
+ * Coverage: Enum completeness, string representations
+ */
+ @Test
+ public void testRunStateEnum() {
+ // Test all enum values exist
+ RunState[] states = RunState.values();
+ assertEquals(5, states.length, "Should have exactly 5 run states");
+
+ // Test specific states exist
+ assertNotNull(RunState.PENDING);
+ assertNotNull(RunState.RUNNING);
+ assertNotNull(RunState.SUCCESS);
+ assertNotNull(RunState.FAILED);
+ assertNotNull(RunState.CANCELLED);
+
+ // Test states can be used in TaskStatus
+ for (RunState state : states) {
+ StatusUpdate status = new StatusUpdate<>(0.5, state);
+ assertEquals(state, status.runstate);
+ }
+ }
+
+ /**
+ * Tests TestableTask basic functionality and lifecycle.
+ *
+ * Purpose: Validates that TestableTask correctly implements
+ * TaskStatus.Provider and provides expected task behavior.
+ *
+ * Coverage: TestableTask creation, progress tracking, state management
+ */
+ @Test
+ public void testTestableTaskBasicFunctionality() {
+ TestableTask task = new TestableTask("test-task");
+
+ // Test initial state
+ assertEquals("test-task", task.getName());
+ assertEquals(0.0, task.getProgress(), 0.0001);
+ assertEquals(RunState.PENDING, task.getState());
+
+ // Test progress setting
+ task.setProgress(0.5);
+ assertEquals(0.5, task.getProgress(), 0.0001);
+ assertEquals(RunState.RUNNING, task.getState());
+
+ // Test completion
+ task.complete();
+ assertEquals(1.0, task.getProgress(), 0.0001);
+ assertEquals(RunState.SUCCESS, task.getState());
+ }
+
+ /**
+ * Tests TestableTask TaskStatus.Provider implementation.
+ *
+ * Purpose: Validates that TestableTask properly implements
+ * the TaskStatus.Provider interface and returns consistent status objects.
+ *
+ * Coverage: TaskStatus.Provider implementation, status consistency
+ */
+ @Test
+ public void testTestableTaskStatusProvider() throws InterruptedException {
+ TestableTask task = new TestableTask("provider-test");
+
+ // Test initial status
+ StatusUpdate status1 = task.getTaskStatus();
+ assertEquals(0.0, status1.progress, 0.0001);
+ assertEquals(RunState.PENDING, status1.runstate);
+
+ // Change task state and test status update
+ task.setProgress(0.75);
+
+ // Small delay to ensure different timestamps
+ Thread.sleep(1);
+
+ StatusUpdate status2 = task.getTaskStatus();
+ assertEquals(0.75, status2.progress, 0.0001);
+ assertEquals(RunState.RUNNING, status2.runstate);
+
+ // Verify timestamps are different
+ assertTrue(status2.timestamp > status1.timestamp,
+ "Status timestamps should be different");
+ }
+
+ /**
+ * Tests TestableTask lifecycle state transitions.
+ *
+ * Purpose: Validates that TestableTask correctly handles
+ * state transitions through its lifecycle methods.
+ *
+ * Coverage: Lifecycle methods, state transitions
+ */
+ @Test
+ public void testTestableTaskLifecycle() {
+ TestableTask task = new TestableTask("lifecycle-test");
+
+ // Test start
+ task.start();
+ assertEquals(RunState.RUNNING, task.getState());
+ assertEquals(0.0, task.getProgress(), 0.0001);
+
+ // Test manual progress
+ task.setProgress(0.3);
+ assertEquals(RunState.RUNNING, task.getState());
+
+ // Test failure
+ task.fail();
+ assertEquals(RunState.FAILED, task.getState());
+ assertEquals(0.3, task.getProgress(), 0.0001); // Progress unchanged
+
+ // Test reset and complete
+ TestableTask task2 = new TestableTask("lifecycle-test-2");
+ task2.complete();
+ assertEquals(RunState.SUCCESS, task2.getState());
+ assertEquals(1.0, task2.getProgress(), 0.0001);
+ }
+
+ /**
+ * Tests TestableTask input validation and error handling.
+ *
+ * Purpose: Validates that TestableTask properly validates
+ * inputs and throws appropriate exceptions for invalid values.
+ *
+ * Coverage: Input validation, exception handling
+ */
+ @Test
+ public void testTestableTaskValidation() {
+ // Test invalid task name
+ try {
+ new TestableTask(null);
+ fail("Should throw exception for null name");
+ } catch (IllegalArgumentException e) {
+ assertTrue(e.getMessage().contains("Task name cannot be null"));
+ }
+
+ try {
+ new TestableTask("");
+ fail("Should throw exception for empty name");
+ } catch (IllegalArgumentException e) {
+ assertTrue(e.getMessage().contains("Task name cannot be null"));
+ }
+
+ // Test invalid progress values
+ TestableTask task = new TestableTask("validation-test");
+
+ try {
+ task.setProgress(-0.1);
+ fail("Should throw exception for negative progress");
+ } catch (IllegalArgumentException e) {
+ assertTrue(e.getMessage().contains("Progress must be between"));
+ }
+
+ try {
+ task.setProgress(1.1);
+ fail("Should throw exception for progress > 1.0");
+ } catch (IllegalArgumentException e) {
+ assertTrue(e.getMessage().contains("Progress must be between"));
+ }
+
+ try {
+ task.setProgress(Double.NaN);
+ fail("Should throw exception for NaN progress");
+ } catch (IllegalArgumentException e) {
+ assertTrue(e.getMessage().contains("Progress must be between"));
+ }
+
+ // Test invalid state
+ try {
+ task.setState(null);
+ fail("Should throw exception for null state");
+ } catch (IllegalArgumentException e) {
+ assertTrue(e.getMessage().contains("State cannot be null"));
+ }
+ }
+
+ /**
+ * Tests TestableTask equality and hash code behavior.
+ *
+ * Purpose: Validates that TestableTask correctly implements
+ * equality based on task name only.
+ *
+ * Coverage: Equals/hashCode contract, name-based equality
+ */
+ @Test
+ public void testTestableTaskEquality() {
+ TestableTask task1 = new TestableTask("same-name");
+ TestableTask task2 = new TestableTask("same-name");
+ TestableTask task3 = new TestableTask("different-name");
+
+ // Test equality
+ assertEquals(task1, task2);
+ assertNotEquals(task1, task3);
+ assertNotEquals(task1, null);
+ assertNotEquals(task1, "not-a-task");
+
+ // Test hash code consistency
+ assertEquals(task1.hashCode(), task2.hashCode());
+
+ // Test that progress/state don't affect equality
+ task1.setProgress(0.5);
+ task2.setProgress(0.8);
+ assertEquals(task1, task2, "Tasks should be equal despite different progress");
+ }
+
+ /**
+ * Tests TestableTask toString representation.
+ *
+ * Purpose: Validates that TestableTask provides meaningful
+ * string representations for debugging and logging.
+ *
+ * Coverage: String representation, debugging output
+ */
+ @Test
+ public void testTestableTaskToString() {
+ TestableTask task = new TestableTask("toString-test");
+ task.setProgress(0.42);
+ task.setState(RunState.RUNNING);
+
+ String str = task.toString();
+ assertTrue(str.contains("toString-test"), "Should contain task name");
+ assertTrue(str.contains("0.42"), "Should contain progress");
+ assertTrue(str.contains("RUNNING"), "Should contain state");
+ assertTrue(str.startsWith("TestableTask{"), "Should be TestableTask format");
+ }
+
+ /**
+ * Tests StatusUpdate with tracked object reference.
+ *
+ * Purpose: Validates that StatusUpdate correctly stores
+ * and provides access to the tracked object reference when provided.
+ *
+ * Coverage: Tracked field initialization, null handling
+ */
+ @Test
+ public void testStatusUpdateWithTrackedObject() {
+ TestableTask task = new TestableTask("tracked-test");
+
+ // Test with tracked object (3-arg constructor)
+ StatusUpdate statusWithTracked = new StatusUpdate<>(0.5, RunState.RUNNING, task);
+ assertEquals(0.5, statusWithTracked.progress, 0.0001);
+ assertEquals(RunState.RUNNING, statusWithTracked.runstate);
+ assertNotNull(statusWithTracked.tracked);
+ assertEquals(task, statusWithTracked.tracked);
+ assertEquals("tracked-test", statusWithTracked.tracked.getName());
+
+ // Test without tracked object (2-arg constructor)
+ StatusUpdate statusWithoutTracked = new StatusUpdate<>(0.75, RunState.SUCCESS);
+ assertEquals(0.75, statusWithoutTracked.progress, 0.0001);
+ assertEquals(RunState.SUCCESS, statusWithoutTracked.runstate);
+ assertNull(statusWithoutTracked.tracked);
+
+ // Test that TestableTask's getTaskStatus includes tracked reference
+ task.setProgress(0.8);
+ StatusUpdate taskStatus = task.getTaskStatus();
+ assertEquals(0.8, taskStatus.progress, 0.0001);
+ assertEquals(RunState.RUNNING, taskStatus.runstate);
+ assertNotNull(taskStatus.tracked);
+ assertEquals(task, taskStatus.tracked);
+ }
+
+ /**
+ * Tests that Provider implementations return StatusUpdate with tracked reference.
+ */
+ @Test
+ public void testProviderTrackedReference() {
+ TestableTask task = new TestableTask("provider-tracked-test");
+
+ // Test PENDING state
+ StatusUpdate pendingStatus = task.getTaskStatus();
+ assertSame(task, pendingStatus.tracked, "Provider should return self as tracked");
+
+ // Test RUNNING state
+ task.setProgress(0.5);
+ StatusUpdate runningStatus = task.getTaskStatus();
+ assertSame(task, runningStatus.tracked, "Provider should return self as tracked");
+
+ // Test SUCCESS state
+ task.complete();
+ StatusUpdate successStatus = task.getTaskStatus();
+ assertSame(task, successStatus.tracked, "Provider should return self as tracked");
+ assertEquals(1.0, successStatus.progress, 0.0001);
+ }
+
+ /**
+ * Tests progress auto-state transitions in TestableTask.
+ *
+ * Purpose: Validates that TestableTask automatically
+ * updates its run state based on progress value changes.
+ *
+ * Coverage: Auto-state transitions, progress-based state logic
+ */
+ @Test
+ public void testTestableTaskAutoStateTransitions() {
+ TestableTask task = new TestableTask("auto-state-test");
+
+ // Initial state should be PENDING
+ assertEquals(RunState.PENDING, task.getState());
+
+ // Setting progress > 0 and < 1 should go to RUNNING
+ task.setProgress(0.1);
+ assertEquals(RunState.RUNNING, task.getState());
+
+ task.setProgress(0.5);
+ assertEquals(RunState.RUNNING, task.getState());
+
+ task.setProgress(0.99);
+ assertEquals(RunState.RUNNING, task.getState());
+
+ // Setting progress to 1.0 should go to SUCCESS
+ task.setProgress(1.0);
+ assertEquals(RunState.SUCCESS, task.getState());
+
+ // Test that setting to 0 from PENDING stays PENDING
+ TestableTask task2 = new TestableTask("auto-state-test-2");
+ task2.setProgress(0.0);
+ assertEquals(RunState.PENDING, task2.getState());
+ }
+}
diff --git a/internal-apis/src/test/java/io/github/jbellis/jvector/status/TaskMonitorTest.java b/internal-apis/src/test/java/io/github/jbellis/jvector/status/TaskMonitorTest.java
new file mode 100644
index 000000000..7c4628c1c
--- /dev/null
+++ b/internal-apis/src/test/java/io/github/jbellis/jvector/status/TaskMonitorTest.java
@@ -0,0 +1,90 @@
+/*
+ * Copyright DataStax, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package io.github.jbellis.jvector.status;
+
+import io.github.jbellis.jvector.status.eventing.RunState;
+import io.github.jbellis.jvector.status.eventing.StatusSource;
+import io.github.jbellis.jvector.status.eventing.StatusUpdate;
+import org.junit.jupiter.api.Test;
+
+import java.time.Duration;
+
+import static org.junit.jupiter.api.Assertions.*;
+
+class TaskMonitorTest {
+
+ private static final class SampleTask implements StatusSource {
+ private volatile RunState state = RunState.PENDING;
+ private volatile double progress = 0.0;
+
+ @Override
+ public StatusUpdate getTaskStatus() {
+ return new StatusUpdate<>(progress, state, this);
+ }
+
+ void start(double progress) {
+ this.state = RunState.RUNNING;
+ this.progress = progress;
+ }
+
+ void finish() {
+ this.progress = 1.0;
+ this.state = RunState.SUCCESS;
+ }
+ }
+
+ @Test
+ void contextPollsTrackedTasks() throws InterruptedException {
+ SampleTask task = new SampleTask();
+ try (StatusContext context = new StatusContext("polling", Duration.ofMillis(20));
+ StatusScope scope = context.createScope("test-scope");
+ StatusTracker tracker = scope.trackTask(task)) {
+
+ assertEquals(RunState.PENDING, tracker.getStatus().runstate);
+
+ task.start(0.25);
+ Thread.sleep(50);
+ assertEquals(RunState.RUNNING, tracker.getStatus().runstate);
+ assertEquals(0.25, tracker.getStatus().progress, 1e-6);
+
+ task.finish();
+ Thread.sleep(50);
+ assertEquals(RunState.SUCCESS, tracker.getStatus().runstate);
+ assertEquals(1.0, tracker.getStatus().progress, 1e-6);
+ }
+ }
+
+ @Test
+ void closingContextStopsPolling() throws InterruptedException {
+ SampleTask task = new SampleTask();
+ StatusContext context = new StatusContext("closable", Duration.ofMillis(10));
+ StatusScope scope = context.createScope("test-scope");
+ StatusTracker tracker = scope.trackTask(task);
+
+ task.start(0.1);
+ Thread.sleep(30);
+ assertEquals(RunState.RUNNING, tracker.getStatus().runstate);
+
+ context.close();
+
+ task.finish();
+ Thread.sleep(30);
+ // Status should remain RUNNING because polling thread is stopped.
+ assertEquals(RunState.RUNNING, tracker.getStatus().runstate);
+ tracker.close();
+ }
+}
diff --git a/internal-apis/src/test/java/io/github/jbellis/jvector/status/TestableTask.java b/internal-apis/src/test/java/io/github/jbellis/jvector/status/TestableTask.java
new file mode 100644
index 000000000..7fd729818
--- /dev/null
+++ b/internal-apis/src/test/java/io/github/jbellis/jvector/status/TestableTask.java
@@ -0,0 +1,193 @@
+/*
+ * Copyright DataStax, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package io.github.jbellis.jvector.status;
+
+import io.github.jbellis.jvector.status.eventing.RunState;
+import io.github.jbellis.jvector.status.eventing.StatusSource;
+import io.github.jbellis.jvector.status.eventing.StatusUpdate;
+
+import java.util.Objects;
+
+/**
+ * A testable task implementation that provides basic progress tracking functionality for testing.
+ * This class serves as a test utility for validating the task tracking framework
+ * without requiring complex domain objects or business logic.
+ *
+ * This implementation provides:
+ *
+ * - Named task identification
+ * - Thread-safe progress tracking (0.0 to 1.0)
+ * - TaskStatus.Provider implementation for use with Tracker
+ * - Simple lifecycle management
+ *
+ *
+ * Usage Examples:
+ *
+ * Basic Progress Tracking in Tests
+ * {@code
+ * TestableTask task = new TestableTask("data-processing");
+ * StatusContext context = new StatusContext("tests");
+ * try (StatusTracker tracker = context.track(task, sinks)) {
+ * task.setProgress(0.25); // 25% complete
+ * doSomeWork();
+ * task.setProgress(0.75); // 75% complete
+ * doMoreWork();
+ * task.setProgress(1.0); // Complete
+ * }
+ * }
+ *
+ * Integration with StatusContext
+ * {@code
+ * StatusContext context = new StatusContext("batch-processing");
+ * TestableTask task = new TestableTask("file-processing");
+ * try (StatusTracker tracker = context.track(task)) {
+ * // Task progress is automatically monitored and reported
+ * for (int i = 0; i < 100; i++) {
+ * processItem(i);
+ * task.setProgress(i / 100.0);
+ * }
+ * }
+ * }
+ *
+ * @since 4.0.0
+ */
+public final class TestableTask implements StatusSource {
+ private final String name;
+ private volatile double progress = 0.0;
+ private volatile RunState state = RunState.PENDING;
+
+ /**
+ * Creates a new testable task with the specified name.
+ *
+ * @param name the name of the task
+ * @throws IllegalArgumentException if name is null or empty
+ */
+ public TestableTask(String name) {
+ if (name == null || name.trim().isEmpty()) {
+ throw new IllegalArgumentException("Task name cannot be null or empty");
+ }
+ this.name = name.trim();
+ }
+
+ /**
+ * Gets the name of this task.
+ *
+ * @return the task name
+ */
+ public String getName() {
+ return name;
+ }
+
+ /**
+ * Gets the current progress of this task.
+ *
+ * @return progress value between 0.0 (0%) and 1.0 (100%)
+ */
+ public double getProgress() {
+ return progress;
+ }
+
+ /**
+ * Sets the current progress of this task.
+ *
+ * @param progress the progress value between 0.0 (0%) and 1.0 (100%)
+ * @throws IllegalArgumentException if progress is not between 0.0 and 1.0, or is NaN/infinite
+ */
+ public void setProgress(double progress) {
+ if (progress < 0.0 || progress > 1.0 || !Double.isFinite(progress)) {
+ throw new IllegalArgumentException("Progress must be between 0.0 and 1.0, inclusive, got: " + progress);
+ }
+ this.progress = progress;
+
+ // Auto-update state based on progress
+ if (progress == 0.0 && state == RunState.PENDING) {
+ // Stay in PENDING state
+ } else if (progress > 0.0 && progress < 1.0) {
+ state = RunState.RUNNING;
+ } else if (progress == 1.0) {
+ state = RunState.SUCCESS;
+ }
+ }
+
+ /**
+ * Gets the current run state of this task.
+ *
+ * @return the current state
+ */
+ public RunState getState() {
+ return state;
+ }
+
+ /**
+ * Sets the current run state of this task.
+ *
+ * @param state the new state
+ * @throws IllegalArgumentException if state is null
+ */
+ public void setState(RunState state) {
+ if (state == null) {
+ throw new IllegalArgumentException("State cannot be null");
+ }
+ this.state = state;
+ }
+
+ /**
+ * Marks this task as started (RUNNING state).
+ */
+ public void start() {
+ this.state = RunState.RUNNING;
+ }
+
+ /**
+ * Marks this task as completed (SUCCESS state and 100% progress).
+ */
+ public void complete() {
+ this.progress = 1.0;
+ this.state = RunState.SUCCESS;
+ }
+
+ /**
+ * Marks this task as failed (FAILED state).
+ */
+ public void fail() {
+ this.state = RunState.FAILED;
+ }
+
+ @Override
+ public StatusUpdate getTaskStatus() {
+ return new StatusUpdate<>(progress, state, this);
+ }
+
+ @Override
+ public String toString() {
+ return String.format("TestableTask{name='%s', progress=%.2f, state=%s}",
+ name, progress, state);
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (this == obj) return true;
+ if (obj == null || getClass() != obj.getClass()) return false;
+ TestableTask that = (TestableTask) obj;
+ return Objects.equals(name, that.name);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(name);
+ }
+}
diff --git a/internal-apis/src/test/java/io/github/jbellis/jvector/status/asyncdag/LayeredTaskTest.java b/internal-apis/src/test/java/io/github/jbellis/jvector/status/asyncdag/LayeredTaskTest.java
new file mode 100644
index 000000000..fec947240
--- /dev/null
+++ b/internal-apis/src/test/java/io/github/jbellis/jvector/status/asyncdag/LayeredTaskTest.java
@@ -0,0 +1,110 @@
+/*
+ * Copyright DataStax, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package io.github.jbellis.jvector.status.asyncdag;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.concurrent.*;
+
+public class LayeredTaskTest {
+
+ @Test
+ public void testThreeLayerTaskStructure() throws Exception {
+ ExecutorService executor = Executors.newCachedThreadPool();
+
+ try {
+ TaskLayer task1a = new TaskLayer("compute-1a", 500, "Result-1a");
+ TaskLayer task1b = new TaskLayer("compute-1b", 700, "Result-1b");
+ TaskLayer task1c = new TaskLayer("compute-1c", 300, "Result-1c");
+ TaskLayer task1d = new TaskLayer("compute-1d", 600, "Result-1d");
+
+ TaskLayer task2a = new TaskLayer("process-2a", 2, executor);
+ task2a.addSubTask(task1a);
+ task2a.addSubTask(task1b);
+
+ TaskLayer task2b = new TaskLayer("process-2b", 2, executor);
+ task2b.addSubTask(task1c);
+ task2b.addSubTask(task1d);
+
+ TaskLayer task3 = new TaskLayer("orchestrate-3", 3, executor);
+ task3.addSubTask(task2a);
+ task3.addSubTask(task2b);
+
+ System.out.println("Starting three-layer task execution...\n");
+
+ Future result = executor.submit(task3);
+
+ String finalResult = result.get(10, TimeUnit.SECONDS);
+
+ System.out.println("\n=== Final Result ===");
+ System.out.println(finalResult);
+
+ } finally {
+ executor.shutdown();
+ if (!executor.awaitTermination(5, TimeUnit.SECONDS)) {
+ executor.shutdownNow();
+ }
+ }
+ }
+
+ @Test
+ public void testParallelTaskExecution() throws Exception {
+ ExecutorService executor = Executors.newFixedThreadPool(4);
+
+ try {
+ TaskLayer[] layer1Tasks = new TaskLayer[6];
+ for (int i = 0; i < 6; i++) {
+ layer1Tasks[i] = new TaskLayer(
+ "worker-" + i,
+ 200 + (i * 100),
+ "Data-" + i
+ );
+ }
+
+ TaskLayer task2a = new TaskLayer("aggregator-A", 2, executor);
+ task2a.addSubTask(layer1Tasks[0]);
+ task2a.addSubTask(layer1Tasks[1]);
+ task2a.addSubTask(layer1Tasks[2]);
+
+ TaskLayer task2b = new TaskLayer("aggregator-B", 2, executor);
+ task2b.addSubTask(layer1Tasks[3]);
+ task2b.addSubTask(layer1Tasks[4]);
+ task2b.addSubTask(layer1Tasks[5]);
+
+ TaskLayer masterTask = new TaskLayer("master", 3, executor);
+ masterTask.addSubTask(task2a);
+ masterTask.addSubTask(task2b);
+
+ System.out.println("Starting parallel task execution with 6 workers...\n");
+
+ long startTime = System.currentTimeMillis();
+ Future result = executor.submit(masterTask);
+ String finalResult = result.get(15, TimeUnit.SECONDS);
+ long endTime = System.currentTimeMillis();
+
+ System.out.println("\n=== Execution Summary ===");
+ System.out.println("Total execution time: " + (endTime - startTime) + "ms");
+ System.out.println(finalResult);
+
+ } finally {
+ executor.shutdown();
+ if (!executor.awaitTermination(5, TimeUnit.SECONDS)) {
+ executor.shutdownNow();
+ }
+ }
+ }
+}
diff --git a/internal-apis/src/test/java/io/github/jbellis/jvector/status/asyncdag/TaskLayer.java b/internal-apis/src/test/java/io/github/jbellis/jvector/status/asyncdag/TaskLayer.java
new file mode 100644
index 000000000..c115a7b84
--- /dev/null
+++ b/internal-apis/src/test/java/io/github/jbellis/jvector/status/asyncdag/TaskLayer.java
@@ -0,0 +1,86 @@
+/*
+ * Copyright DataStax, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package io.github.jbellis.jvector.status.asyncdag;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.*;
+
+public class TaskLayer implements Callable {
+ private final String taskName;
+ private final int layer;
+ private final List subTasks;
+ private final ExecutorService executor;
+ private final long workDurationMillis;
+ private final String workResult;
+
+ public TaskLayer(String taskName, int layer, ExecutorService executor) {
+ this.taskName = taskName;
+ this.layer = layer;
+ this.executor = executor;
+ this.subTasks = new ArrayList<>();
+ this.workDurationMillis = 0;
+ this.workResult = null;
+ }
+
+ public TaskLayer(String taskName, long workDurationMillis, String workResult) {
+ this.taskName = taskName;
+ this.layer = 1;
+ this.executor = null;
+ this.subTasks = new ArrayList<>();
+ this.workDurationMillis = workDurationMillis;
+ this.workResult = workResult;
+ }
+
+ public void addSubTask(TaskLayer task) {
+ subTasks.add(task);
+ }
+
+ @Override
+ public String call() throws Exception {
+ System.out.println("TaskLayer" + layer + "[" + taskName + "] starting");
+
+ if (subTasks.isEmpty()) {
+ long startTime = System.currentTimeMillis();
+ long endTime = startTime + workDurationMillis;
+ while (System.currentTimeMillis() < endTime) {
+ Thread.sleep(Math.min(100, endTime - System.currentTimeMillis()));
+ }
+ System.out.println("TaskLayer" + layer + "[" + taskName + "] completed work, returning: " + workResult);
+ return workResult;
+ } else {
+ System.out.println("TaskLayer" + layer + "[" + taskName + "] delegating to " + subTasks.size() + " subtasks");
+
+ List> futures = new ArrayList<>();
+ for (TaskLayer task : subTasks) {
+ futures.add(executor.submit(task));
+ }
+
+ StringBuilder results = new StringBuilder();
+ results.append("TaskLayer").append(layer).append("[").append(taskName).append("] results: ");
+
+ for (Future future : futures) {
+ String result = future.get();
+ results.append(result).append("; ");
+ }
+
+ String finalResult = results.toString();
+ System.out.println("TaskLayer" + layer + "[" + taskName + "] completed delegation");
+ return finalResult;
+ }
+ }
+}
\ No newline at end of file
diff --git a/internal-apis/src/test/java/io/github/jbellis/jvector/status/examples/ConsolePanelDemo.java b/internal-apis/src/test/java/io/github/jbellis/jvector/status/examples/ConsolePanelDemo.java
new file mode 100644
index 000000000..a88efcb12
--- /dev/null
+++ b/internal-apis/src/test/java/io/github/jbellis/jvector/status/examples/ConsolePanelDemo.java
@@ -0,0 +1,200 @@
+/*
+ * Copyright DataStax, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package io.github.jbellis.jvector.status.examples;
+
+import io.github.jbellis.jvector.status.sinks.ConsolePanelLogIntercept;
+import io.github.jbellis.jvector.status.SimulatedClock;
+import io.github.jbellis.jvector.status.StatusContext;
+import io.github.jbellis.jvector.status.StatusScope;
+import io.github.jbellis.jvector.status.sinks.ConsolePanelSink;
+import io.github.jbellis.jvector.status.sinks.OutputMode;
+
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Demonstration application for {@link ConsolePanelSink} with hierarchical task display and
+ * integrated log panel. This minimal orchestrator creates a context with ConsolePanelSink,
+ * launches a diverse workload via {@link ExampleWorkloadExecutor}, and waits for completion.
+ *
+ * Key Features Demonstrated:
+ *
+ * - Visual Hierarchy: Multi-level task trees with clear parent-child relationships
+ * - Real-time Updates: Live progress bars, status icons, and timing information
+ * - Interactive Controls: Keyboard navigation (ββ for scrolling, q to quit)
+ * - Log Integration: Automatic capture of log4j2 output in scrollable panel
+ * - Terminal Optimization: Efficient differential rendering with JLine3
+ *
+ *
+ * Display Layout
+ * The console panel provides two sections:
+ *
+ * - Task Status Panel (top): Hierarchical view of all active and completed tasks
+ * with progress bars, duration tracking, and color-coded status indicators
+ * - Console Output Panel (bottom): Scrollable log display showing all log4j2
+ * messages with level-based color coding and timestamps
+ *
+ *
+ * Architecture
+ * This demo illustrates the clean separation between organizational scopes and work tasks:
+ *
+ * - {@link ConsolePanelDemo} - Creates context with organizational scope for workload
+ * - {@link StatusScope} - Organizational container with no progress/state
+ * - Task types ({@link ExampleDataProcessingTask}, {@link ExampleComputeTask}, {@link ExampleValidationTask}) -
+ * Leaf nodes with actual progress and state
+ * - {@link StatusContext} - Routes status updates from tasks to sinks
+ * - {@link ConsolePanelSink} - Receives and displays status updates in interactive UI
+ *
+ *
+ * Running the Demo
+ * Execute from the command line with Maven:
+ *
+ * mvn test-compile exec:java -pl internal-apis \
+ * -Dexec.mainClass="io.github.jbellis.jvector.status.examples.ConsolePanelDemo" \
+ * -Dexec.classpathScope=test
+ *
+ *
+ * Interactive Controls
+ *
+ * - β / β: Scroll through console log output
+ * - [ / ]: Adjust split between task and log panels
+ * - PgUp / PgDn: Quick split adjustment
+ * - Home: Reset scroll positions and split to defaults
+ * - End: Jump to end of logs and tasks
+ * - s: Save current display to file
+ * - q: Quit and shutdown (gracefully cancels tasks)
+ *
+ *
+ * Expected Behavior
+ * The demo will:
+ *
+ * - Display startup messages for 3 seconds
+ * - Launch 6 diverse task groups with staggered starts:
+ *
+ * - DataLoad (500 records) - I/O simulation
+ * - VectorIndexing scope - Parallel computation with main task (200 iterations) and 3 worker subtasks
+ * - SchemaValidation (40 checks) - Sequential validation with nested checks
+ * - DataTransform (300 records) - Data processing
+ * - Clustering scope - Concurrent computation with main task (150 iterations) and 2 worker subtasks
+ * - IntegrityCheck (30 checks) - Final validation
+ *
+ *
+ * - Tasks spawn child tasks demonstrating different hierarchy patterns
+ * - All tasks emit log messages at various levels (INFO, DEBUG, WARN, ERROR)
+ * - Tasks complete naturally after their configured work
+ * - Console panel remains visible for 3 seconds after completion
+ * - Application cleans up and exits, printing log history to stdout
+ *
+ *
+ * @see ConsolePanelSink
+ * @see ExampleWorkloadExecutor
+ * @see ExampleDataProcessingTask
+ * @see ExampleComputeTask
+ * @see ExampleValidationTask
+ * @see StatusContext
+ * @since 4.0.0
+ */
+public class ConsolePanelDemo {
+
+ /**
+ * Main entry point for the console panel demonstration. Creates a status context with
+ * ConsolePanelSink, launches a diverse workload with multiple task types, and waits for completion.
+ *
+ * @param args command line arguments (unused)
+ * @throws Exception if initialization or execution fails
+ */
+ public static void main(String[] args) throws Exception {
+ // Configure logging to capture all log events in LogBuffer
+ ConsolePanelLogIntercept.configure(OutputMode.INTERACTIVE);
+
+ // Create simulated clock for time control
+ SimulatedClock clock = new SimulatedClock();
+
+ // Use array to allow lambda to reference the sink before it's fully initialized
+ ConsolePanelSink[] sinkHolder = new ConsolePanelSink[1];
+
+ // Create the enhanced ConsolePanelSink with time control callbacks
+ ConsolePanelSink consolePanelSink = ConsolePanelSink.builder()
+ .withRefreshRateMs(100)
+ .withCompletedTaskRetention(5, TimeUnit.SECONDS)
+ .withColorOutput(true)
+ .withMaxLogLines(100)
+ .withCaptureSystemStreams(true)
+ .withKeyHandler("shift-right", () -> {
+ clock.speedUp();
+ sinkHolder[0].addLogMessage("Time speed: " + clock.getSpeedDescription());
+ })
+ .withKeyHandler("shift-left", () -> {
+ clock.slowDown();
+ sinkHolder[0].addLogMessage("Time speed: " + clock.getSpeedDescription());
+ })
+ .build();
+ sinkHolder[0] = consolePanelSink;
+
+ // Print reminder to terminal before entering interactive mode
+ System.out.println("\n" +
+ "=".repeat(70) + "\n" +
+ " ConsolePanelSink Demo - Interactive Task Monitor\n" +
+ "=".repeat(70) + "\n" +
+ "\n" +
+ " This demo demonstrates:\n" +
+ " β’ DataProcessingTask: I/O-heavy operations (record processing)\n" +
+ " β’ ComputeTask: CPU-intensive work with parallel subtasks\n" +
+ " β’ ValidationTask: Sequential checks with nested validations\n" +
+ "\n" +
+ " Demo-specific feature:\n" +
+ " β’ Shift+β / Shift+β: Control simulated time speed\n" +
+ "\n" +
+ " Press '?' inside the monitor to see all keyboard shortcuts\n" +
+ "\n" +
+ "=".repeat(70) + "\n");
+
+ try (StatusContext context = new StatusContext("console-demo")) {
+ context.addSink(consolePanelSink);
+
+ // Create organizational scope for the workload
+ try (StatusScope workloadScope = context.createScope("DemoWorkload")) {
+
+ // Launch diverse workload - all tasks are within the scope
+ ExampleWorkloadExecutor executor = ExampleWorkloadExecutor.runDemoWorkload(workloadScope, clock);
+
+ // Wait for workload to complete, checking for early shutdown
+ while (!executor.isComplete() && !consolePanelSink.isClosed()) {
+ if (!executor.awaitCompletion(1000)) {
+ if (consolePanelSink.isClosed()) {
+ executor.interruptAll();
+ executor.awaitCompletion(2000);
+ break;
+ }
+ }
+ }
+
+ // Wait for user to close the panel (via 'q' key)
+ // The panel will stay open until user explicitly closes it
+ while (!consolePanelSink.isClosed()) {
+ Thread.sleep(100);
+ }
+ }
+
+ } finally {
+ // Ensure panel is closed (in case of exceptions)
+ if (!consolePanelSink.isClosed()) {
+ consolePanelSink.close();
+ }
+ }
+ }
+
+}
diff --git a/internal-apis/src/test/java/io/github/jbellis/jvector/status/examples/ExampleComputeTask.java b/internal-apis/src/test/java/io/github/jbellis/jvector/status/examples/ExampleComputeTask.java
new file mode 100644
index 000000000..7d4f7ab12
--- /dev/null
+++ b/internal-apis/src/test/java/io/github/jbellis/jvector/status/examples/ExampleComputeTask.java
@@ -0,0 +1,289 @@
+/*
+ * Copyright DataStax, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package io.github.jbellis.jvector.status.examples;
+
+import io.github.jbellis.jvector.status.SimulatedClock;
+import io.github.jbellis.jvector.status.StatusTracker;
+import io.github.jbellis.jvector.status.StatusScope;
+import io.github.jbellis.jvector.status.eventing.RunState;
+import io.github.jbellis.jvector.status.eventing.StatusSource;
+import io.github.jbellis.jvector.status.eventing.StatusUpdate;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+
+/**
+ * Demonstration task that simulates CPU-intensive compute operations with configurable
+ * parallel subtask spawning. This class illustrates hierarchical task organization by
+ * creating its own {@link StatusScope} containing the main computation task and optional
+ * worker subtasks.
+ *
+ * The task performs simulated computation through trigonometric calculations while
+ * emitting log messages at various levels to demonstrate logging integration. When parallel
+ * subtasks are configured, each worker task executes independently with half the iteration
+ * count of the parent task.
+ *
+ *
Task Hierarchy
+ *
+ * ExampleComputeTask (scope owner)
+ * βββ MainTask (actual computation with progress tracking)
+ * βββ Workers (optional nested scope)
+ * βββ Worker1 (ExampleComputeTask with reduced iterations)
+ * βββ Worker2
+ * βββ Worker3...
+ *
+ *
+ * Example Usage
+ * {@code
+ * try (StatusScope scope = context.createScope("MyScope")) {
+ * SimulatedClock clock = new SimulatedClock();
+ * ExampleComputeTask task = new ExampleComputeTask(
+ * "MyComputeTask",
+ * 200, // iterations
+ * 3, // parallel workers
+ * scope,
+ * clock
+ * );
+ * new Thread(task).start();
+ * }
+ * }
+ *
+ * @see StatusScope
+ * @see StatusTracker
+ * @see SimulatedClock
+ * @since 4.0.0
+ */
+class ExampleComputeTask implements Runnable {
+ private static final Logger logger = LogManager.getLogger(ExampleComputeTask.class);
+
+ private final String name;
+ private final int iterations;
+ private final int parallelSubtasks;
+ private final StatusScope parentScope;
+ private final SimulatedClock clock;
+ private final List childThreads = new ArrayList<>();
+
+ /**
+ * Constructs a new compute task with the specified configuration.
+ *
+ * @param name the display name for this task
+ * @param iterations the number of computation iterations to perform
+ * @param parallelSubtasks the number of parallel worker tasks to spawn (0 for no workers)
+ * @param parentScope the parent tracker scope under which this task's scope will be created
+ * @param clock the simulated clock for controlling task timing
+ */
+ ExampleComputeTask(String name, int iterations, int parallelSubtasks, StatusScope parentScope, SimulatedClock clock) {
+ this.name = name;
+ this.iterations = iterations;
+ this.parallelSubtasks = parallelSubtasks;
+ this.parentScope = parentScope;
+ this.clock = clock;
+ }
+
+ /**
+ * Executes the compute task by creating a task-specific scope, spawning parallel
+ * worker tasks if configured, running the main computation, and waiting for all
+ * child tasks to complete. Exceptions are logged but do not propagate.
+ */
+ @Override
+ public void run() {
+ try (StatusScope taskScope = parentScope.createChildScope(name)) {
+ // Spawn parallel worker tasks if needed
+ StatusScope workersScope = null;
+ if (parallelSubtasks > 0) {
+ logger.info("Spawning {} parallel worker tasks for {}", parallelSubtasks, name);
+ workersScope = taskScope.createChildScope("Workers");
+ for (int i = 0; i < parallelSubtasks; i++) {
+ String childName = "Worker" + (i + 1);
+ logger.debug("Creating worker task: {}", childName);
+ ExampleComputeTask childTask = new ExampleComputeTask(childName, iterations / 2, 0, workersScope, clock);
+ Thread childThread = new Thread(childTask);
+ childThreads.add(childThread);
+ childThread.start();
+ logger.trace("Worker {} started with thread ID: {}", childName, childThread.getId());
+ clock.sleep(100);
+ }
+ logger.info("All {} worker tasks spawned successfully", parallelSubtasks);
+ }
+
+ // Execute the main task
+ MainTask mainTask = new MainTask(name, iterations, taskScope, clock);
+ mainTask.execute();
+
+ // Wait for child tasks
+ if (!childThreads.isEmpty()) {
+ logger.debug("Waiting for {} child tasks to complete", childThreads.size());
+ for (Thread child : childThreads) {
+ logger.trace("Joining thread: {}", child.getName());
+ child.join(2000);
+ }
+ logger.info("All child tasks completed for {}", name);
+ }
+
+ // Close workers scope if created
+ if (workersScope != null) {
+ workersScope.close();
+ logger.debug("Workers scope closed for {}", name);
+ }
+ } catch (Exception e) {
+ logger.error("Compute task {} failed: {}", name, e.getMessage());
+ }
+ }
+
+ /**
+ * Inner class representing the actual computation work with progress tracking.
+ * This class implements {@link StatusSource} to provide real-time progress updates
+ * to the status tracking system.
+ *
+ * The task performs computation in iterations, simulating CPU-intensive work through
+ * trigonometric calculations. Progress, state, and completion status are exposed through
+ * the {@link #getTaskStatus()} method.
+ *
+ * @see StatusSource
+ * @see StatusUpdate
+ */
+ private static class MainTask implements StatusSource {
+ private final String name;
+ private final int iterations;
+ private final StatusScope scope;
+ private final SimulatedClock clock;
+ private final Random random = new Random();
+
+ private volatile int iterationsComplete = 0;
+ private volatile RunState state = RunState.PENDING;
+ private volatile boolean interrupted = false;
+
+ /**
+ * Constructs a new main task for computation execution.
+ *
+ * @param name the task name for display purposes
+ * @param iterations the total number of iterations to perform
+ * @param scope the tracker scope for registering this task
+ * @param clock the simulated clock for timing control
+ */
+ MainTask(String name, int iterations, StatusScope scope, SimulatedClock clock) {
+ this.name = name;
+ this.iterations = iterations;
+ this.scope = scope;
+ this.clock = clock;
+ }
+
+ /**
+ * Executes the main computation task with progress tracking. Creates a
+ * {@link StatusTracker} for this task, performs the computation loop,
+ * and handles interruption gracefully.
+ *
+ * @throws InterruptedException if the task is interrupted during execution
+ */
+ void execute() throws InterruptedException {
+ try (StatusTracker tracker = scope.trackTask(this, MainTask::getTaskStatus)) {
+ state = RunState.RUNNING;
+ logger.info("Starting compute task: {} ({} iterations)", name, iterations);
+ logger.debug("Task configuration - Iterations: {}, Tracker: {}", iterations, tracker.getClass().getSimpleName());
+ logger.trace("Initializing computation state for task: {}", name);
+
+ executeComputation();
+
+ if (interrupted) {
+ logger.warn("Compute task {} interrupted at iteration {}/{}", name, iterationsComplete, iterations);
+ logger.debug("Cleanup after interruption for task: {}", name);
+ state = RunState.CANCELLED;
+ } else {
+ iterationsComplete = iterations;
+ state = RunState.SUCCESS;
+ logger.info("β Completed compute task: {} ({} iterations)", name, iterations);
+ logger.debug("Final state: SUCCESS, Total iterations: {}", iterations);
+ logger.trace("Task {} deallocating resources", name);
+ }
+ }
+ }
+
+ private void executeComputation() throws InterruptedException {
+ // Perform computation
+ logger.debug("Beginning main computation loop for {} iterations", iterations);
+ for (int i = 0; i < iterations && !interrupted; i++) {
+ iterationsComplete = i + 1;
+
+ logger.trace("Iteration {} starting for task {}", i, name);
+
+ // Simulate computation (busy work)
+ double result = 0;
+ for (int j = 0; j < 1000; j++) {
+ result += Math.sin(j * 0.01) * Math.cos(i * 0.01);
+ }
+
+ if (i % 10 == 0) {
+ clock.sleep(20); // Small delay between iterations
+ logger.trace("Checkpoint at iteration {}, result: {}", i, result);
+ }
+
+ if (i % 20 == 0 && i > 0) {
+ logger.debug("Compute progress: {}/{} iterations ({}%), result: {}",
+ iterationsComplete, iterations, (iterationsComplete * 100 / iterations), result);
+ }
+
+ if (i % 50 == 0 && i > 0) {
+ logger.info("Milestone: {} completed {}/{} iterations", name, iterationsComplete, iterations);
+ }
+
+ // Simulate various events at different log levels
+ int rand = random.nextInt(100);
+ if (rand > 98) {
+ logger.warn("Compute task {} encountered cache miss at iteration {}", name, i);
+ } else if (rand > 95) {
+ logger.debug("Minor performance hiccup at iteration {} (still within bounds)", i);
+ } else if (rand > 92) {
+ logger.trace("Memory allocation occurred at iteration {}", i);
+ } else if (rand > 90) {
+ logger.info("Checkpoint saved for task {} at iteration {}", name, i);
+ }
+
+ if (i % 100 == 0 && i > 0) {
+ double progress = (i * 100.0) / iterations;
+ if (progress < 30) {
+ logger.debug("Early stage: {}% complete", progress);
+ } else if (progress < 70) {
+ logger.debug("Mid stage: {}% complete", progress);
+ } else {
+ logger.debug("Final stage: {}% complete", progress);
+ }
+ }
+ }
+ }
+
+ /**
+ * Returns the current status of this task including progress fraction and run state.
+ * Progress is calculated as the ratio of completed iterations to total iterations.
+ *
+ * @return a status update containing progress, state, and this task as the source
+ */
+ @Override
+ public StatusUpdate getTaskStatus() {
+ double progress = iterations > 0 ? (double) iterationsComplete / iterations : 0.0;
+ return new StatusUpdate<>(progress, state, this);
+ }
+
+ @Override
+ public String toString() {
+ return name;
+ }
+ }
+}
diff --git a/internal-apis/src/test/java/io/github/jbellis/jvector/status/examples/ExampleDataProcessingTask.java b/internal-apis/src/test/java/io/github/jbellis/jvector/status/examples/ExampleDataProcessingTask.java
new file mode 100644
index 000000000..77b6a2df3
--- /dev/null
+++ b/internal-apis/src/test/java/io/github/jbellis/jvector/status/examples/ExampleDataProcessingTask.java
@@ -0,0 +1,172 @@
+/*
+ * Copyright DataStax, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package io.github.jbellis.jvector.status.examples;
+
+import io.github.jbellis.jvector.status.SimulatedClock;
+import io.github.jbellis.jvector.status.StatusTracker;
+import io.github.jbellis.jvector.status.StatusScope;
+import io.github.jbellis.jvector.status.eventing.RunState;
+import io.github.jbellis.jvector.status.eventing.StatusSource;
+import io.github.jbellis.jvector.status.eventing.StatusUpdate;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.time.Duration;
+import java.util.Random;
+
+/**
+ * Demo task simulating data processing operations (loading, parsing, transforming).
+ * Demonstrates I/O-heavy workloads with periodic status updates as a leaf task within a scope.
+ */
+class ExampleDataProcessingTask implements StatusSource, Runnable {
+ private static final Logger logger = LogManager.getLogger(ExampleDataProcessingTask.class);
+
+ private final String name;
+ private final int recordCount;
+ private final StatusScope parentScope;
+ private final SimulatedClock clock;
+ private final Random random = new Random();
+
+ private volatile int recordsProcessed = 0;
+ private volatile RunState state = RunState.PENDING;
+ private volatile boolean interrupted = false;
+
+ ExampleDataProcessingTask(String name, int recordCount, StatusScope parentScope, SimulatedClock clock) {
+ this.name = name;
+ this.recordCount = recordCount;
+ this.parentScope = parentScope;
+ this.clock = clock;
+ }
+
+ @Override
+ public void run() {
+ try (StatusTracker tracker = createTracker()) {
+ execute();
+ } catch (Exception e) {
+ logger.error("Data processing task {} failed: {}", name, e.getMessage());
+ }
+ }
+
+ private StatusTracker createTracker() {
+ return parentScope.trackTask(this, ExampleDataProcessingTask::getTaskStatus);
+ }
+
+ private void execute() throws InterruptedException {
+ state = RunState.RUNNING;
+ logger.info("Starting data processing: {} ({} records)", name, recordCount);
+ logger.debug("Initializing data pipeline for {}", name);
+ logger.trace("Allocating buffer space for {} records", recordCount);
+
+ long startTime = System.currentTimeMillis();
+ int batchSize = 50;
+ int currentBatch = 0;
+
+ for (int i = 0; i < recordCount && !interrupted; i++) {
+ recordsProcessed = i + 1;
+
+ if (i % batchSize == 0) {
+ currentBatch++;
+ logger.debug("Processing batch {}/{} (records {}-{})",
+ currentBatch, (recordCount + batchSize - 1) / batchSize,
+ i, Math.min(i + batchSize - 1, recordCount - 1));
+ logger.trace("Batch {} memory footprint: ~{}KB", currentBatch, random.nextInt(500) + 100);
+ }
+
+ // Simulate I/O delay
+ clock.sleep(50 + random.nextInt(50));
+
+ if (i % 10 == 0 && i > 0) {
+ logger.trace("Record {} parsed and validated", i);
+ }
+
+ if (i % 50 == 0 && i > 0) {
+ double progress = (recordsProcessed * 100.0) / recordCount;
+ logger.debug("Progress: {}/{} records ({:.1f}%)", recordsProcessed, recordCount, progress);
+ }
+
+ if (i % 100 == 0 && i > 0) {
+ logger.info("Processed {}/{} records in {}", recordsProcessed, recordCount, name);
+ long elapsed = System.currentTimeMillis() - startTime;
+ double rate = recordsProcessed / (elapsed / 1000.0);
+ logger.debug("Processing rate: {:.1f} records/sec", rate);
+ }
+
+ if (i % 250 == 0 && random.nextInt(10) > 7) {
+ logger.info("Batch checkpoint: {} records processed successfully", recordsProcessed);
+ logger.trace("Checkpoint saved to offset: {}", i);
+ }
+
+ // Simulate various data processing events
+ int rand = random.nextInt(1000);
+ if (rand > 995) {
+ logger.warn("Slow record detected in {} at index {} (processing took >200ms)", name, i);
+ } else if (rand > 990) {
+ logger.debug("Data validation passed for record {}", i);
+ } else if (rand > 985) {
+ logger.trace("Schema version mismatch handled for record {}", i);
+ } else if (rand > 980) {
+ logger.info("Transformation pipeline applied to record {}", i);
+ } else if (rand > 975) {
+ logger.debug("Record {} required additional normalization", i);
+ } else if (rand > 970) {
+ logger.trace("Cache hit for lookup key at record {}", i);
+ }
+
+ // Simulate batch commits
+ if (i > 0 && i % 200 == 0) {
+ logger.info("Committing batch of {} records to storage", 200);
+ logger.debug("Transaction ID: {}", random.nextInt(100000));
+ logger.trace("Fsync completed for batch ending at record {}", i);
+ }
+ }
+
+ if (interrupted) {
+ logger.warn("Data processing {} interrupted at record {}/{}", name, recordsProcessed, recordCount);
+ logger.debug("Performing rollback for incomplete batch");
+ logger.trace("Cleaning up partial state at record {}", recordsProcessed);
+ state = RunState.CANCELLED;
+ } else {
+ recordsProcessed = recordCount;
+ state = RunState.SUCCESS;
+ long elapsed = System.currentTimeMillis() - startTime;
+ double rate = recordCount / (elapsed / 1000.0);
+ logger.info("β Completed data processing: {} ({} records)", name, recordCount);
+ logger.debug("Final stats - Total: {} records, Duration: {}ms, Rate: {:.1f} rec/sec",
+ recordCount, elapsed, rate);
+ logger.trace("Pipeline resources released for task {}", name);
+ }
+ }
+
+ public void interrupt() {
+ interrupted = true;
+ }
+
+ public String getName() {
+ return name;
+ }
+
+ @Override
+ public StatusUpdate getTaskStatus() {
+ double progress = recordCount > 0 ? (double) recordsProcessed / recordCount : 0.0;
+ return new StatusUpdate<>(progress, state, this);
+ }
+
+ @Override
+ public String toString() {
+ return name;
+ }
+}
diff --git a/internal-apis/src/test/java/io/github/jbellis/jvector/status/examples/ExampleValidationTask.java b/internal-apis/src/test/java/io/github/jbellis/jvector/status/examples/ExampleValidationTask.java
new file mode 100644
index 000000000..856d0dc5c
--- /dev/null
+++ b/internal-apis/src/test/java/io/github/jbellis/jvector/status/examples/ExampleValidationTask.java
@@ -0,0 +1,214 @@
+/*
+ * Copyright DataStax, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package io.github.jbellis.jvector.status.examples;
+
+import io.github.jbellis.jvector.status.SimulatedClock;
+import io.github.jbellis.jvector.status.StatusTracker;
+import io.github.jbellis.jvector.status.StatusScope;
+import io.github.jbellis.jvector.status.eventing.RunState;
+import io.github.jbellis.jvector.status.eventing.StatusSource;
+import io.github.jbellis.jvector.status.eventing.StatusUpdate;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+
+/**
+ * Demo task simulating validation and verification operations.
+ * Demonstrates sequential validation stages with nested check operations.
+ */
+class ExampleValidationTask implements StatusSource, Runnable {
+ private static final Logger logger = LogManager.getLogger(ExampleValidationTask.class);
+
+ private final String name;
+ private final int checkCount;
+ private final StatusScope parentScope;
+ private final Random random = new Random();
+
+ private volatile int checksComplete = 0;
+ private volatile RunState state = RunState.PENDING;
+ private volatile boolean interrupted = false;
+ private final List childThreads = new ArrayList<>();
+ private StatusScope detailsScope; // Scope for nested validations
+ private final boolean allowNesting; // Only top-level tasks spawn nested validations
+ private final SimulatedClock clock;
+
+ ExampleValidationTask(String name, int checkCount, StatusScope parentScope, SimulatedClock clock) {
+ this(name, checkCount, parentScope, true, clock);
+ }
+
+ ExampleValidationTask(String name, int checkCount, StatusScope parentScope, boolean allowNesting, SimulatedClock clock) {
+ this.name = name;
+ this.checkCount = checkCount;
+ this.parentScope = parentScope;
+ this.allowNesting = allowNesting;
+ this.clock = clock;
+ }
+
+ @Override
+ public void run() {
+ try (StatusTracker tracker = createTracker()) {
+ execute(tracker);
+ } catch (Exception e) {
+ logger.error("Validation task {} failed: {}", name, e.getMessage());
+ }
+ }
+
+ private StatusTracker createTracker() {
+ return parentScope.trackTask(this, ExampleValidationTask::getTaskStatus);
+ }
+
+ private void execute(StatusTracker tracker) throws InterruptedException {
+ state = RunState.RUNNING;
+ logger.info("Starting validation: {} ({} checks)", name, checkCount);
+ logger.debug("Validation scope: {}, Expected duration: ~{}ms", name, checkCount * 150);
+ logger.trace("Loading validation rules for {}", name);
+
+ int warningCount = 0;
+ int errorCount = 0;
+ long startTime = System.currentTimeMillis();
+
+ for (int i = 0; i < checkCount && !interrupted; i++) {
+ checksComplete = i + 1;
+
+ logger.trace("Executing validation check #{} of {}", i + 1, checkCount);
+
+ // Simulate validation check
+ clock.sleep(100 + random.nextInt(100));
+
+ if (i % 3 == 0) {
+ logger.debug("Validation check {}/{} passed in {}", checksComplete, checkCount, name);
+ }
+
+ if (i % 5 == 0 && i > 0) {
+ double progress = (checksComplete * 100.0) / checkCount;
+ logger.debug("Validation progress: {:.1f}% ({}/{})", progress, checksComplete, checkCount);
+ }
+
+ if (i % 10 == 0 && i > 0) {
+ logger.info("Validation milestone: {} checks completed ({}% done)",
+ checksComplete, (checksComplete * 100) / checkCount);
+ }
+
+ // Spawn nested validation at 50% mark (only for top-level validation tasks)
+ if (i == checkCount / 2 && allowNesting && detailsScope == null) {
+ logger.info("Spawning detailed validation subtasks for {}", name);
+ logger.debug("Creating 3 specialized validation tasks");
+ detailsScope = parentScope.createChildScope(name + "-Details");
+ spawnDetailedValidation(tracker, "Schema", 10);
+ spawnDetailedValidation(tracker, "Integrity", 12);
+ spawnDetailedValidation(tracker, "Consistency", 8);
+ logger.trace("Nested validation tasks scheduled");
+ }
+
+ // Simulate various validation outcomes
+ int rand = random.nextInt(100);
+ if (rand > 95) {
+ warningCount++;
+ logger.warn("Validation warning in {} at check {}: minor inconsistency detected (total warnings: {})",
+ name, i, warningCount);
+ logger.trace("Warning details: threshold exceeded by 5%");
+ } else if (rand > 92) {
+ logger.debug("Check {} passed with constraints", i);
+ } else if (rand > 89) {
+ logger.trace("Cached validation result reused for check {}", i);
+ } else if (rand > 86) {
+ logger.info("Complex validation rule #{} evaluated successfully", i);
+ }
+
+ if (random.nextInt(100) > 98) {
+ errorCount++;
+ logger.error("Validation error in {} at check {}: retrying (error count: {})", name, i, errorCount);
+ logger.debug("Attempting retry with relaxed constraints");
+ clock.sleep(200); // Simulate retry delay
+ logger.debug("Retry succeeded for check {}", i);
+ }
+
+ if (i > 0 && i % 20 == 0) {
+ long elapsed = System.currentTimeMillis() - startTime;
+ double rate = checksComplete / (elapsed / 1000.0);
+ logger.debug("Validation rate: {:.1f} checks/sec, Warnings: {}, Errors: {}",
+ rate, warningCount, errorCount);
+ }
+ }
+
+ // Wait for nested validations and close the details scope
+ if (!childThreads.isEmpty()) {
+ logger.debug("Waiting for {} nested validation tasks to complete", childThreads.size());
+ for (Thread child : childThreads) {
+ logger.trace("Joining validation thread: {}", child.getName());
+ child.join(2000);
+ }
+ logger.info("All nested validations completed for {}", name);
+
+ // Close the details scope to clean up resources
+ if (detailsScope != null) {
+ detailsScope.close();
+ logger.debug("Details scope closed for {}", name);
+ }
+ }
+
+ if (interrupted) {
+ logger.warn("Validation {} interrupted at check {}/{}", name, checksComplete, checkCount);
+ logger.debug("Partial results: {} checks passed, {} warnings, {} errors",
+ checksComplete, warningCount, errorCount);
+ state = RunState.CANCELLED;
+ } else {
+ checksComplete = checkCount;
+ state = RunState.SUCCESS;
+ long elapsed = System.currentTimeMillis() - startTime;
+ logger.info("β Completed validation: {} ({} checks passed)", name, checkCount);
+ logger.debug("Validation summary - Duration: {}ms, Warnings: {}, Errors: {}, Success rate: {:.1f}%",
+ elapsed, warningCount, errorCount,
+ ((checkCount - errorCount) * 100.0) / checkCount);
+ logger.trace("Validation state persisted for {}", name);
+ }
+ }
+
+ private void spawnDetailedValidation(StatusTracker tracker, String checkType, int checks) {
+ // Use the shared details scope created earlier, disable nesting for child tasks
+ ExampleValidationTask childTask = new ExampleValidationTask(checkType, checks, detailsScope, false, clock);
+ Thread childThread = new Thread(childTask);
+ childThreads.add(childThread);
+ childThread.start();
+ }
+
+ public void interrupt() {
+ interrupted = true;
+ for (Thread child : childThreads) {
+ child.interrupt();
+ }
+ }
+
+ public String getName() {
+ return name;
+ }
+
+ @Override
+ public StatusUpdate getTaskStatus() {
+ double progress = checkCount > 0 ? (double) checksComplete / checkCount : 0.0;
+ return new StatusUpdate<>(progress, state, this);
+ }
+
+ @Override
+ public String toString() {
+ return name;
+ }
+}
diff --git a/internal-apis/src/test/java/io/github/jbellis/jvector/status/examples/ExampleWorkloadExecutor.java b/internal-apis/src/test/java/io/github/jbellis/jvector/status/examples/ExampleWorkloadExecutor.java
new file mode 100644
index 000000000..4c8d0680e
--- /dev/null
+++ b/internal-apis/src/test/java/io/github/jbellis/jvector/status/examples/ExampleWorkloadExecutor.java
@@ -0,0 +1,137 @@
+/*
+ * Copyright DataStax, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package io.github.jbellis.jvector.status.examples;
+
+import io.github.jbellis.jvector.status.SimulatedClock;
+import io.github.jbellis.jvector.status.StatusScope;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Executor that manages a diverse workload of demo tasks showing different execution patterns.
+ * Coordinates {@link ExampleDataProcessingTask}, {@link ExampleComputeTask}, and {@link ExampleValidationTask}
+ * instances within a {@link StatusScope} to demonstrate the organizational hierarchy.
+ */
+public class ExampleWorkloadExecutor {
+ private final List tasks = new ArrayList<>();
+ private final List threads = new ArrayList<>();
+ private volatile boolean workloadComplete = false;
+ private final SimulatedClock clock;
+
+ private ExampleWorkloadExecutor(SimulatedClock clock) {
+ this.clock = clock;
+ }
+
+ /**
+ * Launches a complete demo workload with diverse task types within the given scope.
+ * The scope provides organizational structure while tasks do the actual work.
+ * ComputeTasks create their own child scopes containing main task and worker subtasks.
+ * This method starts the workload and returns immediately.
+ */
+ public static ExampleWorkloadExecutor runDemoWorkload(StatusScope workloadScope, SimulatedClock clock) {
+ ExampleWorkloadExecutor executor = new ExampleWorkloadExecutor(clock);
+
+ // Create diverse workload showing different patterns
+ // Simple tasks are direct children, compute tasks create their own scopes
+ executor.addTask(new ExampleDataProcessingTask("DataLoad", 500, workloadScope, clock));
+ executor.addTask(new ExampleComputeTask("VectorIndexing", 200, 3, workloadScope, clock));
+ executor.addTask(new ExampleValidationTask("SchemaValidation", 40, workloadScope, clock));
+ executor.addTask(new ExampleDataProcessingTask("DataTransform", 300, workloadScope, clock));
+ executor.addTask(new ExampleComputeTask("Clustering", 150, 2, workloadScope, clock));
+ executor.addTask(new ExampleValidationTask("IntegrityCheck", 30, workloadScope, clock));
+
+ executor.startAll();
+ return executor;
+ }
+
+ private void addTask(Runnable task) {
+ tasks.add(task);
+ }
+
+ private void startAll() {
+ for (Runnable task : tasks) {
+ Thread thread = new Thread(task);
+ threads.add(thread);
+ thread.start();
+
+ // Stagger start times
+ try {
+ clock.sleep(600);
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ break;
+ }
+ }
+ }
+
+ /**
+ * Wait for all tasks in the workload to complete.
+ */
+ public boolean awaitCompletion() {
+ try {
+ for (Thread thread : threads) {
+ thread.join();
+ }
+ workloadComplete = true;
+ return true;
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ return false;
+ }
+ }
+
+ /**
+ * Wait for all tasks with a timeout.
+ */
+ public boolean awaitCompletion(long timeoutMs) {
+ long deadline = System.currentTimeMillis() + timeoutMs;
+ try {
+ for (Thread thread : threads) {
+ long remaining = deadline - System.currentTimeMillis();
+ if (remaining <= 0) {
+ return false;
+ }
+ thread.join(remaining);
+ if (thread.isAlive()) {
+ return false;
+ }
+ }
+ workloadComplete = true;
+ return true;
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ return false;
+ }
+ }
+
+ /**
+ * Check if the workload has completed.
+ */
+ public boolean isComplete() {
+ return workloadComplete;
+ }
+
+ /**
+ * Interrupt all running tasks.
+ */
+ public void interruptAll() {
+ for (Thread thread : threads) {
+ thread.interrupt();
+ }
+ }
+}
diff --git a/internal-apis/src/test/java/io/github/jbellis/jvector/status/examples/ScopeUsageExample.java b/internal-apis/src/test/java/io/github/jbellis/jvector/status/examples/ScopeUsageExample.java
new file mode 100644
index 000000000..6b0c411ad
--- /dev/null
+++ b/internal-apis/src/test/java/io/github/jbellis/jvector/status/examples/ScopeUsageExample.java
@@ -0,0 +1,106 @@
+/*
+ * Copyright DataStax, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package io.github.jbellis.jvector.status.examples;
+
+import io.github.jbellis.jvector.status.SimulatedClock;
+import io.github.jbellis.jvector.status.StatusContext;
+import io.github.jbellis.jvector.status.StatusTracker;
+import io.github.jbellis.jvector.status.StatusScope;
+import io.github.jbellis.jvector.status.sinks.ConsoleLoggerSink;
+
+import java.time.Duration;
+
+/**
+ * Example showing the new scope-based organization vs traditional tracker hierarchy.
+ */
+public class ScopeUsageExample {
+
+ public static void main(String[] args) throws InterruptedException {
+ demonstrateScopes();
+ }
+
+ /**
+ * NEW APPROACH: Use scopes for organization, tasks for work.
+ * Scopes have no progress/state - they're purely organizational.
+ * Tasks within scopes CANNOT have children (enforced).
+ */
+ private static void demonstrateScopes() throws InterruptedException {
+ SimulatedClock clock = new SimulatedClock();
+ try (StatusContext context = new StatusContext("pipeline",
+ Duration.ofMillis(100),
+ java.util.List.of(new ConsoleLoggerSink()))) {
+
+ // Create organizational scopes
+ try (StatusScope ingestionScope = context.createScope("Ingestion");
+ StatusScope processingScope = context.createScope("Processing")) {
+
+ // Track actual work as leaf tasks
+ StatusTracker loadTracker =
+ ingestionScope.trackTask(new ExampleDataProcessingTask("LoadCSV", 100, ingestionScope, clock));
+
+ StatusTracker validateTracker =
+ ingestionScope.trackTask(new ExampleValidationTask("ValidateSchema", 20, ingestionScope, clock));
+
+ // ComputeTask creates its own scope hierarchy internally
+ Thread transformThread = new Thread(new ExampleComputeTask("Transform", 80, 2, processingScope, clock));
+ transformThread.start();
+
+ // Tasks execute...
+ Thread.sleep(500);
+
+ // ComputeTask creates its own scope hierarchy internally
+ Thread indexThread = new Thread(new ExampleComputeTask("BuildIndex", 120, 3, processingScope, clock));
+ indexThread.start();
+
+ System.out.println("\nScope hierarchy:");
+ System.out.println(" Ingestion (scope)");
+ System.out.println(" ββ LoadCSV (task)");
+ System.out.println(" ββ ValidateSchema (task)");
+ System.out.println(" Processing (scope)");
+ System.out.println(" ββ Transform (scope)");
+ System.out.println(" β ββ Transform (main task)");
+ System.out.println(" β ββ Workers (scope)");
+ System.out.println(" β ββ Worker1 (task)");
+ System.out.println(" β ββ Worker2 (task)");
+ System.out.println(" ββ BuildIndex (scope)");
+ System.out.println(" ββ BuildIndex (main task)");
+ System.out.println(" ββ Workers (scope)");
+ System.out.println(" ββ Worker1 (task)");
+ System.out.println(" ββ Worker2 (task)");
+ System.out.println(" ββ Worker3 (task)");
+
+ // Check completion
+ System.out.println("\nIs ingestion scope complete? " + ingestionScope.isComplete());
+ System.out.println("Is processing scope complete? " + processingScope.isComplete());
+
+ // Close trackers
+ loadTracker.close();
+ validateTracker.close();
+
+ // Wait for compute tasks to complete
+ transformThread.join(5000);
+ indexThread.join(5000);
+
+ Thread.sleep(100);
+
+ System.out.println("\nAfter closing all tasks:");
+ System.out.println("Is ingestion scope complete? " + ingestionScope.isComplete());
+ System.out.println("Is processing scope complete? " + processingScope.isComplete());
+ }
+ }
+ }
+}
diff --git a/internal-apis/src/test/java/io/github/jbellis/jvector/status/sinks/ConsolePanelSinkTest.java b/internal-apis/src/test/java/io/github/jbellis/jvector/status/sinks/ConsolePanelSinkTest.java
new file mode 100644
index 000000000..f685fba70
--- /dev/null
+++ b/internal-apis/src/test/java/io/github/jbellis/jvector/status/sinks/ConsolePanelSinkTest.java
@@ -0,0 +1,83 @@
+/*
+ * Copyright DataStax, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package io.github.jbellis.jvector.status.sinks;
+
+import io.github.jbellis.jvector.status.StatusContext;
+import io.github.jbellis.jvector.status.StatusTracker;
+import io.github.jbellis.jvector.status.eventing.RunState;
+import io.github.jbellis.jvector.status.eventing.StatusSource;
+import io.github.jbellis.jvector.status.eventing.StatusUpdate;
+import org.junit.jupiter.api.Test;
+
+import java.time.Duration;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.jupiter.api.Assertions.*;
+
+class ConsolePanelSinkTest {
+
+ private static final class DemoTask implements StatusSource {
+ private final String name;
+ private volatile double progress;
+ private volatile RunState state = RunState.PENDING;
+
+ DemoTask(String name) {
+ this.name = name;
+ }
+
+ @Override
+ public StatusUpdate getTaskStatus() {
+ return new StatusUpdate<>(progress, state, this);
+ }
+
+ void advance(double delta) {
+ progress = Math.min(1.0, progress + delta);
+ state = progress >= 1.0 ? RunState.SUCCESS : RunState.RUNNING;
+ }
+
+ @Override
+ public String toString() {
+ return name;
+ }
+ }
+
+ @Test
+ void panelReceivesLifecycleEvents() throws InterruptedException {
+ ConsolePanelSink sink = ConsolePanelSink.builder()
+ .withRefreshRateMs(50)
+ .withCompletedTaskRetention(1, TimeUnit.SECONDS)
+ .withCaptureSystemStreams(false)
+ .withColorOutput(false)
+ .build();
+
+ try (StatusContext context = new StatusContext("panel", Duration.ofMillis(20), List.of(sink))) {
+ DemoTask task = new DemoTask("panel-demo");
+ try (var scope = context.createScope("test-scope");
+ StatusTracker