Remove dead code in SkyframeLabelVisitor

Two of the functions are never used, several of the parameter are not
used or can be simplified (require callers to convert from Target ->
Label).

--
PiperOrigin-RevId: 146787380
MOS_MIGRATED_REVID=146787380
diff --git a/src/main/java/com/google/devtools/build/lib/pkgcache/TransitivePackageLoader.java b/src/main/java/com/google/devtools/build/lib/pkgcache/TransitivePackageLoader.java
index 45c7c93..0b40bda8 100644
--- a/src/main/java/com/google/devtools/build/lib/pkgcache/TransitivePackageLoader.java
+++ b/src/main/java/com/google/devtools/build/lib/pkgcache/TransitivePackageLoader.java
@@ -13,12 +13,8 @@
 // limitations under the License.
 package com.google.devtools.build.lib.pkgcache;
 
-import com.google.common.collect.Multimap;
 import com.google.devtools.build.lib.cmdline.Label;
-import com.google.devtools.build.lib.cmdline.PackageIdentifier;
 import com.google.devtools.build.lib.events.EventHandler;
-import com.google.devtools.build.lib.packages.Target;
-
 import java.util.Set;
 
 /**
@@ -32,36 +28,12 @@
    * may do an up-to-date check which will not trigger any of the observers.
    *
    * @param eventHandler the error and warnings eventHandler; must be thread-safe
-   * @param targetsToVisit the targets to visit
    * @param labelsToVisit the labels to visit in addition to the targets
-   * @param keepGoing if false, stop visitation upon first error.
-   * @param parallelThreads number of threads to use in the visitation.
-   * @param maxDepth the maximum depth to traverse to.
+   * @param keepGoing if false, stop visitation upon first error
+   * @param parallelThreads number of threads to use in the visitation
    */
   boolean sync(EventHandler eventHandler,
-               Set<Target> targetsToVisit,
                Set<Label> labelsToVisit,
                boolean keepGoing,
-               int parallelThreads,
-               int maxDepth) throws InterruptedException;
-
-  /**
-   * Returns a read-only view of the set of packages visited since this visitor
-   * was constructed.
-   *
-   * <p>Not thread-safe; do not call during visitation.
-   */
-  Set<PackageIdentifier> getVisitedPackageNames();
-
-  /**
-   * Return a mapping between the specified top-level targets and root causes. Note that targets in
-   * the input that are transitively error free will not be in the output map. "Top-level" targets
-   * are the targetsToVisit and labelsToVisit specified in the last sync.
-   *
-   * <p>May only be called once a keep_going visitation is complete, and prior to
-   * trimErrorTracking().
-   *
-   * @return a mapping of targets to root causes
-   */
-  Multimap<Label, Label> getRootCauses();
+               int parallelThreads) throws InterruptedException;
 }
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 0a74cc0..cfd92cf 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
@@ -15,6 +15,7 @@
 
 import com.google.common.base.Function;
 import com.google.common.base.Predicate;
+import com.google.common.collect.Collections2;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Maps;
 import com.google.devtools.build.lib.cmdline.Label;
@@ -67,6 +68,12 @@
  * The environment of a Blaze query. Not thread-safe.
  */
 public class BlazeQueryEnvironment extends AbstractBlazeQueryEnvironment<Target> {
+  private static final Function<Target, Label> TO_LABEL = new Function<Target, Label>() {
+    @Override
+    public Label apply(Target input) {
+      return input.getLabel();
+    }
+  };
 
   private static final int MAX_DEPTH_FULL_SCAN_LIMIT = 20;
   private final Map<String, Set<Target>> resolvedTargetPatterns = new HashMap<>();
@@ -303,12 +310,12 @@
   }
 
   private void preloadTransitiveClosure(Set<Target> targets, int maxDepth)
