Bladeren bron

HADOOP-17115. Replace Guava Sets usage by Hadoop's own Sets in hadoop-common and hadoop-tools (#2985)

Signed-off-by: Sean Busbey <busbey@apache.org>
Viraj Jasani 4 jaren geleden
bovenliggende
commit
e4062ad027
21 gewijzigde bestanden met toevoegingen van 552 en 17 verwijderingen
  1. 32 0
      hadoop-common-project/hadoop-common/pom.xml
  2. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MutableRates.java
  3. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MutableRatesWithAggregation.java
  4. 0 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/JvmPauseMonitor.java
  5. 377 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Sets.java
  6. 1 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestValueQueue.java
  7. 1 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/kms/TestLoadBalancingKMSClientProvider.java
  8. 1 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java
  9. 32 0
      hadoop-common-project/hadoop-kms/pom.xml
  10. 1 1
      hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSAudit.java
  11. 32 0
      hadoop-tools/hadoop-aws/pom.xml
  12. 1 1
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/AssumedRoleCredentialProvider.java
  13. 2 2
      hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/Paths.java
  14. 1 1
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AAWSCredentialsProvider.java
  15. 1 1
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/integration/ITestS3ACommitterMRJob.java
  16. 1 1
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingCommitter.java
  17. 1 1
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingPartitionedTaskCommit.java
  18. 32 0
      hadoop-tools/hadoop-distcp/pom.xml
  19. 1 1
      hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/CopyListing.java
  20. 1 1
      hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/test/java/org/apache/hadoop/tools/dynamometer/TestDynamometerInfra.java
  21. 32 0
      hadoop-tools/hadoop-dynamometer/pom.xml

+ 32 - 0
hadoop-common-project/hadoop-common/pom.xml

@@ -634,6 +634,38 @@
           </filesets>
           </filesets>
         </configuration>
         </configuration>
       </plugin>
       </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-enforcer-plugin</artifactId>
+        <dependencies>
+          <dependency>
+            <groupId>de.skuzzle.enforcer</groupId>
+            <artifactId>restrict-imports-enforcer-rule</artifactId>
+            <version>${restrict-imports.enforcer.version}</version>
+          </dependency>
+        </dependencies>
+        <executions>
+          <execution>
+            <id>banned-illegal-imports</id>
+            <phase>process-sources</phase>
+            <goals>
+              <goal>enforce</goal>
+            </goals>
+            <configuration>
+              <rules>
+                <restrictImports implementation="de.skuzzle.enforcer.restrictimports.rule.RestrictImports">
+                  <includeTestCode>true</includeTestCode>
+                  <reason>Use hadoop-common provided Sets rather than Guava provided Sets</reason>
+                  <bannedImports>
+                    <bannedImport>org.apache.hadoop.thirdparty.com.google.common.collect.Sets</bannedImport>
+                    <bannedImport>com.google.common.collect.Sets</bannedImport>
+                  </bannedImports>
+                </restrictImports>
+              </rules>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
     </plugins>
     </plugins>
   </build>
   </build>
 
 

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MutableRates.java

@@ -22,7 +22,7 @@ import java.lang.reflect.Method;
 import java.util.Set;
 import java.util.Set;
 
 
 import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.*;
 import static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.*;
-import org.apache.hadoop.thirdparty.com.google.common.collect.Sets;
+import org.apache.hadoop.util.Sets;
 
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MutableRatesWithAggregation.java

@@ -18,7 +18,7 @@
 
 
 package org.apache.hadoop.metrics2.lib;
 package org.apache.hadoop.metrics2.lib;
 
 
-import org.apache.hadoop.thirdparty.com.google.common.collect.Sets;
+import org.apache.hadoop.util.Sets;
 import java.lang.ref.WeakReference;
 import java.lang.ref.WeakReference;
 import java.lang.reflect.Method;
 import java.lang.reflect.Method;
 import java.util.Iterator;
 import java.util.Iterator;

+ 0 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/JvmPauseMonitor.java

