You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by ke...@apache.org on 2017/01/24 03:50:12 UTC

[1/2] beam git commit: Revert "Simplified API surface verifications"

Repository: beam
Updated Branches:
  refs/heads/master 6ecbfb9e1 -> cb6e0a80c


Revert "Simplified API surface verifications"

This reverts commit 29ffaf3859ba9b4d8ba8529efc96fd5e105e21a3.

The change to require all whitelisted packages to actually expose something
failed in the Dataflow runner postcommit. To be rolled forward after
fixing.


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/9248befb
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/9248befb
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/9248befb

Branch: refs/heads/master
Commit: 9248befbbba6b2d18cde3b7ee562b13af33681fc
Parents: 26a2c47
Author: Kenneth Knowles <kl...@google.com>
Authored: Mon Jan 23 16:02:24 2017 -0800
Committer: Kenneth Knowles <kl...@google.com>
Committed: Mon Jan 23 16:03:47 2017 -0800

----------------------------------------------------------------------
 .../org/apache/beam/sdk/util/ApiSurface.java    | 420 +++++--------------
 .../org/apache/beam/SdkCoreApiSurfaceTest.java  |  61 ---
 .../apache/beam/sdk/util/ApiSurfaceTest.java    | 152 +++++--
 .../apache/beam/sdk/io/gcp/ApiSurfaceTest.java  | 134 ++++++
 .../beam/sdk/io/gcp/GcpApiSurfaceTest.java      |  76 ----
 5 files changed, 359 insertions(+), 484 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/9248befb/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ApiSurface.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ApiSurface.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ApiSurface.java
index b6b0b32..2040161 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ApiSurface.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ApiSurface.java
@@ -17,21 +17,12 @@
  */
 package org.apache.beam.sdk.util;
 
-import static org.hamcrest.Matchers.anyOf;
-
-import com.google.common.base.Function;
 import com.google.common.base.Joiner;
-import com.google.common.base.Predicate;
 import com.google.common.base.Supplier;
-import com.google.common.collect.FluentIterable;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Multimap;
 import com.google.common.collect.Multimaps;
-import com.google.common.collect.Ordering;
 import com.google.common.collect.Sets;
 import com.google.common.reflect.ClassPath;
 import com.google.common.reflect.ClassPath.ClassInfo;
@@ -54,20 +45,15 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Set;
 import java.util.regex.Pattern;
-import javax.annotation.Nonnull;
-import org.hamcrest.Description;
-import org.hamcrest.Matcher;
-import org.hamcrest.StringDescription;
-import org.hamcrest.TypeSafeDiagnosingMatcher;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * Represents the API surface of a package prefix. Used for accessing public classes, methods, and
- * the types they reference, to control what dependencies are re-exported.
+ * Represents the API surface of a package prefix. Used for accessing public classes,
+ * methods, and the types they reference, to control what dependencies are re-exported.
  *
- * <p>For the purposes of calculating the public API surface, exposure includes any public or
- * protected occurrence of:
+ * <p>For the purposes of calculating the public API surface, exposure includes any public
+ * or protected occurrence of:
  *
  * <ul>
  * <li>superclasses
@@ -80,272 +66,42 @@ import org.slf4j.LoggerFactory;
  * <li>wildcard bounds
  * </ul>
  *
- * <p>Exposure is a transitive property. The resulting map excludes primitives and array classes
- * themselves.
+ * <p>Exposure is a transitive property. The resulting map excludes primitives
+ * and array classes themselves.
  *