-      throws QueryException, InterruptedException {
+      throws InterruptedException {
     if (maxDepth >= MAX_DEPTH_FULL_SCAN_LIMIT && transitivePackageLoader != null) {
       // Only do the full visitation if "maxDepth" is large enough. Otherwise, the benefits of
       // preloading will be outweighed by the cost of doing more work than necessary.
-      transitivePackageLoader.sync(
-          eventHandler, targets, ImmutableSet.<Label>of(), keepGoing, loadingPhaseThreads, -1);
+      Set<Label> labels = ImmutableSet.copyOf(Collections2.transform(targets, TO_LABEL));
+      transitivePackageLoader.sync(eventHandler, labels, keepGoing, loadingPhaseThreads);
     }
   }
 
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 a14e8b6..06017be 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
@@ -90,7 +90,6 @@
 import com.google.devtools.build.lib.packages.Preprocessor.AstAfterPreprocessing;
 import com.google.devtools.build.lib.packages.RuleClassProvider;
 import com.google.devtools.build.lib.packages.RuleVisibility;
-import com.google.devtools.build.lib.packages.Target;
 import com.google.devtools.build.lib.pkgcache.LoadingCallback;
 import com.google.devtools.build.lib.pkgcache.LoadingFailedException;
 import com.google.devtools.build.lib.pkgcache.LoadingOptions;
@@ -1523,18 +1522,14 @@
     /**
      * Loads the specified {@link TransitiveTargetValue}s.
      */
-    EvaluationResult<TransitiveTargetValue> loadTransitiveTargets(EventHandler eventHandler,
-            Iterable<Target> targetsToVisit, Iterable<Label> labelsToVisit, boolean keepGoing,
-            int parallelThreads)
+    EvaluationResult<TransitiveTargetValue> loadTransitiveTargets(
+        EventHandler eventHandler, Iterable<Label> labelsToVisit, boolean keepGoing,
+        int parallelThreads)
         throws InterruptedException {
       List<SkyKey> valueNames = new ArrayList<>();
-      for (Target target : targetsToVisit) {
-        valueNames.add(TransitiveTargetValue.key(target.getLabel()));
-      }
       for (Label label : labelsToVisit) {
         valueNames.add(TransitiveTargetValue.key(label));
       }
-
       return buildDriver.evaluate(valueNames, keepGoing, parallelThreads, eventHandler);
     }
   }
diff --git a/src/main/java/com/google/devtools/build/lib/skyframe/SkyframeLabelVisitor.java b/src/main/java/com/google/devtools/build/lib/skyframe/SkyframeLabelVisitor.java
index 38c2111..6b1e6ca 100644
--- a/src/main/java/com/google/devtools/build/lib/skyframe/SkyframeLabelVisitor.java
+++ b/src/main/java/com/google/devtools/build/lib/skyframe/SkyframeLabelVisitor.java
@@ -13,32 +13,20 @@
 // limitations under the License.
 package com.google.devtools.build.lib.skyframe;
 
-import com.google.common.collect.HashMultimap;
-import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Iterables;
-import com.google.common.collect.Multimap;
 import com.google.devtools.build.lib.cmdline.Label;
-import com.google.devtools.build.lib.cmdline.PackageIdentifier;
-import com.google.devtools.build.lib.collect.nestedset.NestedSetBuilder;
 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.TransitivePackageLoader;
 import com.google.devtools.build.lib.skyframe.SkyframeExecutor.SkyframeTransitivePackageLoader;
 import com.google.devtools.build.lib.util.Preconditions;
-import com.google.devtools.build.skyframe.CycleInfo;
 import com.google.devtools.build.skyframe.CyclesReporter;
 import com.google.devtools.build.skyframe.ErrorInfo;
 import com.google.devtools.build.skyframe.EvaluationResult;
 import com.google.devtools.build.skyframe.SkyKey;
-
-import java.util.Collection;
-import java.util.HashSet;
 import java.util.Map.Entry;