@@ -29,10 +29,8 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.service.AbstractService;
 
 
 import org.apache.hadoop.thirdparty.com.google.common.base.Joiner;
 import org.apache.hadoop.thirdparty.com.google.common.base.Joiner;
-import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
 import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
 import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
 import org.apache.hadoop.thirdparty.com.google.common.collect.Maps;
 import org.apache.hadoop.thirdparty.com.google.common.collect.Maps;
-import org.apache.hadoop.thirdparty.com.google.common.collect.Sets;
 import org.slf4j.Logger;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
 
 

+ 377 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Sets.java

@@ -0,0 +1,377 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.hadoop.util;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Static utility methods pertaining to {@link Set} instances.
+ * This class is Hadoop's internal use alternative to Guava's Sets
+ * utility class.
+ * Javadocs for majority of APIs in this class are taken from Guava's Sets
+ * class from Guava release version 27.0-jre.
+ */
+@InterfaceAudience.Private
+public final class Sets {
+
+  private static final int MAX_POWER_OF_TWO = 1 << (Integer.SIZE - 2);
+
+  private Sets() {
+    // empty
+  }
+
+  /**
+   * Creates a <i>mutable</i>, initially empty {@code HashSet} instance.
+   *
+   * <p><b>Note:</b> if mutability is not required, use ImmutableSet#of()
+   * instead. If {@code E} is an {@link Enum} type, use {@link EnumSet#noneOf}
+   * instead. Otherwise, strongly consider using a {@code LinkedHashSet}
+   * instead, at the cost of increased memory footprint, to get
+   * deterministic iteration behavior.
+   */
+  public static <E> HashSet<E> newHashSet() {
+    return new HashSet<E>();
+  }
+
+  /**
+   * Creates a <i>mutable</i>, empty {@code TreeSet} instance sorted by the
+   * natural sort ordering of its elements.
+   *
+   * <p><b>Note:</b> if mutability is not required, use ImmutableSortedSet#of()
+   * instead.
+   *
+   * @return a new, empty {@code TreeSet}
+   */
+  public static <E extends Comparable> TreeSet<E> newTreeSet() {
+    return new TreeSet<E>();
+  }
+
+  /**
+   * Creates a <i>mutable</i> {@code HashSet} instance initially containing
+   * the given elements.
+   *
+   * <p><b>Note:</b> if elements are non-null and won't be added or removed
+   * after this point, use ImmutableSet#of() or ImmutableSet#copyOf(Object[])
+   * instead. If {@code E} is an {@link Enum} type, use
+   * {@link EnumSet#of(Enum, Enum[])} instead. Otherwise, strongly consider
+   * using a {@code LinkedHashSet} instead, at the cost of increased memory
+   * footprint, to get deterministic iteration behavior.
+   *
+   * <p>This method is just a small convenience, either for
+   * {@code newHashSet(}{@link Arrays#asList}{@code (...))}, or for creating an
+   * empty set then calling {@link Collections#addAll}.
+   */
+  @SafeVarargs
+  public static <E> HashSet<E> newHashSet(E... elements) {
+    HashSet<E> set = newHashSetWithExpectedSize(elements.length);
+    Collections.addAll(set, elements);
+    return set;
+  }
+
+  /**
+   * Creates a <i>mutable</i> {@code HashSet} instance containing the given
+   * elements. A very thin convenience for creating an empty set then calling
+   * {@link Collection#addAll} or Iterables#addAll.
+   *
+   * <p><b>Note:</b> if mutability is not required and the elements are
+   * non-null, use ImmutableSet#copyOf(Iterable) instead. (Or, change
+   * {@code elements} to be a FluentIterable and call {@code elements.toSet()}.)
+   *
+   * <p><b>Note:</b> if {@code E} is an {@link Enum} type, use
+   * newEnumSet(Iterable, Class) instead.
+   */
+  public static <E> HashSet<E> newHashSet(Iterable<? extends E> elements) {
+    return (elements instanceof Collection)
+        ? new HashSet<E>(cast(elements))
+        : newHashSet(elements.iterator());
+  }
+
+  /**
+   * Creates a <i>mutable</i> {@code TreeSet} instance containing the given
+   * elements sorted by their natural ordering.
+   *
+   * <p><b>Note:</b> if mutability is not required, use
+   * ImmutableSortedSet#copyOf(Iterable) instead.
+   *
+   * <p><b>Note:</b> If {@code elements} is a {@code SortedSet} with an
+   * explicit comparator, this method has different behavior than
+   * {@link TreeSet#TreeSet(SortedSet)}, which returns a {@code TreeSet}
+   * with that comparator.
+   *
+   * <p><b>Note for Java 7 and later:</b> this method is now unnecessary and
+   * should be treated as deprecated. Instead, use the {@code TreeSet}
+   * constructor directly, taking advantage of the new
+   * <a href="http://goo.gl/iz2Wi">"diamond" syntax</a>.
+   *
+   * <p>This method is just a small convenience for creating an empty set and
+   * then calling Iterables#addAll. This method is not very useful and will
+   * likely be deprecated in the future.
+   *
+   * @param elements the elements that the set should contain
+   * @return a new {@code TreeSet} containing those elements (minus duplicates)
+   */
+  public static <E extends Comparable> TreeSet<E> newTreeSet(
+      Iterable<? extends E> elements) {
+    TreeSet<E> set = newTreeSet();
+    addAll(set, elements);
+    return set;
+  }
+
+  private static <E extends Comparable> boolean addAll(TreeSet<E> addTo,
+      Iterable<? extends E> elementsToAdd) {
+    if (elementsToAdd instanceof Collection) {
+      Collection<? extends E> c = cast(elementsToAdd);
+      return addTo.addAll(c);
+    }
+    if (elementsToAdd == null) {
+      throw new NullPointerException();
+    }
+    return addAll(addTo, elementsToAdd.iterator());
+  }
+
+  /**
+   * Creates a <i>mutable</i> {@code HashSet} instance containing the given
+   * elements. A very thin convenience for creating an empty set and then
+   * calling Iterators#addAll.
+   *
+   * <p><b>Note:</b> if mutability is not required and the elements are
+   * non-null, use ImmutableSet#copyOf(Iterator) instead.
+   *
+   * <p><b>Note:</b> if {@code E} is an {@link Enum} type, you should create
+   * an {@link EnumSet} instead.
+   *
+   * <p>Overall, this method is not very useful and will likely be deprecated
+   * in the future.
+   */
+  public static <E> HashSet<E> newHashSet(Iterator<? extends E> elements) {
+    HashSet<E> set = newHashSet();
+    addAll(set, elements);
+    return set;
+  }
+
+  /**
+   * Returns a new hash set using the smallest initial table size that can hold
+   * {@code expectedSize} elements without resizing. Note that this is not what
+   * {@link HashSet#HashSet(int)} does, but it is what most users want and
+   * expect it to do.
+   *
+   * <p>This behavior can't be broadly guaranteed, but has been tested with
+   * OpenJDK 1.7 and 1.8.
+   *
+   * @param expectedSize the number of elements you expect to add to the
+   *     returned set
+   * @return a new, empty hash set with enough capacity to hold
+   *     {@code expectedSize} elements without resizing
+   * @throws IllegalArgumentException if {@code expectedSize} is negative
+   */
+  public static <E> HashSet<E> newHashSetWithExpectedSize(int expectedSize) {
+    return new HashSet<E>(capacity(expectedSize));
+  }
+
+  private static <E> Collection<E> cast(Iterable<E> iterable) {
+    return (Collection<E>) iterable;
+  }
+
+  private static <E> boolean addAll(Collection<E> addTo,
+      Iterator<? extends E> iterator) {
+    if (addTo == null) {
+      throw new NullPointerException();
+    }
+    if (iterator == null) {
+      throw new NullPointerException();
+    }
+    boolean wasModified = false;
+    while (iterator.hasNext()) {
+      wasModified |= addTo.add(iterator.next());
+    }
+    return wasModified;
+  }
+
+  /**
+   * Returns the intersection of two sets as an unmodifiable set.
+   * The returned set contains all elements that are contained by both backing
+   * sets.
+   *
+   * <p>Results are undefined if {@code set1} and {@code set2} are sets based
+   * on different equivalence relations (as {@code HashSet}, {@code TreeSet},
+   * and the keySet of an {@code IdentityHashMap} all are).
+   */
+  public static <E> Set<E> intersection(final Set<E> set1,
+      final Set<E> set2) {
+    if (set1 == null) {
+      throw new NullPointerException("set1");
+    }
+    if (set2 == null) {
+      throw new NullPointerException("set2");
+    }
+    Set<E> newSet = new HashSet<>(set1);
+    newSet.retainAll(set2);
+    return Collections.unmodifiableSet(newSet);
+  }
+
+  /**
+   * Returns the union of two sets as an unmodifiable set.
+   * The returned set contains all elements that are contained in either
+   * backing set.
+   *
+   * <p>Results are undefined if {@code set1} and {@code set2} are sets
+   * based on different equivalence relations (as {@link HashSet},
+   * {@link TreeSet}, and the {@link Map#keySet} of an
+   * {@code IdentityHashMap} all are).
+   */
+  public static <E> Set<E> union(
+      final Set<E> set1, final Set<E> set2) {
+    if (set1 == null) {
+      throw new NullPointerException("set1");
+    }
+    if (set2 == null) {
+      throw new NullPointerException("set2");
+    }
+    Set<E> newSet = new HashSet<>(set1);
+    newSet.addAll(set2);
+    return Collections.unmodifiableSet(newSet);
+  }
+
+  /**
+   * Returns the difference of two sets as an unmodifiable set.
+   * The returned set contains all elements that are contained by {@code set1}
+   * and not contained by {@code set2}.
+   *
+   * <p>Results are undefined if {@code set1} and {@code set2} are sets based
+   * on different equivalence relations (as {@code HashSet}, {@code TreeSet},
+   * and the keySet of an {@code IdentityHashMap} all are).
+   *
+   * This method is used to find difference for HashSets. For TreeSets with
+   * strict order requirement, recommended method is
+   * {@link #differenceInTreeSets(Set, Set)}.
+   */
+  public static <E> Set<E> difference(
+      final Set<E> set1, final Set<E> set2) {
+    if (set1 == null) {
+      throw new NullPointerException("set1");
+    }
+    if (set2 == null) {
+      throw new NullPointerException("set2");
+    }
+    Set<E> newSet = new HashSet<>(set1);
+    newSet.removeAll(set2);
+    return Collections.unmodifiableSet(newSet);
+  }
+
+  /**
+   * Returns the difference of two sets as an unmodifiable set.
+   * The returned set contains all elements that are contained by {@code set1}
+   * and not contained by {@code set2}.
+   *
+   * <p>Results are undefined if {@code set1} and {@code set2} are sets based
+   * on different equivalence relations (as {@code HashSet}, {@code TreeSet},
+   * and the keySet of an {@code IdentityHashMap} all are).
+   *
+   * This method is used to find difference for TreeSets. For HashSets,
+   * recommended method is {@link #difference(Set, Set)}.
+   */
+  public static <E> Set<E> differenceInTreeSets(
+      final Set<E> set1, final Set<E> set2) {
+    if (set1 == null) {
+      throw new NullPointerException("set1");
+    }
+    if (set2 == null) {
+      throw new NullPointerException("set2");
+    }
+    Set<E> newSet = new TreeSet<>(set1);
+    newSet.removeAll(set2);
+    return Collections.unmodifiableSet(newSet);
+  }
+
+  /**
+   * Returns the symmetric difference of two sets as an unmodifiable set.
+   * The returned set contains all elements that are contained in either
+   * {@code set1} or {@code set2} but not in both. The iteration order of the
+   * returned set is undefined.
+   *
+   * <p>Results are undefined if {@code set1} and {@code set2} are sets based
+   * on different equivalence relations (as {@code HashSet}, {@code TreeSet},
+   * and the keySet of an {@code IdentityHashMap} all are).
+   */
+  public static <E> Set<E> symmetricDifference(
+      final Set<E> set1, final Set<E> set2) {
+    if (set1 == null) {
+      throw new NullPointerException("set1");
+    }
+    if (set2 == null) {
+      throw new NullPointerException("set2");
+    }
+    Set<E> intersection = new HashSet<>(set1);
+    intersection.retainAll(set2);
+    Set<E> symmetricDifference = new HashSet<>(set1);
+    symmetricDifference.addAll(set2);
+    symmetricDifference.removeAll(intersection);
+    return Collections.unmodifiableSet(symmetricDifference);
+  }
+
+  /**
+   * Creates a thread-safe set backed by a hash map. The set is backed by a
+   * {@link ConcurrentHashMap} instance, and thus carries the same concurrency
+   * guarantees.
+   *
+   * <p>Unlike {@code HashSet}, this class does NOT allow {@code null} to be
+   * used as an element. The set is serializable.
+   *
+   * @return a new, empty thread-safe {@code Set}
+   */
+  public static <E> Set<E> newConcurrentHashSet() {
+    return Collections.newSetFromMap(new ConcurrentHashMap<E, Boolean>());
+  }
+
+  /**
+   * Returns a capacity that is sufficient to keep the map from being resized
+   * as long as it grows no larger than expectedSize and the load factor
+   * is ≥ its default (0.75).
+   * The implementation of this method is adapted from Guava version 27.0-jre.
+   */
+  private static int capacity(int expectedSize) {
+    if (expectedSize < 3) {
+      if (expectedSize < 0) {
+        throw new IllegalArgumentException(
+            "expectedSize cannot be negative but was: " + expectedSize);
+      }
+      return expectedSize + 1;
+    }
+    if (expectedSize < MAX_POWER_OF_TWO) {
+      // This is the calculation used in JDK8 to resize when a putAll
+      // happens; it seems to be the most conservative calculation we
+      // can make.  0.75 is the default load factor.
+      return (int) ((float) expectedSize / 0.75F + 1.0F);
+    }
+    return Integer.MAX_VALUE; // any large value
+  }
+
+}

