Update from Google.
--
MOE_MIGRATED_REVID=85702957
diff --git a/src/main/java/com/google/devtools/build/skyframe/BuildDriver.java b/src/main/java/com/google/devtools/build/skyframe/BuildDriver.java
new file mode 100644
index 0000000..938735b
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/BuildDriver.java
@@ -0,0 +1,32 @@
+// Copyright 2014 Google Inc. 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.skyframe;
+
+import com.google.devtools.build.lib.events.EventHandler;
+
+/**
+ * A BuildDriver wraps a MemoizingEvaluator, passing along the proper Version.
+ */
+public interface BuildDriver {
+ /**
+ * See {@link MemoizingEvaluator#evaluate}, which has the same semantics except for the
+ * inclusion of a {@link Version} value.
+ */
+ <T extends SkyValue> EvaluationResult<T> evaluate(
+ Iterable<SkyKey> roots, boolean keepGoing, int numThreads, EventHandler reporter)
+ throws InterruptedException;
+
+ MemoizingEvaluator getGraphForTesting();
+}
diff --git a/src/main/java/com/google/devtools/build/skyframe/BuildingState.java b/src/main/java/com/google/devtools/build/skyframe/BuildingState.java
new file mode 100644
index 0000000..21deec1
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/BuildingState.java
@@ -0,0 +1,437 @@
+// Copyright 2014 Google Inc. 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.skyframe;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import com.google.devtools.build.lib.concurrent.ThreadSafety.ThreadCompatible;
+import com.google.devtools.build.lib.util.GroupedList;
+import com.google.devtools.build.lib.util.GroupedList.GroupedListHelper;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * Data the NodeEntry uses to maintain its state before it is done building. It allows the
+ * {@link NodeEntry} to keep the current state of the entry across invalidation and successive
+ * evaluations. A done node does not contain any of this data. However, if a node is marked dirty,
+ * its entry acquires a new {@code BuildingState} object, which persists until it is done again.
+ *
+ * <p>This class should be considered a private inner class of {@link NodeEntry} -- no other
+ * classes should instantiate a {@code BuildingState} object or call any of its methods directly.
+ * It is in a separate file solely to keep the {@link NodeEntry} class readable. In particular, the
+ * caller must synchronize access to this class.
+ */
+@ThreadCompatible
+final class BuildingState {
+ enum DirtyState {
+ /**
+ * The node's dependencies need to be checked to see if it needs to be rebuilt. The
+ * dependencies must be obtained through calls to {@link #getNextDirtyDirectDeps} and checked.
+ */
+ CHECK_DEPENDENCIES,
+ /**
+ * All of the node's dependencies are unchanged, and the value itself was not marked changed,
+ * so its current value is still valid -- it need not be rebuilt.
+ */
+ VERIFIED_CLEAN,
+ /**
+ * A rebuilding is required or in progress, because either the node itself changed or one of
+ * its dependencies did.
+ */
+ REBUILDING
+ }
+
+ /**
+ * During its life, a node can go through states as follows:
+ * <ol>
+ * <li>Non-existent
+ * <li>Just created ({@code evaluating} is false)
+ * <li>Evaluating ({@code evaluating} is true)
+ * <li>Done (meaning this buildingState object is null)
+ * <li>Just created (when it is dirtied during evaluation)
+ * <li>Reset (just before it is re-evaluated)
+ * <li>Evaluating
+ * <li>Done
+ * </ol>
+ *
+ * <p>The "just created" state is there to allow the {@link EvaluableGraph#createIfAbsent} and
+ * {@link NodeEntry#addReverseDepAndCheckIfDone} methods to be separate. All callers have to
+ * call both methods in that order if they want to create a node. The second method calls
+ * {@link #startEvaluating}, which transitions the current node to the "evaluating" state and
+ * returns true only the first time it was called. A caller that gets "true" back from that call
+ * must start the evaluation of this node, while any subsequent callers must not.
+ *
+ * <p>An entry is set to "evaluating" as soon as it is scheduled for evaluation. Thus, even a
+ * node that is never actually built (for instance, a dirty node that is verified as clean) is
+ * in the "evaluating" state until it is done.
+ */
+ private boolean evaluating = false;
+
+ /**
+ * The state of a dirty node. A node is marked dirty in the BuildingState constructor, and goes
+ * into either the state {@link DirtyState#CHECK_DEPENDENCIES} or {@link DirtyState#REBUILDING},
+ * depending on whether the caller specified that the node was itself changed or not. A non-null
+ * {@code dirtyState} indicates that the node {@link #isDirty} in some way.
+ */
+ private DirtyState dirtyState = null;
+
+ /**
+ * The number of dependencies that are known to be done in a {@link NodeEntry}. There is a
+ * potential check-then-act race here, so we need to make sure that when this is increased, we
+ * always check if the new value is equal to the number of required dependencies, and if so, we
+ * must re-schedule the node for evaluation.
+ *
+ * <p>There are two potential pitfalls here: 1) If multiple dependencies signal this node in
+ * close succession, this node should be scheduled exactly once. 2) If a thread is still working
+ * on this node, it should not be scheduled.
+ *
+ * <p>The first problem is solved by the {@link #signalDep} method, which also returns if the
+ * node needs to be re-scheduled, and ensures that only one thread gets a true return value.
+ *
+ * <p>The second problem is solved by first adding the newly discovered deps to a node's
+ * {@link #directDeps}, and then looping through the direct deps and registering this node as a
+ * reverse dependency. This ensures that the signaledDeps counter can only reach
+ * {@link #directDeps}.size() on the very last iteration of the loop, i.e., the thread is not
+ * working on the node anymore. Note that this requires that there is no code after the loop in
+ * {@code ParallelEvaluator.Evaluate#run}.
+ */
+ private int signaledDeps = 0;
+
+ /**
+ * Direct dependencies discovered during the build. They will be written to the immutable field
+ * {@code ValueEntry#directDeps} and the dependency group data to {@code ValueEntry#groupData}
+ * once the node is finished building. {@link SkyFunction}s can request deps in groups, and these
+ * groupings are preserved in this field.
+ */
+ private final GroupedList<SkyKey> directDeps = new GroupedList<>();
+
+ /**
+ * The set of reverse dependencies that are registered before the node has finished building.
+ * Upon building, these reverse deps will be signaled and then stored in the permanent
+ * {@code ValueEntry#reverseDeps}.
+ */
+ // TODO(bazel-team): Remove this field. With eager invalidation, all direct deps on this dirty
+ // node will be removed by the time evaluation starts, so reverse deps to signal can just be
+ // reverse deps in the main ValueEntry object.
+ private Object reverseDepsToSignal = ImmutableList.of();
+ private List<SkyKey> reverseDepsToRemove = null;
+ private boolean reverseDepIsSingleObject = false;
+
+ private static final ReverseDepsUtil<BuildingState> REVERSE_DEPS_UTIL =
+ new ReverseDepsUtil<BuildingState>() {
+ @Override
+ void setReverseDepsObject(BuildingState container, Object object) {
+ container.reverseDepsToSignal = object;
+ }
+
+ @Override
+ void setSingleReverseDep(BuildingState container, boolean singleObject) {
+ container.reverseDepIsSingleObject = singleObject;
+ }
+
+ @Override
+ void setReverseDepsToRemove(BuildingState container, List<SkyKey> object) {
+ container.reverseDepsToRemove = object;
+ }
+
+ @Override
+ Object getReverseDepsObject(BuildingState container) {
+ return container.reverseDepsToSignal;
+ }
+
+ @Override
+ boolean isSingleReverseDep(BuildingState container) {
+ return container.reverseDepIsSingleObject;
+ }
+
+ @Override
+ List<SkyKey> getReverseDepsToRemove(BuildingState container) {
+ return container.reverseDepsToRemove;
+ }
+ };
+
+ // Below are fields that are used for dirty nodes.
+
+ /**
+ * The dependencies requested (with group markers) last time the node was built (and below, the
+ * value last time the node was built). They will be compared to dependencies requested on this
+ * build to check whether this node has changed in {@link NodeEntry#setValue}. If they are null,
+ * it means that this node is being built for the first time. See {@link #directDeps} for more on
+ * dependency group storage.
+ */
+ private final GroupedList<SkyKey> lastBuildDirectDeps;
+ private final SkyValue lastBuildValue;
+
+ /**
+ * Which child should be re-evaluated next in the process of determining if this entry needs to
+ * be re-evaluated. Used by {@link #getNextDirtyDirectDeps} and {@link #signalDep(boolean)}.
+ */
+ private Iterator<Iterable<SkyKey>> dirtyDirectDepIterator = null;
+
+ BuildingState() {
+ lastBuildDirectDeps = null;
+ lastBuildValue = null;
+ }
+
+ private BuildingState(boolean isChanged, GroupedList<SkyKey> lastBuildDirectDeps,
+ SkyValue lastBuildValue) {
+ this.lastBuildDirectDeps = lastBuildDirectDeps;
+ this.lastBuildValue = Preconditions.checkNotNull(lastBuildValue);
+ Preconditions.checkState(isChanged || !this.lastBuildDirectDeps.isEmpty(),
+ "is being marked dirty, not changed, but has no children that could have dirtied it", this);
+ dirtyState = isChanged ? DirtyState.REBUILDING : DirtyState.CHECK_DEPENDENCIES;
+ if (dirtyState == DirtyState.CHECK_DEPENDENCIES) {
+ // We need to iterate through the deps to see if they have changed. Initialize the iterator.
+ dirtyDirectDepIterator = lastBuildDirectDeps.iterator();
+ }
+ }
+
+ static BuildingState newDirtyState(boolean isChanged,
+ GroupedList<SkyKey> lastBuildDirectDeps, SkyValue lastBuildValue) {
+ return new BuildingState(isChanged, lastBuildDirectDeps, lastBuildValue);
+ }
+
+ void markChanged() {
+ Preconditions.checkState(isDirty(), this);
+ Preconditions.checkState(!isChanged(), this);
+ Preconditions.checkState(!evaluating, this);
+ dirtyState = DirtyState.REBUILDING;
+ }
+
+ void forceChanged() {
+ Preconditions.checkState(isDirty(), this);
+ Preconditions.checkState(!isChanged(), this);
+ Preconditions.checkState(evaluating, this);
+ Preconditions.checkState(isReady(), this);
+ dirtyState = DirtyState.REBUILDING;
+ }
+
+ /**
+ * Returns whether all known children of this node have signaled that they are done.
+ */
+ boolean isReady() {
+ int directDepsSize = directDeps.size();
+ Preconditions.checkState(signaledDeps <= directDepsSize, "%s %s", directDepsSize, this);
+ return signaledDeps == directDepsSize;
+ }
+
+ /**
+ * Returns true if the entry is marked dirty, meaning that at least one of its transitive
+ * dependencies is marked changed.
+ *
+ * @see NodeEntry#isDirty()
+ */
+ boolean isDirty() {
+ return dirtyState != null;
+ }
+
+ /**
+ * Returns true if the entry is known to require re-evaluation.
+ *
+ * @see NodeEntry#isChanged()
+ */
+ boolean isChanged() {
+ return dirtyState == DirtyState.REBUILDING;
+ }
+
+ private boolean rebuilding() {
+ return dirtyState == DirtyState.REBUILDING;
+ }
+
+ /**
+ * Helper method to assert that node has finished building, as far as we can tell. We would
+ * actually like to check that the node has been evaluated, but that is not available in
+ * this context.
+ */
+ private void checkNotProcessing() {
+ Preconditions.checkState(evaluating, "not started building %s", this);
+ Preconditions.checkState(!isDirty() || dirtyState == DirtyState.VERIFIED_CLEAN
+ || rebuilding(), "not done building %s", this);
+ Preconditions.checkState(isReady(), "not done building %s", this);
+ }
+
+ /**
+ * Puts the node in the "evaluating" state if it is not already in it. Returns whether or not the
+ * node was already evaluating. Should only be called by
+ * {@link NodeEntry#addReverseDepAndCheckIfDone}.
+ */
+ boolean startEvaluating() {
+ boolean result = !evaluating;
+ evaluating = true;
+ return result;
+ }
+
+ /**
+ * Increments the number of children known to be finished. Returns true if the number of children
+ * finished is equal to the number of known children.
+ *
+ * <p>If the node is dirty and checking its deps for changes, this also updates {@link
+ * #dirtyState} as needed -- {@link DirtyState#REBUILDING} if the child has changed,
+ * and {@link DirtyState#VERIFIED_CLEAN} if the child has not changed and this was the last
+ * child to be checked (as determined by {@link #dirtyDirectDepIterator} == null, isReady(), and
+ * a flag set in {@link #getNextDirtyDirectDeps}).
+ *
+ * @see NodeEntry#signalDep(Version)
+ */
+ boolean signalDep(boolean childChanged) {
+ signaledDeps++;
+ if (isDirty() && !rebuilding()) {
+ // Synchronization isn't needed here because the only caller is ValueEntry, which does it
+ // through the synchronized method signalDep(long).
+ if (childChanged) {
+ dirtyState = DirtyState.REBUILDING;
+ } else if (dirtyState == DirtyState.CHECK_DEPENDENCIES && isReady()
+ && dirtyDirectDepIterator == null) {
+ // No other dep already marked this as REBUILDING, no deps outstanding, and this was
+ // the last block of deps to be checked.
+ dirtyState = DirtyState.VERIFIED_CLEAN;
+ }
+ }
+ return isReady();
+ }
+
+ /**
+ * Returns true if {@code newValue}.equals the value from the last time this node was built, and
+ * the deps requested during this evaluation are exactly those requested the last time this node
+ * was built, in the same order. Should only be used by {@link NodeEntry#setValue}.
+ */
+ boolean unchangedFromLastBuild(SkyValue newValue) {
+ checkNotProcessing();
+ return lastBuildValue.equals(newValue) && lastBuildDirectDeps.equals(directDeps);
+ }
+
+ boolean noDepsLastBuild() {
+ return lastBuildDirectDeps.isEmpty();
+ }
+
+ SkyValue getLastBuildValue() {
+ return Preconditions.checkNotNull(lastBuildValue, this);
+ }
+
+ /**
+ * Gets the current state of checking this dirty entry to see if it must be re-evaluated. Must be
+ * called each time evaluation of a dirty entry starts to find the proper action to perform next,
+ * as enumerated by {@link DirtyState}.
+ *
+ * @see NodeEntry#getDirtyState()
+ */
+ DirtyState getDirtyState() {
+ // Entry may not be ready if being built just for its errors.
+ Preconditions.checkState(isDirty(), "must be dirty to get dirty state %s", this);
+ Preconditions.checkState(evaluating, "must be evaluating to get dirty state %s", this);
+ return dirtyState;
+ }
+
+ /**
+ * Gets the next children to be re-evaluated to see if this dirty node needs to be re-evaluated.
+ *
+ * <p>If this is the last group of children to be checked, then sets {@link
+ * #dirtyDirectDepIterator} to null so that the final call to {@link #signalDep(boolean)} will
+ * know to mark this entry as {@link DirtyState#VERIFIED_CLEAN} if no deps have changed.
+ *
+ * See {@link NodeEntry#getNextDirtyDirectDeps}.
+ */
+ Collection<SkyKey> getNextDirtyDirectDeps() {
+ Preconditions.checkState(isDirty(), this);
+ Preconditions.checkState(dirtyState == DirtyState.CHECK_DEPENDENCIES, this);
+ Preconditions.checkState(evaluating, this);
+ List<SkyKey> nextDeps = ImmutableList.copyOf(dirtyDirectDepIterator.next());
+ if (!dirtyDirectDepIterator.hasNext()) {
+ // Done checking deps. If this last group is clean, the state will become VERIFIED_CLEAN.
+ dirtyDirectDepIterator = null;
+ }
+ return nextDeps;
+ }
+
+ void addDirectDeps(GroupedListHelper<SkyKey> depsThisRun) {
+ directDeps.append(depsThisRun);
+ }
+
+ /**
+ * Returns the direct deps found so far on this build. Should only be called before the node has
+ * finished building.
+ *
+ * @see NodeEntry#getTemporaryDirectDeps()
+ */
+ Set<SkyKey> getDirectDepsForBuild() {
+ return directDeps.toSet();
+ }
+
+ /**
+ * Returns the direct deps (in groups) found on this build. Should only be called when the node
+ * is done.
+ *
+ * @see NodeEntry#setStateFinishedAndReturnReverseDeps
+ */
+ GroupedList<SkyKey> getFinishedDirectDeps() {
+ return directDeps;
+ }
+
+ /**
+ * Returns reverse deps to signal that have been registered this build.
+ *
+ * @see NodeEntry#getReverseDeps()
+ */
+ ImmutableSet<SkyKey> getReverseDepsToSignal() {
+ return REVERSE_DEPS_UTIL.getReverseDeps(this);
+ }
+
+ /**
+ * Adds a reverse dependency that should be notified when this entry is done.
+ *
+ * @see NodeEntry#addReverseDepAndCheckIfDone(SkyKey)
+ */
+ void addReverseDepToSignal(SkyKey newReverseDep) {
+ REVERSE_DEPS_UTIL.consolidateReverseDepsRemovals(this);
+ REVERSE_DEPS_UTIL.addReverseDeps(this, Collections.singleton(newReverseDep));
+ }
+
+ /**
+ * @see NodeEntry#removeReverseDep(SkyKey)
+ */
+ void removeReverseDepToSignal(SkyKey reverseDep) {
+ REVERSE_DEPS_UTIL.removeReverseDep(this, reverseDep);
+ }
+
+ /**
+ * Removes a set of deps from the set of known direct deps. This is complicated by the need
+ * to maintain the group data. If we remove a dep that ended a group, then its predecessor's
+ * group data must be changed to indicate that it now ends the group.
+ *
+ * @see NodeEntry#removeUnfinishedDeps
+ */
+ void removeDirectDeps(Set<SkyKey> unfinishedDeps) {
+ directDeps.remove(unfinishedDeps);
+ }
+
+ @Override
+ @SuppressWarnings("deprecation")
+ public String toString() {
+ return Objects.toStringHelper(this) // MoreObjects is not in Guava
+ .add("evaluating", evaluating)
+ .add("dirtyState", dirtyState)
+ .add("signaledDeps", signaledDeps)
+ .add("directDeps", directDeps)
+ .add("reverseDepsToSignal", REVERSE_DEPS_UTIL.toString(this))
+ .add("lastBuildDirectDeps", lastBuildDirectDeps)
+ .add("lastBuildValue", lastBuildValue)
+ .add("dirtyDirectDepIterator", dirtyDirectDepIterator).toString();
+ }
+}
diff --git a/src/main/java/com/google/devtools/build/skyframe/CycleDeduper.java b/src/main/java/com/google/devtools/build/skyframe/CycleDeduper.java
new file mode 100644
index 0000000..f52333c
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/CycleDeduper.java
@@ -0,0 +1,90 @@
+// Copyright 2014 Google Inc. 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.skyframe;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+
+/**
+ * Dedupes C candidate cycles of size O(L) in O(CL) time and memory in the common case and
+ * O(C^2 * L) time and O(CL) memory in the extreme case.
+ *
+ * Two cycles are considered duplicates if they are exactly the same except for the entry point.
+ * For example, 'a' -> 'b' -> 'c' -> 'a' is the considered the same as 'b' -> 'c' -> 'a' -> 'b'.
+ */
+class CycleDeduper<T> {
+
+ private HashMultimap<ImmutableSet<T>, ImmutableList<T>> knownCyclesByMembers =
+ HashMultimap.create();
+
+ /**
+ * Marks a non-empty list representing a cycle of unique values as being seen and returns true
+ * iff the cycle hasn't been seen before, accounting for logical equivalence of cycles.
+ *
+ * For example, the cycle 'a' -> 'b' -> 'c' -> 'a' is represented by the list ['a', 'b', 'c']
+ * and is logically equivalent to the cycle represented by the list ['b', 'c', 'a'].
+ */
+ public boolean seen(ImmutableList<T> cycle) {
+ ImmutableSet<T> cycleMembers = ImmutableSet.copyOf(cycle);
+ Preconditions.checkState(!cycle.isEmpty());
+ Preconditions.checkState(cycle.size() == cycleMembers.size(),
+ "cycle doesn't have unique members: " + cycle);
+
+ if (knownCyclesByMembers.containsEntry(cycleMembers, cycle)) {
+ return false;
+ }
+
+ // Of the C cycles, suppose there are D cycles that have the same members (but are in an
+ // incompatible order). This code path takes O(D * L) time. The common case is that D is
+ // very small.
+ boolean found = false;
+ for (ImmutableList<T> candidateCycle : knownCyclesByMembers.get(cycleMembers)) {
+ int startPos = candidateCycle.indexOf(cycle.get(0));
+ // The use of a multimap keyed by cycle members guarantees that the first element of 'cycle'
+ // is present in 'candidateCycle'.
+ Preconditions.checkState(startPos >= 0);
+ if (equalsWithSingleLoopFrom(cycle, candidateCycle, startPos)) {
+ found = true;
+ break;
+ }
+ }
+ // We add the cycle even if it's a duplicate so that future exact copies of this can be
+ // processed in O(L) time. We are already using O(CL) memory, and this optimization doesn't
+ // change that.
+ knownCyclesByMembers.put(cycleMembers, cycle);
+ return !found;
+ }
+
+ /**
+ * Returns true iff
+ * listA[0], listA[1], ..., listA[listA.size()]
+ * is the same as
+ * listB[start], listB[start+1], ..., listB[listB.size()-1], listB[0], ..., listB[start-1]
+ */
+ private boolean equalsWithSingleLoopFrom(ImmutableList<T> listA, ImmutableList<T> listB,
+ int start) {
+ if (listA.size() != listB.size()) {
+ return false;
+ }
+ int length = listA.size();
+ for (int i = 0; i < length; i++) {
+ if (!listA.get(i).equals(listB.get((i + start) % length))) {
+ return false;
+ }
+ }
+ return true;
+ }
+}
diff --git a/src/main/java/com/google/devtools/build/skyframe/CycleInfo.java b/src/main/java/com/google/devtools/build/skyframe/CycleInfo.java
new file mode 100644
index 0000000..a44d2fa
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/CycleInfo.java
@@ -0,0 +1,144 @@
+// Copyright 2014 Google Inc. 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.skyframe;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Function;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Predicates;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+
+import java.io.Serializable;
+import java.util.HashSet;
+import java.util.Objects;
+import java.util.Set;
+
+/**
+ * Data for a single cycle in the graph, together with the path to the cycle. For any value, the
+ * head of path to the cycle should be the value itself, or, if the value is actually in the cycle,
+ * the cycle should start with the value.
+ */
+public class CycleInfo implements Serializable {
+ private final ImmutableList<SkyKey> cycle;
+ private final ImmutableList<SkyKey> pathToCycle;
+
+ @VisibleForTesting
+ public CycleInfo(Iterable<SkyKey> cycle) {
+ this(ImmutableList.<SkyKey>of(), cycle);
+ }
+
+ CycleInfo(Iterable<SkyKey> pathToCycle, Iterable<SkyKey> cycle) {
+ this.pathToCycle = ImmutableList.copyOf(pathToCycle);
+ this.cycle = ImmutableList.copyOf(cycle);
+ }
+
+ // If a cycle is already known, but we are processing a value in the middle of the cycle, we need
+ // to shift the cycle so that the value is at the head.
+ private CycleInfo(Iterable<SkyKey> cycle, int cycleStart) {
+ Preconditions.checkState(cycleStart >= 0, cycleStart);
+ ImmutableList.Builder<SkyKey> cycleTail = ImmutableList.builder();
+ ImmutableList.Builder<SkyKey> cycleHead = ImmutableList.builder();
+ int index = 0;
+ for (SkyKey key : cycle) {
+ if (index >= cycleStart) {
+ cycleHead.add(key);
+ } else {
+ cycleTail.add(key);
+ }
+ index++;
+ }
+ Preconditions.checkState(cycleStart < index, "%s >= %s ??", cycleStart, index);
+ this.cycle = cycleHead.addAll(cycleTail.build()).build();
+ this.pathToCycle = ImmutableList.of();
+ }
+
+ public ImmutableList<SkyKey> getCycle() {
+ return cycle;
+ }
+
+ public ImmutableList<SkyKey> getPathToCycle() {
+ return pathToCycle;
+ }
+
+ // Given a cycle and a value, if the value is part of the cycle, shift the cycle. Otherwise,
+ // prepend the value to the head of pathToCycle.
+ private static CycleInfo normalizeCycle(final SkyKey value, CycleInfo cycle) {
+ int index = cycle.cycle.indexOf(value);
+ if (index > -1) {
+ if (!cycle.pathToCycle.isEmpty()) {
+ // The head value we are considering is already part of a cycle, but we have reached it by a
+ // roundabout way. Since we should have reached it directly as well, filter this roundabout
+ // way out. Example (c has a dependence on top):
+ // top
+ // / ^
+ // a |
+ // / \ /
+ // b-> c
+ // In the traversal, we start at top, visit a, then c, then top. This yields the
+ // cycle {top,a,c}. Then we visit b, getting (b, {top,a,c}). Then we construct the full
+ // error for a. The error should just be the cycle {top,a,c}, but we have an extra copy of
+ // it via the path through b.
+ return null;
+ }
+ return new CycleInfo(cycle.cycle, index);
+ }
+ return new CycleInfo(Iterables.concat(ImmutableList.of(value), cycle.pathToCycle),
+ cycle.cycle);
+ }
+
+ /**
+ * Normalize multiple cycles. This includes removing multiple paths to the same cycle, so that
+ * a value does not depend on the same cycle multiple ways through the same child value. Note that
+ * a value can still depend on the same cycle multiple ways, it's just that each way must be
+ * through a different child value (a path with a different first element).
+ */
+ static Iterable<CycleInfo> prepareCycles(final SkyKey value, Iterable<CycleInfo> cycles) {
+ final Set<ImmutableList<SkyKey>> alreadyDoneCycles = new HashSet<>();
+ return Iterables.filter(Iterables.transform(cycles,
+ new Function<CycleInfo, CycleInfo>() {
+ @Override
+ public CycleInfo apply(CycleInfo input) {
+ CycleInfo normalized = normalizeCycle(value, input);
+ if (normalized != null && alreadyDoneCycles.add(normalized.cycle)) {
+ return normalized;
+ }
+ return null;
+ }
+ }), Predicates.notNull());
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(cycle, pathToCycle);
+ }
+
+ @Override
+ public boolean equals(Object that) {
+ if (this == that) {
+ return true;
+ }
+ if (!(that instanceof CycleInfo)) {
+ return false;
+ }
+
+ CycleInfo thatCycle = (CycleInfo) that;
+ return thatCycle.cycle.equals(this.cycle) && thatCycle.pathToCycle.equals(this.pathToCycle);
+ }
+
+ @Override
+ public String toString() {
+ return Iterables.toString(pathToCycle) + " -> " + Iterables.toString(cycle);
+ }
+}
diff --git a/src/main/java/com/google/devtools/build/skyframe/CyclesReporter.java b/src/main/java/com/google/devtools/build/skyframe/CyclesReporter.java
new file mode 100644
index 0000000..a9b0d8e
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/CyclesReporter.java
@@ -0,0 +1,102 @@
+// Copyright 2014 Google Inc. 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.skyframe;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.devtools.build.lib.events.EventHandler;
+
+/**
+ * An utility for custom reporting of errors from cycles in the the Skyframe graph. This class is
+ * stateful in order to differentiate between new cycles and cycles that have already been
+ * reported (do not reuse the instances or cache the results as it could end up printing
+ * inconsistent information or leak memory). It treats two cycles as the same if they contain the
+ * same {@link SkyKey}s in the same order, but perhaps with different starting points. See
+ * {@link CycleDeduper} for more information.
+ */
+public class CyclesReporter {
+
+ /**
+ * Interface for reporting custom information about a single cycle.
+ */
+ public interface SingleCycleReporter {
+
+ /**
+ * Reports the given cycle and returns {@code true}, or return {@code false} if this
+ * {@link SingleCycleReporter} doesn't know how to report the cycle.
+ *
+ * @param topLevelKey the top level key that transitively depended on the cycle
+ * @param cycleInfo the cycle
+ * @param alreadyReported whether the cycle has already been reported to the
+ * {@link CyclesReporter}.
+ * @param eventHandler the eventHandler to which to report the error
+ */
+ boolean maybeReportCycle(SkyKey topLevelKey, CycleInfo cycleInfo, boolean alreadyReported,
+ EventHandler eventHandler);
+ }
+
+ private final ImmutableList<SingleCycleReporter> cycleReporters;
+ private final CycleDeduper<SkyKey> cycleDeduper = new CycleDeduper<>();
+
+ /**
+ * Constructs a {@link CyclesReporter} that delegates to the given {@link SingleCycleReporter}s,
+ * in the given order, to report custom information about cycles.
+ */
+ public CyclesReporter(SingleCycleReporter... cycleReporters) {
+ this.cycleReporters = ImmutableList.copyOf(cycleReporters);
+ }
+
+ /**
+ * Reports the given cycles, differentiating between cycles that have already been reported.
+ *
+ * @param cycles The {@code Iterable} of cycles.
+ * @param topLevelKey This key represents the top level value key that returned cycle errors.
+ * @param eventHandler the eventHandler to which to report the error
+ */
+ public void reportCycles(Iterable<CycleInfo> cycles, SkyKey topLevelKey,
+ EventHandler eventHandler) {
+ Preconditions.checkNotNull(eventHandler);
+ for (CycleInfo cycleInfo : cycles) {
+ boolean alreadyReported = false;
+ if (!cycleDeduper.seen(cycleInfo.getCycle())) {
+ alreadyReported = true;
+ }
+ boolean successfullyReported = false;
+ for (SingleCycleReporter cycleReporter : cycleReporters) {
+ if (cycleReporter.maybeReportCycle(topLevelKey, cycleInfo, alreadyReported, eventHandler)) {
+ successfullyReported = true;
+ break;
+ }
+ }
+ Preconditions.checkState(successfullyReported,
+ printArbitraryCycle(topLevelKey, cycleInfo, alreadyReported));
+ }
+ }
+
+ private String printArbitraryCycle(SkyKey topLevelKey, CycleInfo cycleInfo,
+ boolean alreadyReported) {
+ StringBuilder cycleMessage = new StringBuilder()
+ .append("topLevelKey: " + topLevelKey + "\n")
+ .append("alreadyReported: " + alreadyReported + "\n")
+ .append("path to cycle:\n");
+ for (SkyKey skyKey : cycleInfo.getPathToCycle()) {
+ cycleMessage.append(skyKey + "\n");
+ }
+ cycleMessage.append("cycle:\n");
+ for (SkyKey skyKey : cycleInfo.getCycle()) {
+ cycleMessage.append(skyKey + "\n");
+ }
+ return cycleMessage.toString();
+ }
+}
diff --git a/src/main/java/com/google/devtools/build/skyframe/Differencer.java b/src/main/java/com/google/devtools/build/skyframe/Differencer.java
new file mode 100644
index 0000000..f6433ac
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/Differencer.java
@@ -0,0 +1,45 @@
+// Copyright 2014 Google Inc. 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.skyframe;
+
+import java.util.Map;
+
+/**
+ * Calculate set of changed values in a graph.
+ */
+public interface Differencer {
+
+ /**
+ * Represents a set of changed values.
+ */
+ interface Diff {
+ /**
+ * Returns the value keys whose values have changed, but for which we don't have the new values.
+ */
+ Iterable<SkyKey> changedKeysWithoutNewValues();
+
+ /**
+ * Returns the value keys whose values have changed, along with their new values.
+ *
+ * <p> The values in here cannot have any dependencies. This is required in order to prevent
+ * conflation of injected values and derived values.
+ */
+ Map<SkyKey, ? extends SkyValue> changedKeysWithNewValues();
+ }
+
+ /**
+ * Returns the value keys that have changed between the two Versions.
+ */
+ Diff getDiff(Version fromVersion, Version toVersion) throws InterruptedException;
+}
diff --git a/src/main/java/com/google/devtools/build/skyframe/DirtiableGraph.java b/src/main/java/com/google/devtools/build/skyframe/DirtiableGraph.java
new file mode 100644
index 0000000..0781222
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/DirtiableGraph.java
@@ -0,0 +1,28 @@
+// Copyright 2014 Google Inc. 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.skyframe;
+
+/**
+ * Interface for classes that need to remove values from graph. Currently just used by {@link
+ * EagerInvalidator}.
+ *
+ * <p>This class is not intended for direct use, and is only exposed as public for use in
+ * evaluation implementations outside of this package.
+ */
+public interface DirtiableGraph extends QueryableGraph {
+ /**
+ * Remove the value with given name from the graph.
+ */
+ void remove(SkyKey key);
+}
diff --git a/src/main/java/com/google/devtools/build/skyframe/DirtyKeyTracker.java b/src/main/java/com/google/devtools/build/skyframe/DirtyKeyTracker.java
new file mode 100644
index 0000000..b0b5074
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/DirtyKeyTracker.java
@@ -0,0 +1,43 @@
+// Copyright 2014 Google Inc. 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.skyframe;
+
+import com.google.devtools.build.lib.concurrent.ThreadSafety.ThreadSafe;
+
+import java.util.Set;
+
+/**
+ * Interface for implementations that need to keep track of dirty SkyKeys.
+ */
+public interface DirtyKeyTracker {
+
+ /**
+ * Marks the {@code skyKey} as dirty.
+ */
+ @ThreadSafe
+ void dirty(SkyKey skyKey);
+
+ /**
+ * Marks the {@code skyKey} as not dirty.
+ */
+ @ThreadSafe
+ void notDirty(SkyKey skyKey);
+
+ /**
+ * Returns the set of keys k for which there was a call to dirty(k) but not a subsequent call
+ * to notDirty(k).
+ */
+ @ThreadSafe
+ Set<SkyKey> getDirtyKeys();
+}
diff --git a/src/main/java/com/google/devtools/build/skyframe/DirtyKeyTrackerImpl.java b/src/main/java/com/google/devtools/build/skyframe/DirtyKeyTrackerImpl.java
new file mode 100644
index 0000000..e3e070cb
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/DirtyKeyTrackerImpl.java
@@ -0,0 +1,40 @@
+// Copyright 2014 Google Inc. 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.skyframe;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Sets;
+
+import java.util.Set;
+
+/** Encapsulates a thread-safe set of SkyKeys. */
+public class DirtyKeyTrackerImpl implements DirtyKeyTracker {
+
+ private final Set<SkyKey> dirtyKeys = Sets.newConcurrentHashSet();
+
+ @Override
+ public void dirty(SkyKey skyKey) {
+ dirtyKeys.add(skyKey);
+ }
+
+ @Override
+ public void notDirty(SkyKey skyKey) {
+ dirtyKeys.remove(skyKey);
+ }
+
+ @Override
+ public Set<SkyKey> getDirtyKeys() {
+ return ImmutableSet.copyOf(dirtyKeys);
+ }
+}
diff --git a/src/main/java/com/google/devtools/build/skyframe/EagerInvalidator.java b/src/main/java/com/google/devtools/build/skyframe/EagerInvalidator.java
new file mode 100644
index 0000000..fc2a2c7
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/EagerInvalidator.java
@@ -0,0 +1,85 @@
+// Copyright 2014 Google Inc. 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.skyframe;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.devtools.build.skyframe.InvalidatingNodeVisitor.DeletingNodeVisitor;
+import com.google.devtools.build.skyframe.InvalidatingNodeVisitor.DirtyingNodeVisitor;
+import com.google.devtools.build.skyframe.InvalidatingNodeVisitor.InvalidationState;
+
+/**
+ * Utility class for performing eager invalidation on Skyframe graphs.
+ *
+ * <p>This is intended only for use in alternative {@code MemoizingEvaluator} implementations.
+ */
+public final class EagerInvalidator {
+
+ private EagerInvalidator() {}
+
+ /**
+ * Deletes given values. The {@code traverseGraph} parameter controls whether this method deletes
+ * (transitive) dependents of these nodes and relevant graph edges, or just the nodes themselves.
+ * Deleting just the nodes is inconsistent unless the graph will not be used for incremental
+ * builds in the future, but unfortunately there is a case where we delete nodes intra-build. As
+ * long as the full upward transitive closure of the nodes is specified for deletion, the graph
+ * remains consistent.
+ */
+ public static void delete(DirtiableGraph graph, Iterable<SkyKey> diff,
+ EvaluationProgressReceiver invalidationReceiver, InvalidationState state,
+ boolean traverseGraph, DirtyKeyTracker dirtyKeyTracker) throws InterruptedException {
+ InvalidatingNodeVisitor visitor =
+ createVisitor(/*delete=*/true, graph, diff, invalidationReceiver, state, traverseGraph,
+ dirtyKeyTracker);
+ if (visitor != null) {
+ visitor.run();
+ }
+ }
+
+ /**
+ * Creates an invalidation visitor that is ready to run. Caller should call #run() on the visitor.
+ * Allows test classes to keep a reference to the visitor, and await exceptions/interrupts.
+ */
+ @VisibleForTesting
+ static InvalidatingNodeVisitor createVisitor(boolean delete, DirtiableGraph graph,
+ Iterable<SkyKey> diff, EvaluationProgressReceiver invalidationReceiver,
+ InvalidationState state, boolean traverseGraph, DirtyKeyTracker dirtyKeyTracker) {
+ state.update(diff);
+ if (state.isEmpty()) {
+ return null;
+ }
+ return delete
+ ? new DeletingNodeVisitor(graph, invalidationReceiver, state, traverseGraph,
+ dirtyKeyTracker)
+ : new DirtyingNodeVisitor(graph, invalidationReceiver, state, dirtyKeyTracker);
+ }
+
+ /**
+ * Invalidates given values and their upward transitive closure in the graph.
+ */
+ public static void invalidate(DirtiableGraph graph, Iterable<SkyKey> diff,
+ EvaluationProgressReceiver invalidationReceiver, InvalidationState state,
+ DirtyKeyTracker dirtyKeyTracker)
+ throws InterruptedException {
+ // If we are invalidating, we must be in an incremental build by definition, so we must
+ // maintain a consistent graph state by traversing the graph and invalidating transitive
+ // dependencies. If edges aren't present, it would be impossible to check the dependencies of
+ // a dirty node in any case.
+ InvalidatingNodeVisitor visitor =
+ createVisitor(/*delete=*/false, graph, diff, invalidationReceiver, state,
+ /*traverseGraph=*/true, dirtyKeyTracker);
+ if (visitor != null) {
+ visitor.run();
+ }
+ }
+}
diff --git a/src/main/java/com/google/devtools/build/skyframe/EdgelessNodeEntry.java b/src/main/java/com/google/devtools/build/skyframe/EdgelessNodeEntry.java
new file mode 100644
index 0000000..98fb61e
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/EdgelessNodeEntry.java
@@ -0,0 +1,32 @@
+// Copyright 2014 Google Inc. 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.skyframe;
+
+/**
+ * NodeEntry that does not store edges (directDeps and reverseDeps) when the node is done. Used to
+ * save memory when it is known that the graph will not be reused.
+ *
+ * <p>Graph edges must be stored for incremental builds, but if this program will terminate after a
+ * single run, edges can be thrown away in order to save memory. The edges will be stored in the
+ * {@link BuildingState} as usual while the node is being built, but will not be stored once the
+ * node is done and written to the graph. Any attempt to access the edges once the node is done will
+ * fail the build fast.
+ */
+class EdgelessNodeEntry extends NodeEntry {
+ @Override
+ protected boolean keepEdges() {
+ return false;
+ }
+}
diff --git a/src/main/java/com/google/devtools/build/skyframe/ErrorInfo.java b/src/main/java/com/google/devtools/build/skyframe/ErrorInfo.java
new file mode 100644
index 0000000..6873d19
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/ErrorInfo.java
@@ -0,0 +1,157 @@
+// Copyright 2014 Google Inc. 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.skyframe;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.devtools.build.lib.collect.nestedset.NestedSet;
+import com.google.devtools.build.lib.collect.nestedset.NestedSetBuilder;
+import com.google.devtools.build.lib.collect.nestedset.Order;
+import com.google.devtools.build.skyframe.SkyFunctionException.ReifiedSkyFunctionException;
+
+import java.io.Serializable;
+import java.util.Collection;
+
+import javax.annotation.Nullable;
+
+/**
+ * Information about why a {@link SkyValue} failed to evaluate successfully.
+ *
+ * <p>This is intended only for use in alternative {@code MemoizingEvaluator} implementations.
+ */
+public class ErrorInfo implements Serializable {
+ /**
+ * The set of descendants of this value that failed to build
+ */
+ private final NestedSet<SkyKey> rootCauses;
+
+ /**
+ * An exception thrown upon a value's failure to build. The exception is used for reporting, and
+ * thus may ultimately be rethrown by the caller. As well, during a --nokeep_going evaluation, if
+ * an error value is encountered from an earlier --keep_going build, the exception to be thrown is
+ * taken from here.
+ */
+ @Nullable private final Exception exception;
+ private final SkyKey rootCauseOfException;
+
+ private final Iterable<CycleInfo> cycles;
+
+ private final boolean isTransient;
+ private final boolean isCatastrophic;
+
+ public ErrorInfo(ReifiedSkyFunctionException builderException) {
+ this.rootCauseOfException = builderException.getRootCauseSkyKey();
+ this.rootCauses = NestedSetBuilder.create(Order.STABLE_ORDER, rootCauseOfException);
+ this.exception = Preconditions.checkNotNull(builderException.getCause(), builderException);
+ this.cycles = ImmutableList.of();
+ this.isTransient = builderException.isTransient();
+ this.isCatastrophic = builderException.isCatastrophic();
+ }
+
+ ErrorInfo(CycleInfo cycleInfo) {
+ this.rootCauses = NestedSetBuilder.emptySet(Order.STABLE_ORDER);
+ this.exception = null;
+ this.rootCauseOfException = null;
+ this.cycles = ImmutableList.of(cycleInfo);
+ this.isTransient = false;
+ this.isCatastrophic = false;
+ }
+
+ public ErrorInfo(SkyKey currentValue, Collection<ErrorInfo> childErrors) {
+ Preconditions.checkNotNull(currentValue);
+ Preconditions.checkState(!childErrors.isEmpty(),
+ "Error value %s with no exception must depend on another error value", currentValue);
+ NestedSetBuilder<SkyKey> builder = NestedSetBuilder.stableOrder();
+ ImmutableList.Builder<CycleInfo> cycleBuilder = ImmutableList.builder();
+ Exception firstException = null;
+ SkyKey firstChildKey = null;
+ boolean isTransient = false;
+ boolean isCatastrophic = false;
+ // Arbitrarily pick the first error.
+ for (ErrorInfo child : childErrors) {
+ if (firstException == null) {
+ firstException = child.getException();
+ firstChildKey = child.getRootCauseOfException();
+ }
+ builder.addTransitive(child.rootCauses);
+ cycleBuilder.addAll(CycleInfo.prepareCycles(currentValue, child.cycles));
+ isTransient |= child.isTransient();
+ isCatastrophic |= child.isCatastrophic();
+ }
+ this.rootCauses = builder.build();
+ this.exception = firstException;
+ this.rootCauseOfException = firstChildKey;
+ this.cycles = cycleBuilder.build();
+ this.isTransient = isTransient;
+ this.isCatastrophic = isCatastrophic;
+ }
+
+ @Override
+ public String toString() {
+ return String.format("<ErrorInfo exception=%s rootCauses=%s cycles=%s>",
+ exception, rootCauses, cycles);
+ }
+
+ /**
+ * The root causes of a value that failed to build are its descendant values that failed to build.
+ * If a value's descendants all built successfully, but it failed to, its root cause will be
+ * itself. If a value depends on a cycle, but has no other errors, this method will return
+ * the empty set.
+ */
+ public Iterable<SkyKey> getRootCauses() {
+ return rootCauses;
+ }
+
+ /**
+ * The exception thrown when building a value. May be null if value's only error is depending
+ * on a cycle.
+ */
+ @Nullable public Exception getException() {
+ return exception;
+ }
+
+ public SkyKey getRootCauseOfException() {
+ return rootCauseOfException;
+ }
+
+ /**
+ * Any cycles found when building this value.
+ *
+ * <p>If there are a large number of cycles, only a limited number are returned here.
+ *
+ * <p>If this value has a child through which there are multiple paths to the same cycle, only one
+ * path is returned here. However, if there are multiple paths to the same cycle, each of which
+ * goes through a different child, each of them is returned here.
+ */
+ public Iterable<CycleInfo> getCycleInfo() {
+ return cycles;
+ }
+
+ /**
+ * Returns true iff the error is transient, i.e. if retrying the same computation could lead to a
+ * different result.
+ */
+ public boolean isTransient() {
+ return isTransient;
+ }
+
+
+ /**
+ * Returns true iff the error is catastrophic, i.e. it should halt even for a keepGoing update()
+ * call.
+ */
+ public boolean isCatastrophic() {
+ return isCatastrophic;
+ }
+}
diff --git a/src/main/java/com/google/devtools/build/skyframe/ErrorTransienceValue.java b/src/main/java/com/google/devtools/build/skyframe/ErrorTransienceValue.java
new file mode 100644
index 0000000..c0c445d
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/ErrorTransienceValue.java
@@ -0,0 +1,29 @@
+// Copyright 2014 Google Inc. 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.skyframe;
+
+/**
+ * A value that represents "error transience", i.e. anything which may have caused an unexpected
+ * failure.
+ */
+public final class ErrorTransienceValue implements SkyValue {
+ public static final SkyFunctionName FUNCTION_NAME =
+ new SkyFunctionName("ERROR_TRANSIENCE", false);
+
+ ErrorTransienceValue() {}
+
+ public static SkyKey key() {
+ return new SkyKey(FUNCTION_NAME, "ERROR_TRANSIENCE");
+ }
+}
diff --git a/src/main/java/com/google/devtools/build/skyframe/EvaluableGraph.java b/src/main/java/com/google/devtools/build/skyframe/EvaluableGraph.java
new file mode 100644
index 0000000..3d9a934
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/EvaluableGraph.java
@@ -0,0 +1,26 @@
+// Copyright 2014 Google Inc. 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.skyframe;
+
+/**
+ * Interface between a single version of the graph and the evaluator. Supports mutation of that
+ * single version of the graph.
+ */
+interface EvaluableGraph extends QueryableGraph {
+ /**
+ * Creates a new node with the specified key if it does not exist yet. Returns the node entry
+ * (either the existing one or the one just created), never {@code null}.
+ */
+ NodeEntry createIfAbsent(SkyKey key);
+}
diff --git a/src/main/java/com/google/devtools/build/skyframe/EvaluationProgressReceiver.java b/src/main/java/com/google/devtools/build/skyframe/EvaluationProgressReceiver.java
new file mode 100644
index 0000000..7928878
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/EvaluationProgressReceiver.java
@@ -0,0 +1,77 @@
+// Copyright 2014 Google Inc. 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.skyframe;
+
+import com.google.devtools.build.lib.concurrent.ThreadSafety;
+
+/**
+ * Receiver to inform callers which values have been invalidated. Values may be invalidated and then
+ * re-validated if they have been found not to be changed.
+ */
+public interface EvaluationProgressReceiver {
+ /**
+ * New state of the value entry after evaluation.
+ */
+ enum EvaluationState {
+ /** The value was successfully re-evaluated. */
+ BUILT,
+ /** The value is clean or re-validated. */
+ CLEAN,
+ }
+
+ /**
+ * New state of the value entry after invalidation.
+ */
+ enum InvalidationState {
+ /** The value is dirty, although it might get re-validated again. */
+ DIRTY,
+ /** The value is dirty and got deleted, cannot get re-validated again. */
+ DELETED,
+ }
+
+ /**
+ * Notifies that {@code value} has been invalidated.
+ *
+ * <p>{@code state} indicates the new state of the value.
+ *
+ * <p>This method is not called on invalidation of values which do not have a value (usually
+ * because they are in error).
+ *
+ * <p>May be called concurrently from multiple threads, possibly with the same {@code value}
+ * object.
+ */
+ @ThreadSafety.ThreadSafe
+ void invalidated(SkyValue value, InvalidationState state);
+
+ /**
+ * Notifies that {@code skyKey} is about to get queued for evaluation.
+ *
+ * <p>Note that we don't guarantee that it actually got enqueued or will, only that if
+ * everything "goes well" (e.g. no interrupts happen) it will.
+ *
+ * <p>This guarantee is intentionally vague to encourage writing robust implementations.
+ */
+ @ThreadSafety.ThreadSafe
+ void enqueueing(SkyKey skyKey);
+
+ /**
+ * Notifies that {@code value} has been evaluated.
+ *
+ * <p>{@code state} indicates the new state of the value.
+ *
+ * <p>This method is not called if the value builder threw an error when building this value.
+ */
+ @ThreadSafety.ThreadSafe
+ void evaluated(SkyKey skyKey, SkyValue value, EvaluationState state);
+}
diff --git a/src/main/java/com/google/devtools/build/skyframe/EvaluationResult.java b/src/main/java/com/google/devtools/build/skyframe/EvaluationResult.java
new file mode 100644
index 0000000..e518dca
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/EvaluationResult.java
@@ -0,0 +1,163 @@
+// Copyright 2014 Google Inc. 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.skyframe;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * The result of a Skyframe {@link Evaluator#eval} call. Will contain all the
+ * successfully evaluated values, retrievable through {@link #get}. As well, the {@link ErrorInfo}
+ * for the first value that failed to evaluate (in the non-keep-going case), or any remaining values
+ * that failed to evaluate (in the keep-going case) will be retrievable.
+ *
+ * @param <T> The type of the values that the caller has requested.
+ */
+public class EvaluationResult<T extends SkyValue> {
+
+ private final boolean hasError;
+
+ private final Map<SkyKey, T> resultMap;
+ private final Map<SkyKey, ErrorInfo> errorMap;
+
+ /**
+ * Constructor for the "completed" case. Used only by {@link Builder}.
+ */
+ private EvaluationResult(Map<SkyKey, T> result, Map<SkyKey, ErrorInfo> errorMap,
+ boolean hasError) {
+ Preconditions.checkState(errorMap.isEmpty() || hasError,
+ "result=%s, errorMap=%s", result, errorMap);
+ this.resultMap = Preconditions.checkNotNull(result);
+ this.errorMap = Preconditions.checkNotNull(errorMap);
+ this.hasError = hasError;
+ }
+
+ /**
+ * Get a successfully evaluated value.
+ */
+ public T get(SkyKey key) {
+ Preconditions.checkNotNull(resultMap, key);
+ return resultMap.get(key);
+ }
+
+ /**
+ * @return Whether or not the eval successfully evaluated all requested values. Note that this
+ * may return true even if all values returned are available in get(). This happens if a top-level
+ * value depends transitively on some value that recovered from a {@link SkyFunctionException}.
+ */
+ public boolean hasError() {
+ return hasError;
+ }
+
+ /**
+ * @return All successfully evaluated {@link SkyValue}s.
+ */
+ public Collection<T> values() {
+ return Collections.unmodifiableCollection(resultMap.values());
+ }
+
+ /**
+ * Returns {@link Map} of {@link SkyKey}s to {@link ErrorInfo}. Note that currently some
+ * of the returned SkyKeys may not be the ones requested by the user. Moreover, the SkyKey
+ * is not necessarily the cause of the error -- it is just the value that was being evaluated
+ * when the error was discovered. For the cause of the error, use
+ * {@link ErrorInfo#getRootCauses()} on each ErrorInfo.
+ */
+ public Map<SkyKey, ErrorInfo> errorMap() {
+ return ImmutableMap.copyOf(errorMap);
+ }
+
+ /**
+ * @param key {@link SkyKey} to get {@link ErrorInfo} for.
+ */
+ public ErrorInfo getError(SkyKey key) {
+ return Preconditions.checkNotNull(errorMap, key).get(key);
+ }
+
+ /**
+ * @return Names of all values that were successfully evaluated.
+ */
+ public <S> Collection<? extends S> keyNames() {
+ return this.<S>getNames(resultMap.keySet());
+ }
+
+ @SuppressWarnings("unchecked")
+ private <S> Collection<? extends S> getNames(Collection<SkyKey> keys) {
+ Collection<S> names = Lists.newArrayListWithCapacity(keys.size());
+ for (SkyKey key : keys) {
+ names.add((S) key.argument());
+ }
+ return names;
+ }
+
+ /**
+ * Returns some error info. Convenience method equivalent to
+ * Iterables.getFirst({@link #errorMap()}, null).getValue().
+ */
+ public ErrorInfo getError() {
+ return Iterables.getFirst(errorMap.entrySet(), null).getValue();
+ }
+
+ @Override
+ @SuppressWarnings("deprecation")
+ public String toString() {
+ return Objects.toStringHelper(this) // MoreObjects is not in Guava
+ .add("hasError", hasError)
+ .add("errorMap", errorMap)
+ .add("resultMap", resultMap)
+ .toString();
+ }
+
+ public static <T extends SkyValue> Builder<T> builder() {
+ return new Builder<>();
+ }
+
+ /**
+ * Builder for {@link EvaluationResult}.
+ *
+ * <p>This is intended only for use in alternative {@code MemoizingEvaluator} implementations.
+ */
+ public static class Builder<T extends SkyValue> {
+ private final Map<SkyKey, T> result = new HashMap<>();
+ private final Map<SkyKey, ErrorInfo> errors = new HashMap<>();
+ private boolean hasError = false;
+
+ @SuppressWarnings("unchecked")
+ public Builder<T> addResult(SkyKey key, SkyValue value) {
+ result.put(key, Preconditions.checkNotNull((T) value, key));
+ return this;
+ }
+
+ public Builder<T> addError(SkyKey key, ErrorInfo error) {
+ errors.put(key, Preconditions.checkNotNull(error, key));
+ return this;
+ }
+
+ public EvaluationResult<T> build() {
+ return new EvaluationResult<>(result, errors, hasError);
+ }
+
+ public void setHasError(boolean hasError) {
+ this.hasError = hasError;
+ }
+ }
+}
diff --git a/src/main/java/com/google/devtools/build/skyframe/Evaluator.java b/src/main/java/com/google/devtools/build/skyframe/Evaluator.java
new file mode 100644
index 0000000..342eff1
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/Evaluator.java
@@ -0,0 +1,43 @@
+// Copyright 2014 Google Inc. 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.skyframe;
+
+import com.google.devtools.build.lib.events.EventHandler;
+
+/**
+ * An interface for the evaluator for a particular graph version.
+ */
+public interface Evaluator {
+ /**
+ * Factory to create Evaluator instances.
+ */
+ interface Factory {
+ /**
+ * @param graph the graph to operate on
+ * @param graphVersion the version at which to write entries in the graph.
+ * @param reporter where to write warning/error/progress messages.
+ * @param keepGoing whether {@link #eval} should continue if building a {link Value} fails.
+ * Otherwise, we throw an exception on failure.
+ */
+ Evaluator create(ProcessableGraph graph, long graphVersion, EventHandler reporter,
+ boolean keepGoing);
+ }
+
+ /**
+ * Evaluates a set of values. Returns an {@link EvaluationResult}. All elements of skyKeys must
+ * be keys for Values of subtype T.
+ */
+ <T extends SkyValue> EvaluationResult<T> eval(Iterable<SkyKey> skyKeys)
+ throws InterruptedException;
+}
diff --git a/src/main/java/com/google/devtools/build/skyframe/ImmutableDiff.java b/src/main/java/com/google/devtools/build/skyframe/ImmutableDiff.java
new file mode 100644
index 0000000..46ab29e
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/ImmutableDiff.java
@@ -0,0 +1,43 @@
+// Copyright 2014 Google Inc. 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.skyframe;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+
+import java.util.Map;
+
+/**
+ * Immutable implementation of {@link Differencer.Diff}.
+ */
+public class ImmutableDiff implements Differencer.Diff {
+
+ private final ImmutableList<SkyKey> valuesToInvalidate;
+ private final ImmutableMap<SkyKey, SkyValue> valuesToInject;
+
+ public ImmutableDiff(Iterable<SkyKey> valuesToInvalidate, Map<SkyKey, SkyValue> valuesToInject) {
+ this.valuesToInvalidate = ImmutableList.copyOf(valuesToInvalidate);
+ this.valuesToInject = ImmutableMap.copyOf(valuesToInject);
+ }
+
+ @Override
+ public Iterable<SkyKey> changedKeysWithoutNewValues() {
+ return valuesToInvalidate;
+ }
+
+ @Override
+ public Map<SkyKey, SkyValue> changedKeysWithNewValues() {
+ return valuesToInject;
+ }
+}
diff --git a/src/main/java/com/google/devtools/build/skyframe/InMemoryGraph.java b/src/main/java/com/google/devtools/build/skyframe/InMemoryGraph.java
new file mode 100644
index 0000000..44956da
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/InMemoryGraph.java
@@ -0,0 +1,126 @@
+// Copyright 2014 Google Inc. 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.skyframe;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Function;
+import com.google.common.base.Predicate;
+import com.google.common.base.Predicates;
+import com.google.common.collect.MapMaker;
+import com.google.common.collect.Maps;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+
+import javax.annotation.Nullable;
+
+/**
+ * An in-memory graph implementation. All operations are thread-safe with ConcurrentMap semantics.
+ * Also see {@link NodeEntry}.
+ *
+ * <p>This class is public only for use in alternative graph implementations.
+ */
+public class InMemoryGraph implements ProcessableGraph {
+
+ protected final ConcurrentMap<SkyKey, NodeEntry> nodeMap =
+ new MapMaker().initialCapacity(1024).concurrencyLevel(200).makeMap();
+ private final boolean keepEdges;
+
+ InMemoryGraph() {
+ this(/*keepEdges=*/true);
+ }
+
+ public InMemoryGraph(boolean keepEdges) {
+ this.keepEdges = keepEdges;
+ }
+
+ @Override
+ public void remove(SkyKey skyKey) {
+ nodeMap.remove(skyKey);
+ }
+
+ @Override
+ public NodeEntry get(SkyKey skyKey) {
+ return nodeMap.get(skyKey);
+ }
+
+ @Override
+ public NodeEntry createIfAbsent(SkyKey key) {
+ NodeEntry newval = keepEdges ? new NodeEntry() : new EdgelessNodeEntry();
+ NodeEntry oldval = nodeMap.putIfAbsent(key, newval);
+ return oldval == null ? newval : oldval;
+ }
+
+ /** Only done nodes exist to the outside world. */
+ private static final Predicate<NodeEntry> NODE_DONE_PREDICATE =
+ new Predicate<NodeEntry>() {
+ @Override
+ public boolean apply(NodeEntry entry) {
+ return entry != null && entry.isDone();
+ }
+ };
+
+ /**
+ * Returns a value, if it exists. If not, returns null.
+ */
+ @Nullable public SkyValue getValue(SkyKey key) {
+ NodeEntry entry = get(key);
+ return NODE_DONE_PREDICATE.apply(entry) ? entry.getValue() : null;
+ }
+
+ /**
+ * Returns a read-only live view of the nodes in the graph. All node are included. Dirty values
+ * include their Node value. Values in error have a null value.
+ */
+ Map<SkyKey, SkyValue> getValues() {
+ return Collections.unmodifiableMap(Maps.transformValues(
+ nodeMap,
+ new Function<NodeEntry, SkyValue>() {
+ @Override
+ public SkyValue apply(NodeEntry entry) {
+ return entry.toValue();
+ }
+ }));
+ }
+
+ /**
+ * Returns a read-only live view of the done values in the graph. Dirty, changed, and error values
+ * are not present in the returned map
+ */
+ Map<SkyKey, SkyValue> getDoneValues() {
+ return Collections.unmodifiableMap(Maps.filterValues(Maps.transformValues(
+ nodeMap,
+ new Function<NodeEntry, SkyValue>() {
+ @Override
+ public SkyValue apply(NodeEntry entry) {
+ return entry.isDone() ? entry.getValue() : null;
+ }
+ }), Predicates.notNull()));
+ }
+
+ // Only for use by MemoizingEvaluator#delete
+ Map<SkyKey, NodeEntry> getAllValues() {
+ return Collections.unmodifiableMap(nodeMap);
+ }
+
+ @VisibleForTesting
+ protected ConcurrentMap<SkyKey, NodeEntry> getNodeMap() {
+ return nodeMap;
+ }
+
+ boolean keepsEdges() {
+ return keepEdges;
+ }
+}
diff --git a/src/main/java/com/google/devtools/build/skyframe/InMemoryMemoizingEvaluator.java b/src/main/java/com/google/devtools/build/skyframe/InMemoryMemoizingEvaluator.java
new file mode 100644
index 0000000..827cc7b
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/InMemoryMemoizingEvaluator.java
@@ -0,0 +1,317 @@
+// Copyright 2014 Google Inc. 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.skyframe;
+
+import com.google.common.base.Function;
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Predicate;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+import com.google.devtools.build.lib.events.EventHandler;
+import com.google.devtools.build.skyframe.Differencer.Diff;
+import com.google.devtools.build.skyframe.InvalidatingNodeVisitor.DeletingInvalidationState;
+import com.google.devtools.build.skyframe.InvalidatingNodeVisitor.DirtyingInvalidationState;
+import com.google.devtools.build.skyframe.InvalidatingNodeVisitor.InvalidationState;
+import com.google.devtools.build.skyframe.NodeEntry.DependencyState;
+
+import java.io.PrintStream;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.LinkedHashSet;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import javax.annotation.Nullable;
+
+/**
+ * An inmemory implementation that uses the eager invalidation strategy. This class is, by itself,
+ * not thread-safe. Neither is it thread-safe to use this class in parallel with any of the
+ * returned graphs. However, it is allowed to access the graph from multiple threads as long as
+ * that does not happen in parallel with an {@link #evaluate} call.
+ *
+ * <p>This memoizing evaluator requires a sequential versioning scheme. Evaluations
+ * must pass in a monotonically increasing {@link IntVersion}.
+ */
+public final class InMemoryMemoizingEvaluator implements MemoizingEvaluator {
+
+ private final ImmutableMap<? extends SkyFunctionName, ? extends SkyFunction> skyFunctions;
+ @Nullable private final EvaluationProgressReceiver progressReceiver;
+ // Not final only for testing.
+ private InMemoryGraph graph;
+ private IntVersion lastGraphVersion = null;
+
+ // State related to invalidation and deletion.
+ private Set<SkyKey> valuesToDelete = new LinkedHashSet<>();
+ private Set<SkyKey> valuesToDirty = new LinkedHashSet<>();
+ private Map<SkyKey, SkyValue> valuesToInject = new HashMap<>();
+ private final DirtyKeyTracker dirtyKeyTracker = new DirtyKeyTrackerImpl();
+ private final InvalidationState deleterState = new DeletingInvalidationState();
+ private final Differencer differencer;
+
+ // Keep edges in graph. Can be false to save memory, in which case incremental builds are
+ // not possible.
+ private final boolean keepEdges;
+
+ // Values that the caller explicitly specified are assumed to be changed -- they will be
+ // re-evaluated even if none of their children are changed.
+ private final InvalidationState invalidatorState = new DirtyingInvalidationState();
+
+ private final EmittedEventState emittedEventState;
+
+ private final AtomicBoolean evaluating = new AtomicBoolean(false);
+
+ public InMemoryMemoizingEvaluator(
+ Map<? extends SkyFunctionName, ? extends SkyFunction> skyFunctions, Differencer differencer) {
+ this(skyFunctions, differencer, null);
+ }
+
+ public InMemoryMemoizingEvaluator(
+ Map<? extends SkyFunctionName, ? extends SkyFunction> skyFunctions, Differencer differencer,
+ @Nullable EvaluationProgressReceiver invalidationReceiver) {
+ this(skyFunctions, differencer, invalidationReceiver, new EmittedEventState(), true);
+ }
+
+ public InMemoryMemoizingEvaluator(
+ Map<? extends SkyFunctionName, ? extends SkyFunction> skyFunctions, Differencer differencer,
+ @Nullable EvaluationProgressReceiver invalidationReceiver,
+ EmittedEventState emittedEventState, boolean keepEdges) {
+ this.skyFunctions = ImmutableMap.copyOf(skyFunctions);
+ this.differencer = Preconditions.checkNotNull(differencer);
+ this.progressReceiver = invalidationReceiver;
+ this.graph = new InMemoryGraph(keepEdges);
+ this.emittedEventState = emittedEventState;
+ this.keepEdges = keepEdges;
+ }
+
+ private void invalidate(Iterable<SkyKey> diff) {
+ Iterables.addAll(valuesToDirty, diff);
+ }
+
+ @Override
+ public void delete(final Predicate<SkyKey> deletePredicate) {
+ valuesToDelete.addAll(
+ Maps.filterEntries(graph.getAllValues(), new Predicate<Entry<SkyKey, NodeEntry>>() {
+ @Override
+ public boolean apply(Entry<SkyKey, NodeEntry> input) {
+ return input.getValue().isDirty() || deletePredicate.apply(input.getKey());
+ }
+ }).keySet());
+ }
+
+ @Override
+ public void deleteDirty(long versionAgeLimit) {
+ Preconditions.checkArgument(versionAgeLimit >= 0);
+ final Version threshold = new IntVersion(lastGraphVersion.getVal() - versionAgeLimit);
+ valuesToDelete.addAll(
+ Sets.filter(dirtyKeyTracker.getDirtyKeys(), new Predicate<SkyKey>() {
+ @Override
+ public boolean apply(SkyKey skyKey) {
+ NodeEntry entry = graph.get(skyKey);
+ Preconditions.checkNotNull(entry, skyKey);
+ Preconditions.checkState(entry.isDirty(), skyKey);
+ return entry.getVersion().atMost(threshold);
+ }
+ }));
+ }
+
+ @Override
+ public <T extends SkyValue> EvaluationResult<T> evaluate(Iterable<SkyKey> roots, Version version,
+ boolean keepGoing, int numThreads, EventHandler eventHandler)
+ throws InterruptedException {
+ // NOTE: Performance critical code. See bug "Null build performance parity".
+ IntVersion intVersion = (IntVersion) version;
+ Preconditions.checkState((lastGraphVersion == null && intVersion.getVal() == 0)
+ || version.equals(lastGraphVersion.next()),
+ "InMemoryGraph supports only monotonically increasing Integer versions: %s %s",
+ lastGraphVersion, version);
+ setAndCheckEvaluateState(true, roots);
+ try {
+ // The RecordingDifferencer implementation is not quite working as it should be at this point.
+ // It clears the internal data structures after getDiff is called and will not return
+ // diffs for historical versions. This makes the following code sensitive to interrupts.
+ // Ideally we would simply not update lastGraphVersion if an interrupt occurs.
+ Diff diff = differencer.getDiff(lastGraphVersion, version);
+ valuesToInject.putAll(diff.changedKeysWithNewValues());
+ invalidate(diff.changedKeysWithoutNewValues());
+ pruneInjectedValues(valuesToInject);
+ invalidate(valuesToInject.keySet());
+
+ performInvalidation();
+ injectValues(intVersion);
+
+ ParallelEvaluator evaluator = new ParallelEvaluator(graph, intVersion,
+ skyFunctions, eventHandler, emittedEventState, keepGoing, numThreads, progressReceiver,
+ dirtyKeyTracker);
+ return evaluator.eval(roots);
+ } finally {
+ lastGraphVersion = intVersion;
+ setAndCheckEvaluateState(false, roots);
+ }
+ }
+
+ /**
+ * Removes entries in {@code valuesToInject} whose values are equal to the present values in the
+ * graph.
+ */
+ private void pruneInjectedValues(Map<SkyKey, SkyValue> valuesToInject) {
+ for (Iterator<Entry<SkyKey, SkyValue>> it = valuesToInject.entrySet().iterator();
+ it.hasNext();) {
+ Entry<SkyKey, SkyValue> entry = it.next();
+ SkyKey key = entry.getKey();
+ SkyValue newValue = entry.getValue();
+ NodeEntry prevEntry = graph.get(key);
+ if (prevEntry != null && prevEntry.isDone()) {
+ Iterable<SkyKey> directDeps = prevEntry.getDirectDeps();
+ Preconditions.checkState(Iterables.isEmpty(directDeps),
+ "existing entry for %s has deps: %s", key, directDeps);
+ if (newValue.equals(prevEntry.getValue())
+ && !valuesToDirty.contains(key) && !valuesToDelete.contains(key)) {
+ it.remove();
+ }
+ }
+ }
+ }
+
+ /**
+ * Injects values in {@code valuesToInject} into the graph.
+ */
+ private void injectValues(IntVersion version) {
+ if (valuesToInject.isEmpty()) {
+ return;
+ }
+ for (Entry<SkyKey, SkyValue> entry : valuesToInject.entrySet()) {
+ SkyKey key = entry.getKey();
+ SkyValue value = entry.getValue();
+ Preconditions.checkState(value != null, key);
+ NodeEntry prevEntry = graph.createIfAbsent(key);
+ if (prevEntry.isDirty()) {
+ // There was an existing entry for this key in the graph.
+ // Get the node in the state where it is able to accept a value.
+ Preconditions.checkState(prevEntry.getTemporaryDirectDeps().isEmpty(), key);
+
+ DependencyState newState = prevEntry.addReverseDepAndCheckIfDone(null);
+ Preconditions.checkState(newState == DependencyState.NEEDS_SCHEDULING, key);
+
+ // Check that the previous node has no dependencies. Overwriting a value with deps with an
+ // injected value (which is by definition deps-free) needs a little additional bookkeeping
+ // (removing reverse deps from the dependencies), but more importantly it's something that
+ // we want to avoid, because it indicates confusion of input values and derived values.
+ Preconditions.checkState(prevEntry.noDepsLastBuild(),
+ "existing entry for %s has deps: %s", key, prevEntry);
+ }
+ prevEntry.setValue(value, version);
+ // The evaluate method previously invalidated all keys in valuesToInject that survived the
+ // pruneInjectedValues call. Now that this key's injected value is set, it is no longer dirty.
+ dirtyKeyTracker.notDirty(key);
+ }
+ // Start with a new map to avoid bloat since clear() does not downsize the map.
+ valuesToInject = new HashMap<>();
+ }
+
+ private void performInvalidation() throws InterruptedException {
+ EagerInvalidator.delete(graph, valuesToDelete, progressReceiver, deleterState, keepEdges,
+ dirtyKeyTracker);
+ // Note that clearing the valuesToDelete would not do an internal resizing. Therefore, if any
+ // build has a large set of dirty values, subsequent operations (even clearing) will be slower.
+ // Instead, just start afresh with a new LinkedHashSet.
+ valuesToDelete = new LinkedHashSet<>();
+
+ EagerInvalidator.invalidate(graph, valuesToDirty, progressReceiver, invalidatorState,
+ dirtyKeyTracker);
+ // Ditto.
+ valuesToDirty = new LinkedHashSet<>();
+ }
+
+ private void setAndCheckEvaluateState(boolean newValue, Object requestInfo) {
+ Preconditions.checkState(evaluating.getAndSet(newValue) != newValue,
+ "Re-entrant evaluation for request: %s", requestInfo);
+ }
+
+ @Override
+ public Map<SkyKey, SkyValue> getValues() {
+ return graph.getValues();
+ }
+
+ @Override
+ public Map<SkyKey, SkyValue> getDoneValues() {
+ return graph.getDoneValues();
+ }
+
+ @Override
+ @Nullable public SkyValue getExistingValueForTesting(SkyKey key) {
+ return graph.getValue(key);
+ }
+
+ @Override
+ @Nullable public ErrorInfo getExistingErrorForTesting(SkyKey key) {
+ NodeEntry entry = graph.get(key);
+ return (entry == null || !entry.isDone()) ? null : entry.getErrorInfo();
+ }
+
+ public void setGraphForTesting(InMemoryGraph graph) {
+ this.graph = graph;
+ }
+
+ @Override
+ public void dump(boolean summarize, PrintStream out) {
+ if (summarize) {
+ long nodes = 0;
+ long edges = 0;
+ for (NodeEntry entry : graph.getAllValues().values()) {
+ nodes++;
+ if (entry.isDone()) {
+ edges += Iterables.size(entry.getDirectDeps());
+ }
+ }
+ out.println("Node count: " + nodes);
+ out.println("Edge count: " + edges);
+ } else {
+ Function<SkyKey, String> keyFormatter =
+ new Function<SkyKey, String>() {
+ @Override
+ public String apply(SkyKey key) {
+ return String.format("%s:%s",
+ key.functionName(), key.argument().toString().replace('\n', '_'));
+ }
+ };
+
+ for (Entry<SkyKey, NodeEntry> mapPair : graph.getAllValues().entrySet()) {
+ SkyKey key = mapPair.getKey();
+ NodeEntry entry = mapPair.getValue();
+ if (entry.isDone()) {
+ out.print(keyFormatter.apply(key));
+ out.print("|");
+ out.println(Joiner.on('|').join(
+ Iterables.transform(entry.getDirectDeps(), keyFormatter)));
+ }
+ }
+ }
+ }
+
+ public static final EvaluatorSupplier SUPPLIER = new EvaluatorSupplier() {
+ @Override
+ public MemoizingEvaluator create(
+ Map<? extends SkyFunctionName, ? extends SkyFunction> skyFunctions, Differencer differencer,
+ @Nullable EvaluationProgressReceiver invalidationReceiver,
+ EmittedEventState emittedEventState, boolean keepEdges) {
+ return new InMemoryMemoizingEvaluator(skyFunctions, differencer, invalidationReceiver,
+ emittedEventState, keepEdges);
+ }
+ };
+}
diff --git a/src/main/java/com/google/devtools/build/skyframe/Injectable.java b/src/main/java/com/google/devtools/build/skyframe/Injectable.java
new file mode 100644
index 0000000..5325df3
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/Injectable.java
@@ -0,0 +1,23 @@
+// Copyright 2014 Google Inc. 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.skyframe;
+
+import java.util.Map;
+
+/**
+ * An object that accepts Skyframe key / value mapping.
+ */
+public interface Injectable {
+ void inject(Map<SkyKey, ? extends SkyValue> values);
+}
diff --git a/src/main/java/com/google/devtools/build/skyframe/IntVersion.java b/src/main/java/com/google/devtools/build/skyframe/IntVersion.java
new file mode 100644
index 0000000..3d2a31d
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/IntVersion.java
@@ -0,0 +1,61 @@
+// Copyright 2014 Google Inc. 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.skyframe;
+
+/**
+ * Versioning scheme based on integers.
+ */
+public final class IntVersion implements Version {
+
+ private final long val;
+
+ public IntVersion(long val) {
+ this.val = val;
+ }
+
+ public long getVal() {
+ return val;
+ }
+
+ public IntVersion next() {
+ return new IntVersion(val + 1);
+ }
+
+ @Override
+ public boolean atMost(Version other) {
+ if (!(other instanceof IntVersion)) {
+ return false;
+ }
+ return val <= ((IntVersion) other).val;
+ }
+
+ @Override
+ public int hashCode() {
+ return Long.valueOf(val).hashCode();
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj instanceof IntVersion) {
+ IntVersion other = (IntVersion) obj;
+ return other.val == val;
+ }
+ return false;
+ }
+
+ @Override
+ public String toString() {
+ return "IntVersion: " + val;
+ }
+}
diff --git a/src/main/java/com/google/devtools/build/skyframe/InvalidatingNodeVisitor.java b/src/main/java/com/google/devtools/build/skyframe/InvalidatingNodeVisitor.java
new file mode 100644
index 0000000..7abf6c6
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/InvalidatingNodeVisitor.java
@@ -0,0 +1,350 @@
+// Copyright 2014 Google Inc. 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.skyframe;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Function;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Sets;
+import com.google.devtools.build.lib.concurrent.AbstractQueueVisitor;
+import com.google.devtools.build.lib.concurrent.ThreadSafety.ThreadSafe;
+import com.google.devtools.build.lib.util.Pair;
+
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+import javax.annotation.Nullable;
+
+/**
+ * A visitor that is useful for invalidating transitive dependencies of Skyframe nodes.
+ *
+ * <p>Interruptibility: It is safe to interrupt the invalidation process at any time. Consider a
+ * graph and a set of modified nodes. Then the reverse transitive closure of the modified nodes is
+ * the set of dirty nodes. We provide interruptibility by making sure that the following invariant
+ * holds at any time:
+ *
+ * <p>If a node is dirty, but not removed (or marked as dirty) yet, then either it or any of its
+ * transitive dependencies must be in the {@link #pendingVisitations} set. Furthermore, reverse dep
+ * pointers must always point to existing nodes.
+ *
+ * <p>Thread-safety: This class should only be instantiated and called on a single thread, but
+ * internally it spawns many worker threads to process the graph. The thread-safety of the workers
+ * on the graph can be delicate, and is documented below. Moreover, no other modifications to the
+ * graph can take place while invalidation occurs.
+ *
+ * <p>This is intended only for use in alternative {@code MemoizingEvaluator} implementations.
+ */
+public abstract class InvalidatingNodeVisitor extends AbstractQueueVisitor {
+
+ // Default thread count is equal to the number of cores to exploit
+ // that level of hardware parallelism, since invalidation should be CPU-bound.
+ // We may consider increasing this in the future.
+ private static final int DEFAULT_THREAD_COUNT = Runtime.getRuntime().availableProcessors();
+
+ private static final boolean MUST_EXIST = true;
+
+ protected final DirtiableGraph graph;
+ @Nullable protected final EvaluationProgressReceiver invalidationReceiver;
+ protected final DirtyKeyTracker dirtyKeyTracker;
+ // Aliased to InvalidationState.pendingVisitations.
+ protected final Set<Pair<SkyKey, InvalidationType>> pendingVisitations;
+
+ protected InvalidatingNodeVisitor(
+ DirtiableGraph graph, @Nullable EvaluationProgressReceiver invalidationReceiver,
+ InvalidationState state, DirtyKeyTracker dirtyKeyTracker) {
+ super(/*concurrent*/true,
+ /*corePoolSize*/DEFAULT_THREAD_COUNT,
+ /*maxPoolSize*/DEFAULT_THREAD_COUNT,
+ 1, TimeUnit.SECONDS,
+ /*failFastOnException*/true,
+ /*failFastOnInterrupt*/true,
+ "skyframe-invalidator");
+ this.graph = Preconditions.checkNotNull(graph);
+ this.invalidationReceiver = invalidationReceiver;
+ this.dirtyKeyTracker = Preconditions.checkNotNull(dirtyKeyTracker);
+ this.pendingVisitations = state.pendingValues;
+ }
+
+ /**
+ * Initiates visitation and waits for completion.
+ */
+ void run() throws InterruptedException {
+ // Make a copy to avoid concurrent modification confusing us as to which nodes were passed by
+ // the caller, and which are added by other threads during the run. Since no tasks have been
+ // started yet (the queueDirtying calls start them), this is thread-safe.
+ for (Pair<SkyKey, InvalidationType> visitData : ImmutableList.copyOf(pendingVisitations)) {
+ // The caller may have specified non-existent SkyKeys, or there may be stale SkyKeys in
+ // pendingVisitations that have already been deleted. In both these cases, the nodes will not
+ // exist in the graph, so we must be tolerant of that case.
+ visit(visitData.first, visitData.second, !MUST_EXIST);
+ }
+ work(/*failFastOnInterrupt=*/true);
+ Preconditions.checkState(pendingVisitations.isEmpty(),
+ "All dirty nodes should have been processed: %s", pendingVisitations);
+ }
+
+ protected void informInvalidationReceiver(SkyValue value,
+ EvaluationProgressReceiver.InvalidationState state) {
+ if (invalidationReceiver != null && value != null) {
+ invalidationReceiver.invalidated(value, state);
+ }
+ }
+
+ /**
+ * Enqueues a node for invalidation.
+ */
+ @ThreadSafe
+ abstract void visit(SkyKey key, InvalidationType second, boolean mustExist);
+
+ @VisibleForTesting
+ enum InvalidationType {
+ /**
+ * The node is dirty and must be recomputed.
+ */
+ CHANGED,
+ /**
+ * The node is dirty, but may be marked clean later during change pruning.
+ */
+ DIRTIED,
+ /**
+ * The node is deleted.
+ */
+ DELETED;
+ }
+
+ /**
+ * Invalidation state object that keeps track of which nodes need to be invalidated, but have not
+ * been dirtied/deleted yet. This supports interrupts - by only deleting a node from this set
+ * when all its parents have been invalidated, we ensure that no information is lost when an
+ * interrupt comes in.
+ */
+ static class InvalidationState {
+ private final Set<Pair<SkyKey, InvalidationType>> pendingValues = Sets.newConcurrentHashSet();
+ private final InvalidationType defaultUpdateType;
+
+ private InvalidationState(InvalidationType defaultUpdateType) {
+ this.defaultUpdateType = Preconditions.checkNotNull(defaultUpdateType);
+ }
+
+ void update(Iterable<SkyKey> diff) {
+ Iterables.addAll(pendingValues, Iterables.transform(diff,
+ new Function<SkyKey, Pair<SkyKey, InvalidationType>>() {
+ @Override
+ public Pair<SkyKey, InvalidationType> apply(SkyKey skyKey) {
+ return Pair.of(skyKey, defaultUpdateType);
+ }
+ }));
+ }
+
+ @VisibleForTesting
+ boolean isEmpty() {
+ return pendingValues.isEmpty();
+ }
+
+ @VisibleForTesting
+ Set<Pair<SkyKey, InvalidationType>> getInvalidationsForTesting() {
+ return ImmutableSet.copyOf(pendingValues);
+ }
+ }
+
+ public static class DirtyingInvalidationState extends InvalidationState {
+ public DirtyingInvalidationState() {
+ super(InvalidationType.CHANGED);
+ }
+ }
+
+ static class DeletingInvalidationState extends InvalidationState {
+ public DeletingInvalidationState() {
+ super(InvalidationType.DELETED);
+ }
+ }
+
+ /**
+ * A node-deleting implementation.
+ */
+ static class DeletingNodeVisitor extends InvalidatingNodeVisitor {
+
+ private final Set<SkyKey> visitedValues = Sets.newConcurrentHashSet();
+ private final boolean traverseGraph;
+
+ protected DeletingNodeVisitor(DirtiableGraph graph,
+ EvaluationProgressReceiver invalidationReceiver, InvalidationState state,
+ boolean traverseGraph, DirtyKeyTracker dirtyKeyTracker) {
+ super(graph, invalidationReceiver, state, dirtyKeyTracker);
+ this.traverseGraph = traverseGraph;
+ }
+
+ @Override
+ public void visit(final SkyKey key, InvalidationType invalidationType, boolean mustExist) {
+ Preconditions.checkState(invalidationType == InvalidationType.DELETED, key);
+ if (!visitedValues.add(key)) {
+ return;
+ }
+ final Pair<SkyKey, InvalidationType> invalidationPair = Pair.of(key, invalidationType);
+ pendingVisitations.add(invalidationPair);
+ enqueue(new Runnable() {
+ @Override
+ public void run() {
+ NodeEntry entry = graph.get(key);
+ if (entry == null) {
+ pendingVisitations.remove(invalidationPair);
+ return;
+ }
+
+ if (traverseGraph) {
+ // Propagate deletion upwards.
+ for (SkyKey reverseDep : entry.getReverseDeps()) {
+ visit(reverseDep, InvalidationType.DELETED, !MUST_EXIST);
+ }
+ }
+
+ if (entry.isDone()) {
+ // Only process this node's value and children if it is done, since dirty nodes have
+ // no awareness of either.
+
+ // Unregister this node from direct deps, since reverse dep edges cannot point to
+ // non-existent nodes.
+ if (traverseGraph) {
+ for (SkyKey directDep : entry.getDirectDeps()) {
+ NodeEntry dep = graph.get(directDep);
+ if (dep != null) {
+ dep.removeReverseDep(key);
+ }
+ }
+ }
+ // Allow custom Value-specific logic to update dirtiness status.
+ informInvalidationReceiver(entry.getValue(),
+ EvaluationProgressReceiver.InvalidationState.DELETED);
+ }
+ if (traverseGraph) {
+ // Force reverseDeps consolidation (validates that attempts to remove reverse deps were
+ // really successful.
+ entry.getReverseDeps();
+ }
+ // Actually remove the node.
+ graph.remove(key);
+ dirtyKeyTracker.notDirty(key);
+
+ // Remove the node from the set as the last operation.
+ pendingVisitations.remove(invalidationPair);
+ }
+ });
+ }
+ }
+
+ /**
+ * A node-dirtying implementation.
+ */
+ static class DirtyingNodeVisitor extends InvalidatingNodeVisitor {
+
+ private final Set<Pair<SkyKey, InvalidationType>> visited = Sets.newConcurrentHashSet();
+
+ protected DirtyingNodeVisitor(DirtiableGraph graph,
+ EvaluationProgressReceiver invalidationReceiver, InvalidationState state,
+ DirtyKeyTracker dirtyKeyTracker) {
+ super(graph, invalidationReceiver, state, dirtyKeyTracker);
+ }
+
+ /**
+ * Queues a task to dirty the node named by {@code key}. May be called from multiple threads.
+ * It is possible that the same node is enqueued many times. However, we require that a node
+ * is only actually marked dirty/changed once, with two exceptions:
+ *
+ * (1) If a node is marked dirty, it can subsequently be marked changed. This can occur if, for
+ * instance, FileValue workspace/foo/foo.cc is marked dirty because FileValue workspace/foo is
+ * marked changed (and every FileValue depends on its parent). Then FileValue
+ * workspace/foo/foo.cc is itself changed (this can even happen on the same build).
+ *
+ * (2) If a node is going to be marked both dirty and changed, as, for example, in the previous
+ * case if both workspace/foo/foo.cc and workspace/foo have been changed in the same build, the
+ * thread marking workspace/foo/foo.cc dirty may race with the one marking it changed, and so
+ * try to mark it dirty after it has already been marked changed. In that case, the
+ * {@link NodeEntry} ignores the second marking.
+ *
+ * The invariant that we do not process a (SkyKey, InvalidationType) pair twice is enforced by
+ * the {@link #visited} set.
+ *
+ * The "invariant" is also enforced across builds by checking to see if the entry is already
+ * marked changed, or if it is already marked dirty and we are just going to mark it dirty
+ * again.
+ *
+ * If either of the above tests shows that we have already started a task to mark this entry
+ * dirty/changed, or that it is already marked dirty/changed, we do not continue this task.
+ */
+ @Override
+ @ThreadSafe
+ public void visit(final SkyKey key, final InvalidationType invalidationType,
+ final boolean mustExist) {
+ Preconditions.checkState(invalidationType != InvalidationType.DELETED, key);
+ final boolean isChanged = (invalidationType == InvalidationType.CHANGED);
+ final Pair<SkyKey, InvalidationType> invalidationPair = Pair.of(key, invalidationType);
+ if (!visited.add(invalidationPair)) {
+ return;
+ }
+ pendingVisitations.add(invalidationPair);
+ enqueue(new Runnable() {
+ @Override
+ public void run() {
+ NodeEntry entry = graph.get(key);
+
+ if (entry == null) {
+ Preconditions.checkState(!mustExist,
+ "%s does not exist in the graph but was enqueued for dirtying by another node",
+ key);
+ pendingVisitations.remove(invalidationPair);
+ return;
+ }
+
+ if (entry.isChanged() || (!isChanged && entry.isDirty())) {
+ // If this node is already marked changed, or we are only marking this node dirty, and
+ // it already is, move along.
+ pendingVisitations.remove(invalidationPair);
+ return;
+ }
+
+ // This entry remains in the graph in this dirty state until it is re-evaluated.
+ Pair<? extends Iterable<SkyKey>, ? extends SkyValue> depsAndValue =
+ entry.markDirty(isChanged);
+ // It is not safe to interrupt the logic from this point until the end of the method.
+ // Any exception thrown should be unrecoverable.
+ if (depsAndValue == null) {
+ // Another thread has already dirtied this node. Don't do anything in this thread.
+ pendingVisitations.remove(invalidationPair);
+ return;
+ }
+ // Propagate dirtiness upwards and mark this node dirty/changed. Reverse deps should only
+ // be marked dirty (because only a dependency of theirs has changed).
+ for (SkyKey reverseDep : entry.getReverseDeps()) {
+ visit(reverseDep, InvalidationType.DIRTIED, MUST_EXIST);
+ }
+
+ // Remove this node as a reverse dep from its children, since we have reset it and it no
+ // longer lists its children as direct deps.
+ for (SkyKey dep : depsAndValue.first) {
+ graph.get(dep).removeReverseDep(key);
+ }
+
+ SkyValue value = ValueWithMetadata.justValue(depsAndValue.second);
+ informInvalidationReceiver(value, EvaluationProgressReceiver.InvalidationState.DIRTY);
+ dirtyKeyTracker.dirty(key);
+ // Remove the node from the set as the last operation.
+ pendingVisitations.remove(invalidationPair);
+ }
+ });
+ }
+ }
+}
diff --git a/src/main/java/com/google/devtools/build/skyframe/MemoizingEvaluator.java b/src/main/java/com/google/devtools/build/skyframe/MemoizingEvaluator.java
new file mode 100644
index 0000000..2c7f14e
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/MemoizingEvaluator.java
@@ -0,0 +1,143 @@
+// Copyright 2014 Google Inc. 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.skyframe;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Predicate;
+import com.google.devtools.build.lib.collect.nestedset.NestedSetVisitor;
+import com.google.devtools.build.lib.concurrent.ThreadSafety.ThreadHostile;
+import com.google.devtools.build.lib.events.EventHandler;
+
+import java.io.PrintStream;
+import java.util.Map;
+
+import javax.annotation.Nullable;
+
+/**
+ * A graph, defined by a set of functions that can construct values from value keys.
+ *
+ * <p>The value constructor functions ({@link SkyFunction}s) can declare dependencies on
+ * prerequisite {@link SkyValue}s. The {@link MemoizingEvaluator} implementation makes sure that
+ * those are created beforehand.
+ *
+ * <p>The graph caches previously computed value values. Arbitrary values can be invalidated between
+ * calls to {@link #evaluate}; they will be recreated the next time they are requested.
+ */
+public interface MemoizingEvaluator {
+
+ /**
+ * Computes the transitive closure of a given set of values at the given {@link Version}. See
+ * {@link EagerInvalidator#invalidate}.
+ *
+ * <p>The returned EvaluationResult is guaranteed to contain a result for at least one root if
+ * keepGoing is false. It will contain a result for every root if keepGoing is true, <i>unless</i>
+ * the evaluation failed with a "catastrophic" error. In that case, some or all results may be
+ * missing.
+ */
+ <T extends SkyValue> EvaluationResult<T> evaluate(
+ Iterable<SkyKey> roots,
+ Version version,
+ boolean keepGoing,
+ int numThreads,
+ EventHandler reporter)
+ throws InterruptedException;
+
+ /**
+ * Ensures that after the next completed {@link #evaluate} call the current values of any value
+ * matching this predicate (and all values that transitively depend on them) will be removed from
+ * the value cache. All values that were already marked dirty in the graph will also be deleted,
+ * regardless of whether or not they match the predicate.
+ *
+ * <p>If a later call to {@link #evaluate} requests some of the deleted values, those values will
+ * be recomputed and the new values stored in the cache again.
+ *
+ * <p>To delete all dirty values, you can specify a predicate that's always false.
+ */
+ void delete(Predicate<SkyKey> pred);
+
+ /**
+ * Marks dirty values for deletion if they have been dirty for at least as many graph versions
+ * as the specified limit.
+ *
+ * <p>This ensures that after the next completed {@link #evaluate} call, all such values, along
+ * with all values that transitively depend on them, will be removed from the value cache. Values
+ * that were marked dirty after the threshold version will not be affected by this call.
+ *
+ * <p>If a later call to {@link #evaluate} requests some of the deleted values, those values will
+ * be recomputed and the new values stored in the cache again.
+ *
+ * <p>To delete all dirty values, you can specify 0 for the limit.
+ */
+ void deleteDirty(long versionAgeLimit);
+
+ /**
+ * Returns the values in the graph.
+ *
+ * <p>The returned map may be a live view of the graph.
+ */
+ Map<SkyKey, SkyValue> getValues();
+
+
+ /**
+ * Returns the done (without error) values in the graph.
+ *
+ * <p>The returned map may be a live view of the graph.
+ */
+ Map<SkyKey, SkyValue> getDoneValues();
+
+ /**
+ * Returns a value if and only if an earlier call to {@link #evaluate} created it; null otherwise.
+ *
+ * <p>This method should only be used by tests that need to verify the presence of a value in the
+ * graph after an {@link #evaluate} call.
+ */
+ @VisibleForTesting
+ @Nullable
+ SkyValue getExistingValueForTesting(SkyKey key);
+
+ /**
+ * Returns an error if and only if an earlier call to {@link #evaluate} created it; null
+ * otherwise.
+ *
+ * <p>This method should only be used by tests that need to verify the presence of an error in the
+ * graph after an {@link #evaluate} call.
+ */
+ @VisibleForTesting
+ @Nullable
+ ErrorInfo getExistingErrorForTesting(SkyKey key);
+
+ /**
+ * Write the graph to the output stream. Not necessarily thread-safe. Use only for debugging
+ * purposes.
+ */
+ @ThreadHostile
+ void dump(boolean summarize, PrintStream out);
+
+ /**
+ * A supplier for creating instances of a particular evaluator implementation.
+ */
+ public static interface EvaluatorSupplier {
+ MemoizingEvaluator create(
+ Map<? extends SkyFunctionName, ? extends SkyFunction> skyFunctions, Differencer differencer,
+ @Nullable EvaluationProgressReceiver invalidationReceiver,
+ EmittedEventState emittedEventState, boolean keepEdges);
+ }
+
+ /**
+ * Keeps track of already-emitted events. Users of the graph should instantiate an
+ * {@code EmittedEventState} first and pass it to the graph during creation. This allows them to
+ * determine whether or not to replay events.
+ */
+ public static class EmittedEventState extends NestedSetVisitor.VisitedState<TaggedEvents> {}
+}
diff --git a/src/main/java/com/google/devtools/build/skyframe/MinimalVersion.java b/src/main/java/com/google/devtools/build/skyframe/MinimalVersion.java
new file mode 100644
index 0000000..6f75c15
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/MinimalVersion.java
@@ -0,0 +1,31 @@
+// Copyright 2014 Google Inc. 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.skyframe;
+
+/**
+ * A Version "less than" all other versions, other than itself.
+ *
+ * <p>Only use in custom evaluator implementations.
+ */
+public class MinimalVersion implements Version {
+ public static final MinimalVersion INSTANCE = new MinimalVersion();
+
+ private MinimalVersion() {
+ }
+
+ @Override
+ public boolean atMost(Version other) {
+ return true;
+ }
+}
diff --git a/src/main/java/com/google/devtools/build/skyframe/NodeEntry.java b/src/main/java/com/google/devtools/build/skyframe/NodeEntry.java
new file mode 100644
index 0000000..243189d
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/NodeEntry.java
@@ -0,0 +1,581 @@
+// Copyright 2014 Google Inc. 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.skyframe;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import com.google.devtools.build.lib.util.GroupedList;
+import com.google.devtools.build.lib.util.GroupedList.GroupedListHelper;
+import com.google.devtools.build.lib.util.Pair;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Set;
+
+import javax.annotation.Nullable;
+
+/**
+ * A node in the graph. All operations on this class are thread-safe. Care was taken to provide
+ * certain compound operations to avoid certain check-then-act races. That means this class is
+ * somewhat closely tied to the exact Evaluator implementation.
+ *
+ * <p>Consider the example with two threads working on two nodes, where one depends on the other,
+ * say b depends on a. If a completes first, it's done. If it completes second, it needs to signal
+ * b, and potentially re-schedule it. If b completes first, it must exit, because it will be
+ * signaled (and re-scheduled) by a. If it completes second, it must signal (and re-schedule)
+ * itself. However, if the Evaluator supported re-entrancy for a node, then this wouldn't have to
+ * be so strict, because duplicate scheduling would be less problematic.
+ *
+ * <p>The transient state of a {@code NodeEntry} is kept in a {@link BuildingState} object. Many of
+ * the methods of {@code NodeEntry} are just wrappers around the corresponding
+ * {@link BuildingState} methods.
+ *
+ * <p>This class is non-final only for testing purposes.
+ * <p>This class is public only for the benefit of alternative graph implementations outside of the
+ * package.
+ */
+public class NodeEntry {
+ /**
+ * Return code for {@link #addReverseDepAndCheckIfDone(SkyKey)}.
+ */
+ enum DependencyState {
+ /** The node is done. */
+ DONE,
+
+ /**
+ * The node was just created and needs to be scheduled for its first evaluation pass. The
+ * evaluator is responsible for signaling the reverse dependency node.
+ */
+ NEEDS_SCHEDULING,
+
+ /**
+ * The node was already created, but isn't done yet. The evaluator is responsible for
+ * signaling the reverse dependency node.
+ */
+ ADDED_DEP;
+ }
+
+ /** Actual data stored in this entry when it is done. */
+ private SkyValue value = null;
+
+ /**
+ * The last version of the graph at which this node entry was changed. In {@link #setValue} it
+ * may be determined that the data being written to the graph at a given version is the same as
+ * the already-stored data. In that case, the version will remain the same. The version can be
+ * thought of as the latest timestamp at which this entry was changed.
+ */
+ private Version version = MinimalVersion.INSTANCE;
+
+ /**
+ * This object represents a {@link GroupedList}<SkyKey> in a memory-efficient way. It stores the
+ * direct dependencies of this node, in groups if the {@code SkyFunction} requested them that way.
+ */
+ private Object directDeps = null;
+
+ /**
+ * This list stores the reverse dependencies of this node that have been declared so far.
+ *
+ * <p>In case of a single object we store the object unwrapped, without the list, for
+ * memory-efficiency.
+ */
+ @VisibleForTesting
+ protected Object reverseDeps = ImmutableList.of();
+
+ /**
+ * We take advantage of memory alignment to avoid doing a nasty {@code instanceof} for knowing
+ * if {@code reverseDeps} is a single object or a list.
+ */
+ protected boolean reverseDepIsSingleObject = false;
+
+ /**
+ * During the invalidation we keep the reverse deps to be removed in this list instead of directly
+ * removing them from {@code reverseDeps}. That is because removals from reverseDeps are O(N).
+ * Originally reverseDeps was a HashSet, but because of memory consumption we switched to a list.
+ *
+ * <p>This requires that any usage of reverseDeps (contains, add, the list of reverse deps) call
+ * {@code consolidateReverseDepsRemovals} first. While this operation is not free, it can be done
+ * more effectively than trying to remove each dirty reverse dependency individually (O(N) each
+ * time).
+ */
+ private List<SkyKey> reverseDepsToRemove = null;
+
+ private static final ReverseDepsUtil<NodeEntry> REVERSE_DEPS_UTIL =
+ new ReverseDepsUtil<NodeEntry>() {
+ @Override
+ void setReverseDepsObject(NodeEntry container, Object object) {
+ container.reverseDeps = object;
+ }
+
+ @Override
+ void setSingleReverseDep(NodeEntry container, boolean singleObject) {
+ container.reverseDepIsSingleObject = singleObject;
+ }
+
+ @Override
+ void setReverseDepsToRemove(NodeEntry container, List<SkyKey> object) {
+ container.reverseDepsToRemove = object;
+ }
+
+ @Override
+ Object getReverseDepsObject(NodeEntry container) {
+ return container.reverseDeps;
+ }
+
+ @Override
+ boolean isSingleReverseDep(NodeEntry container) {
+ return container.reverseDepIsSingleObject;
+ }
+
+ @Override
+ List<SkyKey> getReverseDepsToRemove(NodeEntry container) {
+ return container.reverseDepsToRemove;
+ }
+ };
+
+ /**
+ * The transient state of this entry, after it has been created but before it is done. It allows
+ * us to keep the current state of the entry across invalidation and successive evaluations.
+ */
+ @VisibleForTesting
+ protected BuildingState buildingState = new BuildingState();
+
+ /**
+ * Construct a NodeEntry. Use ONLY in Skyframe evaluation and graph implementations.
+ */
+ public NodeEntry() {
+ }
+
+ protected boolean keepEdges() {
+ return true;
+ }
+
+ /** Returns whether the entry has been built and is finished evaluating. */
+ synchronized boolean isDone() {
+ return buildingState == null;
+ }
+
+ /**
+ * Returns the value stored in this entry. This method may only be called after the evaluation of
+ * this node is complete, i.e., after {@link #setValue} has been called.
+ */
+ synchronized SkyValue getValue() {
+ Preconditions.checkState(isDone(), "no value until done. ValueEntry: %s", this);
+ return ValueWithMetadata.justValue(value);
+ }
+
+ /**
+ * Returns the {@link SkyValue} for this entry and the metadata associated with it (Like events
+ * and errors). This method may only be called after the evaluation of this node is complete,
+ * i.e., after {@link #setValue} has been called.
+ */
+ synchronized ValueWithMetadata getValueWithMetadata() {
+ Preconditions.checkState(isDone(), "no value until done: %s", this);
+ return ValueWithMetadata.wrapWithMetadata(value);
+ }
+
+ /**
+ * Returns the value, even if dirty or changed. Returns null otherwise.
+ */
+ public synchronized SkyValue toValue() {
+ if (isDone()) {
+ return getErrorInfo() == null ? getValue() : null;
+ } else if (isChanged() || isDirty()) {
+ return (buildingState.getLastBuildValue() == null)
+ ? null
+ : ValueWithMetadata.justValue(buildingState.getLastBuildValue());
+ }
+ throw new AssertionError("Value in bad state: " + this);
+ }
+
+ /**
+ * Returns an immutable iterable of the direct deps of this node. This method may only be called
+ * after the evaluation of this node is complete, i.e., after {@link #setValue} has been called.
+ *
+ * <p>This method is not very efficient, but is only be called in limited circumstances --
+ * when the node is about to be deleted, or when the node is expected to have no direct deps (in
+ * which case the overhead is not so bad). It should not be called repeatedly for the same node,
+ * since each call takes time proportional to the number of direct deps of the node.
+ */
+ synchronized Iterable<SkyKey> getDirectDeps() {
+ assertKeepEdges();
+ Preconditions.checkState(isDone(), "no deps until done. ValueEntry: %s", this);
+ return GroupedList.<SkyKey>create(directDeps).toSet();
+ }
+
+ /**
+ * Returns the error, if any, associated to this node. This method may only be called after
+ * the evaluation of this node is complete, i.e., after {@link #setValue} has been called.
+ */
+ @Nullable
+ synchronized ErrorInfo getErrorInfo() {
+ Preconditions.checkState(isDone(), "no errors until done. ValueEntry: %s", this);
+ return ValueWithMetadata.getMaybeErrorInfo(value);
+ }
+
+ private synchronized Set<SkyKey> setStateFinishedAndReturnReverseDeps() {
+ // Get reverse deps that need to be signaled.
+ ImmutableSet<SkyKey> reverseDepsToSignal = buildingState.getReverseDepsToSignal();
+ REVERSE_DEPS_UTIL.consolidateReverseDepsRemovals(this);
+ REVERSE_DEPS_UTIL.addReverseDeps(this, reverseDepsToSignal);
+ this.directDeps = buildingState.getFinishedDirectDeps().compress();
+
+ // Set state of entry to done.
+ buildingState = null;
+
+ if (!keepEdges()) {
+ this.directDeps = null;
+ this.reverseDeps = null;
+ }
+ return reverseDepsToSignal;
+ }
+
+ /**
+ * Returns the set of reverse deps that have been declared so far this build. Only for use in
+ * debugging and when bubbling errors up in the --nokeep_going case, where we need to know what
+ * parents this entry has.
+ */
+ synchronized Set<SkyKey> getInProgressReverseDeps() {
+ Preconditions.checkState(!isDone(), this);
+ return buildingState.getReverseDepsToSignal();
+ }
+
+ /**
+ * Transitions the node from the EVALUATING to the DONE state and simultaneously sets it to the
+ * given value and error state. It then returns the set of reverse dependencies that need to be
+ * signaled.
+ *
+ * <p>This is an atomic operation to avoid a race where two threads work on two nodes, where one
+ * node depends on another (b depends on a). When a finishes, it signals <b>exactly</b> the set
+ * of reverse dependencies that are registered at the time of the {@code setValue} call. If b
+ * comes in before a, it is signaled (and re-scheduled) by a, otherwise it needs to do that
+ * itself.
+ *
+ * <p>{@code version} indicates the graph version at which this node is being written. If the
+ * entry determines that the new value is equal to the previous value, the entry will keep its
+ * current version. Callers can query that version to see if the node considers its value to have
+ * changed.
+ */
+ public synchronized Set<SkyKey> setValue(SkyValue value, Version version) {
+ Preconditions.checkState(isReady(), "%s %s", this, value);
+ // This check may need to be removed when we move to a non-linear versioning sequence.
+ Preconditions.checkState(this.version.atMost(version),
+ "%s %s %s", this, version, value);
+
+ if (isDirty() && buildingState.unchangedFromLastBuild(value)) {
+ // If the value is the same as before, just use the old value. Note that we don't use the new
+ // value, because preserving == equality is even better than .equals() equality.
+ this.value = buildingState.getLastBuildValue();
+ } else {
+ // If this is a new value, or it has changed since the last build, set the version to the
+ // current graph version.
+ this.version = version;
+ this.value = value;
+ }
+
+ return setStateFinishedAndReturnReverseDeps();
+ }
+
+ /**
+ * Queries if the node is done and adds the given key as a reverse dependency. The return code
+ * indicates whether a) the node is done, b) the reverse dependency is the first one, so the
+ * node needs to be scheduled, or c) the reverse dependency was added, and the node does not
+ * need to be scheduled.
+ *
+ * <p>This method <b>must</b> be called before any processing of the entry. This encourages
+ * callers to check that the entry is ready to be processed.
+ *
+ * <p>Adding the dependency and checking if the node needs to be scheduled is an atomic operation
+ * to avoid a race where two threads work on two nodes, where one depends on the other (b depends
+ * on a). In that case, we need to ensure that b is re-scheduled exactly once when a is done.
+ * However, a may complete first, in which case b has to re-schedule itself. Also see {@link
+ * #setValue}.
+ *
+ * <p>If the parameter is {@code null}, then no reverse dependency is added, but we still check
+ * if the node needs to be scheduled.
+ */
+ synchronized DependencyState addReverseDepAndCheckIfDone(SkyKey reverseDep) {
+ if (reverseDep != null) {
+ if (keepEdges()) {
+ REVERSE_DEPS_UTIL.consolidateReverseDepsRemovals(this);
+ REVERSE_DEPS_UTIL.maybeCheckReverseDepNotPresent(this, reverseDep);
+ }
+ if (isDone()) {
+ if (keepEdges()) {
+ REVERSE_DEPS_UTIL.addReverseDeps(this, ImmutableList.of(reverseDep));
+ }
+ } else {
+ // Parent should never register itself twice in the same build.
+ buildingState.addReverseDepToSignal(reverseDep);
+ }
+ }
+ if (isDone()) {
+ return DependencyState.DONE;
+ }
+ return buildingState.startEvaluating() ? DependencyState.NEEDS_SCHEDULING
+ : DependencyState.ADDED_DEP;
+ }
+
+ /**
+ * Removes a reverse dependency.
+ */
+ synchronized void removeReverseDep(SkyKey reverseDep) {
+ if (!keepEdges()) {
+ return;
+ }
+ REVERSE_DEPS_UTIL.removeReverseDep(this, reverseDep);
+ if (!isDone()) {
+ // This is currently unnecessary -- the only time we remove a reverse dep that was added this
+ // build is during the clean following a build failure. In that case, this node that is not
+ // done will be deleted soon, so clearing the reverse dep is not required.
+ buildingState.removeReverseDepToSignal(reverseDep);
+ }
+ }
+
+ /**
+ * Returns a copy of the set of reverse dependencies. Note that this introduces a potential
+ * check-then-act race; {@link #removeReverseDep} may fail for a key that is returned here.
+ */
+ synchronized Iterable<SkyKey> getReverseDeps() {
+ assertKeepEdges();
+ Preconditions.checkState(isDone() || buildingState.getReverseDepsToSignal().isEmpty(),
+ "Reverse deps should only be queried before the build has begun "
+ + "or after the node is done %s", this);
+ return REVERSE_DEPS_UTIL.getReverseDeps(this);
+ }
+
+ /**
+ * Tell this node that one of its dependencies is now done. Callers must check the return value,
+ * and if true, they must re-schedule this node for evaluation. Equivalent to
+ * {@code #signalDep(Long.MAX_VALUE)}. Since this entry's version is less than
+ * {@link Long#MAX_VALUE}, informing this entry that a child of it has version
+ * {@link Long#MAX_VALUE} will force it to re-evaluate.
+ */
+ synchronized boolean signalDep() {
+ return signalDep(/*childVersion=*/new IntVersion(Long.MAX_VALUE));
+ }
+
+ /**
+ * Tell this entry that one of its dependencies is now done. Callers must check the return value,
+ * and if true, they must re-schedule this node for evaluation.
+ *
+ * @param childVersion If this entry {@link #isDirty()} and {@code childVersion} is not at most
+ * {@link #getVersion()}, then this entry records that one of its children has changed since it
+ * was last evaluated (namely, it was last evaluated at version {@link #getVersion()} and the
+ * child was last evaluated at {@code childVersion}. Thus, the next call to
+ * {@link #getDirtyState()} will return {@link BuildingState.DirtyState#REBUILDING}.
+ */
+ synchronized boolean signalDep(Version childVersion) {
+ Preconditions.checkState(!isDone(), "Value must not be done in signalDep %s", this);
+ return buildingState.signalDep(/*childChanged=*/!childVersion.atMost(getVersion()));
+ }
+
+ /**
+ * Returns true if the entry is marked dirty, meaning that at least one of its transitive
+ * dependencies is marked changed.
+ */
+ public synchronized boolean isDirty() {
+ return !isDone() && buildingState.isDirty();
+ }
+
+ /**
+ * Returns true if the entry is marked changed, meaning that it must be re-evaluated even if its
+ * dependencies' values have not changed.
+ */
+ synchronized boolean isChanged() {
+ return !isDone() && buildingState.isChanged();
+ }
+
+ /** Checks that a caller is not trying to access not-stored graph edges. */
+ private void assertKeepEdges() {
+ Preconditions.checkState(keepEdges(), "Graph edges not stored. %s", this);
+ }
+
+ /**
+ * Marks this node dirty, or changed if {@code isChanged} is true. The node is put in the
+ * just-created state. It will be re-evaluated if necessary during the evaluation phase,
+ * but if it has not changed, it will not force a re-evaluation of its parents.
+ *
+ * @return The direct deps and value of this entry, or null if the entry has already been marked
+ * dirty. In the latter case, the caller should abort its handling of this node, since another
+ * thread is already dealing with it.
+ */
+ @Nullable
+ synchronized Pair<? extends Iterable<SkyKey>, ? extends SkyValue> markDirty(boolean isChanged) {
+ assertKeepEdges();
+ if (isDone()) {
+ GroupedList<SkyKey> lastDirectDeps = GroupedList.create(directDeps);
+ buildingState = BuildingState.newDirtyState(isChanged, lastDirectDeps, value);
+ Pair<? extends Iterable<SkyKey>, ? extends SkyValue> result =
+ Pair.of(lastDirectDeps.toSet(), value);
+ value = null;
+ directDeps = null;
+ return result;
+ }
+ // The caller may be simultaneously trying to mark this node dirty and changed, and the dirty
+ // thread may have lost the race, but it is the caller's responsibility not to try to mark
+ // this node changed twice. The end result of racing markers must be a changed node, since one
+ // of the markers is trying to mark the node changed.
+ Preconditions.checkState(isChanged != isChanged(),
+ "Cannot mark node dirty twice or changed twice: %s", this);
+ Preconditions.checkState(value == null, "Value should have been reset already %s", this);
+ Preconditions.checkState(directDeps == null, "direct deps not already reset %s", this);
+ if (isChanged) {
+ // If the changed marker lost the race, we just need to mark changed in this method -- all
+ // other work was done by the dirty marker.
+ buildingState.markChanged();
+ }
+ return null;
+ }
+
+ /**
+ * Marks this entry as up-to-date at this version.
+ *
+ * @return {@link Set} of reverse dependencies to signal that this node is done.
+ */
+ synchronized Set<SkyKey> markClean() {
+ this.value = buildingState.getLastBuildValue();
+ // This checks both the value and the direct deps, but since we're passing in the same value,
+ // the value check should be trivial.
+ Preconditions.checkState(buildingState.unchangedFromLastBuild(this.value),
+ "Direct deps must be the same as those found last build for node to be marked clean: %s",
+ this);
+ Preconditions.checkState(isDirty(), this);
+ Preconditions.checkState(!buildingState.isChanged(), "shouldn't be changed: %s", this);
+ return setStateFinishedAndReturnReverseDeps();
+ }
+
+ /**
+ * Forces this node to be reevaluated, even if none of its dependencies are known to have
+ * changed.
+ *
+ * <p>Used when an external caller has reason to believe that re-evaluating may yield a new
+ * result. This method should not be used if one of the normal deps of this node has changed,
+ * the usual change-pruning process should work in that case.
+ */
+ synchronized void forceRebuild() {
+ buildingState.forceChanged();
+ }
+
+ /**
+ * Gets the current version of this entry.
+ */
+ synchronized Version getVersion() {
+ return version;
+ }
+
+ /**
+ * Gets the current state of checking this dirty entry to see if it must be re-evaluated. Must be
+ * called each time evaluation of a dirty entry starts to find the proper action to perform next,
+ * as enumerated by {@link BuildingState.DirtyState}.
+ *
+ * @see BuildingState#getDirtyState()
+ */
+ synchronized BuildingState.DirtyState getDirtyState() {
+ return buildingState.getDirtyState();
+ }
+
+ /**
+ * Should only be called if the entry is dirty. During the examination to see if the entry must be
+ * re-evaluated, this method returns the next group of children to be checked. Callers should
+ * have already called {@link #getDirtyState} and received a return value of
+ * {@link BuildingState.DirtyState#CHECK_DEPENDENCIES} before calling this method -- any other
+ * return value from {@link #getDirtyState} means that this method must not be called, since
+ * whether or not the node needs to be rebuilt is already known.
+ *
+ * <p>Deps are returned in groups. The deps in each group were requested in parallel by the
+ * {@code SkyFunction} last build, meaning independently of the values of any other deps in this
+ * group (although possibly depending on deps in earlier groups). Thus the caller may check all
+ * the deps in this group in parallel, since the deps in all previous groups are verified
+ * unchanged. See {@link SkyFunction.Environment#getValues} for more on dependency groups.
+ *
+ * <p>The caller should register these as deps of this entry using {@link #addTemporaryDirectDeps}
+ * before checking them.
+ *
+ * @see BuildingState#getNextDirtyDirectDeps()
+ */
+ synchronized Collection<SkyKey> getNextDirtyDirectDeps() {
+ return buildingState.getNextDirtyDirectDeps();
+ }
+
+ /**
+ * Returns the set of direct dependencies. This may only be called while the node is being
+ * evaluated, that is, before {@link #setValue} and after {@link #markDirty}.
+ */
+ synchronized Set<SkyKey> getTemporaryDirectDeps() {
+ Preconditions.checkState(!isDone(), "temporary shouldn't be done: %s", this);
+ return buildingState.getDirectDepsForBuild();
+ }
+
+ synchronized boolean noDepsLastBuild() {
+ return buildingState.noDepsLastBuild();
+ }
+
+ /**
+ * Remove dep from direct deps. This should only be called if this entry is about to be
+ * committed as a cycle node, but some of its children were not checked for cycles, either
+ * because the cycle was discovered before some children were checked; some children didn't have a
+ * chance to finish before the evaluator aborted; or too many cycles were found when it came time
+ * to check the children.
+ */
+ synchronized void removeUnfinishedDeps(Set<SkyKey> unfinishedDeps) {
+ buildingState.removeDirectDeps(unfinishedDeps);
+ }
+
+ synchronized void addTemporaryDirectDeps(GroupedListHelper<SkyKey> helper) {
+ Preconditions.checkState(!isDone(), "add temp shouldn't be done: %s %s", helper, this);
+ buildingState.addDirectDeps(helper);
+ }
+
+ /**
+ * Returns true if the node is ready to be evaluated, i.e., it has been signaled exactly as many
+ * times as it has temporary dependencies. This may only be called while the node is being
+ * evaluated, that is, before {@link #setValue} and after {@link #markDirty}.
+ */
+ synchronized boolean isReady() {
+ Preconditions.checkState(!isDone(), "can't be ready if done: %s", this);
+ return buildingState.isReady();
+ }
+
+ @Override
+ @SuppressWarnings("deprecation")
+ public String toString() {
+ return Objects.toStringHelper(this) // MoreObjects is not in Guava
+ .add("value", value)
+ .add("version", version)
+ .add("directDeps", directDeps == null ? null : GroupedList.create(directDeps))
+ .add("reverseDeps", REVERSE_DEPS_UTIL.toString(this))
+ .add("buildingState", buildingState).toString();
+ }
+
+ /**
+ * Do not use except in custom evaluator implementations! Added only temporarily.
+ *
+ * <p>Clones a NodeEntry iff it is a done node. Otherwise it fails.
+ */
+ @Deprecated
+ public synchronized NodeEntry cloneNodeEntry() {
+ // As this is temporary, for now lets limit to done nodes
+ Preconditions.checkState(isDone(), "Only done nodes can be copied");
+ NodeEntry nodeEntry = new NodeEntry();
+ nodeEntry.value = value;
+ nodeEntry.version = this.version;
+ REVERSE_DEPS_UTIL.addReverseDeps(nodeEntry, REVERSE_DEPS_UTIL.getReverseDeps(this));
+ nodeEntry.directDeps = directDeps;
+ nodeEntry.buildingState = null;
+ return nodeEntry;
+ }
+}
diff --git a/src/main/java/com/google/devtools/build/skyframe/NullDirtyKeyTrackerImpl.java b/src/main/java/com/google/devtools/build/skyframe/NullDirtyKeyTrackerImpl.java
new file mode 100644
index 0000000..937f1cb
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/NullDirtyKeyTrackerImpl.java
@@ -0,0 +1,37 @@
+// Copyright 2014 Google Inc. 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.skyframe;
+
+import com.google.common.collect.ImmutableSet;
+
+import java.util.Set;
+
+/**
+ * Tracks nothing. Should be used by evaluators that don't do dirty node garbage collection.
+ */
+public class NullDirtyKeyTrackerImpl implements DirtyKeyTracker {
+
+ @Override
+ public void dirty(SkyKey skyKey) {
+ }
+
+ @Override
+ public void notDirty(SkyKey skyKey) {
+ }
+
+ @Override
+ public Set<SkyKey> getDirtyKeys() {
+ return ImmutableSet.of();
+ }
+}
diff --git a/src/main/java/com/google/devtools/build/skyframe/ParallelEvaluator.java b/src/main/java/com/google/devtools/build/skyframe/ParallelEvaluator.java
new file mode 100644
index 0000000..39f11d7
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/ParallelEvaluator.java
@@ -0,0 +1,1786 @@
+// Copyright 2014 Google Inc. 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.skyframe;
+
+import com.google.common.base.Function;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Predicate;
+import com.google.common.base.Predicates;
+import com.google.common.base.Throwables;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Interner;
+import com.google.common.collect.Interners;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import com.google.devtools.build.lib.collect.nestedset.NestedSet;
+import com.google.devtools.build.lib.collect.nestedset.NestedSetBuilder;
+import com.google.devtools.build.lib.collect.nestedset.NestedSetVisitor;
+import com.google.devtools.build.lib.concurrent.AbstractQueueVisitor;
+import com.google.devtools.build.lib.concurrent.ExecutorShutdownUtil;
+import com.google.devtools.build.lib.concurrent.ThreadSafety.ThreadCompatible;
+import com.google.devtools.build.lib.concurrent.ThrowableRecordingRunnableWrapper;
+import com.google.devtools.build.lib.events.Event;
+import com.google.devtools.build.lib.events.EventHandler;
+import com.google.devtools.build.lib.events.StoredEventHandler;
+import com.google.devtools.build.lib.profiler.Profiler;
+import com.google.devtools.build.lib.profiler.ProfilerTask;
+import com.google.devtools.build.lib.util.GroupedList.GroupedListHelper;
+import com.google.devtools.build.skyframe.BuildingState.DirtyState;
+import com.google.devtools.build.skyframe.EvaluationProgressReceiver.EvaluationState;
+import com.google.devtools.build.skyframe.NodeEntry.DependencyState;
+import com.google.devtools.build.skyframe.Scheduler.SchedulerException;
+import com.google.devtools.build.skyframe.SkyFunctionException.ReifiedSkyFunctionException;
+import com.google.devtools.build.skyframe.ValueOrExceptionUtils.BottomException;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import javax.annotation.Nullable;
+
+/**
+ * Evaluates a set of given functions ({@code SkyFunction}s) with arguments ({@code SkyKey}s).
+ * Cycles are not allowed and are detected during the traversal.
+ *
+ * <p>This class implements multi-threaded evaluation. This is a fairly complex process that has
+ * strong consistency requirements between the {@link ProcessableGraph}, the nodes in the graph of
+ * type {@link NodeEntry}, the work queue, and the set of in-flight nodes.
+ *
+ * <p>The basic invariants are:
+ *
+ * <p>A node can be in one of three states: ready, waiting, and done. A node is ready if and only
+ * if all of its dependencies have been signaled. A node is done if it has a value. It is waiting
+ * if not all of its dependencies have been signaled.
+ *
+ * <p>A node must be in the work queue if and only if it is ready. It is an error for a node to be
+ * in the work queue twice at the same time.
+ *
+ * <p>A node is considered in-flight if it has been created, and is not done yet. In case of an
+ * interrupt, the work queue is discarded, and the in-flight set is used to remove partially
+ * computed values.
+ *
+ * <p>Each evaluation of the graph takes place at a "version," which is currently given by a
+ * non-negative {@code long}. The version can also be thought of as an "mtime." Each node in the
+ * graph has a version, which is the last version at which its value changed. This version data is
+ * used to avoid unnecessary re-evaluation of values. If a node is re-evaluated and found to have
+ * the same data as before, its version (mtime) remains the same. If all of a node's children's
+ * have the same version as before, its re-evaluation can be skipped.
+ *
+ * <p>This class is not intended for direct use, and is only exposed as public for use in
+ * evaluation implementations outside of this package.
+ */
+public final class ParallelEvaluator implements Evaluator {
+ private final ProcessableGraph graph;
+ private final Version graphVersion;
+
+ private final Predicate<SkyKey> nodeEntryIsDone = new Predicate<SkyKey>() {
+ @Override
+ public boolean apply(SkyKey skyKey) {
+ return isDoneForBuild(graph.get(skyKey));
+ }
+ };
+
+ private final ImmutableMap<? extends SkyFunctionName, ? extends SkyFunction> skyFunctions;
+
+ private final EventHandler reporter;
+ private final NestedSetVisitor<TaggedEvents> replayingNestedSetEventVisitor;
+ private final boolean keepGoing;
+ private final int threadCount;
+ @Nullable private final EvaluationProgressReceiver progressReceiver;
+ private final DirtyKeyTracker dirtyKeyTracker;
+ private final AtomicBoolean errorEncountered = new AtomicBoolean(false);
+
+ private static final Interner<SkyKey> KEY_CANONICALIZER = Interners.newWeakInterner();
+
+ public ParallelEvaluator(ProcessableGraph graph, Version graphVersion,
+ ImmutableMap<? extends SkyFunctionName, ? extends SkyFunction> skyFunctions,
+ final EventHandler reporter,
+ MemoizingEvaluator.EmittedEventState emittedEventState,
+ boolean keepGoing, int threadCount,
+ @Nullable EvaluationProgressReceiver progressReceiver,
+ DirtyKeyTracker dirtyKeyTracker) {
+ this.graph = graph;
+ this.skyFunctions = skyFunctions;
+ this.graphVersion = graphVersion;
+ this.reporter = Preconditions.checkNotNull(reporter);
+ this.keepGoing = keepGoing;
+ this.threadCount = threadCount;
+ this.progressReceiver = progressReceiver;
+ this.dirtyKeyTracker = Preconditions.checkNotNull(dirtyKeyTracker);
+ this.replayingNestedSetEventVisitor =
+ new NestedSetVisitor<>(new NestedSetEventReceiver(reporter), emittedEventState);
+ }
+
+ /**
+ * Receives the events from the NestedSet and delegates to the reporter.
+ */
+ private static class NestedSetEventReceiver implements NestedSetVisitor.Receiver<TaggedEvents> {
+
+ private final EventHandler reporter;
+
+ public NestedSetEventReceiver(EventHandler reporter) {
+ this.reporter = reporter;
+ }
+ @Override
+ public void accept(TaggedEvents events) {
+ String tag = events.getTag();
+ for (Event e : events.getEvents()) {
+ reporter.handle(e.withTag(tag));
+ }
+ }
+ }
+
+ /**
+ * A suitable {@link SkyFunction.Environment} implementation.
+ */
+ class SkyFunctionEnvironment implements SkyFunction.Environment {
+ private boolean building = true;
+ private boolean valuesMissing = false;
+ private SkyKey depErrorKey = null;
+ private final SkyKey skyKey;
+ private SkyValue value = null;
+ private ErrorInfo errorInfo = null;
+ private final Map<SkyKey, ValueWithMetadata> bubbleErrorInfo;
+ /** The set of values previously declared as dependencies. */
+ private final Set<SkyKey> directDeps;
+
+ /**
+ * The grouped list of values requested during this build as dependencies. On a subsequent
+ * build, if this value is dirty, all deps in the same dependency group can be checked in
+ * parallel for changes. In other words, if dep1 and dep2 are in the same group, then dep1 will
+ * be checked in parallel with dep2. See {@link #getValues} for more.
+ */
+ private final GroupedListHelper<SkyKey> newlyRequestedDeps = new GroupedListHelper<>();
+
+ /**
+ * The value visitor managing the thread pool. Used to enqueue parents when this value is
+ * finished, and, during testing, to block until an exception is thrown if a value builder
+ * requests that.
+ */
+ private final ValueVisitor visitor;
+
+ /** The set of errors encountered while fetching children. */
+ private final Collection<ErrorInfo> childErrorInfos = new LinkedHashSet<>();
+ private final StoredEventHandler eventHandler = new StoredEventHandler() {
+ @Override
+ public void handle(Event e) {
+ checkActive();
+ switch (e.getKind()) {
+ case INFO:
+ throw new UnsupportedOperationException("Values should not display INFO messages: " +
+ skyKey + " printed " + e.getLocation() + ": " + e.getMessage());
+ case PROGRESS:
+ reporter.handle(e);
+ break;
+ default:
+ super.handle(e);
+ }
+ }
+ };
+
+ private SkyFunctionEnvironment(SkyKey skyKey, Set<SkyKey> directDeps, ValueVisitor visitor) {
+ this(skyKey, directDeps, null, visitor);
+ }
+
+ private SkyFunctionEnvironment(SkyKey skyKey, Set<SkyKey> directDeps,
+ @Nullable Map<SkyKey, ValueWithMetadata> bubbleErrorInfo, ValueVisitor visitor) {
+ this.skyKey = skyKey;
+ this.directDeps = Collections.unmodifiableSet(directDeps);
+ this.bubbleErrorInfo = bubbleErrorInfo;
+ this.childErrorInfos.addAll(childErrorInfos);
+ this.visitor = visitor;
+ }
+
+ private void checkActive() {
+ Preconditions.checkState(building, skyKey);
+ }
+
+ private NestedSet<TaggedEvents> buildEvents(boolean missingChildren) {
+ // Aggregate the nested set of events from the direct deps, also adding the events from
+ // building this value.
+ NestedSetBuilder<TaggedEvents> eventBuilder = NestedSetBuilder.stableOrder();
+ ImmutableList<Event> events = eventHandler.getEvents();
+ if (!events.isEmpty()) {
+ eventBuilder.add(new TaggedEvents(getTagFromKey(), events));
+ }
+ for (SkyKey dep : graph.get(skyKey).getTemporaryDirectDeps()) {
+ ValueWithMetadata value = getValueMaybeFromError(dep, bubbleErrorInfo);
+ if (value != null) {
+ eventBuilder.addTransitive(value.getTransitiveEvents());
+ } else {
+ Preconditions.checkState(missingChildren, "", dep, skyKey);
+ }
+ }
+ return eventBuilder.build();
+ }
+
+ /**
+ * If this node has an error, that is, if errorInfo is non-null, do nothing. Otherwise, set
+ * errorInfo to the union of the child errors that were recorded earlier by getValueOrException,
+ * if there are any.
+ */
+ private void finalizeErrorInfo() {
+ if (errorInfo == null && !childErrorInfos.isEmpty()) {
+ errorInfo = new ErrorInfo(skyKey, childErrorInfos);
+ }
+ }
+
+ private void setValue(SkyValue newValue) {
+ Preconditions.checkState(errorInfo == null && bubbleErrorInfo == null,
+ "%s %s %s %s", skyKey, newValue, errorInfo, bubbleErrorInfo);
+ Preconditions.checkState(value == null, "%s %s %s", skyKey, value, newValue);
+ value = newValue;
+ }
+
+ /**
+ * Set this node to be in error. The node's value must not have already been set. However, all
+ * dependencies of this node <i>must</i> already have been registered, since this method may
+ * register a dependence on the error transience node, which should always be the last dep.
+ */
+ private void setError(ErrorInfo errorInfo) {
+ Preconditions.checkState(value == null, "%s %s %s", skyKey, value, errorInfo);
+ Preconditions.checkState(this.errorInfo == null,
+ "%s %s %s", skyKey, this.errorInfo, errorInfo);
+
+ if (errorInfo.isTransient()) {
+ DependencyState triState =
+ graph.get(ErrorTransienceValue.key()).addReverseDepAndCheckIfDone(skyKey);
+ Preconditions.checkState(triState == DependencyState.DONE,
+ "%s %s %s", skyKey, triState, errorInfo);
+
+ final NodeEntry state = graph.get(skyKey);
+ state.addTemporaryDirectDeps(
+ GroupedListHelper.create(ImmutableList.of(ErrorTransienceValue.key())));
+ state.signalDep();
+ }
+
+ this.errorInfo = Preconditions.checkNotNull(errorInfo, skyKey);
+ }
+
+ /** Get a child of the value being evaluated, for use by the value builder. */
+ private ValueOrUntypedException getValueOrUntypedException(SkyKey depKey) {
+ checkActive();
+ depKey = KEY_CANONICALIZER.intern(depKey); // Canonicalize SkyKeys to save memory.
+ ValueWithMetadata value = getValueMaybeFromError(depKey, bubbleErrorInfo);
+ if (value == null) {
+ // If this entry is not yet done then (optionally) record the missing dependency and return
+ // null.
+ valuesMissing = true;
+ if (bubbleErrorInfo != null) {
+ // Values being built just for their errors don't get to request new children.
+ return ValueOrExceptionUtils.ofNull();
+ }
+ Preconditions.checkState(!directDeps.contains(depKey), "%s %s %s", skyKey, depKey, value);
+ addDep(depKey);
+ valuesMissing = true;
+ return ValueOrExceptionUtils.ofNull();
+ }
+
+ if (!directDeps.contains(depKey)) {
+ // If this child is done, we will return it, but also record that it was newly requested so
+ // that the dependency can be properly registered in the graph.
+ addDep(depKey);
+ }
+
+ replayingNestedSetEventVisitor.visit(value.getTransitiveEvents());
+ ErrorInfo errorInfo = value.getErrorInfo();
+
+ if (errorInfo != null) {
+ childErrorInfos.add(errorInfo);
+ }
+
+ if (value.getValue() != null && (keepGoing || errorInfo == null)) {
+ // The caller is given the value of the value if there was no error computing the value, or
+ // if this is a keepGoing build (in which case each value should get child values even if
+ // there are also errors).
+ return ValueOrExceptionUtils.ofValueUntyped(value.getValue());
+ }
+
+ // There was an error building the value, which we will either report by throwing an exception
+ // or insulate the caller from by returning null.
+ Preconditions.checkNotNull(errorInfo, "%s %s %s", skyKey, depKey, value);
+
+ if (!keepGoing && errorInfo.getException() != null && bubbleErrorInfo == null) {
+ // Child errors should not be propagated in noKeepGoing mode (except during error bubbling).
+ // Instead we should fail fast.
+
+ // We arbitrarily record the first child error.
+ if (depErrorKey == null) {
+ depErrorKey = depKey;
+ }
+ valuesMissing = true;
+ return ValueOrExceptionUtils.ofNull();
+ }
+
+ if (bubbleErrorInfo != null) {
+ // Set interrupted status, so that builder doesn't try anything fancy after this.
+ Thread.currentThread().interrupt();
+ }
+ if (errorInfo.getException() != null) {
+ // Give builder a chance to handle this exception.
+ Exception e = errorInfo.getException();
+ return ValueOrExceptionUtils.ofExn(e);
+ }
+ // In a cycle.
+ Preconditions.checkState(!Iterables.isEmpty(errorInfo.getCycleInfo()), "%s %s %s %s", skyKey,
+ depKey, errorInfo, value);
+ valuesMissing = true;
+ return ValueOrExceptionUtils.ofNull();
+ }
+
+ private <E extends Exception> ValueOrException<E> getValueOrException(SkyKey depKey,
+ Class<E> exceptionClass) {
+ return ValueOrExceptionUtils.downcovert(getValueOrException(depKey, exceptionClass,
+ BottomException.class), exceptionClass);
+ }
+
+ private <E1 extends Exception, E2 extends Exception> ValueOrException2<E1, E2>
+ getValueOrException(SkyKey depKey, Class<E1> exceptionClass1, Class<E2> exceptionClass2) {
+ return ValueOrExceptionUtils.downconvert(getValueOrException(depKey, exceptionClass1,
+ exceptionClass2, BottomException.class), exceptionClass1, exceptionClass2);
+ }
+
+ private <E1 extends Exception, E2 extends Exception, E3 extends Exception>
+ ValueOrException3<E1, E2, E3> getValueOrException(SkyKey depKey, Class<E1> exceptionClass1,
+ Class<E2> exceptionClass2, Class<E3> exceptionClass3) {
+ return ValueOrExceptionUtils.downconvert(getValueOrException(depKey, exceptionClass1,
+ exceptionClass2, exceptionClass3, BottomException.class), exceptionClass1,
+ exceptionClass2, exceptionClass3);
+ }
+
+ private <E1 extends Exception, E2 extends Exception, E3 extends Exception,
+ E4 extends Exception> ValueOrException4<E1, E2, E3, E4> getValueOrException(SkyKey depKey,
+ Class<E1> exceptionClass1, Class<E2> exceptionClass2, Class<E3> exceptionClass3,
+ Class<E4> exceptionClass4) {
+ SkyFunctionException.validateExceptionType(exceptionClass1);
+ SkyFunctionException.validateExceptionType(exceptionClass2);
+ SkyFunctionException.validateExceptionType(exceptionClass3);
+ SkyFunctionException.validateExceptionType(exceptionClass4);
+ ValueOrUntypedException voe = getValueOrUntypedException(depKey);
+ SkyValue value = voe.getValue();
+ if (value != null) {
+ return ValueOrExceptionUtils.ofValue(value);
+ }
+ Exception e = voe.getException();
+ if (e != null) {
+ if (exceptionClass1.isInstance(e)) {
+ return ValueOrExceptionUtils.ofExn1(exceptionClass1.cast(e));
+ }
+ if (exceptionClass2.isInstance(e)) {
+ return ValueOrExceptionUtils.ofExn2(exceptionClass2.cast(e));
+ }
+ if (exceptionClass3.isInstance(e)) {
+ return ValueOrExceptionUtils.ofExn3(exceptionClass3.cast(e));
+ }
+ if (exceptionClass4.isInstance(e)) {
+ return ValueOrExceptionUtils.ofExn4(exceptionClass4.cast(e));
+ }
+ }
+ valuesMissing = true;
+ return ValueOrExceptionUtils.ofNullValue();
+ }
+
+ @Override
+ @Nullable
+ public SkyValue getValue(SkyKey depKey) {
+ try {
+ return getValueOrThrow(depKey, BottomException.class);
+ } catch (BottomException e) {
+ throw new IllegalStateException("shouldn't reach here");
+ }
+ }
+
+ @Override
+ @Nullable
+ public <E extends Exception> SkyValue getValueOrThrow(SkyKey depKey, Class<E> exceptionClass)
+ throws E {
+ return getValueOrException(depKey, exceptionClass).get();
+ }
+
+ @Override
+ @Nullable
+ public <E1 extends Exception, E2 extends Exception> SkyValue getValueOrThrow(SkyKey depKey,
+ Class<E1> exceptionClass1, Class<E2> exceptionClass2) throws E1, E2 {
+ return getValueOrException(depKey, exceptionClass1, exceptionClass2).get();
+ }
+
+ @Override
+ @Nullable
+ public <E1 extends Exception, E2 extends Exception,
+ E3 extends Exception> SkyValue getValueOrThrow(SkyKey depKey, Class<E1> exceptionClass1,
+ Class<E2> exceptionClass2, Class<E3> exceptionClass3) throws E1, E2, E3 {
+ return getValueOrException(depKey, exceptionClass1, exceptionClass2, exceptionClass3).get();
+ }
+
+ @Override
+ public <E1 extends Exception, E2 extends Exception, E3 extends Exception,
+ E4 extends Exception> SkyValue getValueOrThrow(SkyKey depKey, Class<E1> exceptionClass1,
+ Class<E2> exceptionClass2, Class<E3> exceptionClass3, Class<E4> exceptionClass4) throws E1,
+ E2, E3, E4 {
+ return getValueOrException(depKey, exceptionClass1, exceptionClass2, exceptionClass3,
+ exceptionClass4).get();
+ }
+
+ @Override
+ public Map<SkyKey, SkyValue> getValues(Iterable<SkyKey> depKeys) {
+ return Maps.transformValues(getValuesOrThrow(depKeys, BottomException.class),
+ GET_VALUE_FROM_VOE);
+ }
+
+ @Override
+ public <E extends Exception> Map<SkyKey, ValueOrException<E>> getValuesOrThrow(
+ Iterable<SkyKey> depKeys, Class<E> exceptionClass) {
+ return Maps.transformValues(getValuesOrThrow(depKeys, exceptionClass, BottomException.class),
+ makeSafeDowncastToVOEFunction(exceptionClass));
+ }
+
+ @Override
+ public <E1 extends Exception,
+ E2 extends Exception> Map<SkyKey, ValueOrException2<E1, E2>> getValuesOrThrow(
+ Iterable<SkyKey> depKeys, Class<E1> exceptionClass1, Class<E2> exceptionClass2) {
+ return Maps.transformValues(getValuesOrThrow(depKeys, exceptionClass1, exceptionClass2,
+ BottomException.class), makeSafeDowncastToVOE2Function(exceptionClass1,
+ exceptionClass2));
+ }
+
+ @Override
+ public <E1 extends Exception, E2 extends Exception, E3 extends Exception> Map<SkyKey,
+ ValueOrException3<E1, E2, E3>> getValuesOrThrow(Iterable<SkyKey> depKeys,
+ Class<E1> exceptionClass1, Class<E2> exceptionClass2, Class<E3> exceptionClass3) {
+ return Maps.transformValues(getValuesOrThrow(depKeys, exceptionClass1, exceptionClass2,
+ exceptionClass3, BottomException.class), makeSafeDowncastToVOE3Function(exceptionClass1,
+ exceptionClass2, exceptionClass3));
+ }
+
+ @Override
+ public <E1 extends Exception, E2 extends Exception, E3 extends Exception,
+ E4 extends Exception> Map<SkyKey, ValueOrException4<E1, E2, E3, E4>> getValuesOrThrow(
+ Iterable<SkyKey> depKeys, Class<E1> exceptionClass1, Class<E2> exceptionClass2,
+ Class<E3> exceptionClass3, Class<E4> exceptionClass4) {
+ Map<SkyKey, ValueOrException4<E1, E2, E3, E4>> result = new HashMap<>();
+ newlyRequestedDeps.startGroup();
+ for (SkyKey depKey : depKeys) {
+ if (result.containsKey(depKey)) {
+ continue;
+ }
+ result.put(depKey, getValueOrException(depKey, exceptionClass1, exceptionClass2,
+ exceptionClass3, exceptionClass4));
+ }
+ newlyRequestedDeps.endGroup();
+ return Collections.unmodifiableMap(result);
+ }
+
+ private void addDep(SkyKey key) {
+ if (!newlyRequestedDeps.contains(key)) {
+ // dep may have been requested already this evaluation. If not, add it.
+ newlyRequestedDeps.add(key);
+ }
+ }
+
+ @Override
+ public boolean valuesMissing() {
+ return valuesMissing;
+ }
+
+ /**
+ * If {@code !keepGoing} and there is at least one dep in error, returns a dep in error.
+ * Otherwise returns {@code null}.
+ */
+ @Nullable
+ private SkyKey getDepErrorKey() {
+ return depErrorKey;
+ }
+
+ @Override
+ public EventHandler getListener() {
+ checkActive();
+ return eventHandler;
+ }
+
+ private void doneBuilding() {
+ building = false;
+ }
+
+ /**
+ * Apply the change to the graph (mostly) atomically and signal all nodes that are waiting for
+ * this node to complete. Adding nodes and signaling is not atomic, but may need to be changed
+ * for interruptibility.
+ *
+ * <p>Parents are only enqueued if {@code enqueueParents} holds. Parents should be enqueued
+ * unless (1) this node is being built after the main evaluation has aborted, or (2) this node
+ * is being built with --nokeep_going, and so we are about to shut down the main evaluation
+ * anyway.
+ *
+ * <p>The node entry is informed if the node's value and error are definitive via the flag
+ * {@code completeValue}.
+ */
+ void commit(boolean enqueueParents) {
+ NodeEntry primaryEntry = Preconditions.checkNotNull(graph.get(skyKey), skyKey);
+ // Construct the definitive error info, if there is one.
+ finalizeErrorInfo();
+
+ // We have the following implications:
+ // errorInfo == null => value != null => enqueueParents.
+ // All these implications are strict:
+ // (1) errorInfo != null && value != null happens for values with recoverable errors.
+ // (2) value == null && enqueueParents happens for values that are found to have errors
+ // during a --keep_going build.
+
+ NestedSet<TaggedEvents> events = buildEvents(/*missingChildren=*/false);
+ if (value == null) {
+ Preconditions.checkNotNull(errorInfo, "%s %s", skyKey, primaryEntry);
+ // We could consider using max(childVersions) here instead of graphVersion. When full
+ // versioning is implemented, this would allow evaluation at a version between
+ // max(childVersions) and graphVersion to re-use this result.
+ Set<SkyKey> reverseDeps = primaryEntry.setValue(
+ ValueWithMetadata.error(errorInfo, events), graphVersion);
+ signalValuesAndEnqueueIfReady(enqueueParents ? visitor : null, reverseDeps, graphVersion);
+ } else {
+ // We must be enqueueing parents if we have a value.
+ Preconditions.checkState(enqueueParents, "%s %s", skyKey, primaryEntry);
+ Set<SkyKey> reverseDeps;
+ Version valueVersion;
+ // If this entry is dirty, setValue may not actually change it, if it determines that
+ // the data being written now is the same as the data already present in the entry.
+ // We could consider using max(childVersions) here instead of graphVersion. When full
+ // versioning is implemented, this would allow evaluation at a version between
+ // max(childVersions) and graphVersion to re-use this result.
+ reverseDeps = primaryEntry.setValue(
+ ValueWithMetadata.normal(value, errorInfo, events), graphVersion);
+ // Note that if this update didn't actually change the value entry, this version may not
+ // be the graph version.
+ valueVersion = primaryEntry.getVersion();
+ Preconditions.checkState(valueVersion.atMost(graphVersion),
+ "%s should be at most %s in the version partial ordering",
+ valueVersion, graphVersion);
+ if (progressReceiver != null) {
+ // Tell the receiver that this value was built. If valueVersion.equals(graphVersion), it
+ // was evaluated this run, and so was changed. Otherwise, it is less than graphVersion,
+ // by the Preconditions check above, and was not actually changed this run -- when it was
+ // written above, its version stayed below this update's version, so its value remains the
+ // same as before.
+ progressReceiver.evaluated(skyKey, value,
+ valueVersion.equals(graphVersion) ? EvaluationState.BUILT : EvaluationState.CLEAN);
+ }
+ signalValuesAndEnqueueIfReady(visitor, reverseDeps, valueVersion);
+ }
+
+ visitor.notifyDone(skyKey);
+ replayingNestedSetEventVisitor.visit(events);
+ }
+
+ @Nullable
+ private String getTagFromKey() {
+ return skyFunctions.get(skyKey.functionName()).extractTag(skyKey);
+ }
+
+ /**
+ * Gets the latch that is counted down when an exception is thrown in {@code
+ * AbstractQueueVisitor}. For use in tests to check if an exception actually was thrown. Calling
+ * {@code AbstractQueueVisitor#awaitExceptionForTestingOnly} can throw a spurious {@link
+ * InterruptedException} because {@link CountDownLatch#await} checks the interrupted bit before
+ * returning, even if the latch is already at 0. See bug "testTwoErrors is flaky".
+ */
+ CountDownLatch getExceptionLatchForTesting() {
+ return visitor.getExceptionLatchForTestingOnly();
+ }
+
+ @Override
+ public boolean inErrorBubblingForTesting() {
+ return bubbleErrorInfo != null;
+ }
+ }
+
+ private static final Function<ValueOrException<BottomException>, SkyValue> GET_VALUE_FROM_VOE =
+ new Function<ValueOrException<BottomException>, SkyValue>() {
+ @Override
+ public SkyValue apply(ValueOrException<BottomException> voe) {
+ return ValueOrExceptionUtils.downcovert(voe);
+ }
+ };
+
+ private static <E extends Exception>
+ Function<ValueOrException2<E, BottomException>, ValueOrException<E>>
+ makeSafeDowncastToVOEFunction(final Class<E> exceptionClass) {
+ return new Function<ValueOrException2<E, BottomException>, ValueOrException<E>>() {
+ @Override
+ public ValueOrException<E> apply(ValueOrException2<E, BottomException> voe) {
+ return ValueOrExceptionUtils.downcovert(voe, exceptionClass);
+ }
+ };
+ }
+
+ private static <E1 extends Exception, E2 extends Exception>
+ Function<ValueOrException3<E1, E2, BottomException>, ValueOrException2<E1, E2>>
+ makeSafeDowncastToVOE2Function(final Class<E1> exceptionClass1,
+ final Class<E2> exceptionClass2) {
+ return new Function<ValueOrException3<E1, E2, BottomException>,
+ ValueOrException2<E1, E2>>() {
+ @Override
+ public ValueOrException2<E1, E2> apply(ValueOrException3<E1, E2, BottomException> voe) {
+ return ValueOrExceptionUtils.downconvert(voe, exceptionClass1, exceptionClass2);
+ }
+ };
+ }
+
+ private static <E1 extends Exception, E2 extends Exception, E3 extends Exception>
+ Function<ValueOrException4<E1, E2, E3, BottomException>, ValueOrException3<E1, E2, E3>>
+ makeSafeDowncastToVOE3Function(final Class<E1> exceptionClass1,
+ final Class<E2> exceptionClass2, final Class<E3> exceptionClass3) {
+ return new Function<ValueOrException4<E1, E2, E3, BottomException>,
+ ValueOrException3<E1, E2, E3>>() {
+ @Override
+ public ValueOrException3<E1, E2, E3> apply(ValueOrException4<E1, E2, E3,
+ BottomException> voe) {
+ return ValueOrExceptionUtils.downconvert(voe, exceptionClass1, exceptionClass2,
+ exceptionClass3);
+ }
+ };
+ }
+
+ private class ValueVisitor extends AbstractQueueVisitor {
+ private AtomicBoolean preventNewEvaluations = new AtomicBoolean(false);
+ private final Set<SkyKey> inflightNodes = Sets.newConcurrentHashSet();
+
+ private ValueVisitor(int threadCount) {
+ super(/*concurrent*/true,
+ threadCount,
+ threadCount,
+ 1, TimeUnit.SECONDS,
+ /*failFastOnException*/true,
+ /*failFastOnInterrupt*/true,
+ "skyframe-evaluator");
+ }
+
+ @Override
+ protected boolean isCriticalError(Throwable e) {
+ return e instanceof RuntimeException;
+ }
+
+ protected void waitForCompletion() throws InterruptedException {
+ work(/*failFastOnInterrupt=*/true);
+ }
+
+ public void enqueueEvaluation(final SkyKey key) {
+ // We unconditionally add the key to the set of in-flight nodes because even if evaluation is
+ // never scheduled we still want to remove the previously created NodeEntry from the graph.
+ // Otherwise we would leave the graph in a weird state (wasteful garbage in the best case and
+ // inconsistent in the worst case).
+ boolean newlyEnqueued = inflightNodes.add(key);
+ // All nodes enqueued for evaluation will be either verified clean, re-evaluated, or cleaned
+ // up after being in-flight when an error happens in nokeep_going mode or in the event of an
+ // interrupt. In any of these cases, they won't be dirty anymore.
+ if (newlyEnqueued) {
+ dirtyKeyTracker.notDirty(key);
+ }
+ if (preventNewEvaluations.get()) {
+ return;
+ }
+ if (newlyEnqueued && progressReceiver != null) {
+ progressReceiver.enqueueing(key);
+ }
+ enqueue(new Evaluate(this, key));
+ }
+
+ public void preventNewEvaluations() {
+ preventNewEvaluations.set(true);
+ }
+
+ public void notifyDone(SkyKey key) {
+ inflightNodes.remove(key);
+ }
+
+ private boolean isInflight(SkyKey key) {
+ return inflightNodes.contains(key);
+ }
+ }
+
+ /**
+ * An action that evaluates a value.
+ */
+ private class Evaluate implements Runnable {
+ private final ValueVisitor visitor;
+ /** The name of the value to be evaluated. */
+ private final SkyKey skyKey;
+
+ private Evaluate(ValueVisitor visitor, SkyKey skyKey) {
+ this.visitor = visitor;
+ this.skyKey = skyKey;
+ }
+
+ private void enqueueChild(SkyKey skyKey, NodeEntry entry, SkyKey child) {
+ Preconditions.checkState(!entry.isDone(), "%s %s", skyKey, entry);
+ Preconditions.checkState(!ErrorTransienceValue.key().equals(child),
+ "%s cannot request ErrorTransienceValue as a dep: %s", skyKey, entry);
+
+ NodeEntry depEntry = graph.createIfAbsent(child);
+ switch (depEntry.addReverseDepAndCheckIfDone(skyKey)) {
+ case DONE :
+ if (entry.signalDep(depEntry.getVersion())) {
+ // This can only happen if there are no more children to be added.
+ visitor.enqueueEvaluation(skyKey);
+ }
+ break;
+ case ADDED_DEP :
+ break;
+ case NEEDS_SCHEDULING :
+ visitor.enqueueEvaluation(child);
+ break;
+ }
+ }
+
+ /**
+ * Returns true if this depGroup consists of the error transience value and the error transience
+ * value is newer than the entry, meaning that the entry must be re-evaluated.
+ */
+ private boolean invalidatedByErrorTransience(Collection<SkyKey> depGroup, NodeEntry entry) {
+ return depGroup.size() == 1
+ && depGroup.contains(ErrorTransienceValue.key())
+ && !graph.get(ErrorTransienceValue.key()).getVersion().atMost(entry.getVersion());
+ }
+
+ @Override
+ public void run() {
+ NodeEntry state = graph.get(skyKey);
+ Preconditions.checkNotNull(state, "%s %s", skyKey, state);
+ Preconditions.checkState(state.isReady(), "%s %s", skyKey, state);
+
+ if (state.isDirty()) {
+ switch (state.getDirtyState()) {
+ case CHECK_DEPENDENCIES:
+ // Evaluating a dirty node for the first time, and checking its children to see if any
+ // of them have changed. Note that there must be dirty children for this to happen.
+
+ // Check the children group by group -- we don't want to evaluate a value that is no
+ // longer needed because an earlier dependency changed. For example, //foo:foo depends
+ // on target //bar:bar and is built. Then foo/BUILD is modified to remove the dependence
+ // on bar, and bar/BUILD is deleted. Reloading //bar:bar would incorrectly throw an
+ // exception. To avoid this, we must reload foo/BUILD first, at which point we will
+ // discover that it has changed, and re-evaluate target //foo:foo from scratch.
+ // On the other hand, when an action requests all of its inputs, we can safely check all
+ // of them in parallel on a subsequent build. So we allow checking an entire group in
+ // parallel here, if the node builder requested a group last build.
+ Collection<SkyKey> directDepsToCheck = state.getNextDirtyDirectDeps();
+
+ if (invalidatedByErrorTransience(directDepsToCheck, state)) {
+ // If this dep is the ErrorTransienceValue and the ErrorTransienceValue has been
+ // updated then we need to force a rebuild. We would like to just signal the entry as
+ // usual, but we can't, because then the ErrorTransienceValue would remain as a dep,
+ // which would be incorrect if, for instance, the value re-evaluated to a non-error.
+ state.forceRebuild();
+ break; // Fall through to re-evaluation.
+ } else {
+ // If this isn't the error transience value, it is safe to add these deps back to the
+ // node -- even if one of them has changed, the contract of pruning is that the node
+ // will request these deps again when it rebuilds. We must add these deps before
+ // enqueuing them, so that the node knows that it depends on them.
+ state.addTemporaryDirectDeps(GroupedListHelper.create(directDepsToCheck));
+ }
+
+ for (SkyKey directDep : directDepsToCheck) {
+ enqueueChild(skyKey, state, directDep);
+ }
+ return;
+ case VERIFIED_CLEAN:
+ // No child has a changed value. This node can be marked done and its parents signaled
+ // without any re-evaluation.
+ visitor.notifyDone(skyKey);
+ Set<SkyKey> reverseDeps = state.markClean();
+ SkyValue value = state.getValue();
+ if (progressReceiver != null && value != null) {
+ // Tell the receiver that the value was not actually changed this run.
+ progressReceiver.evaluated(skyKey, value, EvaluationState.CLEAN);
+ }
+ signalValuesAndEnqueueIfReady(visitor, reverseDeps, state.getVersion());
+ return;
+ case REBUILDING:
+ // Nothing to be done if we are already rebuilding.
+ }
+ }
+
+ // TODO(bazel-team): Once deps are requested in a deterministic order within a group, or the
+ // framework is resilient to rearranging group order, change this so that
+ // SkyFunctionEnvironment "follows along" as the node builder runs, iterating through the
+ // direct deps that were requested on a previous run. This would allow us to avoid the
+ // conversion of the direct deps into a set.
+ Set<SkyKey> directDeps = state.getTemporaryDirectDeps();
+ Preconditions.checkState(!directDeps.contains(ErrorTransienceValue.key()),
+ "%s cannot have a dep on ErrorTransienceValue during building: %s", skyKey, state);
+ // Get the corresponding SkyFunction and call it on this value.
+ SkyFunctionEnvironment env = new SkyFunctionEnvironment(skyKey, directDeps, visitor);
+ SkyFunctionName functionName = skyKey.functionName();
+ SkyFunction factory = skyFunctions.get(functionName);
+ Preconditions.checkState(factory != null, "%s %s", functionName, state);
+
+ SkyValue value = null;
+ Profiler.instance().startTask(ProfilerTask.SKYFUNCTION, skyKey);
+ try {
+ // TODO(bazel-team): count how many of these calls returns null vs. non-null
+ value = factory.compute(skyKey, env);
+ } catch (final SkyFunctionException builderException) {
+ ReifiedSkyFunctionException reifiedBuilderException =
+ new ReifiedSkyFunctionException(builderException, skyKey);
+ // Propagated transitive errors are treated the same as missing deps.
+ if (reifiedBuilderException.getRootCauseSkyKey().equals(skyKey)) {
+ boolean shouldFailFast = !keepGoing || builderException.isCatastrophic();
+ if (shouldFailFast) {
+ // After we commit this error to the graph but before the eval call completes with the
+ // error there is a race-like opportunity for the error to be used, either by an
+ // in-flight computation or by a future computation.
+ if (errorEncountered.compareAndSet(false, true)) {
+ // This is the first error encountered.
+ visitor.preventNewEvaluations();
+ } else {
+ // This is not the first error encountered, so we ignore it so that we can terminate
+ // with the first error.
+ return;
+ }
+ }
+
+ registerNewlyDiscoveredDepsForDoneEntry(skyKey, state, env);
+ ErrorInfo errorInfo = new ErrorInfo(reifiedBuilderException);
+ env.setError(errorInfo);
+ env.commit(/*enqueueParents=*/keepGoing);
+ if (!shouldFailFast) {
+ return;
+ }
+ throw SchedulerException.ofError(errorInfo, skyKey);
+ }
+ } catch (InterruptedException ie) {
+ // InterruptedException cannot be thrown by Runnable.run, so we must wrap it.
+ // Interrupts can be caught by both the Evaluator and the AbstractQueueVisitor.
+ // The former will unwrap the IE and propagate it as is; the latter will throw a new IE.
+ throw SchedulerException.ofInterruption(ie, skyKey);
+ } catch (RuntimeException re) {
+ // Programmer error (most likely NPE or a failed precondition in a SkyFunction). Output
+ // some context together with the exception.
+ String msg = prepareCrashMessage(skyKey, state.getInProgressReverseDeps());
+ throw new RuntimeException(msg, re);
+ } finally {
+ env.doneBuilding();
+ Profiler.instance().completeTask(ProfilerTask.SKYFUNCTION);
+ }
+
+ GroupedListHelper<SkyKey> newDirectDeps = env.newlyRequestedDeps;
+
+ if (value != null) {
+ Preconditions.checkState(!env.valuesMissing,
+ "%s -> %s, ValueEntry: %s", skyKey, newDirectDeps, state);
+ env.setValue(value);
+ registerNewlyDiscoveredDepsForDoneEntry(skyKey, state, env);
+ env.commit(/*enqueueParents=*/true);
+ return;
+ }
+
+ if (!newDirectDeps.isEmpty() && env.getDepErrorKey() != null) {
+ Preconditions.checkState(!keepGoing);
+ // We encountered a child error in noKeepGoing mode, so we want to fail fast. But we first
+ // need to add the edge between the current node and the child error it requested so that
+ // error bubbling can occur. Note that this edge will subsequently be removed during graph
+ // cleaning (since the current node will never be committed to the graph).
+ SkyKey childErrorKey = env.getDepErrorKey();
+ NodeEntry childErrorEntry = Preconditions.checkNotNull(graph.get(childErrorKey),
+ "skyKey: %s, state: %s childErrorKey: %s", skyKey, state, childErrorKey);
+ if (!state.getTemporaryDirectDeps().contains(childErrorKey)) {
+ // This means the cached error was freshly requested (e.g. the parent has never been
+ // built before).
+ Preconditions.checkState(newDirectDeps.contains(childErrorKey), "%s %s %s", state,
+ childErrorKey, newDirectDeps);
+ state.addTemporaryDirectDeps(GroupedListHelper.create(ImmutableList.of(childErrorKey)));
+ DependencyState childErrorState = childErrorEntry.addReverseDepAndCheckIfDone(skyKey);
+ Preconditions.checkState(childErrorState == DependencyState.DONE,
+ "skyKey: %s, state: %s childErrorKey: %s", skyKey, state, childErrorKey,
+ childErrorEntry);
+ } else {
+ // This means the cached error was previously requested, and was then subsequently (after
+ // a restart) requested along with another sibling dep. This can happen on an incremental
+ // eval call when the parent is dirty and the child error is in a separate dependency
+ // group from the sibling dep.
+ Preconditions.checkState(!newDirectDeps.contains(childErrorKey), "%s %s %s", state,
+ childErrorKey, newDirectDeps);
+ Preconditions.checkState(childErrorEntry.isDone(),
+ "skyKey: %s, state: %s childErrorKey: %s", skyKey, state, childErrorKey,
+ childErrorEntry);
+ }
+ ErrorInfo childErrorInfo = Preconditions.checkNotNull(childErrorEntry.getErrorInfo());
+ throw SchedulerException.ofError(childErrorInfo, childErrorKey);
+ }
+
+ // TODO(bazel-team): This code is not safe to interrupt, because we would lose the state in
+ // newDirectDeps.
+
+ // TODO(bazel-team): An ill-behaved SkyFunction can throw us into an infinite loop where we
+ // add more dependencies on every run. [skyframe-core]
+
+ // Add all new keys to the set of known deps.
+ state.addTemporaryDirectDeps(newDirectDeps);
+
+ // If there were no newly requested dependencies, at least one of them was in error or there
+ // is a bug in the SkyFunction implementation. The environment has collected its errors, so we
+ // just order it to be built.
+ if (newDirectDeps.isEmpty()) {
+ // TODO(bazel-team): This means a bug in the SkyFunction. What to do?
+ Preconditions.checkState(!env.childErrorInfos.isEmpty(), "%s %s", skyKey, state);
+ env.commit(/*enqueueParents=*/keepGoing);
+ if (!keepGoing) {
+ throw SchedulerException.ofError(state.getErrorInfo(), skyKey);
+ }
+ return;
+ }
+
+ for (SkyKey newDirectDep : newDirectDeps) {
+ enqueueChild(skyKey, state, newDirectDep);
+ }
+ // It is critical that there is no code below this point.
+ }
+
+ private String prepareCrashMessage(SkyKey skyKey, Iterable<SkyKey> reverseDeps) {
+ StringBuilder reverseDepDump = new StringBuilder();
+ for (SkyKey key : reverseDeps) {
+ if (reverseDepDump.length() > MAX_REVERSEDEP_DUMP_LENGTH) {
+ reverseDepDump.append(", ...");
+ break;
+ }
+ if (reverseDepDump.length() > 0) {
+ reverseDepDump.append(", ");
+ }
+ reverseDepDump.append("'");
+ reverseDepDump.append(key.toString());
+ reverseDepDump.append("'");
+ }
+
+ return String.format(
+ "Unrecoverable error while evaluating node '%s' (requested by nodes %s)",
+ skyKey, reverseDepDump);
+ }
+
+ private static final int MAX_REVERSEDEP_DUMP_LENGTH = 1000;
+ }
+
+ /**
+ * Signals all parents that this node is finished. If visitor is not null, also enqueues any
+ * parents that are ready. If visitor is null, indicating that we are building this node after
+ * the main build aborted, then skip any parents that are already done (that can happen with
+ * cycles).
+ */
+ private void signalValuesAndEnqueueIfReady(@Nullable ValueVisitor visitor, Iterable<SkyKey> keys,
+ Version version) {
+ if (visitor != null) {
+ for (SkyKey key : keys) {
+ if (graph.get(key).signalDep(version)) {
+ visitor.enqueueEvaluation(key);
+ }
+ }
+ } else {
+ for (SkyKey key : keys) {
+ NodeEntry entry = Preconditions.checkNotNull(graph.get(key), key);
+ if (!entry.isDone()) {
+ // In cycles, we can have parents that are already done.
+ entry.signalDep(version);
+ }
+ }
+ }
+ }
+
+ /**
+ * If child is not done, removes key from child's reverse deps. Returns whether child should be
+ * removed from key's entry's direct deps.
+ */
+ private boolean removeIncompleteChild(SkyKey key, SkyKey child) {
+ NodeEntry childEntry = graph.get(child);
+ if (!isDoneForBuild(childEntry)) {
+ childEntry.removeReverseDep(key);
+ return true;
+ }
+ return false;
+ }
+
+ /**
+ * Add any additional deps that were registered during the run of a builder that finished by
+ * creating a node or throwing an error. Builders may throw errors even if all their deps were
+ * not provided -- we trust that a SkyFunction may be know it should throw an error even if not
+ * all of its requested deps are done. However, that means we're assuming the SkyFunction would
+ * throw that same error if all of its requested deps were done. Unfortunately, there is no way to
+ * enforce that condition.
+ */
+ private void registerNewlyDiscoveredDepsForDoneEntry(SkyKey skyKey, NodeEntry entry,
+ SkyFunctionEnvironment env) {
+ Set<SkyKey> unfinishedDeps = new HashSet<>();
+ Iterables.addAll(unfinishedDeps,
+ Iterables.filter(env.newlyRequestedDeps, Predicates.not(nodeEntryIsDone)));
+ env.newlyRequestedDeps.remove(unfinishedDeps);
+ entry.addTemporaryDirectDeps(env.newlyRequestedDeps);
+ for (SkyKey newDep : env.newlyRequestedDeps) {
+ NodeEntry depEntry = graph.get(newDep);
+ DependencyState triState = depEntry.addReverseDepAndCheckIfDone(skyKey);
+ Preconditions.checkState(DependencyState.DONE == triState,
+ "new dep %s was not already done for %s. ValueEntry: %s. DepValueEntry: %s",
+ newDep, skyKey, entry, depEntry);
+ entry.signalDep();
+ }
+ Preconditions.checkState(entry.isReady(), "%s %s %s", skyKey, entry, env.newlyRequestedDeps);
+ }
+
+ private void informProgressReceiverThatValueIsDone(SkyKey key) {
+ if (progressReceiver != null) {
+ NodeEntry entry = graph.get(key);
+ Preconditions.checkState(entry.isDone(), entry);
+ SkyValue value = entry.getValue();
+ if (value != null) {
+ Version valueVersion = entry.getVersion();
+ Preconditions.checkState(valueVersion.atMost(graphVersion),
+ "%s should be at most %s in the version partial ordering", valueVersion, graphVersion);
+ // Nodes with errors will have no value. Don't inform the receiver in that case.
+ // For most nodes we do not inform the progress receiver if they were already done
+ // when we retrieve them, but top-level nodes are presumably of more interest.
+ // If valueVersion is not equal to graphVersion, it must be less than it (by the
+ // Preconditions check above), and so the node is clean.
+ progressReceiver.evaluated(key, value, valueVersion.equals(graphVersion)
+ ? EvaluationState.BUILT
+ : EvaluationState.CLEAN);
+ }
+ }
+ }
+
+ @Override
+ @ThreadCompatible
+ public <T extends SkyValue> EvaluationResult<T> eval(Iterable<SkyKey> skyKeys)
+ throws InterruptedException {
+ ImmutableSet<SkyKey> skyKeySet = ImmutableSet.copyOf(skyKeys);
+
+ // Optimization: if all required node values are already present in the cache, return them
+ // directly without launching the heavy machinery, spawning threads, etc.
+ // Inform progressReceiver that these nodes are done to be consistent with the main code path.
+ if (Iterables.all(skyKeySet, nodeEntryIsDone)) {
+ for (SkyKey skyKey : skyKeySet) {
+ informProgressReceiverThatValueIsDone(skyKey);
+ }
+ // Note that the 'catastrophe' parameter doesn't really matter here (it's only used for
+ // sanity checking).
+ return constructResult(null, skyKeySet, null, /*catastrophe=*/false);
+ }
+
+ if (!keepGoing) {
+ Set<SkyKey> cachedErrorKeys = new HashSet<>();
+ for (SkyKey skyKey : skyKeySet) {
+ NodeEntry entry = graph.get(skyKey);
+ if (entry == null) {
+ continue;
+ }
+ if (entry.isDone() && entry.getErrorInfo() != null) {
+ informProgressReceiverThatValueIsDone(skyKey);
+ cachedErrorKeys.add(skyKey);
+ }
+ }
+
+ // Errors, even cached ones, should halt evaluations not in keepGoing mode.
+ if (!cachedErrorKeys.isEmpty()) {
+ // Note that the 'catastrophe' parameter doesn't really matter here (it's only used for
+ // sanity checking).
+ return constructResult(null, cachedErrorKeys, null, /*catastrophe=*/false);
+ }
+ }
+
+ // We delay this check until we know that some kind of evaluation is necessary, since !keepGoing
+ // and !keepsEdges are incompatible only in the case of a failed evaluation -- there is no
+ // need to be overly harsh to callers who are just trying to retrieve a cached result.
+ Preconditions.checkState(keepGoing || !(graph instanceof InMemoryGraph)
+ || ((InMemoryGraph) graph).keepsEdges(),
+ "nokeep_going evaluations are not allowed if graph edges are not kept: %s", skyKeys);
+
+ Profiler.instance().startTask(ProfilerTask.SKYFRAME_EVAL, skyKeySet);
+ try {
+ return eval(skyKeySet, new ValueVisitor(threadCount));
+ } finally {
+ Profiler.instance().completeTask(ProfilerTask.SKYFRAME_EVAL);
+ }
+ }
+
+ @ThreadCompatible
+ private <T extends SkyValue> EvaluationResult<T> eval(ImmutableSet<SkyKey> skyKeys,
+ ValueVisitor visitor) throws InterruptedException {
+ // We unconditionally add the ErrorTransienceValue here, to ensure that it will be created, and
+ // in the graph, by the time that it is needed. Creating it on demand in a parallel context sets
+ // up a race condition, because there is no way to atomically create a node and set its value.
+ NodeEntry errorTransienceEntry = graph.createIfAbsent(ErrorTransienceValue.key());
+ DependencyState triState = errorTransienceEntry.addReverseDepAndCheckIfDone(null);
+ Preconditions.checkState(triState != DependencyState.ADDED_DEP,
+ "%s %s", errorTransienceEntry, triState);
+ if (triState != DependencyState.DONE) {
+ errorTransienceEntry.setValue(new ErrorTransienceValue(), graphVersion);
+ // The error transience entry is always invalidated by the RecordingDifferencer.
+ // Now that the entry's value is set, it is no longer dirty.
+ dirtyKeyTracker.notDirty(ErrorTransienceValue.key());
+
+ Preconditions.checkState(
+ errorTransienceEntry.addReverseDepAndCheckIfDone(null) != DependencyState.ADDED_DEP,
+ errorTransienceEntry);
+ }
+ for (SkyKey skyKey : skyKeys) {
+ NodeEntry entry = graph.createIfAbsent(skyKey);
+ // This must be equivalent to the code in enqueueChild above, in order to be thread-safe.
+ switch (entry.addReverseDepAndCheckIfDone(null)) {
+ case NEEDS_SCHEDULING:
+ visitor.enqueueEvaluation(skyKey);
+ break;
+ case DONE:
+ informProgressReceiverThatValueIsDone(skyKey);
+ break;
+ case ADDED_DEP:
+ break;
+ default:
+ throw new IllegalStateException(entry + " for " + skyKey + " in unknown state");
+ }
+ }
+ try {
+ return waitForCompletionAndConstructResult(visitor, skyKeys);
+ } finally {
+ // TODO(bazel-team): In nokeep_going mode or in case of an interrupt, we need to remove
+ // partial values from the graph. Find a better way to handle those cases.
+ clean(visitor.inflightNodes);
+ }
+ }
+
+ private void clean(Set<SkyKey> inflightNodes) throws InterruptedException {
+ boolean alreadyInterrupted = Thread.interrupted();
+ // This parallel computation is fully cpu-bound, so we use a thread for each processor.
+ ExecutorService executor = Executors.newFixedThreadPool(
+ Runtime.getRuntime().availableProcessors(),
+ new ThreadFactoryBuilder().setNameFormat("ParallelEvaluator#clean %d").build());
+ ThrowableRecordingRunnableWrapper wrapper =
+ new ThrowableRecordingRunnableWrapper("ParallelEvaluator#clean");
+ for (final SkyKey key : inflightNodes) {
+ final NodeEntry entry = graph.get(key);
+ if (entry.isDone()) {
+ // Entry may be done in case of a RuntimeException or other programming bug. Do nothing,
+ // since (a) we're about to crash anyway, and (b) getTemporaryDirectDeps cannot be called
+ // on a done node, so the call below would crash, which would mask the actual exception
+ // that caused this state.
+ continue;
+ }
+ executor.execute(wrapper.wrap(new Runnable() {
+ @Override
+ public void run() {
+ cleanInflightNode(key, entry);
+ }
+ }));
+ }
+ // We uninterruptibly wait for all nodes to be cleaned because we want to make sure the graph
+ // is left in a good state.
+ //
+ // TODO(bazel-team): Come up with a better design for graph cleaning such that we can respond
+ // to interrupts in constant time.
+ boolean newlyInterrupted = ExecutorShutdownUtil.uninterruptibleShutdown(executor);
+ Throwables.propagateIfPossible(wrapper.getFirstThrownError());
+ if (newlyInterrupted || alreadyInterrupted) {
+ throw new InterruptedException();
+ }
+ }
+
+ private void cleanInflightNode(SkyKey key, NodeEntry entry) {
+ Set<SkyKey> temporaryDeps = entry.getTemporaryDirectDeps();
+ graph.remove(key);
+ for (SkyKey dep : temporaryDeps) {
+ NodeEntry nodeEntry = graph.get(dep);
+ // The direct dep might have already been cleaned from the graph.
+ if (nodeEntry != null) {
+ // Only bother removing the reverse dep on done nodes since other in-flight nodes will be
+ // cleaned too.
+ if (nodeEntry.isDone()) {
+ nodeEntry.removeReverseDep(key);
+ }
+ }
+ }
+ }
+
+ private <T extends SkyValue> EvaluationResult<T> waitForCompletionAndConstructResult(
+ ValueVisitor visitor, Iterable<SkyKey> skyKeys) throws InterruptedException {
+ Map<SkyKey, ValueWithMetadata> bubbleErrorInfo = null;
+ boolean catastrophe = false;
+ try {
+ visitor.waitForCompletion();
+ } catch (final SchedulerException e) {
+ Throwables.propagateIfPossible(e.getCause(), InterruptedException.class);
+ if (Thread.interrupted()) {
+ // As per the contract of AbstractQueueVisitor#work, if an unchecked exception is thrown and
+ // the build is interrupted, the thrown exception is what will be rethrown. Since the user
+ // presumably wanted to interrupt the build, we ignore the thrown SchedulerException (which
+ // doesn't indicate a programming bug) and throw an InterruptedException.
+ throw new InterruptedException();
+ }
+
+ SkyKey errorKey = Preconditions.checkNotNull(e.getFailedValue(), e);
+ // ErrorInfo could only be null if SchedulerException wrapped an InterruptedException, but
+ // that should have been propagated.
+ ErrorInfo errorInfo = Preconditions.checkNotNull(e.getErrorInfo(), errorKey);
+ catastrophe = errorInfo.isCatastrophic();
+ if (!catastrophe || !keepGoing) {
+ bubbleErrorInfo = bubbleErrorUp(errorInfo, errorKey, skyKeys, visitor);
+ } else {
+ // Bubbling the error up requires that graph edges are present for done nodes. This is not
+ // always the case in a keepGoing evaluation, since it is assumed that done nodes do not
+ // need to be traversed. In this case, we hope the caller is tolerant of a possibly empty
+ // result, and return prematurely.
+ bubbleErrorInfo = ImmutableMap.of(errorKey, graph.get(errorKey).getValueWithMetadata());
+ }
+ }
+
+ // Successful evaluation, either because keepGoing or because we actually did succeed.
+ // TODO(bazel-team): Maybe report root causes during the build for lower latency.
+ return constructResult(visitor, skyKeys, bubbleErrorInfo, catastrophe);
+ }
+
+ /**
+ * Walk up graph to find a top-level node (without parents) that wanted this failure. Store
+ * the failed nodes along the way in a map, with ErrorInfos that are appropriate for that layer.
+ * Example:
+ * foo bar
+ * \ /
+ * unrequested baz
+ * \ |
+ * failed-node
+ * User requests foo, bar. When failed-node fails, we look at its parents. unrequested is not
+ * in-flight, so we replace failed-node by baz and repeat. We look at baz's parents. foo is
+ * in-flight, so we replace baz by foo. Since foo is a top-level node and doesn't have parents,
+ * we then break, since we know a top-level node, foo, that depended on the failed node.
+ *
+ * There's the potential for a weird "track jump" here in the case:
+ * foo
+ * / \
+ * fail1 fail2
+ * If fail1 and fail2 fail simultaneously, fail2 may start propagating up in the loop below.
+ * However, foo requests fail1 first, and then throws an exception based on that. This is not
+ * incorrect, but may be unexpected.
+ *
+ * <p>Returns a map of errors that have been constructed during the bubbling up, so that the
+ * appropriate error can be returned to the caller, even though that error was not written to the
+ * graph. If a cycle is detected during the bubbling, this method aborts and returns null so that
+ * the normal cycle detection can handle the cycle.
+ *
+ * <p>Note that we are not propagating error to the first top-level node but to the highest one,
+ * because during this process we can add useful information about error from other nodes.
+ */
+ private Map<SkyKey, ValueWithMetadata> bubbleErrorUp(final ErrorInfo leafFailure,
+ SkyKey errorKey, Iterable<SkyKey> skyKeys, ValueVisitor visitor) {
+ Set<SkyKey> rootValues = ImmutableSet.copyOf(skyKeys);
+ ErrorInfo error = leafFailure;
+ Map<SkyKey, ValueWithMetadata> bubbleErrorInfo = new HashMap<>();
+ boolean externalInterrupt = false;
+ while (true) {
+ NodeEntry errorEntry = graph.get(errorKey);
+ Iterable<SkyKey> reverseDeps = errorEntry.isDone()
+ ? errorEntry.getReverseDeps()
+ : errorEntry.getInProgressReverseDeps();
+ // We should break from loop only when node doesn't have any parents.
+ if (Iterables.isEmpty(reverseDeps)) {
+ Preconditions.checkState(rootValues.contains(errorKey),
+ "Current key %s has to be a top-level key: %s", errorKey, rootValues);
+ break;
+ }
+ SkyKey parent = null;
+ NodeEntry parentEntry = null;
+ for (SkyKey bubbleParent : reverseDeps) {
+ if (bubbleErrorInfo.containsKey(bubbleParent)) {
+ // We are in a cycle. Don't try to bubble anything up -- cycle detection will kick in.
+ return null;
+ }
+ NodeEntry bubbleParentEntry = Preconditions.checkNotNull(graph.get(bubbleParent),
+ "parent %s of %s not in graph", bubbleParent, errorKey);
+ // Might be the parent that requested the error.
+ if (bubbleParentEntry.isDone()) {
+ // This parent is cached from a previous evaluate call. We shouldn't bubble up to it
+ // since any error message produced won't be meaningful to this evaluate call.
+ // The child error must also be cached from a previous build.
+ Preconditions.checkState(errorEntry.isDone(), "%s %s", errorEntry, bubbleParentEntry);
+ Version parentVersion = bubbleParentEntry.getVersion();
+ Version childVersion = errorEntry.getVersion();
+ Preconditions.checkState(childVersion.atMost(graphVersion)
+ && !childVersion.equals(graphVersion),
+ "child entry is not older than the current graph version, but had a done parent. "
+ + "child: %s childEntry: %s, childVersion: %s"
+ + "bubbleParent: %s bubbleParentEntry: %s, parentVersion: %s, graphVersion: %s",
+ errorKey, errorEntry, childVersion,
+ bubbleParent, bubbleParentEntry, parentVersion, graphVersion);
+ Preconditions.checkState(parentVersion.atMost(graphVersion)
+ && !parentVersion.equals(graphVersion),
+ "parent entry is not older than the current graph version. "
+ + "child: %s childEntry: %s, childVersion: %s"
+ + "bubbleParent: %s bubbleParentEntry: %s, parentVersion: %s, graphVersion: %s",
+ errorKey, errorEntry, childVersion,
+ bubbleParent, bubbleParentEntry, parentVersion, graphVersion);
+ continue;
+ }
+ // Arbitrarily pick the first in-flight parent.
+ Preconditions.checkState(visitor.isInflight(bubbleParent),
+ "errorKey: %s, errorEntry: %s, bubbleParent: %s, bubbleParentEntry: %s", errorKey,
+ errorEntry, bubbleParent, bubbleParentEntry);
+ parent = bubbleParent;
+ parentEntry = bubbleParentEntry;
+ break;
+ }
+ Preconditions.checkNotNull(parent, "", errorKey, bubbleErrorInfo);
+ errorKey = parent;
+ SkyFunction factory = skyFunctions.get(parent.functionName());
+ if (parentEntry.isDirty()) {
+ switch (parentEntry.getDirtyState()) {
+ case CHECK_DEPENDENCIES:
+ // If this value's child was bubbled up to, it did not signal this value, and so we must
+ // manually make it ready to build.
+ parentEntry.signalDep();
+ // Fall through to REBUILDING, since state is now REBUILDING.
+ case REBUILDING:
+ // Nothing to be done.
+ break;
+ default:
+ throw new AssertionError(parent + " not in valid dirty state: " + parentEntry);
+ }
+ }
+ SkyFunctionEnvironment env =
+ new SkyFunctionEnvironment(parent, parentEntry.getTemporaryDirectDeps(),
+ bubbleErrorInfo, visitor);
+ externalInterrupt = externalInterrupt || Thread.currentThread().isInterrupted();
+ try {
+ // This build is only to check if the parent node can give us a better error. We don't
+ // care about a return value.
+ factory.compute(parent, env);
+ } catch (SkyFunctionException builderException) {
+ ReifiedSkyFunctionException reifiedBuilderException =
+ new ReifiedSkyFunctionException(builderException, parent);
+ if (reifiedBuilderException.getRootCauseSkyKey().equals(parent)) {
+ error = new ErrorInfo(reifiedBuilderException);
+ bubbleErrorInfo.put(errorKey,
+ ValueWithMetadata.error(new ErrorInfo(errorKey, ImmutableSet.of(error)),
+ env.buildEvents(/*missingChildren=*/true)));
+ continue;
+ }
+ } catch (InterruptedException interruptedException) {
+ // Do nothing.
+ // This throw happens if the builder requested the failed node, and then checked the
+ // interrupted state later -- getValueOrThrow sets the interrupted bit after the failed
+ // value is requested, to prevent the builder from doing too much work.
+ } finally {
+ // Clear interrupted status. We're not listening to interrupts here.
+ Thread.interrupted();
+ }
+ // Builder didn't throw an exception, so just propagate this one up.
+ bubbleErrorInfo.put(errorKey,
+ ValueWithMetadata.error(new ErrorInfo(errorKey, ImmutableSet.of(error)),
+ env.buildEvents(/*missingChildren=*/true)));
+ }
+
+ // Reset the interrupt bit if there was an interrupt from outside this evaluator interrupt.
+ // Note that there are internal interrupts set in the node builder environment if an error
+ // bubbling node calls getValueOrThrow() on a node in error.
+ if (externalInterrupt) {
+ Thread.currentThread().interrupt();
+ }
+ return bubbleErrorInfo;
+ }
+
+ /**
+ * Constructs an {@link EvaluationResult} from the {@link #graph}. Looks for cycles if there
+ * are unfinished nodes but no error was already found through bubbling up
+ * (as indicated by {@code bubbleErrorInfo} being null).
+ *
+ * <p>{@code visitor} may be null, but only in the case where all graph entries corresponding to
+ * {@code skyKeys} are known to be in the DONE state ({@code entry.isDone()} returns true).
+ */
+ private <T extends SkyValue> EvaluationResult<T> constructResult(
+ @Nullable ValueVisitor visitor, Iterable<SkyKey> skyKeys,
+ Map<SkyKey, ValueWithMetadata> bubbleErrorInfo, boolean catastrophe) {
+ Preconditions.checkState(!keepGoing || catastrophe || bubbleErrorInfo == null,
+ "", skyKeys, bubbleErrorInfo);
+ EvaluationResult.Builder<T> result = EvaluationResult.builder();
+ List<SkyKey> cycleRoots = new ArrayList<>();
+ boolean hasError = false;
+ for (SkyKey skyKey : skyKeys) {
+ ValueWithMetadata valueWithMetadata = getValueMaybeFromError(skyKey, bubbleErrorInfo);
+ // Cycle checking: if there is a cycle, evaluation cannot progress, therefore,
+ // the final values will not be in DONE state when the work runs out.
+ if (valueWithMetadata == null) {
+ // Don't look for cycles if the build failed for a known reason.
+ if (bubbleErrorInfo == null) {
+ cycleRoots.add(skyKey);
+ }
+ hasError = true;
+ continue;
+ }
+ SkyValue value = valueWithMetadata.getValue();
+ // TODO(bazel-team): Verify that message replay is fast and works in failure
+ // modes [skyframe-core]
+ // Note that replaying events here is only necessary on null builds, because otherwise we
+ // would have already printed the transitive messages after building these values.
+ replayingNestedSetEventVisitor.visit(valueWithMetadata.getTransitiveEvents());
+ ErrorInfo errorInfo = valueWithMetadata.getErrorInfo();
+ Preconditions.checkState(value != null || errorInfo != null, skyKey);
+ hasError = hasError || (errorInfo != null);
+ if (!keepGoing && errorInfo != null) {
+ // value will be null here unless the value was already built on a prior keepGoing build.
+ result.addError(skyKey, errorInfo);
+ continue;
+ }
+ if (value == null) {
+ // Note that we must be in the keepGoing case. Only make this value an error if it doesn't
+ // have a value. The error shouldn't matter to the caller since the value succeeded after a
+ // fashion.
+ result.addError(skyKey, errorInfo);
+ } else {
+ result.addResult(skyKey, value);
+ }
+ }
+ if (!cycleRoots.isEmpty()) {
+ Preconditions.checkState(visitor != null, skyKeys);
+ checkForCycles(cycleRoots, result, visitor, keepGoing);
+ }
+ Preconditions.checkState(bubbleErrorInfo == null || hasError,
+ "If an error bubbled up, some top-level node must be in error", bubbleErrorInfo, skyKeys);
+ result.setHasError(hasError);
+ return result.build();
+ }
+
+ private <T extends SkyValue> void checkForCycles(
+ Iterable<SkyKey> badRoots, EvaluationResult.Builder<T> result, final ValueVisitor visitor,
+ boolean keepGoing) {
+ for (SkyKey root : badRoots) {
+ ErrorInfo errorInfo = checkForCycles(root, visitor, keepGoing);
+ if (errorInfo == null) {
+ // This node just wasn't finished when evaluation aborted -- there were no cycles below it.
+ Preconditions.checkState(!keepGoing, "", root, badRoots);
+ continue;
+ }
+ Preconditions.checkState(!Iterables.isEmpty(errorInfo.getCycleInfo()),
+ "%s was not evaluated, but was not part of a cycle", root);
+ result.addError(root, errorInfo);
+ if (!keepGoing) {
+ return;
+ }
+ }
+ }
+
+ /**
+ * Marker value that we push onto a stack before we push a node's children on. When the marker
+ * value is popped, we know that all the children are finished. We would use null instead, but
+ * ArrayDeque does not permit null elements.
+ */
+ private static final SkyKey CHILDREN_FINISHED =
+ new SkyKey(new SkyFunctionName("MARKER", false), "MARKER");
+
+ /** The max number of cycles we will report to the user for a given root, to avoid OOMing. */
+ private static final int MAX_CYCLES = 20;
+
+ /**
+ * The algorithm for this cycle detector is as follows. We visit the graph depth-first, keeping
+ * track of the path we are currently on. We skip any DONE nodes (they are transitively
+ * error-free). If we come to a node already on the path, we immediately construct a cycle. If
+ * we are in the noKeepGoing case, we return ErrorInfo with that cycle to the caller. Otherwise,
+ * we continue. Once all of a node's children are done, we construct an error value for it, based
+ * on those children. Finally, when the original root's node is constructed, we return its
+ * ErrorInfo.
+ */
+ private ErrorInfo checkForCycles(SkyKey root, ValueVisitor visitor, boolean keepGoing) {
+ // The number of cycles found. Do not keep on searching for more cycles after this many were
+ // found.
+ int cyclesFound = 0;
+ // The path through the graph currently being visited.
+ List<SkyKey> graphPath = new ArrayList<>();
+ // Set of nodes on the path, to avoid expensive searches through the path for cycles.
+ Set<SkyKey> pathSet = new HashSet<>();
+
+ // Maintain a stack explicitly instead of recursion to avoid stack overflows
+ // on extreme graphs (with long dependency chains).
+ Deque<SkyKey> toVisit = new ArrayDeque<>();
+
+ toVisit.push(root);
+
+ // The procedure for this check is as follows: we visit a node, push it onto the graph stack,
+ // push a marker value onto the toVisit stack, and then push all of its children onto the
+ // toVisit stack. Thus, when the marker node comes to the top of the toVisit stack, we have
+ // visited the downward transitive closure of the value. At that point, all of its children must
+ // be finished, and so we can build the definitive error info for the node, popping it off the
+ // graph stack.
+ while (!toVisit.isEmpty()) {
+ SkyKey key = toVisit.pop();
+ NodeEntry entry = graph.get(key);
+
+ if (key == CHILDREN_FINISHED) {
+ // A marker node means we are done with all children of a node. Since all nodes have
+ // errors, we must have found errors in the children when that happens.
+ key = graphPath.remove(graphPath.size() - 1);
+ entry = graph.get(key);
+ pathSet.remove(key);
+ // Skip this node if it was first/last node of a cycle, and so has already been processed.
+ if (entry.isDone()) {
+ continue;
+ }
+ if (!keepGoing) {
+ // in the --nokeep_going mode, we would have already returned if we'd found a cycle below
+ // this node. The fact that we haven't means that there were no cycles below this node
+ // -- it just hadn't finished evaluating. So skip it.
+ continue;
+ }
+ if (cyclesFound < MAX_CYCLES) {
+ // Value must be ready, because all of its children have finished, so we can build its
+ // error.
+ Preconditions.checkState(entry.isReady(), "%s not ready. ValueEntry: %s", key, entry);
+ } else if (!entry.isReady()) {
+ removeIncompleteChildrenForCycle(key, entry, entry.getTemporaryDirectDeps());
+ }
+ Set<SkyKey> directDeps = entry.getTemporaryDirectDeps();
+ // Find out which children have errors. Similar logic to that in Evaluate#run().
+ List<ErrorInfo> errorDeps = getChildrenErrorsForCycle(directDeps);
+ Preconditions.checkState(!errorDeps.isEmpty(),
+ "Value %s was not successfully evaluated, but had no child errors. ValueEntry: %s", key,
+ entry);
+ SkyFunctionEnvironment env = new SkyFunctionEnvironment(key, directDeps, visitor);
+ env.setError(new ErrorInfo(key, errorDeps));
+ env.commit(/*enqueueParents=*/false);
+ }
+
+ // Nothing to be done for this node if it already has an entry.
+ if (entry.isDone()) {
+ continue;
+ }
+ if (cyclesFound == MAX_CYCLES) {
+ // Do not keep on searching for cycles indefinitely, to avoid excessive runtime/OOMs.
+ continue;
+ }
+
+ if (pathSet.contains(key)) {
+ int cycleStart = graphPath.indexOf(key);
+ // Found a cycle!
+ cyclesFound++;
+ Iterable<SkyKey> cycle = graphPath.subList(cycleStart, graphPath.size());
+ // Put this node into a consistent state for building if it is dirty.
+ if (entry.isDirty() && entry.getDirtyState() == DirtyState.CHECK_DEPENDENCIES) {
+ // In the check deps state, entry has exactly one child not done yet. Note that this node
+ // must be part of the path to the cycle we have found (since done nodes cannot be in
+ // cycles, and this is the only missing one). Thus, it will not be removed below in
+ // removeDescendantsOfCycleValue, so it is safe here to signal that it is done.
+ entry.signalDep();
+ }
+ if (keepGoing) {
+ // Any children of this node that we haven't already visited are not worth visiting,
+ // since this node is about to be done. Thus, the only child worth visiting is the one in
+ // this cycle, the cycleChild (which may == key if this cycle is a self-edge).
+ SkyKey cycleChild = selectCycleChild(key, graphPath, cycleStart);
+ removeDescendantsOfCycleValue(key, entry, cycleChild, toVisit,
+ graphPath.size() - cycleStart);
+ ValueWithMetadata dummyValue = ValueWithMetadata.wrapWithMetadata(new SkyValue() {});
+
+
+ SkyFunctionEnvironment env =
+ new SkyFunctionEnvironment(key, entry.getTemporaryDirectDeps(),
+ ImmutableMap.of(cycleChild, dummyValue), visitor);
+
+ // Construct error info for this node. Get errors from children, which are all done
+ // except possibly for the cycleChild.
+ List<ErrorInfo> allErrors =
+ getChildrenErrors(entry.getTemporaryDirectDeps(), /*unfinishedChild=*/cycleChild);
+ CycleInfo cycleInfo = new CycleInfo(cycle);
+ // Add in this cycle.
+ allErrors.add(new ErrorInfo(cycleInfo));
+ env.setError(new ErrorInfo(key, allErrors));
+ env.commit(/*enqueueParents=*/false);
+ continue;
+ } else {
+ // We need to return right away in the noKeepGoing case, so construct the cycle (with the
+ // path) and return.
+ Preconditions.checkState(graphPath.get(0).equals(root),
+ "%s not reached from %s. ValueEntry: %s", key, root, entry);
+ return new ErrorInfo(new CycleInfo(graphPath.subList(0, cycleStart), cycle));
+ }
+ }
+
+ // This node is not yet known to be in a cycle. So process its children.
+ Iterable<? extends SkyKey> children = graph.get(key).getTemporaryDirectDeps();
+ if (Iterables.isEmpty(children)) {
+ continue;
+ }
+
+ // This marker flag will tell us when all this node's children have been processed.
+ toVisit.push(CHILDREN_FINISHED);
+ // This node is now part of the path through the graph.
+ graphPath.add(key);
+ pathSet.add(key);
+ for (SkyKey nextValue : children) {
+ toVisit.push(nextValue);
+ }
+ }
+ return keepGoing ? getAndCheckDone(root).getErrorInfo() : null;
+ }
+
+ /**
+ * Returns the child of this node that is in the cycle that was just found. If the cycle is a
+ * self-edge, returns the node itself.
+ */
+ private static SkyKey selectCycleChild(SkyKey key, List<SkyKey> graphPath, int cycleStart) {
+ return cycleStart + 1 == graphPath.size() ? key : graphPath.get(cycleStart + 1);
+ }
+
+ /**
+ * Get all the errors of child nodes. There must be at least one cycle amongst them.
+ *
+ * @param children child nodes to query for errors.
+ * @return List of ErrorInfos from all children that had errors.
+ */
+ private List<ErrorInfo> getChildrenErrorsForCycle(Iterable<SkyKey> children) {
+ List<ErrorInfo> allErrors = new ArrayList<>();
+ boolean foundCycle = false;
+ for (SkyKey child : children) {
+ ErrorInfo errorInfo = getAndCheckDone(child).getErrorInfo();
+ if (errorInfo != null) {
+ foundCycle |= !Iterables.isEmpty(errorInfo.getCycleInfo());
+ allErrors.add(errorInfo);
+ }
+ }
+ Preconditions.checkState(foundCycle, "", children, allErrors);
+ return allErrors;
+ }
+
+ /**
+ * Get all the errors of child nodes.
+ *
+ * @param children child nodes to query for errors.
+ * @param unfinishedChild child which is allowed to not be done.
+ * @return List of ErrorInfos from all children that had errors.
+ */
+ private List<ErrorInfo> getChildrenErrors(Iterable<SkyKey> children, SkyKey unfinishedChild) {
+ List<ErrorInfo> allErrors = new ArrayList<>();
+ for (SkyKey child : children) {
+ ErrorInfo errorInfo = getErrorMaybe(child, /*allowUnfinished=*/child.equals(unfinishedChild));
+ if (errorInfo != null) {
+ allErrors.add(errorInfo);
+ }
+ }
+ return allErrors;
+ }
+
+ @Nullable
+ private ErrorInfo getErrorMaybe(SkyKey key, boolean allowUnfinished) {
+ if (!allowUnfinished) {
+ return getAndCheckDone(key).getErrorInfo();
+ }
+ NodeEntry entry = Preconditions.checkNotNull(graph.get(key), key);
+ return entry.isDone() ? entry.getErrorInfo() : null;
+ }
+
+ /**
+ * Removes direct children of key from toVisit and from the entry itself, and makes the entry
+ * ready if necessary. We must do this because it would not make sense to try to build the
+ * children after building the entry. It would violate the invariant that a parent can only be
+ * built after its children are built; See bug "Precondition error while evaluating a Skyframe
+ * graph with a cycle".
+ *
+ * @param key SkyKey of node in a cycle.
+ * @param entry NodeEntry of node in a cycle.
+ * @param cycleChild direct child of key in the cycle, or key itself if the cycle is a self-edge.
+ * @param toVisit list of remaining nodes to visit by the cycle-checker.
+ * @param cycleLength the length of the cycle found.
+ */
+ private void removeDescendantsOfCycleValue(SkyKey key, NodeEntry entry,
+ @Nullable SkyKey cycleChild, Iterable<SkyKey> toVisit, int cycleLength) {
+ Set<SkyKey> unvisitedDeps = new HashSet<>(entry.getTemporaryDirectDeps());
+ unvisitedDeps.remove(cycleChild);
+ // Remove any children from this node that are not part of the cycle we just found. They are
+ // irrelevant to the node as it stands, and if they are deleted from the graph because they are
+ // not built by the end of cycle-checking, we would have dangling references.
+ removeIncompleteChildrenForCycle(key, entry, unvisitedDeps);
+ if (!entry.isReady()) {
+ // The entry has at most one undone dep now, its cycleChild. Signal to make entry ready. Note
+ // that the entry can conceivably be ready if its cycleChild already found a different cycle
+ // and was built.
+ entry.signalDep();
+ }
+ Preconditions.checkState(entry.isReady(), "%s %s %s", key, cycleChild, entry);
+ Iterator<SkyKey> it = toVisit.iterator();
+ while (it.hasNext()) {
+ SkyKey descendant = it.next();
+ if (descendant == CHILDREN_FINISHED) {
+ // Marker value, delineating the end of a group of children that were enqueued.
+ cycleLength--;
+ if (cycleLength == 0) {
+ // We have seen #cycleLength-1 marker values, and have arrived at the one for this value,
+ // so we are done.
+ return;
+ }
+ continue; // Don't remove marker values.
+ }
+ if (cycleLength == 1) {
+ // Remove the direct children remaining to visit of the cycle node.
+ Preconditions.checkState(unvisitedDeps.contains(descendant),
+ "%s %s %s %s %s", key, descendant, cycleChild, unvisitedDeps, entry);
+ it.remove();
+ }
+ }
+ throw new IllegalStateException("There were not " + cycleLength + " groups of children in "
+ + toVisit + " when trying to remove children of " + key + " other than " + cycleChild);
+ }
+
+ private void removeIncompleteChildrenForCycle(SkyKey key, NodeEntry entry,
+ Iterable<SkyKey> children) {
+ Set<SkyKey> unfinishedDeps = new HashSet<>();
+ for (SkyKey child : children) {
+ if (removeIncompleteChild(key, child)) {
+ unfinishedDeps.add(child);
+ }
+ }
+ entry.removeUnfinishedDeps(unfinishedDeps);
+ }
+
+ private NodeEntry getAndCheckDone(SkyKey key) {
+ NodeEntry entry = graph.get(key);
+ Preconditions.checkNotNull(entry, key);
+ Preconditions.checkState(entry.isDone(), "%s %s", key, entry);
+ return entry;
+ }
+
+ private ValueWithMetadata getValueMaybeFromError(SkyKey key,
+ @Nullable Map<SkyKey, ValueWithMetadata> bubbleErrorInfo) {
+ SkyValue value = bubbleErrorInfo == null ? null : bubbleErrorInfo.get(key);
+ NodeEntry entry = graph.get(key);
+ if (value != null) {
+ Preconditions.checkNotNull(entry,
+ "Value cannot have error before evaluation started", key, value);
+ return ValueWithMetadata.wrapWithMetadata(value);
+ }
+ return isDoneForBuild(entry) ? entry.getValueWithMetadata() : null;
+ }
+
+ /**
+ * Return true if the entry does not need to be re-evaluated this build. The entry will need to
+ * be re-evaluated if it is not done, but also if it was not completely evaluated last build and
+ * this build is keepGoing.
+ */
+ private boolean isDoneForBuild(@Nullable NodeEntry entry) {
+ return entry != null && entry.isDone();
+ }
+}
diff --git a/src/main/java/com/google/devtools/build/skyframe/ProcessableGraph.java b/src/main/java/com/google/devtools/build/skyframe/ProcessableGraph.java
new file mode 100644
index 0000000..8bf8a38
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/ProcessableGraph.java
@@ -0,0 +1,24 @@
+// Copyright 2014 Google Inc. 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.skyframe;
+
+/**
+ * A graph that is both Dirtiable (values can be deleted) and Evaluable (values can be added). All
+ * methods in this interface (as inherited from super-interfaces) should be thread-safe.
+ *
+ * <p>This class is not intended for direct use, and is only exposed as public for use in
+ * evaluation implementations outside of this package.
+ */
+public interface ProcessableGraph extends DirtiableGraph, EvaluableGraph {
+}
diff --git a/src/main/java/com/google/devtools/build/skyframe/QueryableGraph.java b/src/main/java/com/google/devtools/build/skyframe/QueryableGraph.java
new file mode 100644
index 0000000..e1cfc0a
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/QueryableGraph.java
@@ -0,0 +1,24 @@
+// Copyright 2014 Google Inc. 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.skyframe;
+
+/**
+ * A graph that exposes its entries and structure, for use by classes that must traverse it.
+ */
+public interface QueryableGraph {
+ /**
+ * Returns the node with the given name, or {@code null} if the node does not exist.
+ */
+ NodeEntry get(SkyKey key);
+}
diff --git a/src/main/java/com/google/devtools/build/skyframe/RecordingDifferencer.java b/src/main/java/com/google/devtools/build/skyframe/RecordingDifferencer.java
new file mode 100644
index 0000000..3ebbf33
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/RecordingDifferencer.java
@@ -0,0 +1,76 @@
+// Copyright 2014 Google Inc. 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.skyframe;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+import com.google.devtools.build.lib.concurrent.ThreadSafety;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A simple Differencer which just records the invalidated values it's been given.
+ */
+@ThreadSafety.ThreadCompatible
+public class RecordingDifferencer implements Differencer, Injectable {
+
+ private List<SkyKey> valuesToInvalidate;
+ private Map<SkyKey, SkyValue> valuesToInject;
+
+ public RecordingDifferencer() {
+ clear();
+ }
+
+ private void clear() {
+ valuesToInvalidate = new ArrayList<>();
+ valuesToInject = new HashMap<>();
+ }
+
+ @Override
+ public Diff getDiff(Version fromVersion, Version toVersion) {
+ Diff diff = new ImmutableDiff(valuesToInvalidate, valuesToInject);
+ clear();
+ return diff;
+ }
+
+ /**
+ * Store the given values for invalidation.
+ */
+ public void invalidate(Iterable<SkyKey> values) {
+ Iterables.addAll(valuesToInvalidate, values);
+ }
+
+ /**
+ * Invalidates the cached values of any values in error transiently.
+ *
+ * <p>If a future call to {@link MemoizingEvaluator#evaluate} requests a value that transitively
+ * depends on any value that was in an error state (or is one of these), they will be re-computed.
+ */
+ public void invalidateTransientErrors() {
+ // All transient error values have a dependency on the single global ERROR_TRANSIENCE value,
+ // so we only have to invalidate that one value to catch everything.
+ invalidate(ImmutableList.of(ErrorTransienceValue.key()));
+ }
+
+ /**
+ * Store the given values for injection.
+ */
+ @Override
+ public void inject(Map<SkyKey, ? extends SkyValue> values) {
+ valuesToInject.putAll(values);
+ }
+}
diff --git a/src/main/java/com/google/devtools/build/skyframe/ReverseDepsUtil.java b/src/main/java/com/google/devtools/build/skyframe/ReverseDepsUtil.java
new file mode 100644
index 0000000..13d8c4b
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/ReverseDepsUtil.java
@@ -0,0 +1,211 @@
+// Copyright 2014 Google Inc. 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.skyframe;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * A utility class that allows us to keep the reverse dependencies as an array list instead of a
+ * set. This is more memory-efficient. At the same time it allows us to group the removals and
+ * uniqueness checks so that it also performs well.
+ *
+ * <p>The reason of this class it to share non-trivial code between BuildingState and NodeEntry. We
+ * could simply make those two classes extend this class instead, but we would be less
+ * memory-efficient since object memory alignment does not cross classes ( you would have two memory
+ * alignments, one for the base class and one for the extended one).
+ */
+abstract class ReverseDepsUtil<T> {
+
+ static final int MAYBE_CHECK_THRESHOLD = 10;
+
+ abstract void setReverseDepsObject(T container, Object object);
+
+ abstract void setSingleReverseDep(T container, boolean singleObject);
+
+ abstract void setReverseDepsToRemove(T container, List<SkyKey> object);
+
+ abstract Object getReverseDepsObject(T container);
+
+ abstract boolean isSingleReverseDep(T container);
+
+ abstract List<SkyKey> getReverseDepsToRemove(T container);
+
+ /**
+ * We check that the reverse dependency is not already present. We only do that if reverseDeps is
+ * small, so that it does not impact performance.
+ */
+ void maybeCheckReverseDepNotPresent(T container, SkyKey reverseDep) {
+ if (isSingleReverseDep(container)) {
+ Preconditions.checkState(!getReverseDepsObject(container).equals(reverseDep),
+ "Reverse dep %s already present", reverseDep);
+ return;
+ }
+ @SuppressWarnings("unchecked")
+ List<SkyKey> asList = (List<SkyKey>) getReverseDepsObject(container);
+ if (asList.size() < MAYBE_CHECK_THRESHOLD) {
+ Preconditions.checkState(!asList.contains(reverseDep), "Reverse dep %s already present"
+ + " in %s", reverseDep, asList);
+ }
+ }
+
+ /**
+ * We use a memory-efficient trick to keep reverseDeps memory usage low. Edges in Bazel are
+ * dominant over the number of nodes.
+ *
+ * <p>Most of the nodes have zero or one reverse dep. That is why we use immutable versions of the
+ * lists for those cases. In case of the size being > 1 we switch to an ArrayList. That is because
+ * we also have a decent number of nodes for which the reverseDeps are huge (for example almost
+ * everything depends on BuildInfo node).
+ *
+ * <p>We also optimize for the case where we have only one dependency. In that case we keep the
+ * object directly instead of a wrapper list.
+ */
+ @SuppressWarnings("unchecked")
+ void addReverseDeps(T container, Collection<SkyKey> newReverseDeps) {
+ if (newReverseDeps.isEmpty()) {
+ return;
+ }
+ Object reverseDeps = getReverseDepsObject(container);
+ int reverseDepsSize = isSingleReverseDep(container) ? 1 : ((List<SkyKey>) reverseDeps).size();
+ int newSize = reverseDepsSize + newReverseDeps.size();
+ if (newSize == 1) {
+ overwriteReverseDepsWithObject(container, Iterables.getOnlyElement(newReverseDeps));
+ } else if (reverseDepsSize == 0) {
+ overwriteReverseDepsList(container, Lists.newArrayList(newReverseDeps));
+ } else if (reverseDepsSize == 1) {
+ List<SkyKey> newList = Lists.newArrayListWithExpectedSize(newSize);
+ newList.add((SkyKey) reverseDeps);
+ newList.addAll(newReverseDeps);
+ overwriteReverseDepsList(container, newList);
+ } else {
+ ((List<SkyKey>) reverseDeps).addAll(newReverseDeps);
+ }
+ }
+
+ /**
+ * See {@code addReverseDeps} method.
+ */
+ void removeReverseDep(T container, SkyKey reverseDep) {
+ if (isSingleReverseDep(container)) {
+ // This removal is cheap so let's do it and not keep it in reverseDepsToRemove.
+ // !equals should only happen in case of catastrophe.
+ if (getReverseDepsObject(container).equals(reverseDep)) {
+ overwriteReverseDepsList(container, ImmutableList.<SkyKey>of());
+ }
+ return;
+ }
+ @SuppressWarnings("unchecked")
+ List<SkyKey> reverseDepsAsList = (List<SkyKey>) getReverseDepsObject(container);
+ if (reverseDepsAsList.isEmpty()) {
+ return;
+ }
+ List<SkyKey> reverseDepsToRemove = getReverseDepsToRemove(container);
+ if (reverseDepsToRemove == null) {
+ reverseDepsToRemove = Lists.newArrayListWithExpectedSize(1);
+ setReverseDepsToRemove(container, reverseDepsToRemove);
+ }
+ reverseDepsToRemove.add(reverseDep);
+ }
+
+ ImmutableSet<SkyKey> getReverseDeps(T container) {
+ consolidateReverseDepsRemovals(container);
+
+ // TODO(bazel-team): Unfortunately, we need to make a copy here right now to be on the safe side
+ // wrt. thread-safety. The parents of a node get modified when any of the parents is deleted,
+ // and we can't handle that right now.
+ if (isSingleReverseDep(container)) {
+ return ImmutableSet.of((SkyKey) getReverseDepsObject(container));
+ } else {
+ @SuppressWarnings("unchecked")
+ List<SkyKey> reverseDeps = (List<SkyKey>) getReverseDepsObject(container);
+ ImmutableSet<SkyKey> set = ImmutableSet.copyOf(reverseDeps);
+ Preconditions.checkState(set.size() == reverseDeps.size(),
+ "Duplicate reverse deps present in %s: %s. %s", this, reverseDeps, container);
+ return set;
+ }
+ }
+
+ void consolidateReverseDepsRemovals(T container) {
+ List<SkyKey> reverseDepsToRemove = getReverseDepsToRemove(container);
+ Object reverseDeps = getReverseDepsObject(container);
+ if (reverseDepsToRemove == null) {
+ return;
+ }
+ Preconditions.checkState(!isSingleReverseDep(container),
+ "We do not use reverseDepsToRemove for single lists: %s", container);
+ // Should not happen, as we only create reverseDepsToRemove in case we have at least one
+ // reverse dep to remove.
+ Preconditions.checkState((!((List<?>) reverseDeps).isEmpty()),
+ "Could not remove %s elements from %s.\nReverse deps to remove: %s. %s",
+ reverseDepsToRemove.size(), reverseDeps, reverseDepsToRemove, container);
+
+ Set<SkyKey> toRemove = Sets.newHashSet(reverseDepsToRemove);
+ int expectedRemovals = toRemove.size();
+ Preconditions.checkState(expectedRemovals == reverseDepsToRemove.size(),
+ "A reverse dependency tried to remove itself twice: %s. %s", reverseDepsToRemove,
+ container);
+
+ @SuppressWarnings("unchecked")
+ List<SkyKey> reverseDepsAsList = (List<SkyKey>) reverseDeps;
+ List<SkyKey> newReverseDeps = Lists
+ .newArrayListWithExpectedSize(Math.max(0, reverseDepsAsList.size() - expectedRemovals));
+
+ for (SkyKey reverseDep : reverseDepsAsList) {
+ if (!toRemove.contains(reverseDep)) {
+ newReverseDeps.add(reverseDep);
+ }
+ }
+ Preconditions.checkState(newReverseDeps.size() == reverseDepsAsList.size() - expectedRemovals,
+ "Could not remove some elements from %s.\nReverse deps to remove: %s. %s", reverseDeps,
+ toRemove, container);
+
+ if (newReverseDeps.isEmpty()) {
+ overwriteReverseDepsList(container, ImmutableList.<SkyKey>of());
+ } else if (newReverseDeps.size() == 1) {
+ overwriteReverseDepsWithObject(container, newReverseDeps.get(0));
+ } else {
+ overwriteReverseDepsList(container, newReverseDeps);
+ }
+ setReverseDepsToRemove(container, null);
+ }
+
+ @SuppressWarnings("deprecation")
+ String toString(T container) {
+ return Objects.toStringHelper("ReverseDeps") // MoreObjects is not in Guava
+ .add("reverseDeps", getReverseDepsObject(container))
+ .add("singleReverseDep", isSingleReverseDep(container))
+ .add("reverseDepsToRemove", getReverseDepsToRemove(container))
+ .toString();
+ }
+
+ private void overwriteReverseDepsWithObject(T container, SkyKey newObject) {
+ setReverseDepsObject(container, newObject);
+ setSingleReverseDep(container, true);
+ }
+
+ private void overwriteReverseDepsList(T container, List<SkyKey> list) {
+ setReverseDepsObject(container, list);
+ setSingleReverseDep(container, false);
+ }
+}
diff --git a/src/main/java/com/google/devtools/build/skyframe/Scheduler.java b/src/main/java/com/google/devtools/build/skyframe/Scheduler.java
new file mode 100644
index 0000000..f05860f
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/Scheduler.java
@@ -0,0 +1,78 @@
+// Copyright 2014 Google Inc. 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.skyframe;
+
+import com.google.common.base.Preconditions;
+
+import javax.annotation.Nullable;
+
+/**
+ * A work queue -- takes {@link Runnable}s and runs them when requested.
+ */
+interface Scheduler {
+ /**
+ * Schedules a new action to be eventually done.
+ */
+ void schedule(Runnable action);
+
+ /**
+ * Runs the actions that have been scheduled. These actions can in turn schedule new actions,
+ * which will be run as well.
+ *
+ * @throw SchedulerException wrapping a scheduled action's exception.
+ */
+ void run() throws SchedulerException;
+
+ /**
+ * Wrapper exception that {@link Runnable}s can throw, to be caught and handled
+ * by callers of {@link #run}.
+ */
+ static class SchedulerException extends RuntimeException {
+ private final SkyKey failedValue;
+ private final ErrorInfo errorInfo;
+
+ private SchedulerException(@Nullable Throwable cause, @Nullable ErrorInfo errorInfo,
+ SkyKey failedValue) {
+ super(errorInfo != null ? errorInfo.getException() : cause);
+ this.errorInfo = errorInfo;
+ this.failedValue = Preconditions.checkNotNull(failedValue, errorInfo);
+ }
+
+ /**
+ * Returns a SchedulerException wrapping an expected error, e.g. an error describing an expected
+ * build failure when trying to evaluate the given value, that should cause Skyframe to produce
+ * useful error information to the user.
+ */
+ static SchedulerException ofError(ErrorInfo errorInfo, SkyKey failedValue) {
+ Preconditions.checkNotNull(errorInfo);
+ return new SchedulerException(errorInfo.getException(), errorInfo, failedValue);
+ }
+
+ /**
+ * Returns a SchedulerException wrapping an InterruptedException, e.g. if the user interrupts
+ * the build, that should cause Skyframe to exit as soon as possible.
+ */
+ static SchedulerException ofInterruption(InterruptedException cause, SkyKey failedValue) {
+ return new SchedulerException(cause, null, failedValue);
+ }
+
+ SkyKey getFailedValue() {
+ return failedValue;
+ }
+
+ @Nullable ErrorInfo getErrorInfo() {
+ return errorInfo;
+ }
+ }
+}
diff --git a/src/main/java/com/google/devtools/build/skyframe/SequentialBuildDriver.java b/src/main/java/com/google/devtools/build/skyframe/SequentialBuildDriver.java
new file mode 100644
index 0000000..9b7f036
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/SequentialBuildDriver.java
@@ -0,0 +1,46 @@
+// Copyright 2014 Google Inc. 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.skyframe;
+
+import com.google.common.base.Preconditions;
+import com.google.devtools.build.lib.events.EventHandler;
+
+/**
+ * A driver for auto-updating graphs which operate over monotonically increasing integer versions.
+ */
+public class SequentialBuildDriver implements BuildDriver {
+ private final MemoizingEvaluator memoizingEvaluator;
+ private IntVersion curVersion;
+
+ public SequentialBuildDriver(MemoizingEvaluator evaluator) {
+ this.memoizingEvaluator = Preconditions.checkNotNull(evaluator);
+ this.curVersion = new IntVersion(0);
+ }
+
+ @Override
+ public <T extends SkyValue> EvaluationResult<T> evaluate(
+ Iterable<SkyKey> roots, boolean keepGoing, int numThreads, EventHandler reporter)
+ throws InterruptedException {
+ try {
+ return memoizingEvaluator.evaluate(roots, curVersion, keepGoing, numThreads, reporter);
+ } finally {
+ curVersion = curVersion.next();
+ }
+ }
+
+ @Override
+ public MemoizingEvaluator getGraphForTesting() {
+ return memoizingEvaluator;
+ }
+}
diff --git a/src/main/java/com/google/devtools/build/skyframe/SkyFunction.java b/src/main/java/com/google/devtools/build/skyframe/SkyFunction.java
new file mode 100644
index 0000000..324c03d
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/SkyFunction.java
@@ -0,0 +1,187 @@
+// Copyright 2014 Google Inc. 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.skyframe;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.devtools.build.lib.events.EventHandler;
+
+import java.util.Map;
+
+import javax.annotation.Nullable;
+
+/**
+ * Machinery to evaluate a single value.
+ *
+ * <p>The builder is supposed to access only direct dependencies of the value. However, the direct
+ * dependencies need not be known in advance. The builder can request arbitrary values using
+ * {@link Environment#getValue}. If the values are not ready, the call will return null; in that
+ * case the builder can either try to proceed (and potentially indicate more dependencies by
+ * additional {@code getValue} calls), or just return null, in which case the missing dependencies
+ * will be computed and the builder will be started again.
+ */
+public interface SkyFunction {
+
+ /**
+ * When a value is requested, this method is called with the name of the value and a value
+ * building environment.
+ *
+ * <p>This method should return a constructed value, or null if any dependencies were missing
+ * ({@link Environment#valuesMissing} was true before returning). In that case the missing
+ * dependencies will be computed and the value builder restarted.
+ *
+ * <p>Implementations must be threadsafe and reentrant.
+ *
+ * @throws SkyFunctionException on failure
+ * @throws InterruptedException when the user interrupts the build
+ */
+ @Nullable SkyValue compute(SkyKey skyKey, Environment env) throws SkyFunctionException,
+ InterruptedException;
+
+ /**
+ * Extracts a tag (target label) from a SkyKey if it has one. Otherwise return null.
+ *
+ * <p>The tag is used for filtering out non-error event messages that do not match --output_filter
+ * flag. If a SkyFunction returns null in this method it means that all the info/warning messages
+ * associated with this value will be shown, no matter what --output_filter says.
+ */
+ @Nullable
+ String extractTag(SkyKey skyKey);
+
+ /**
+ * The services provided to the value builder by the graph implementation.
+ */
+ interface Environment {
+ /**
+ * Returns a direct dependency. If the specified value is not in the set of already evaluated
+ * direct dependencies, returns null. Also returns null if the specified value has already been
+ * evaluated and found to be in error.
+ *
+ * <p>On a subsequent build, if any of this value's dependencies have changed they will be
+ * re-evaluated in the same order as originally requested by the {@code SkyFunction} using
+ * this {@code getValue} call (see {@link #getValues} for when preserving the order is not
+ * important).
+ */
+ @Nullable
+ SkyValue getValue(SkyKey valueName);
+
+ /**
+ * Returns a direct dependency. If the specified value is not in the set of already evaluated
+ * direct dependencies, returns null. If the specified value has already been evaluated and
+ * found to be in error, throws the exception coming from the error. Value builders may
+ * use this method to continue evaluation even if one of their children is in error by catching
+ * the thrown exception and proceeding. The caller must specify the exception that might be
+ * thrown using the {@code exceptionClass} argument. If the child's exception is not an instance
+ * of {@code exceptionClass}, returns null without throwing.
+ *
+ * <p>The exception class given cannot be a supertype or a subtype of {@link RuntimeException},
+ * or a subtype of {@link InterruptedException}. See
+ * {@link SkyFunctionException#validateExceptionType} for details.
+ */
+ @Nullable
+ <E extends Exception> SkyValue getValueOrThrow(SkyKey depKey, Class<E> exceptionClass) throws E;
+ @Nullable
+ <E1 extends Exception, E2 extends Exception> SkyValue getValueOrThrow(SkyKey depKey,
+ Class<E1> exceptionClass1, Class<E2> exceptionClass2) throws E1, E2;
+ @Nullable
+ <E1 extends Exception, E2 extends Exception, E3 extends Exception> SkyValue getValueOrThrow(
+ SkyKey depKey, Class<E1> exceptionClass1, Class<E2> exceptionClass2,
+ Class<E3> exceptionClass3) throws E1, E2, E3;
+ @Nullable
+ <E1 extends Exception, E2 extends Exception, E3 extends Exception, E4 extends Exception>
+ SkyValue getValueOrThrow(SkyKey depKey, Class<E1> exceptionClass1,
+ Class<E2> exceptionClass2, Class<E3> exceptionClass3, Class<E4> exceptionClass4)
+ throws E1, E2, E3, E4;
+
+ /**
+ * Returns true iff any of the past {@link #getValue}(s) or {@link #getValueOrThrow} method
+ * calls for this instance returned null (because the value was not yet present and done in the
+ * graph).
+ *
+ * <p>If this returns true, the {@link SkyFunction} must return {@code null}.
+ */
+ boolean valuesMissing();
+
+ /**
+ * Requests {@code depKeys} "in parallel", independent of each others' values. These keys may be
+ * thought of as a "dependency group" -- they are requested together by this value.
+ *
+ * <p>In general, if the result of one getValue call can affect the argument of a later getValue
+ * call, the two calls cannot be merged into a single getValues call, since the result of the
+ * first call might change on a later build. Inversely, if the result of one getValue call
+ * cannot affect the parameters of the next getValue call, the two keys can form a dependency
+ * group and the two getValue calls merged into one getValues call.
+ *
+ * <p>This means that on subsequent builds, when checking to see if a value requires rebuilding,
+ * all the values in this group may be simultaneously checked. A SkyFunction should request a
+ * dependency group if checking the deps serially on a subsequent build would take too long, and
+ * if the builder would request all deps anyway as long as no earlier deps had changed.
+ * SkyFunction.Environment implementations may also choose to request these deps in
+ * parallel on the first build, potentially speeding up the build.
+ *
+ * <p>While re-evaluating every value in the group may take longer than re-evaluating just the
+ * first one and finding that it has changed, no extra work is done: the contract of the
+ * dependency group means that the builder, when called to rebuild this value, will request all
+ * values in the group again anyway, so they would have to have been built in any case.
+ *
+ * <p>Example of when to use getValues: A ListProcessor value is built with key inputListRef.
+ * The builder first calls getValue(InputList.key(inputListRef)), and retrieves inputList. It
+ * then iterates through inputList, calling getValue on each input. Finally, it processes the
+ * whole list and returns. Say inputList is (a, b, c). Since the builder will unconditionally
+ * call getValue(a), getValue(b), and getValue(c), the builder can instead just call
+ * getValues({a, b, c}). If the value is later dirtied the evaluator will build a, b, and c in
+ * parallel (assuming the inputList value was unchanged), and re-evaluate the ListProcessor
+ * value only if at least one of them was changed. On the other hand, if the InputList changes
+ * to be (a, b, d), then the evaluator will see that the first dep has changed, and call the
+ * builder to rebuild from scratch, without considering the dep group of {a, b, c}.
+ *
+ * <p>Example of when not to use getValues: A BestMatch value is built with key
+ * <potentialMatchesRef, matchCriterion>. The builder first calls
+ * getValue(PotentialMatches.key(potentialMatchesRef) and retrieves potentialMatches. It then
+ * iterates through potentialMatches, calling getValue on each potential match until it finds
+ * one that satisfies matchCriterion. In this case, if potentialMatches is (a, b, c), it would
+ * be <i>incorrect</i> to call getValues({a, b, c}), because it is not known yet whether
+ * requesting b or c will be necessary -- if a matches, then we will never call b or c.
+ */
+ Map<SkyKey, SkyValue> getValues(Iterable<SkyKey> depKeys);
+
+ /**
+ * The same as {@link #getValues} but the returned objects may throw when attempting to retrieve
+ * their value. Note that even if the requested values can throw different kinds of exceptions,
+ * only exceptions of type {@code E} will be preserved in the returned objects. All others will
+ * be null.
+ */
+ <E extends Exception> Map<SkyKey, ValueOrException<E>> getValuesOrThrow(
+ Iterable<SkyKey> depKeys, Class<E> exceptionClass);
+ <E1 extends Exception, E2 extends Exception> Map<SkyKey, ValueOrException2<E1, E2>>
+ getValuesOrThrow(Iterable<SkyKey> depKeys, Class<E1> exceptionClass1,
+ Class<E2> exceptionClass2);
+ <E1 extends Exception, E2 extends Exception, E3 extends Exception>
+ Map<SkyKey, ValueOrException3<E1, E2, E3>> getValuesOrThrow(Iterable<SkyKey> depKeys,
+ Class<E1> exceptionClass1, Class<E2> exceptionClass2, Class<E3> exceptionClass3);
+ <E1 extends Exception, E2 extends Exception, E3 extends Exception, E4 extends Exception>
+ Map<SkyKey, ValueOrException4<E1, E2, E3, E4>> getValuesOrThrow(Iterable<SkyKey> depKeys,
+ Class<E1> exceptionClass1, Class<E2> exceptionClass2, Class<E3> exceptionClass3,
+ Class<E4> exceptionClass4);
+
+ /**
+ * Returns the {@link EventHandler} that a SkyFunction should use to print any errors,
+ * warnings, or progress messages while building.
+ */
+ EventHandler getListener();
+
+ /** Returns whether we are currently in error bubbling. */
+ @VisibleForTesting
+ boolean inErrorBubblingForTesting();
+ }
+}
diff --git a/src/main/java/com/google/devtools/build/skyframe/SkyFunctionException.java b/src/main/java/com/google/devtools/build/skyframe/SkyFunctionException.java
new file mode 100644
index 0000000..71b4710
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/SkyFunctionException.java
@@ -0,0 +1,133 @@
+// Copyright 2014 Google Inc. 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.skyframe;
+
+
+import com.google.common.base.Preconditions;
+
+import javax.annotation.Nullable;
+
+/**
+ * Base class of exceptions thrown by {@link SkyFunction#compute} on failure.
+ *
+ * SkyFunctions should declare a subclass {@code C} of {@link SkyFunctionException} whose
+ * constructors forward fine-grained exception types (e.g. {@link IOException}) to
+ * {@link SkyFunctionException}'s constructor, and they should also declare
+ * {@link SkyFunction#compute} to throw {@code C}. This way the type system checks that no
+ * unexpected exceptions are thrown by the {@link SkyFunction}.
+ *
+ * <p>We took this approach over using a generic exception class since Java disallows it because of
+ * type erasure
+ * (see http://docs.oracle.com/javase/tutorial/java/generics/restrictions.html#cannotCatch).
+ *
+ * <p> Note that there are restrictions on what Exception types are allowed to be wrapped in this
+ * manner. See {@link SkyFunctionException#validateExceptionType}.
+ *
+ * <p>Failures are explicitly either transient or persistent. The transience of the failure from
+ * {@link SkyFunction#compute} should be influenced only by the computations done, and not by the
+ * transience of the failures from computations requested via
+ * {@link SkyFunction.Environment#getValueOrThrow}.
+ */
+public abstract class SkyFunctionException extends Exception {
+
+ /** The transience of the error. */
+ public enum Transience {
+ // An error that may or may not occur again if the computation were re-run. If a computation
+ // results in a transient error and is needed on a subsequent MemoizingEvaluator#evaluate call,
+ // it will be re-executed.
+ TRANSIENT,
+
+ // An error that is completely deterministic and persistent in terms of the computation's
+ // inputs. Persistent errors may be cached.
+ PERSISTENT;
+ }
+
+ private final Transience transience;
+ @Nullable
+ private final SkyKey rootCause;
+
+ public SkyFunctionException(Exception cause, Transience transience) {
+ this(cause, transience, null);
+ }
+
+ /** Used to rethrow a child error that the parent cannot handle. */
+ public SkyFunctionException(Exception cause, SkyKey childKey) {
+ this(cause, Transience.PERSISTENT, childKey);
+ }
+
+ private SkyFunctionException(Exception cause, Transience transience, SkyKey rootCause) {
+ super(Preconditions.checkNotNull(cause));
+ SkyFunctionException.validateExceptionType(cause.getClass());
+ this.transience = transience;
+ this.rootCause = rootCause;
+ }
+
+ @Nullable
+ final SkyKey getRootCauseSkyKey() {
+ return rootCause;
+ }
+
+ final boolean isTransient() {
+ return transience == Transience.TRANSIENT;
+ }
+
+ /**
+ * Catastrophic failures halt the build even when in keepGoing mode.
+ */
+ public boolean isCatastrophic() {
+ return false;
+ }
+
+ @Override
+ public Exception getCause() {
+ return (Exception) super.getCause();
+ }
+
+ static <E extends Throwable> void validateExceptionType(Class<E> exceptionClass) {
+ if (exceptionClass.equals(ValueOrExceptionUtils.BottomException.class)) {
+ return;
+ }
+
+ if (exceptionClass.isAssignableFrom(RuntimeException.class)) {
+ throw new IllegalStateException(exceptionClass.getSimpleName() + " is a supertype of "
+ + "RuntimeException. Don't do this since then you would potentially swallow all "
+ + "RuntimeExceptions, even those from Skyframe");
+ }
+ if (RuntimeException.class.isAssignableFrom(exceptionClass)) {
+ throw new IllegalStateException(exceptionClass.getSimpleName() + " is a subtype of "
+ + "RuntimeException. You should rewrite your code to use checked exceptions.");
+ }
+ if (InterruptedException.class.isAssignableFrom(exceptionClass)) {
+ throw new IllegalStateException(exceptionClass.getSimpleName() + " is a subtype of "
+ + "InterruptedException. Don't do this; Skyframe handles interrupts separately from the "
+ + "general SkyFunctionException mechanism.");
+ }
+ }
+
+ /** A {@link SkyFunctionException} with a definite root cause. */
+ static class ReifiedSkyFunctionException extends SkyFunctionException {
+ private final boolean isCatastrophic;
+
+ ReifiedSkyFunctionException(SkyFunctionException e, SkyKey key) {
+ super(e.getCause(), e.transience, Preconditions.checkNotNull(e.getRootCauseSkyKey() == null
+ ? key : e.getRootCauseSkyKey()));
+ this.isCatastrophic = e.isCatastrophic();
+ }
+
+ @Override
+ public boolean isCatastrophic() {
+ return isCatastrophic;
+ }
+ }
+}
diff --git a/src/main/java/com/google/devtools/build/skyframe/SkyFunctionName.java b/src/main/java/com/google/devtools/build/skyframe/SkyFunctionName.java
new file mode 100644
index 0000000..389d4d8
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/SkyFunctionName.java
@@ -0,0 +1,90 @@
+// Copyright 2014 Google Inc. 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.skyframe;
+
+import com.google.common.base.Predicate;
+
+import java.io.Serializable;
+import java.util.Set;
+
+/**
+ * An identifier for a {@code SkyFunction}.
+ */
+public final class SkyFunctionName implements Serializable {
+ public static SkyFunctionName computed(String name) {
+ return new SkyFunctionName(name, true);
+ }
+
+ private final String name;
+ private final boolean isComputed;
+
+ public SkyFunctionName(String name, boolean isComputed) {
+ this.name = name;
+ this.isComputed = isComputed;
+ }
+
+ @Override
+ public String toString() {
+ return name;
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (this == obj) {
+ return true;
+ }
+ if (!(obj instanceof SkyFunctionName)) {
+ return false;
+ }
+ SkyFunctionName other = (SkyFunctionName) obj;
+ return name.equals(other.name);
+ }
+
+ @Override
+ public int hashCode() {
+ return name.hashCode();
+ }
+
+ /**
+ * Returns whether the values of this type are computed. The computation of a computed value must
+ * be deterministic and may only access requested dependencies.
+ */
+ public boolean isComputed() {
+ return isComputed;
+ }
+
+ /**
+ * A predicate that returns true for {@link SkyKey}s that have the given {@link SkyFunctionName}.
+ */
+ public static Predicate<SkyKey> functionIs(final SkyFunctionName functionName) {
+ return new Predicate<SkyKey>() {
+ @Override
+ public boolean apply(SkyKey skyKey) {
+ return functionName.equals(skyKey.functionName());
+ }
+ };
+ }
+
+ /**
+ * A predicate that returns true for {@link SkyKey}s that have the given {@link SkyFunctionName}.
+ */
+ public static Predicate<SkyKey> functionIsIn(final Set<SkyFunctionName> functionNames) {
+ return new Predicate<SkyKey>() {
+ @Override
+ public boolean apply(SkyKey skyKey) {
+ return functionNames.contains(skyKey.functionName());
+ }
+ };
+ }
+}
diff --git a/src/main/java/com/google/devtools/build/skyframe/SkyKey.java b/src/main/java/com/google/devtools/build/skyframe/SkyKey.java
new file mode 100644
index 0000000..cc1dd1f
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/SkyKey.java
@@ -0,0 +1,86 @@
+// Copyright 2014 Google Inc. 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.skyframe;
+
+import com.google.common.base.Function;
+import com.google.common.base.Preconditions;
+
+import java.io.Serializable;
+
+/**
+ * A {@link SkyKey} is effectively a pair (type, name) that identifies a Skyframe value.
+ */
+public final class SkyKey implements Serializable {
+ private final SkyFunctionName functionName;
+
+ /**
+ * The name of the value.
+ *
+ * <p>This is deliberately an untyped Object so that we can use arbitrary value types (e.g.,
+ * Labels, PathFragments, BuildConfigurations, etc.) as value names without incurring
+ * serialization costs in the in-memory implementation of the graph.
+ */
+ private final Object argument;
+
+ /**
+ * Cache the hash code for this object. It might be expensive to compute.
+ */
+ private final int hashCode;
+
+ public SkyKey(SkyFunctionName functionName, Object valueName) {
+ this.functionName = Preconditions.checkNotNull(functionName);
+ this.argument = Preconditions.checkNotNull(valueName);
+ this.hashCode = 31 * functionName.hashCode() + argument.hashCode();
+ }
+
+ public SkyFunctionName functionName() {
+ return functionName;
+ }
+
+ public Object argument() {
+ return argument;
+ }
+
+ @Override
+ public String toString() {
+ return functionName + ":" + argument;
+ }
+
+ @Override
+ public int hashCode() {
+ return hashCode;
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (this == obj) {
+ return true;
+ }
+ if (obj == null) {
+ return false;
+ }
+ if (getClass() != obj.getClass()) {
+ return false;
+ }
+ SkyKey other = (SkyKey) obj;
+ return argument.equals(other.argument) && functionName.equals(other.functionName);
+ }
+
+ public static final Function<SkyKey, Object> NODE_NAME = new Function<SkyKey, Object>() {
+ @Override
+ public Object apply(SkyKey input) {
+ return input.argument();
+ }
+ };
+}
diff --git a/src/main/java/com/google/devtools/build/skyframe/SkyValue.java b/src/main/java/com/google/devtools/build/skyframe/SkyValue.java
new file mode 100644
index 0000000..7cfaa78
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/SkyValue.java
@@ -0,0 +1,22 @@
+// Copyright 2014 Google Inc. 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.skyframe;
+
+import java.io.Serializable;
+
+/**
+ * A return value of a {@code SkyFunction}.
+ */
+public interface SkyValue extends Serializable {
+}
diff --git a/src/main/java/com/google/devtools/build/skyframe/TaggedEvents.java b/src/main/java/com/google/devtools/build/skyframe/TaggedEvents.java
new file mode 100644
index 0000000..056175e
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/TaggedEvents.java
@@ -0,0 +1,62 @@
+// Copyright 2014 Google Inc. 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.skyframe;
+
+import com.google.common.collect.ImmutableCollection;
+import com.google.common.collect.Iterables;
+import com.google.devtools.build.lib.events.Event;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.Immutable;
+
+/**
+ * A wrapper of {@link Event} that contains a tag of the label where the event was generated. This
+ * class allows us to tell where the events are coming from when we group all the tags in a
+ * NestedSet.
+ *
+ * <p>The only usage of this code for now is to be able to use --output_filter in Skyframe
+ *
+ * <p>This is intended only for use in alternative {@code MemoizingEvaluator} implementations.
+ */
+@Immutable
+public final class TaggedEvents {
+
+ @Nullable
+ private final String tag;
+ private final ImmutableCollection<Event> events;
+
+ TaggedEvents(@Nullable String tag, ImmutableCollection<Event> events) {
+
+ this.tag = tag;
+ this.events = events;
+ }
+
+ @Nullable
+ String getTag() {
+ return tag;
+ }
+
+ ImmutableCollection<Event> getEvents() {
+ return events;
+ }
+
+ /**
+ * Returns <i>some</i> moderately sane representation of the events. Should never be used in
+ * user-visible places, only for debugging and testing.
+ */
+ @Override
+ public String toString() {
+ return tag == null ? "<unknown>" : tag + ": " + Iterables.toString(events);
+ }
+}
diff --git a/src/main/java/com/google/devtools/build/skyframe/ValueOrException.java b/src/main/java/com/google/devtools/build/skyframe/ValueOrException.java
new file mode 100644
index 0000000..d682095
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/ValueOrException.java
@@ -0,0 +1,24 @@
+// Copyright 2014 Google Inc. 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.skyframe;
+
+import javax.annotation.Nullable;
+
+/** Wrapper for a value or the typed exception thrown when trying to compute it. */
+public abstract class ValueOrException<E extends Exception> extends ValueOrUntypedException {
+
+ /** Gets the stored value. Throws an exception if one was thrown when computing this value. */
+ @Nullable
+ public abstract SkyValue get() throws E;
+}
diff --git a/src/main/java/com/google/devtools/build/skyframe/ValueOrException2.java b/src/main/java/com/google/devtools/build/skyframe/ValueOrException2.java
new file mode 100644
index 0000000..deedbb1
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/ValueOrException2.java
@@ -0,0 +1,25 @@
+// Copyright 2014 Google Inc. 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.skyframe;
+
+import javax.annotation.Nullable;
+
+/** Wrapper for a value or the typed exception thrown when trying to compute it. */
+public abstract class ValueOrException2<E1 extends Exception, E2 extends Exception>
+ extends ValueOrUntypedException {
+
+ /** Gets the stored value. Throws an exception if one was thrown when computing this value. */
+ @Nullable
+ public abstract SkyValue get() throws E1, E2;
+}
diff --git a/src/main/java/com/google/devtools/build/skyframe/ValueOrException3.java b/src/main/java/com/google/devtools/build/skyframe/ValueOrException3.java
new file mode 100644
index 0000000..e737c55
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/ValueOrException3.java
@@ -0,0 +1,25 @@
+// Copyright 2014 Google Inc. 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.skyframe;
+
+import javax.annotation.Nullable;
+
+/** Wrapper for a value or the typed exception thrown when trying to compute it. */
+public abstract class ValueOrException3<E1 extends Exception, E2 extends Exception,
+ E3 extends Exception> extends ValueOrUntypedException {
+
+ /** Gets the stored value. Throws an exception if one was thrown when computing this value. */
+ @Nullable
+ public abstract SkyValue get() throws E1, E2, E3;
+}
diff --git a/src/main/java/com/google/devtools/build/skyframe/ValueOrException4.java b/src/main/java/com/google/devtools/build/skyframe/ValueOrException4.java
new file mode 100644
index 0000000..176f405
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/ValueOrException4.java
@@ -0,0 +1,25 @@
+// Copyright 2014 Google Inc. 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.skyframe;
+
+import javax.annotation.Nullable;
+
+/** Wrapper for a value or the typed exception thrown when trying to compute it. */
+public abstract class ValueOrException4<E1 extends Exception, E2 extends Exception,
+ E3 extends Exception, E4 extends Exception> extends ValueOrUntypedException {
+
+ /** Gets the stored value. Throws an exception if one was thrown when computing this value. */
+ @Nullable
+ public abstract SkyValue get() throws E1, E2, E3, E4;
+}
diff --git a/src/main/java/com/google/devtools/build/skyframe/ValueOrExceptionUtils.java b/src/main/java/com/google/devtools/build/skyframe/ValueOrExceptionUtils.java
new file mode 100644
index 0000000..e66f4fa
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/ValueOrExceptionUtils.java
@@ -0,0 +1,520 @@
+// Copyright 2014 Google Inc. 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.skyframe;
+
+import javax.annotation.Nullable;
+
+/** Utilities for producing and consuming ValueOrException(2|3|4)? instances. */
+class ValueOrExceptionUtils {
+
+ /** The bottom exception type. */
+ class BottomException extends Exception {
+ }
+
+ @Nullable
+ public static SkyValue downcovert(ValueOrException<BottomException> voe) {
+ return voe.getValue();
+ }
+
+ public static <E1 extends Exception> ValueOrException<E1> downcovert(
+ ValueOrException2<E1, BottomException> voe, Class<E1> exceptionClass1) {
+ Exception e = voe.getException();
+ if (e == null) {
+ return new ValueOrExceptionValueImpl<>(voe.getValue());
+ }
+ // Here and below, we use type-safe casts for performance reasons. Another approach would be
+ // cascading try-catch-rethrow blocks, but that has a higher performance penalty.
+ if (exceptionClass1.isInstance(e)) {
+ return new ValueOrExceptionExnImpl<>(exceptionClass1.cast(e));
+ }
+ throw new IllegalStateException("shouldn't reach here " + e.getClass() + " " + exceptionClass1,
+ e);
+ }
+
+ public static <E1 extends Exception, E2 extends Exception> ValueOrException2<E1, E2> downconvert(
+ ValueOrException3<E1, E2, BottomException> voe, Class<E1> exceptionClass1,
+ Class<E2> exceptionClass2) {
+ Exception e = voe.getException();
+ if (e == null) {
+ return new ValueOrException2ValueImpl<>(voe.getValue());
+ }
+ if (exceptionClass1.isInstance(e)) {
+ return new ValueOrException2Exn1Impl<>(exceptionClass1.cast(e));
+ }
+ if (exceptionClass2.isInstance(e)) {
+ return new ValueOrException2Exn2Impl<>(exceptionClass2.cast(e));
+ }
+ throw new IllegalStateException("shouldn't reach here " + e.getClass() + " " + exceptionClass1
+ + " " + exceptionClass2, e);
+ }
+
+ public static <E1 extends Exception, E2 extends Exception, E3 extends Exception>
+ ValueOrException3<E1, E2, E3> downconvert(ValueOrException4<E1, E2, E3, BottomException> voe,
+ Class<E1> exceptionClass1, Class<E2> exceptionClass2, Class<E3> exceptionClass3) {
+ Exception e = voe.getException();
+ if (e == null) {
+ return new ValueOrException3ValueImpl<>(voe.getValue());
+ }
+ if (exceptionClass1.isInstance(e)) {
+ return new ValueOrException3Exn1Impl<>(exceptionClass1.cast(e));
+ }
+ if (exceptionClass2.isInstance(e)) {
+ return new ValueOrException3Exn2Impl<>(exceptionClass2.cast(e));
+ }
+ if (exceptionClass3.isInstance(e)) {
+ return new ValueOrException3Exn3Impl<>(exceptionClass3.cast(e));
+ }
+ throw new IllegalStateException("shouldn't reach here " + e.getClass() + " " + exceptionClass1
+ + " " + exceptionClass2 + " " + exceptionClass3, e);
+ }
+
+ public static <E extends Exception> ValueOrException<E> ofNull() {
+ return ValueOrExceptionValueImpl.ofNull();
+ }
+
+ public static ValueOrUntypedException ofValueUntyped(SkyValue value) {
+ return new ValueOrUntypedExceptionImpl(value);
+ }
+
+ public static <E extends Exception> ValueOrException<E> ofExn(E e) {
+ return new ValueOrExceptionExnImpl<>(e);
+ }
+
+ public static <E1 extends Exception, E2 extends Exception, E3 extends Exception,
+ E4 extends Exception> ValueOrException4<E1, E2, E3, E4> ofNullValue() {
+ return ValueOrException4ValueImpl.ofNullValue();
+ }
+
+ public static <E1 extends Exception, E2 extends Exception, E3 extends Exception,
+ E4 extends Exception> ValueOrException4<E1, E2, E3, E4> ofValue(SkyValue value) {
+ return new ValueOrException4ValueImpl<>(value);
+ }
+
+ public static <E1 extends Exception, E2 extends Exception, E3 extends Exception,
+ E4 extends Exception> ValueOrException4<E1, E2, E3, E4> ofExn1(E1 e) {
+ return new ValueOrException4Exn1Impl<>(e);
+ }
+
+ public static <E1 extends Exception, E2 extends Exception, E3 extends Exception,
+ E4 extends Exception> ValueOrException4<E1, E2, E3, E4> ofExn2(E2 e) {
+ return new ValueOrException4Exn2Impl<>(e);
+ }
+
+ public static <E1 extends Exception, E2 extends Exception, E3 extends Exception,
+ E4 extends Exception> ValueOrException4<E1, E2, E3, E4> ofExn3(E3 e) {
+ return new ValueOrException4Exn3Impl<>(e);
+ }
+
+ public static <E1 extends Exception, E2 extends Exception, E3 extends Exception,
+ E4 extends Exception> ValueOrException4<E1, E2, E3, E4> ofExn4(E4 e) {
+ return new ValueOrException4Exn4Impl<>(e);
+ }
+
+ private static class ValueOrUntypedExceptionImpl extends ValueOrUntypedException {
+ @Nullable
+ private final SkyValue value;
+
+ ValueOrUntypedExceptionImpl(@Nullable SkyValue value) {
+ this.value = value;
+ }
+
+ @Override
+ @Nullable
+ public SkyValue getValue() {
+ return value;
+ }
+
+ @Override
+ public Exception getException() {
+ return null;
+ }
+ }
+
+ private static class ValueOrExceptionValueImpl<E extends Exception> extends ValueOrException<E> {
+ private static final ValueOrExceptionValueImpl<Exception> NULL =
+ new ValueOrExceptionValueImpl<Exception>((SkyValue) null);
+
+ @Nullable
+ private final SkyValue value;
+
+ private ValueOrExceptionValueImpl(@Nullable SkyValue value) {
+ this.value = value;
+ }
+
+ @Override
+ @Nullable
+ public SkyValue get() {
+ return value;
+ }
+
+ @Override
+ @Nullable
+ public SkyValue getValue() {
+ return value;
+ }
+
+ @Override
+ @Nullable
+ public Exception getException() {
+ return null;
+ }
+
+ @SuppressWarnings("unchecked")
+ public static <E extends Exception> ValueOrExceptionValueImpl<E> ofNull() {
+ return (ValueOrExceptionValueImpl<E>) NULL;
+ }
+ }
+
+ private static class ValueOrExceptionExnImpl<E extends Exception> extends ValueOrException<E> {
+ private final E e;
+
+ private ValueOrExceptionExnImpl(E e) {
+ this.e = e;
+ }
+
+ @Override
+ public SkyValue get() throws E {
+ throw e;
+ }
+
+ @Override
+ @Nullable
+ public SkyValue getValue() {
+ return null;
+ }
+
+ @Override
+ public Exception getException() {
+ return e;
+ }
+ }
+
+ private static class ValueOrException2ValueImpl<E1 extends Exception, E2 extends Exception>
+ extends ValueOrException2<E1, E2> {
+ @Nullable
+ private final SkyValue value;
+
+ ValueOrException2ValueImpl(@Nullable SkyValue value) {
+ this.value = value;
+ }
+
+ @Override
+ @Nullable
+ public SkyValue get() throws E1, E2 {
+ return value;
+ }
+
+ @Override
+ @Nullable
+ public Exception getException() {
+ return null;
+ }
+
+ @Override
+ @Nullable
+ public SkyValue getValue() {
+ return value;
+ }
+ }
+
+ private static class ValueOrException2Exn1Impl<E1 extends Exception, E2 extends Exception>
+ extends ValueOrException2<E1, E2> {
+ private final E1 e;
+
+ private ValueOrException2Exn1Impl(E1 e) {
+ this.e = e;
+ }
+
+ @Override
+ public SkyValue get() throws E1 {
+ throw e;
+ }
+
+ @Override
+ public Exception getException() {
+ return e;
+ }
+
+ @Override
+ @Nullable
+ public SkyValue getValue() {
+ return null;
+ }
+ }
+
+ private static class ValueOrException2Exn2Impl<E1 extends Exception, E2 extends Exception>
+ extends ValueOrException2<E1, E2> {
+ private final E2 e;
+
+ private ValueOrException2Exn2Impl(E2 e) {
+ this.e = e;
+ }
+
+ @Override
+ public SkyValue get() throws E2 {
+ throw e;
+ }
+
+ @Override
+ public Exception getException() {
+ return e;
+ }
+
+ @Override
+ @Nullable
+ public SkyValue getValue() {
+ return null;
+ }
+ }
+
+ private static class ValueOrException3ValueImpl<E1 extends Exception, E2 extends Exception,
+ E3 extends Exception> extends ValueOrException3<E1, E2, E3> {
+ @Nullable
+ private final SkyValue value;
+
+ ValueOrException3ValueImpl(@Nullable SkyValue value) {
+ this.value = value;
+ }
+
+ @Override
+ @Nullable
+ public SkyValue get() throws E1, E2 {
+ return value;
+ }
+
+ @Override
+ @Nullable
+ public Exception getException() {
+ return null;
+ }
+
+ @Override
+ @Nullable
+ public SkyValue getValue() {
+ return value;
+ }
+ }
+
+ private static class ValueOrException3Exn1Impl<E1 extends Exception, E2 extends Exception,
+ E3 extends Exception> extends ValueOrException3<E1, E2, E3> {
+ private final E1 e;
+
+ private ValueOrException3Exn1Impl(E1 e) {
+ this.e = e;
+ }
+
+ @Override
+ public SkyValue get() throws E1 {
+ throw e;
+ }
+
+ @Override
+ public Exception getException() {
+ return e;
+ }
+
+ @Override
+ @Nullable
+ public SkyValue getValue() {
+ return null;
+ }
+ }
+
+ private static class ValueOrException3Exn2Impl<E1 extends Exception, E2 extends Exception,
+ E3 extends Exception> extends ValueOrException3<E1, E2, E3> {
+ private final E2 e;
+
+ private ValueOrException3Exn2Impl(E2 e) {
+ this.e = e;
+ }
+
+ @Override
+ public SkyValue get() throws E2 {
+ throw e;
+ }
+
+ @Override
+ public Exception getException() {
+ return e;
+ }
+
+ @Override
+ @Nullable
+ public SkyValue getValue() {
+ return null;
+ }
+ }
+
+ private static class ValueOrException3Exn3Impl<E1 extends Exception, E2 extends Exception,
+ E3 extends Exception> extends ValueOrException3<E1, E2, E3> {
+ private final E3 e;
+
+ private ValueOrException3Exn3Impl(E3 e) {
+ this.e = e;
+ }
+
+ @Override
+ public SkyValue get() throws E3 {
+ throw e;
+ }
+
+ @Override
+ public Exception getException() {
+ return e;
+ }
+
+ @Override
+ @Nullable
+ public SkyValue getValue() {
+ return null;
+ }
+ }
+
+ private static class ValueOrException4ValueImpl<E1 extends Exception, E2 extends Exception,
+ E3 extends Exception, E4 extends Exception> extends ValueOrException4<E1, E2, E3, E4> {
+ private static final ValueOrException4ValueImpl<Exception, Exception, Exception,
+ Exception> NULL = new ValueOrException4ValueImpl<>((SkyValue) null);
+
+ @Nullable
+ private final SkyValue value;
+
+ ValueOrException4ValueImpl(@Nullable SkyValue value) {
+ this.value = value;
+ }
+
+ @Override
+ @Nullable
+ public SkyValue get() throws E1, E2 {
+ return value;
+ }
+
+ @Override
+ @Nullable
+ public Exception getException() {
+ return null;
+ }
+
+ @Override
+ @Nullable
+ public SkyValue getValue() {
+ return value;
+ }
+
+ @SuppressWarnings("unchecked")
+ private static <E1 extends Exception, E2 extends Exception, E3 extends Exception,
+ E4 extends Exception>ValueOrException4ValueImpl<E1, E2, E3, E4> ofNullValue() {
+ return (ValueOrException4ValueImpl<E1, E2, E3, E4>) NULL;
+ }
+ }
+
+ private static class ValueOrException4Exn1Impl<E1 extends Exception, E2 extends Exception,
+ E3 extends Exception, E4 extends Exception> extends ValueOrException4<E1, E2, E3, E4> {
+ private final E1 e;
+
+ private ValueOrException4Exn1Impl(E1 e) {
+ this.e = e;
+ }
+
+ @Override
+ public SkyValue get() throws E1 {
+ throw e;
+ }
+
+ @Override
+ public Exception getException() {
+ return e;
+ }
+
+ @Override
+ @Nullable
+ public SkyValue getValue() {
+ return null;
+ }
+ }
+
+ private static class ValueOrException4Exn2Impl<E1 extends Exception, E2 extends Exception,
+ E3 extends Exception, E4 extends Exception> extends ValueOrException4<E1, E2, E3, E4> {
+ private final E2 e;
+
+ private ValueOrException4Exn2Impl(E2 e) {
+ this.e = e;
+ }
+
+ @Override
+ public SkyValue get() throws E2 {
+ throw e;
+ }
+
+ @Override
+ public Exception getException() {
+ return e;
+ }
+
+ @Override
+ @Nullable
+ public SkyValue getValue() {
+ return null;
+ }
+ }
+
+ private static class ValueOrException4Exn3Impl<E1 extends Exception, E2 extends Exception,
+ E3 extends Exception, E4 extends Exception> extends ValueOrException4<E1, E2, E3, E4> {
+ private final E3 e;
+
+ private ValueOrException4Exn3Impl(E3 e) {
+ this.e = e;
+ }
+
+ @Override
+ public SkyValue get() throws E3 {
+ throw e;
+ }
+
+ @Override
+ public Exception getException() {
+ return e;
+ }
+
+ @Override
+ @Nullable
+ public SkyValue getValue() {
+ return null;
+ }
+ }
+
+ private static class ValueOrException4Exn4Impl<E1 extends Exception, E2 extends Exception,
+ E3 extends Exception, E4 extends Exception> extends ValueOrException4<E1, E2, E3, E4> {
+ private final E4 e;
+
+ private ValueOrException4Exn4Impl(E4 e) {
+ this.e = e;
+ }
+
+ @Override
+ public SkyValue get() throws E4 {
+ throw e;
+ }
+
+ @Override
+ public Exception getException() {
+ return e;
+ }
+
+ @Override
+ @Nullable
+ public SkyValue getValue() {
+ return null;
+ }
+ }
+}
diff --git a/src/main/java/com/google/devtools/build/skyframe/ValueOrUntypedException.java b/src/main/java/com/google/devtools/build/skyframe/ValueOrUntypedException.java
new file mode 100644
index 0000000..c7ea7d4
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/ValueOrUntypedException.java
@@ -0,0 +1,34 @@
+// Copyright 2014 Google Inc. 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.skyframe;
+
+import javax.annotation.Nullable;
+
+/**
+ * Wrapper for a value or the untyped exception thrown when trying to compute it.
+ *
+ * <p>This is an implementation detail of {@link ParallelEvaluator} and
+ * {@link ValueOrExceptionUtils}. It's an abstract class (as opposed to an interface) to avoid
+ * exposing the methods outside the package.
+ */
+abstract class ValueOrUntypedException {
+
+ /** Returns the stored value, if there was one. */
+ @Nullable
+ abstract SkyValue getValue();
+
+ /** Returns the stored exception, if there was one. */
+ @Nullable
+ abstract Exception getException();
+}
diff --git a/src/main/java/com/google/devtools/build/skyframe/ValueWithMetadata.java b/src/main/java/com/google/devtools/build/skyframe/ValueWithMetadata.java
new file mode 100644
index 0000000..956e404
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/ValueWithMetadata.java
@@ -0,0 +1,209 @@
+// Copyright 2014 Google Inc. 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
+package com.google.devtools.build.skyframe;
+
+import com.google.common.base.Preconditions;
+import com.google.devtools.build.lib.collect.nestedset.NestedSet;
+import com.google.devtools.build.lib.collect.nestedset.NestedSetBuilder;
+import com.google.devtools.build.lib.collect.nestedset.Order;
+
+import java.util.Objects;
+
+import javax.annotation.Nullable;
+
+/**
+ * Encapsulation of data stored by {@link NodeEntry} when the value has finished building.
+ *
+ * <p>This is intended only for use in alternative {@code MemoizingEvaluator} implementations.
+ */
+public abstract class ValueWithMetadata implements SkyValue {
+ protected final SkyValue value;
+
+ private static final NestedSet<TaggedEvents> NO_EVENTS =
+ NestedSetBuilder.<TaggedEvents>emptySet(Order.STABLE_ORDER);
+
+ public ValueWithMetadata(SkyValue value) {
+ this.value = value;
+ }
+
+ /** Builds a value entry value that has an error (and no value value).
+ *
+ * <p>This is intended only for use in alternative {@code MemoizingEvaluator} implementations.
+ */
+ public static ValueWithMetadata error(ErrorInfo errorInfo,
+ NestedSet<TaggedEvents> transitiveEvents) {
+ return new ErrorInfoValue(errorInfo, null, transitiveEvents);
+ }
+
+ /**
+ * Builds a value entry value that has a value value, and possibly an error (constructed from its
+ * children's errors).
+ *
+ * <p>This is intended only for use in alternative {@code MemoizingEvaluator} implementations.
+ */
+ static SkyValue normal(@Nullable SkyValue value, @Nullable ErrorInfo errorInfo,
+ NestedSet<TaggedEvents> transitiveEvents) {
+ Preconditions.checkState(value != null || errorInfo != null,
+ "Value and error cannot both be null");
+ if (errorInfo == null) {
+ return transitiveEvents.isEmpty()
+ ? value
+ : new ValueWithEvents(value, transitiveEvents);
+ }
+ return new ErrorInfoValue(errorInfo, value, transitiveEvents);
+ }
+
+
+ @Nullable SkyValue getValue() {
+ return value;
+ }
+
+ @Nullable
+ abstract ErrorInfo getErrorInfo();
+
+ abstract NestedSet<TaggedEvents> getTransitiveEvents();
+
+ static final class ValueWithEvents extends ValueWithMetadata {
+
+ private final NestedSet<TaggedEvents> transitiveEvents;
+
+ ValueWithEvents(SkyValue value, NestedSet<TaggedEvents> transitiveEvents) {
+ super(Preconditions.checkNotNull(value));
+ this.transitiveEvents = Preconditions.checkNotNull(transitiveEvents);
+ }
+
+ @Nullable
+ @Override
+ ErrorInfo getErrorInfo() { return null; }
+
+ @Override
+ NestedSet<TaggedEvents> getTransitiveEvents() { return transitiveEvents; }
+
+ /**
+ * We override equals so that if the same value is written to a {@link NodeEntry} twice, it can
+ * verify that the two values are equal, and avoid incrementing its version.
+ */
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+
+ ValueWithEvents that = (ValueWithEvents) o;
+
+ // Shallow equals is a middle ground between using default equals, which might miss
+ // nested sets with the same elements, and deep equality checking, which would be expensive.
+ // All three choices are sound, since shallow equals and default equals are more
+ // conservative than deep equals. Using shallow equals means that we may unnecessarily
+ // consider some values unequal that are actually equal, but this is still a net win over
+ // deep equals.
+ return value.equals(that.value) && transitiveEvents.shallowEquals(that.transitiveEvents);
+ }
+
+ @Override
+ public int hashCode() {
+ return 31 * value.hashCode() + transitiveEvents.hashCode();
+ }
+
+ @Override
+ public String toString() { return value.toString(); }
+ }
+
+ static final class ErrorInfoValue extends ValueWithMetadata {
+
+ private final ErrorInfo errorInfo;
+ private final NestedSet<TaggedEvents> transitiveEvents;
+
+ ErrorInfoValue(ErrorInfo errorInfo, @Nullable SkyValue value,
+ NestedSet<TaggedEvents> transitiveEvents) {
+ super(value);
+ this.errorInfo = Preconditions.checkNotNull(errorInfo);
+ this.transitiveEvents = Preconditions.checkNotNull(transitiveEvents);
+ }
+
+ @Nullable
+ @Override
+ ErrorInfo getErrorInfo() { return errorInfo; }
+
+ @Override
+ NestedSet<TaggedEvents> getTransitiveEvents() { return transitiveEvents; }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+
+ ErrorInfoValue that = (ErrorInfoValue) o;
+
+ // Shallow equals is a middle ground between using default equals, which might miss
+ // nested sets with the same elements, and deep equality checking, which would be expensive.
+ // All three choices are sound, since shallow equals and default equals are more
+ // conservative than deep equals. Using shallow equals means that we may unnecessarily
+ // consider some values unequal that are actually equal, but this is still a net win over
+ // deep equals.
+ return Objects.equals(this.value, that.value)
+ && Objects.equals(this.errorInfo, that.errorInfo)
+ && transitiveEvents.shallowEquals(that.transitiveEvents);
+ }
+
+ @Override
+ public int hashCode() {
+ return 31 * Objects.hash(value, errorInfo) + transitiveEvents.shallowHashCode();
+ }
+
+ @Override
+ public String toString() {
+ StringBuilder result = new StringBuilder();
+ if (value != null) {
+ result.append("Value: ").append(value);
+ }
+ if (errorInfo != null) {
+ if (result.length() > 0) {
+ result.append("; ");
+ }
+ result.append("Error: ").append(errorInfo);
+ }
+ return result.toString();
+ }
+ }
+
+ static SkyValue justValue(SkyValue value) {
+ if (value instanceof ValueWithMetadata) {
+ return ((ValueWithMetadata) value).getValue();
+ }
+ return value;
+ }
+
+ static ValueWithMetadata wrapWithMetadata(SkyValue value) {
+ if (value instanceof ValueWithMetadata) {
+ return (ValueWithMetadata) value;
+ }
+ return new ValueWithEvents(value, NO_EVENTS);
+ }
+
+ @Nullable
+ public static ErrorInfo getMaybeErrorInfo(SkyValue value) {
+ if (value.getClass() == ErrorInfoValue.class) {
+ return ((ValueWithMetadata) value).getErrorInfo();
+ }
+ return null;
+
+ }
+}
\ No newline at end of file
diff --git a/src/main/java/com/google/devtools/build/skyframe/Version.java b/src/main/java/com/google/devtools/build/skyframe/Version.java
new file mode 100644
index 0000000..90a6020
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/Version.java
@@ -0,0 +1,32 @@
+// Copyright 2014 Google Inc. 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.skyframe;
+
+/**
+ * A Version defines a value in a version tree used in persistent data structures.
+ * See http://en.wikipedia.org/wiki/Persistent_data_structure.
+ */
+public interface Version {
+ /**
+ * Defines a partial order relation on versions. Returns true if this object is at most
+ * {@code other} in that partial order. If x.equals(y), then x.atMost(y).
+ *
+ * <p>If x.atMost(y) returns false, then there are two possibilities: y < x in the partial order,
+ * so y.atMost(x) returns true and !x.equals(y), or x and y are incomparable in this partial
+ * order. This may be because x and y are instances of different Version implementations (although
+ * it is legal for different Version implementations to be comparable as well).
+ * See http://en.wikipedia.org/wiki/Partially_ordered_set.
+ */
+ boolean atMost(Version other);
+}