Move critical path computation out of runtime

... so that we can use it for posting a machine-readable form of the critical path as build metric in the build event stream.

Closes #16611.

PiperOrigin-RevId: 522304794
Change-Id: Ieb7e1dbc1a4a330341a5fd4f32eae8bc5edc5390
diff --git a/src/main/java/com/google/devtools/build/lib/runtime/AggregatedCriticalPath.java b/src/main/java/com/google/devtools/build/lib/runtime/AggregatedCriticalPath.java
deleted file mode 100644
index bd38d02..0000000
--- a/src/main/java/com/google/devtools/build/lib/runtime/AggregatedCriticalPath.java
+++ /dev/null
@@ -1,106 +0,0 @@
-// Copyright 2014 The Bazel Authors. All rights reserved.
-//
-// 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 com.google.devtools.build.lib.runtime;
-
-import com.google.common.base.Joiner;
-import com.google.common.collect.ImmutableList;
-import com.google.devtools.build.lib.actions.AggregatedSpawnMetrics;
-import com.google.devtools.build.lib.actions.SpawnMetrics;
-import java.time.Duration;
-
-/**
- * Aggregates all the critical path components in one object. This allows us to easily access the
- * components data and have a proper toString().
- */
-@SuppressWarnings("GoodTime") // Use ints instead of Durations to improve build time (cl/505728570)
-public class AggregatedCriticalPath {
-  public static final AggregatedCriticalPath EMPTY =
-      new AggregatedCriticalPath(0, AggregatedSpawnMetrics.EMPTY, ImmutableList.of());
-
-  private final int totalTimeInMs;
-  private final AggregatedSpawnMetrics aggregatedSpawnMetrics;
-  private final ImmutableList<CriticalPathComponent> criticalPathComponents;
-
-  public AggregatedCriticalPath(
-      int totalTimeInMs,
-      AggregatedSpawnMetrics aggregatedSpawnMetrics,
-      ImmutableList<CriticalPathComponent> criticalPathComponents) {
-    this.totalTimeInMs = totalTimeInMs;
-    this.aggregatedSpawnMetrics = aggregatedSpawnMetrics;
-    this.criticalPathComponents = criticalPathComponents;
-  }
-
-  /** Total wall time spent running the critical path actions. */
-  public int totalTimeInMs() {
-    return totalTimeInMs;
-  }
-
-  public AggregatedSpawnMetrics getSpawnMetrics() {
-    return aggregatedSpawnMetrics;
-  }
-
-  /** Returns a list of all the component stats for the critical path. */
-  public ImmutableList<CriticalPathComponent> components() {
-    return criticalPathComponents;
-  }
-
-  public String getNewStringSummary() {
-    int executionWallTimeInMs =
-        aggregatedSpawnMetrics.getTotalDuration(SpawnMetrics::executionWallTimeInMs);
-    int overheadTimeInMs =
-        aggregatedSpawnMetrics.getTotalDuration(SpawnMetrics::totalTimeInMs)
-            - executionWallTimeInMs;
-    return String.format(
-        "Execution critical path %.2fs (setup %.2fs, action wall time %.2fs)",
-        totalTimeInMs / 1000.0, overheadTimeInMs / 1000.0, executionWallTimeInMs / 1000.0);
-  }
-
-  @Override
-  public String toString() {
-    return toString(false, true);
-  }
-
-  private String toString(boolean summary, boolean remote) {
-    StringBuilder sb = new StringBuilder("Critical Path: ");
-    sb.append(String.format("%.2f", totalTimeInMs / 1000.0));
-    sb.append("s");
-    if (remote) {
-      sb.append(", ");
-      sb.append(getSpawnMetrics().toString(Duration.ofMillis(totalTimeInMs), summary));
-    }
-    if (summary || criticalPathComponents.isEmpty()) {
-      return sb.toString();
-    }
-    sb.append("\n  ");
-    Joiner.on("\n  ").appendTo(sb, criticalPathComponents);
-    return sb.toString();
-  }
-
-  /**
-   * Returns a summary version of the critical path stats that omits stats that are not useful to
-   * the user.
-   */
-  public String toStringSummary() {
-    return toString(true, true);
-  }
-
-  /**
-   * Same as toStringSummary but also omits remote stats. This is to be used in Bazel because
-   * currently the Remote stats are not calculated correctly.
-   */
-  public String toStringSummaryNoRemote() {
-    return toString(true, false);
-  }
-}
diff --git a/src/main/java/com/google/devtools/build/lib/runtime/BuildSummaryStatsModule.java b/src/main/java/com/google/devtools/build/lib/runtime/BuildSummaryStatsModule.java
index f051d65..9a1fc76 100644
--- a/src/main/java/com/google/devtools/build/lib/runtime/BuildSummaryStatsModule.java
+++ b/src/main/java/com/google/devtools/build/lib/runtime/BuildSummaryStatsModule.java
@@ -31,6 +31,9 @@
 import com.google.devtools.build.lib.events.Reporter;
 import com.google.devtools.build.lib.exec.ExecutionOptions;
 import com.google.devtools.build.lib.exec.ExecutorBuilder;