+ 1 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestValueQueue.java

@@ -32,7 +32,7 @@ import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Assert;
 import org.junit.Assert;
 import org.junit.Test;
 import org.junit.Test;
 
 
-import org.apache.hadoop.thirdparty.com.google.common.collect.Sets;
+import org.apache.hadoop.util.Sets;
 
 
 public class TestValueQueue {
 public class TestValueQueue {
   Logger LOG = LoggerFactory.getLogger(TestValueQueue.class);
   Logger LOG = LoggerFactory.getLogger(TestValueQueue.class);

+ 1 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/kms/TestLoadBalancingKMSClientProvider.java

@@ -63,7 +63,7 @@ import org.junit.Test;
 import org.junit.rules.Timeout;
 import org.junit.rules.Timeout;
 import org.mockito.Mockito;
 import org.mockito.Mockito;
 
 
-import org.apache.hadoop.thirdparty.com.google.common.collect.Sets;
+import org.apache.hadoop.util.Sets;
 
 
 public class TestLoadBalancingKMSClientProvider {
 public class TestLoadBalancingKMSClientProvider {
 
 

+ 1 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java

@@ -72,7 +72,7 @@ import org.mockito.stubbing.Answer;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
 
 
 import org.apache.hadoop.thirdparty.com.google.common.base.Joiner;
 import org.apache.hadoop.thirdparty.com.google.common.base.Joiner;
-import org.apache.hadoop.thirdparty.com.google.common.collect.Sets;
+import org.apache.hadoop.util.Sets;
 
 
 import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
 import static org.apache.hadoop.util.functional.CommonCallableSupplier.submit;
 import static org.apache.hadoop.util.functional.CommonCallableSupplier.submit;

+ 32 - 0
hadoop-common-project/hadoop-kms/pom.xml

@@ -244,6 +244,38 @@
           </excludeFilterFile>
           </excludeFilterFile>
         </configuration>
         </configuration>
       </plugin>
       </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-enforcer-plugin</artifactId>
+        <dependencies>
+          <dependency>
+            <groupId>de.skuzzle.enforcer</groupId>
+            <artifactId>restrict-imports-enforcer-rule</artifactId>
+            <version>${restrict-imports.enforcer.version}</version>
+          </dependency>
+        </dependencies>
+        <executions>
+          <execution>
+            <id>banned-illegal-imports</id>
+            <phase>process-sources</phase>
+            <goals>
+              <goal>enforce</goal>
+            </goals>
+            <configuration>
+              <rules>
+                <restrictImports implementation="de.skuzzle.enforcer.restrictimports.rule.RestrictImports">
+                  <includeTestCode>true</includeTestCode>
+                  <reason>Use hadoop-common provided Sets rather than Guava provided Sets</reason>
+                  <bannedImports>
+                    <bannedImport>org.apache.hadoop.thirdparty.com.google.common.collect.Sets</bannedImport>
+                    <bannedImport>com.google.common.collect.Sets</bannedImport>
+                  </bannedImports>
+                </restrictImports>
+              </rules>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
     </plugins>
     </plugins>
   </build>
   </build>
 
 

+ 1 - 1
hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSAudit.java

@@ -36,7 +36,7 @@ import org.apache.hadoop.thirdparty.com.google.common.cache.Cache;
 import org.apache.hadoop.thirdparty.com.google.common.cache.CacheBuilder;
 import org.apache.hadoop.thirdparty.com.google.common.cache.CacheBuilder;
 import org.apache.hadoop.thirdparty.com.google.common.cache.RemovalListener;
 import org.apache.hadoop.thirdparty.com.google.common.cache.RemovalListener;
 import org.apache.hadoop.thirdparty.com.google.common.cache.RemovalNotification;
 import org.apache.hadoop.thirdparty.com.google.common.cache.RemovalNotification;
-import org.apache.hadoop.thirdparty.com.google.common.collect.Sets;
+import org.apache.hadoop.util.Sets;
 import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
 import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
 
 
 import java.util.HashSet;
 import java.util.HashSet;

+ 32 - 0
hadoop-tools/hadoop-aws/pom.xml

@@ -467,6 +467,38 @@
           </execution>
           </execution>
         </executions>
         </executions>
       </plugin>
       </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-enforcer-plugin</artifactId>
+        <dependencies>
+          <dependency>
+            <groupId>de.skuzzle.enforcer</groupId>
+            <artifactId>restrict-imports-enforcer-rule</artifactId>
+            <version>${restrict-imports.enforcer.version}</version>
+          </dependency>
+        </dependencies>
+        <executions>
+          <execution>
+            <id>banned-illegal-imports</id>
+            <phase>process-sources</phase>
+            <goals>
+              <goal>enforce</goal>
+            </goals>
+            <configuration>
+              <rules>
+                <restrictImports implementation="de.skuzzle.enforcer.restrictimports.rule.RestrictImports">
+                  <includeTestCode>true</includeTestCode>
+                  <reason>Use hadoop-common provided Sets rather than Guava provided Sets</reason>
+                  <bannedImports>
+                    <bannedImport>org.apache.hadoop.thirdparty.com.google.common.collect.Sets</bannedImport>
+                    <bannedImport>com.google.common.collect.Sets</bannedImport>
+                  </bannedImports>
+                </restrictImports>
+              </rules>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
     </plugins>
     </plugins>
   </build>
   </build>
 
 

+ 1 - 1
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/auth/AssumedRoleCredentialProvider.java

@@ -33,7 +33,7 @@ import com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider;
 import com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClientBuilder;
 import com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClientBuilder;
 import com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException;
 import com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException;
 import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.thirdparty.com.google.common.collect.Sets;
+import org.apache.hadoop.util.Sets;
 import org.slf4j.Logger;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
 
 

+ 2 - 2
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/Paths.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.fs.s3a.commit.staging;
 package org.apache.hadoop.fs.s3a.commit.staging;
 
 
 import java.io.IOException;
 import java.io.IOException;
+import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.List;
 import java.util.Set;
 import java.util.Set;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutionException;
@@ -27,7 +28,6 @@ import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTest
 import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
 import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
 import org.apache.hadoop.thirdparty.com.google.common.cache.Cache;
 import org.apache.hadoop.thirdparty.com.google.common.cache.Cache;
 import org.apache.hadoop.thirdparty.com.google.common.cache.CacheBuilder;
 import org.apache.hadoop.thirdparty.com.google.common.cache.CacheBuilder;
-import org.apache.hadoop.thirdparty.com.google.common.collect.Sets;
 import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.UncheckedExecutionException;
 import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.UncheckedExecutionException;
 
 
 import org.apache.commons.lang3.StringUtils;
 import org.apache.commons.lang3.StringUtils;
@@ -290,7 +290,7 @@ public final class Paths {
       List<? extends FileStatus> taskOutput)
       List<? extends FileStatus> taskOutput)
       throws IOException {
       throws IOException {
     // get a list of partition directories
     // get a list of partition directories
-    Set<String> partitions = Sets.newLinkedHashSet();
+    Set<String> partitions = new LinkedHashSet<>();
     for (FileStatus fileStatus : taskOutput) {
     for (FileStatus fileStatus : taskOutput) {
       // sanity check the output paths
       // sanity check the output paths
       Path outputFile = fileStatus.getPath();
       Path outputFile = fileStatus.getPath();

+ 1 - 1
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AAWSCredentialsProvider.java

@@ -30,7 +30,7 @@ import com.amazonaws.auth.AWSCredentials;
 import com.amazonaws.auth.AWSCredentialsProvider;
 import com.amazonaws.auth.AWSCredentialsProvider;
 import com.amazonaws.auth.EnvironmentVariableCredentialsProvider;
 import com.amazonaws.auth.EnvironmentVariableCredentialsProvider;
 import com.amazonaws.auth.InstanceProfileCredentialsProvider;
 import com.amazonaws.auth.InstanceProfileCredentialsProvider;
-import org.apache.hadoop.thirdparty.com.google.common.collect.Sets;
+import org.apache.hadoop.util.Sets;
 import org.junit.Rule;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
 import org.junit.rules.ExpectedException;

+ 1 - 1
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/integration/ITestS3ACommitterMRJob.java

@@ -34,7 +34,7 @@ import java.util.Set;
 import java.util.UUID;
 import java.util.UUID;
 import java.util.stream.Collectors;
 import java.util.stream.Collectors;
 
 
-import org.apache.hadoop.thirdparty.com.google.common.collect.Sets;
+import org.apache.hadoop.util.Sets;
 import org.assertj.core.api.Assertions;
 import org.assertj.core.api.Assertions;
 import org.junit.FixMethodOrder;
 import org.junit.FixMethodOrder;
 import org.junit.Rule;
 import org.junit.Rule;

+ 1 - 1
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingCommitter.java

@@ -35,7 +35,7 @@ import com.amazonaws.services.s3.AmazonS3;
 import com.amazonaws.services.s3.model.AbortMultipartUploadRequest;
 import com.amazonaws.services.s3.model.AbortMultipartUploadRequest;
 import com.amazonaws.services.s3.model.CompleteMultipartUploadRequest;
 import com.amazonaws.services.s3.model.CompleteMultipartUploadRequest;
 
 
-import org.apache.hadoop.thirdparty.com.google.common.collect.Sets;
+import org.apache.hadoop.util.Sets;
 import org.assertj.core.api.Assertions;
 import org.assertj.core.api.Assertions;
 import org.hamcrest.core.StringStartsWith;
 import org.hamcrest.core.StringStartsWith;
 import org.junit.After;
 import org.junit.After;

+ 1 - 1
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingPartitionedTaskCommit.java

@@ -26,7 +26,7 @@ import java.util.UUID;
 
 
 import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest;
 import com.amazonaws.services.s3.model.InitiateMultipartUploadRequest;
 import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
 import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
-import org.apache.hadoop.thirdparty.com.google.common.collect.Sets;
+import org.apache.hadoop.util.Sets;
 import org.assertj.core.api.Assertions;
 import org.assertj.core.api.Assertions;
 import org.junit.BeforeClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.Test;

+ 32 - 0
hadoop-tools/hadoop-distcp/pom.xml

@@ -223,6 +223,38 @@
           </execution>
           </execution>
         </executions>
         </executions>
       </plugin>
       </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-enforcer-plugin</artifactId>
+        <dependencies>
+          <dependency>
+            <groupId>de.skuzzle.enforcer</groupId>
+            <artifactId>restrict-imports-enforcer-rule</artifactId>
+            <version>${restrict-imports.enforcer.version}</version>
+          </dependency>
+        </dependencies>
+        <executions>
+          <execution>
+            <id>banned-illegal-imports</id>
+            <phase>process-sources</phase>
+            <goals>
+              <goal>enforce</goal>
+            </goals>
+            <configuration>
+              <rules>
+                <restrictImports implementation="de.skuzzle.enforcer.restrictimports.rule.RestrictImports">
+                  <includeTestCode>true</includeTestCode>
+                  <reason>Use hadoop-common provided Sets rather than Guava provided Sets</reason>
+                  <bannedImports>
+                    <bannedImport>org.apache.hadoop.thirdparty.com.google.common.collect.Sets</bannedImport>
+                    <bannedImport>com.google.common.collect.Sets</bannedImport>
+                  </bannedImports>
+                </restrictImports>
+              </rules>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
     </plugins>
     </plugins>
   </build>
   </build>
 </project>
 </project>

+ 1 - 1
hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/CopyListing.java

@@ -35,7 +35,7 @@ import java.lang.reflect.Constructor;
 import java.net.URI;
 import java.net.URI;
 import java.util.Set;
 import java.util.Set;
 
 
-import org.apache.hadoop.thirdparty.com.google.common.collect.Sets;
+import org.apache.hadoop.util.Sets;
 
 
 /**
 /**
  * The CopyListing abstraction is responsible for how the list of
  * The CopyListing abstraction is responsible for how the list of

+ 1 - 1
hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/test/java/org/apache/hadoop/tools/dynamometer/TestDynamometerInfra.java

@@ -17,7 +17,7 @@
  */
  */
 package org.apache.hadoop.tools.dynamometer;
 package org.apache.hadoop.tools.dynamometer;
 
 
-import org.apache.hadoop.thirdparty.com.google.common.collect.Sets;
+import org.apache.hadoop.util.Sets;
 import java.util.Optional;
 import java.util.Optional;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeUnit;

+ 32 - 0
hadoop-tools/hadoop-dynamometer/pom.xml

@@ -50,6 +50,38 @@
         <configuration>
         <configuration>
         </configuration>
         </configuration>
       </plugin>
       </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-enforcer-plugin</artifactId>
+        <dependencies>
+          <dependency>
+            <groupId>de.skuzzle.enforcer</groupId>
+            <artifactId>restrict-imports-enforcer-rule</artifactId>
+            <version>${restrict-imports.enforcer.version}</version>
+          </dependency>
+        </dependencies>
+        <executions>
+          <execution>
+            <id>banned-illegal-imports</id>
+            <phase>process-sources</phase>
+            <goals>
+              <goal>enforce</goal>
+            </goals>
+            <configuration>
+              <rules>
+                <restrictImports implementation="de.skuzzle.enforcer.restrictimports.rule.RestrictImports">
+                  <includeTestCode>true</includeTestCode>
+                  <reason>Use hadoop-common provided Sets rather than Guava provided Sets</reason>
+                  <bannedImports>
+                    <bannedImport>org.apache.hadoop.thirdparty.com.google.common.collect.Sets</bannedImport>
+                    <bannedImport>com.google.common.collect.Sets</bannedImport>
+                  </bannedImports>
+                </restrictImports>
+              </rules>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
     </plugins>
     </plugins>
   </build>
   </build>
 </project>
 </project>