Move ConfiguredTargetFunction dynamic config selection to its own class.

Also clarify the interfaces *TransitionResolver* - which determines what
transition to apply to an input configuration and *ConfigurationResolver*
 - which determines the output configuration from that transition.

PiperOrigin-RevId: 169311986
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 715ecf2..c157351 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
@@ -13,20 +13,12 @@
 // limitations under the License.
 package com.google.devtools.build.lib.skyframe;
 
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Joiner;
 import com.google.common.base.Supplier;
-import com.google.common.base.Verify;
-import com.google.common.base.VerifyException;
 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.LinkedHashMultimap;
-import com.google.common.collect.LinkedListMultimap;
 import com.google.common.collect.Maps;
-import com.google.common.collect.Multimap;
-import com.google.common.collect.Sets;
 import com.google.devtools.build.lib.actions.Actions;
 import com.google.devtools.build.lib.actions.Actions.GeneratingActions;
 import com.google.devtools.build.lib.actions.MutableActionGraph.ActionConflictException;
@@ -42,18 +34,15 @@
 import com.google.devtools.build.lib.analysis.TargetAndConfiguration;
 import com.google.devtools.build.lib.analysis.ToolchainContext;
 import com.google.devtools.build.lib.analysis.config.BuildConfiguration;
-import com.google.devtools.build.lib.analysis.config.BuildOptions;
 import com.google.devtools.build.lib.analysis.config.ConfigMatchingProvider;
-import com.google.devtools.build.lib.analysis.config.HostTransition;
+import com.google.devtools.build.lib.analysis.config.ConfigurationResolver;
 import com.google.devtools.build.lib.analysis.config.InvalidConfigurationException;
-import com.google.devtools.build.lib.analysis.config.PatchTransition;
 import com.google.devtools.build.lib.analysis.configuredtargets.MergedConfiguredTarget;
 import com.google.devtools.build.lib.analysis.configuredtargets.MergedConfiguredTarget.DuplicateException;
 import com.google.devtools.build.lib.cmdline.Label;
 import com.google.devtools.build.lib.collect.nestedset.NestedSet;
 import com.google.devtools.build.lib.collect.nestedset.NestedSetBuilder;
 import com.google.devtools.build.lib.collect.nestedset.Order;
-import com.google.devtools.build.lib.concurrent.ThreadSafety.Immutable;
 import com.google.devtools.build.lib.events.Event;
 import com.google.devtools.build.lib.events.StoredEventHandler;
 import com.google.devtools.build.lib.packages.Aspect;
@@ -81,17 +70,12 @@
 import com.google.devtools.build.skyframe.SkyValue;
 import com.google.devtools.build.skyframe.ValueOrException;
 import com.google.devtools.build.skyframe.ValueOrException2;
-import java.util.ArrayList;
+
 import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
 import java.util.HashMap;
 import java.util.HashSet;
-import java.util.Iterator;
 import java.util.LinkedHashMap;
-import java.util.List;
 import java.util.Map;
-import java.util.Objects;
 import java.util.Set;
 import java.util.concurrent.Semaphore;
 import javax.annotation.Nullable;
