Add a SkyQueryEnvironment which runs queries by examining the Skyframe graph.

This environment eagerly preloads the transitive closure of a specified query "universe", and so may not be as efficient as the standard query for limited-scope queries. It is activated when the universe is specified and ordered results are not requested (since it is currently unable to order results).

Tests were modified/added to exercise this environment where deemed interesting. Some ugly hacks were done to add coverage in AbstractQueryTest and friends, because currently even if the full depot is loaded (using //...), individual target patterns most likely won't be present in the graph. A better way to deal with this situation, suggested by felly, is probably to extract target pattern resolution logic to an auxiliary function so that query is able to resolve target patterns without mutating the graph, and then call into the read-only graph with the resolved patterns. That may be done in a follow-up, in which case the "scope" of every query could be //... .

--
MOS_MIGRATED_REVID=87257028
diff --git a/src/main/java/com/google/devtools/build/lib/graph/Digraph.java b/src/main/java/com/google/devtools/build/lib/graph/Digraph.java
index bea2f5a..92e973a 100644
--- a/src/main/java/com/google/devtools/build/lib/graph/Digraph.java
+++ b/src/main/java/com/google/devtools/build/lib/graph/Digraph.java
@@ -597,7 +597,7 @@
    * starting node, returns the path from the root of the tree to 'node' as a
    * list.
    */
-  private static <X> List<X> getPathToTreeNode(Map<X, X> tree, X node) {
+  public static <X> List<X> getPathToTreeNode(Map<X, X> tree, X node) {
     List<X> path = new ArrayList<X>();
     while (node != null) {
       path.add(node);
diff --git a/src/main/java/com/google/devtools/build/lib/pkgcache/TargetPatternEvaluator.java b/src/main/java/com/google/devtools/build/lib/pkgcache/TargetPatternEvaluator.java
index 94e956b..7732519 100644
--- a/src/main/java/com/google/devtools/build/lib/pkgcache/TargetPatternEvaluator.java
+++ b/src/main/java/com/google/devtools/build/lib/pkgcache/TargetPatternEvaluator.java
@@ -43,6 +43,8 @@
  */
 @ThreadSafety.ConditionallyThreadSafe // as long as you don't call updateOffset.
 public interface TargetPatternEvaluator {
+  static FilteringPolicy DEFAULT_FILTERING_POLICY = FilteringPolicies.NO_FILTER;
+
   /**
    * Attempts to parse an ordered list of target patterns, computing the union
    * of the set of targets represented by each pattern, unless it is preceded by
@@ -78,7 +80,6 @@
       Collection<String> patterns, boolean keepGoing)
           throws TargetParsingException, InterruptedException;
 
-
   /**
    * Update the parser's offset, given the workspace and working directory.
    *
diff --git a/src/main/java/com/google/devtools/build/lib/query2/AbstractBlazeQueryEnvironment.java b/src/main/java/com/google/devtools/build/lib/query2/AbstractBlazeQueryEnvironment.java
index de56c1b..96e8509 100644
--- a/src/main/java/com/google/devtools/build/lib/query2/AbstractBlazeQueryEnvironment.java
+++ b/src/main/java/com/google/devtools/build/lib/query2/AbstractBlazeQueryEnvironment.java
@@ -34,6 +34,7 @@
 import com.google.devtools.build.lib.query2.engine.QueryExpression;
 import com.google.devtools.build.lib.syntax.Label;
 import com.google.devtools.build.lib.util.BinaryPredicate;
+import com.google.devtools.build.skyframe.WalkableGraph.WalkableGraphFactory;
 
 import java.util.Collection;
 import java.util.HashMap;
@@ -48,7 +49,6 @@
  */
 public abstract class AbstractBlazeQueryEnvironment<T> implements QueryEnvironment<T> {
   protected final ErrorSensingEventHandler eventHandler;
-  private final TargetPatternEvaluator targetPatternEvaluator;
   private final Map<String, Set<T>> letBindings = new HashMap<>();
   protected final Map<String, ResolvedTargets<Target>> resolvedTargetPatterns = new HashMap<>();
   protected final boolean keepGoing;
@@ -60,15 +60,13 @@
   private final Set<Setting> settings;
   private final List<QueryFunction> extraFunctions;
 
-  protected AbstractBlazeQueryEnvironment(TargetPatternEvaluator targetPatternEvaluator,
-      boolean keepGoing,
+  protected AbstractBlazeQueryEnvironment(boolean keepGoing,
       boolean strictScope,
       Predicate<Label> labelFilter,
       EventHandler eventHandler,
       Set<Setting> settings,
       Iterable<QueryFunction> extraFunctions) {
     this.eventHandler = new ErrorSensingEventHandler(eventHandler);
-    this.targetPatternEvaluator = targetPatternEvaluator;
     this.keepGoing = keepGoing;
     this.strictScope = strictScope;
     this.dependencyFilter = constructDependencyFilter(settings);
@@ -90,23 +88,31 @@
   }
 
   public static AbstractBlazeQueryEnvironment<Target> newQueryEnvironment(
-      TransitivePackageLoader transitivePackageLoader, PackageProvider packageProvider,
+      TransitivePackageLoader transitivePackageLoader, WalkableGraphFactory graphFactory,
+      PackageProvider packageProvider,
       TargetPatternEvaluator targetPatternEvaluator, boolean keepGoing, boolean orderedResults,
-      int loadingPhaseThreads,
+      List<String> universeScope, int loadingPhaseThreads,
       EventHandler eventHandler, Set<Setting> settings, Iterable<QueryFunction> functions) {
-    return newQueryEnvironment(transitivePackageLoader, packageProvider,
+    return newQueryEnvironment(transitivePackageLoader, graphFactory, packageProvider,
         targetPatternEvaluator, keepGoing, /*strictScope=*/true, orderedResults,
-        loadingPhaseThreads, Rule.ALL_LABELS, eventHandler, settings, functions);
+        universeScope, loadingPhaseThreads, Rule.ALL_LABELS, eventHandler, settings, functions);
   }
 
   public static AbstractBlazeQueryEnvironment<Target> newQueryEnvironment(
-      TransitivePackageLoader transitivePackageLoader, PackageProvider packageProvider,
+      TransitivePackageLoader transitivePackageLoader, WalkableGraphFactory graphFactory,
+      PackageProvider packageProvider,
       TargetPatternEvaluator targetPatternEvaluator, boolean keepGoing, boolean strictScope,
-      boolean orderedResults, int loadingPhaseThreads, Predicate<Label> labelFilter,
+      boolean orderedResults, List<String> universeScope, int loadingPhaseThreads,
+      Predicate<Label> labelFilter,
       EventHandler eventHandler, Set<Setting> settings, Iterable<QueryFunction> functions) {
-    return new BlazeQueryEnvironment(transitivePackageLoader, packageProvider,
+    Preconditions.checkNotNull(universeScope);
+    return orderedResults || universeScope.isEmpty()
+        ? new BlazeQueryEnvironment(transitivePackageLoader, packageProvider,
         targetPatternEvaluator, keepGoing, strictScope, loadingPhaseThreads,
-        labelFilter, eventHandler, settings, functions);
+        labelFilter, eventHandler, settings, functions)
+        : new SkyQueryEnvironment(
+            keepGoing, strictScope, loadingPhaseThreads, labelFilter, eventHandler, settings,
+            functions, targetPatternEvaluator.getOffset(), graphFactory, universeScope);
   }
 
   /**
@@ -119,9 +125,6 @@
    *   effect
    */
   public QueryEvalResult<T> evaluateQuery(QueryExpression expr) throws QueryException {
-    // Some errors are reported as QueryExceptions and others as ERROR events
-    // (if --keep_going).
-    eventHandler.resetErrors();
     resolvedTargetPatterns.clear();
 
     // In the --nokeep_going case, errors are reported in the order in which the patterns are
@@ -210,18 +213,8 @@
     return getTargetsMatchingPattern(caller, pattern);
   }
 
-  private Map<String, ResolvedTargets<Target>> preloadOrThrow(Collection<String> patterns)
-      throws TargetParsingException {
-    try {
-      // Note that this may throw a RuntimeException if deps are missing in Skyframe and this is
-      // being called from within a SkyFunction.
-      return targetPatternEvaluator.preloadTargetPatterns(
-          eventHandler, patterns, keepGoing);
-    } catch (InterruptedException e) {
-      // TODO(bazel-team): Propagate the InterruptedException from here [skyframe-loading].
-      throw new TargetParsingException("interrupted");
-    }
-  }
+  protected abstract Map<String, ResolvedTargets<Target>> preloadOrThrow(
+      Collection<String> patterns) throws QueryException, TargetParsingException;
 
   @Override
   public boolean isSettingEnabled(Setting setting) {
@@ -235,5 +228,4 @@
     builder.addAll(extraFunctions);
     return builder.build();
   }
-
 }
diff --git a/src/main/java/com/google/devtools/build/lib/query2/BlazeQueryEnvironment.java b/src/main/java/com/google/devtools/build/lib/query2/BlazeQueryEnvironment.java
index ede8ece..8c1c2b1 100644
--- a/src/main/java/com/google/devtools/build/lib/query2/BlazeQueryEnvironment.java
+++ b/src/main/java/com/google/devtools/build/lib/query2/BlazeQueryEnvironment.java
@@ -17,6 +17,8 @@
 import com.google.common.base.Predicate;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Iterables;
+import com.google.devtools.build.lib.cmdline.ResolvedTargets;
+import com.google.devtools.build.lib.cmdline.TargetParsingException;
 import com.google.devtools.build.lib.events.EventHandler;
 import com.google.devtools.build.lib.graph.Digraph;
 import com.google.devtools.build.lib.graph.Node;
@@ -43,6 +45,7 @@
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.LinkedHashSet;
+import java.util.Map;
 import java.util.Set;
 
 /**
@@ -51,6 +54,7 @@
 public class BlazeQueryEnvironment extends AbstractBlazeQueryEnvironment<Target> {
 
   private static final int MAX_DEPTH_FULL_SCAN_LIMIT = 20;
+  private final TargetPatternEvaluator targetPatternEvaluator;
   private final TransitivePackageLoader transitivePackageLoader;
   private final TargetProvider targetProvider;
   private final Digraph<Target> graph = new Digraph<>();
@@ -81,9 +85,8 @@
       EventHandler eventHandler,
       Set<Setting> settings,
       Iterable<QueryFunction> extraFunctions) {
-    super(targetPatternEvaluator, keepGoing, strictScope, labelFilter, eventHandler, settings,
-        extraFunctions
-    );
+    super(keepGoing, strictScope, labelFilter, eventHandler, settings, extraFunctions);
+    this.targetPatternEvaluator = targetPatternEvaluator;
     this.transitivePackageLoader = transitivePackageLoader;
     this.targetProvider = packageProvider;
     this.errorObserver = new ErrorPrintingTargetEdgeErrorObserver(this.eventHandler);
@@ -93,6 +96,10 @@
 
   @Override
   public BlazeQueryEvalResult<Target> evaluateQuery(QueryExpression expr) throws QueryException {
+    // Some errors are reported as QueryExceptions and others as ERROR events (if --keep_going). The
+    // result is set to have an error iff there were errors emitted during the query, so we reset
+    // errors here.
+    eventHandler.resetErrors();
     QueryEvalResult<Target> queryEvalResult = super.evaluateQuery(expr);
     return new BlazeQueryEvalResult<>(queryEvalResult.getSuccess(), queryEvalResult.getResultSet(),
         graph);
@@ -334,6 +341,19 @@
     return dependentFiles;
   }
 
+  protected Map<String, ResolvedTargets<Target>> preloadOrThrow(Collection<String> patterns)
+      throws TargetParsingException {
+    try {
+      // Note that this may throw a RuntimeException if deps are missing in Skyframe and this is
+      // being called from within a SkyFunction.
+      return targetPatternEvaluator.preloadTargetPatterns(
+          eventHandler, patterns, keepGoing);
+    } catch (InterruptedException e) {
+      // TODO(bazel-team): Propagate the InterruptedException from here [skyframe-loading].
+      throw new TargetParsingException("interrupted");
+    }
+  }
+
   private static void addIfUniqueLabel(Node<Target> node, Set<Label> labels, Set<Target> nodes) {
     if (labels.add(node.getLabel().getLabel())) {
       nodes.add(node.getLabel());
diff --git a/src/main/java/com/google/devtools/build/lib/query2/SkyQueryEnvironment.java b/src/main/java/com/google/devtools/build/lib/query2/SkyQueryEnvironment.java
new file mode 100644
index 0000000..4a0dae6
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/lib/query2/SkyQueryEnvironment.java
@@ -0,0 +1,351 @@
+// Copyright 2015 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.lib.query2;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Predicate;
+import com.google.common.collect.Collections2;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Maps;
+import com.google.devtools.build.lib.cmdline.ResolvedTargets;
+import com.google.devtools.build.lib.cmdline.TargetParsingException;
+import com.google.devtools.build.lib.events.EventHandler;
+import com.google.devtools.build.lib.graph.Digraph;
+import com.google.devtools.build.lib.packages.NoSuchThingException;
+import com.google.devtools.build.lib.packages.Package;
+import com.google.devtools.build.lib.packages.Rule;
+import com.google.devtools.build.lib.packages.Target;
+import com.google.devtools.build.lib.pkgcache.TargetPatternEvaluator;
+import com.google.devtools.build.lib.query2.engine.QueryEvalResult;
+import com.google.devtools.build.lib.query2.engine.QueryException;
+import com.google.devtools.build.lib.query2.engine.QueryExpression;
+import com.google.devtools.build.lib.skyframe.PackageValue;
+import com.google.devtools.build.lib.skyframe.SkyFunctions;
+import com.google.devtools.build.lib.skyframe.TargetPatternValue;
+import com.google.devtools.build.lib.skyframe.TransitiveTargetValue;
+import com.google.devtools.build.lib.syntax.Label;
+import com.google.devtools.build.skyframe.SkyFunctionName;
+import com.google.devtools.build.skyframe.SkyKey;
+import com.google.devtools.build.skyframe.WalkableGraph;
+import com.google.devtools.build.skyframe.WalkableGraph.WalkableGraphFactory;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collection;
+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 javax.annotation.Nullable;
+
+/**
+ * {@link AbstractBlazeQueryEnvironment} that introspects the Skyframe graph to find forward and
+ * reverse edges. Results obtained by calling {@link #evaluateQuery} are not guaranteed to be in
+ * any particular order. As well, this class eagerly loads the full transitive closure of targets,
+ * even if the full closure isn't needed.
+ */
+public class SkyQueryEnvironment extends AbstractBlazeQueryEnvironment<Target> {
+  private WalkableGraph graph;
+
+  private final BlazeTargetAccessor accessor = new BlazeTargetAccessor(this);
+  private final int loadingPhaseThreads;
+  private final WalkableGraphFactory graphFactory;
+  private final List<String> universeScope;
+  private final String parserPrefix;
+
+  public SkyQueryEnvironment(boolean keepGoing, boolean strictScope, int loadingPhaseThreads,
+      Predicate<Label> labelFilter,
+      EventHandler eventHandler,
+      Set<Setting> settings,
+      Iterable<QueryFunction> extraFunctions, String parserPrefix,
+      WalkableGraphFactory graphFactory,
+      List<String> universeScope) {
+    super(keepGoing, strictScope, labelFilter,
+        eventHandler,
+        settings,
+        extraFunctions);
+    this.loadingPhaseThreads = loadingPhaseThreads;
+    this.graphFactory = graphFactory;
+    this.universeScope = Preconditions.checkNotNull(universeScope);
+    this.parserPrefix = parserPrefix;
+    Preconditions.checkState(!universeScope.isEmpty(),
+        "No queries can be performed with an empty universe");
+  }
+
+  private void init() throws InterruptedException {
+    graph = graphFactory.prepareAndGet(universeScope, loadingPhaseThreads, eventHandler);
+  }
+
+  @Override
+  public QueryEvalResult<Target> evaluateQuery(QueryExpression expr)
+      throws QueryException {
+    // Some errors are reported as QueryExceptions and others as ERROR events (if --keep_going). The
+    // result is set to have an error iff there were errors emitted during the query, so we reset
+    // errors here.
+    eventHandler.resetErrors();
+    try {
+      init();
+    } catch (InterruptedException e) {
+      throw new QueryException(e.getMessage());
+    }
+    return super.evaluateQuery(expr);
+  }
+
+  private static SkyKey transformToKey(Target value) {
+    return TransitiveTargetValue.key(value.getLabel());
+  }
+
+  @Nullable
+  private Target transformToValue(SkyKey key) {
+    SkyFunctionName functionName = key.functionName();
+    if (functionName != SkyFunctions.TRANSITIVE_TARGET) {
+      return null;
+    }
+    try {
+      return getTarget(((Label) key.argument()));
+    } catch (QueryException | TargetNotFoundException e) {
+      // Any problems with targets were already reported during #buildTransitiveClosure.
+      return null;
+    }
+  }
+
+  private Collection<Target> getRawFwdDeps(Target target) {
+    return makeTargets(graph.getDirectDeps(transformToKey(target)));
+  }
+
+  private Collection<Target> getRawReverseDeps(Target target) {
+    return makeTargets(graph.getReverseDeps(transformToKey(target)));
+  }
+
+  private Set<Label> getAllowedDeps(Rule rule) {
+    Set<Label> allowedLabels = new HashSet<>(rule.getLabels(dependencyFilter));
+    allowedLabels.addAll(rule.getVisibility().getDependencyLabels());
+    return allowedLabels;
+  }
+
+  public Collection<Target> getFwdDeps(Target target) {
+    Collection<Target> unfilteredDeps = getRawFwdDeps(target);
+    if (!(target instanceof Rule)) {
+      return getRawFwdDeps(target);
+    }
+    final Set<Label> allowedLabels = getAllowedDeps((Rule) target);
+    return Collections2.filter(unfilteredDeps,
+        new Predicate<Target>() {
+          @Override
+          public boolean apply(Target target) {
+            return allowedLabels.contains(target.getLabel());
+          }
+        });
+  }
+
+  public Collection<Target> getReverseDeps(final Target target) {
+    return Collections2.filter(getRawReverseDeps(target), new Predicate<Target>() {
+      @Override
+      public boolean apply(Target parent) {
+        return !(parent instanceof Rule)
+            || getAllowedDeps((Rule) parent).contains(target.getLabel());
+      }
+    });
+  }
+
+  @Override
+  public Set<Target> getTransitiveClosure(Set<Target> targets) {
+    Set<Target> visited = new HashSet<>();
+    List<Target> result = new ArrayList<>(targets);
+    int i = 0;
+    while (i < result.size()) {
+      for (Target dep : getFwdDeps(result.get(i))) {
+        if (visited.add(dep)) {
+          result.add(dep);
+        }
+      }
+      i++;
+    }
+    return ImmutableSet.copyOf(result);
+  }
+
+  // Implemented with a breadth-first search.
+  @Override
+  public Set<Target> getNodesOnPath(Target from, Target to) {
+    // Tree of nodes visited so far.
+    Map<Target, Target> nodeToParent = new HashMap<>();
+    // Contains all nodes left to visit in a (LIFO) stack.
+    Deque<Target> toVisit = new ArrayDeque<>();
+    toVisit.add(from);
+    nodeToParent.put(from, null);
+    while (!toVisit.isEmpty()) {
+      Target current = toVisit.removeFirst();
+      if (to.equals(current)) {
+        return ImmutableSet.copyOf(Digraph.getPathToTreeNode(nodeToParent, to));
+      }
+      for (Target dep : getFwdDeps(current)) {
+        if (!nodeToParent.containsKey(dep)) {
+          nodeToParent.put(dep, current);
+          toVisit.addFirst(dep);
+        }
+      }
+    }
+    // Note that the only current caller of this method checks first to see if there is a path
+    // before calling this method. It is not clear what the return value should be here.
+    return null;
+  }
+
+  @Override
+  public Set<Target> getTargetsMatchingPattern(QueryExpression owner, String pattern)
+      throws QueryException {
+    Set<Target> targets = new LinkedHashSet<>(resolvedTargetPatterns.get(pattern).getTargets());
+
+    // Sets.filter would be more convenient here, but can't deal with exceptions.
+    Iterator<Target> targetIterator = targets.iterator();
+    while (targetIterator.hasNext()) {
+      Target target = targetIterator.next();
+      if (!validateScope(target.getLabel(), strictScope)) {
+        targetIterator.remove();
+      }
+    }
+    return targets;
+  }
+
+  @Override
+  public Set<Target> getBuildFiles(QueryExpression caller, Set<Target> nodes)
+      throws QueryException {
+    Set<Target> dependentFiles = new LinkedHashSet<>();
+    Set<Package> seenPackages = new HashSet<>();
+    // Keep track of seen labels, to avoid adding a fake subinclude label that also exists as a
+    // real target.
+    Set<Label> seenLabels = new HashSet<>();
+
+    // Adds all the package definition files (BUILD files and build
+    // extensions) for package "pkg", to "buildfiles".
+    for (Target x : nodes) {
+      Package pkg = x.getPackage();
+      if (seenPackages.add(pkg)) {
+        addIfUniqueLabel(pkg.getBuildFile(), seenLabels, dependentFiles);
+        for (Label subinclude
+            : Iterables.concat(pkg.getSubincludeLabels(), pkg.getSkylarkFileDependencies())) {
+          addIfUniqueLabel(getSubincludeTarget(subinclude, pkg), seenLabels, dependentFiles);
+
+          // Also add the BUILD file of the subinclude.
+          try {
+            addIfUniqueLabel(getSubincludeTarget(
+                subinclude.getLocalTargetLabel("BUILD"), pkg), seenLabels, dependentFiles);
+          } catch (Label.SyntaxException e) {
+            throw new AssertionError("BUILD should always parse as a target name", e);
+          }
+        }
+      }
+    }
+    return dependentFiles;
+  }
+
+  private static void addIfUniqueLabel(Target node, Set<Label> labels, Set<Target> nodes) {
+    if (labels.add(node.getLabel())) {
+      nodes.add(node);
+    }
+  }
+
+  private static Target getSubincludeTarget(final Label label, Package pkg) {
+    return new FakeSubincludeTarget(label, pkg.getBuildFile().getLocation());
+  }
+
+  @Override
+  public TargetAccessor<Target> getAccessor() {
+    return accessor;
+  }
+
+  @Override
+  public Target getTarget(Label label) throws TargetNotFoundException, QueryException {
+    // Can't use strictScope here because we are expecting a target back.
+    validateScope(label, true);
+    SkyKey packageKey = PackageValue.key(label.getPackageIdentifier());
+    checkExistence(packageKey);
+    try {
+      PackageValue packageValue =
+          (PackageValue) graph.getValue(packageKey);
+      if (packageValue != null) {
+        return packageValue.getPackage().getTarget(label.getName());
+      } else {
+        throw (NoSuchThingException) Preconditions.checkNotNull(
+            graph.getException(packageKey), label);
+      }
+    } catch (NoSuchThingException e) {
+      throw new TargetNotFoundException(e);
+    }
+  }
+
+  @Override
+  public void buildTransitiveClosure(QueryExpression caller, Set<Target> targets, int maxDepth)
+      throws QueryException {
+    // Everything has already been loaded, so here we just check for errors so that we can
+    // pre-emptively throw/report if needed.
+    for (Target target : targets) {
+      SkyKey targetKey = TransitiveTargetValue.key(target.getLabel());
+      checkExistence(targetKey);
+      Exception exception = graph.getException(targetKey);
+      if (exception != null) {
+        reportBuildFileError(caller, exception.getMessage());
+      }
+    }
+  }
+
+  protected Map<String, ResolvedTargets<Target>> preloadOrThrow(Collection<String> patterns)
+      throws QueryException, TargetParsingException {
+    Map<String, ResolvedTargets<Target>> result =
+        Maps.newHashMapWithExpectedSize(patterns.size());
+    for (String pattern : patterns) {
+      SkyKey patternKey = TargetPatternValue.key(pattern,
+          TargetPatternEvaluator.DEFAULT_FILTERING_POLICY, parserPrefix);
+      checkExistence(patternKey);
+      TargetPatternValue value = (TargetPatternValue) graph.getValue(patternKey);
+      if (value != null) {
+        result.put(pattern, value.getTargets());
+      } else if (!keepGoing) {
+        throw (TargetParsingException) Preconditions.checkNotNull(graph.getException(patternKey),
+            pattern);
+      } else {
+        result.put(pattern, ResolvedTargets.<Target>builder().setError().build());
+      }
+    }
+    return result;
+  }
+
+  private Set<Target> makeTargets(Iterable<SkyKey> keys) {
+    ImmutableSet.Builder<Target> builder = ImmutableSet.builder();
+    for (SkyKey key : keys) {
+      Target value = transformToValue(key);
+      if (value != null) {
+        // Some values may be filtered out because they are not Targets.
+        builder.add(value);
+      }
+    }
+    return builder.build();
+  }
+
+  private void checkExistence(SkyKey key) throws QueryException {
+    if (!graph.exists(key)) {
+      throw new QueryException(key + " does not exist in graph");
+    }
+  }
+
+  @Override
+  public Target getOrCreate(Target target) {
+    return target;
+  }
+}
diff --git a/src/main/java/com/google/devtools/build/lib/query2/output/QueryOptions.java b/src/main/java/com/google/devtools/build/lib/query2/output/QueryOptions.java
index 810e8c7..c43610b 100644
--- a/src/main/java/com/google/devtools/build/lib/query2/output/QueryOptions.java
+++ b/src/main/java/com/google/devtools/build/lib/query2/output/QueryOptions.java
@@ -14,10 +14,12 @@
 package com.google.devtools.build.lib.query2.output;
 
 import com.google.devtools.build.lib.query2.engine.QueryEnvironment.Setting;
+import com.google.devtools.common.options.Converters;
 import com.google.devtools.common.options.Option;
 import com.google.devtools.common.options.OptionsBase;
 
 import java.util.EnumSet;
+import java.util.List;
 import java.util.Set;
 
 /**
@@ -117,6 +119,15 @@
           + "containing non-test targets.")
   public boolean strictTestSuite;
 
+  @Option(name = "universe_scope",
+      converter = Converters.CommaSeparatedOptionListConverter.class,
+      defaultValue = "",
+      category = "query",
+      help = "A comma-separated set of target patterns (additive and subtractive). The query will "
+          + "be performed in the universe which is the transitive closure of the specified "
+          + "targets.")
+  public List<String> universeScope;
+
   /**
    * Return the current options as a set of QueryEnvironment settings.
    */
diff --git a/src/main/java/com/google/devtools/build/lib/rules/gen/GenQuery.java b/src/main/java/com/google/devtools/build/lib/rules/gen/GenQuery.java
index 0d639fc..ea0d439 100644
--- a/src/main/java/com/google/devtools/build/lib/rules/gen/GenQuery.java
+++ b/src/main/java/com/google/devtools/build/lib/rules/gen/GenQuery.java
@@ -115,6 +115,10 @@
       ruleContext.attributeError("opts", "option --keep_going is not allowed");
       return null;
     }
+    if (!queryOptions.universeScope.isEmpty()) {
+      ruleContext.attributeError("opts", "option --universe_scope is not allowed");
+      return null;
+    }
 
     final byte[] result = executeQuery(ruleContext, queryOptions, getScope(ruleContext), query);
     if (result == null || ruleContext.hasErrors()) {
@@ -251,12 +255,12 @@
           Preconditions.checkNotNull(outputFormatters), queryOptions.outputFormat);
       queryResult = (BlazeQueryEvalResult<Target>) AbstractBlazeQueryEnvironment
           .newQueryEnvironment(
-          /*transitivePackageLoader=*/null, packageProvider,
+          /*transitivePackageLoader=*/null, /*graph=*/null, packageProvider,
               evaluator,
           /* keepGoing = */ false,
               ruleContext.attributes().get("strict", Type.BOOLEAN),
           /*orderedResults=*/QueryOutputUtils.orderResults(queryOptions, formatter),
-              4,
+              /*universeScope=*/ImmutableList.<String>of(), 4,
               labelFilter,
               getEventHandler(ruleContext),
               settings,
diff --git a/src/main/java/com/google/devtools/build/lib/runtime/commands/QueryCommand.java b/src/main/java/com/google/devtools/build/lib/runtime/commands/QueryCommand.java
index 4b5cbf0..c1d986c 100644
--- a/src/main/java/com/google/devtools/build/lib/runtime/commands/QueryCommand.java
+++ b/src/main/java/com/google/devtools/build/lib/runtime/commands/QueryCommand.java
@@ -40,6 +40,7 @@
 import java.io.IOException;
 import java.io.PrintStream;
 import java.nio.channels.ClosedByInterruptException;
+import java.util.List;
 import java.util.Set;
 
 /**
@@ -105,7 +106,7 @@
         runtime,
         queryOptions.keepGoing,
         QueryOutputUtils.orderResults(queryOptions, formatter),
-        queryOptions.loadingPhaseThreads,
+        queryOptions.universeScope, queryOptions.loadingPhaseThreads,
         settings);
 
     // 1. Parse query:
@@ -150,7 +151,15 @@
 
   @VisibleForTesting // for com.google.devtools.deps.gquery.test.QueryResultTestUtil
   public static AbstractBlazeQueryEnvironment<Target> newQueryEnvironment(BlazeRuntime runtime,
-      boolean keepGoing, boolean orderedResults,  int loadingPhaseThreads,
+      boolean keepGoing, boolean orderedResults, int loadingPhaseThreads,
+      Set<Setting> settings) {
+    return newQueryEnvironment(runtime, keepGoing, orderedResults, ImmutableList.<String>of(),
+        loadingPhaseThreads, settings);
+  }
+
+  public static AbstractBlazeQueryEnvironment<Target> newQueryEnvironment(BlazeRuntime runtime,
+      boolean keepGoing, boolean orderedResults, List<String> universeScope,
+      int loadingPhaseThreads,
       Set<Setting> settings) {
     ImmutableList.Builder<QueryFunction> functions = ImmutableList.builder();
     for (BlazeModule module : runtime.getBlazeModules()) {
@@ -158,9 +167,11 @@
     }
     return AbstractBlazeQueryEnvironment.newQueryEnvironment(
         runtime.getPackageManager().newTransitiveLoader(),
+        runtime.getSkyframeExecutor(),
         runtime.getPackageManager(),
         runtime.getTargetPatternEvaluator(),
-        keepGoing, orderedResults, loadingPhaseThreads, runtime.getReporter(), settings,
+        keepGoing, orderedResults, universeScope, loadingPhaseThreads, runtime.getReporter(),
+        settings,
         functions.build());
   }
 }
diff --git a/src/main/java/com/google/devtools/build/lib/skyframe/SkyframeExecutor.java b/src/main/java/com/google/devtools/build/lib/skyframe/SkyframeExecutor.java
index 9be7077..0c5b503 100644
--- a/src/main/java/com/google/devtools/build/lib/skyframe/SkyframeExecutor.java
+++ b/src/main/java/com/google/devtools/build/lib/skyframe/SkyframeExecutor.java
@@ -57,6 +57,8 @@
 import com.google.devtools.build.lib.analysis.config.ConfigurationFactory;
 import com.google.devtools.build.lib.analysis.config.ConfigurationFragmentFactory;
 import com.google.devtools.build.lib.analysis.config.InvalidConfigurationException;
+import com.google.devtools.build.lib.cmdline.ResolvedTargets;
+import com.google.devtools.build.lib.cmdline.TargetParsingException;
 import com.google.devtools.build.lib.concurrent.ThreadSafety;
 import com.google.devtools.build.lib.concurrent.ThreadSafety.ThreadCompatible;
 import com.google.devtools.build.lib.events.EventHandler;
@@ -70,9 +72,11 @@
 import com.google.devtools.build.lib.packages.Preprocessor;
 import com.google.devtools.build.lib.packages.RuleVisibility;
 import com.google.devtools.build.lib.packages.Target;
+import com.google.devtools.build.lib.pkgcache.FilteringPolicy;
 import com.google.devtools.build.lib.pkgcache.PackageCacheOptions;
 import com.google.devtools.build.lib.pkgcache.PackageManager;
 import com.google.devtools.build.lib.pkgcache.PathPackageLocator;
+import com.google.devtools.build.lib.pkgcache.TargetPatternEvaluator;
 import com.google.devtools.build.lib.pkgcache.TransitivePackageLoader;
 import com.google.devtools.build.lib.skyframe.SkyframeActionExecutor.ActionCompletedReceiver;
 import com.google.devtools.build.lib.skyframe.SkyframeActionExecutor.ProgressSupplier;
@@ -101,6 +105,8 @@
 import com.google.devtools.build.skyframe.SkyFunctionName;
 import com.google.devtools.build.skyframe.SkyKey;
 import com.google.devtools.build.skyframe.SkyValue;
+import com.google.devtools.build.skyframe.WalkableGraph;
+import com.google.devtools.build.skyframe.WalkableGraph.WalkableGraphFactory;
 
 import java.io.IOException;
 import java.io.PrintStream;
@@ -128,7 +134,7 @@
  * some additional artifacts (workspace status and build info artifacts) into SkyFunctions
  * for use during the build.
  */
-public abstract class SkyframeExecutor {
+public abstract class SkyframeExecutor implements WalkableGraphFactory {
   private final EvaluatorSupplier evaluatorSupplier;
   protected MemoizingEvaluator memoizingEvaluator;
   private final MemoizingEvaluator.EmittedEventState emittedEventState =
@@ -1163,6 +1169,43 @@
   }
 
   /**
+   * For internal use in queries: performs two graph updates to make sure the transitive closure of
+   * the specified target {@code patterns} is present in the graph, and returns a traversable view
+   * of the graph.
+   *
+   * <p>The graph updates here are unconditionally done in keep-going mode, so that the query is
+   * guaranteed a complete graph to work on.
+   */
+  @Override
+  public WalkableGraph prepareAndGet(Collection<String> patterns, int numThreads,
+      EventHandler eventHandler) throws InterruptedException {
+    SkyframeTargetPatternEvaluator patternEvaluator =
+        (SkyframeTargetPatternEvaluator) packageManager.getTargetPatternEvaluator();
+    String offset = patternEvaluator.getOffset();
+    FilteringPolicy policy = TargetPatternEvaluator.DEFAULT_FILTERING_POLICY;
+    Iterable<SkyKey> patternSkyKeys = TargetPatternValue.keys(patterns, policy, offset);
+    ResolvedTargets<Target> result;
+    try {
+      result = patternEvaluator.parseTargetPatternKeys(patternSkyKeys, /*keepGoing=*/true,
+          eventHandler);
+    } catch (TargetParsingException e) {
+      // Can't happen, since we ran with keepGoing.
+      throw new IllegalStateException(e);
+    }
+    List<SkyKey> targetKeys = new ArrayList<>();
+    for (Target target : result.getTargets()) {
+      targetKeys.add(TransitiveTargetValue.key(target.getLabel()));
+    }
+    // We request all the keys here, even the ones that were already evaluated, because we want a
+    // single graph that contains all these keys, and if the evaluator keys graphs based on
+    // top-level keys, we must request the union of all our desired keys in a single evaluate call.
+    Iterable<SkyKey> allKeys = ImmutableList.copyOf(Iterables.concat(patternSkyKeys, targetKeys));
+    return Preconditions.checkNotNull(
+        buildDriver.evaluate(allKeys, true, numThreads, eventHandler).getWalkableGraph(),
+        patterns);
+  }
+
+  /**
    * Returns the generating {@link Action} of the given {@link Artifact}.
    *
    * <p>For use for legacy support from {@code BuildView} only.
diff --git a/src/main/java/com/google/devtools/build/lib/skyframe/SkyframeTargetPatternEvaluator.java b/src/main/java/com/google/devtools/build/lib/skyframe/SkyframeTargetPatternEvaluator.java
index 9e619e3..050bdab 100644
--- a/src/main/java/com/google/devtools/build/lib/skyframe/SkyframeTargetPatternEvaluator.java
+++ b/src/main/java/com/google/devtools/build/lib/skyframe/SkyframeTargetPatternEvaluator.java
@@ -23,7 +23,6 @@
 import com.google.devtools.build.lib.events.Event;
 import com.google.devtools.build.lib.events.EventHandler;
 import com.google.devtools.build.lib.packages.Target;
-import com.google.devtools.build.lib.pkgcache.FilteringPolicies;
 import com.google.devtools.build.lib.pkgcache.FilteringPolicy;
 import com.google.devtools.build.lib.pkgcache.ParseFailureListener;
 import com.google.devtools.build.lib.pkgcache.TargetPatternEvaluator;
@@ -58,7 +57,7 @@
   public ResolvedTargets<Target> parseTargetPattern(EventHandler eventHandler,
       String pattern, boolean keepGoing) throws TargetParsingException, InterruptedException {
     return parseTargetPatternList(eventHandler, ImmutableList.of(pattern),
-        FilteringPolicies.NO_FILTER, keepGoing);
+        DEFAULT_FILTERING_POLICY, keepGoing);
   }
 
   @Override
@@ -92,7 +91,12 @@
   ResolvedTargets<Target> parseTargetPatternList(String offset, EventHandler eventHandler,
       List<String> targetPatterns, FilteringPolicy policy, boolean keepGoing)
       throws InterruptedException, TargetParsingException {
-    Iterable<SkyKey> patternSkyKeys = TargetPatternValue.keys(targetPatterns, policy, offset);
+    return parseTargetPatternKeys(TargetPatternValue.keys(targetPatterns, policy, offset),
+        keepGoing, eventHandler);
+  }
+
+  ResolvedTargets<Target> parseTargetPatternKeys(Iterable<SkyKey> patternSkyKeys, boolean keepGoing,
+      EventHandler eventHandler) throws InterruptedException, TargetParsingException {
     EvaluationResult<TargetPatternValue> result =
         skyframeExecutor.targetPatterns(patternSkyKeys, keepGoing, eventHandler);
 
diff --git a/src/main/java/com/google/devtools/build/skyframe/DelegatingWalkableGraph.java b/src/main/java/com/google/devtools/build/skyframe/DelegatingWalkableGraph.java
new file mode 100644
index 0000000..5328a48
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/DelegatingWalkableGraph.java
@@ -0,0 +1,64 @@
+// Copyright 2015 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;
+
+/**
+ * {@link WalkableGraph} that looks nodes up in a {@link QueryableGraph}.
+ */
+public class DelegatingWalkableGraph implements WalkableGraph {
+  private final QueryableGraph graph;
+
+  public DelegatingWalkableGraph(QueryableGraph graph) {
+    this.graph = graph;
+  }
+
+  private NodeEntry getEntry(SkyKey key) {
+    NodeEntry entry = Preconditions.checkNotNull(graph.get(key), key);
+    Preconditions.checkState(entry.isDone(), "%s %s", key, entry);
+    return entry;
+  }
+
+  @Override
+  public boolean exists(SkyKey key) {
+    NodeEntry entry = graph.get(key);
+    return entry != null && entry.isDone();
+  }
+
+  @Nullable
+  @Override
+  public SkyValue getValue(SkyKey key) {
+    return getEntry(key).getValue();
+  }
+
+  @Nullable
+  @Override
+  public Exception getException(SkyKey key) {
+    ErrorInfo errorInfo = getEntry(key).getErrorInfo();
+    return errorInfo == null ? null : errorInfo.getException();
+  }
+
+  @Override
+  public Iterable<SkyKey> getDirectDeps(SkyKey key) {
+    return getEntry(key).getDirectDeps();
+  }
+
+  @Override
+  public Iterable<SkyKey> getReverseDeps(SkyKey key) {
+    return getEntry(key).getReverseDeps();
+  }
+}
diff --git a/src/main/java/com/google/devtools/build/skyframe/EvaluationResult.java b/src/main/java/com/google/devtools/build/skyframe/EvaluationResult.java
index e518dca..92738b0 100644
--- a/src/main/java/com/google/devtools/build/skyframe/EvaluationResult.java
+++ b/src/main/java/com/google/devtools/build/skyframe/EvaluationResult.java
@@ -24,6 +24,8 @@
 import java.util.HashMap;
 import java.util.Map;
 
+import javax.annotation.Nullable;
+
 /**
  * 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}
@@ -38,17 +40,19 @@
 
   private final Map<SkyKey, T> resultMap;
   private final Map<SkyKey, ErrorInfo> errorMap;
+  private final WalkableGraph walkableGraph;
 
   /**
    * Constructor for the "completed" case. Used only by {@link Builder}.
    */
   private EvaluationResult(Map<SkyKey, T> result, Map<SkyKey, ErrorInfo> errorMap,
-      boolean hasError) {
+      boolean hasError, @Nullable WalkableGraph walkableGraph) {
     Preconditions.checkState(errorMap.isEmpty() || hasError,
         "result=%s, errorMap=%s", result, errorMap);
     this.resultMap = Preconditions.checkNotNull(result);
     this.errorMap = Preconditions.checkNotNull(errorMap);
     this.hasError = hasError;
+    this.walkableGraph = walkableGraph;
   }
 
   /**
@@ -109,6 +113,11 @@
     return names;
   }
 
+  @Nullable
+  public WalkableGraph getWalkableGraph() {
+    return walkableGraph;
+  }
+
   /**
    * Returns some error info. Convenience method equivalent to
    * Iterables.getFirst({@link #errorMap()}, null).getValue().
@@ -140,6 +149,7 @@
     private final Map<SkyKey, T> result = new HashMap<>();
     private final Map<SkyKey, ErrorInfo> errors = new HashMap<>();
     private boolean hasError = false;
+    private WalkableGraph walkableGraph = null;
 
     @SuppressWarnings("unchecked")
     public Builder<T> addResult(SkyKey key, SkyValue value) {
@@ -152,8 +162,20 @@
       return this;
     }
 
+    public Builder<T> setWalkableGraph(WalkableGraph walkableGraph) {
+      this.walkableGraph = walkableGraph;
+      return this;
+    }
+
+    public Builder<T> mergeFrom(EvaluationResult<T> otherResult) {
+      result.putAll(otherResult.resultMap);
+      errors.putAll(otherResult.errorMap);
+      hasError |= otherResult.hasError;
+      return this;
+    }
+
     public EvaluationResult<T> build() {
-      return new EvaluationResult<>(result, errors, hasError);
+      return new EvaluationResult<>(result, errors, hasError, walkableGraph);
     }
 
     public void setHasError(boolean hasError) {
diff --git a/src/main/java/com/google/devtools/build/skyframe/InMemoryMemoizingEvaluator.java b/src/main/java/com/google/devtools/build/skyframe/InMemoryMemoizingEvaluator.java
index 827cc7b..695d025 100644
--- a/src/main/java/com/google/devtools/build/skyframe/InMemoryMemoizingEvaluator.java
+++ b/src/main/java/com/google/devtools/build/skyframe/InMemoryMemoizingEvaluator.java
@@ -158,7 +158,11 @@
       ParallelEvaluator evaluator = new ParallelEvaluator(graph, intVersion,
           skyFunctions, eventHandler, emittedEventState, keepGoing, numThreads, progressReceiver,
           dirtyKeyTracker);
-      return evaluator.eval(roots);
+      EvaluationResult<T> result = evaluator.eval(roots);
+      return EvaluationResult.<T>builder()
+          .mergeFrom(result)
+          .setWalkableGraph(new DelegatingWalkableGraph(graph))
+          .build();
     } finally {
       lastGraphVersion = intVersion;
       setAndCheckEvaluateState(false, roots);
diff --git a/src/main/java/com/google/devtools/build/skyframe/NodeEntry.java b/src/main/java/com/google/devtools/build/skyframe/NodeEntry.java
index ac10844..585b4e1 100644
--- a/src/main/java/com/google/devtools/build/skyframe/NodeEntry.java
+++ b/src/main/java/com/google/devtools/build/skyframe/NodeEntry.java
@@ -82,7 +82,7 @@
    */
   @ThreadSafe
   SkyValue getValue();
-   
+
 
   /**
    * Returns the {@link SkyValue} for this entry and the metadata associated with it (Like events
diff --git a/src/main/java/com/google/devtools/build/skyframe/WalkableGraph.java b/src/main/java/com/google/devtools/build/skyframe/WalkableGraph.java
new file mode 100644
index 0000000..68c39c9
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/skyframe/WalkableGraph.java
@@ -0,0 +1,65 @@
+// Copyright 2015 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;
+
+import java.util.Collection;
+
+import javax.annotation.Nullable;
+
+/**
+ * Read-only graph that exposes the dependents, dependencies (reverse dependents), and value and
+ * exception (if any) of a given node.
+ */
+public interface WalkableGraph {
+
+  /**
+   * Returns whether the given key exists as a done node in the graph. If there is a chance that the
+   * given node does not exist, this method should be called before any others, since the others
+   * throw a {@link RuntimeException} on failure to access a node.
+   */
+  boolean exists(SkyKey key);
+
+  /**
+   * Returns the value of the given key, or {@code null} if it has no value due to an error during
+   * its computation. A node with this key must exist in the graph.
+   */
+  @Nullable
+  SkyValue getValue(SkyKey key);
+
+  /**
+   * Returns the exception thrown when computing the node with the given key, if any. If the node
+   * was computed successfully, returns null. A node with this key must exist in the graph.
+   */
+  @Nullable Exception getException(SkyKey key);
+
+  /**
+   * Returns the direct dependencies of the node with the given key. A node with this key must exist
+   * in the graph.
+   */
+  Iterable<SkyKey> getDirectDeps(SkyKey key);
+
+  /**
+   * Returns the reverse dependencies of the node with the given key. A node with this key must
+   * exist in the graph.
+   */
+  Iterable<SkyKey> getReverseDeps(SkyKey key);
+
+  /** Provides a WalkableGraph on demand after preparing it. */
+  interface WalkableGraphFactory {
+    WalkableGraph prepareAndGet(Collection<String> roots, int numThreads,
+        EventHandler eventHandler) throws InterruptedException;
+  }
+}