Initial (partial) implementation of configured target query. Activated by passing the --post_build_query flag to a build command, with a query expression as the argument. Bazel then executes this query on the configured target graph as constructed by the build command.

Since the prepare graph -> query workflow is how SkyQueryEnvironment works, this is mostly just copying that.

Main missing features/code cleanups:
* Recursive target patterns (/...) are not supported.
* There is no way to specify the configuration of the targets in your query.
* Configuration output is totally opaque (just the hash, or null if no configuration).
* More generally, no output options.
* Some features (visibility, label attrs) not supported.
* No edge filtering (host deps, implicit deps).
* Aspects are totally ignored.
* Graceful failure on errors, edge cases, incompatible flags (like the TAP flags that discard edges).
* Code hygiene issues (calling test-only method to get to Skyframe graph, some code duplication across ConfiguredTargetQueryEnvironment and SkyQueryEnvironment).

Most of the features I plan to leave to rules-side people, since I think they won't be too hard for a general Blaze developer to implement, and designing the right features and user interfaces for these things is better left to the rules side.

PiperOrigin-RevId: 165747829
diff --git a/src/main/java/com/google/devtools/build/lib/buildtool/BuildRequest.java b/src/main/java/com/google/devtools/build/lib/buildtool/BuildRequest.java
index cff4572..75ba131 100644
--- a/src/main/java/com/google/devtools/build/lib/buildtool/BuildRequest.java
+++ b/src/main/java/com/google/devtools/build/lib/buildtool/BuildRequest.java
@@ -228,6 +228,14 @@
     public boolean dumpToStdout;
 
     @Option(
+      name = "experimental_post_build_query",
+      defaultValue = "null",
+      documentationCategory = OptionDocumentationCategory.LOGGING,
+      effectTags = {OptionEffectTag.UNKNOWN}
+    )
+    public String queryExpression;
+
+    @Option(
       name = "analyze",
       defaultValue = "true",
       documentationCategory = OptionDocumentationCategory.UNDOCUMENTED,
diff --git a/src/main/java/com/google/devtools/build/lib/buildtool/BuildTool.java b/src/main/java/com/google/devtools/build/lib/buildtool/BuildTool.java
index 127b01d..d04e23b 100644
--- a/src/main/java/com/google/devtools/build/lib/buildtool/BuildTool.java
+++ b/src/main/java/com/google/devtools/build/lib/buildtool/BuildTool.java
@@ -56,13 +56,19 @@
 import com.google.devtools.build.lib.pkgcache.LoadingResult;
 import com.google.devtools.build.lib.profiler.ProfilePhase;
 import com.google.devtools.build.lib.profiler.Profiler;
+import com.google.devtools.build.lib.query2.ConfiguredTargetQueryEnvironment;
+import com.google.devtools.build.lib.query2.engine.QueryException;
+import com.google.devtools.build.lib.query2.engine.ThreadSafeOutputFormatterCallback;
 import com.google.devtools.build.lib.runtime.BlazeRuntime;
 import com.google.devtools.build.lib.runtime.CommandEnvironment;
+import com.google.devtools.build.lib.skyframe.SkyframeExecutorWrappingWalkableGraph;
 import com.google.devtools.build.lib.util.AbruptExitException;
 import com.google.devtools.build.lib.util.ExitCode;
 import com.google.devtools.build.lib.util.Preconditions;
 import com.google.devtools.build.lib.util.RegexFilter;
+import com.google.devtools.build.skyframe.WalkableGraph;
 import com.google.devtools.common.options.OptionsParsingException;
+import java.io.IOException;
 import java.util.Collection;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
@@ -132,6 +138,7 @@
     env.setupPackageCache(request, DefaultsPackage.getDefaultsPackageContent(buildOptions));
 
     ExecutionTool executionTool = null;
+    BuildConfigurationCollection configurations;
     boolean catastrophe = false;
     try {
       env.getEventBus().post(new BuildStartingEvent(env, request));
@@ -184,7 +191,7 @@
       // TODO(gregce): BuildConfigurationCollection is important for static configs, less so for
       // dynamic configs. Consider dropping it outright and passing on-the-fly target / host configs
       // directly when needed (although this could be hard when Skyframe is unavailable).
-      BuildConfigurationCollection configurations =
+      configurations =
           env.getSkyframeExecutor()
               .createConfigurations(
                   env.getReporter(),
@@ -218,6 +225,22 @@
                 "Target %s build was skipped.", target.getLabel())));
         }
 