@@ -401,8 +385,8 @@
     // Trim each dep's configuration so it only includes the fragments needed by its transitive
     // closure.
     if (ctgValue.getConfiguration() != null) {
-      depValueNames = getDynamicConfigurations(env, ctgValue, depValueNames, hostConfiguration,
-          ruleClassProvider);
+      depValueNames = ConfigurationResolver.resolveConfigurations(env, ctgValue, depValueNames,
+          hostConfiguration, ruleClassProvider);
       // It's important that we don't use "if (env.missingValues()) { return null }" here (or
       // in the following lines). See the comments in getDynamicConfigurations' Skyframe call
       // for explanation.
@@ -438,454 +422,6 @@
   }
 
   /**
-   * Helper class for {@link #getDynamicConfigurations} - encapsulates a set of config fragments and
-   * a dynamic transition. This can be used to determine the exact build options needed to
-   * set a dynamic configuration.
-   */
-  @Immutable
-  private static final class FragmentsAndTransition {
-    // Treat this as immutable. The only reason this isn't an ImmutableSet is because it
-    // gets bound to a NestedSet.toSet() reference, which returns a Set interface.
-    final Set<Class<? extends BuildConfiguration.Fragment>> fragments;
-    final Attribute.Transition transition;
-    private final int hashCode;
-
-    FragmentsAndTransition(Set<Class<? extends BuildConfiguration.Fragment>> fragments,
-        Attribute.Transition transition) {
-      this.fragments = fragments;
-      this.transition = transition;
-      hashCode = Objects.hash(this.fragments, this.transition);
-    }
-
-    @Override
-    public boolean equals(Object o) {
-      if (o == this) {
-        return true;
-      } else if (o == null) {
-        return false;
-      } else {
-        FragmentsAndTransition other = (FragmentsAndTransition) o;
-        return other.transition.equals(transition) && other.fragments.equals(fragments);
-      }
-    }
-
-    @Override
-    public int hashCode() {
-      return hashCode;
-    }
-  }
-
-  /**
-   * Helper class for {@link #getDynamicConfigurations} - encapsulates an <attribute, label> pair
-   * that can be used to map from an input dependency to a trimmed dependency.
-   */
-  @Immutable
-  private static final class AttributeAndLabel {
-    final Attribute attribute;
-    final Label label;
-    Integer hashCode;
-
-    AttributeAndLabel(Attribute attribute, Label label) {
-      this.attribute = attribute;
-      this.label = label;
-    }
-
-    @Override
-    public boolean equals(Object o) {
-      if (!(o instanceof AttributeAndLabel)) {
-        return false;
-      }
-      AttributeAndLabel other = (AttributeAndLabel) o;
-      return Objects.equals(other.attribute, attribute) && other.label.equals(label);
-    }
-
-    @Override
-    public int hashCode() {
-      if (hashCode == null) {
-        // Not every <Attribute, Label> pair gets hashed. So only evaluate for the instances that
-        // need it. This can significantly reduce the number of evaluations.
-        hashCode = Objects.hash(this.attribute, this.label);
-      }
-      return hashCode;
-    }
-  }
-
-  /**
-   * Variation of {@link Multimap#put} that triggers an exception if a value already exists.
-   */
-  @VisibleForTesting
-  static <K, V> void putOnlyEntry(Multimap<K, V> map, K key, V value) {
-    // Performance note: while "Verify.verify(!map.containsKey(key, value), String.format(...)))"
-    // is simpler code, profiling shows a substantial performance penalty to that approach
-    // (~10% extra analysis phase time on a simple cc_binary). Most of that is from the cost of
-    // evaluating value.toString() on every call. This approach essentially eliminates the overhead.
-    if (map.containsKey(key)) {
-      throw new VerifyException(
-          String.format("couldn't insert %s: map already has key %s",
-              value.toString(), key.toString()));
-    }
-    map.put(key, value);
-  }
-
-  /**
-   * Creates a dynamic configuration for each dep that's custom-fitted specifically for that dep.
-   *
-   * <p>More specifically: given a set of {@link Dependency} instances holding dynamic config
-   * transition requests (e.g. {@link Dependency#hasExplicitConfiguration()} == false}), returns
-   * equivalent dependencies containing dynamically created configurations applying those
-   * transitions. If {@link BuildConfiguration.Options#trimConfigurations()} is true, these
-   * configurations only contain the fragments needed by the dep and its transitive closure. Else
-   * the configurations unconditionally include all fragments.
-   *
-   * <p>This method is heavily performance-optimized. Because it, in aggregate, reads over every
-   * edge in the configured target graph, small inefficiencies can have observable impact on
-   * analysis time. Keep this in mind when making modifications and performance-test any changes you
-   * make.
-   *
-   * @param env Skyframe evaluation environment
-   * @param ctgValue the label and the configuration of the node
-   * @param originalDeps the set of configuration transition requests for this target's attributes
-   * @param hostConfiguration the host configuration
-   * @param ruleClassProvider the rule class provider for determining the right configuration
-   *    fragments to apply to deps
-   *
-   * @return a mapping from each attribute to the {@link BuildConfiguration}s and {@link Label}s
-   *    to use for that attribute's deps. Returns null if not all Skyframe dependencies are
-   *    available yet.
-   */
-  @Nullable
-  static OrderedSetMultimap<Attribute, Dependency> getDynamicConfigurations(
-      Environment env,
-      TargetAndConfiguration ctgValue,
-      OrderedSetMultimap<Attribute, Dependency> originalDeps,
-      BuildConfiguration hostConfiguration,
-      RuleClassProvider ruleClassProvider)
-      throws DependencyEvaluationException, InterruptedException {
-
-    // Maps each Skyframe-evaluated BuildConfiguration to the dependencies that need that
-    // configuration. For cases where Skyframe isn't needed to get the configuration (e.g. when
-    // we just re-used the original rule's configuration), we should skip this outright.
-    Multimap<SkyKey, Map.Entry<Attribute, Dependency>> keysToEntries = LinkedListMultimap.create();
-
-    // Stores the result of applying a dynamic transition to the current configuration using a
-    // particular subset of fragments. By caching this, we save from redundantly computing the
-    // same transition for every dependency edge that requests that transition. This can have
-    // real effect on analysis time for commonly triggered transitions.
-    //
-    // Split transitions may map to multiple values. All other transitions map to one.
-    Map<FragmentsAndTransition, List<BuildOptions>> transitionsMap = new LinkedHashMap<>();
-
-    // The fragments used by the current target's configuration.
-    Set<Class<? extends BuildConfiguration.Fragment>> ctgFragments =
-        ctgValue.getConfiguration().fragmentClasses();
-    BuildOptions ctgOptions = ctgValue.getConfiguration().getOptions();
-
-    // Stores the dynamically configured versions of each dependency. This method must preserve the
-    // original label ordering of each attribute. For example, if originalDeps.get("data") is
-    // [":a", ":b"], the dynamic variant must also be [":a", ":b"] in the same order. Because we may
-    // not actualize the results in order (some results need Skyframe-evaluated configurations while
-    // others can be computed trivially), we dump them all into this map, then as a final step
-    // iterate through the original list and pluck out values from here for the final value.
-    //
-    // For split transitions, originaldeps.get("data") = [":a", ":b"] can produce the output
-    // [":a"<config1>, ":a"<config2>, ..., ":b"<config1>, ":b"<config2>, ...]. All instances of ":a"
-    // still appear before all instances of ":b". But the [":a"<config1>, ":a"<config2>"] subset may
-    // be in any (deterministic) order. In particular, this may not be the same order as
-    // SplitTransition.split. If needed, this code can be modified to use that order, but that
-    // involves more runtime in performance-critical code, so we won't make that change without a
-    // clear need.
-    //
-    // This map is used heavily by all builds. Inserts and gets should be as fast as possible.
-    Multimap<AttributeAndLabel, Dependency> dynamicDeps = LinkedHashMultimap.create();
-
-    // Performance optimization: This method iterates over originalDeps twice. By storing
-    // AttributeAndLabel instances in this list, we avoid having to recreate them the second time
-    // (particularly avoid recomputing their hash codes). Profiling shows this shaves 25% off this
-    // method's execution time (at the time of this comment).
-    ArrayList<AttributeAndLabel> attributesAndLabels = new ArrayList<>(originalDeps.size());
-
-    for (Map.Entry<Attribute, Dependency> depsEntry : originalDeps.entries()) {
-      Dependency dep = depsEntry.getValue();
-      AttributeAndLabel attributeAndLabel =
-          new AttributeAndLabel(depsEntry.getKey(), dep.getLabel());
-      attributesAndLabels.add(attributeAndLabel);
-      // Certain targets (like output files) trivially re-use their input configuration. Likewise,
-      // deps with null configurations (e.g. source files), can be trivially computed. So we skip
-      // all logic in this method for these cases and just reinsert their original configurations
-      // back at the end (note that null-configured targets will have a static
-      // NullConfigurationDependency instead of dynamic
-      // Dependency(label, transition=Attribute.Configuration.Transition.NULL)).
-      //
-      // A *lot* of targets have null deps, so this produces real savings. Profiling tests over a
-      // simple cc_binary show this saves ~1% of total analysis phase time.
-      if (dep.hasExplicitConfiguration()) {
-        continue;
-      }
-
-      // Figure out the required fragments for this dep and its transitive closure.
-      Set<Class<? extends BuildConfiguration.Fragment>> depFragments =
-          getTransitiveFragments(env, dep.getLabel(), ctgValue.getConfiguration());
-      if (depFragments == null) {
-        return null;
-      }
-      // TODO(gregce): remove the below call once we have confidence dynamic configurations always
-      // provide needed fragments. This unnecessarily drags performance on the critical path (up
-      // to 0.5% of total analysis time as profiled over a simple cc_binary).
-      if (ctgValue.getConfiguration().trimConfigurations()) {
-        checkForMissingFragments(env, ctgValue, attributeAndLabel.attribute.getName(), dep,
-            depFragments);
-      }
-
-      boolean sameFragments = depFragments.equals(ctgFragments);
-      Attribute.Transition transition = dep.getTransition();
-
-      if (sameFragments) {
-        if (transition == Attribute.ConfigurationTransition.NONE) {
-          // The dep uses the same exact configuration.
-          putOnlyEntry(
-              dynamicDeps,
-              attributeAndLabel,
-              Dependency.withConfigurationAndAspects(
-                  dep.getLabel(), ctgValue.getConfiguration(), dep.getAspects()));
-          continue;
-        } else if (transition == HostTransition.INSTANCE) {
-          // The current rule's host configuration can also be used for the dep. We short-circuit
-          // the standard transition logic for host transitions because these transitions are
-          // uniquely frequent. It's possible, e.g., for every node in the configured target graph
-          // to incur multiple host transitions. So we aggressively optimize to avoid hurting
-          // analysis time.
-          putOnlyEntry(
-              dynamicDeps,
-              attributeAndLabel,
-              Dependency.withConfigurationAndAspects(
-                  dep.getLabel(), hostConfiguration, dep.getAspects()));
-          continue;
-        }
-      }
-
-      // Apply the transition or use the cached result if it was already applied.
-      FragmentsAndTransition transitionKey = new FragmentsAndTransition(depFragments, transition);
-      List<BuildOptions> toOptions = transitionsMap.get(transitionKey);
-      if (toOptions == null) {
-        toOptions = getDynamicTransitionOptions(ctgOptions, transition, depFragments,
-            ruleClassProvider, !sameFragments);
-        transitionsMap.put(transitionKey, toOptions);
-      }
-
-      // If the transition doesn't change the configuration, trivially re-use the original
-      // configuration.
-      if (sameFragments && toOptions.size() == 1
-          && Iterables.getOnlyElement(toOptions).equals(ctgOptions)) {
-        putOnlyEntry(
-            dynamicDeps,
-            attributeAndLabel,
-            Dependency.withConfigurationAndAspects(
-                dep.getLabel(), ctgValue.getConfiguration(), dep.getAspects()));
-        continue;
-      }
-
-      // If we get here, we have to get the configuration from Skyframe.
-      for (BuildOptions options : toOptions) {
-        keysToEntries.put(BuildConfigurationValue.key(depFragments, options), depsEntry);
-      }
-    }
-
-    // Get all BuildConfigurations we need from Skyframe. While not every value might be available,
-    // we don't call env.valuesMissing() here because that could be true from the earlier
-    // resolver.dependentNodeMap call in computeDependencies, which also calls Skyframe. This method
-    // doesn't need those missing values, but it still has to be called after
-    // resolver.dependentNodeMap because it consumes that method's output. The reason the missing
-    // values don't matter is because resolver.dependentNodeMap still returns "partial" results
-    // and this method runs over whatever's available.
-    //
-    // While there would be no *correctness* harm in nulling out early, there's significant
-    // *performance* harm. Profiling shows that putting "if (env.valuesMissing()) { return null; }"
-    // here (or even after resolver.dependentNodeMap) produces a ~30% performance hit on the
-    // analysis phase. That's because resolveConfiguredTargetDependencies and
-    // resolveAspectDependencies don't get a chance to make their own Skyframe requests before
-    // bailing out of this ConfiguredTargetFunction call. Ideally we could batch all requests
-    // from all methods into a single Skyframe call, but there are enough subtle data flow
-    // dependencies in ConfiguredTargetFucntion to make that impractical.
-    Map<SkyKey, ValueOrException<InvalidConfigurationException>> depConfigValues =
-        env.getValuesOrThrow(keysToEntries.keySet(), InvalidConfigurationException.class);
-
-    // Now fill in the remaining unresolved deps with the now-resolved configurations.
-    try {
-      for (Map.Entry<SkyKey, ValueOrException<InvalidConfigurationException>> entry :
-          depConfigValues.entrySet()) {
-        SkyKey key = entry.getKey();
-        ValueOrException<InvalidConfigurationException> valueOrException = entry.getValue();
-        if (valueOrException.get() == null) {
-          // Instead of env.missingValues(), check for missing values here. This guarantees we only
-          // null out on missing values from *this specific Skyframe request*.
-          return null;
-        }
-        BuildConfigurationValue trimmedConfig = (BuildConfigurationValue) valueOrException.get();
-        for (Map.Entry<Attribute, Dependency> info : keysToEntries.get(key)) {
-          Dependency originalDep = info.getValue();
-          AttributeAndLabel attr = new AttributeAndLabel(info.getKey(), originalDep.getLabel());
-          Dependency resolvedDep = Dependency.withConfigurationAndAspects(originalDep.getLabel(),
-              trimmedConfig.getConfiguration(), originalDep.getAspects());
-          if (attr.attribute.hasSplitConfigurationTransition()) {
-            dynamicDeps.put(attr, resolvedDep);
-          } else {
-            putOnlyEntry(dynamicDeps, attr, resolvedDep);
-          }
-        }
-      }
-    } catch (InvalidConfigurationException e) {
-      throw new DependencyEvaluationException(e);
-    }
-
-    return sortDynamicallyConfiguredDeps(originalDeps, dynamicDeps, attributesAndLabels);
-  }
-
-  /**
-   * Returns the configuration fragments required by a dep and its transitive closure.
-   * Returns null if Skyframe dependencies aren't yet available.
-   *
-   * @param env Skyframe evaluation environment
-   * @param dep label of the dep to check
-   * @param parentConfig configuration of the rule depending on the dep
-   */
-  @Nullable
-  private static Set<Class<? extends BuildConfiguration.Fragment>> getTransitiveFragments(
-      Environment env, Label dep, BuildConfiguration parentConfig) throws InterruptedException {
-    if (!parentConfig.trimConfigurations()) {
-      return parentConfig.getAllFragments().keySet();
-    }
-    SkyKey fragmentsKey = TransitiveTargetValue.key(dep);
-    TransitiveTargetValue transitiveDepInfo = (TransitiveTargetValue) env.getValue(fragmentsKey);
-    if (transitiveDepInfo == null) {
-      // This should only be possible for tests. In actual runs, this was already called
-      // as a routine part of the loading phase.
-      // TODO(bazel-team): check this only occurs in a test context.
-      return null;
-    }
-    return transitiveDepInfo.getTransitiveConfigFragments().toSet();
-  }
-
-  /**
-   * Applies a dynamic configuration transition over a set of build options.
-   *
-   * @return the build options for the transitioned configuration. If trimResults is true,
-   *     only options needed by the required fragments are included. Else the same options as the
-   *     original input are included (with different possible values, of course).
-   */
-  static List<BuildOptions> getDynamicTransitionOptions(BuildOptions fromOptions,
-      Attribute.Transition transition,
-      Iterable<Class<? extends BuildConfiguration.Fragment>> requiredFragments,
-      RuleClassProvider ruleClassProvider, boolean trimResults) {
-    List<BuildOptions> result;
-    if (transition == Attribute.ConfigurationTransition.NONE) {
-      result = ImmutableList.<BuildOptions>of(fromOptions);
-    } else if (transition instanceof PatchTransition) {
-      // TODO(bazel-team): safety-check that this never mutates fromOptions.
-      result = ImmutableList.<BuildOptions>of(((PatchTransition) transition).apply(fromOptions));
-    } else if (transition instanceof Attribute.SplitTransition) {
-      @SuppressWarnings("unchecked") // Attribute.java doesn't have the BuildOptions symbol.
-      List<BuildOptions> toOptions =
-          ((Attribute.SplitTransition<BuildOptions>) transition).split(fromOptions);
-      if (toOptions.isEmpty()) {
-        // When the split returns an empty list, it's signaling it doesn't apply to this instance.
-        // So return the original options.
-        result = ImmutableList.<BuildOptions>of(fromOptions);
-      } else {
-        result = toOptions;
-      }
-    } else {
-      throw new IllegalStateException(String.format(
-          "unsupported dynamic transition type: %s", transition.getClass().getName()));
-    }
-
-    if (!trimResults) {
-      return result;
-    } else {
-      ImmutableList.Builder<BuildOptions> trimmedOptions = ImmutableList.builder();
-      for (BuildOptions toOptions : result) {
-        trimmedOptions.add(toOptions.trim(
-            BuildConfiguration.getOptionsClasses(requiredFragments, ruleClassProvider)));
-      }
-      return trimmedOptions.build();
-    }
-  }
-
-  /**
-   * Diagnostic helper method for dynamic configurations: checks the config fragments required by
-   * a dep against the fragments in its actual configuration. If any are missing, triggers a
-   * descriptive "missing fragments" error.
-   */
-  private static void checkForMissingFragments(Environment env, TargetAndConfiguration ctgValue,
-      String attribute, Dependency dep,
-      Set<Class<? extends BuildConfiguration.Fragment>> expectedDepFragments)
-      throws DependencyEvaluationException {
-    Set<String> ctgFragmentNames = new HashSet<>();
-    for (BuildConfiguration.Fragment fragment :
-        ctgValue.getConfiguration().getAllFragments().values()) {
-      ctgFragmentNames.add(fragment.getClass().getSimpleName());
-    }
-    Set<String> depFragmentNames = new HashSet<>();
-    for (Class<? extends BuildConfiguration.Fragment> fragmentClass : expectedDepFragments) {
-     depFragmentNames.add(fragmentClass.getSimpleName());
-    }
-    Set<String> missing = Sets.difference(depFragmentNames, ctgFragmentNames);
-    if (!missing.isEmpty()) {
-      String msg = String.format(
-          "%s: dependency %s from attribute \"%s\" is missing required config fragments: %s",
-          ctgValue.getLabel(), dep.getLabel(), attribute, Joiner.on(", ").join(missing));
-      env.getListener().handle(Event.error(msg));
-      throw new DependencyEvaluationException(new InvalidConfigurationException(msg));
-    }
-  }
-
-  /**
-   * Determines the output ordering of each <attribute, depLabel> ->
-   * [dep<config1>, dep<config2>, ...] collection produced by a split transition.
-   */
-  @VisibleForTesting
-  static final Comparator<Dependency> DYNAMIC_SPLIT_DEP_ORDERING =
-      new Comparator<Dependency>() {
-        @Override
-        public int compare(Dependency d1, Dependency d2) {
-          return d1.getConfiguration().getMnemonic().compareTo(d2.getConfiguration().getMnemonic());
-        }
-      };
-
-  /**
-   * Helper method for {@link #getDynamicConfigurations}: returns a copy of the output deps
-   * using the same key and value ordering as the input deps.
-   *
-   * @param originalDeps the input deps with the ordering to preserve
-   * @param dynamicDeps the unordered output deps
-   * @param attributesAndLabels collection of <attribute, depLabel> pairs guaranteed to match
-   *   the ordering of originalDeps.entries(). This is a performance optimization: see
-   *   {@link #getDynamicConfigurations#attributesAndLabels} for details.
-   */
-  private static OrderedSetMultimap<Attribute, Dependency> sortDynamicallyConfiguredDeps(
-      OrderedSetMultimap<Attribute, Dependency> originalDeps,
-      Multimap<AttributeAndLabel, Dependency> dynamicDeps,
-      ArrayList<AttributeAndLabel> attributesAndLabels) {
-    Iterator<AttributeAndLabel> iterator = attributesAndLabels.iterator();
-    OrderedSetMultimap<Attribute, Dependency> result = OrderedSetMultimap.create();
-    for (Map.Entry<Attribute, Dependency> depsEntry : originalDeps.entries()) {
-      AttributeAndLabel attrAndLabel = iterator.next();
-      if (depsEntry.getValue().hasExplicitConfiguration()) {
-        result.put(attrAndLabel.attribute, depsEntry.getValue());
-      } else {
-        Collection<Dependency> dynamicAttrDeps = dynamicDeps.get(attrAndLabel);
-        Verify.verify(!dynamicAttrDeps.isEmpty());
-        if (dynamicAttrDeps.size() > 1) {
-          List<Dependency> sortedSplitList = new ArrayList<>(dynamicAttrDeps);
-          Collections.sort(sortedSplitList, DYNAMIC_SPLIT_DEP_ORDERING);
-          dynamicAttrDeps = sortedSplitList;
-        }
-        result.putAll(depsEntry.getKey(), dynamicAttrDeps);
-      }
-    }
-    return result;
-  }
-
-  /**
    * Merges the each direct dependency configured target with the aspects associated with it.
    *
    * <p>Note that the combination of a configured target and its associated aspects are not