+import com.google.devtools.build.lib.metrics.criticalpath.AggregatedCriticalPath;
+import com.google.devtools.build.lib.metrics.criticalpath.CriticalPathComponent;
+import com.google.devtools.build.lib.metrics.criticalpath.CriticalPathComputer;
 import com.google.devtools.build.lib.profiler.Profiler;
 import com.google.devtools.build.lib.profiler.ProfilerTask;
 import com.google.devtools.build.lib.profiler.SilentCloseable;
diff --git a/src/main/java/com/google/devtools/build/lib/runtime/CriticalPathComponent.java b/src/main/java/com/google/devtools/build/lib/runtime/CriticalPathComponent.java
deleted file mode 100644
index c69619f..0000000
--- a/src/main/java/com/google/devtools/build/lib/runtime/CriticalPathComponent.java
+++ /dev/null
@@ -1,335 +0,0 @@
-// Copyright 2014 The Bazel Authors. All rights reserved.
-//
-// 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 com.google.devtools.build.lib.runtime;
-
-import com.google.common.base.Preconditions;
-import com.google.devtools.build.lib.actions.Action;
-import com.google.devtools.build.lib.actions.ActionOwner;
-import com.google.devtools.build.lib.actions.AggregatedSpawnMetrics;
-import com.google.devtools.build.lib.actions.Artifact;
-import com.google.devtools.build.lib.actions.SpawnMetrics;
-import com.google.devtools.build.lib.actions.SpawnResult;
-import com.google.devtools.build.lib.clock.BlazeClock.NanosToMillisSinceEpochConverter;
-import com.google.devtools.build.lib.cmdline.Label;
-import com.google.devtools.build.lib.concurrent.ThreadSafety.ThreadCompatible;
-import java.time.Duration;
-import javax.annotation.Nullable;
-
-/**
- * A component of the graph over which the critical path is computed. This may be identical to the
- * action graph, but does not have to be - it may also take into account individual spawns run as
- * part of an action.
- */
-@ThreadCompatible
-public class CriticalPathComponent {
-  /** Empty metrics used to simplify handling of {@link #phaseMaxMetrics}. */
-  private static final SpawnMetrics EMPTY_PLACEHOLDER_METRICS =
-      SpawnMetrics.Builder.forOtherExec().build();
-
-  // These two fields are values of BlazeClock.nanoTime() at the relevant points in time.
-  private long startNanos;
-  private long finishNanos = 0;
-  private volatile boolean isRunning = false;
-
-  /** The longest aggregate runtime of this component and its critical path. */
-  private long aggregatedElapsedTime = 0;
-
-  private final Action action;
-  private final Artifact primaryOutput;
-
-  /** Spawn metrics for this action. */
-  private SpawnMetrics phaseMaxMetrics = EMPTY_PLACEHOLDER_METRICS;
-
-  private AggregatedSpawnMetrics totalSpawnMetrics = AggregatedSpawnMetrics.EMPTY;
-  private int longestRunningTotalDurationInMs = 0;
-  private boolean phaseChange;
-
-  /** Name of the runner used for the spawn. */
-  @Nullable private String longestPhaseSpawnRunnerName;
-  /** Details about the runner used for the spawn. */
-  @Nullable private String longestPhaseSpawnRunnerSubtype;
-  /** An unique identifier of the component for one build execution */
-  private final int id;
-
-  /** Child with the maximum critical path. */
-  @Nullable private CriticalPathComponent child;
-
-  /** Indication that there is at least one remote spawn metrics received. */
-  private boolean remote = false;
-
-  public CriticalPathComponent(int id, Action action, long startNanos) {
-    this.id = id;
-    this.action = Preconditions.checkNotNull(action);
-    this.primaryOutput = action.getPrimaryOutput();
-    this.startNanos = startNanos;
-  }
-
-  /**
-   * Record the elapsed time in case the new duration is greater. This method could be called
-   * multiple times in the following cases:
-   *
-   * <ol>
-   *   <li>Shared actions run concurrently, and the one that really gets executed takes more time to
-   *       send the finish event and the one that was a cache hit manages to send the event before.
-   *   <li>An action gets rewound, and is later reattempted.
-   * </ol>
-   *
-   * <p>In both these cases we overwrite the components' times if the later call specifies a greater
-   * duration.
-   *
-   * <p>In the former case the logic is known to be incorrect, as other actions that depend on this
-   * action will not necessarily use the correct getElapsedTimeNanos(). But we do not want to block
-   * action execution because of this. So in certain conditions we might see another path as the
-   * critical path.
-   *
-   * <p>In addition, in the case of sequential spawns, Aggregate the last phase's duration values
-   * with the total spawn metrics. To make sure not to add the last phase's duration multiple times,
-   * only add if there is duration and reset the phase metrics once it has been aggregated.
-   */
-  public synchronized void finishActionExecution(
-      long startNanos, long finishNanos, String finalizeReason) {
-    if (isRunning || finishNanos - startNanos > getElapsedTimeNanos()) {
-      this.startNanos = startNanos;
-      this.finishNanos = finishNanos;
-      // In case aggregatedElapsedTime was never set (such as a leaf node with no depedencies) with
-      // #addDepInfo, we want to set it here in which case the elapsed time is just the run time of
-      // this component.
-      aggregatedElapsedTime = Math.max(aggregatedElapsedTime, this.finishNanos - this.startNanos);
-      isRunning = false;
-      if (longestPhaseSpawnRunnerName == null && !finalizeReason.isEmpty()) {
-        // This is probably not the best way to do it in face of getting called multiple times.
-        longestPhaseSpawnRunnerName = finalizeReason;
-        longestPhaseSpawnRunnerSubtype = "";
-        longestRunningTotalDurationInMs =
-            (int) Duration.ofNanos(this.finishNanos - this.startNanos).toMillis();
-      }
-    }
-
-    // If the phaseMaxMetrics has Duration, then we want to aggregate it to the total.
-    if (!this.phaseMaxMetrics.isEmpty()) {
-      this.totalSpawnMetrics = this.totalSpawnMetrics.sumDurationsMaxOther(phaseMaxMetrics);
-      this.phaseMaxMetrics = EMPTY_PLACEHOLDER_METRICS;
-    }
-  }
-
-  @SuppressWarnings("ReferenceEquality")
-  boolean isPrimaryOutput(Artifact possiblePrimaryOutput) {
-    // We know that the keys in the CriticalPathComputer are exactly the values returned from
-    // action.getPrimaryOutput(), so pointer equality is safe here.
-    return possiblePrimaryOutput == primaryOutput;
-  }
-
-  /** The action for which we are storing the stat. */
-  public final Action getAction() {
-    return action;
-  }
-
-  /**
-   * This is called by {@link CriticalPathComputer#actionStarted} to start running the action. The
-   * three scenarios where this would occur is:
-   *
-   * <ol>
-   *   <li>A new CriticalPathComponent is created and should start running.
-   *   <li>A CriticalPathComponent has been created with discover inputs and beginning to execute.
-   *   <li>An action was rewound and starts again.
-   * </ol>
-   */
-  void startRunning() {
-    isRunning = true;
-  }
-
-  public boolean isRunning() {
-    return isRunning;
-  }
-
-  public String prettyPrintAction() {
-    return action.prettyPrint();
-  }
-
-  @Nullable
-  public Label getOwner() {
-    ActionOwner owner = action.getOwner();
-    if (owner != null && owner.getLabel() != null) {
-      return owner.getLabel();
-    }
-    return null;
-  }
-
-  public String getMnemonic() {
-    return action.getMnemonic();
-  }
-
-  /** An unique identifier of the component for one build execution */
-  public int getId() {
-    return id;
-  }
-
-  /**
-   * An action can run multiple spawns. Those calls can be sequential or parallel. If action is a
-   * sequence of calls we aggregate the SpawnMetrics of all the SpawnResults. If there are multiples
-   * of the same action run in parallel, we keep the maximum runtime SpawnMetrics. We will also set
-   * the longestPhaseSpawnRunnerName to the longest running spawn runner name across all phases if
-   * it exists.
-   */
-  void addSpawnResult(
-      SpawnMetrics metrics, @Nullable String runnerName, String runnerSubtype, boolean wasRemote) {
-    // Mark this component as having remote components if _any_ spawn result contributing
-    // to it contains meaningful remote metrics. Subsequent non-remote spawns in an action
-    // must not reset this flag.
-    if (wasRemote) {
-      this.remote = true;
-    }
-    if (this.phaseChange) {
-      if (!this.phaseMaxMetrics.isEmpty()) {
-        this.totalSpawnMetrics = this.totalSpawnMetrics.sumDurationsMaxOther(phaseMaxMetrics);
-      }
-      this.phaseMaxMetrics = metrics;
-      this.phaseChange = false;
-    } else if (metrics.totalTimeInMs() > phaseMaxMetrics.totalTimeInMs()) {
-      this.phaseMaxMetrics = metrics;
-    }
-
-    if (runnerName != null && metrics.totalTimeInMs() > this.longestRunningTotalDurationInMs) {
-      this.longestPhaseSpawnRunnerName = runnerName;
-      this.longestPhaseSpawnRunnerSubtype = runnerSubtype;
-      this.longestRunningTotalDurationInMs = metrics.totalTimeInMs();
-    }
-  }
-
-  /** Set the phaseChange flag as true so we will aggregate incoming spawnMetrics. */
-  void changePhase() {
-    this.phaseChange = true;
-  }
-
-  /**
-   * Returns total spawn metrics of the maximum (longest running) spawn metrics of all phases for
-   * the execution of the action.
-   */
-  public AggregatedSpawnMetrics getSpawnMetrics() {
-    return totalSpawnMetrics;
-  }
-
-  /**
-   * Returns name of the maximum runner used for the finished spawn which took most time (see {@link
-   * #addSpawnResult(SpawnResult)}), null if no spawns have finished for this action (either there
-   * are no spawns or we asked before any have finished).
-   */
-  @Nullable
-  public String getLongestPhaseSpawnRunnerName() {
-    return longestPhaseSpawnRunnerName;
-  }
-
-  /** Like getLongestPhaseSpawnRunnerName(), but returns the runner details. */
-  @Nullable
-  public String getLongestPhaseSpawnRunnerSubtype() {
-    return longestPhaseSpawnRunnerSubtype;
-  }
-
-  /**
-   * Updates the child component if the union of the new dependency component runtime and the
-   * current component runtime is greater than the union of the current child runtime and current
-   * component runtime. The caller should ensure the dependency component is not running.
-   */
-  synchronized void addDepInfo(CriticalPathComponent dep, long componentFinishNanos) {
-    long currentElapsedTime = componentFinishNanos - startNanos;
-    long aggregatedElapsedTime = dep.aggregatedElapsedTime + currentElapsedTime;
-    // This corrects the overlapping run time.
-    if (dep.finishNanos > startNanos) {
-      aggregatedElapsedTime -= dep.finishNanos - startNanos;
-    }
-    if (child == null || aggregatedElapsedTime > this.aggregatedElapsedTime) {
-      this.aggregatedElapsedTime = aggregatedElapsedTime;
-      child = dep;
-    }
-  }
-
-  public long getStartTimeNanos() {
-    return startNanos;
-  }
-
-  public long getStartTimeMillisSinceEpoch(NanosToMillisSinceEpochConverter converter) {
-    return converter.toEpochMillis(startNanos);
-  }
-
-  public Duration getElapsedTime() {
-    return Duration.ofNanos(getElapsedTimeNanos());
-  }
-
-  long getElapsedTimeNanos() {
-    if (isRunning) {
-      // It can happen that we're being asked to compute a critical path even though the build was
-      // interrupted. In that case, we may not have gotten an action completion event. We don't have
-      // access to the clock from here, so we have to return 0.
-      // Note that the critical path never includes interrupted actions, so getAggregatedElapsedTime
-      // does not get called in this state.
-      // If we want the critical path to contain partially executed actions in a case of interrupt,
-      // then we need to tell the critical path computer that the build was interrupt, and let it
-      // artificially mark all such actions as done.
-      return 0;
-    }
-    return getElapsedTimeNanosNoCheck();
-  }
-
-  /** To be used only in debugging: skips state invariance checks to avoid crash-looping. */
-  private Duration getElapsedTimeNoCheck() {
-    return Duration.ofNanos(getElapsedTimeNanosNoCheck());
-  }
-
-  private long getElapsedTimeNanosNoCheck() {
-    // The delta value may be negative, see note in {@link Clock#nanoTime}.
-    return Math.max(0, finishNanos - startNanos);
-  }
-
-  /**
-   * Returns the current critical path for the action.
-   *
-   * <p>Critical path is defined as : action_execution_time + max(child_critical_path).
-   */
-  Duration getAggregatedElapsedTime() {
-    return Duration.ofNanos(aggregatedElapsedTime);
-  }
-
-  /**
-   * Get the child critical path component.
-   *
-   * <p>The component dependency with the maximum total critical path time.
-   */
-  @Nullable
-  public CriticalPathComponent getChild() {
-    return child;
-  }
-
-  /** Returns a string representation of the action. Only for use in crash messages and the like. */
-  private String getActionString() {
-    return action.prettyPrint();
-  }
-
-  /** Returns a user readable representation of the critical path stats with all the details. */
-  @Override
-  public String toString() {
-    StringBuilder sb = new StringBuilder();
-    String currentTime = "still running";
-    if (!isRunning) {
-      currentTime = String.format("%.2f", getElapsedTimeNoCheck().toMillis() / 1000.0) + "s";
-    }
-    sb.append(currentTime);
-    if (remote) {
-      sb.append(", ");
-      sb.append(getSpawnMetrics().toString(getElapsedTimeNoCheck(), /* summary= */ false));
-    }
-    sb.append(" ");
-    sb.append(getActionString());
-    return sb.toString();
-  }
-}
diff --git a/src/main/java/com/google/devtools/build/lib/runtime/CriticalPathComputer.java b/src/main/java/com/google/devtools/build/lib/runtime/CriticalPathComputer.java
deleted file mode 100644
index 6479de7..0000000
--- a/src/main/java/com/google/devtools/build/lib/runtime/CriticalPathComputer.java
+++ /dev/null
@@ -1,379 +0,0 @@
-// Copyright 2014 The Bazel Authors. All rights reserved.
-//
-// 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 com.google.devtools.build.lib.runtime;
-
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Comparators;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Maps;
-import com.google.common.eventbus.AllowConcurrentEvents;
-import com.google.common.eventbus.Subscribe;
-import com.google.common.flogger.GoogleLogger;
-import com.google.common.flogger.StackSize;
-import com.google.devtools.build.lib.actions.Action;
-import com.google.devtools.build.lib.actions.ActionAnalysisMetadata;
-import com.google.devtools.build.lib.actions.ActionCompletionEvent;
-import com.google.devtools.build.lib.actions.ActionKeyContext;
-import com.google.devtools.build.lib.actions.ActionMiddlemanEvent;
-import com.google.devtools.build.lib.actions.ActionStartedEvent;
-import com.google.devtools.build.lib.actions.Actions;
-import com.google.devtools.build.lib.actions.AggregatedSpawnMetrics;
-import com.google.devtools.build.lib.actions.Artifact;
-import com.google.devtools.build.lib.actions.CachedActionEvent;
-import com.google.devtools.build.lib.actions.DiscoveredInputsEvent;
-import com.google.devtools.build.lib.actions.SpawnExecutedEvent;
-import com.google.devtools.build.lib.actions.SpawnMetrics;
-import com.google.devtools.build.lib.actions.SpawnResult;
-import com.google.devtools.build.lib.clock.Clock;
-import com.google.devtools.build.lib.skyframe.rewinding.ActionRewoundEvent;
-import java.time.Duration;
-import java.util.Comparator;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.function.BinaryOperator;
-import java.util.stream.Stream;
-import javax.annotation.concurrent.ThreadSafe;
-
-/**
- * Computes the critical path in the action graph based on events published to the event bus.
- *
- * <p>After instantiation, this object needs to be registered on the event bus to work.
- */
-@ThreadSafe
-public class CriticalPathComputer {
-  private static final GoogleLogger logger = GoogleLogger.forEnclosingClass();
-
-  /** Number of top actions to record. */
-  static final int SLOWEST_COMPONENTS_SIZE = 30;
-
-  private static final int LARGEST_MEMORY_COMPONENTS_SIZE = 20;
-  private static final int LARGEST_INPUT_SIZE_COMPONENTS_SIZE = 20;
-  private static final int LARGEST_INPUT_COUNT_COMPONENTS_SIZE = 20;
-
-  /** Selects and returns the longer of two components (the first may be {@code null}). */
-  private static final BinaryOperator<CriticalPathComponent> SELECT_LONGER_COMPONENT =
-      (a, b) ->
-          a == null || a.getAggregatedElapsedTime().compareTo(b.getAggregatedElapsedTime()) < 0
-              ? b
-              : a;
-
-  private final AtomicInteger idGenerator = new AtomicInteger();
-  // outputArtifactToComponent is accessed from multiple event handlers.
-  private final ConcurrentMap<Artifact, CriticalPathComponent> outputArtifactToComponent =
-      Maps.newConcurrentMap();
-  private final ActionKeyContext actionKeyContext;
-
-  /** Maximum critical path found. */
-  private final AtomicReference<CriticalPathComponent> maxCriticalPath = new AtomicReference<>();
-
-  private final Clock clock;
-
-  public CriticalPathComputer(ActionKeyContext actionKeyContext, Clock clock) {
-    this.actionKeyContext = actionKeyContext;
-    this.clock = clock;
-  }
-
-  /**
-   * Creates a critical path component for an action.
-   *
-   * @param action the action for the critical path component
-   * @param relativeStartNanos time when the action started to run in nanos. Only meant to be used
-   *     for computing time differences.
-   */
-  private CriticalPathComponent createComponent(Action action, long relativeStartNanos) {
-    return new CriticalPathComponent(idGenerator.getAndIncrement(), action, relativeStartNanos);
-  }
-
-  /**
-   * Return the critical path stats for the current command execution.
-   *
-   * <p>This method allows us to calculate lazily the aggregate statistics of the critical path,
-   * avoiding the memory and cpu penalty for doing it for all the actions executed.
-   */
-  public AggregatedCriticalPath aggregate() {
-    CriticalPathComponent criticalPath = getMaxCriticalPath();
-    if (criticalPath == null) {
-      return AggregatedCriticalPath.EMPTY;
-    }
-
-    ImmutableList.Builder<CriticalPathComponent> components = ImmutableList.builder();
-    AggregatedSpawnMetrics.Builder metricsBuilder = new AggregatedSpawnMetrics.Builder();
-    CriticalPathComponent child = criticalPath;
-
-    while (child != null) {
-      AggregatedSpawnMetrics childSpawnMetrics = child.getSpawnMetrics();
-      if (childSpawnMetrics != null) {
-        metricsBuilder.addDurations(childSpawnMetrics);
-        metricsBuilder.addNonDurations(childSpawnMetrics);
-      }
-      components.add(child);
-      child = child.getChild();
-    }
-
-    return new AggregatedCriticalPath(
-        (int) criticalPath.getAggregatedElapsedTime().toMillis(),
-        metricsBuilder.build(),
-        components.build());
-  }
-
-  public Map<Artifact, CriticalPathComponent> getCriticalPathComponentsMap() {
-    return outputArtifactToComponent;
-  }
-
-  /** Changes the phase of the action */
-  @Subscribe
-  @AllowConcurrentEvents
-  public void nextCriticalPathPhase(SpawnExecutedEvent.ChangePhase phase) {
-    CriticalPathComponent stats =
-        outputArtifactToComponent.get(phase.getAction().getPrimaryOutput());
-    if (stats != null) {
-      stats.changePhase();
-    }
-  }
-
-  /** Adds spawn metrics to the action stats. */
-  @Subscribe
-  @AllowConcurrentEvents
-  public void spawnExecuted(SpawnExecutedEvent event) {
-    ActionAnalysisMetadata action = event.getActionMetadata();
-    Artifact primaryOutput = action.getPrimaryOutput();
-    if (primaryOutput == null) {
-      // Despite the documentation to the contrary, the SpawnIncludeScanner creates an
-      // ActionExecutionMetadata instance that returns a null primary output. That said, this
-      // class is incorrect wrt. multiple Spawns in a single action. See b/111583707.
-      return;
-    }
-    CriticalPathComponent stats =
-        Preconditions.checkNotNull(outputArtifactToComponent.get(primaryOutput));
-
-    SpawnResult spawnResult = event.getSpawnResult();
-    stats.addSpawnResult(
-        spawnResult.getMetrics(),
-        spawnResult.getRunnerName(),
-        spawnResult.getRunnerSubtype(),
-        spawnResult.wasRemote());
-  }
-
-  /** Returns the list of components using the most memory. */
-  public List<CriticalPathComponent> getLargestMemoryComponents() {
-    return uniqueActions()
-        .collect(
-            Comparators.greatest(
-                LARGEST_MEMORY_COMPONENTS_SIZE,
-                Comparator.comparingLong(
-                    (c) ->
-                        c.getSpawnMetrics().getMaxNonDuration(0, SpawnMetrics::memoryEstimate))));
-  }
-
-  /** Returns the list of components with the largest input sizes. */
-  public List<CriticalPathComponent> getLargestInputSizeComponents() {
-    return uniqueActions()
-        .collect(
-            Comparators.greatest(
-                LARGEST_INPUT_SIZE_COMPONENTS_SIZE,
-                Comparator.comparingLong(
-                    (c) -> c.getSpawnMetrics().getMaxNonDuration(0, SpawnMetrics::inputBytes))));
-  }
-
-  /** Returns the list of components with the largest input counts. */
-  public List<CriticalPathComponent> getLargestInputCountComponents() {
-    return uniqueActions()
-        .collect(
-            Comparators.greatest(
-                LARGEST_INPUT_COUNT_COMPONENTS_SIZE,
-                Comparator.comparingLong(
-                    (c) -> c.getSpawnMetrics().getMaxNonDuration(0, SpawnMetrics::inputFiles))));
-  }
-
-  /** Returns the list of slowest components. */
-  public List<CriticalPathComponent> getSlowestComponents() {
-    return uniqueActions()
-        .collect(
-            Comparators.greatest(
-                SLOWEST_COMPONENTS_SIZE,
-                Comparator.comparingLong(CriticalPathComponent::getElapsedTimeNanos)));
-  }
-
-  private Stream<CriticalPathComponent> uniqueActions() {
-    return outputArtifactToComponent.entrySet().stream()
-        .filter(e -> e.getValue().isPrimaryOutput(e.getKey()))
-        .map(Map.Entry::getValue);
-  }
-
-  /** Creates a CriticalPathComponent and adds the duration of input discovery and changes phase. */
-  @Subscribe
-  @AllowConcurrentEvents
-  public void discoverInputs(DiscoveredInputsEvent event) throws InterruptedException {
-    CriticalPathComponent stats =
-        tryAddComponent(createComponent(event.getAction(), event.getStartTimeNanos()));
-    stats.addSpawnResult(event.getMetrics(), null, "", /* wasRemote=*/ false);
-    stats.changePhase();
-  }
-
-  /**
-   * Record an action that has started to run. If the CriticalPathComponent has not been created,
-   * initialize it and then start running.
-   *
-   * @param event information about the started action
-   */
-  @Subscribe
-  @AllowConcurrentEvents
-  public void actionStarted(ActionStartedEvent event) throws InterruptedException {
-    Action action = event.getAction();
-    tryAddComponent(createComponent(action, event.getNanoTimeStart())).startRunning();
-  }
-
-  /**
-   * Record a middleman action execution. Even if middleman are almost instant, we record them
-   * because they depend on other actions and we need them for constructing the critical path.
-   *
-   * <p>For some rules with incorrect configuration transitions we might get notified several times
-   * for the same middleman. This should only happen if the actions are shared.
-   */
-  @Subscribe
-  @AllowConcurrentEvents
-  public void middlemanAction(ActionMiddlemanEvent event) throws InterruptedException {
-    Action action = event.getAction();
-    CriticalPathComponent component =
-        tryAddComponent(createComponent(action, event.getNanoTimeStart()));
-    finalizeActionStat(event.getNanoTimeStart(), action, component, "middleman action");
-  }
-
-  /**
-   * Try to add the component to the map of critical path components. If there is an existing
-   * component for its primary output it uses that to update the rest of the outputs.
-   *
-   * @return The component to be used for updating the time stats.
-   */
-  @SuppressWarnings("ReferenceEquality")
-  private CriticalPathComponent tryAddComponent(CriticalPathComponent newComponent)
-      throws InterruptedException {
-    Action newAction = newComponent.getAction();
-    Artifact primaryOutput = newAction.getPrimaryOutput();
-    CriticalPathComponent storedComponent =
-        outputArtifactToComponent.putIfAbsent(primaryOutput, newComponent);
-    if (storedComponent != null) {
-      Action oldAction = storedComponent.getAction();
-      // TODO(b/120663721) Replace this fragile reference equality check with something principled.
-      if (oldAction != newAction && !Actions.canBeShared(actionKeyContext, newAction, oldAction)) {
-        throw new IllegalStateException(
-            "Duplicate output artifact found for unsharable actions."
-                + "This can happen if a previous event registered the action.\n"
-                + "Old action: "
-                + oldAction
-                + "\n\nNew action: "
-                + newAction
-                + "\n\nArtifact: "
-                + primaryOutput
-                + "\n");
-      }
-    } else {
-      storedComponent = newComponent;
-    }
-    // Try to insert the existing component for the rest of the outputs even if we failed to be
-    // the ones inserting the component so that at the end of this method we guarantee that all the
-    // outputs have a component.
-    for (Artifact output : newAction.getOutputs()) {
-      if (output == primaryOutput) {
-        continue;
-      }
-      CriticalPathComponent old = outputArtifactToComponent.putIfAbsent(output, storedComponent);
-      // If two actions run concurrently maybe we find a component by primary output but we are
-      // the first updating the rest of the outputs.
-      Preconditions.checkState(
-          old == null || old == storedComponent, "Inconsistent state for %s", newAction);
-    }
-    return storedComponent;
-  }
-
-  /**
-   * Record an action that was not executed because it was in the (disk) cache. This is needed so
-   * that we can calculate correctly the dependencies tree if we have some cached actions in the
-   * middle of the critical path.
-   */
-  @Subscribe
-  @AllowConcurrentEvents
-  public void actionCached(CachedActionEvent event) throws InterruptedException {
-    Action action = event.getAction();
-    CriticalPathComponent component =
-        tryAddComponent(createComponent(action, event.getNanoTimeStart()));
-    finalizeActionStat(event.getNanoTimeStart(), action, component, "action cache hit");
-  }
-
-  /**
-   * Records the elapsed time stats for the action. For each input artifact, it finds the real
-   * dependent artifacts and records the critical path stats.
-   */
-  @Subscribe
-  @AllowConcurrentEvents
-  public void actionComplete(ActionCompletionEvent event) {
-    Action action = event.getAction();
-    CriticalPathComponent component =
-        Preconditions.checkNotNull(
-            outputArtifactToComponent.get(action.getPrimaryOutput()), action);
-    finalizeActionStat(event.getRelativeActionStartTime(), action, component, "");
-  }
-
-  /**
-   * Record that the failed rewound action is no longer running. The action may or may not start
-   * again later.
-   */
-  @Subscribe
-  @AllowConcurrentEvents
-  public void actionRewound(ActionRewoundEvent event) {
-    Action action = event.getFailedRewoundAction();
-    CriticalPathComponent component =
-        Preconditions.checkNotNull(outputArtifactToComponent.get(action.getPrimaryOutput()));
-    component.finishActionExecution(
-        event.getRelativeActionStartTime(), clock.nanoTime(), "action rewound");
-  }
-
-  /** Maximum critical path component found during the build. */
-  CriticalPathComponent getMaxCriticalPath() {
-    return maxCriticalPath.get();
-  }
-
-  private void finalizeActionStat(
-      long startTimeNanos, Action action, CriticalPathComponent component, String finalizeReason) {
-    long finishTimeNanos = clock.nanoTime();
-    for (Artifact input : action.getInputs().toList()) {
-      addArtifactDependency(component, input, finishTimeNanos);
-    }
-    if (Duration.ofNanos(finishTimeNanos - startTimeNanos).compareTo(Duration.ofMillis(-5)) < 0) {
-      // See note in {@link Clock#nanoTime} about non increasing subsequent #nanoTime calls.
-      logger.atWarning().withStackTrace(StackSize.MEDIUM).log(
-          "Negative duration time for [%s] %s with start: %s, finish: %s.",
-          action.getMnemonic(), action.getPrimaryOutput(), startTimeNanos, finishTimeNanos);
-    }
-    component.finishActionExecution(startTimeNanos, finishTimeNanos, finalizeReason);
-    maxCriticalPath.accumulateAndGet(component, SELECT_LONGER_COMPONENT);
-  }
-
-  /** If "input" is a generated artifact, link its critical path to the one we're building. */
-  private void addArtifactDependency(
-      CriticalPathComponent actionStats, Artifact input, long componentFinishNanos) {
-    CriticalPathComponent depComponent = outputArtifactToComponent.get(input);
-    // Typically, the dep component should already be finished since its output was used as an input
-    // for a just-completed action. However, we tolerate it still running for (a) action rewinding
-    // and (b) the rare case that an action depending on a previously-cached shared action sees a
-    // different shared action that is in the midst of being an action cache hit.
-    if (depComponent != null && !depComponent.isRunning()) {
-      actionStats.addDepInfo(depComponent, componentFinishNanos);
-    }
-  }
-}