Clean up DependencyResolver's interface for the dynamic config migration and for general readability.

Major changes:
 - Remove the intermediate Attribute -> LabelAndConfiguration multimap (computed in resolveAttributes). Instead, feed discovered values directly into the final Attribute -> Dependency map via a new RuleResolver interface.
 - Remove all references to LabelAndConfiguration. The configuration is always the owning rule's configuration except for two special cases: late-bound attributes with splits and late-bound attributes with LateBoundDefault.useHostConfiguration. The original interface made this very unclear and required a lot of awkward and sometimes incorrect logic. The new interface only involves configurations for the cases that actually need them.
 - Remove an ugly hack caused by BuildConfiguration.evaluateTransition mixing poorly with LateBoundDefault.useHostConfiguration (https://github.com/bazelbuild/bazel/blo[]e172693c27f3efc95ed163e43a9f0a7a6fb4017/src/main/java/com/google/devtools/build/lib/analysis/DependencyResolver.java#L488).
 - Remove a hack that applies split transitions twice because of BuildConfiguration.evaluateTransition mixing poorly with late-bound split attributes (https://github.com/bazelbuild/bazel/blo[]e172693c27f3efc95ed163e43a9f0a7a6fb4017/src/main/java/com/google/devtools/build/lib/analysis/DependencyResolver.java#L319). This happens to be innocent now but won't be when nested splits are possible.
- Solidifies the API contract for Attribute.LateBoundDefault.useHostConfiguration.
- Applies clearer naming and more consistent ordering to method parameters.
- Better documentation.

This is all also prep work for dynamic split transitions.

tl;dr: late-bound attributes are legitimately special. Treat them that way to make the rest of DependencyResolver cleaner and hack-free.

--
MOS_MIGRATED_REVID=128582618
diff --git a/src/main/java/com/google/devtools/build/lib/analysis/DependencyResolver.java b/src/main/java/com/google/devtools/build/lib/analysis/DependencyResolver.java
index f71f853..ec52681 100644
--- a/src/main/java/com/google/devtools/build/lib/analysis/DependencyResolver.java
+++ b/src/main/java/com/google/devtools/build/lib/analysis/DependencyResolver.java
@@ -13,20 +13,20 @@
 // limitations under the License.
 package com.google.devtools.build.lib.analysis;
 
+import com.google.common.base.Verify;
 import com.google.common.collect.ArrayListMultimap;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterables;
 import com.google.common.collect.ListMultimap;
 import com.google.common.collect.Sets;
 import com.google.devtools.build.lib.analysis.config.BuildConfiguration;
 import com.google.devtools.build.lib.analysis.config.ConfigMatchingProvider;
 import com.google.devtools.build.lib.cmdline.Label;
-import com.google.devtools.build.lib.collect.ImmutableSortedKeyListMultimap;
 import com.google.devtools.build.lib.collect.nestedset.NestedSetBuilder;
 import com.google.devtools.build.lib.packages.Aspect;
 import com.google.devtools.build.lib.packages.AspectClass;
-import com.google.devtools.build.lib.packages.AspectDefinition;
 import com.google.devtools.build.lib.packages.Attribute;
 import com.google.devtools.build.lib.packages.Attribute.LateBoundDefault;
 import com.google.devtools.build.lib.packages.AttributeMap;
@@ -80,16 +80,24 @@
    * <p>The long-term goal is that most configuration transitions be applied here. However, in order
    * to do that, we first have to eliminate transitions that depend on the rule class of the
    * dependency.
+   *
+   * @param node the target/configuration being evaluated
+   * @param hostConfig the configuration this target would use if it was evaluated as a host
+   *     tool. This is needed to support {@link LateBoundDefault#useHostConfiguration()}.
+   * @param aspect the aspect applied to this target (if any)
+   * @param configConditions resolver for config_setting labels
+   *
+   * @return a mapping of each attribute in this rule or aspect to its dependent nodes
    */
   public final ListMultimap<Attribute, Dependency> dependentNodeMap(
       TargetAndConfiguration node,
       BuildConfiguration hostConfig,
-      Aspect aspect,
+      @Nullable Aspect aspect,
       ImmutableMap<Label, ConfigMatchingProvider> configConditions)
       throws EvalException, InterruptedException {
     NestedSetBuilder<Label> rootCauses = NestedSetBuilder.<Label>stableOrder();
-    ListMultimap<Attribute, Dependency> outgoingEdges =
-        dependentNodeMap(node, hostConfig, aspect, configConditions, rootCauses);
+    ListMultimap<Attribute, Dependency> outgoingEdges = dependentNodeMap(
+        node, hostConfig, aspect, configConditions, rootCauses);
     if (!rootCauses.isEmpty()) {
       throw new IllegalStateException(rootCauses.build().iterator().next().toString());
     }
@@ -114,11 +122,20 @@
    * <p>The long-term goal is that most configuration transitions be applied here. However, in order
    * to do that, we first have to eliminate transitions that depend on the rule class of the
    * dependency.
+   *
+   * @param node the target/configuration being evaluated
+   * @param hostConfig the configuration this target would use if it was evaluated as a host
+   *     tool. This is needed to support {@link LateBoundDefault#useHostConfiguration()}.
+   * @param aspect the aspect applied to this target (if any)
+   * @param configConditions resolver for config_setting labels
+   * @param rootCauses collector for dep labels that can't be (loading phase) loaded
+   *
+   * @return a mapping of each attribute in this rule or aspect to its dependent nodes
    */
   public final ListMultimap<Attribute, Dependency> dependentNodeMap(
       TargetAndConfiguration node,
       BuildConfiguration hostConfig,
-      Aspect aspect,
+      @Nullable Aspect aspect,
       ImmutableMap<Label, ConfigMatchingProvider> configConditions,
       NestedSetBuilder<Label> rootCauses)
       throws EvalException, InterruptedException {
@@ -135,17 +152,7 @@
     } else if (target instanceof EnvironmentGroup) {
       visitTargetVisibility(node, rootCauses, outgoingEdges.get(null));
     } else if (target instanceof Rule) {
-      Preconditions.checkNotNull(config);
-      visitTargetVisibility(node, rootCauses, outgoingEdges.get(null));
-      Rule rule = (Rule) target;
-      ListMultimap<Attribute, LabelAndConfiguration> labelMap =
-          resolveAttributes(
-              rule,
-              aspect != null ? aspect.getDefinition() : null,
-              config,
-              hostConfig,
-              configConditions);
-      visitRule(rule, aspect, labelMap, rootCauses, outgoingEdges);
+      visitRule(node, hostConfig, aspect, configConditions, rootCauses, outgoingEdges);
     } else if (target instanceof PackageGroup) {
       visitPackageGroup(node, (PackageGroup) target, rootCauses, outgoingEdges.get(null));
     } else {
@@ -154,33 +161,41 @@
     return outgoingEdges;
   }
 
-  private ListMultimap<Attribute, LabelAndConfiguration> resolveAttributes(
-      Rule rule, AspectDefinition aspect, BuildConfiguration configuration,
-      BuildConfiguration hostConfiguration,
-      ImmutableMap<Label, ConfigMatchingProvider> configConditions)
+  private void visitRule(
+      TargetAndConfiguration node,
+      BuildConfiguration hostConfig,
+      @Nullable Aspect aspect,
+      ImmutableMap<Label, ConfigMatchingProvider> configConditions,
+      NestedSetBuilder<Label> rootCauses,
+      ListMultimap<Attribute, Dependency> outgoingEdges)
       throws EvalException, InterruptedException {
+    Preconditions.checkArgument(node.getTarget() instanceof Rule);
+    BuildConfiguration ruleConfig = Preconditions.checkNotNull(node.getConfiguration());
+    Rule rule = (Rule) node.getTarget();
+
     ConfiguredAttributeMapper attributeMap = ConfiguredAttributeMapper.of(rule, configConditions);
     attributeMap.validateAttributes();
-    List<Attribute> attributes;
-    if (aspect == null) {
-      attributes = rule.getRuleClassObject().getAttributes();
-    } else {
-      attributes = new ArrayList<>();
-      attributes.addAll(rule.getRuleClassObject().getAttributes());
-      attributes.addAll(aspect.getAttributes().values());
-    }
+    RuleResolver depResolver =
+        new RuleResolver(rule, ruleConfig, aspect, attributeMap, rootCauses, outgoingEdges);
 
-    ImmutableSortedKeyListMultimap.Builder<Attribute, LabelAndConfiguration> result =
-        ImmutableSortedKeyListMultimap.builder();
+    visitTargetVisibility(node, rootCauses, outgoingEdges.get(null));
+    resolveEarlyBoundAttributes(depResolver);
+    resolveLateBoundAttributes(depResolver, ruleConfig, hostConfig);
+  }
 
-    resolveExplicitAttributes(configuration, attributeMap, result);
-    resolveImplicitAttributes(rule, configuration, attributeMap, attributes, result);
-    resolveLateBoundAttributes(rule, configuration, hostConfiguration, attributeMap, attributes,
-        result);
+  /**
+   * Resolves the dependencies for all attributes in this rule except late-bound attributes
+   * (which require special processing: see {@link #resolveLateBoundAttributes}).
+   */
+  private void resolveEarlyBoundAttributes(RuleResolver depResolver)
+      throws EvalException, InterruptedException {
+    Rule rule = depResolver.rule;
+
+    resolveExplicitAttributes(depResolver);
+    resolveImplicitAttributes(depResolver);
 
     // Add the rule's visibility labels (which may come from the rule or from package defaults).
-    addExplicitDeps(result, rule, "visibility", rule.getVisibility().getDependencyLabels(),
-        configuration);
+    addExplicitDeps(depResolver, "visibility", rule.getVisibility().getDependencyLabels());
 
     // Add package default constraints when the rule doesn't explicitly declare them.
     //
@@ -206,46 +221,17 @@
     // above). But within the scope of a single package it seems better to keep the model simple and
     // make the user responsible for resolving ambiguities.
     if (!rule.isAttributeValueExplicitlySpecified(RuleClass.COMPATIBLE_ENVIRONMENT_ATTR)) {
-      addExplicitDeps(result, rule, RuleClass.COMPATIBLE_ENVIRONMENT_ATTR,
-          rule.getPackage().getDefaultCompatibleWith(), configuration);
+      addExplicitDeps(depResolver, RuleClass.COMPATIBLE_ENVIRONMENT_ATTR,
+          rule.getPackage().getDefaultCompatibleWith());
     }
     if (!rule.isAttributeValueExplicitlySpecified(RuleClass.RESTRICTED_ENVIRONMENT_ATTR)) {
-      addExplicitDeps(result, rule, RuleClass.RESTRICTED_ENVIRONMENT_ATTR,
-          rule.getPackage().getDefaultRestrictedTo(), configuration);
-    }
-
-    return result.build();
-  }
-
-  /**
-   * Adds new dependencies to the given rule under the given attribute name
-   *
-   * @param result the builder for the attribute --> dependency labels map
-   * @param rule the rule being evaluated
-   * @param attrName the name of the attribute to add dependency labels to
-   * @param labels the dependencies to add
-   * @param configuration the configuration to apply to those dependencies
-   */
-  private void addExplicitDeps(
-      ImmutableSortedKeyListMultimap.Builder<Attribute, LabelAndConfiguration> result, Rule rule,
-      String attrName, Iterable<Label> labels, BuildConfiguration configuration) {
-    if (!rule.isAttrDefined(attrName, BuildType.LABEL_LIST)
-        && !rule.isAttrDefined(attrName, BuildType.NODEP_LABEL_LIST)) {
-      return;
-    }
-    Attribute attribute = rule.getRuleClassObject().getAttributeByName(attrName);
-    for (Label label : labels) {
-      // The configuration must be the configuration after the first transition step (applying
-      // split configurations). The proper configuration (null) for package groups will be set
-      // later.
-      result.put(attribute, LabelAndConfiguration.of(label, configuration));
+      addExplicitDeps(depResolver, RuleClass.RESTRICTED_ENVIRONMENT_ATTR,
+          rule.getPackage().getDefaultRestrictedTo());
     }
   }
 
-  private void resolveExplicitAttributes(final BuildConfiguration configuration,
-      AttributeMap attributes,
-      final ImmutableSortedKeyListMultimap.Builder<Attribute, LabelAndConfiguration> builder) {
-    attributes.visitLabels(
+  private void resolveExplicitAttributes(final RuleResolver depResolver) {
+    depResolver.attributeMap.visitLabels(
         new AttributeMap.AcceptsLabelAttribute() {
           @Override
           public void acceptLabelAttribute(Label label, Attribute attribute) {
@@ -253,21 +239,24 @@
                 || attribute.isLateBound()) {
               return;
             }
-            builder.put(attribute, LabelAndConfiguration.of(label, configuration));
+            depResolver.resolveDep(attribute, label);
           }
         });
   }
 
-  private void resolveImplicitAttributes(Rule rule, BuildConfiguration configuration,
-      AttributeMap attributeMap, Iterable<Attribute> attributes,
-      ImmutableSortedKeyListMultimap.Builder<Attribute, LabelAndConfiguration> builder) {
+  /**
+   * Resolves the dependencies for all implicit attributes in this rule.
+   */
+  private void resolveImplicitAttributes(RuleResolver depResolver) {
     // Since the attributes that come from aspects do not appear in attributeMap, we have to get
     // their values from somewhere else. This incidentally means that aspects attributes are not
     // configurable. It would be nice if that wasn't the case, but we'd have to revamp how
     // attribute mapping works, which is a large chunk of work.
+    Rule rule = depResolver.rule;
     Label ruleLabel = rule.getLabel();
+    ConfiguredAttributeMapper attributeMap = depResolver.attributeMap;
     ImmutableSet<String> mappedAttributes = ImmutableSet.copyOf(attributeMap.getAttributeNames());
-    for (Attribute attribute : attributes) {
+    for (Attribute attribute : depResolver.attributes) {
       if (!attribute.isImplicit() || !attribute.getCondition().apply(attributeMap)) {
         continue;
       }
@@ -279,7 +268,7 @@
 
         if (label != null) {
           label = ruleLabel.resolveRepositoryRelative(label);
-          builder.put(attribute, LabelAndConfiguration.of(label, configuration));
+          depResolver.resolveDep(attribute, label);
         }
       } else if (attribute.getType() == BuildType.LABEL_LIST) {
         List<Label> labelList;
@@ -291,101 +280,181 @@
         } else {
           labelList = BuildType.LABEL_LIST.cast(attribute.getDefaultValue(rule));
         }
-
         for (Label label : labelList) {
-          label = ruleLabel.resolveRepositoryRelative(label);
-          builder.put(attribute, LabelAndConfiguration.of(label, configuration));
+          depResolver.resolveDep(attribute, ruleLabel.resolveRepositoryRelative(label));
         }
       }
     }
   }
 
+  /**
+   * Resolves the dependencies for all late-bound attributes in this rule.
+   *
+   * <p>Late-bound attributes need special handling because they require configuration
+   * transitions to determine their values.
+   *
+   * <p>In other words, the normal process of dependency resolution is:
+   * <ol>
+   *   <li>Find every label value in the rule's attributes</li>
+   *   <li>Apply configuration transitions over each value to get its dep configuration
+   *   <li>Return each value with its dep configuration</li>
+   * </ol>
+   *
+   * This doesn't work for late-bound attributes because you can't get their values without
+   * knowing the configuration first. And that configuration may not be the owning rule's
+   * configuration. Specifically, {@link LateBoundDefault#useHostConfiguration()} switches to the
+   * host config and late-bound split attributes branch into multiple split configs.
+   *
+   * <p>This method implements that logic and makes sure the normal configuration
+   * transition logic mixes with it cleanly.
+   *
+   * @param depResolver the resolver for this rule's deps
+   * @param ruleConfig the rule's configuration
+   * @param hostConfig the equivalent host configuration
+   */
   private void resolveLateBoundAttributes(
-      Rule rule,
-      BuildConfiguration configuration,
-      BuildConfiguration hostConfiguration,
-      AttributeMap attributeMap,
-      Iterable<Attribute> attributes,
-      ImmutableSortedKeyListMultimap.Builder<Attribute, LabelAndConfiguration> builder)
+      RuleResolver depResolver,
+      BuildConfiguration ruleConfig,
+      BuildConfiguration hostConfig)
       throws EvalException, InterruptedException {
-    for (Attribute attribute : attributes) {
+    ConfiguredAttributeMapper attributeMap = depResolver.attributeMap;
+    for (Attribute attribute : depResolver.attributes) {
       if (!attribute.isLateBound() || !attribute.getCondition().apply(attributeMap)) {
         continue;
       }
 
-      List<BuildConfiguration> actualConfigurations = ImmutableList.of(configuration);
+      @SuppressWarnings("unchecked")
+      LateBoundDefault<BuildConfiguration> lateBoundDefault =
+        (LateBoundDefault<BuildConfiguration>) attribute.getLateBoundDefault();
+
       if (attribute.hasSplitConfigurationTransition()) {
-        Preconditions.checkState(attribute.getConfigurator() == null);
-        // TODO(bazel-team): This ends up applying the split transition twice, both here and in the
-        // visitRule method below - this is not currently a problem, because the configuration graph
-        // never contains nested split transitions, so the second application is idempotent.
-        actualConfigurations =
-            configuration.getSplitConfigurations(attribute.getSplitTransition(rule));
-      }
-
-      for (BuildConfiguration actualConfig : actualConfigurations) {
-        @SuppressWarnings("unchecked")
-        LateBoundDefault<BuildConfiguration> lateBoundDefault =
-            (LateBoundDefault<BuildConfiguration>) attribute.getLateBoundDefault();
-        if (lateBoundDefault.useHostConfiguration()) {
-          actualConfig = hostConfiguration;
-        }
-        // TODO(bazel-team): This might be too expensive - can we cache this somehow?
-        if (!lateBoundDefault.getRequiredConfigurationFragments().isEmpty()) {
-          if (!actualConfig.hasAllFragments(lateBoundDefault.getRequiredConfigurationFragments())) {
-            continue;
+        // Late-bound attribute with a split transition:
+        // Since we want to get the same results as BuildConfiguration.evaluateTransition (but
+        // skip it since we've already applied the split), we want to make sure this logic
+        // doesn't do anything differently. evaluateTransition has additional logic
+        // for host configs and attributes with configurators. So we check here that neither of
+        // of those apply, in the name of keeping the fork as simple as possible.
+        Verify.verify(attribute.getConfigurator() == null);
+        Verify.verify(!lateBoundDefault.useHostConfiguration());
+        for (BuildConfiguration splitConfig : ruleConfig.getSplitConfigurations(
+            attribute.getSplitTransition(depResolver.rule))) {
+          // TODO(gregce): support dynamic split transitions
+          for (Label dep : resolveLateBoundAttribute(
+              depResolver.rule, attribute, splitConfig, attributeMap)) {
+            // Skip the normal config transition pipeline and directly feed the split config. This
+            // is because the split already had to be applied to determine the attribute's value.
+            // This makes the split logic in the normal pipeline redundant and potentially
+            // incorrect.
+            depResolver.resolveDep(attribute, dep, splitConfig);
           }
         }
-
-        // TODO(bazel-team): We should check if the implementation tries to access an undeclared
-        // fragment.
-        Object actualValue = lateBoundDefault.resolve(rule, attributeMap, actualConfig);
-        if (EvalUtils.isNullOrNone(actualValue)) {
-          continue;
-        }
-        try {
-          if (attribute.getType() == BuildType.LABEL) {
-            Label label = rule.getLabel().resolveRepositoryRelative(
-                BuildType.LABEL.cast(actualValue));
-            builder.put(attribute, LabelAndConfiguration.of(label, actualConfig));
-          } else if (attribute.getType() == BuildType.LABEL_LIST) {
-            for (Label label : BuildType.LABEL_LIST.cast(actualValue)) {
-              builder.put(attribute, LabelAndConfiguration.of(
-                  rule.getLabel().resolveRepositoryRelative(label),
-                  actualConfig));
-            }
-          } else {
-            throw new IllegalStateException(
-                String.format(
-                    "Late bound attribute '%s' is not a label or a label list",
-                    attribute.getName()));
-          }
-        } catch (ClassCastException e) { // From either of the cast calls above.
-          throw new EvalException(
-              rule.getLocation(),
-              String.format(
-                  "When computing the default value of %s, expected '%s', got '%s'",
-                  attribute.getName(),
-                  attribute.getType(),
-                  EvalUtils.getDataTypeName(actualValue, true)));
+      } else {
+        // Late-bound attribute without a split transition:
+        for (Label dep : resolveLateBoundAttribute(depResolver.rule, attribute,
+            lateBoundDefault.useHostConfiguration() ? hostConfig : ruleConfig, attributeMap)) {
+          // Process this dep like a normal attribute.
+          depResolver.resolveDep(attribute, dep);
         }
       }
     }
   }
 
   /**
+   * Returns the label dependencies for the given late-bound attribute in this rule.
+   *
+   * @param rule the rule being evaluated
+   * @param attribute the attribute to evaluate
+   * @param config the configuration to evaluate the attribute in
+   * @param attributeMap mapper to attribute values
+   */
+  private Iterable<Label> resolveLateBoundAttribute(
+      Rule rule,
+      Attribute attribute,
+      BuildConfiguration config,
+      AttributeMap attributeMap)
+      throws EvalException, InterruptedException {
+    Preconditions.checkArgument(attribute.isLateBound());
+
+    @SuppressWarnings("unchecked")
+    LateBoundDefault<BuildConfiguration> lateBoundDefault =
+      (LateBoundDefault<BuildConfiguration>) attribute.getLateBoundDefault();
+
+    // TODO(bazel-team): This might be too expensive - can we cache this somehow?
+    if (!lateBoundDefault.getRequiredConfigurationFragments().isEmpty()) {
+      if (!config.hasAllFragments(lateBoundDefault.getRequiredConfigurationFragments())) {
+        return ImmutableList.<Label>of();
+      }
+    }
+
+    // TODO(bazel-team): We should check if the implementation tries to access an undeclared
+    // fragment.
+    Object actualValue = lateBoundDefault.resolve(rule, attributeMap, config);
+    if (EvalUtils.isNullOrNone(actualValue)) {
+      return ImmutableList.<Label>of();
+    }
+    try {
+      ImmutableList.Builder<Label> deps = ImmutableList.builder();
+      if (attribute.getType() == BuildType.LABEL) {
+        deps.add(rule.getLabel().resolveRepositoryRelative(BuildType.LABEL.cast(actualValue)));
+      } else if (attribute.getType() == BuildType.LABEL_LIST) {
+        for (Label label : BuildType.LABEL_LIST.cast(actualValue)) {
+          deps.add(rule.getLabel().resolveRepositoryRelative(label));
+        }
+      } else {
+        throw new IllegalStateException(
+            String.format(
+                "Late bound attribute '%s' is not a label or a label list",
+                attribute.getName()));
+      }
+      return deps.build();
+    } catch (ClassCastException e) { // From either of the cast calls above.
+      throw new EvalException(
+          rule.getLocation(),
+          String.format(
+              "When computing the default value of %s, expected '%s', got '%s'",
+              attribute.getName(),
+              attribute.getType(),
+              EvalUtils.getDataTypeName(actualValue, true)));
+    }
+  }
+
+  /**
+   * Adds new dependencies to the given rule under the given attribute name
+   *
+   * @param depResolver the resolver for this rule's deps
+   * @param attrName the name of the attribute to add dependency labels to
+   * @param labels the dependencies to add
+   */
+  private void addExplicitDeps(RuleResolver depResolver, String attrName, Iterable<Label> labels) {
+    Rule rule = depResolver.rule;
+    if (!rule.isAttrDefined(attrName, BuildType.LABEL_LIST)
+        && !rule.isAttrDefined(attrName, BuildType.NODEP_LABEL_LIST)) {
+      return;
+    }
+    Attribute attribute = rule.getRuleClassObject().getAttributeByName(attrName);
+    for (Label label : labels) {
+      depResolver.resolveDep(attribute, label);
+    }
+  }
+
+  /**
    * Converts the given multimap of attributes to labels into a multi map of attributes to
    * {@link Dependency} objects using the proper configuration transition for each attribute.
    *
    * @throws IllegalArgumentException if the {@code node} does not refer to a {@link Rule} instance
    */
-  public final Collection<Dependency> resolveRuleLabels(
-      TargetAndConfiguration node, ListMultimap<Attribute,
-      LabelAndConfiguration> labelMap, NestedSetBuilder<Label> rootCauses) {
+  public final Collection<Dependency> resolveRuleLabels(TargetAndConfiguration node,
+      ListMultimap<Attribute, Label> depLabels, NestedSetBuilder<Label> rootCauses) {
     Preconditions.checkArgument(node.getTarget() instanceof Rule);
     Rule rule = (Rule) node.getTarget();
     ListMultimap<Attribute, Dependency> outgoingEdges = ArrayListMultimap.create();
-    visitRule(rule, labelMap, rootCauses, outgoingEdges);
+    RuleResolver depResolver = new RuleResolver(rule, node.getConfiguration(), /*aspect=*/null,
+        /*attributeMap=*/null, rootCauses, outgoingEdges);
+    for (Map.Entry<Attribute, Collection<Label>> entry : depLabels.asMap().entrySet()) {
+      for (Label depLabel : entry.getValue()) {
+        depResolver.resolveDep(entry.getKey(), depLabel);
+      }
+    }
     return outgoingEdges.values();
   }
 
@@ -409,7 +478,7 @@
   }
 
   private ImmutableSet<AspectDescriptor> requiredAspects(
-      Aspect aspect, Attribute attribute, final Target target, Rule originalRule) {
+      @Nullable Aspect aspect, Attribute attribute, final Target target, Rule originalRule) {
     if (!(target instanceof Rule)) {
       return ImmutableSet.of();
     }
@@ -432,7 +501,7 @@
   }
 
   private static Iterable<Aspect> extractAspectCandidates(
-      Aspect aspect, Attribute attribute, Rule originalRule) {
+      @Nullable Aspect aspect, Attribute attribute, Rule originalRule) {
     // The order of this set will be deterministic. This is necessary because this order eventually
     // influences the order in which aspects are merged into the main configured target, which in
     // turn influences which aspect takes precedence if two emit the same provider (maybe this
@@ -458,59 +527,102 @@
     return aspectCandidates;
   }
 
-  private void visitRule(Rule rule,
-      ListMultimap<Attribute, LabelAndConfiguration> labelMap, NestedSetBuilder<Label> rootCauses,
-      ListMultimap<Attribute, Dependency> outgoingEdges) {
-    visitRule(rule, /*aspect=*/ null, labelMap, rootCauses, outgoingEdges);
-  }
+  /**
+   * Supplies the logic for translating <Attribute, Label> pairs for a rule into the
+   * <Attribute, Dependency> pairs DependencyResolver ultimately returns.
+   *
+   * <p>The main difference between the two is that the latter applies configuration transitions,
+   * i.e. it specifies not just which deps a rule has but also the configurations those deps
+   * should take.
+   */
+  private class RuleResolver {
+    private final Rule rule;
+    private final BuildConfiguration ruleConfig;
+    private final Aspect aspect;
+    private final ConfiguredAttributeMapper attributeMap;
+    private final NestedSetBuilder<Label> rootCauses;
+    private final ListMultimap<Attribute, Dependency> outgoingEdges;
+    private final List<Attribute> attributes;
 
-  private void visitRule(
-      Rule rule,
-      Aspect aspect,
-      ListMultimap<Attribute, LabelAndConfiguration> labelMap,
-      NestedSetBuilder<Label> rootCauses,
-      ListMultimap<Attribute, Dependency> outgoingEdges) {
-    Preconditions.checkNotNull(labelMap);
-    for (Map.Entry<Attribute, Collection<LabelAndConfiguration>> entry :
-        labelMap.asMap().entrySet()) {
-      Attribute attribute = entry.getKey();
-      for (LabelAndConfiguration dep : entry.getValue()) {
-        Label label = dep.getLabel();
-        BuildConfiguration config = dep.getConfiguration();
+    /**
+     * Constructs a new dependency resolver for the specified rule context.
+     *
+     * @param rule the rule being evaluated
+     * @param ruleConfig the rule's configuration
+     * @param aspect the aspect applied to this rule (if any)
+     * @param attributeMap mapper for the rule's attribute values
+     * @param rootCauses output collector for dep labels that can't be (loading phase) loaded
+     * @param outgoingEdges output collector for the resolved dependencies
+     */
+    RuleResolver(Rule rule, BuildConfiguration ruleConfig, Aspect aspect,
+        ConfiguredAttributeMapper attributeMap, NestedSetBuilder<Label> rootCauses,
+        ListMultimap<Attribute, Dependency> outgoingEdges) {
+      this.rule = rule;
+      this.ruleConfig = ruleConfig;
+      this.aspect = aspect;
+      this.attributeMap = attributeMap;
+      this.rootCauses = rootCauses;
+      this.outgoingEdges = outgoingEdges;
+      this.attributes = getAttributes(rule, aspect);
+    }
 
-        Target toTarget = getTarget(rule, label, rootCauses);
-        if (toTarget == null) {
-          continue;
-        }
-        BuildConfiguration.TransitionApplier transitionApplier = config.getTransitionApplier();
-        if (config.useDynamicConfigurations() && config.isHostConfiguration()
-            && !BuildConfiguration.usesNullConfiguration(toTarget)) {
-          // This condition is needed because resolveLateBoundAttributes may switch config to
-          // the host configuration, which is the only case DependencyResolver applies a
-          // configuration transition outside of this method. We need to reflect that
-          // transition in the results of this method, but config.evaluateTransition is hard-set
-          // to return a NONE transition when the input is a host config. Since the outside
-          // caller originally passed the *original* value of config (before the possible
-          // switch), it can mistakenly interpret the result as a NONE transition from the
-          // original value of config. This condition fixes that. Another fix would be to have
-          // config.evaluateTransition return a HOST transition when the input config is a host,
-          // but since this blemish is specific to DependencyResolver it seems best to keep the
-          // fix here.
-          // TODO(bazel-team): eliminate this special case by passing transitionApplier to
-          // resolveLateBoundAttributes, so that method uses the same interface for transitions.
-          transitionApplier.applyTransition(Attribute.ConfigurationTransition.HOST);
-        } else {
-          config.evaluateTransition(rule, attribute, toTarget, transitionApplier);
-        }
-        for (Dependency dependency :
-            transitionApplier.getDependencies(
-                label, requiredAspects(aspect, attribute, toTarget, rule))) {
-          outgoingEdges.put(
-              entry.getKey(),
-              dependency);
-        }
+    /**
+     * Returns the attributes that should be visited for this rule/aspect combination.
+     */
+    private List<Attribute> getAttributes(Rule rule, @Nullable Aspect aspect) {
+      List<Attribute> ruleDefs = rule.getRuleClassObject().getAttributes();
+      if (aspect == null) {
+        return ruleDefs;
+      } else {
+        return ImmutableList.<Attribute>builder()
+            .addAll(ruleDefs)
+            .addAll(aspect.getDefinition().getAttributes().values())
+            .build();
       }
     }
+
+    /**
+     * Resolves the given dep for the given attribute, including determining which
+     * configurations to apply to it.
+     */
+    void resolveDep(Attribute attribute, Label depLabel) {
+      // Late-bound split attributes are separately handled (see LateBoundSplitResolver).
+      Verify.verify(!(attribute.isLateBound() && attribute.hasSplitConfigurationTransition()));
+      Target toTarget = getTarget(rule, depLabel, rootCauses);
+      if (toTarget == null) {
+        return; // Skip this round: we still need to Skyframe-evaluate the dep's target.
+      }
+      BuildConfiguration.TransitionApplier resolver = ruleConfig.getTransitionApplier();
+      ruleConfig.evaluateTransition(rule, attribute, toTarget, resolver);
+      // An <Attribute, Label> pair can resolve to multiple deps because of split transitions.
+      for (Dependency dependency :
+          resolver.getDependencies(depLabel, requiredAspects(aspect, attribute, toTarget, rule))) {
+        outgoingEdges.put(attribute, dependency);
+      }
+    }
+
+    /**
+     * Resolves the given dep for the given attribute using a pre-prepared configuration.
+     *
+     * <p>Use this method with care: it skips Bazel's standard config transition semantics
+     * ({@link BuildConfiguration#evaluateTransition}). That means attributes passed through here
+     * won't obey standard rules on which configurations apply to their deps. This should only
+     * be done for special circumstances that really justify the difference. When in doubt, use
+     * {@link #resolveDep(Attribute, Label)}.
+     */
+    void resolveDep(Attribute attribute, Label depLabel, BuildConfiguration config) {
+      Target toTarget = getTarget(rule, depLabel, rootCauses);
+      if (toTarget == null) {
+        return; // Skip this round: this is either a loading error or unevaluated Skyframe dep.
+      }
+      BuildConfiguration.TransitionApplier transitionApplier = config.getTransitionApplier();
+      if (BuildConfiguration.usesNullConfiguration(toTarget)) {
+        transitionApplier.applyTransition(Attribute.ConfigurationTransition.NULL);
+      }
+      Dependency dep = Iterables.getOnlyElement(transitionApplier.getDependencies(depLabel,
+          requiredAspects(aspect, attribute, toTarget, rule)));
+      outgoingEdges.put(attribute, dep);
+    }
   }
 
   private void visitTargetVisibility(TargetAndConfiguration node,
diff --git a/src/main/java/com/google/devtools/build/lib/packages/Attribute.java b/src/main/java/com/google/devtools/build/lib/packages/Attribute.java
index 5277aef..2ac4851 100644
--- a/src/main/java/com/google/devtools/build/lib/packages/Attribute.java
+++ b/src/main/java/com/google/devtools/build/lib/packages/Attribute.java
@@ -1090,8 +1090,12 @@
    */
   public interface LateBoundDefault<T> {
     /**
-     * Whether to look up the label in the host configuration. This is only here for the host JDK -
-     * we usually need to look up labels in the target configuration.
+     * Whether to look up the label in the host configuration. This is only here for host
+     * compilation tools - we usually need to look up labels in the target configuration.
+     *
+     * <p>This method only sets the configuration passed to {@link #resolve}. If you want the
+     * dependency to also be analyzed in the host configuration, use
+     * {@link ConfigurationTransition#HOST}.
      */
     boolean useHostConfiguration();
 
diff --git a/src/main/java/com/google/devtools/build/lib/skyframe/ConfiguredTargetFunction.java b/src/main/java/com/google/devtools/build/lib/skyframe/ConfiguredTargetFunction.java
index b0a8137..18fbe8c 100644
--- a/src/main/java/com/google/devtools/build/lib/skyframe/ConfiguredTargetFunction.java
+++ b/src/main/java/com/google/devtools/build/lib/skyframe/ConfiguredTargetFunction.java
@@ -710,14 +710,12 @@
     Map<Label, ConfigMatchingProvider> configConditions = new LinkedHashMap<>();
 
     // Collect the labels of the configured targets we need to resolve.
-    ListMultimap<Attribute, LabelAndConfiguration> configLabelMap = ArrayListMultimap.create();
+    ListMultimap<Attribute, Label> configLabelMap = ArrayListMultimap.create();
     RawAttributeMapper attributeMap = RawAttributeMapper.of(((Rule) target));
     for (Attribute a : ((Rule) target).getAttributes()) {
       for (Label configLabel : attributeMap.getConfigurabilityKeys(a.getName(), a.getType())) {
         if (!BuildType.Selector.isReservedLabel(configLabel)) {
-          configLabelMap.put(a, LabelAndConfiguration.of(
-              target.getLabel().resolveRepositoryRelative(configLabel),
-              ctgValue.getConfiguration()));
+          configLabelMap.put(a, target.getLabel().resolveRepositoryRelative(configLabel));
         }
       }
     }