This is an automated email from the ASF dual-hosted git repository.

iemejia pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/beam.git


The following commit(s) were added to refs/heads/master by this push:
     new 28197ff  [BEAM-6426] Enforce ban of non-vendored Guava, with exceptions
     new a87b2db  Merge pull request #7695: [BEAM-6426] Enforce ban of 
non-vendored Guava, with exceptions
28197ff is described below

commit 28197ff31af75eb7647bfc0b2cb20ace35c5fc88
Author: Ismaël Mejía <[email protected]>
AuthorDate: Fri Feb 1 01:06:49 2019 +0100

    [BEAM-6426] Enforce ban of non-vendored Guava, with exceptions
---
 sdks/java/build-tools/src/main/resources/beam/checkstyle.xml  | 11 +++++++++++
 .../java/build-tools/src/main/resources/beam/suppressions.xml |  3 +++
 .../src/main/java/org/apache/beam/sdk/io/LocalFileSystem.java |  3 ++-
 .../core/src/main/java/org/apache/beam/sdk/io/Source.java     |  4 ++--
 .../main/java/org/apache/beam/sdk/transforms/Reshuffle.java   |  3 ++-
 .../sdk/transforms/splittabledofn/RestrictionTracker.java     |  2 +-
 6 files changed, 21 insertions(+), 5 deletions(-)

diff --git a/sdks/java/build-tools/src/main/resources/beam/checkstyle.xml 
b/sdks/java/build-tools/src/main/resources/beam/checkstyle.xml
index 4a1b547..c0fe78a 100644
--- a/sdks/java/build-tools/src/main/resources/beam/checkstyle.xml
+++ b/sdks/java/build-tools/src/main/resources/beam/checkstyle.xml
@@ -95,6 +95,17 @@ page at http://checkstyle.sourceforge.net/config.html -->
       <property name="format" 
value="com\.google\.api\.client\.util\.(ByteStreams|Charsets|Collections2|Joiner|Lists|Maps|Objects|Preconditions|Sets|Strings|Throwables)"/>
     </module>
 
+    <!--
+        Forbid non-vendored guava imports.
+        testing import is allowed because it comes from guava-testing for the 
tests.
+    -->
+    <module name="RegexpSinglelineJava">
+      <property name="id" value="ForbidNonVendoredGuava"/>
+      <property name="format" value="\scom\.google\.common\.(?!testing)"/>
+      <property name="format" value="\scom\.google\.thirdparty\."/>
+      <property name="severity" value="error"/>
+    </module>
+
     <module name="UnusedImports">
       <property name="severity" value="error"/>
       <property name="processJavadoc" value="true"/>
diff --git a/sdks/java/build-tools/src/main/resources/beam/suppressions.xml 
b/sdks/java/build-tools/src/main/resources/beam/suppressions.xml
index 4bd1e78..e2541a5 100644
--- a/sdks/java/build-tools/src/main/resources/beam/suppressions.xml
+++ b/sdks/java/build-tools/src/main/resources/beam/suppressions.xml
@@ -93,4 +93,7 @@
   <suppress checks="JavadocMethod" 
files=".*flink.*wrappers.*ExecutableStageDoFnOperator\.java" lines="119"/>
   <!-- End of Suppression on JavadocMethod violations. -->
 
+  <!-- Guava use exceptions -->
+  <!-- Cassandra driver exposes guava in its API so we need to suppress the 
error for this case. -->
+  <suppress id="ForbidNonVendoredGuava" 
files=".*cassandra.*CassandraServiceImpl\.java" />
 </suppressions>
diff --git 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/LocalFileSystem.java 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/LocalFileSystem.java
index 8d231ac..dc4a918 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/LocalFileSystem.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/LocalFileSystem.java
@@ -18,6 +18,7 @@
 package org.apache.beam.sdk.io;
 
 import static 
org.apache.beam.vendor.guava.v20_0.com.google.common.base.Preconditions.checkArgument;
+import static 
org.apache.beam.vendor.guava.v20_0.com.google.common.io.Files.fileTreeTraverser;
 
 import java.io.BufferedOutputStream;
 import java.io.File;
@@ -243,7 +244,7 @@ class LocalFileSystem extends FileSystem<LocalResourceId> {
         java.nio.file.FileSystems.getDefault().getPathMatcher("glob:" + 
pathToMatch);
 
     // TODO: Avoid iterating all files: 
https://issues.apache.org/jira/browse/BEAM-1309
-    Iterable<File> files = 
com.google.common.io.Files.fileTreeTraverser().preOrderTraversal(parent);
+    Iterable<File> files = fileTreeTraverser().preOrderTraversal(parent);
     Iterable<File> matchedFiles =
         StreamSupport.stream(files.spliterator(), false)
             .filter(
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Source.java 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Source.java
index fb41773..b32d7a8 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Source.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Source.java
@@ -24,6 +24,7 @@ import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.transforms.display.DisplayData;
 import org.apache.beam.sdk.transforms.display.HasDisplayData;
+import org.apache.beam.vendor.guava.v20_0.com.google.common.base.Preconditions;
 import org.joda.time.Instant;
 
 /**
@@ -54,8 +55,7 @@ public abstract class Source<T> implements Serializable, 
HasDisplayData {
   /**
    * Checks that this source is valid, before it can be used in a pipeline.
    *
-   * <p>It is recommended to use {@link com.google.common.base.Preconditions} 
for implementing this
-   * method.
+   * <p>It is recommended to use {@link Preconditions} for implementing this 
method.
    */
   public void validate() {}
 
diff --git 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Reshuffle.java 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Reshuffle.java
index abf6928..ca7615d 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Reshuffle.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Reshuffle.java
@@ -134,7 +134,8 @@ public class Reshuffle<K, V> extends 
PTransform<PCollection<KV<K, V>>, PCollecti
         // which for Integer is a no-op and it is an issue:
         // 
http://hydronitrogen.com/poor-hash-partitioning-of-timestamps-integers-and-longs-in-
         // spark.html
-        // This hashing strategy is copied from 
com.google.common.collect.Hashing.smear().
+        // This hashing strategy is copied from
+        // 
org.apache.beam.vendor.guava.v20_0.com.google.common.collect.Hashing.smear().
         int hashOfShard = 0x1b873593 * Integer.rotateLeft(shard * 0xcc9e2d51, 
15);
         r.output(KV.of(hashOfShard, element));
       }
diff --git 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/RestrictionTracker.java
 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/RestrictionTracker.java
index 8b59f05..11ddcac 100644
--- 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/RestrictionTracker.java
+++ 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/RestrictionTracker.java
@@ -18,7 +18,7 @@
 package org.apache.beam.sdk.transforms.splittabledofn;
 
 import static com.google.common.base.Preconditions.checkNotNull;
-import static com.google.common.base.Preconditions.checkState;
+import static 
org.apache.beam.vendor.guava.v20_0.com.google.common.base.Preconditions.checkState;
 
 import javax.annotation.Nullable;
 import org.apache.beam.sdk.annotations.Internal;

Reply via email to