- * <p>It is prudent (though not required) to prune prefixes like "java" via the builder method
- * {@link #pruningPrefix} to halt the traversal so it does not uselessly catalog references that are
- * not interesting.
+ * <p>It is prudent (though not required) to prune prefixes like "java" via the builder
+ * method {@link #pruningPrefix} to halt the traversal so it does not uselessly catalog references
+ * that are not interesting.
  */
 @SuppressWarnings("rawtypes")
 public class ApiSurface {
   private static final Logger LOG = LoggerFactory.getLogger(ApiSurface.class);
 
-  /** A factory method to create a {@link Class} matcher for classes residing in a given package. */
-  public static Matcher<Class<?>> classesInPackage(final String packageName) {
-    return new Matchers.ClassInPackage(packageName);
-  }
-
-  /**
-   * A factory method to create an {@link ApiSurface} matcher, producing a positive match if the
-   * queried api surface contains ONLY classes described by the provided matchers.
-   */
-  public static Matcher<ApiSurface> containsOnlyClassesMatching(
-      final Set<Matcher<Class<?>>> classMatchers) {
-    return new Matchers.ClassesInSurfaceMatcher(classMatchers);
-  }
-
-  /** See {@link ApiSurface#containsOnlyClassesMatching(Set)}. */
-  @SafeVarargs
-  public static Matcher<ApiSurface> containsOnlyClassesMatching(
-      final Matcher<Class<?>>... classMatchers) {
-    return new Matchers.ClassesInSurfaceMatcher(Sets.newHashSet(classMatchers));
-  }
-
-  /** See {@link ApiSurface#containsOnlyPackages(Set)}. */
-  public static Matcher<ApiSurface> containsOnlyPackages(final String... packageNames) {
-    return containsOnlyPackages(Sets.newHashSet(packageNames));
-  }
-
-  /**
-   * A factory method to create an {@link ApiSurface} matcher, producing a positive match if the
-   * queried api surface contains classes ONLY from specified package names.
-   */
-  public static Matcher<ApiSurface> containsOnlyPackages(final Set<String> packageNames) {
-
-    final Function<String, Matcher<Class<?>>> packageNameToClassMatcher =
-        new Function<String, Matcher<Class<?>>>() {
-
-          @Override
-          public Matcher<Class<?>> apply(@Nonnull final String packageName) {
-            return classesInPackage(packageName);
-          }
-        };
-
-    final ImmutableSet<Matcher<Class<?>>> classesInPackages =
-        FluentIterable.from(packageNames).transform(packageNameToClassMatcher).toSet();
-
-    return containsOnlyClassesMatching(classesInPackages);
-  }
-
   /**
-   * {@link Matcher}s for use in {@link ApiSurface} related tests that aim to keep the public API
-   * conformant to a hard-coded policy by controlling what classes are allowed to be exposed by an
-   * API surface.
+   * Returns an empty {@link ApiSurface}.
    */
-  // based on previous code by @kennknowles and others.
-  private static class Matchers {
-
-    private static class ClassInPackage extends TypeSafeDiagnosingMatcher<Class<?>> {
-
-      private final String packageName;
-
-      private ClassInPackage(final String packageName) {
-        this.packageName = packageName;
-      }
-
-      @Override
-      public void describeTo(final Description description) {
-        description.appendText("Classes in package \"");
-        description.appendText(packageName);
-        description.appendText("\"");
-      }
-
-      @Override
-      protected boolean matchesSafely(final Class<?> clazz, final Description mismatchDescription) {
-        return clazz.getName().startsWith(packageName + ".");
-      }
-    }
-
-    private static class ClassesInSurfaceMatcher extends TypeSafeDiagnosingMatcher<ApiSurface> {
-
-      private final Set<Matcher<Class<?>>> classMatchers;
-
-      private ClassesInSurfaceMatcher(final Set<Matcher<Class<?>>> classMatchers) {
-        this.classMatchers = classMatchers;
-      }
-
-      private boolean verifyNoAbandoned(
-          final ApiSurface checkedApiSurface,
-          final Set<Matcher<Class<?>>> allowedClasses,
-          final Description mismatchDescription) {
-
-        // <helper_lambdas>
-
-        final Function<Matcher<Class<?>>, String> toMessage =
-            new Function<Matcher<Class<?>>, String>() {
-
-              @Override
-              public String apply(@Nonnull final Matcher<Class<?>> abandonedClassMacther) {
-                final StringDescription description = new StringDescription();
-                description.appendText("No ");
-                abandonedClassMacther.describeTo(description);
-                return description.toString();
-              }
-            };
-
-        final Predicate<Matcher<Class<?>>> matchedByExposedClasses =
-            new Predicate<Matcher<Class<?>>>() {
-
-              @Override
-              public boolean apply(@Nonnull final Matcher<Class<?>> classMatcher) {
-                return FluentIterable.from(checkedApiSurface.getExposedClasses())
-                    .anyMatch(
-                        new Predicate<Class<?>>() {
-
-                          @Override
-                          public boolean apply(@Nonnull final Class<?> aClass) {
-                            return classMatcher.matches(aClass);
-                          }
-                        });
-              }
-            };
-
-        // </helper_lambdas>
-
-        final ImmutableSet<Matcher<Class<?>>> matchedClassMatchers =
-            FluentIterable.from(allowedClasses).filter(matchedByExposedClasses).toSet();
-
-        final Sets.SetView<Matcher<Class<?>>> abandonedClassMatchers =
-            Sets.difference(allowedClasses, matchedClassMatchers);
-
-        final ImmutableList<String> messages =
-            FluentIterable.from(abandonedClassMatchers)
-                .transform(toMessage)
-                .toSortedList(Ordering.<String>natural());
-
-        if (!messages.isEmpty()) {
-          mismatchDescription.appendText(
-              "The following white-listed scopes did not have matching classes on the API surface:"
-                  + "\n\t"
-                  + Joiner.on("\n\t").join(messages));
-        }
-
-        return messages.isEmpty();
-      }
-
-      private boolean verifyNoDisallowed(
-          final ApiSurface checkedApiSurface,
-          final Set<Matcher<Class<?>>> allowedClasses,
-          final Description mismatchDescription) {
-
-        /* <helper_lambdas> */
-
-        final Function<Class<?>, List<Class<?>>> toExposure =
-            new Function<Class<?>, List<Class<?>>>() {
-
-              @Override
-              public List<Class<?>> apply(@Nonnull final Class<?> aClass) {
-                return checkedApiSurface.getAnyExposurePath(aClass);
-              }
-            };
-
-        final Maps.EntryTransformer<Class<?>, List<Class<?>>, String> toMessage =
-            new Maps.EntryTransformer<Class<?>, List<Class<?>>, String>() {
-
-              @Override
-              public String transformEntry(
-                  @Nonnull final Class<?> aClass, @Nonnull final List<Class<?>> exposure) {
-                return aClass + " exposed via:\n\t\t" + Joiner.on("\n\t\t").join(exposure);
-              }
-            };
-
-        final Predicate<Class<?>> disallowed =
-            new Predicate<Class<?>>() {
-
-              @Override
-              public boolean apply(@Nonnull final Class<?> aClass) {
-                return !classIsAllowed(aClass, allowedClasses);
-              }
-            };
-
-        /* </helper_lambdas> */
-
-        final FluentIterable<Class<?>> disallowedClasses =
-            FluentIterable.from(checkedApiSurface.getExposedClasses()).filter(disallowed);
-
-        final ImmutableMap<Class<?>, List<Class<?>>> exposures =
-            Maps.toMap(disallowedClasses, toExposure);
-
-        final ImmutableList<String> messages =
-            FluentIterable.from(Maps.transformEntries(exposures, toMessage).values())
-                .toSortedList(Ordering.<String>natural());
-
-        if (!messages.isEmpty()) {
-          mismatchDescription.appendText(
-              "The following disallowed classes appeared on the API surface:\n\t"
-                  + Joiner.on("\n\t").join(messages));
-        }
-
-        return messages.isEmpty();
-      }
-
-      @SuppressWarnings({"rawtypes", "unchecked"})
-      private boolean classIsAllowed(
-          final Class<?> clazz, final Set<Matcher<Class<?>>> allowedClasses) {
-        // Safe cast inexpressible in Java without rawtypes
-        return anyOf((Iterable) allowedClasses).matches(clazz);
-      }
-
-      @Override
-      protected boolean matchesSafely(
-          final ApiSurface apiSurface, final Description mismatchDescription) {
-        final boolean noDisallowed =
-            verifyNoDisallowed(apiSurface, classMatchers, mismatchDescription);
-
-        final boolean noAbandoned =
-            verifyNoAbandoned(apiSurface, classMatchers, mismatchDescription);
-
-        return noDisallowed & noAbandoned;
-      }
-
-      @Override
-      public void describeTo(final Description description) {
-        description.appendText("API surface to include only:" + "\n\t");
-        for (final Matcher<Class<?>> classMatcher : classMatchers) {
-          classMatcher.describeTo(description);
-          description.appendText("\n\t");
-        }
-      }
-    }
-  }
-
-  ///////////////
-
-  /** Returns an empty {@link ApiSurface}. */
   public static ApiSurface empty() {
     LOG.debug("Returning an empty ApiSurface");
     return new ApiSurface(Collections.<Class<?>>emptySet(), Collections.<Pattern>emptySet());
   }
 
-  /** Returns an {@link ApiSurface} object representing the given package and all subpackages. */
+  /**
+   * Returns an {@link ApiSurface} object representing the given package and all subpackages.
+   */
   public static ApiSurface ofPackage(String packageName) throws IOException {
     return ApiSurface.empty().includingPackage(packageName);
   }
 
-  /** Returns an {@link ApiSurface} object representing the given package and all subpackages. */
-  public static ApiSurface ofPackage(Package aPackage) throws IOException {
-    return ApiSurface.empty().includingPackage(aPackage.getName());
-  }
-
-  /** Returns an {@link ApiSurface} object representing just the surface of the given class. */
+  /**
+   * Returns an {@link ApiSurface} object representing just the surface of the given class.
+   */
   public static ApiSurface ofClass(Class<?> clazz) {
     return ApiSurface.empty().includingClass(clazz);
   }
 
   /**
-   * Returns an {@link ApiSurface} like this one, but also including the named package and all of
-   * its subpackages.
+   * Returns an {@link ApiSurface} like this one, but also including the named
+   * package and all of its subpackages.
    */
   public ApiSurface includingPackage(String packageName) throws IOException {
     ClassPath classPath = ClassPath.from(ClassLoader.getSystemClassLoader());
@@ -363,7 +119,9 @@ public class ApiSurface {
     return new ApiSurface(newRootClasses, patternsToPrune);
   }
 
-  /** Returns an {@link ApiSurface} like this one, but also including the given class. */
+  /**
+   * Returns an {@link ApiSurface} like this one, but also including the given class.
+   */
   public ApiSurface includingClass(Class<?> clazz) {
     Set<Class<?>> newRootClasses = Sets.newHashSet();
     LOG.debug("Including class {}", clazz);
@@ -373,28 +131,32 @@ public class ApiSurface {
   }
 
   /**
-   * Returns an {@link ApiSurface} like this one, but pruning transitive references from classes
-   * whose full name (including package) begins with the provided prefix.
+   * Returns an {@link ApiSurface} like this one, but pruning transitive
+   * references from classes whose full name (including package) begins with the provided prefix.
    */
   public ApiSurface pruningPrefix(String prefix) {
     return pruningPattern(Pattern.compile(Pattern.quote(prefix) + ".*"));
   }
 
-  /** Returns an {@link ApiSurface} like this one, but pruning references from the named class. */
+  /**
+   * Returns an {@link ApiSurface} like this one, but pruning references from the named
+   * class.
+   */
   public ApiSurface pruningClassName(String className) {
     return pruningPattern(Pattern.compile(Pattern.quote(className)));
   }
 
   /**
-   * Returns an {@link ApiSurface} like this one, but pruning references from the provided class.
+   * Returns an {@link ApiSurface} like this one, but pruning references from the
+   * provided class.
    */
   public ApiSurface pruningClass(Class<?> clazz) {
     return pruningClassName(clazz.getName());
   }
 
   /**
-   * Returns an {@link ApiSurface} like this one, but pruning transitive references from classes
-   * whose full name (including package) begins with the provided prefix.
+   * Returns an {@link ApiSurface} like this one, but pruning transitive
+   * references from classes whose full name (including package) begins with the provided prefix.
    */
   public ApiSurface pruningPattern(Pattern pattern) {
     Set<Pattern> newPatterns = Sets.newHashSet();
@@ -403,26 +165,35 @@ public class ApiSurface {
     return new ApiSurface(rootClasses, newPatterns);
   }
 
-  /** See {@link #pruningPattern(Pattern)}. */
+  /**
+   * See {@link #pruningPattern(Pattern)}.
+   */
   public ApiSurface pruningPattern(String patternString) {
     return pruningPattern(Pattern.compile(patternString));
   }
 
-  /** Returns all public classes originally belonging to the package in the {@link ApiSurface}. */
+  /**
+   * Returns all public classes originally belonging to the package
+   * in the {@link ApiSurface}.
+   */
   public Set<Class<?>> getRootClasses() {
     return rootClasses;
   }
 
-  /** Returns exposed types in this set, including arrays and primitives as specified. */
+  /**
+   * Returns exposed types in this set, including arrays and primitives as
+   * specified.
+   */
   public Set<Class<?>> getExposedClasses() {
     return getExposedToExposers().keySet();
   }
 
   /**
-   * Returns a path from an exposed class to a root class. There may be many, but this gives only
-   * one.
+   * Returns a path from an exposed class to a root class. There may be many, but this
+   * gives only one.
    *
-   * <p>If there are only cycles, with no path back to a root class, throws IllegalStateException.
+   * <p>If there are only cycles, with no path back to a root class, throws
+   * IllegalStateException.
    */
   public List<Class<?>> getAnyExposurePath(Class<?> exposedClass) {
     Set<Class<?>> excluded = Sets.newHashSet();
@@ -430,18 +201,16 @@ public class ApiSurface {
     List<Class<?>> path = getAnyExposurePath(exposedClass, excluded);
     if (path == null) {
       throw new IllegalArgumentException(
-          "Class "
-              + exposedClass
-              + " has no path back to any root class."
-              + " It should never have been considered exposed.");
+          "Class " + exposedClass + " has no path back to any root class."
+          + " It should never have been considered exposed.");
     } else {
       return path;
     }
   }
 
   /**
-   * Returns a path from an exposed class to a root class. There may be many, but this gives only
-   * one. It will not return a path that crosses the excluded classes.
+   * Returns a path from an exposed class to a root class. There may be many, but this
+   * gives only one. It will not return a path that crosses the excluded classes.
    *
    * <p>If there are only cycles or paths through the excluded classes, returns null.
    *
@@ -466,8 +235,9 @@ public class ApiSurface {
         return exposurePath;
       }
 
-      List<Class<?>> restOfPath =
-          getAnyExposurePath(exposer, Sets.union(excluded, Sets.newHashSet(exposer)));
+      List<Class<?>> restOfPath = getAnyExposurePath(
+          exposer,
+          Sets.union(excluded, Sets.newHashSet(exposer)));
 
       if (restOfPath != null) {
         exposurePath.addAll(restOfPath);
@@ -494,8 +264,8 @@ public class ApiSurface {
   }
 
   /**
-   * A map from exposed types to place where they are exposed, in the sense of being a part of a
-   * public-facing API surface.
+   * A map from exposed types to place where they are exposed, in the sense of being a part
+   * of a public-facing API surface.
    *
    * <p>This map is the adjencency list representation of a directed graph, where an edge from type
    * {@code T1} to type {@code T2} indicates that {@code T2} directly exposes {@code T1} in its API
@@ -511,25 +281,28 @@ public class ApiSurface {
     return exposedToExposers;
   }
 
-  /** See {@link #getExposedToExposers}. */
+  /**
+   * See {@link #getExposedToExposers}.
+   */
   private void constructExposedToExposers() {
     visited = Sets.newHashSet();
-    exposedToExposers =
-        Multimaps.newSetMultimap(
-            Maps.<Class<?>, Collection<Class<?>>>newHashMap(),
-            new Supplier<Set<Class<?>>>() {
-              @Override
-              public Set<Class<?>> get() {
-                return Sets.newHashSet();
-              }
-            });
+    exposedToExposers = Multimaps.newSetMultimap(
+        Maps.<Class<?>, Collection<Class<?>>>newHashMap(),
+        new Supplier<Set<Class<?>>>() {
+          @Override
+          public Set<Class<?>> get() {
+            return Sets.newHashSet();
+          }
+        });
 
     for (Class<?> clazz : rootClasses) {
       addExposedTypes(clazz, null);
     }
   }
 
-  /** A combined {@code Pattern} that implements all the pruning specified. */
+  /**
+   * A combined {@code Pattern} that implements all the pruning specified.
+   */
   private Pattern getPrunedPattern() {
     if (prunedPattern == null) {
       constructPrunedPattern();
@@ -537,7 +310,9 @@ public class ApiSurface {
     return prunedPattern;
   }
 
-  /** See {@link #getPrunedPattern}. */
+  /**
+   * See {@link #getPrunedPattern}.
+   */
   private void constructPrunedPattern() {
     Set<String> prunedPatternStrings = Sets.newHashSet();
     for (Pattern patternToPrune : patternsToPrune) {
@@ -546,19 +321,25 @@ public class ApiSurface {
     prunedPattern = Pattern.compile("(" + Joiner.on(")|(").join(prunedPatternStrings) + ")");
   }
 
-  /** Whether a type and all that it references should be pruned from the graph. */
+  /**
+   * Whether a type and all that it references should be pruned from the graph.
+   */
   private boolean pruned(Type type) {
     return pruned(TypeToken.of(type).getRawType());
   }
 
-  /** Whether a class and all that it references should be pruned from the graph. */
+  /**
+   * Whether a class and all that it references should be pruned from the graph.
+   */
   private boolean pruned(Class<?> clazz) {
     return clazz.isPrimitive()
         || clazz.isArray()
         || getPrunedPattern().matcher(clazz.getName()).matches();
   }
 
-  /** Whether a type has already beens sufficiently processed. */
+  /**
+   * Whether a type has already beens sufficiently processed.
+   */
   private boolean done(Type type) {
     return visited.contains(type);
   }
@@ -575,7 +356,9 @@ public class ApiSurface {
     visited.add(type);
   }
 
-  /** See {@link #addExposedTypes(Type, Class)}. */
+  /**
+   * See {@link #addExposedTypes(Type, Class)}.
+   */
   private void addExposedTypes(TypeToken type, Class<?> cause) {
     LOG.debug(
         "Adding exposed types from {}, which is the type in type token {}", type.getType(), type);
@@ -583,9 +366,9 @@ public class ApiSurface {
   }
 
   /**
-   * Adds any references learned by following a link from {@code cause} to {@code type}. This will
-   * dispatch according to the concrete {@code Type} implementation. See the other overloads of
-   * {@code addExposedTypes} for their details.
+   * Adds any references learned by following a link from {@code cause} to {@code type}.
+   * This will dispatch according to the concrete {@code Type} implementation. See the
+   * other overloads of {@code addExposedTypes} for their details.
    */
   private void addExposedTypes(Type type, Class<?> cause) {
     if (type instanceof TypeVariable) {
@@ -609,7 +392,8 @@ public class ApiSurface {
   }
 
   /**
-   * Adds any types exposed to this set. These will come from the (possibly absent) bounds on the
+   * Adds any types exposed to this set. These will
+   * come from the (possibly absent) bounds on the
    * type variable.
    */
   private void addExposedTypes(TypeVariable type, Class<?> cause) {
@@ -646,8 +430,9 @@ public class ApiSurface {
   }
 
   /**
-   * Adds any types exposed from the given array type. The array type itself is not added. The cause
-   * of the exposure of the underlying type is considered whatever type exposed the array type.
+   * Adds any types exposed from the given array type. The array type itself is not added. The
+   * cause of the exposure of the underlying type is considered whatever type exposed the array
+   * type.
    */
   private void addExposedTypes(GenericArrayType type, Class<?> cause) {
     if (done(type)) {
@@ -662,8 +447,9 @@ public class ApiSurface {
   }
 
   /**
-   * Adds any types exposed to this set. Even if the root type is to be pruned, the actual type
-   * arguments are processed.
+   * Adds any types exposed to this set. Even if the
+   * root type is to be pruned, the actual type arguments
+   * are processed.
    */
   private void addExposedTypes(ParameterizedType type, Class<?> cause) {
     // Even if the type is already done, this link to it may be new
@@ -696,8 +482,9 @@ public class ApiSurface {
   }
 
   /**
-   * Adds a class and all of the types it exposes. The cause of the class being exposed is given,
-   * and the cause of everything within the class is that class itself.
+   * Adds a class and all of the types it exposes. The cause
+   * of the class being exposed is given, and the cause
+   * of everything within the class is that class itself.
    */
   private void addExposedTypes(Class<?> clazz, Class<?> cause) {
     if (pruned(clazz)) {
@@ -748,7 +535,7 @@ public class ApiSurface {
           "Adding exposed types from {}, which is an annotation on invokable {}",
           annotation,
           invokable);
-      addExposedTypes(annotation.annotationType(), cause);
+     addExposedTypes(annotation.annotationType(), cause);
     }
     for (Parameter parameter : invokable.getParameters()) {
       LOG.debug(
@@ -790,7 +577,9 @@ public class ApiSurface {
     }
   }
 
-  /** Returns an {@link Invokable} for each public methods or constructors of a type. */
+  /**
+   * Returns an {@link Invokable} for each public methods or constructors of a type.
+   */
   private Set<Invokable> getExposedInvokables(TypeToken<?> type) {
     Set<Invokable> invokables = Sets.newHashSet();
 
@@ -809,11 +598,14 @@ public class ApiSurface {
     return invokables;
   }
 
-  /** Returns true of the given modifier bitmap indicates exposure (public or protected access). */
+  /**
+   * Returns true of the given modifier bitmap indicates exposure (public or protected access).
+   */
   private boolean exposed(int modifiers) {
     return 0 != (modifiers & (Modifier.PUBLIC | Modifier.PROTECTED));
   }
 
+
   ////////////////////////////////////////////////////////////////////////////
 
   /**

http://git-wip-us.apache.org/repos/asf/beam/blob/9248befb/sdks/java/core/src/test/java/org/apache/beam/SdkCoreApiSurfaceTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/SdkCoreApiSurfaceTest.java b/sdks/java/core/src/test/java/org/apache/beam/SdkCoreApiSurfaceTest.java
deleted file mode 100644
index 547d760..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/SdkCoreApiSurfaceTest.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*
- * 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.beam;
-
-import static org.apache.beam.sdk.util.ApiSurface.containsOnlyPackages;
-import static org.hamcrest.MatcherAssert.assertThat;
-
-import com.google.common.collect.ImmutableSet;
-import java.util.Set;
-import org.apache.beam.sdk.util.ApiSurface;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/** API surface verification for {@link org.apache.beam}. */
-@RunWith(JUnit4.class)
-public class SdkCoreApiSurfaceTest {
-
-  @Test
-  public void testSdkApiSurface() throws Exception {
-
-    @SuppressWarnings("unchecked")
-    final Set<String> allowed =
-        ImmutableSet.of(
-            "org.apache.beam",
-            "com.google.api.client",
-            "com.google.api.services.bigquery",
-            "com.google.api.services.cloudresourcemanager",
-            "com.google.api.services.pubsub",
-            "com.google.api.services.storage",
-            "com.google.auth",
-            "com.google.protobuf",
-            "com.fasterxml.jackson.annotation",
-            "com.fasterxml.jackson.core",
-            "com.fasterxml.jackson.databind",
-            "org.apache.avro",
-            "org.hamcrest",
-            // via DataflowMatchers
-            "org.codehaus.jackson",
-            // via Avro
-            "org.joda.time",
-            "org.junit");
-
-    assertThat(ApiSurface.getSdkApiSurface(), containsOnlyPackages(allowed));
-  }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/9248befb/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ApiSurfaceTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ApiSurfaceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ApiSurfaceTest.java
index 9ed6e6c..0f3e2ff 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ApiSurfaceTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ApiSurfaceTest.java
@@ -17,50 +17,138 @@
  */
 package org.apache.beam.sdk.util;
 
-import static org.apache.beam.sdk.util.ApiSurface.containsOnlyClassesMatching;
+import static org.hamcrest.Matchers.anyOf;
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.hamcrest.Matchers.emptyIterable;
 import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
 
-import com.google.common.base.Function;
-import com.google.common.collect.FluentIterable;
+import com.google.common.base.Joiner;
 import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
+import java.util.Collections;
 import java.util.List;
-import javax.annotation.Nonnull;
+import java.util.Map;
+import java.util.Set;
+import org.hamcrest.Description;
 import org.hamcrest.Matcher;
-import org.hamcrest.Matchers;
+import org.hamcrest.TypeSafeDiagnosingMatcher;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
 
-/** Functionality tests for ApiSurface. */
+/**
+ * Tests for ApiSurface. These both test the functionality and also that our
+ * public API is conformant to a hard-coded policy.
+ */
 @RunWith(JUnit4.class)
 public class ApiSurfaceTest {
 
+  @Test
+  public void testOurApiSurface() throws Exception {
+    ApiSurface checkedApiSurface = ApiSurface.getSdkApiSurface();
+
+    Map<Class<?>, List<Class<?>>> disallowedClasses = Maps.newHashMap();
+    for (Class<?> clazz : checkedApiSurface.getExposedClasses()) {
+      if (!classIsAllowed(clazz)) {
+        disallowedClasses.put(clazz, checkedApiSurface.getAnyExposurePath(clazz));
+      }
+    }
+
+    List<String> disallowedMessages = Lists.newArrayList();
+    for (Map.Entry<Class<?>, List<Class<?>>> entry : disallowedClasses.entrySet()) {
+      disallowedMessages.add(entry.getKey() + " exposed via:\n\t\t"
+      + Joiner.on("\n\t\t").join(entry.getValue()));
+    }
+    Collections.sort(disallowedMessages);
+
+    if (!disallowedMessages.isEmpty()) {
+      fail("The following disallowed classes appear in the public API surface of the SDK:\n\t"
+        + Joiner.on("\n\t").join(disallowedMessages));
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  private static final Set<Matcher<Class<?>>> ALLOWED_PACKAGES =
+      ImmutableSet.of(
+          inPackage("org.apache.beam"),
+          inPackage("com.google.api.client"),
+          inPackage("com.google.api.services.bigquery"),
+          inPackage("com.google.api.services.cloudresourcemanager"),
+          inPackage("com.google.api.services.dataflow"),
+          inPackage("com.google.api.services.pubsub"),
+          inPackage("com.google.api.services.storage"),
+          inPackage("com.google.auth"),
+          inPackage("com.google.bigtable.v1"),
+          inPackage("com.google.cloud.bigtable.config"),
+          inPackage("com.google.cloud.bigtable.grpc"),
+          inPackage("com.google.datastore"),
+          inPackage("com.google.protobuf"),
+          inPackage("com.google.rpc"),
+          inPackage("com.google.type"),
+          inPackage("com.fasterxml.jackson.annotation"),
+          inPackage("com.fasterxml.jackson.core"),
+          inPackage("com.fasterxml.jackson.databind"),
+          inPackage("com.fasterxml.jackson.deser"),
+          inPackage("io.grpc"),
+          inPackage("org.apache.avro"),
+          inPackage("org.apache.commons.logging"), // via BigTable
+          inPackage("org.hamcrest"), // via DataflowMatchers
+          inPackage("org.codehaus.jackson"), // via Avro
+          inPackage("org.joda.time"),
+          inPackage("org.junit"),
+          inPackage("java"));
+
   @SuppressWarnings({"rawtypes", "unchecked"})
-  private void assertExposed(final Class classToExamine, final Class... exposedClasses) {
+  private boolean classIsAllowed(Class<?> clazz) {
+    // Safe cast inexpressible in Java without rawtypes
+    return anyOf((Iterable) ALLOWED_PACKAGES).matches(clazz);
+  }
+
+  private static Matcher<Class<?>> inPackage(String packageName) {
+    return new ClassInPackage(packageName);
+  }
+
+  private static class ClassInPackage extends TypeSafeDiagnosingMatcher<Class<?>> {
+
+    private final String packageName;
 
-    final ApiSurface apiSurface = ApiSurface.ofClass(classToExamine).pruningPrefix("java");
+    public ClassInPackage(String packageName) {
+      this.packageName = packageName;
+    }
 
-    final ImmutableSet<Matcher<Class<?>>> allowed =
-        FluentIterable.from(
-                Iterables.concat(Sets.newHashSet(classToExamine), Sets.newHashSet(exposedClasses)))
-            .transform(
-                new Function<Class, Matcher<Class<?>>>() {
+    @Override
+    public void describeTo(Description description) {
+      description.appendText("Class in package \"");
+      description.appendText(packageName);
+      description.appendText("\"");
+    }
 
-                  @Override
-                  public Matcher<Class<?>> apply(@Nonnull final Class input) {
-                    return Matchers.<Class<?>>equalTo(input);
-                  }
-                })
-            .toSet();
+    @Override
+    protected boolean matchesSafely(Class<?> clazz, Description mismatchDescription) {
+      return clazz.getName().startsWith(packageName + ".");
+    }
 
-    assertThat(apiSurface, containsOnlyClassesMatching(allowed));
   }
 
-  private interface Exposed {}
+  //////////////////////////////////////////////////////////////////////////////////
+
+  @SuppressWarnings({"rawtypes", "unchecked"})
+  private void assertExposed(Class classToExamine, Class... exposedClasses) {
+    ApiSurface apiSurface = ApiSurface
+        .ofClass(classToExamine)
+        .pruningPrefix("java");
+
+    Set<Class> expectedExposed = Sets.newHashSet(classToExamine);
+    for (Class clazz : exposedClasses) {
+      expectedExposed.add(clazz);
+    }
+    assertThat(apiSurface.getExposedClasses(), containsInAnyOrder(expectedExposed.toArray()));
+  }
+
+  private interface Exposed { }
 
   private interface ExposedReturnType {
     Exposed zero();
@@ -89,7 +177,7 @@ public class ApiSurfaceTest {
     assertExposed(ExposedWildcardBound.class, Exposed.class);
   }
 
-  private interface ExposedActualTypeArgument extends List<Exposed> {}
+  private interface ExposedActualTypeArgument extends List<Exposed> { }
 
   @Test
   public void testExposedActualTypeArgument() throws Exception {
@@ -98,27 +186,25 @@ public class ApiSurfaceTest {
 
   @Test
   public void testIgnoreAll() throws Exception {
-    ApiSurface apiSurface =
-        ApiSurface.ofClass(ExposedWildcardBound.class)
-            .includingClass(Object.class)
-            .includingClass(ApiSurface.class)
-            .pruningPattern(".*");
+    ApiSurface apiSurface = ApiSurface.ofClass(ExposedWildcardBound.class)
+        .includingClass(Object.class)
+        .includingClass(ApiSurface.class)
+        .pruningPattern(".*");
     assertThat(apiSurface.getExposedClasses(), emptyIterable());
   }
 
-  private interface PrunedPattern {}
-
-  private interface NotPruned extends PrunedPattern {}
+  private interface PrunedPattern { }
+  private interface NotPruned extends PrunedPattern { }
 
   @Test
   public void testprunedPattern() throws Exception {
-    ApiSurface apiSurface = ApiSurface.ofClass(NotPruned.class).pruningClass(PrunedPattern.class);
+    ApiSurface apiSurface = ApiSurface.ofClass(NotPruned.class)
+        .pruningClass(PrunedPattern.class);
     assertThat(apiSurface.getExposedClasses(), containsInAnyOrder((Class) NotPruned.class));
   }
 
   private interface ExposedTwice {
     Exposed zero();
-
     Exposed one();
   }
 

http://git-wip-us.apache.org/repos/asf/beam/blob/9248befb/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/ApiSurfaceTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/ApiSurfaceTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/ApiSurfaceTest.java
new file mode 100644
index 0000000..0abf01d
--- /dev/null
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/ApiSurfaceTest.java
@@ -0,0 +1,134 @@
+/*
+ * 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.beam.sdk.io.gcp;
+
+import static org.hamcrest.Matchers.anyOf;
+import static org.hamcrest.Matchers.equalTo;
+import static org.junit.Assert.fail;
+
+import com.google.cloud.bigtable.grpc.BigtableInstanceName;
+import com.google.cloud.bigtable.grpc.BigtableTableName;
+import com.google.common.base.Joiner;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.beam.sdk.util.ApiSurface;
+import org.hamcrest.Description;
+import org.hamcrest.Matcher;
+import org.hamcrest.TypeSafeDiagnosingMatcher;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests for the API surface of the gcp-io module. Tests that our public API is conformant to a
+ * hard-coded policy.
+ */
+@RunWith(JUnit4.class)
+public class ApiSurfaceTest {
+
+  @Test
+  public void testOurApiSurface() throws Exception {
+    ApiSurface checkedApiSurface =
+        ApiSurface.ofPackage("org.apache.beam.sdk.io.gcp")
+            .pruningPattern("org[.]apache[.]beam[.].*Test.*")
+            .pruningPattern("org[.]apache[.]beam[.].*IT")
+            .pruningPattern("java[.]lang.*");
+
+    Map<Class<?>, List<Class<?>>> disallowedClasses = Maps.newHashMap();
+    for (Class<?> clazz : checkedApiSurface.getExposedClasses()) {
+      if (!classIsAllowed(clazz)) {
+        disallowedClasses.put(clazz, checkedApiSurface.getAnyExposurePath(clazz));
+      }
+    }
+
+    List<String> disallowedMessages = Lists.newArrayList();
+    for (Map.Entry<Class<?>, List<Class<?>>> entry : disallowedClasses.entrySet()) {
+      disallowedMessages.add(entry.getKey() + " exposed via:\n\t\t"
+      + Joiner.on("\n\t\t").join(entry.getValue()));
+    }
+    Collections.sort(disallowedMessages);
+
+    if (!disallowedMessages.isEmpty()) {
+      fail("The following disallowed classes appear in the public API surface of the SDK:\n\t"
+        + Joiner.on("\n\t").join(disallowedMessages));
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  private static final Set<Matcher<? extends Class<?>>> ALLOWED_PACKAGES =
+      ImmutableSet.<Matcher<? extends Class<?>>>of(
+          inPackage("com.google.api.client.json"),
+          inPackage("com.google.api.client.util"),
+          inPackage("com.google.api.services.bigquery.model"),
+          inPackage("com.google.auth"),
+          inPackage("com.google.bigtable.v2"),
+          inPackage("com.google.cloud.bigtable.config"),
+          equalTo(BigtableInstanceName.class),
+          equalTo(BigtableTableName.class),
+          // https://github.com/GoogleCloudPlatform/cloud-bigtable-client/pull/1056
+          inPackage("com.google.common.collect"), // via Bigtable, PR above out to fix.
+          inPackage("com.google.datastore.v1"),
+          inPackage("com.google.protobuf"),
+          inPackage("com.google.type"),
+          inPackage("com.fasterxml.jackson.annotation"),
+          inPackage("com.fasterxml.jackson.core"),
+          inPackage("com.fasterxml.jackson.databind"),
+          inPackage("io.grpc"),
+          inPackage("java"),
+          inPackage("javax"),
+          inPackage("org.apache.beam"),
+          inPackage("org.apache.commons.logging"), // via Bigtable
+          inPackage("org.joda.time"));
+
+  @SuppressWarnings({"rawtypes", "unchecked"})
+  private boolean classIsAllowed(Class<?> clazz) {
+    // Safe cast inexpressible in Java without rawtypes
+    return anyOf((Iterable) ALLOWED_PACKAGES).matches(clazz);
+  }
+
+  private static Matcher<Class<?>> inPackage(String packageName) {
+    return new ClassInPackage(packageName);
+  }
+
+  private static class ClassInPackage extends TypeSafeDiagnosingMatcher<Class<?>> {
+
+    private final String packageName;
+
+    public ClassInPackage(String packageName) {
+      this.packageName = packageName;
+    }
+
+    @Override
+    public void describeTo(Description description) {
+      description.appendText("Class in package \"");
+      description.appendText(packageName);
+      description.appendText("\"");
+    }
+
+    @Override
+    protected boolean matchesSafely(Class<?> clazz, Description mismatchDescription) {
+      return clazz.getName().startsWith(packageName + ".");
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/9248befb/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/GcpApiSurfaceTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/GcpApiSurfaceTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/GcpApiSurfaceTest.java
deleted file mode 100644
index 542fd53..0000000
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/GcpApiSurfaceTest.java
+++ /dev/null
@@ -1,76 +0,0 @@
-/*
- * 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.beam.sdk.io.gcp;
-
-import static org.apache.beam.sdk.util.ApiSurface.classesInPackage;
-import static org.apache.beam.sdk.util.ApiSurface.containsOnlyClassesMatching;
-import static org.hamcrest.MatcherAssert.assertThat;
-
-import com.google.common.collect.ImmutableSet;
-import java.util.Set;
-import org.apache.beam.sdk.util.ApiSurface;
-import org.hamcrest.Matcher;
-import org.hamcrest.Matchers;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-/** API surface verification for {@link org.apache.beam.sdk.io.gcp}. */
-@RunWith(JUnit4.class)
-public class GcpApiSurfaceTest {
-
-  @Test
-  public void testGcpApiSurface() throws Exception {
-
-    final ApiSurface apiSurface =
-        ApiSurface.ofPackage(getClass().getPackage())
-            .pruningPattern("org[.]apache[.]beam[.].*Test.*")
-            .pruningPattern("org[.]apache[.]beam[.].*IT")
-            .pruningPattern("java[.]lang.*");
-
-    @SuppressWarnings("unchecked")
-    final Set<Matcher<Class<?>>> allowedClasses =
-        ImmutableSet.of(
-            classesInPackage("com.google.api.client.json"),
-            classesInPackage("com.google.api.client.util"),
-            classesInPackage("com.google.api.services.bigquery.model"),
-            classesInPackage("com.google.auth"),
-            classesInPackage("com.google.bigtable.v2"),
-            classesInPackage("com.google.cloud.bigtable.config"),
-            Matchers.<Class<?>>equalTo(com.google.cloud.bigtable.grpc.BigtableInstanceName.class),
-            Matchers.<Class<?>>equalTo(com.google.cloud.bigtable.grpc.BigtableTableName.class),
-            // https://github.com/GoogleCloudPlatform/cloud-bigtable-client/pull/1056
-            classesInPackage("com.google.common.collect"),
-            // via Bigtable, PR above out to fix.
-            classesInPackage("com.google.datastore.v1"),
-            classesInPackage("com.google.protobuf"),
-            classesInPackage("com.google.type"),
-            classesInPackage("com.fasterxml.jackson.annotation"),
-            classesInPackage("com.fasterxml.jackson.core"),
-            classesInPackage("com.fasterxml.jackson.databind"),
-            classesInPackage("io.grpc"),
-            classesInPackage("java"),
-            classesInPackage("javax"),
-            classesInPackage("org.apache.beam"),
-            classesInPackage("org.apache.commons.logging"),
-            // via Bigtable
-            classesInPackage("org.joda.time"));
-
-    assertThat(apiSurface, containsOnlyClassesMatching(allowedClasses));
-  }
-}


[2/2] beam git commit: This closes #1822: Revert "Simplified API surface verifications"

Posted by ke...@apache.org.
This closes #1822: Revert "Simplified API surface verifications"


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/cb6e0a80
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/cb6e0a80
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/cb6e0a80

Branch: refs/heads/master
Commit: cb6e0a80c57b056489d447cde092cffdd041eed5
Parents: 6ecbfb9 9248bef
Author: Kenneth Knowles <kl...@google.com>
Authored: Mon Jan 23 19:47:47 2017 -0800
Committer: Kenneth Knowles <kl...@google.com>
Committed: Mon Jan 23 19:47:47 2017 -0800

----------------------------------------------------------------------
 .../org/apache/beam/sdk/util/ApiSurface.java    | 420 +++++--------------
 .../org/apache/beam/SdkCoreApiSurfaceTest.java  |  61 ---
 .../apache/beam/sdk/util/ApiSurfaceTest.java    | 152 +++++--
 .../apache/beam/sdk/io/gcp/ApiSurfaceTest.java  | 134 ++++++
 .../beam/sdk/io/gcp/GcpApiSurfaceTest.java      |  76 ----
 5 files changed, 359 insertions(+), 484 deletions(-)
----------------------------------------------------------------------