-import java.util.Objects;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicReference;
-
 import javax.annotation.Nullable;
 
 /**
@@ -49,27 +37,18 @@
   private final SkyframeTransitivePackageLoader transitivePackageLoader;
   private final AtomicReference<CyclesReporter> skyframeCyclesReporter;
 
-  private Set<PackageIdentifier> allVisitedPackages;
-  private Set<TransitiveTargetValue> previousBuildTargetValueSet = null;
-  private boolean lastBuildKeepGoing;
-  private final Multimap<Label, Label> rootCauses = HashMultimap.create();
-
   SkyframeLabelVisitor(SkyframeTransitivePackageLoader transitivePackageLoader,
       AtomicReference<CyclesReporter> skyframeCyclesReporter) {
     this.transitivePackageLoader = transitivePackageLoader;
     this.skyframeCyclesReporter = skyframeCyclesReporter;
   }
 
+  // The only remaining non-test caller of this code is BlazeQueryEnvironment.
   @Override
-  public boolean sync(EventHandler eventHandler, Set<Target> targetsToVisit,
-      Set<Label> labelsToVisit, boolean keepGoing, int parallelThreads, int maxDepth)
-      throws InterruptedException {
-    rootCauses.clear();
-    lastBuildKeepGoing = false;
+  public boolean sync(EventHandler eventHandler, Set<Label> labelsToVisit, boolean keepGoing,
+      int parallelThreads) throws InterruptedException {
     EvaluationResult<TransitiveTargetValue> result = transitivePackageLoader.loadTransitiveTargets(
-        eventHandler, targetsToVisit, labelsToVisit, keepGoing, parallelThreads);
-    updateVisitedValues(result.values());
-    lastBuildKeepGoing = keepGoing;
+        eventHandler, labelsToVisit, keepGoing, parallelThreads);
 
     if (!hasErrors(result)) {
       return true;
@@ -104,8 +83,6 @@
       Label topLevelLabel = (Label) key.argument();
       if (!Iterables.isEmpty(errorInfo.getCycleInfo())) {
         skyframeCyclesReporter.get().reportCycles(errorInfo.getCycleInfo(), key, eventHandler);
-        rootCauses.putAll(
-            topLevelLabel, getRootCausesOfCycles(topLevelLabel, errorInfo.getCycleInfo()));
       }
       if (isDirectErrorFromTopLevelLabel(topLevelLabel, labelsToVisit, errorInfo)) {
         // Unlike top-level targets, which have already gone through target parsing,
@@ -115,23 +92,11 @@
         eventHandler.handle(Event.error(errorInfo.getException().getMessage()));
       }
       warnAboutLoadingFailure(topLevelLabel, eventHandler);
-      for (SkyKey badKey : errorInfo.getRootCauses()) {
-        if (badKey.functionName().equals(SkyFunctions.PACKAGE)) {
-          // Transitive target function may ask for a Package, but don't include this in the root
-          // causes. We'll get more precise information from dependencies on transitive and direct
-          // target dependencies.
-          continue;
-        }
-        Preconditions.checkState(badKey.argument() instanceof Label,
-            "%s %s %s", key, errorInfo, badKey);
-        rootCauses.put(topLevelLabel, (Label) badKey.argument());
-      }
     }
     for (Label topLevelLabel : result.<Label>keyNames()) {
       SkyKey topLevelTransitiveTargetKey = TransitiveTargetValue.key(topLevelLabel);
       TransitiveTargetValue topLevelTransitiveTargetValue = result.get(topLevelTransitiveTargetKey);
       if (topLevelTransitiveTargetValue.getTransitiveRootCauses() != null) {
-        rootCauses.putAll(topLevelLabel, topLevelTransitiveTargetValue.getTransitiveRootCauses());
         warnAboutLoadingFailure(topLevelLabel, eventHandler);
       }
     }
@@ -166,62 +131,4 @@
   private static void warnAboutLoadingFailure(Label label, EventHandler eventHandler) {
     eventHandler.handle(Event.warn("errors encountered while loading target '" + label + "'"));
   }
-
-  private static Set<Label> getRootCausesOfCycles(Label labelToLoad, Iterable<CycleInfo> cycles) {
-    ImmutableSet.Builder<Label> builder = ImmutableSet.builder();
-    for (CycleInfo cycleInfo : cycles) {
-      // The root cause of a cycle depends on the type of a cycle.
-
-      SkyKey culprit = Iterables.getFirst(cycleInfo.getCycle(), null);
-      if (culprit == null) {
-        continue;
-      }
-      if (culprit.functionName().equals(SkyFunctions.TRANSITIVE_TARGET)) {
-        // For a cycle between build targets, the root cause is the first element of the cycle.
-        builder.add((Label) culprit.argument());
-      } else {
-        // For other types of cycles (e.g. file symlink cycles), the root cause is the furthest
-        // target dependency that itself depended on the cycle.
-        Label furthestTarget = labelToLoad;
-        for (SkyKey skyKey : cycleInfo.getPathToCycle()) {
-          if (skyKey.functionName().equals(SkyFunctions.TRANSITIVE_TARGET)) {
-            furthestTarget = (Label) skyKey.argument();
-          } else {
-            break;
-          }
-        }
-        builder.add(furthestTarget);
-      }
-    }
-    return builder.build();
-  }
-
-  // Unfortunately we have to do an effective O(TC) visitation after the eval() call above to
-  // determine all of the packages in the closure.
-  private void updateVisitedValues(Collection<TransitiveTargetValue> targetValues) {
-    Set<TransitiveTargetValue> currentBuildTargetValueSet = new HashSet<>(targetValues);
-    if (Objects.equals(previousBuildTargetValueSet, currentBuildTargetValueSet)) {
-      // The next stanza is slow (and scales with the edge count of the target graph), so avoid
-      // the computation if the previous build already did it.
-      return;
-    }
-    NestedSetBuilder<PackageIdentifier> nestedAllPkgsBuilder = NestedSetBuilder.stableOrder();
-    for (TransitiveTargetValue value : targetValues) {
-      nestedAllPkgsBuilder.addTransitive(value.getTransitiveSuccessfulPackages());
-      nestedAllPkgsBuilder.addTransitive(value.getTransitiveUnsuccessfulPackages());
-    }
-    allVisitedPackages = nestedAllPkgsBuilder.build().toSet();
-    previousBuildTargetValueSet = currentBuildTargetValueSet;
-  }
-
-  @Override
-  public Set<PackageIdentifier> getVisitedPackageNames() {
-    return allVisitedPackages;
-  }
-
-  @Override
-  public Multimap<Label, Label> getRootCauses() {
-    Preconditions.checkState(lastBuildKeepGoing);
-    return rootCauses;
-  }
 }