+        // TODO(janakr): this query will operate over the graph as constructed by analysis, but will
+        // also pick up any nodes that are in the graph from prior builds. This makes the results
+        // not reproducible at the level of a single command. Either tolerate, or wipe the analysis
+        // graph beforehand if this option is specified, or add another option to wipe if desired
+        // (SkyframeExecutor#dropConfiguredTargets should be sufficient).
+        if (request.getBuildOptions().queryExpression != null) {
+          try {
+            doConfiguredTargetQuery(request, configurations);
+          } catch (QueryException | IOException e) {
+            if (!request.getViewOptions().keepGoing) {
+              throw new ViewCreationFailedException("Error doing configured target query", e);
+            }
+            env.getReporter().error(null, "Error doing configured target query", e);
+          }
+        }
+
         // Execution phase.
         if (needsExecutionPhase(request.getBuildOptions())) {
           executionTool.executeBuild(
@@ -368,6 +391,40 @@
     return result;
   }
 
+  private void doConfiguredTargetQuery(
+      BuildRequest request, BuildConfigurationCollection configurations)
+      throws InterruptedException, QueryException, IOException {
+    WalkableGraph walkableGraph =
+        SkyframeExecutorWrappingWalkableGraph.of(env.getSkyframeExecutor());
+    ConfiguredTargetQueryEnvironment configuredTargetQueryEnvironment =
+        new ConfiguredTargetQueryEnvironment(
+            request.getViewOptions().keepGoing,
+            env.getReporter(),
+            env.getRuntime().getQueryFunctions(),
+            configurations.getTargetConfigurations().get(0),
+            configurations.getHostConfiguration(),
+            env.newTargetPatternEvaluator().getOffset(),
+            env.getPackageManager().getPackagePath(),
+            () -> walkableGraph);
+    configuredTargetQueryEnvironment.evaluateQuery(
+        request.getBuildOptions().queryExpression,
+        new ThreadSafeOutputFormatterCallback<ConfiguredTarget>() {
+          @Override
+          public void processOutput(Iterable<ConfiguredTarget> partialResult)
+              throws IOException, InterruptedException {
+            for (ConfiguredTarget configuredTarget : partialResult) {
+              env.getReporter()
+                  .getOutErr()
+                  .printOutLn(
+                      configuredTarget.getLabel()
+                          + " ("
+                          + configuredTarget.getConfiguration()
+                          + ")");
+            }
+          }
+        });
+  }
+
   private void maybeSetStopOnFirstFailure(BuildRequest request, BuildResult result) {
     if (shouldStopOnFailure(request)) {
       result.setStopOnFirstFailure(true);
diff --git a/src/main/java/com/google/devtools/build/lib/packages/TargetUtils.java b/src/main/java/com/google/devtools/build/lib/packages/TargetUtils.java
index a2babbc..bb890cb 100644
--- a/src/main/java/com/google/devtools/build/lib/packages/TargetUtils.java
+++ b/src/main/java/com/google/devtools/build/lib/packages/TargetUtils.java
@@ -14,12 +14,17 @@
 
 package com.google.devtools.build.lib.packages;
 
+import static com.google.devtools.build.lib.packages.BuildType.TRISTATE;
+import static com.google.devtools.build.lib.syntax.Type.BOOLEAN;
+
 import com.google.common.base.Predicate;
 import com.google.common.collect.ImmutableMap;
 import com.google.devtools.build.lib.cmdline.Label;
 import com.google.devtools.build.lib.events.Location;
 import com.google.devtools.build.lib.syntax.Type;
 import com.google.devtools.build.lib.util.Pair;
+import com.google.devtools.build.lib.util.Preconditions;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.List;
@@ -122,6 +127,54 @@
     return hasConstraint(rule, "external");
   }
 
+  public static List<String> getStringListAttr(Target target, String attrName) {
+    Preconditions.checkArgument(target instanceof Rule);
+    return NonconfigurableAttributeMapper.of((Rule) target).get(attrName, Type.STRING_LIST);
+  }
+
+  public static String getStringAttr(Target target, String attrName) {
+    Preconditions.checkArgument(target instanceof Rule);
+    return NonconfigurableAttributeMapper.of((Rule) target).get(attrName, Type.STRING);
+  }
+
+  public static Iterable<String> getAttrAsString(Target target, String attrName) {
+    Preconditions.checkArgument(target instanceof Rule);
+    List<String> values = new ArrayList<>(); // May hold null values.
+    Attribute attribute = ((Rule) target).getAttributeDefinition(attrName);
+    if (attribute != null) {
+      Type<?> attributeType = attribute.getType();
+      for (Object attrValue :
+          AggregatingAttributeMapper.of((Rule) target)
+              .visitAttribute(attribute.getName(), attributeType)) {
+
+        // Ugly hack to maintain backward 'attr' query compatibility for BOOLEAN and TRISTATE
+        // attributes. These are internally stored as actual Boolean or TriState objects but were
+        // historically queried as integers. To maintain compatibility, we inspect their actual
+        // value and return the integer equivalent represented as a String. This code is the
+        // opposite of the code in BooleanType and TriStateType respectively.
+        if (attributeType == BOOLEAN) {
+          values.add(Type.BOOLEAN.cast(attrValue) ? "1" : "0");
+        } else if (attributeType == TRISTATE) {
+          switch (BuildType.TRISTATE.cast(attrValue)) {
+            case AUTO:
+              values.add("-1");
+              break;
+            case NO:
+              values.add("0");
+              break;
+            case YES:
+              values.add("1");
+              break;
+            default:
+              throw new AssertionError("This can't happen!");
+          }
+        } else {
+          values.add(attrValue == null ? null : attrValue.toString());
+        }
+      }
+    }
+    return values;
+  }
 
   /**
    * If the given target is a rule, returns its <code>deprecation<code/> value, or null if unset.
diff --git a/src/main/java/com/google/devtools/build/lib/query2/BUILD b/src/main/java/com/google/devtools/build/lib/query2/BUILD
index 72fb496..968d550 100644
--- a/src/main/java/com/google/devtools/build/lib/query2/BUILD
+++ b/src/main/java/com/google/devtools/build/lib/query2/BUILD
@@ -14,6 +14,7 @@
         "//src/main/java/com/google/devtools/build/lib:packages-internal",
         "//src/main/java/com/google/devtools/build/lib:util",
         "//src/main/java/com/google/devtools/build/lib:vfs",
+        "//src/main/java/com/google/devtools/build/lib/actions",
         "//src/main/java/com/google/devtools/build/skyframe",
         "//src/main/java/com/google/devtools/build/skyframe:skyframe-objects",
         "//third_party:guava",
diff --git a/src/main/java/com/google/devtools/build/lib/query2/BlazeTargetAccessor.java b/src/main/java/com/google/devtools/build/lib/query2/BlazeTargetAccessor.java
index d345a27..1579483 100644
--- a/src/main/java/com/google/devtools/build/lib/query2/BlazeTargetAccessor.java
+++ b/src/main/java/com/google/devtools/build/lib/query2/BlazeTargetAccessor.java
@@ -13,17 +13,11 @@
 // limitations under the License.
 package com.google.devtools.build.lib.query2;
 
-import static com.google.devtools.build.lib.packages.BuildType.TRISTATE;
-import static com.google.devtools.build.lib.syntax.Type.BOOLEAN;
-
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
 import com.google.devtools.build.lib.cmdline.Label;
 import com.google.devtools.build.lib.packages.AggregatingAttributeMapper;
-import com.google.devtools.build.lib.packages.Attribute;
-import com.google.devtools.build.lib.packages.BuildType;
 import com.google.devtools.build.lib.packages.ConstantRuleVisibility;
-import com.google.devtools.build.lib.packages.NonconfigurableAttributeMapper;
 import com.google.devtools.build.lib.packages.PackageGroup;
 import com.google.devtools.build.lib.packages.PackageGroupsRuleVisibility;
 import com.google.devtools.build.lib.packages.PackageSpecification;
@@ -98,53 +92,17 @@
 
   @Override
   public List<String> getStringListAttr(Target target, String attrName) {
-    Preconditions.checkArgument(target instanceof Rule);
-    return NonconfigurableAttributeMapper.of((Rule) target).get(attrName, Type.STRING_LIST);
+    return TargetUtils.getStringListAttr(target, attrName);
   }
 
   @Override
   public String getStringAttr(Target target, String attrName) {
-    Preconditions.checkArgument(target instanceof Rule);
-    return NonconfigurableAttributeMapper.of((Rule) target).get(attrName, Type.STRING);
+    return TargetUtils.getStringAttr(target, attrName);
   }
 
   @Override
   public Iterable<String> getAttrAsString(Target target, String attrName) {
-    Preconditions.checkArgument(target instanceof Rule);
-    List<String> values = new ArrayList<>(); // May hold null values.
-    Attribute attribute = ((Rule) target).getAttributeDefinition(attrName);
-    if (attribute != null) {
-      Type<?> attributeType = attribute.getType();
-      for (Object attrValue : AggregatingAttributeMapper.of((Rule) target).visitAttribute(
-          attribute.getName(), attributeType)) {
-
-        // Ugly hack to maintain backward 'attr' query compatibility for BOOLEAN and TRISTATE
-        // attributes. These are internally stored as actual Boolean or TriState objects but were
-        // historically queried as integers. To maintain compatibility, we inspect their actual
-        // value and return the integer equivalent represented as a String. This code is the
-        // opposite of the code in BooleanType and TriStateType respectively.
-        if (attributeType == BOOLEAN) {
-          values.add(Type.BOOLEAN.cast(attrValue) ? "1" : "0");
-        } else if (attributeType == TRISTATE) {
-            switch (BuildType.TRISTATE.cast(attrValue)) {
-              case AUTO :
-                values.add("-1");
-                break;
-              case NO :
-                values.add("0");
-                break;
-              case YES :
-                values.add("1");
-                break;
-              default :
-                throw new AssertionError("This can't happen!");
-            }
-        } else {
-          values.add(attrValue == null ? null : attrValue.toString());
-        }
-      }
-    }
-    return values;
+    return TargetUtils.getAttrAsString(target, attrName);
   }
 
   @Override
diff --git a/src/main/java/com/google/devtools/build/lib/query2/ConfiguredTargetAccessor.java b/src/main/java/com/google/devtools/build/lib/query2/ConfiguredTargetAccessor.java
new file mode 100644
index 0000000..7d60fc7
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/lib/query2/ConfiguredTargetAccessor.java
@@ -0,0 +1,90 @@
+// Copyright 2017 The Bazel Authors. All rights reserved.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//    http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+package com.google.devtools.build.lib.query2;
+
+import com.google.devtools.build.lib.analysis.ConfiguredTarget;
+import com.google.devtools.build.lib.packages.Rule;
+import com.google.devtools.build.lib.packages.TargetUtils;
+import com.google.devtools.build.lib.query2.engine.QueryEnvironment.TargetAccessor;
+import com.google.devtools.build.lib.query2.engine.QueryException;
+import com.google.devtools.build.lib.query2.engine.QueryExpression;
+import com.google.devtools.build.lib.query2.engine.QueryVisibility;
+import java.util.List;
+import java.util.Set;
+
+/** A {@link TargetAccessor} for {@link ConfiguredTarget} objects. Incomplete. */
+class ConfiguredTargetAccessor implements TargetAccessor<ConfiguredTarget> {
+  @Override
+  public String getTargetKind(ConfiguredTarget target) {
+    return target.getTarget().getTargetKind();
+  }
+
+  @Override
+  public String getLabel(ConfiguredTarget target) {
+    return target.getLabel().toString();
+  }
+
+  @Override
+  public String getPackage(ConfiguredTarget target) {
+    return target.getTarget().getPackage().getNameFragment().toString();
+  }
+
+  @Override
+  public boolean isRule(ConfiguredTarget target) {
+    return target.getTarget() instanceof Rule;
+  }
+
+  @Override
+  public boolean isTestRule(ConfiguredTarget target) {
+    return TargetUtils.isTestRule(target.getTarget());
+  }
+
+  @Override
+  public boolean isTestSuite(ConfiguredTarget target) {
+    return TargetUtils.isTestSuiteRule(target.getTarget());
+  }
+
+  @Override
+  public List<ConfiguredTarget> getLabelListAttr(
+      QueryExpression caller,
+      ConfiguredTarget configuredTarget,
+      String attrName,
+      String errorMsgPrefix)
+      throws QueryException, InterruptedException {
+    // TODO(bazel-team): implement this if needed.
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public List<String> getStringListAttr(ConfiguredTarget target, String attrName) {
+    return TargetUtils.getStringListAttr(target.getTarget(), attrName);
+  }
+
+  @Override
+  public String getStringAttr(ConfiguredTarget target, String attrName) {
+    return TargetUtils.getStringAttr(target.getTarget(), attrName);
+  }
+
+  @Override
+  public Iterable<String> getAttrAsString(ConfiguredTarget target, String attrName) {
+    return TargetUtils.getAttrAsString(target.getTarget(), attrName);
+  }
+
+  @Override
+  public Set<QueryVisibility<ConfiguredTarget>> getVisibility(ConfiguredTarget from)
+      throws QueryException, InterruptedException {
+    // TODO(bazel-team): implement this if needed.
+    throw new UnsupportedOperationException();
+  }
+}
diff --git a/src/main/java/com/google/devtools/build/lib/query2/ConfiguredTargetQueryEnvironment.java b/src/main/java/com/google/devtools/build/lib/query2/ConfiguredTargetQueryEnvironment.java
new file mode 100644
index 0000000..3628a76
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/lib/query2/ConfiguredTargetQueryEnvironment.java
@@ -0,0 +1,410 @@
+// Copyright 2017 The Bazel Authors. All rights reserved.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//    http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+package com.google.devtools.build.lib.query2;
+
+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.common.util.concurrent.AsyncFunction;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.MoreExecutors;
+import com.google.devtools.build.lib.analysis.ConfiguredTarget;
+import com.google.devtools.build.lib.analysis.LabelAndConfiguration;
+import com.google.devtools.build.lib.analysis.config.BuildConfiguration;
+import com.google.devtools.build.lib.cmdline.Label;
+import com.google.devtools.build.lib.cmdline.TargetParsingException;
+import com.google.devtools.build.lib.cmdline.TargetPattern;
+import com.google.devtools.build.lib.cmdline.TargetPattern.Type;
+import com.google.devtools.build.lib.concurrent.MultisetSemaphore;
+import com.google.devtools.build.lib.events.Event;
+import com.google.devtools.build.lib.events.ExtendedEventHandler;
+import com.google.devtools.build.lib.packages.Rule;
+import com.google.devtools.build.lib.packages.Target;
+import com.google.devtools.build.lib.pkgcache.FilteringPolicies;
+import com.google.devtools.build.lib.pkgcache.PathPackageLocator;
+import com.google.devtools.build.lib.pkgcache.TargetPatternEvaluator;
+import com.google.devtools.build.lib.query2.engine.Callback;
+import com.google.devtools.build.lib.query2.engine.KeyExtractor;
+import com.google.devtools.build.lib.query2.engine.MinDepthUniquifier;
+import com.google.devtools.build.lib.query2.engine.QueryEnvironment;
+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.query2.engine.QueryUtil.MinDepthUniquifierImpl;
+import com.google.devtools.build.lib.query2.engine.QueryUtil.MutableKeyExtractorBackedMapImpl;
+import com.google.devtools.build.lib.query2.engine.QueryUtil.ThreadSafeMutableKeyExtractorBackedSetImpl;
+import com.google.devtools.build.lib.query2.engine.QueryUtil.UniquifierImpl;
+import com.google.devtools.build.lib.query2.engine.ThreadSafeOutputFormatterCallback;
+import com.google.devtools.build.lib.query2.engine.Uniquifier;
+import com.google.devtools.build.lib.skyframe.ConfiguredTargetKey;
+import com.google.devtools.build.lib.skyframe.ConfiguredTargetValue;
+import com.google.devtools.build.lib.skyframe.GraphBackedRecursivePackageProvider;
+import com.google.devtools.build.lib.skyframe.RecursivePackageProviderBackedTargetPatternResolver;
+import com.google.devtools.build.lib.skyframe.SkyFunctions;
+import com.google.devtools.build.lib.skyframe.TargetPatternValue;
+import com.google.devtools.build.lib.skyframe.TargetPatternValue.TargetPatternKey;
+import com.google.devtools.build.skyframe.SkyKey;
+import com.google.devtools.build.skyframe.WalkableGraph;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.function.Function;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+
+/**
+ * {@link QueryEnvironment} that runs queries over the configured target (analysis) graph.
+ *
+ * <p>Currently no edges are filtered out, in contrast to query as implemented on the target graph
+ * (host_deps and implicit_deps are important ones). Because of the higher fidelity that users of
+ * the configured target graph presumably want, this may be ok, but also may not be.
+ *
+ * <p>This object can theoretically be used for multiple queries, but currently is only ever used
+ * for one over the course of its lifetime.
+ *
+ * <p>There is currently no way to specify a configuration in the query syntax. Instead, the default
+ * configuration that will be used for any raw labels is provided in the constructor of this
+ * environment. That will probably have to change.
+ *
+ * <p>On the other end, recursive target patterns are not supported.
+ *
+ * <p>Aspects are also not supported, but probably should be in some fashion.
+ */
+public class ConfiguredTargetQueryEnvironment
+    extends AbstractBlazeQueryEnvironment<ConfiguredTarget> {
+  private final BuildConfiguration defaultTargetConfiguration;
+  private final BuildConfiguration hostConfiguration;
+  private final String parserPrefix;
+  protected final PathPackageLocator pkgPath;
+  private final Supplier<WalkableGraph> walkableGraphSupplier;
+  private final ConfiguredTargetAccessor accessor = new ConfiguredTargetAccessor();
+  protected WalkableGraph graph;
+
+  private static final Function<ConfiguredTarget, SkyKey> CT_TO_SKYKEY =
+      target -> ConfiguredTargetValue.key(target.getLabel(), target.getConfiguration());
+  private static final Function<SkyKey, LabelAndConfiguration> SKYKEY_TO_LANDC =
+      skyKey -> {
+        ConfiguredTargetKey key = (ConfiguredTargetKey) skyKey.argument();
+        return LabelAndConfiguration.of(key.getLabel(), key.getConfiguration());
+      };
+  private static final ImmutableList<TargetPatternKey> ALL_PATTERNS;
+  private static final KeyExtractor<ConfiguredTarget, LabelAndConfiguration>
+      CONFIGURED_TARGET_KEY_EXTRACTOR = LabelAndConfiguration::of;
+
+  static {
+    TargetPattern targetPattern;
+    try {
+      targetPattern = TargetPattern.defaultParser().parse("//...");
+    } catch (TargetParsingException e) {
+      throw new IllegalStateException(e);
+    }
+    ALL_PATTERNS =
+        ImmutableList.of(
+            new TargetPatternKey(
+                targetPattern, FilteringPolicies.NO_FILTER, false, "", ImmutableSet.of()));
+  }
+
+  private RecursivePackageProviderBackedTargetPatternResolver resolver;
+
+  public ConfiguredTargetQueryEnvironment(
+      boolean keepGoing,
+      ExtendedEventHandler eventHandler,
+      Iterable<QueryFunction> extraFunctions,
+      BuildConfiguration defaultTargetConfiguration,
+      BuildConfiguration hostConfiguration,
+      String parserPrefix,
+      PathPackageLocator pkgPath,
+      Supplier<WalkableGraph> walkableGraphSupplier) {
+    super(
+        keepGoing,
+        true,
+        Rule.ALL_LABELS,
+        eventHandler,
+        // TODO(janakr): decide whether to support host and implicit dep filtering.
+        EnumSet.noneOf(Setting.class),
+        extraFunctions);
+    this.defaultTargetConfiguration = defaultTargetConfiguration;
+    this.hostConfiguration = hostConfiguration;
+    this.parserPrefix = parserPrefix;
+    this.pkgPath = pkgPath;
+    this.walkableGraphSupplier = walkableGraphSupplier;
+  }
+
+  private void beforeEvaluateQuery() throws InterruptedException {
+    graph = walkableGraphSupplier.get();
+    GraphBackedRecursivePackageProvider graphBackedRecursivePackageProvider =
+        new GraphBackedRecursivePackageProvider(graph, ALL_PATTERNS, pkgPath);
+    resolver =
+        new RecursivePackageProviderBackedTargetPatternResolver(
+            graphBackedRecursivePackageProvider,
+            eventHandler,
+            FilteringPolicies.NO_FILTER,
+            MultisetSemaphore.unbounded());
+  }
+
+  @Nullable
+  private ConfiguredTarget getConfiguredTarget(SkyKey key) throws InterruptedException {
+    ConfiguredTargetValue value =
+        ((ConfiguredTargetValue) walkableGraphSupplier.get().getValue(key));
+    return value == null ? null : value.getConfiguredTarget();
+  }
+
+  @Override
+  public void close() {}
+
+  @Override
+  public QueryEvalResult evaluateQuery(
+      QueryExpression expr, ThreadSafeOutputFormatterCallback<ConfiguredTarget> callback)
+      throws QueryException, InterruptedException, IOException {
+    beforeEvaluateQuery();
+    return super.evaluateQuery(expr, callback);
+  }
+
+  private TargetPattern getPattern(String pattern)
+      throws TargetParsingException, InterruptedException {
+    TargetPatternKey targetPatternKey =
+        ((TargetPatternKey)
+            TargetPatternValue.key(
+                    pattern, TargetPatternEvaluator.DEFAULT_FILTERING_POLICY, parserPrefix)
+                .argument());
+    return targetPatternKey.getParsedPattern();
+  }
+
+  @Override
+  public Collection<ConfiguredTarget> getSiblingTargetsInPackage(ConfiguredTarget target) {
+    throw new UnsupportedOperationException("siblings() not supported");
+  }
+
+  @Override
+  public QueryTaskFuture<Void> getTargetsMatchingPattern(
+      QueryExpression owner, String pattern, Callback<ConfiguredTarget> callback) {
+    TargetPattern patternToEval;
+    try {
+      patternToEval = getPattern(pattern);
+    } catch (TargetParsingException tpe) {
+      try {
+        reportBuildFileError(owner, tpe.getMessage());
+      } catch (QueryException qe) {
+        return immediateFailedFuture(qe);
+      }
+      return immediateSuccessfulFuture(null);
+    } catch (InterruptedException ie) {
+      return immediateCancelledFuture();
+    }
+    AsyncFunction<TargetParsingException, Void> reportBuildFileErrorAsyncFunction =
+        exn -> {
+          reportBuildFileError(owner, exn.getMessage());
+          return Futures.immediateFuture(null);
+        };
+    return QueryTaskFutureImpl.ofDelegate(
+        Futures.catchingAsync(
+            patternToEval.evalAdaptedForAsync(
+                resolver,
+                ImmutableSet.of(),
+                ImmutableSet.of(),
+                (Callback<Target>)
+                    partialResult -> {
+                      List<ConfiguredTarget> transformedResult = new ArrayList<>();
+                      for (Target target : partialResult) {
+                        ConfiguredTarget configuredTarget = getConfiguredTarget(target.getLabel());
+                        if (configuredTarget != null) {
+                          transformedResult.add(configuredTarget);
+                        }
+                      }
+                      callback.process(transformedResult);
+                    },
+                QueryException.class),
+            TargetParsingException.class,
+            reportBuildFileErrorAsyncFunction,
+            MoreExecutors.directExecutor()));
+  }
+
+  @Override
+  public ConfiguredTarget getOrCreate(ConfiguredTarget target) {
+    return target;
+  }
+
+  private Map<SkyKey, Collection<ConfiguredTarget>> targetifyValues(
+      Map<SkyKey, ? extends Iterable<SkyKey>> input) throws InterruptedException {
+    Map<SkyKey, Collection<ConfiguredTarget>> result = new HashMap<>();
+    for (Map.Entry<SkyKey, ? extends Iterable<SkyKey>> entry : input.entrySet()) {
+      Collection<ConfiguredTarget> value = new ArrayList<>();
+      for (SkyKey key : entry.getValue()) {
+        if (key.functionName().equals(SkyFunctions.CONFIGURED_TARGET)) {
+          value.add(getConfiguredTarget(key));
+        }
+      }
+      result.put(entry.getKey(), value);
+    }
+    return result;
+  }
+
+  @Override
+  public ThreadSafeMutableSet<ConfiguredTarget> getFwdDeps(Iterable<ConfiguredTarget> targets)
+      throws InterruptedException {
+    Map<SkyKey, ConfiguredTarget> targetsByKey = new HashMap<>(Iterables.size(targets));
+    for (ConfiguredTarget target : targets) {
+      targetsByKey.put(CT_TO_SKYKEY.apply(target), target);
+    }
+    Map<SkyKey, Collection<ConfiguredTarget>> directDeps =
+        targetifyValues(graph.getDirectDeps(targetsByKey.keySet()));
+    if (targetsByKey.keySet().size() != directDeps.keySet().size()) {
+      Iterable<LabelAndConfiguration> missingTargets =
+          Sets.difference(targetsByKey.keySet(), directDeps.keySet())
+              .stream()
+              .map(SKYKEY_TO_LANDC)
+              .collect(Collectors.toList());
+      eventHandler.handle(Event.warn("Targets were missing from graph: " + missingTargets));
+    }
+    ThreadSafeMutableSet<ConfiguredTarget> result = createThreadSafeMutableSet();
+    for (Entry<SkyKey, Collection<ConfiguredTarget>> entry : directDeps.entrySet()) {
+      result.addAll(entry.getValue());
+    }
+    return result;
+  }
+
+  @Override
+  public Collection<ConfiguredTarget> getReverseDeps(Iterable<ConfiguredTarget> targets)
+      throws InterruptedException {
+    Map<SkyKey, ConfiguredTarget> targetsByKey = new HashMap<>(Iterables.size(targets));
+    for (ConfiguredTarget target : targets) {
+      targetsByKey.put(CT_TO_SKYKEY.apply(target), target);
+    }
+    Map<SkyKey, Collection<ConfiguredTarget>> reverseDeps =
+        targetifyValues(graph.getReverseDeps(targetsByKey.keySet()));
+    if (targetsByKey.keySet().size() != reverseDeps.keySet().size()) {
+      Iterable<LabelAndConfiguration> missingTargets =
+          Sets.difference(targetsByKey.keySet(), reverseDeps.keySet())
+              .stream()
+              .map(SKYKEY_TO_LANDC)
+              .collect(Collectors.toList());
+      eventHandler.handle(Event.warn("Targets were missing from graph: " + missingTargets));
+    }
+    ThreadSafeMutableSet<ConfiguredTarget> result = createThreadSafeMutableSet();
+    for (Entry<SkyKey, Collection<ConfiguredTarget>> entry : reverseDeps.entrySet()) {
+      result.addAll(entry.getValue());
+    }
+    return result;
+  }
+
+  @Override
+  public ThreadSafeMutableSet<ConfiguredTarget> getTransitiveClosure(
+      ThreadSafeMutableSet<ConfiguredTarget> targets) throws InterruptedException {
+    return SkyQueryUtils.getTransitiveClosure(
+        targets, this::getFwdDeps, createThreadSafeMutableSet());
+  }
+
+  @Override
+  public void buildTransitiveClosure(
+      QueryExpression caller, ThreadSafeMutableSet<ConfiguredTarget> targetNodes, int maxDepth)
+      throws QueryException, InterruptedException {
+    // TODO(bazel-team): implement this. Just needed for error-checking.
+  }
+
+  @Override
+  public ImmutableList<ConfiguredTarget> getNodesOnPath(ConfiguredTarget from, ConfiguredTarget to)
+      throws InterruptedException {
+    return SkyQueryUtils.getNodesOnPath(from, to, this::getFwdDeps, LabelAndConfiguration::of);
+  }
+
+  @Override
+  public TargetAccessor<ConfiguredTarget> getAccessor() {
+    return accessor;
+  }
+
+  // TODO(bazel-team): It's weird that this untemplated function exists. Fix? Or don't implement?
+  @Override
+  public Target getTarget(Label label)
+      throws TargetNotFoundException, QueryException, InterruptedException {
+    ConfiguredTarget configuredTarget = getConfiguredTarget(label);
+    return configuredTarget == null ? null : configuredTarget.getTarget();
+  }
+
+  private ConfiguredTarget getConfiguredTarget(Label label) throws InterruptedException {
+    // Try with host configuration.
+    ConfiguredTarget configuredTarget =
+        getConfiguredTarget(ConfiguredTargetValue.key(label, hostConfiguration));
+    if (configuredTarget != null) {
+      return configuredTarget;
+    }
+    configuredTarget =
+        getConfiguredTarget(ConfiguredTargetValue.key(label, defaultTargetConfiguration));
+    if (configuredTarget != null) {
+      return configuredTarget;
+    }
+    // Last chance: source file.
+    return getConfiguredTarget(ConfiguredTargetValue.key(label, null));
+  }
+
+  @Override
+  public ThreadSafeMutableSet<ConfiguredTarget> createThreadSafeMutableSet() {
+    return new ThreadSafeMutableKeyExtractorBackedSetImpl<>(
+        CONFIGURED_TARGET_KEY_EXTRACTOR,
+        ConfiguredTarget.class,
+        SkyQueryEnvironment.DEFAULT_THREAD_COUNT);
+  }
+
+  @Override
+  public <V> MutableMap<ConfiguredTarget, V> createMutableMap() {
+    return new MutableKeyExtractorBackedMapImpl<>(CONFIGURED_TARGET_KEY_EXTRACTOR);
+  }
+
+  @Override
+  public Uniquifier<ConfiguredTarget> createUniquifier() {
+    return new UniquifierImpl<>(
+        CONFIGURED_TARGET_KEY_EXTRACTOR, SkyQueryEnvironment.DEFAULT_THREAD_COUNT);
+  }
+
+  @Override
+  public MinDepthUniquifier<ConfiguredTarget> createMinDepthUniquifier() {
+    return new MinDepthUniquifierImpl<>(
+        CONFIGURED_TARGET_KEY_EXTRACTOR, SkyQueryEnvironment.DEFAULT_THREAD_COUNT);
+  }
+
+  @Override
+  public ThreadSafeMutableSet<ConfiguredTarget> getBuildFiles(
+      QueryExpression caller,
+      ThreadSafeMutableSet<ConfiguredTarget> nodes,
+      boolean buildFiles,
+      boolean subincludes,
+      boolean loads)
+      throws QueryException, InterruptedException {
+    throw new QueryException("buildfiles() doesn't make sense for the configured target graph");
+  }
+
+  @Override
+  protected void preloadOrThrow(QueryExpression caller, Collection<String> patterns)
+      throws QueryException, TargetParsingException, InterruptedException {
+    for (String pattern : patterns) {
+      if (TargetPattern.defaultParser()
+          .parse(pattern)
+          .getType()
+          .equals(Type.TARGETS_BELOW_DIRECTORY)) {
+        // TODO(bazel-team): allow recursive patterns if the pattern is present in the graph? We
+        // could do a mini-eval here to update the graph to contain the necessary nodes for
+        // GraphBackedRecursivePackageProvider, since all the package loading and directory
+        // traversal should already be done.
+        throw new QueryException(
+            "Recursive pattern '" + pattern + "' is not supported in configured target query");
+      }
+    }
+  }
+}
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
index 383a28f..0fb0283 100644
--- a/src/main/java/com/google/devtools/build/lib/query2/SkyQueryEnvironment.java
+++ b/src/main/java/com/google/devtools/build/lib/query2/SkyQueryEnvironment.java
@@ -49,7 +49,6 @@
 import com.google.devtools.build.lib.events.Event;
 import com.google.devtools.build.lib.events.EventKind;
 import com.google.devtools.build.lib.events.ExtendedEventHandler;
-import com.google.devtools.build.lib.graph.Digraph;
 import com.google.devtools.build.lib.packages.BuildFileContainsErrorsException;
 import com.google.devtools.build.lib.packages.DependencyFilter;
 import com.google.devtools.build.lib.packages.NoSuchPackageException;
@@ -107,11 +106,8 @@
 import com.google.devtools.build.skyframe.WalkableGraph;
 import com.google.devtools.build.skyframe.WalkableGraph.WalkableGraphFactory;
 import java.io.IOException;
-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.LinkedList;
 import java.util.List;
@@ -513,7 +509,7 @@
     return getReverseDepsOfTransitiveTraversalKeys(Iterables.transform(targets, TARGET_TO_SKY_KEY));
   }
 
-  Collection<Target> getReverseDepsOfTransitiveTraversalKeys(
+  private Collection<Target> getReverseDepsOfTransitiveTraversalKeys(
       Iterable<SkyKey> transitiveTraversalKeys) throws InterruptedException {
     Map<SkyKey, Collection<Target>> rawReverseDeps = getRawReverseDeps(transitiveTraversalKeys);
     return processRawReverseDeps(rawReverseDeps);
@@ -563,51 +559,14 @@
   @Override
   public ThreadSafeMutableSet<Target> getTransitiveClosure(ThreadSafeMutableSet<Target> targets)
       throws InterruptedException {
-    ThreadSafeMutableSet<Target> visited = createThreadSafeMutableSet();
-    ThreadSafeMutableSet<Target> current = targets;
-    while (!current.isEmpty()) {
-      Iterable<Target> toVisit = Iterables.filter(current,
-          Predicates.not(Predicates.in(visited)));
-      current = getFwdDeps(toVisit);
-      Iterables.addAll(visited, toVisit);
-    }
-    return visited;
+    return SkyQueryUtils.getTransitiveClosure(
+        targets, this::getFwdDeps, createThreadSafeMutableSet());
   }
 
-  // Implemented with a breadth-first search.
   @Override
   public ImmutableList<Target> getNodesOnPath(Target from, Target to)
       throws InterruptedException {
-    // Tree of nodes visited so far.
-    Map<Label, Label> nodeToParent = new HashMap<>();
-    Map<Label, Target> labelToTarget = new HashMap<>();
-    // Contains all nodes left to visit in a (LIFO) stack.
-    Deque<Target> toVisit = new ArrayDeque<>();
-    toVisit.add(from);
-    nodeToParent.put(from.getLabel(), null);
-    labelToTarget.put(from.getLabel(), from);
-    while (!toVisit.isEmpty()) {
-      Target current = toVisit.removeFirst();
-      if (to.getLabel().equals(current.getLabel())) {
-        List<Label> labelPath = Digraph.getPathToTreeNode(nodeToParent, to.getLabel());
-        ImmutableList.Builder<Target> targetPathBuilder = ImmutableList.builder();
-        for (Label label : labelPath) {
-          targetPathBuilder.add(Preconditions.checkNotNull(labelToTarget.get(label), label));
-        }
-        return targetPathBuilder.build();
-      }
-      for (Target dep : getFwdDeps(ImmutableList.of(current))) {
-        Label depLabel = dep.getLabel();
-        if (!nodeToParent.containsKey(depLabel)) {
-          nodeToParent.put(depLabel, current.getLabel());
-          labelToTarget.put(depLabel, dep);
-          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;
+    return SkyQueryUtils.getNodesOnPath(from, to, this::getFwdDeps, Target::getLabel);
   }
 
   private <R> ListenableFuture<R> safeSubmit(Callable<R> callable) {
diff --git a/src/main/java/com/google/devtools/build/lib/query2/SkyQueryUtils.java b/src/main/java/com/google/devtools/build/lib/query2/SkyQueryUtils.java
new file mode 100644
index 0000000..002b9c0
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/lib/query2/SkyQueryUtils.java
@@ -0,0 +1,95 @@
+// Copyright 2017 The Bazel Authors. All rights reserved.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//    http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package com.google.devtools.build.lib.query2;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+import com.google.devtools.build.lib.cmdline.Label;
+import com.google.devtools.build.lib.graph.Digraph;
+import com.google.devtools.build.lib.packages.Target;
+import com.google.devtools.build.lib.query2.engine.QueryEnvironment.ThreadSafeMutableSet;
+import com.google.devtools.build.lib.util.Preconditions;
+import java.util.ArrayDeque;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+/** Utility class for Skyframe-based query implementations. */
+class SkyQueryUtils {
+  interface GetFwdDeps<T> {
+    ThreadSafeMutableSet<T> getFwdDeps(Iterable<T> t) throws InterruptedException;
+  }
+
+  static <T> ThreadSafeMutableSet<T> getTransitiveClosure(
+      ThreadSafeMutableSet<T> targets, GetFwdDeps<T> getFwdDeps, ThreadSafeMutableSet<T> visited)
+      throws InterruptedException {
+    ThreadSafeMutableSet<T> current = targets;
+    while (!current.isEmpty()) {
+      Iterable<T> toVisit =
+          current.stream().filter(obj -> !visited.contains(obj)).collect(Collectors.toList());
+      current = getFwdDeps.getFwdDeps(toVisit);
+      Iterables.addAll(visited, toVisit);
+    }
+    return visited;
+  }
+
+  /**
+   * Gets a path from {@code from} to {@code to}, walking the graph revealed by {@code getFwdDeps}.
+   *
+   * <p>In case the type {@link T} does not implement equality, {@code label} will be used to map
+   * elements of type {@link T} to elements of type {@link L} which does implement equality. {@code
+   * label} should be an injective function. For instance, if {@link T} is of type {@link Target}
+   * then {@link L} could be of type {@link Label} and {@code label} could be {@link
+   * Target::getLabel}.
+   *
+   * <p>Implemented with a breadth-first search.
+   */
+  static <T, L> ImmutableList<T> getNodesOnPath(
+      T from, T to, GetFwdDeps<T> getFwdDeps, Function<T, L> label) throws InterruptedException {
+    // Tree of nodes visited so far.
+    Map<L, L> nodeToParent = new HashMap<>();
+    Map<L, T> labelToTarget = new HashMap<>();
+    // Contains all nodes left to visit in a (LIFO) stack.
+    Deque<T> toVisit = new ArrayDeque<>();
+    toVisit.add(from);
+    nodeToParent.put(label.apply(from), null);
+    labelToTarget.put(label.apply(from), from);
+    while (!toVisit.isEmpty()) {
+      T current = toVisit.removeFirst();
+      if (label.apply(to).equals(label.apply(current))) {
+        List<L> labelPath = Digraph.getPathToTreeNode(nodeToParent, label.apply(to));
+        ImmutableList.Builder<T> targetPathBuilder = ImmutableList.builder();
+        for (L item : labelPath) {
+          targetPathBuilder.add(Preconditions.checkNotNull(labelToTarget.get(item), item));
+        }
+        return targetPathBuilder.build();
+      }
+      for (T dep : getFwdDeps.getFwdDeps(ImmutableList.of(current))) {
+        L depLabel = label.apply(dep);
+        if (!nodeToParent.containsKey(depLabel)) {
+          nodeToParent.put(depLabel, label.apply(current));
+          labelToTarget.put(depLabel, dep);
+          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;
+  }
+}
diff --git a/src/main/java/com/google/devtools/build/lib/skyframe/SkyframeExecutorWrappingWalkableGraph.java b/src/main/java/com/google/devtools/build/lib/skyframe/SkyframeExecutorWrappingWalkableGraph.java
new file mode 100644
index 0000000..42029c1
--- /dev/null
+++ b/src/main/java/com/google/devtools/build/lib/skyframe/SkyframeExecutorWrappingWalkableGraph.java
@@ -0,0 +1,64 @@
+// Copyright 2017 The Bazel Authors. All rights reserved.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//    http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+package com.google.devtools.build.lib.skyframe;
+
+import com.google.devtools.build.skyframe.DelegatingWalkableGraph;
+import com.google.devtools.build.skyframe.MemoizingEvaluator;
+import com.google.devtools.build.skyframe.NodeEntry;
+import com.google.devtools.build.skyframe.QueryableGraph;
+import com.google.devtools.build.skyframe.SkyKey;
+import com.google.devtools.build.skyframe.WalkableGraph;
+import java.util.HashMap;
+import java.util.Map;
+import javax.annotation.Nullable;
+
+/** {@link WalkableGraph} backed by a {@link SkyframeExecutor}. */
+public class SkyframeExecutorWrappingWalkableGraph extends DelegatingWalkableGraph {
+
+  public SkyframeExecutorWrappingWalkableGraph(MemoizingEvaluator evaluator) {
+    super(
+        new QueryableGraph() {
+          @Nullable
+          @Override
+          public NodeEntry get(@Nullable SkyKey requestor, Reason reason, SkyKey key)
+              throws InterruptedException {
+            return evaluator.getGraphMap().get(key);
+          }
+
+          @Override
+          public Map<SkyKey, ? extends NodeEntry> getBatch(
+              @Nullable SkyKey requestor, Reason reason, Iterable<? extends SkyKey> keys)
+              throws InterruptedException {
+            Map<SkyKey, NodeEntry> result = new HashMap<>();
+            for (SkyKey key : keys) {
+              NodeEntry nodeEntry = get(requestor, reason, key);
+              if (nodeEntry != null) {
+                result.put(key, nodeEntry);
+              }
+            }
+            return result;
+          }
+
+          @Override
+          public Iterable<SkyKey> getCurrentlyAvailableNodes(Iterable<SkyKey> keys, Reason reason) {
+            return keys;
+          }
+        });
+  }
+
+  public static SkyframeExecutorWrappingWalkableGraph of(SkyframeExecutor skyframeExecutor) {
+    // TODO(janakr): Provide the graph in a more principled way.
+    return new SkyframeExecutorWrappingWalkableGraph(skyframeExecutor.getEvaluatorForTesting());
+  }
+}