You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by lc...@apache.org on 2016/07/06 17:20:07 UTC

[01/50] [abbrv] incubator-beam git commit: Remove many definitions of named methods

Repository: incubator-beam
Updated Branches:
  refs/heads/runners-spark2 f57e66c48 -> 19dd6ccef


Remove many definitions of named methods

Specifically, remove the occurrences in:
  - Window
  - AvroIO
  - PubsubIO
  - TextIO
  - BigQueryIO
  - Read


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

Branch: refs/heads/runners-spark2
Commit: 37f4ad497c5f669aba7efaeb4aaf6f55e1b01fc4
Parents: 0fd0bd5
Author: Ben Chambers <bc...@google.com>
Authored: Thu Jun 23 17:55:24 2016 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Jul 6 10:18:49 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/beam/sdk/io/AvroIO.java     | 53 ++++---------------
 .../java/org/apache/beam/sdk/io/BigQueryIO.java | 42 +--------------
 .../java/org/apache/beam/sdk/io/PubsubIO.java   | 35 +------------
 .../main/java/org/apache/beam/sdk/io/Read.java  | 29 +----------
 .../java/org/apache/beam/sdk/io/TextIO.java     | 55 ++++----------------
 .../org/apache/beam/sdk/io/package-info.java    |  6 +--
 .../beam/sdk/transforms/windowing/Window.java   | 42 ---------------
 7 files changed, 25 insertions(+), 237 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/37f4ad49/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java
index 4b40c01..604051b 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java
@@ -55,9 +55,7 @@ import javax.annotation.Nullable;
  * {@link AvroIO.Read}, specifying {@link AvroIO.Read#from} to specify
  * the path of the file(s) to read from (e.g., a local filename or
  * filename pattern if running locally, or a Google Cloud Storage
- * filename or filename pattern of the form
- * {@code "gs://<bucket>/<filepath>"}), and optionally
- * {@link AvroIO.Read#named} to specify the name of the pipeline step.
+ * filename or filename pattern of the form {@code "gs://<bucket>/<filepath>"}).
  *
  * <p>It is required to specify {@link AvroIO.Read#withSchema}. To
  * read specific records, such as Avro-generated classes, provide an
@@ -73,15 +71,15 @@ import javax.annotation.Nullable;
  * // A simple Read of a local file (only runs locally):
  * PCollection<AvroAutoGenClass> records =
  *     p.apply(AvroIO.Read.from("/path/to/file.avro")
- *                        .withSchema(AvroAutoGenClass.class));
+ *                 .withSchema(AvroAutoGenClass.class));
  *
  * // A Read from a GCS file (runs locally and via the Google Cloud
  * // Dataflow service):
  * Schema schema = new Schema.Parser().parse(new File("schema.avsc"));
  * PCollection<GenericRecord> records =
- *     p.apply(AvroIO.Read.named("ReadFromAvro")
- *                        .from("gs://my_bucket/path/to/records-*.avro")
- *                        .withSchema(schema));
+ *     p.apply(AvroIO.Read
+ *                .from("gs://my_bucket/path/to/records-*.avro")
+ *                .withSchema(schema));
  * } </pre>
  *
  * <p>To write a {@link PCollection} to one or more Avro files, use
@@ -110,10 +108,10 @@ import javax.annotation.Nullable;
  * // Dataflow service):
  * Schema schema = new Schema.Parser().parse(new File("schema.avsc"));
  * PCollection<GenericRecord> records = ...;
- * records.apply(AvroIO.Write.named("WriteToAvro")
- *                           .to("gs://my_bucket/path/to/numbers")
- *                           .withSchema(schema)
- *                           .withSuffix(".avro"));
+ * records.apply("WriteToAvro", AvroIO.Write
+ *     .to("gs://my_bucket/path/to/numbers")
+ *     .withSchema(schema)
+ *     .withSuffix(".avro"));
  * } </pre>
  *
  * <p><h3>Permissions</h3>
@@ -128,12 +126,6 @@ public class AvroIO {
    * the decoding of each record.
    */
   public static class Read {
-    /**
-     * Returns a {@link PTransform} with the given step name.
-     */
-    public static Bound<GenericRecord> named(String name) {
-      return new Bound<>(GenericRecord.class).named(name);
-    }
 
     /**
      * Returns a {@link PTransform} that reads from the file(s)
@@ -223,16 +215,6 @@ public class AvroIO {
 
       /**
        * Returns a new {@link PTransform} that's like this one but
-       * with the given step name.
-       *
-       * <p>Does not modify this object.
-       */
-      public Bound<T> named(String name) {
-        return new Bound<>(name, filepattern, type, schema, validate);
-      }
-
-      /**
-       * Returns a new {@link PTransform} that's like this one but
        * that reads from the file(s) with the given name or pattern.
        * (See {@link AvroIO.Read#from} for a description of
        * filepatterns.)
@@ -366,12 +348,6 @@ public class AvroIO {
    * multiple Avro files matching a sharding pattern).
    */
   public static class Write {
-    /**
-     * Returns a {@link PTransform} with the given step name.
-     */
-    public static Bound<GenericRecord> named(String name) {
-      return new Bound<>(GenericRecord.class).named(name);
-    }
 
     /**
      * Returns a {@link PTransform} that writes to the file(s)
@@ -522,17 +498,6 @@ public class AvroIO {
 
       /**
        * Returns a new {@link PTransform} that's like this one but
-       * with the given step name.
-       *
-       * <p>Does not modify this object.
-       */
-      public Bound<T> named(String name) {
-        return new Bound<>(
-            name, filenamePrefix, filenameSuffix, numShards, shardTemplate, type, schema, validate);
-      }
-
-      /**
-       * Returns a new {@link PTransform} that's like this one but
        * that writes to the file(s) with the given filename prefix.
        *
        * <p>See {@link AvroIO.Write#to(String)} for more information

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/37f4ad49/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BigQueryIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BigQueryIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BigQueryIO.java
index 7cac705..a9d85b8 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BigQueryIO.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BigQueryIO.java
@@ -162,8 +162,7 @@ import javax.annotation.Nullable;
  * This produces a {@link PCollection} of {@link TableRow TableRows} as output:
  * <pre>{@code
  * PCollection<TableRow> shakespeare = pipeline.apply(
- *     BigQueryIO.Read.named("Read")
- *                    .from("clouddataflow-readonly:samples.weather_stations"));
+ *     BigQueryIO.Read.from("clouddataflow-readonly:samples.weather_stations"));
  * }</pre>
  *
  * <p>See {@link TableRow} for more information on the {@link TableRow} object.
@@ -174,8 +173,7 @@ import javax.annotation.Nullable;
  *
  * <pre>{@code
  * PCollection<TableRow> shakespeare = pipeline.apply(
- *     BigQueryIO.Read.named("Read")
- *                    .fromQuery("SELECT year, mean_temp FROM samples.weather_stations"));
+ *     BigQueryIO.Read.fromQuery("SELECT year, mean_temp FROM samples.weather_stations"));
  * }</pre>
  *
  * <p>When creating a BigQuery input transform, users should provide either a query or a table.
@@ -193,7 +191,6 @@ import javax.annotation.Nullable;
  * TableSchema schema = new TableSchema().setFields(fields);
  *
  * quotes.apply(BigQueryIO.Write
- *     .named("Write")
  *     .to("my-project:output.output_table")
  *     .withSchema(schema)
  *     .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_TRUNCATE));
@@ -214,7 +211,6 @@ import javax.annotation.Nullable;
  * PCollection<TableRow> quotes = ...
  * quotes.apply(Window.<TableRow>into(CalendarWindows.days(1)))
  *       .apply(BigQueryIO.Write
- *         .named("Write")
  *         .withSchema(schema)
  *         .to(new SerializableFunction<BoundedWindow, String>() {
  *           public String apply(BoundedWindow window) {
@@ -345,13 +341,6 @@ public class BigQueryIO {
    * }}</pre>
    */
   public static class Read {
-    /**
-     * Returns a {@link Read.Bound} with the given name. The BigQuery table or query to be read
-     * from has not yet been configured.
-     */
-    public static Bound named(String name) {
-      return new Bound().named(name);
-    }
 
     /**
      * Reads a BigQuery table specified as {@code "[project_id]:[dataset_id].[table_id]"} or
@@ -429,15 +418,6 @@ public class BigQueryIO {
       }
 
       /**
-       * Returns a copy of this transform using the name associated with this transformation.
-       *
-       * <p>Does not modify this object.
-       */
-      public Bound named(String name) {
-        return new Bound(name, query, jsonTableRef, validate, flattenResults, testBigQueryServices);
-      }
-
-      /**
        * Returns a copy of this transform that reads from the specified table. Refer to
        * {@link #parseTableSpec(String)} for the specification format.
        *
@@ -1372,14 +1352,6 @@ public class BigQueryIO {
     }
 
     /**
-     * Creates a write transformation with the given transform name. The BigQuery table to be
-     * written has not yet been configured.
-     */
-    public static Bound named(String name) {
-      return new Bound().named(name);
-    }
-
-    /**
      * Creates a write transformation for the given table specification.
      *
      * <p>Refer to {@link #parseTableSpec(String)} for the specification format.
@@ -1522,16 +1494,6 @@ public class BigQueryIO {
       }
 
       /**
-       * Returns a copy of this write transformation, but with the specified transform name.
-       *
-       * <p>Does not modify this object.
-       */
-      public Bound named(String name) {
-        return new Bound(name, jsonTableRef, tableRefFunction, jsonSchema, createDisposition,
-            writeDisposition, validate, testBigQueryServices);
-      }
-
-      /**
        * Returns a copy of this write transformation, but writing to the specified table. Refer to
        * {@link #parseTableSpec(String)} for the specification format.
        *

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/37f4ad49/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java
index c6de8b4..ecb1f0a 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java
@@ -55,6 +55,7 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
+
 import javax.annotation.Nullable;
 
 /**
@@ -369,13 +370,6 @@ public class PubsubIO {
    * {@link Bound#maxNumRecords(int)} or {@link Bound#maxReadTime(Duration)} must be set.
    */
   public static class Read {
-    /**
-     * Creates and returns a transform for reading from Cloud Pub/Sub with the specified transform
-     * name.
-     */
-    public static Bound<String> named(String name) {
-      return new Bound<>(DEFAULT_PUBSUB_CODER).named(name);
-    }
 
     /**
      * Creates and returns a transform for reading from a Cloud Pub/Sub topic. Mutually exclusive
@@ -531,16 +525,6 @@ public class PubsubIO {
       }
 
       /**
-       * Returns a transform that's like this one but with the given step name.
-       *
-       * <p>Does not modify this object.
-       */
-      public Bound<T> named(String name) {
-        return new Bound<>(
-            name, subscription, topic, timestampLabel, coder, idLabel, maxNumRecords, maxReadTime);
-      }
-
-      /**
        * Returns a transform that's like this one but reading from the
        * given subscription.
        *
@@ -834,13 +818,6 @@ public class PubsubIO {
   // TODO: Support non-String encodings.
   public static class Write {
     /**
-     * Creates a transform that writes to Pub/Sub with the given step name.
-     */
-    public static Bound<String> named(String name) {
-      return new Bound<>(DEFAULT_PUBSUB_CODER).named(name);
-    }
-
-    /**
      * Creates a transform that publishes to the specified topic.
      *
      * <p>See {@link PubsubIO.PubsubTopic#fromPath(String)} for more details on the format of the
@@ -917,16 +894,6 @@ public class PubsubIO {
       }
 
       /**
-       * Returns a new transform that's like this one but with the specified step
-       * name.
-       *
-       * <p>Does not modify this object.
-       */
-      public Bound<T> named(String name) {
-        return new Bound<>(name, topic, timestampLabel, idLabel, coder);
-      }
-
-      /**
        * Returns a new transform that's like this one but that writes to the specified
        * topic.
        *

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/37f4ad49/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java
index c0440f2..e13ff06 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java
@@ -38,17 +38,10 @@ import javax.annotation.Nullable;
  * <p>Usage example:
  * <pre>
  * Pipeline p = Pipeline.create();
- * p.apply(Read.from(new MySource().withFoo("foo").withBar("bar"))
- *             .named("foobar"));
+ * p.apply(Read.from(new MySource().withFoo("foo").withBar("bar")));
  * </pre>
  */
 public class Read {
-  /**
-   * Returns a new {@code Read} {@code PTransform} builder with the given name.
-   */
-  public static Builder named(String name) {
-    return new Builder(name);
-  }
 
   /**
    * Returns a new {@code Read.Bounded} {@code PTransform} reading from the given
@@ -104,16 +97,6 @@ public class Read {
       this.source = SerializableUtils.ensureSerializable(source);
     }
 
-    /**
-     * Returns a new {@code Bounded} {@code PTransform} that's like this one but
-     * has the given name.
-     *
-     * <p>Does not modify this object.
-     */
-    public Bounded<T> named(String name) {
-      return new Bounded<T>(name, source);
-    }
-
     @Override
     protected Coder<T> getDefaultOutputCoder() {
       return source.getDefaultOutputCoder();
@@ -162,16 +145,6 @@ public class Read {
     }
 
     /**
-     * Returns a new {@code Unbounded} {@code PTransform} that's like this one but
-     * has the given name.
-     *
-     * <p>Does not modify this object.
-     */
-    public Unbounded<T> named(String name) {
-      return new Unbounded<T>(name, source);
-    }
-
-    /**
      * Returns a new {@link BoundedReadFromUnboundedSource} that reads a bounded amount
      * of data from the given {@link UnboundedSource}.  The bound is specified as a number
      * of records to read.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/37f4ad49/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java
index a7e5e29..7e7a3e6 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java
@@ -58,8 +58,7 @@ import javax.annotation.Nullable;
  * the path of the file(s) to read from (e.g., a local filename or
  * filename pattern if running locally, or a Google Cloud Storage
  * filename or filename pattern of the form
- * {@code "gs://<bucket>/<filepath>"}). You may optionally call
- * {@link TextIO.Read#named(String)} to specify the name of the pipeline step.
+ * {@code "gs://<bucket>/<filepath>"}).
  *
  * <p>By default, {@link TextIO.Read} returns a {@link PCollection} of {@link String Strings},
  * each corresponding to one line of an input UTF-8 text file. To convert directly from the raw
@@ -78,9 +77,9 @@ import javax.annotation.Nullable;
  * // A fully-specified Read from a GCS file (runs locally and via the
  * // Google Cloud Dataflow service):
  * PCollection<Integer> numbers =
- *     p.apply(TextIO.Read.named("ReadNumbers")
- *                        .from("gs://my_bucket/path/to/numbers-*.txt")
- *                        .withCoder(TextualIntegerCoder.of()));
+ *     p.apply("ReadNumbers", TextIO.Read
+ *         .from("gs://my_bucket/path/to/numbers-*.txt")
+ *         .withCoder(TextualIntegerCoder.of()));
  * }</pre>
  *
  * <p>To write a {@link PCollection} to one or more text files, use
@@ -88,9 +87,8 @@ import javax.annotation.Nullable;
  * the path of the file to write to (e.g., a local filename or sharded
  * filename pattern if running locally, or a Google Cloud Storage
  * filename or sharded filename pattern of the form
- * {@code "gs://<bucket>/<filepath>"}). You can optionally name the resulting transform using
- * {@link TextIO.Write#named(String)}, and you can use {@link TextIO.Write#withCoder(Coder)}
- * to specify the Coder to use to encode the Java values into text lines.
+ * {@code "gs://<bucket>/<filepath>"}). You can use {@link TextIO.Write#withCoder(Coder)}
+ * to specify the {@link Coder} to use to encode the Java values into text lines.
  *
  * <p>Any existing files with the same names as generated output files
  * will be overwritten.
@@ -104,10 +102,10 @@ import javax.annotation.Nullable;
  * // A fully-specified Write to a sharded GCS file (runs locally and via the
  * // Google Cloud Dataflow service):
  * PCollection<Integer> numbers = ...;
- * numbers.apply(TextIO.Write.named("WriteNumbers")
- *                           .to("gs://my_bucket/path/to/numbers")
- *                           .withSuffix(".txt")
- *                           .withCoder(TextualIntegerCoder.of()));
+ * numbers.apply("WriteNumbers", TextIO.Write
+ *      .to("gs://my_bucket/path/to/numbers")
+ *      .withSuffix(".txt")
+ *      .withCoder(TextualIntegerCoder.of()));
  * }</pre>
  *
  * <h3>Permissions</h3>
@@ -130,12 +128,6 @@ public class TextIO {
    * {@link #withCoder(Coder)} to change the return type.
    */
   public static class Read {
-    /**
-     * Returns a transform for reading text files that uses the given step name.
-     */
-    public static Bound<String> named(String name) {
-      return new Bound<>(DEFAULT_TEXT_CODER).named(name);
-    }
 
     /**
      * Returns a transform for reading text files that reads from the file(s)
@@ -228,16 +220,6 @@ public class TextIO {
 
       /**
        * Returns a new transform for reading from text files that's like this one but
-       * with the given step name.
-       *
-       * <p>Does not modify this object.
-       */
-      public Bound<T> named(String name) {
-        return new Bound<>(name, filepattern, coder, validate, compressionType);
-      }
-
-      /**
-       * Returns a new transform for reading from text files that's like this one but
        * that reads from the file(s) with the given name or pattern. See {@link TextIO.Read#from}
        * for a description of filepatterns.
        *
@@ -387,12 +369,6 @@ public class TextIO {
    * element of the input collection encoded into its own line.
    */
   public static class Write {
-    /**
-     * Returns a transform for writing to text files with the given step name.
-     */
-    public static Bound<String> named(String name) {
-      return new Bound<>(DEFAULT_TEXT_CODER).named(name);
-    }
 
     /**
      * Returns a transform for writing to text files that writes to the file(s)
@@ -521,17 +497,6 @@ public class TextIO {
 
       /**
        * Returns a transform for writing to text files that's like this one but
-       * with the given step name.
-       *
-       * <p>Does not modify this object.
-       */
-      public Bound<T> named(String name) {
-        return new Bound<>(name, filenamePrefix, filenameSuffix, coder, numShards,
-            shardTemplate, validate);
-      }
-
-      /**
-       * Returns a transform for writing to text files that's like this one but
        * that writes to the file(s) with the given filename prefix.
        *
        * <p>See {@link TextIO.Write#to(String) Write.to(String)} for more information.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/37f4ad49/sdks/java/core/src/main/java/org/apache/beam/sdk/io/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/package-info.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/package-info.java
index c2c0685..432c5df 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/package-info.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/package-info.java
@@ -25,14 +25,12 @@
  * from existing storage:
  * <pre>{@code
  * PCollection<TableRow> inputData = pipeline.apply(
- *     BigQueryIO.Read.named("Read")
- *                    .from("clouddataflow-readonly:samples.weather_stations");
+ *     BigQueryIO.Read.from("clouddataflow-readonly:samples.weather_stations"));
  * }</pre>
  * and {@code Write} transforms that persist PCollections to external storage:
  * <pre> {@code
  * PCollection<Integer> numbers = ...;
- * numbers.apply(TextIO.Write.named("WriteNumbers")
- *                           .to("gs://my_bucket/path/to/numbers"));
+ * numbers.apply(TextIO.Write.to("gs://my_bucket/path/to/numbers"));
  * } </pre>
  */
 package org.apache.beam.sdk.io;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/37f4ad49/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java
index dde5c05..bc122e2 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java
@@ -159,21 +159,6 @@ public class Window {
   }
 
   /**
-   * Creates a {@code Window} {@code PTransform} with the given name.
-   *
-   * <p>See the discussion of Naming in
-   * {@link org.apache.beam.sdk.transforms.ParDo} for more explanation.
-   *
-   * <p>The resulting {@code PTransform} is incomplete, and its input/output
-   * type is not yet bound.  Use {@link Window.Unbound#into} to specify the
-   * {@link WindowFn} to use, which will also bind the input/output type of this
-   * {@code PTransform}.
-   */
-  public static Unbound named(String name) {
-    return new Unbound().named(name);
-  }
-
-  /**
    * Creates a {@code Window} {@code PTransform} that uses the given
    * {@link WindowFn} to window the data.
    *
@@ -255,19 +240,6 @@ public class Window {
     }
 
     /**
-     * Returns a new {@code Window} transform that's like this
-     * transform but with the specified name.  Does not modify this
-     * transform.  The resulting transform is still incomplete.
-     *
-     * <p>See the discussion of Naming in
-     * {@link org.apache.beam.sdk.transforms.ParDo} for more
-     * explanation.
-     */
-    public Unbound named(String name) {
-      return new Unbound(name);
-    }
-
-    /**
      * Returns a new {@code Window} {@code PTransform} that's like this
      * transform but that will use the given {@link WindowFn}, and that has
      * its input and output types bound.  Does not modify this transform.  The
@@ -408,20 +380,6 @@ public class Window {
     }
 
     /**
-     * Returns a new {@code Window} {@code PTransform} that's like this
-     * {@code PTransform} but with the specified name.  Does not
-     * modify this {@code PTransform}.
-     *
-     * <p>See the discussion of Naming in
-     * {@link org.apache.beam.sdk.transforms.ParDo} for more
-     * explanation.
-     */
-    public Bound<T> named(String name) {
-      return new Bound<>(
-          name, windowFn, trigger, mode, allowedLateness, closingBehavior, outputTimeFn);
-    }
-
-    /**
      * Sets a non-default trigger for this {@code Window} {@code PTransform}.
      * Elements that are assigned to a specific window will be output when
      * the trigger fires.


[19/50] [abbrv] incubator-beam git commit: Static import Preconditions.checkX everywhere

Posted by lc...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/HadoopIO.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/HadoopIO.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/HadoopIO.java
index 00c10d4..1177a57 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/HadoopIO.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/HadoopIO.java
@@ -17,6 +17,9 @@
  */
 package org.apache.beam.runners.spark.io.hadoop;
 
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+
 import org.apache.beam.sdk.io.ShardNameTemplate;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.util.WindowingStrategy;
@@ -25,8 +28,6 @@ import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PDone;
 import org.apache.beam.sdk.values.PInput;
 
-import com.google.common.base.Preconditions;
-
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 
@@ -68,14 +69,10 @@ public final class HadoopIO {
 
       Bound(String filepattern, Class<? extends FileInputFormat<K, V>> format, Class<K> key,
           Class<V> value) {
-        Preconditions.checkNotNull(filepattern,
-                                   "need to set the filepattern of an HadoopIO.Read transform");
-        Preconditions.checkNotNull(format,
-                                   "need to set the format class of an HadoopIO.Read transform");
-        Preconditions.checkNotNull(key,
-                                   "need to set the key class of an HadoopIO.Read transform");
-        Preconditions.checkNotNull(value,
-                                   "need to set the value class of an HadoopIO.Read transform");
+        checkNotNull(filepattern, "need to set the filepattern of an HadoopIO.Read transform");
+        checkNotNull(format, "need to set the format class of an HadoopIO.Read transform");
+        checkNotNull(key, "need to set the key class of an HadoopIO.Read transform");
+        checkNotNull(value, "need to set the value class of an HadoopIO.Read transform");
         this.filepattern = filepattern;
         this.formatClass = format;
         this.keyClass = key;
@@ -203,17 +200,16 @@ public final class HadoopIO {
 
       @Override
       public PDone apply(PCollection<KV<K, V>> input) {
-        Preconditions.checkNotNull(filenamePrefix,
-            "need to set the filename prefix of an HadoopIO.Write transform");
-        Preconditions.checkNotNull(formatClass,
-            "need to set the format class of an HadoopIO.Write transform");
-        Preconditions.checkNotNull(keyClass,
-            "need to set the key class of an HadoopIO.Write transform");
-        Preconditions.checkNotNull(valueClass,
-            "need to set the value class of an HadoopIO.Write transform");
-
-        Preconditions.checkArgument(ShardNameTemplateAware.class.isAssignableFrom(formatClass),
-            "Format class must implement " + ShardNameTemplateAware.class.getName());
+        checkNotNull(
+            filenamePrefix, "need to set the filename prefix of an HadoopIO.Write transform");
+        checkNotNull(formatClass, "need to set the format class of an HadoopIO.Write transform");
+        checkNotNull(keyClass, "need to set the key class of an HadoopIO.Write transform");
+        checkNotNull(valueClass, "need to set the value class of an HadoopIO.Write transform");
+
+        checkArgument(
+            ShardNameTemplateAware.class.isAssignableFrom(formatClass),
+            "Format class must implement %s",
+            ShardNameTemplateAware.class.getName());
 
         return PDone.in(input.getPipeline());
       }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java
index e264bc6..31ae2dc 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java
@@ -17,6 +17,8 @@
  */
 package org.apache.beam.sdk;
 
+import static com.google.common.base.Preconditions.checkState;
+
 import org.apache.beam.sdk.coders.CoderRegistry;
 import org.apache.beam.sdk.io.Read;
 import org.apache.beam.sdk.options.PipelineOptions;
@@ -34,7 +36,6 @@ import org.apache.beam.sdk.values.PInput;
 import org.apache.beam.sdk.values.POutput;
 import org.apache.beam.sdk.values.PValue;
 
-import com.google.common.base.Preconditions;
 import com.google.common.collect.HashMultimap;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Multimap;
@@ -492,8 +493,8 @@ public class Pipeline {
   public String getFullNameForTesting(PTransform<?, ?> transform) {
     Collection<AppliedPTransform<?, ?, ?>> uses =
         transformApplicationsForTesting.get(transform);
-    Preconditions.checkState(uses.size() > 0, "Unknown transform: " + transform);
-    Preconditions.checkState(uses.size() <= 1, "Transform used multiple times: " + transform);
+    checkState(uses.size() > 0, "Unknown transform: " + transform);
+    checkState(uses.size() <= 1, "Transform used multiple times: " + transform);
     return Iterables.getOnlyElement(uses).getFullName();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java
index 71e99f7..d2666f0 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java
@@ -17,6 +17,8 @@
  */
 package org.apache.beam.sdk.coders;
 
+import static com.google.common.base.Preconditions.checkArgument;
+
 import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.annotations.Experimental.Kind;
 import org.apache.beam.sdk.util.CloudObject;
@@ -25,7 +27,6 @@ import org.apache.beam.sdk.util.common.ElementByteSizeObserver;
 import com.google.common.base.Joiner;
 import com.google.common.base.MoreObjects;
 import com.google.common.base.Objects;
-import com.google.common.base.Preconditions;
 
 import java.io.IOException;
 import java.io.InputStream;
@@ -281,8 +282,7 @@ public interface Coder<T> extends Serializable {
         List<String> reasons,
         @Nullable NonDeterministicException cause) {
       super(cause);
-      Preconditions.checkArgument(reasons.size() > 0,
-          "Reasons must not be empty.");
+      checkArgument(reasons.size() > 0, "Reasons must not be empty.");
       this.reasons = reasons;
       this.coder = coder;
     }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java
index 3e4c6d2..9c3fabe 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java
@@ -17,6 +17,8 @@
  */
 package org.apache.beam.sdk.coders;
 
+import static com.google.common.base.Preconditions.checkArgument;
+
 import org.apache.beam.sdk.coders.CannotProvideCoderException.ReasonCode;
 import org.apache.beam.sdk.coders.protobuf.ProtoCoder;
 import org.apache.beam.sdk.transforms.SerializableFunction;
@@ -27,7 +29,6 @@ import org.apache.beam.sdk.values.TypeDescriptor;
 
 import com.google.api.services.bigquery.model.TableRow;
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Maps;
 import com.google.protobuf.ByteString;
@@ -161,7 +162,7 @@ public class CoderRegistry implements CoderProvider {
    * appropriate static methods and register it directly via {@link #registerCoder(Class, Class)}.
    */
   public <T> void registerCoder(Class<T> rawClazz, Coder<T> coder) {
-    Preconditions.checkArgument(
+    checkArgument(
       rawClazz.getTypeParameters().length == 0,
       "CoderRegistry.registerCoder(Class<T>, Coder<T>) may not be used "
       + "with unspecialized generic classes");

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CollectionCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CollectionCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CollectionCoder.java
index 9f997c0..f40d5f0 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CollectionCoder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CollectionCoder.java
@@ -17,9 +17,9 @@
  */
 package org.apache.beam.sdk.coders;
 
-import org.apache.beam.sdk.util.PropertyNames;
+import static com.google.common.base.Preconditions.checkArgument;
 
-import com.google.common.base.Preconditions;
+import org.apache.beam.sdk.util.PropertyNames;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
@@ -55,8 +55,7 @@ public class CollectionCoder<T> extends IterableLikeCoder<T, Collection<T>> {
   public static CollectionCoder<?> of(
       @JsonProperty(PropertyNames.COMPONENT_ENCODINGS)
       List<Object> components) {
-    Preconditions.checkArgument(components.size() == 1,
-        "Expecting 1 component, got " + components.size());
+    checkArgument(components.size() == 1, "Expecting 1 component, got %s", components.size());
     return of((Coder<?>) components.get(0));
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/IterableCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/IterableCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/IterableCoder.java
index 546a240..2e5a8cc 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/IterableCoder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/IterableCoder.java
@@ -19,11 +19,11 @@ package org.apache.beam.sdk.coders;
 
 import static org.apache.beam.sdk.util.Structs.addBoolean;
 
+import static com.google.common.base.Preconditions.checkArgument;
+
 import org.apache.beam.sdk.util.CloudObject;
 import org.apache.beam.sdk.util.PropertyNames;
 
-import com.google.common.base.Preconditions;
-
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 
@@ -53,8 +53,7 @@ public class IterableCoder<T> extends IterableLikeCoder<T, Iterable<T>> {
   public static IterableCoder<?> of(
       @JsonProperty(PropertyNames.COMPONENT_ENCODINGS)
       List<Coder<?>> components) {
-    Preconditions.checkArgument(components.size() == 1,
-        "Expecting 1 component, got " + components.size());
+    checkArgument(components.size() == 1, "Expecting 1 component, got %s", components.size());
     return of(components.get(0));
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/IterableLikeCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/IterableLikeCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/IterableLikeCoder.java
index 74e3dd3..9417d85 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/IterableLikeCoder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/IterableLikeCoder.java
@@ -17,13 +17,13 @@
  */
 package org.apache.beam.sdk.coders;
 
+import static com.google.common.base.Preconditions.checkArgument;
+
 import org.apache.beam.sdk.util.BufferedElementCountingOutputStream;
 import org.apache.beam.sdk.util.VarInt;
 import org.apache.beam.sdk.util.common.ElementByteSizeObservableIterable;
 import org.apache.beam.sdk.util.common.ElementByteSizeObserver;
 
-import com.google.common.base.Preconditions;
-
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
@@ -89,10 +89,8 @@ public abstract class IterableLikeCoder<T, IterableT extends Iterable<T>>
   }
 
   protected IterableLikeCoder(Coder<T> elementCoder, String  iterableName) {
-    Preconditions.checkArgument(elementCoder != null,
-        "element Coder for IterableLikeCoder must not be null");
-    Preconditions.checkArgument(iterableName != null,
-        "iterable name for IterableLikeCoder must not be null");
+    checkArgument(elementCoder != null, "element Coder for IterableLikeCoder must not be null");
+    checkArgument(iterableName != null, "iterable name for IterableLikeCoder must not be null");
     this.elementCoder = elementCoder;
     this.iterableName = iterableName;
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/KvCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/KvCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/KvCoder.java
index 1dd28c9..beeb9c5 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/KvCoder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/KvCoder.java
@@ -19,13 +19,13 @@ package org.apache.beam.sdk.coders;
 
 import static org.apache.beam.sdk.util.Structs.addBoolean;
 
+import static com.google.common.base.Preconditions.checkArgument;
+
 import org.apache.beam.sdk.util.CloudObject;
 import org.apache.beam.sdk.util.PropertyNames;
 import org.apache.beam.sdk.util.common.ElementByteSizeObserver;
 import org.apache.beam.sdk.values.KV;
 
-import com.google.common.base.Preconditions;
-
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 
@@ -51,8 +51,7 @@ public class KvCoder<K, V> extends StandardCoder<KV<K, V>> {
   public static KvCoder<?, ?> of(
       @JsonProperty(PropertyNames.COMPONENT_ENCODINGS)
       List<Coder<?>> components) {
-    Preconditions.checkArgument(components.size() == 2,
-        "Expecting 2 components, got " + components.size());
+    checkArgument(components.size() == 2, "Expecting 2 components, got %s", components.size());
     return of(components.get(0), components.get(1));
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ListCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ListCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ListCoder.java
index c45559a..f8437a4 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ListCoder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ListCoder.java
@@ -17,9 +17,9 @@
  */
 package org.apache.beam.sdk.coders;
 
-import org.apache.beam.sdk.util.PropertyNames;
+import static com.google.common.base.Preconditions.checkArgument;
 
-import com.google.common.base.Preconditions;
+import org.apache.beam.sdk.util.PropertyNames;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
@@ -49,8 +49,7 @@ public class ListCoder<T> extends IterableLikeCoder<T, List<T>> {
   public static ListCoder<?> of(
       @JsonProperty(PropertyNames.COMPONENT_ENCODINGS)
       List<Coder<?>> components) {
-    Preconditions.checkArgument(components.size() == 1,
-        "Expecting 1 component, got " + components.size());
+    checkArgument(components.size() == 1, "Expecting 1 component, got %s", components.size());
     return of((Coder<?>) components.get(0));
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/MapCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/MapCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/MapCoder.java
index b77a39f..044fff1 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/MapCoder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/MapCoder.java
@@ -17,10 +17,11 @@
  */
 package org.apache.beam.sdk.coders;
 
+import static com.google.common.base.Preconditions.checkArgument;
+
 import org.apache.beam.sdk.util.PropertyNames;
 import org.apache.beam.sdk.util.common.ElementByteSizeObserver;
 
-import com.google.common.base.Preconditions;
 import com.google.common.collect.Maps;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
@@ -57,8 +58,7 @@ public class MapCoder<K, V> extends StandardCoder<Map<K, V>> {
   public static MapCoder<?, ?> of(
       @JsonProperty(PropertyNames.COMPONENT_ENCODINGS)
       List<Coder<?>> components) {
-    Preconditions.checkArgument(components.size() == 2,
-        "Expecting 2 components, got " + components.size());
+    checkArgument(components.size() == 2, "Expecting 2 components, got %s", components.size());
     return of((Coder<?>) components.get(0), (Coder<?>) components.get(1));
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/NullableCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/NullableCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/NullableCoder.java
index 6fa7305..a8ce873 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/NullableCoder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/NullableCoder.java
@@ -17,11 +17,12 @@
  */
 package org.apache.beam.sdk.coders;
 
+import static com.google.common.base.Preconditions.checkArgument;
+
 import org.apache.beam.sdk.util.PropertyNames;
 import org.apache.beam.sdk.util.common.ElementByteSizeObserver;
 
 import com.google.common.base.Optional;
-import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
@@ -54,8 +55,7 @@ public class NullableCoder<T> extends StandardCoder<T> {
   public static NullableCoder<?> of(
       @JsonProperty(PropertyNames.COMPONENT_ENCODINGS)
       List<Coder<?>> components) {
-    Preconditions.checkArgument(components.size() == 1,
-        "Expecting 1 components, got " + components.size());
+    checkArgument(components.size() == 1, "Expecting 1 components, got %s", components.size());
     return of(components.get(0));
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SetCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SetCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SetCoder.java
index d194a53..fcbb379 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SetCoder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SetCoder.java
@@ -17,9 +17,9 @@
  */
 package org.apache.beam.sdk.coders;
 
-import org.apache.beam.sdk.util.PropertyNames;
+import static com.google.common.base.Preconditions.checkArgument;
 
-import com.google.common.base.Preconditions;
+import org.apache.beam.sdk.util.PropertyNames;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
@@ -50,8 +50,7 @@ public class SetCoder<T> extends IterableLikeCoder<T, Set<T>> {
   public static SetCoder<?> of(
       @JsonProperty(PropertyNames.COMPONENT_ENCODINGS)
       List<Object> components) {
-    Preconditions.checkArgument(components.size() == 1,
-        "Expecting 1 component, got " + components.size());
+    checkArgument(components.size() == 1, "Expecting 1 component, got %s", components.size());
     return of((Coder<?>) components.get(0));
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java
index 604051b..280cd12 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java
@@ -17,6 +17,7 @@
  */
 package org.apache.beam.sdk.io;
 
+import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkState;
 
 import org.apache.beam.sdk.coders.AvroCoder;
@@ -34,7 +35,6 @@ import org.apache.beam.sdk.values.PDone;
 import org.apache.beam.sdk.values.PInput;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
 
 import org.apache.avro.Schema;
 import org.apache.avro.file.DataFileWriter;
@@ -540,7 +540,7 @@ public class AvroIO {
        * @see ShardNameTemplate
        */
       public Bound<T> withNumShards(int numShards) {
-        Preconditions.checkArgument(numShards >= 0);
+        checkArgument(numShards >= 0);
         return new Bound<>(
             name, filenamePrefix, filenameSuffix, numShards, shardTemplate, type, schema, validate);
       }
@@ -729,7 +729,7 @@ public class AvroIO {
   private static final Pattern SHARD_OUTPUT_PATTERN = Pattern.compile("@([0-9]+|\\*)");
 
   private static void validateOutputComponent(String partialFilePattern) {
-    Preconditions.checkArgument(
+    checkArgument(
         !SHARD_OUTPUT_PATTERN.matcher(partialFilePattern).find(),
         "Output name components are not allowed to contain @* or @N patterns: "
         + partialFilePattern);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSource.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSource.java
index 9cc0b98..22e9dfe 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSource.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSource.java
@@ -17,6 +17,8 @@
  */
 package org.apache.beam.sdk.io;
 
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
 import static com.google.common.base.Preconditions.checkState;
 
 import org.apache.beam.sdk.annotations.Experimental;
@@ -27,8 +29,6 @@ import org.apache.beam.sdk.util.AvroUtils;
 import org.apache.beam.sdk.util.AvroUtils.AvroMetadata;
 import org.apache.beam.sdk.values.PCollection;
 
-import com.google.common.base.Preconditions;
-
 import org.apache.avro.Schema;
 import org.apache.avro.file.CodecFactory;
 import org.apache.avro.file.DataFileConstants;
@@ -265,7 +265,7 @@ public class AvroSource<T> extends BlockBasedSource<T> {
       AvroMetadata metadata;
       try {
         Collection<String> files = FileBasedSource.expandFilePattern(fileName);
-        Preconditions.checkArgument(files.size() <= 1, "More than 1 file matched %s");
+        checkArgument(files.size() <= 1, "More than 1 file matched %s");
         metadata = AvroUtils.readMetadataFromFile(fileName);
       } catch (IOException e) {
         throw new RuntimeException("Error reading metadata from file " + fileName, e);
@@ -343,10 +343,8 @@ public class AvroSource<T> extends BlockBasedSource<T> {
   private DatumReader<T> createDatumReader() {
     Schema readSchema = getReadSchema();
     Schema fileSchema = getFileSchema();
-    Preconditions.checkNotNull(
-        readSchema, "No read schema has been initialized for source %s", this);
-    Preconditions.checkNotNull(
-        fileSchema, "No file schema has been initialized for source %s", this);
+    checkNotNull(readSchema, "No read schema has been initialized for source %s", this);
+    checkNotNull(fileSchema, "No file schema has been initialized for source %s", this);
     if (type == GenericRecord.class) {
       return new GenericDatumReader<>(fileSchema, readSchema);
     } else {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CompressedSource.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CompressedSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CompressedSource.java
index 48e0b7a..ee4b84b 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CompressedSource.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CompressedSource.java
@@ -17,12 +17,14 @@
  */
 package org.apache.beam.sdk.io;
 
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+
 import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.transforms.display.DisplayData;
 
-import com.google.common.base.Preconditions;
 import com.google.common.io.ByteStreams;
 import com.google.common.primitives.Ints;
 
@@ -316,7 +318,7 @@ public class CompressedSource<T> extends FileBasedSource<T> {
       DecompressingChannelFactory channelFactory, String filePatternOrSpec, long minBundleSize,
       long startOffset, long endOffset) {
     super(filePatternOrSpec, minBundleSize, startOffset, endOffset);
-    Preconditions.checkArgument(
+    checkArgument(
         startOffset == 0,
         "CompressedSources must start reading at offset 0. Requested offset: " + startOffset);
     this.sourceDelegate = sourceDelegate;
@@ -329,9 +331,9 @@ public class CompressedSource<T> extends FileBasedSource<T> {
   @Override
   public void validate() {
     super.validate();
-    Preconditions.checkNotNull(sourceDelegate);
+    checkNotNull(sourceDelegate);
     sourceDelegate.validate();
-    Preconditions.checkNotNull(channelFactory);
+    checkNotNull(channelFactory);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/main/java/org/apache/beam/sdk/io/DatastoreIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/DatastoreIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/DatastoreIO.java
index 7fab79d..7abda71 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/DatastoreIO.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/DatastoreIO.java
@@ -47,7 +47,6 @@ import com.google.api.client.util.BackOff;
 import com.google.api.client.util.BackOffUtils;
 import com.google.api.client.util.Sleeper;
 import com.google.common.base.MoreObjects;
-import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import com.google.common.primitives.Ints;
 import com.google.datastore.v1beta3.CommitRequest;
@@ -307,8 +306,8 @@ public class DatastoreIO {
 
     @Override
     public void validate() {
-      Preconditions.checkNotNull(query, "query");
-      Preconditions.checkNotNull(projectId, "projectId");
+      checkNotNull(query, "query");
+      checkNotNull(projectId, "projectId");
     }
 
     @Override
@@ -548,7 +547,7 @@ public class DatastoreIO {
      */
     @Override
     public void validate(PipelineOptions options) {
-      Preconditions.checkNotNull(
+      checkNotNull(
           projectId,
           "Project ID is a required parameter. Please use withProject to to set the projectId.");
     }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSink.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSink.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSink.java
index 521f54b..02fc63a 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSink.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSink.java
@@ -17,6 +17,10 @@
  */
 package org.apache.beam.sdk.io;
 
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+import static com.google.common.base.Preconditions.checkState;
+
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.SerializableCoder;
 import org.apache.beam.sdk.options.PipelineOptions;
@@ -29,7 +33,6 @@ import org.apache.beam.sdk.util.IOChannelFactory;
 import org.apache.beam.sdk.util.IOChannelUtils;
 import org.apache.beam.sdk.util.MimeTypes;
 
-import com.google.common.base.Preconditions;
 import com.google.common.collect.Ordering;
 
 import org.slf4j.Logger;
@@ -112,7 +115,7 @@ public abstract class FileBasedSink<T> extends Sink<T> {
   /**
    * Perform pipeline-construction-time validation. The default implementation is a no-op.
    * Subclasses should override to ensure the sink is valid and can be written to. It is recommended
-   * to use {@link Preconditions} in the implementation of this method.
+   * to use {@link Preconditions#checkState(boolean)} in the implementation of this method.
    */
   @Override
   public void validate(PipelineOptions options) {}
@@ -366,7 +369,7 @@ public abstract class FileBasedSink<T> extends Sink<T> {
       }
 
       int numDistinctShards = new HashSet<String>(destFilenames).size();
-      Preconditions.checkState(numDistinctShards == numFiles,
+      checkState(numDistinctShards == numFiles,
           "Shard name template '%s' only generated %s distinct file names for %s files.",
           fileNamingTemplate, numDistinctShards, numFiles);
 
@@ -458,7 +461,7 @@ public abstract class FileBasedSink<T> extends Sink<T> {
      * Construct a new FileBasedWriter with a base filename.
      */
     public FileBasedWriter(FileBasedWriteOperation<T> writeOperation) {
-      Preconditions.checkNotNull(writeOperation);
+      checkNotNull(writeOperation);
       this.writeOperation = writeOperation;
     }
 
@@ -628,7 +631,7 @@ public abstract class FileBasedSink<T> extends Sink<T> {
 
     @Override
     public void copy(List<String> srcFilenames, List<String> destFilenames) throws IOException {
-      Preconditions.checkArgument(
+      checkArgument(
           srcFilenames.size() == destFilenames.size(),
           "Number of source files %s must equal number of destination files %s",
           srcFilenames.size(),

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/main/java/org/apache/beam/sdk/io/OffsetBasedSource.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/OffsetBasedSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/OffsetBasedSource.java
index 295eab9..d5a6801 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/OffsetBasedSource.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/OffsetBasedSource.java
@@ -17,13 +17,13 @@
  */
 package org.apache.beam.sdk.io;
 
+import static com.google.common.base.Preconditions.checkArgument;
+
 import org.apache.beam.sdk.io.range.OffsetRangeTracker;
 import org.apache.beam.sdk.io.range.RangeTracker;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.transforms.display.DisplayData;
 
-import com.google.common.base.Preconditions;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -141,17 +141,17 @@ public abstract class OffsetBasedSource<T> extends BoundedSource<T> {
 
   @Override
   public void validate() {
-    Preconditions.checkArgument(
+    checkArgument(
         this.startOffset >= 0,
         "Start offset has value %s, must be non-negative", this.startOffset);
-    Preconditions.checkArgument(
+    checkArgument(
         this.endOffset >= 0,
         "End offset has value %s, must be non-negative", this.endOffset);
-    Preconditions.checkArgument(
+    checkArgument(
         this.startOffset < this.endOffset,
         "Start offset %s must be before end offset %s",
         this.startOffset, this.endOffset);
-    Preconditions.checkArgument(
+    checkArgument(
         this.minBundleSize >= 0,
         "minBundleSize has value %s, must be non-negative",
         this.minBundleSize);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java
index 7e7a3e6..9dd3679 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java
@@ -17,6 +17,7 @@
  */
 package org.apache.beam.sdk.io;
 
+import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkState;
 
 import org.apache.beam.sdk.coders.Coder;
@@ -34,7 +35,6 @@ import org.apache.beam.sdk.values.PDone;
 import org.apache.beam.sdk.values.PInput;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
 import com.google.protobuf.ByteString;
 
 import java.io.IOException;
@@ -538,7 +538,7 @@ public class TextIO {
        * @see ShardNameTemplate
        */
       public Bound<T> withNumShards(int numShards) {
-        Preconditions.checkArgument(numShards >= 0);
+        checkArgument(numShards >= 0);
         return new Bound<>(name, filenamePrefix, filenameSuffix, coder, numShards,
             shardTemplate, validate);
       }
@@ -718,7 +718,7 @@ public class TextIO {
   private static final Pattern SHARD_OUTPUT_PATTERN = Pattern.compile("@([0-9]+|\\*)");
 
   private static void validateOutputComponent(String partialFilePattern) {
-    Preconditions.checkArgument(
+    checkArgument(
         !SHARD_OUTPUT_PATTERN.matcher(partialFilePattern).find(),
         "Output name components are not allowed to contain @* or @N patterns: "
         + partialFilePattern);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/main/java/org/apache/beam/sdk/io/XmlSink.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/XmlSink.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/XmlSink.java
index bbe68d9..ef50a1c 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/XmlSink.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/XmlSink.java
@@ -17,6 +17,8 @@
  */
 package org.apache.beam.sdk.io;
 
+import static com.google.common.base.Preconditions.checkNotNull;
+
 import org.apache.beam.sdk.coders.StringUtf8Coder;
 import org.apache.beam.sdk.io.FileBasedSink.FileBasedWriteOperation;
 import org.apache.beam.sdk.io.FileBasedSink.FileBasedWriter;
@@ -25,8 +27,6 @@ import org.apache.beam.sdk.transforms.display.DisplayData;
 import org.apache.beam.sdk.util.CoderUtils;
 import org.apache.beam.sdk.values.PCollection;
 
-import com.google.common.base.Preconditions;
-
 import java.io.OutputStream;
 import java.nio.channels.Channels;
 import java.nio.channels.WritableByteChannel;
@@ -205,9 +205,9 @@ public class XmlSink {
      */
     @Override
     public void validate(PipelineOptions options) {
-      Preconditions.checkNotNull(classToBind, "Missing a class to bind to a JAXB context.");
-      Preconditions.checkNotNull(rootElementName, "Missing a root element name.");
-      Preconditions.checkNotNull(baseOutputFilename, "Missing a filename to write to.");
+      checkNotNull(classToBind, "Missing a class to bind to a JAXB context.");
+      checkNotNull(rootElementName, "Missing a root element name.");
+      checkNotNull(baseOutputFilename, "Missing a filename to write to.");
       try {
         JAXBContext.newInstance(classToBind);
       } catch (JAXBException e) {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/main/java/org/apache/beam/sdk/io/XmlSource.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/XmlSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/XmlSource.java
index 56651e8..83443a3 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/XmlSource.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/XmlSource.java
@@ -17,14 +17,14 @@
  */
 package org.apache.beam.sdk.io;
 
+import static com.google.common.base.Preconditions.checkNotNull;
+
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.JAXBCoder;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.runners.PipelineRunner;
 import org.apache.beam.sdk.transforms.display.DisplayData;
 
-import com.google.common.base.Preconditions;
-
 import org.codehaus.stax2.XMLInputFactory2;
 
 import java.io.ByteArrayInputStream;
@@ -39,6 +39,7 @@ import java.nio.channels.Channels;
 import java.nio.channels.ReadableByteChannel;
 import java.nio.charset.StandardCharsets;
 import java.util.NoSuchElementException;
+
 import javax.xml.bind.JAXBContext;
 import javax.xml.bind.JAXBElement;
 import javax.xml.bind.JAXBException;
@@ -209,12 +210,12 @@ public class XmlSource<T> extends FileBasedSource<T> {
   @Override
   public void validate() {
     super.validate();
-    Preconditions.checkNotNull(
+    checkNotNull(
         rootElement, "rootElement is null. Use builder method withRootElement() to set this.");
-    Preconditions.checkNotNull(
+    checkNotNull(
         recordElement,
         "recordElement is null. Use builder method withRecordElement() to set this.");
-    Preconditions.checkNotNull(
+    checkNotNull(
         recordClass, "recordClass is null. Use builder method withRecordClass() to set this.");
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java
index 5f2dd11..3cb1377 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java
@@ -18,6 +18,7 @@
 package org.apache.beam.sdk.options;
 
 import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
 
 import org.apache.beam.sdk.options.Validation.Required;
 import org.apache.beam.sdk.runners.PipelineRunner;
@@ -30,7 +31,6 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Function;
 import com.google.common.base.Joiner;
 import com.google.common.base.Optional;
-import com.google.common.base.Preconditions;
 import com.google.common.base.Predicate;
 import com.google.common.base.Strings;
 import com.google.common.collect.Collections2;
@@ -239,7 +239,7 @@ public class PipelineOptionsFactory {
      * {@link PipelineOptionsFactory#printHelp(PrintStream, Class)}.
      */
     public Builder fromArgs(String[] args) {
-      Preconditions.checkNotNull(args, "Arguments should not be null.");
+      checkNotNull(args, "Arguments should not be null.");
       return new Builder(args, validation, strictParsing);
     }
 
@@ -549,8 +549,8 @@ public class PipelineOptionsFactory {
    * @param iface The interface object to manually register.
    */
   public static synchronized void register(Class<? extends PipelineOptions> iface) {
-    Preconditions.checkNotNull(iface);
-    Preconditions.checkArgument(iface.isInterface(), "Only interface types are supported.");
+    checkNotNull(iface);
+    checkArgument(iface.isInterface(), "Only interface types are supported.");
 
     if (REGISTERED_OPTIONS.contains(iface)) {
       return;
@@ -608,7 +608,7 @@ public class PipelineOptionsFactory {
    */
   static synchronized <T extends PipelineOptions> Registration<T> validateWellFormed(
       Class<T> iface, Set<Class<? extends PipelineOptions>> validatedPipelineOptionsInterfaces) {
-    Preconditions.checkArgument(iface.isInterface(), "Only interface types are supported.");
+    checkArgument(iface.isInterface(), "Only interface types are supported.");
 
     @SuppressWarnings("unchecked")
     Set<Class<? extends PipelineOptions>> combinedPipelineOptionsInterfaces =
@@ -659,7 +659,7 @@ public class PipelineOptionsFactory {
    * format its output to be compatible with a terminal window.
    */
   public static void printHelp(PrintStream out) {
-    Preconditions.checkNotNull(out);
+    checkNotNull(out);
     out.println("The set of registered options are:");
     Set<Class<? extends PipelineOptions>> sortedOptions =
         new TreeSet<>(ClassNameComparator.INSTANCE);
@@ -691,8 +691,8 @@ public class PipelineOptionsFactory {
    * This method will attempt to format its output to be compatible with a terminal window.
    */
   public static void printHelp(PrintStream out, Class<? extends PipelineOptions> iface) {
-    Preconditions.checkNotNull(out);
-    Preconditions.checkNotNull(iface);
+    checkNotNull(out);
+    checkNotNull(iface);
     validateWellFormed(iface, REGISTERED_OPTIONS);
 
     Set<PipelineOptionSpec> properties =
@@ -1116,7 +1116,7 @@ public class PipelineOptionsFactory {
         Sets.filter(
             Sets.difference(Sets.newHashSet(klass.getMethods()), methods),
             NOT_SYNTHETIC_PREDICATE));
-    Preconditions.checkArgument(unknownMethods.isEmpty(),
+    checkArgument(unknownMethods.isEmpty(),
         "Methods %s on [%s] do not conform to being bean properties.",
         FluentIterable.from(unknownMethods).transform(ReflectHelpers.METHOD_FORMATTER),
         iface.getName());
@@ -1329,11 +1329,11 @@ public class PipelineOptionsFactory {
         continue;
       }
       try {
-        Preconditions.checkArgument(arg.startsWith("--"),
+        checkArgument(arg.startsWith("--"),
             "Argument '%s' does not begin with '--'", arg);
         int index = arg.indexOf("=");
         // Make sure that '=' isn't the first character after '--' or the last character
-        Preconditions.checkArgument(index != 2,
+        checkArgument(index != 2,
             "Argument '%s' starts with '--=', empty argument name not allowed", arg);
         if (index > 0) {
           builder.put(arg.substring(2, index), arg.substring(index + 1, arg.length()));
@@ -1448,7 +1448,7 @@ public class PipelineOptionsFactory {
 
           if (returnType.isArray() && !returnType.getComponentType().equals(String.class)) {
             for (String value : values) {
-              Preconditions.checkArgument(!value.isEmpty(),
+              checkArgument(!value.isEmpty(),
                   "Empty argument value is only allowed for String, String Array, and Collection,"
                   + " but received: " + returnType);
             }
@@ -1456,13 +1456,13 @@ public class PipelineOptionsFactory {
           convertedOptions.put(entry.getKey(), MAPPER.convertValue(values, type));
         } else if (SIMPLE_TYPES.contains(returnType) || returnType.isEnum()) {
           String value = Iterables.getOnlyElement(entry.getValue());
-          Preconditions.checkArgument(returnType.equals(String.class) || !value.isEmpty(),
+          checkArgument(returnType.equals(String.class) || !value.isEmpty(),
               "Empty argument value is only allowed for String, String Array, and Collection,"
                + " but received: " + returnType);
           convertedOptions.put(entry.getKey(), MAPPER.convertValue(value, type));
         } else {
           String value = Iterables.getOnlyElement(entry.getValue());
-          Preconditions.checkArgument(returnType.equals(String.class) || !value.isEmpty(),
+          checkArgument(returnType.equals(String.class) || !value.isEmpty(),
               "Empty argument value is only allowed for String, String Array, and Collection,"
                + " but received: " + returnType);
           try {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsValidator.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsValidator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsValidator.java
index 4946c59..70be65f 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsValidator.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsValidator.java
@@ -17,10 +17,12 @@
  */
 package org.apache.beam.sdk.options;
 
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+
 import org.apache.beam.sdk.options.Validation.Required;
 import org.apache.beam.sdk.util.common.ReflectHelpers;
 
-import com.google.common.base.Preconditions;
 import com.google.common.collect.Collections2;
 import com.google.common.collect.Ordering;
 import com.google.common.collect.SortedSetMultimap;
@@ -46,11 +48,10 @@ public class PipelineOptionsValidator {
    * @return The type
    */
   public static <T extends PipelineOptions> T validate(Class<T> klass, PipelineOptions options) {
-    Preconditions.checkNotNull(klass);
-    Preconditions.checkNotNull(options);
-    Preconditions.checkArgument(Proxy.isProxyClass(options.getClass()));
-    Preconditions.checkArgument(Proxy.getInvocationHandler(options)
-        instanceof ProxyInvocationHandler);
+    checkNotNull(klass);
+    checkNotNull(options);
+    checkArgument(Proxy.isProxyClass(options.getClass()));
+    checkArgument(Proxy.getInvocationHandler(options) instanceof ProxyInvocationHandler);
 
     // Ensure the methods for T are registered on the ProxyInvocationHandler
     T asClassOptions = options.as(klass);
@@ -68,8 +69,9 @@ public class PipelineOptionsValidator {
             requiredGroups.put(requiredGroup, method);
           }
         } else {
-          Preconditions.checkArgument(handler.invoke(asClassOptions, method, null) != null,
-              "Missing required value for [" + method + ", \"" + getDescription(method) + "\"]. ");
+          checkArgument(handler.invoke(asClassOptions, method, null) != null,
+              "Missing required value for [%s, \"%s\"]. ",
+              method, getDescription(method));
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ProxyInvocationHandler.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ProxyInvocationHandler.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ProxyInvocationHandler.java
index a0c4ea3..e3d763b 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ProxyInvocationHandler.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ProxyInvocationHandler.java
@@ -17,6 +17,9 @@
  */
 package org.apache.beam.sdk.options;
 
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+
 import org.apache.beam.sdk.options.PipelineOptionsFactory.JsonIgnorePredicate;
 import org.apache.beam.sdk.options.PipelineOptionsFactory.Registration;
 import org.apache.beam.sdk.transforms.display.DisplayData;
@@ -27,7 +30,6 @@ import org.apache.beam.sdk.util.common.ReflectHelpers;
 import com.google.auto.value.AutoValue;
 import com.google.common.base.Defaults;
 import com.google.common.base.Function;
-import com.google.common.base.Preconditions;
 import com.google.common.collect.ClassToInstanceMap;
 import com.google.common.collect.FluentIterable;
 import com.google.common.collect.HashMultimap;
@@ -67,6 +69,7 @@ import java.util.Map;
 import java.util.Set;
 import java.util.SortedMap;
 import java.util.TreeMap;
+
 import javax.annotation.Nullable;
 import javax.annotation.concurrent.ThreadSafe;
 
@@ -202,8 +205,8 @@ class ProxyInvocationHandler implements InvocationHandler, HasDisplayData {
    * @return An object that implements the interface <T>.
    */
   synchronized <T extends PipelineOptions> T as(Class<T> iface) {
-    Preconditions.checkNotNull(iface);
-    Preconditions.checkArgument(iface.isInterface());
+    checkNotNull(iface);
+    checkArgument(iface.isInterface());
     if (!interfaceToProxyCache.containsKey(iface)) {
       Registration<T> registration =
           PipelineOptionsFactory.validateWellFormed(iface, knownInterfaces);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunner.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunner.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunner.java
index 941debb..ede1507 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunner.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunner.java
@@ -17,6 +17,8 @@
  */
 package org.apache.beam.sdk.runners;
 
+import static com.google.common.base.Preconditions.checkNotNull;
+
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.PipelineResult;
 import org.apache.beam.sdk.options.GcsOptions;
@@ -28,8 +30,6 @@ import org.apache.beam.sdk.util.InstanceBuilder;
 import org.apache.beam.sdk.values.PInput;
 import org.apache.beam.sdk.values.POutput;
 
-import com.google.common.base.Preconditions;
-
 /**
  * A {@link PipelineRunner} can execute, translate, or otherwise process a
  * {@link Pipeline}.
@@ -45,7 +45,7 @@ public abstract class PipelineRunner<ResultT extends PipelineResult> {
    */
   public static PipelineRunner<? extends PipelineResult> fromOptions(PipelineOptions options) {
     GcsOptions gcsOptions = PipelineOptionsValidator.validate(GcsOptions.class, options);
-    Preconditions.checkNotNull(options);
+    checkNotNull(options);
 
     // (Re-)register standard IO factories. Clobbers any prior credentials.
     IOChannelUtils.registerStandardIOFactories(gcsOptions);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java
index 6aeaf5f..6b93bbd 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java
@@ -17,13 +17,13 @@
  */
 package org.apache.beam.sdk.runners;
 
+import static com.google.common.base.Preconditions.checkState;
+
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.values.PInput;
 import org.apache.beam.sdk.values.POutput;
 import org.apache.beam.sdk.values.PValue;
 
-import com.google.common.base.Preconditions;
-
 import java.util.Deque;
 import java.util.HashMap;
 import java.util.LinkedList;
@@ -65,7 +65,7 @@ public class TransformHierarchy {
    */
   public void popNode() {
     transformStack.pop();
-    Preconditions.checkState(!transformStack.isEmpty());
+    checkState(!transformStack.isEmpty());
   }
 
   /**
@@ -76,9 +76,7 @@ public class TransformHierarchy {
   public void addInput(TransformTreeNode node, PInput input) {
     for (PValue i : input.expand()) {
       TransformTreeNode producer = producingTransformNode.get(i);
-      if (producer == null) {
-        throw new IllegalStateException("Producer unknown for input: " + i);
-      }
+      checkState(producer != null, "Producer unknown for input: %s", i);
 
       producer.finishSpecifying();
       node.addInputProducer(i, producer);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformTreeNode.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformTreeNode.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformTreeNode.java
index 59edd52..b5a250b 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformTreeNode.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformTreeNode.java
@@ -17,6 +17,9 @@
  */
 package org.apache.beam.sdk.runners;
 
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkState;
+
 import org.apache.beam.sdk.Pipeline.PipelineVisitor;
 import org.apache.beam.sdk.Pipeline.PipelineVisitor.CompositeBehavior;
 import org.apache.beam.sdk.transforms.PTransform;
@@ -24,8 +27,6 @@ import org.apache.beam.sdk.values.PInput;
 import org.apache.beam.sdk.values.POutput;
 import org.apache.beam.sdk.values.PValue;
 
-import com.google.common.base.Preconditions;
-
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -82,7 +83,7 @@ public class TransformTreeNode {
                            @Nullable PInput input) {
     this.enclosingNode = enclosingNode;
     this.transform = transform;
-    Preconditions.checkArgument((enclosingNode == null && transform == null)
+    checkArgument((enclosingNode == null && transform == null)
         || (enclosingNode != null && transform != null),
         "EnclosingNode and transform must both be specified, or both be null");
     this.fullName = fullName;
@@ -146,7 +147,7 @@ public class TransformTreeNode {
    * Adds an input to the transform node.
    */
   public void addInputProducer(PValue expandedInput, TransformTreeNode producer) {
-    Preconditions.checkState(!finishedSpecifying);
+    checkState(!finishedSpecifying);
     inputs.put(expandedInput, producer);
   }
 
@@ -169,8 +170,8 @@ public class TransformTreeNode {
    * Adds an output to the transform node.
    */
   public void setOutput(POutput output) {
-    Preconditions.checkState(!finishedSpecifying);
-    Preconditions.checkState(this.output == null);
+    checkState(!finishedSpecifying);
+    checkState(this.output == null);
     this.output = output;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateQuantiles.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateQuantiles.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateQuantiles.java
index b056830..fa87ed8 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateQuantiles.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateQuantiles.java
@@ -17,6 +17,8 @@
  */
 package org.apache.beam.sdk.transforms;
 
+import static com.google.common.base.Preconditions.checkArgument;
+
 import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.CoderException;
@@ -31,7 +33,6 @@ import org.apache.beam.sdk.util.common.ElementByteSizeObserver;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 
-import com.google.common.base.Preconditions;
 import com.google.common.collect.Iterators;
 import com.google.common.collect.Lists;
 import com.google.common.collect.UnmodifiableIterator;
@@ -254,9 +255,9 @@ public class ApproximateQuantiles {
         int bufferSize,
         int numBuffers,
         long maxNumElements) {
-      Preconditions.checkArgument(numQuantiles >= 2);
-      Preconditions.checkArgument(bufferSize >= 2);
-      Preconditions.checkArgument(numBuffers >= 2);
+      checkArgument(numQuantiles >= 2);
+      checkArgument(bufferSize >= 2);
+      checkArgument(numBuffers >= 2);
       this.numQuantiles = numQuantiles;
       this.compareFn = compareFn;
       this.bufferSize = bufferSize;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java
index 7871672..5faf4e3 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java
@@ -17,6 +17,8 @@
  */
 package org.apache.beam.sdk.transforms;
 
+import static com.google.common.base.Preconditions.checkState;
+
 import org.apache.beam.sdk.coders.CannotProvideCoderException;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.CoderException;
@@ -28,7 +30,6 @@ import org.apache.beam.sdk.coders.KvCoder;
 import org.apache.beam.sdk.coders.StandardCoder;
 import org.apache.beam.sdk.coders.VarIntCoder;
 import org.apache.beam.sdk.coders.VoidCoder;
-import org.apache.beam.sdk.transforms.Combine.AccumulatingCombineFn;
 import org.apache.beam.sdk.transforms.CombineFnBase.AbstractGlobalCombineFn;
 import org.apache.beam.sdk.transforms.CombineFnBase.AbstractPerKeyCombineFn;
 import org.apache.beam.sdk.transforms.CombineFnBase.GlobalCombineFn;
@@ -58,7 +59,6 @@ import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.sdk.values.TupleTagList;
 import org.apache.beam.sdk.values.TypeDescriptor;
 
-import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
 
@@ -1349,7 +1349,7 @@ public class Combine {
      */
     public Globally<InputT, OutputT> withSideInputs(
         Iterable<? extends PCollectionView<?>> sideInputs) {
-      Preconditions.checkState(fn instanceof RequiresContextInternal);
+      checkState(fn instanceof RequiresContextInternal);
       return new Globally<>(name, fn, fnDisplayData, insertDefault, fanout,
           ImmutableList.copyOf(sideInputs));
     }
@@ -1733,7 +1733,7 @@ public class Combine {
      */
     public PerKey<K, InputT, OutputT> withSideInputs(
         Iterable<? extends PCollectionView<?>> sideInputs) {
-      Preconditions.checkState(fn instanceof RequiresContextInternal);
+      checkState(fn instanceof RequiresContextInternal);
       return new PerKey<>(name, fn, fnDisplayData, fewKeys,
           ImmutableList.copyOf(sideInputs));
     }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Create.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Create.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Create.java
index 538966d..fa645ab 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Create.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Create.java
@@ -17,6 +17,8 @@
  */
 package org.apache.beam.sdk.transforms;
 
+import static com.google.common.base.Preconditions.checkArgument;
+
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.CannotProvideCoderException;
 import org.apache.beam.sdk.coders.Coder;
@@ -39,7 +41,6 @@ import org.apache.beam.sdk.values.TypeDescriptor;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Function;
 import com.google.common.base.Optional;
-import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
 
@@ -210,7 +211,7 @@ public class Create<T> {
     while (valueIter.hasNext() && timestampIter.hasNext()) {
       elems.add(TimestampedValue.of(valueIter.next(), new Instant(timestampIter.next())));
     }
-    Preconditions.checkArgument(
+    checkArgument(
         !valueIter.hasNext() && !timestampIter.hasNext(),
         "Expect sizes of values and timestamps are same.");
     return timestamped(elems);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/IntraBundleParallelization.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/IntraBundleParallelization.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/IntraBundleParallelization.java
index 3cb8415..ef1e3c6 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/IntraBundleParallelization.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/IntraBundleParallelization.java
@@ -17,6 +17,8 @@
  */
 package org.apache.beam.sdk.transforms;
 
+import static com.google.common.base.Preconditions.checkArgument;
+
 import org.apache.beam.sdk.options.GcsOptions;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.transforms.Combine.CombineFn;
@@ -29,7 +31,6 @@ import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.sdk.values.TypeDescriptor;
 
-import com.google.common.base.Preconditions;
 import com.google.common.base.Throwables;
 
 import org.joda.time.Instant;
@@ -102,7 +103,7 @@ public class IntraBundleParallelization {
     }
 
     Unbound(int maxParallelism) {
-      Preconditions.checkArgument(maxParallelism > 0,
+      checkArgument(maxParallelism > 0,
           "Expected parallelism factor greater than zero, received %s.", maxParallelism);
       this.maxParallelism = maxParallelism;
     }
@@ -143,7 +144,7 @@ public class IntraBundleParallelization {
     private final int maxParallelism;
 
     Bound(DoFn<InputT, OutputT> doFn, int maxParallelism) {
-      Preconditions.checkArgument(maxParallelism > 0,
+      checkArgument(maxParallelism > 0,
           "Expected parallelism factor greater than zero, received %s.", maxParallelism);
       this.doFn = doFn;
       this.maxParallelism = maxParallelism;
@@ -198,7 +199,7 @@ public class IntraBundleParallelization {
       extends DoFn<InputT, OutputT> {
 
     public MultiThreadedIntraBundleProcessingDoFn(DoFn<InputT, OutputT> doFn, int maxParallelism) {
-      Preconditions.checkArgument(maxParallelism > 0,
+      checkArgument(maxParallelism > 0,
           "Expected parallelism factor greater than zero, received %s.", maxParallelism);
       this.doFn = doFn;
       this.maxParallelism = maxParallelism;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Sample.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Sample.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Sample.java
index 1d98668..1495c01 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Sample.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Sample.java
@@ -17,6 +17,8 @@
  */
 package org.apache.beam.sdk.transforms;
 
+import static com.google.common.base.Preconditions.checkArgument;
+
 import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.CoderRegistry;
@@ -29,8 +31,6 @@ import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionView;
 
-import com.google.common.base.Preconditions;
-
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Random;
@@ -139,7 +139,7 @@ public class Sample {
      * elements of its input {@code PCollection}.
      */
     private SampleAny(long limit) {
-      Preconditions.checkArgument(limit >= 0, "Expected non-negative limit, received %s.", limit);
+      checkArgument(limit >= 0, "Expected non-negative limit, received %s.", limit);
       this.limit = limit;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Top.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Top.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Top.java
index 257f77b..04cf9e3 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Top.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Top.java
@@ -17,6 +17,8 @@
  */
 package org.apache.beam.sdk.transforms;
 
+import static com.google.common.base.Preconditions.checkArgument;
+
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.CoderException;
 import org.apache.beam.sdk.coders.CoderRegistry;
@@ -32,7 +34,6 @@ import org.apache.beam.sdk.util.common.ElementByteSizeObserver;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 
-import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 
 import java.io.IOException;
@@ -375,9 +376,7 @@ new TopCombineFn<>(count, new Largest<V>()).<K>asKeyedFn())
     private final ComparatorT compareFn;
 
     public TopCombineFn(int count, ComparatorT compareFn) {
-      Preconditions.checkArgument(
-          count >= 0,
-          "count must be >= 0");
+      checkArgument(count >= 0, "count must be >= 0 (not %s)", count);
       this.count = count;
       this.compareFn = compareFn;
     }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/DisplayData.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/DisplayData.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/DisplayData.java
index 59a2cf4..6180dbf 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/DisplayData.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/DisplayData.java
@@ -17,12 +17,12 @@
  */
 package org.apache.beam.sdk.transforms.display;
 
+import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkNotNull;
 
 import org.apache.beam.sdk.transforms.PTransform;
 
 import com.google.auto.value.AutoValue;
-import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
@@ -657,7 +657,7 @@ public class DisplayData implements Serializable {
       }
 
       Identifier id = Identifier.of(item.getNamespace(), item.getKey());
-      Preconditions.checkArgument(!entries.containsKey(id),
+      checkArgument(!entries.containsKey(id),
           "Display data key (%s) is not unique within the specified namespace (%s).",
           item.getKey(), item.getNamespace());
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java
index f868b9c..f4cf84d 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java
@@ -19,6 +19,8 @@ package org.apache.beam.sdk.transforms.join;
 
 import static org.apache.beam.sdk.util.Structs.addObject;
 
+import static com.google.common.base.Preconditions.checkArgument;
+
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.CoderException;
 import org.apache.beam.sdk.coders.IterableCoder;
@@ -30,7 +32,6 @@ import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.sdk.values.TupleTagList;
 
-import com.google.common.base.Preconditions;
 import com.google.common.collect.Iterators;
 import com.google.common.collect.PeekingIterator;
 
@@ -233,8 +234,7 @@ public class CoGbkResult {
         @JsonProperty(PropertyNames.COMPONENT_ENCODINGS)
         List<Coder<?>> components,
         @JsonProperty(PropertyNames.CO_GBK_RESULT_SCHEMA) CoGbkResultSchema schema) {
-      Preconditions.checkArgument(components.size() == 1,
-          "Expecting 1 component, got " + components.size());
+      checkArgument(components.size() == 1, "Expecting 1 component, got %s", components.size());
       return new CoGbkResultCoder(schema, (UnionCoder) components.get(0));
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterAll.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterAll.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterAll.java
index e6d8e84..27dd129 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterAll.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterAll.java
@@ -17,12 +17,13 @@
  */
 package org.apache.beam.sdk.transforms.windowing;
 
+import static com.google.common.base.Preconditions.checkArgument;
+
 import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger;
 import org.apache.beam.sdk.util.ExecutableTrigger;
 
 import com.google.common.base.Joiner;
-import com.google.common.base.Preconditions;
 
 import org.joda.time.Instant;
 
@@ -37,13 +38,12 @@ public class AfterAll extends OnceTrigger {
 
   private AfterAll(List<Trigger> subTriggers) {
     super(subTriggers);
-    Preconditions.checkArgument(subTriggers.size() > 1);
+    checkArgument(subTriggers.size() > 1);
   }
 
   /**
    * Returns an {@code AfterAll} {@code Trigger} with the given subtriggers.
    */
-  @SafeVarargs
   public static OnceTrigger of(OnceTrigger... triggers) {
     return new AfterAll(Arrays.<Trigger>asList(triggers));
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterFirst.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterFirst.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterFirst.java
index a43d9ac..1034cac 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterFirst.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterFirst.java
@@ -17,12 +17,13 @@
  */
 package org.apache.beam.sdk.transforms.windowing;
 
+import static com.google.common.base.Preconditions.checkArgument;
+
 import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger;
 import org.apache.beam.sdk.util.ExecutableTrigger;
 
 import com.google.common.base.Joiner;
-import com.google.common.base.Preconditions;
 
 import org.joda.time.Instant;
 
@@ -38,15 +39,13 @@ public class AfterFirst extends OnceTrigger {
 
   AfterFirst(List<Trigger> subTriggers) {
     super(subTriggers);
-    Preconditions.checkArgument(subTriggers.size() > 1);
+    checkArgument(subTriggers.size() > 1);
   }
 
   /**
    * Returns an {@code AfterFirst} {@code Trigger} with the given subtriggers.
    */
-  @SafeVarargs
-  public static OnceTrigger of(
-      OnceTrigger... triggers) {
+  public static OnceTrigger of(OnceTrigger... triggers) {
     return new AfterFirst(Arrays.<Trigger>asList(triggers));
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PaneInfo.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PaneInfo.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PaneInfo.java
index fff5001..77c857c 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PaneInfo.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PaneInfo.java
@@ -17,6 +17,9 @@
  */
 package org.apache.beam.sdk.transforms.windowing;
 
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+
 import org.apache.beam.sdk.coders.AtomicCoder;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.CoderException;
@@ -25,7 +28,6 @@ import org.apache.beam.sdk.transforms.GroupByKey;
 import org.apache.beam.sdk.util.VarInt;
 
 import com.google.common.base.MoreObjects;
-import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableMap;
 
 import java.io.IOException;
@@ -191,7 +193,7 @@ public final class PaneInfo {
   }
 
   public static PaneInfo createPane(boolean isFirst, boolean isLast, Timing timing) {
-    Preconditions.checkArgument(isFirst, "Indices must be provided for non-first pane info.");
+    checkArgument(isFirst, "Indices must be provided for non-first pane info.");
     return createPane(isFirst, isLast, timing, 0, timing == Timing.EARLY ? -1 : 0);
   }
 
@@ -201,15 +203,14 @@ public final class PaneInfo {
   public static PaneInfo createPane(
       boolean isFirst, boolean isLast, Timing timing, long index, long onTimeIndex) {
     if (isFirst || timing == Timing.UNKNOWN) {
-      return Preconditions.checkNotNull(
-          BYTE_TO_PANE_INFO.get(encodedByte(isFirst, isLast, timing)));
+      return checkNotNull(BYTE_TO_PANE_INFO.get(encodedByte(isFirst, isLast, timing)));
     } else {
       return new PaneInfo(isFirst, isLast, timing, index, onTimeIndex);
     }
   }
 
   public static PaneInfo decodePane(byte encodedPane) {
-    return Preconditions.checkNotNull(BYTE_TO_PANE_INFO.get(encodedPane));
+    return checkNotNull(BYTE_TO_PANE_INFO.get(encodedPane));
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AttemptAndTimeBoundedExponentialBackOff.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AttemptAndTimeBoundedExponentialBackOff.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AttemptAndTimeBoundedExponentialBackOff.java
index 833e9b3..e26b7bd 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AttemptAndTimeBoundedExponentialBackOff.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AttemptAndTimeBoundedExponentialBackOff.java
@@ -17,9 +17,10 @@
  */
 package org.apache.beam.sdk.util;
 
+import static com.google.common.base.Preconditions.checkArgument;
+
 import com.google.api.client.util.BackOff;
 import com.google.api.client.util.NanoClock;
-import com.google.common.base.Preconditions;
 
 import java.util.concurrent.TimeUnit;
 
@@ -112,13 +113,13 @@ public class AttemptAndTimeBoundedExponentialBackOff extends AttemptBoundedExpon
       ResetPolicy resetPolicy,
       NanoClock nanoClock) {
     super(maximumNumberOfAttempts, initialIntervalMillis);
-    Preconditions.checkArgument(
+    checkArgument(
         maximumTotalWaitTimeMillis > 0, "Maximum total wait time must be greater than zero.");
-    Preconditions.checkArgument(
+    checkArgument(
         maximumTotalWaitTimeMillis < MAX_ELAPSED_TIME_MILLIS,
         "Maximum total wait time must be less than " + MAX_ELAPSED_TIME_MILLIS + " milliseconds");
-    Preconditions.checkArgument(resetPolicy != null, "resetPolicy may not be null");
-    Preconditions.checkArgument(nanoClock != null, "nanoClock may not be null");
+    checkArgument(resetPolicy != null, "resetPolicy may not be null");
+    checkArgument(nanoClock != null, "nanoClock may not be null");
     this.maximumTotalWaitTimeMillis = maximumTotalWaitTimeMillis;
     this.resetPolicy = resetPolicy;
     this.nanoClock = nanoClock;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AttemptBoundedExponentialBackOff.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AttemptBoundedExponentialBackOff.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AttemptBoundedExponentialBackOff.java
index cfc53f0..5707293 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AttemptBoundedExponentialBackOff.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AttemptBoundedExponentialBackOff.java
@@ -17,8 +17,10 @@
  */
 package org.apache.beam.sdk.util;
 
+import static com.google.common.base.Preconditions.checkArgument;
+
 import com.google.api.client.util.BackOff;
-import com.google.common.base.Preconditions;
+
 
 /**
  * Implementation of {@link BackOff} that increases the back off period for each retry attempt
@@ -51,10 +53,9 @@ public class AttemptBoundedExponentialBackOff implements BackOff {
   private int currentAttempt;
 
   public AttemptBoundedExponentialBackOff(int maximumNumberOfAttempts, long initialIntervalMillis) {
-    Preconditions.checkArgument(maximumNumberOfAttempts > 0,
+    checkArgument(maximumNumberOfAttempts > 0,
         "Maximum number of attempts must be greater than zero.");
-    Preconditions.checkArgument(initialIntervalMillis > 0,
-        "Initial interval must be greater than zero.");
+    checkArgument(initialIntervalMillis > 0, "Initial interval must be greater than zero.");
     this.maximumNumberOfAttempts = maximumNumberOfAttempts;
     this.initialIntervalMillis = initialIntervalMillis;
     reset();

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryTableInserter.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryTableInserter.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryTableInserter.java
index bcb4137..f87a3c4 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryTableInserter.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryTableInserter.java
@@ -17,6 +17,8 @@
  */
 package org.apache.beam.sdk.util;
 
+import static com.google.common.base.Preconditions.checkNotNull;
+
 import org.apache.beam.sdk.io.BigQueryIO;
 import org.apache.beam.sdk.io.BigQueryIO.Write.CreateDisposition;
 import org.apache.beam.sdk.io.BigQueryIO.Write.WriteDisposition;
@@ -36,7 +38,6 @@ import com.google.api.services.bigquery.model.TableRow;
 import com.google.api.services.bigquery.model.TableSchema;
 import com.google.cloud.hadoop.util.ApiErrorExtractor;
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
 import com.google.common.util.concurrent.MoreExecutors;
 
 import org.slf4j.Logger;
@@ -170,7 +171,7 @@ public class BigQueryTableInserter {
   public void insertAll(TableReference ref, List<TableRow> rowList,
       @Nullable List<String> insertIdList, Aggregator<Long, Long> byteCountAggregator)
       throws IOException {
-    Preconditions.checkNotNull(ref, "ref");
+    checkNotNull(ref, "ref");
     if (insertIdList != null && rowList.size() != insertIdList.size()) {
       throw new AssertionError("If insertIdList is not null it needs to have at least "
           + "as many elements as rowList");

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Credentials.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Credentials.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Credentials.java
index 6fbbcb5..41065cd 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Credentials.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Credentials.java
@@ -17,6 +17,8 @@
  */
 package org.apache.beam.sdk.util;
 
+import static com.google.common.base.Preconditions.checkArgument;
+
 import org.apache.beam.sdk.options.GcpOptions;
 
 import com.google.api.client.auth.oauth2.Credential;
@@ -32,7 +34,6 @@ import com.google.api.client.http.HttpTransport;
 import com.google.api.client.json.JsonFactory;
 import com.google.api.client.json.jackson2.JacksonFactory;
 import com.google.api.client.util.store.FileDataStoreFactory;
-import com.google.common.base.Preconditions;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -157,7 +158,7 @@ public class Credentials {
       throws IOException, GeneralSecurityException {
     String clientSecretsFile = options.getSecretsFile();
 
-    Preconditions.checkArgument(clientSecretsFile != null);
+    checkArgument(clientSecretsFile != null);
     HttpTransport httpTransport = GoogleNetHttpTransport.newTrustedTransport();
 
     JsonFactory jsonFactory = JacksonFactory.getDefaultInstance();

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExecutableTrigger.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExecutableTrigger.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExecutableTrigger.java
index 9fcd21b..bb71fcc 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExecutableTrigger.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExecutableTrigger.java
@@ -17,12 +17,13 @@
  */
 package org.apache.beam.sdk.util;
 
+import static com.google.common.base.Preconditions.checkNotNull;
+import static com.google.common.base.Preconditions.checkState;
+
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.transforms.windowing.Trigger;
 import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger;
 
-import com.google.common.base.Preconditions;
-
 import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.List;
@@ -59,7 +60,7 @@ public class ExecutableTrigger implements Serializable {
   }
 
   private ExecutableTrigger(Trigger trigger, int nextUnusedIndex) {
-    this.trigger = Preconditions.checkNotNull(trigger, "trigger must not be null");
+    this.trigger = checkNotNull(trigger, "trigger must not be null");
     this.triggerIndex = nextUnusedIndex++;
 
     if (trigger.subTriggers() != null) {
@@ -101,8 +102,8 @@ public class ExecutableTrigger implements Serializable {
   }
 
   public ExecutableTrigger getSubTriggerContaining(int index) {
-    Preconditions.checkNotNull(subTriggers);
-    Preconditions.checkState(index > triggerIndex && index < firstIndexAfterSubtree,
+    checkNotNull(subTriggers);
+    checkState(index > triggerIndex && index < firstIndexAfterSubtree,
         "Cannot find sub-trigger containing index not in this tree.");
     ExecutableTrigger previous = null;
     for (ExecutableTrigger subTrigger : subTriggers) {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsUtil.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsUtil.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsUtil.java
index 9e8d175..4e9ee6e 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsUtil.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsUtil.java
@@ -17,6 +17,9 @@
  */
 package org.apache.beam.sdk.util;
 
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+
 import org.apache.beam.sdk.options.DefaultValueFactory;
 import org.apache.beam.sdk.options.GcsOptions;
 import org.apache.beam.sdk.options.PipelineOptions;
@@ -43,7 +46,6 @@ import com.google.cloud.hadoop.util.ClientRequestHelper;
 import com.google.cloud.hadoop.util.ResilientOperation;
 import com.google.cloud.hadoop.util.RetryDeterminer;
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 
 import org.slf4j.Logger;
@@ -158,7 +160,7 @@ public class GcsUtil {
    * exists.
    */
   public List<GcsPath> expand(GcsPath gcsPattern) throws IOException {
-    Preconditions.checkArgument(isGcsPatternSupported(gcsPattern.getObject()));
+    checkArgument(isGcsPatternSupported(gcsPattern.getObject()));
     Matcher m = GLOB_PREFIX.matcher(gcsPattern.getObject());
     Pattern p = null;
     String prefix = null;
@@ -218,7 +220,7 @@ public class GcsUtil {
             +  ", prefix " + prefix + " against pattern " + p.toString(), e);
       }
       //Objects objects = listObject.execute();
-      Preconditions.checkNotNull(objects);
+      checkNotNull(objects);
 
       if (objects.getItems() == null) {
         break;
@@ -371,7 +373,7 @@ public class GcsUtil {
   }
 
   public void copy(List<String> srcFilenames, List<String> destFilenames) throws IOException {
-    Preconditions.checkArgument(
+    checkArgument(
         srcFilenames.size() == destFilenames.size(),
         "Number of source files %s must equal number of destination files %s",
         srcFilenames.size(),


[47/50] [abbrv] incubator-beam git commit: DatastoreIO: add Read/Write Transforms and version-specific class

Posted by lc...@apache.org.
DatastoreIO: add Read/Write Transforms and version-specific class

* Also move DatastoreIO to io.datastore package


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

Branch: refs/heads/runners-spark2
Commit: d1e38443be936da918f6e3115145d0bc0b46c762
Parents: b36aeb9
Author: Vikas Kedigehalli <vi...@google.com>
Authored: Fri Jun 24 16:19:40 2016 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Jul 6 10:18:53 2016 -0700

----------------------------------------------------------------------
 .../beam/examples/complete/AutoComplete.java    |   4 +-
 .../examples/cookbook/DatastoreWordCount.java   |  12 +-
 .../dataflow/io/DataflowDatastoreIOTest.java    |  18 +-
 .../org/apache/beam/sdk/io/DatastoreIO.java     | 937 ------------------
 .../beam/sdk/io/datastore/DatastoreIO.java      |  41 +
 .../apache/beam/sdk/io/datastore/V1Beta3.java   | 992 +++++++++++++++++++
 .../beam/sdk/io/datastore/package-info.java     |  24 +
 .../org/apache/beam/sdk/io/DatastoreIOTest.java | 594 -----------
 .../beam/sdk/io/datastore/V1Beta3Test.java      | 584 +++++++++++
 9 files changed, 1659 insertions(+), 1547 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/d1e38443/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java b/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java
index f278ce3..ca27faa 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java
@@ -28,9 +28,9 @@ import org.apache.beam.sdk.PipelineResult;
 import org.apache.beam.sdk.coders.AvroCoder;
 import org.apache.beam.sdk.coders.DefaultCoder;
 import org.apache.beam.sdk.io.BigQueryIO;
-import org.apache.beam.sdk.io.DatastoreIO;
 import org.apache.beam.sdk.io.PubsubIO;
 import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.io.datastore.DatastoreIO;
 import org.apache.beam.sdk.options.Default;
 import org.apache.beam.sdk.options.Description;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
@@ -482,7 +482,7 @@ public class AutoComplete {
     if (options.getOutputToDatastore()) {
       toWrite
       .apply("FormatForDatastore", ParDo.of(new FormatForDatastore(options.getKind())))
-      .apply(DatastoreIO.writeTo(MoreObjects.firstNonNull(
+      .apply(DatastoreIO.v1beta3().write().withProjectId(MoreObjects.firstNonNull(
           options.getOutputProject(), options.getProject())));
     }
     if (options.getOutputToBigQuery()) {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/d1e38443/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java
index b070f94..36af202 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java
@@ -24,9 +24,9 @@ import static com.google.datastore.v1beta3.client.DatastoreHelper.makeValue;
 
 import org.apache.beam.examples.WordCount;
 import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.io.DatastoreIO;
-import org.apache.beam.sdk.io.Read;
 import org.apache.beam.sdk.io.TextIO;
+import org.apache.beam.sdk.io.datastore.DatastoreIO;
+import org.apache.beam.sdk.io.datastore.V1Beta3;
 import org.apache.beam.sdk.options.Default;
 import org.apache.beam.sdk.options.Description;
 import org.apache.beam.sdk.options.PipelineOptions;
@@ -195,7 +195,7 @@ public class DatastoreWordCount {
       Pipeline p = Pipeline.create(options);
       p.apply("ReadLines", TextIO.Read.from(options.getInput()))
        .apply(ParDo.of(new CreateEntityFn(options.getNamespace(), options.getKind())))
-       .apply(DatastoreIO.writeTo(options.getProject()));
+       .apply(DatastoreIO.v1beta3().write().withProjectId(options.getProject()));
 
       p.run();
   }
@@ -226,13 +226,13 @@ public class DatastoreWordCount {
     Query query = makeAncestorKindQuery(options);
 
     // For Datastore sources, the read namespace can be set on the entire query.
-    DatastoreIO.Source source = DatastoreIO.source()
-        .withProject(options.getProject())
+    V1Beta3.Read read = DatastoreIO.v1beta3().read()
+        .withProjectId(options.getProject())
         .withQuery(query)
         .withNamespace(options.getNamespace());
 
     Pipeline p = Pipeline.create(options);
-    p.apply("ReadShakespeareFromDatastore", Read.from(source))
+    p.apply("ReadShakespeareFromDatastore", read)
         .apply("StringifyEntity", ParDo.of(new GetContentFn()))
         .apply("CountWords", new WordCount.CountWords())
         .apply("PrintWordCount", MapElements.via(new WordCount.FormatAsTextFn()))

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/d1e38443/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/io/DataflowDatastoreIOTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/io/DataflowDatastoreIOTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/io/DataflowDatastoreIOTest.java
index e7c0791..8cdf611 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/io/DataflowDatastoreIOTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/io/DataflowDatastoreIOTest.java
@@ -23,12 +23,13 @@ import static org.hamcrest.Matchers.hasItem;
 import static org.junit.Assert.assertThat;
 
 import org.apache.beam.runners.dataflow.transforms.DataflowDisplayDataEvaluator;
-import org.apache.beam.sdk.io.DatastoreIO;
+import org.apache.beam.sdk.io.datastore.DatastoreIO;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.display.DisplayData;
 import org.apache.beam.sdk.transforms.display.DisplayDataEvaluator;
+import org.apache.beam.sdk.values.PBegin;
 import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
 
 import com.google.datastore.v1beta3.Entity;
 import com.google.datastore.v1beta3.Query;
@@ -44,21 +45,22 @@ public class DataflowDatastoreIOTest {
   @Test
   public void testSourcePrimitiveDisplayData() {
     DisplayDataEvaluator evaluator = DataflowDisplayDataEvaluator.create();
-    PTransform<PInput, ?> read = DatastoreIO.readFrom(
-        "myProject", Query.newBuilder().build());
+    PTransform<PBegin, ? extends POutput> read = DatastoreIO.v1beta3().read().withProjectId(
+        "myProject").withQuery(Query.newBuilder().build());
 
-    Set<DisplayData> displayData = evaluator.displayDataForPrimitiveTransforms(read);
+    Set<DisplayData> displayData = evaluator.displayDataForPrimitiveSourceTransforms(read);
     assertThat("DatastoreIO read should include the project in its primitive display data",
-        displayData, hasItem(hasDisplayItem("project")));
+        displayData, hasItem(hasDisplayItem("projectId")));
   }
 
   @Test
   public void testSinkPrimitiveDisplayData() {
     DisplayDataEvaluator evaluator = DataflowDisplayDataEvaluator.create();
-    PTransform<PCollection<Entity>, ?> write = DatastoreIO.writeTo("myProject");
+    PTransform<PCollection<Entity>, ?> write =
+        DatastoreIO.v1beta3().write().withProjectId("myProject");
 
     Set<DisplayData> displayData = evaluator.displayDataForPrimitiveTransforms(write);
     assertThat("DatastoreIO write should include the project in its primitive display data",
-        displayData, hasItem(hasDisplayItem("project")));
+        displayData, hasItem(hasDisplayItem("projectId")));
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/d1e38443/sdks/java/core/src/main/java/org/apache/beam/sdk/io/DatastoreIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/DatastoreIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/DatastoreIO.java
deleted file mode 100644
index 7abda71..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/DatastoreIO.java
+++ /dev/null
@@ -1,937 +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;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkNotNull;
-import static com.google.common.base.Verify.verify;
-import static com.google.datastore.v1beta3.PropertyFilter.Operator.EQUAL;
-import static com.google.datastore.v1beta3.PropertyOrder.Direction.DESCENDING;
-import static com.google.datastore.v1beta3.QueryResultBatch.MoreResultsType.NOT_FINISHED;
-import static com.google.datastore.v1beta3.client.DatastoreHelper.makeAndFilter;
-import static com.google.datastore.v1beta3.client.DatastoreHelper.makeFilter;
-import static com.google.datastore.v1beta3.client.DatastoreHelper.makeOrder;
-import static com.google.datastore.v1beta3.client.DatastoreHelper.makeUpsert;
-import static com.google.datastore.v1beta3.client.DatastoreHelper.makeValue;
-
-import org.apache.beam.sdk.annotations.Experimental;
-import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.sdk.coders.SerializableCoder;
-import org.apache.beam.sdk.coders.protobuf.ProtoCoder;
-import org.apache.beam.sdk.io.Sink.WriteOperation;
-import org.apache.beam.sdk.io.Sink.Writer;
-import org.apache.beam.sdk.options.GcpOptions;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.transforms.display.DisplayData;
-import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff;
-import org.apache.beam.sdk.util.RetryHttpRequestInitializer;
-import org.apache.beam.sdk.values.PCollection;
-
-import com.google.api.client.auth.oauth2.Credential;
-import com.google.api.client.util.BackOff;
-import com.google.api.client.util.BackOffUtils;
-import com.google.api.client.util.Sleeper;
-import com.google.common.base.MoreObjects;
-import com.google.common.collect.ImmutableList;
-import com.google.common.primitives.Ints;
-import com.google.datastore.v1beta3.CommitRequest;
-import com.google.datastore.v1beta3.Entity;
-import com.google.datastore.v1beta3.EntityResult;
-import com.google.datastore.v1beta3.Key;
-import com.google.datastore.v1beta3.Key.PathElement;
-import com.google.datastore.v1beta3.PartitionId;
-import com.google.datastore.v1beta3.Query;
-import com.google.datastore.v1beta3.QueryResultBatch;
-import com.google.datastore.v1beta3.RunQueryRequest;
-import com.google.datastore.v1beta3.RunQueryResponse;
-import com.google.datastore.v1beta3.client.Datastore;
-import com.google.datastore.v1beta3.client.DatastoreException;
-import com.google.datastore.v1beta3.client.DatastoreFactory;
-import com.google.datastore.v1beta3.client.DatastoreHelper;
-import com.google.datastore.v1beta3.client.DatastoreOptions;
-import com.google.datastore.v1beta3.client.QuerySplitter;
-import com.google.protobuf.Int32Value;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import java.util.NoSuchElementException;
-
-import javax.annotation.Nullable;
-
-/**
- * <p>{@link DatastoreIO} provides an API to Read and Write {@link PCollection PCollections} of
- * <a href="https://developers.google.com/datastore/">Google Cloud Datastore</a>
- * {@link Entity} objects.
- *
- * <p>Google Cloud Datastore is a fully managed NoSQL data storage service.
- * An {@code Entity} is an object in Datastore, analogous to a row in traditional
- * database table.
- *
- * <p>This API currently requires an authentication workaround. To use {@link DatastoreIO}, users
- * must use the {@code gcloud} command line tool to get credentials for Datastore:
- * <pre>
- * $ gcloud auth login
- * </pre>
- *
- * <p>To read a {@link PCollection} from a query to Datastore, use {@link DatastoreIO#source} and
- * its methods {@link DatastoreIO.Source#withProject} and {@link DatastoreIO.Source#withQuery} to
- * specify the project to query and the query to read from. You can optionally provide a namespace
- * to query within using {@link DatastoreIO.Source#withNamespace}.
- *
- * <p>For example:
- *
- * <pre> {@code
- * // Read a query from Datastore
- * PipelineOptions options = PipelineOptionsFactory.fromArgs(args).create();
- * Query query = ...;
- * String projectId = "...";
- *
- * Pipeline p = Pipeline.create(options);
- * PCollection<Entity> entities = p.apply(
- *     Read.from(DatastoreIO.source()
- *         .withProject(projectId)
- *         .withQuery(query));
- * } </pre>
- *
- * <p>or:
- *
- * <pre> {@code
- * // Read a query from Datastore using the default namespace
- * PipelineOptions options = PipelineOptionsFactory.fromArgs(args).create();
- * Query query = ...;
- * String projectId = "...";
- *
- * Pipeline p = Pipeline.create(options);
- * PCollection<Entity> entities = p.apply(DatastoreIO.readFrom(projectId, query));
- * p.run();
- * } </pre>
- *
- * <p><b>Note:</b> Normally, a Cloud Dataflow job will read from Cloud Datastore in parallel across
- * many workers. However, when the {@link Query} is configured with a limit using
- * {@link com.google.datastore.v1beta3.Query.Builder#setLimit(Int32Value)}, then
- * all returned results will be read by a single Dataflow worker in order to ensure correct data.
- *
- * <p>To write a {@link PCollection} to a Datastore, use {@link DatastoreIO#writeTo},
- * specifying the datastore to write to:
- *
- * <pre> {@code
- * PCollection<Entity> entities = ...;
- * entities.apply(DatastoreIO.writeTo(projectId));
- * p.run();
- * } </pre>
- *
- * <p>{@link Entity Entities} in the {@code PCollection} to be written must have complete
- * {@link Key Keys}. Complete {@code Keys} specify the {@code name} and {@code id} of the
- * {@code Entity}, where incomplete {@code Keys} do not. A {@code namespace} other than the
- * project default may be written to by specifying it in the {@code Entity} {@code Keys}.
- *
- * <pre>{@code
- * Key.Builder keyBuilder = DatastoreHelper.makeKey(...);
- * keyBuilder.getPartitionIdBuilder().setNamespace(namespace);
- * }</pre>
- *
- * <p>{@code Entities} will be committed as upsert (update or insert) mutations. Please read
- * <a href="https://cloud.google.com/datastore/docs/concepts/entities">Entities, Properties, and
- * Keys</a> for more information about {@code Entity} keys.
- *
- * <p><h3>Permissions</h3>
- * Permission requirements depend on the {@code PipelineRunner} that is used to execute the
- * Dataflow job. Please refer to the documentation of corresponding {@code PipelineRunner}s for
- * more details.
- *
- * <p>Please see <a href="https://cloud.google.com/datastore/docs/activate">Cloud Datastore Sign Up
- * </a>for security and permission related information specific to Datastore.
- *
- * @see org.apache.beam.sdk.runners.PipelineRunner
- */
-@Experimental(Experimental.Kind.SOURCE_SINK)
-public class DatastoreIO {
-  /**
-   * Datastore has a limit of 500 mutations per batch operation, so we flush
-   * changes to Datastore every 500 entities.
-   */
-  public static final int DATASTORE_BATCH_UPDATE_LIMIT = 500;
-
-  /**
-   * Returns an empty {@link DatastoreIO.Source} builder.
-   * Configure the {@code project}, {@code query}, and {@code namespace} using
-   * {@link DatastoreIO.Source#withProject}, {@link DatastoreIO.Source#withQuery},
-   * and {@link DatastoreIO.Source#withNamespace}.
-   *
-   * @deprecated the name and return type do not match. Use {@link #source()}.
-   */
-  @Deprecated
-  public static Source read() {
-    return source();
-  }
-
-  /**
-   * Returns an empty {@link DatastoreIO.Source} builder.
-   * Configure the {@code project}, {@code query}, and {@code namespace} using
-   * {@link DatastoreIO.Source#withProject}, {@link DatastoreIO.Source#withQuery},
-   * and {@link DatastoreIO.Source#withNamespace}.
-   *
-   * <p>The resulting {@link Source} object can be passed to {@link Read} to create a
-   * {@code PTransform} that will read from Datastore.
-   */
-  public static Source source() {
-    return new Source(null, null, null);
-  }
-
-  /**
-   * Returns a {@code PTransform} that reads Datastore entities from the query
-   * against the given project.
-   */
-  public static Read.Bounded<Entity> readFrom(String projectId, Query query) {
-    return Read.from(new Source(projectId, query, null));
-  }
-
-  /**
-   * A {@link Source} that reads the result rows of a Datastore query as {@code Entity} objects.
-   */
-  public static class Source extends BoundedSource<Entity> {
-    public String getProjectId() {
-      return projectId;
-    }
-
-    public Query getQuery() {
-      return query;
-    }
-
-    @Nullable
-    public String getNamespace() {
-      return namespace;
-    }
-
-    public Source withProject(String projectId) {
-      checkNotNull(projectId, "projectId");
-      return new Source(projectId, query, namespace);
-    }
-
-    /**
-     * Returns a new {@link Source} that reads the results of the specified query.
-     *
-     * <p>Does not modify this object.
-     *
-     * <p><b>Note:</b> Normally, a Cloud Dataflow job will read from Cloud Datastore in parallel
-     * across many workers. However, when the {@link Query} is configured with a limit using
-     * {@link com.google.datastore.v1beta3.Query.Builder#setLimit(Int32Value)}, then all
-     * returned results will be read by a single Dataflow worker in order to ensure correct data.
-     */
-    public Source withQuery(Query query) {
-      checkNotNull(query, "query");
-      checkArgument(!query.hasLimit() || query.getLimit().getValue() > 0,
-          "Invalid query limit %s: must be positive", query.getLimit().getValue());
-      return new Source(projectId, query, namespace);
-    }
-
-    public Source withNamespace(@Nullable String namespace) {
-      return new Source(projectId, query, namespace);
-    }
-
-    @Override
-    public Coder<Entity> getDefaultOutputCoder() {
-      return ProtoCoder.of(Entity.class);
-    }
-
-    @Override
-    public boolean producesSortedKeys(PipelineOptions options) {
-      // TODO: Perhaps this can be implemented by inspecting the query.
-      return false;
-    }
-
-    @Override
-    public List<Source> splitIntoBundles(long desiredBundleSizeBytes, PipelineOptions options)
-        throws Exception {
-      // Users may request a limit on the number of results. We can currently support this by
-      // simply disabling parallel reads and using only a single split.
-      if (query.hasLimit()) {
-        return ImmutableList.of(this);
-      }
-
-      long numSplits;
-      try {
-        numSplits = Math.round(((double) getEstimatedSizeBytes(options)) / desiredBundleSizeBytes);
-      } catch (Exception e) {
-        // Fallback in case estimated size is unavailable. TODO: fix this, it's horrible.
-        numSplits = 12;
-      }
-
-      // If the desiredBundleSize or number of workers results in 1 split, simply return
-      // a source that reads from the original query.
-      if (numSplits <= 1) {
-        return ImmutableList.of(this);
-      }
-
-      List<Query> datastoreSplits;
-      try {
-        datastoreSplits = getSplitQueries(Ints.checkedCast(numSplits), options);
-      } catch (IllegalArgumentException | DatastoreException e) {
-        LOG.warn("Unable to parallelize the given query: {}", query, e);
-        return ImmutableList.of(this);
-      }
-
-      ImmutableList.Builder<Source> splits = ImmutableList.builder();
-      for (Query splitQuery : datastoreSplits) {
-        splits.add(new Source(projectId, splitQuery, namespace));
-      }
-      return splits.build();
-    }
-
-    @Override
-    public BoundedReader<Entity> createReader(PipelineOptions pipelineOptions) throws IOException {
-      return new DatastoreReader(this, getDatastore(pipelineOptions));
-    }
-
-    @Override
-    public void validate() {
-      checkNotNull(query, "query");
-      checkNotNull(projectId, "projectId");
-    }
-
-    @Override
-    public long getEstimatedSizeBytes(PipelineOptions options) throws Exception {
-      // Datastore provides no way to get a good estimate of how large the result of a query
-      // will be. As a rough approximation, we attempt to fetch the statistics of the whole
-      // entity kind being queried, using the __Stat_Kind__ system table, assuming exactly 1 kind
-      // is specified in the query.
-      //
-      // See https://cloud.google.com/datastore/docs/concepts/stats
-      if (mockEstimateSizeBytes != null) {
-        return mockEstimateSizeBytes;
-      }
-
-      Datastore datastore = getDatastore(options);
-      if (query.getKindCount() != 1) {
-        throw new UnsupportedOperationException(
-            "Can only estimate size for queries specifying exactly 1 kind.");
-      }
-      String ourKind = query.getKind(0).getName();
-      long latestTimestamp = queryLatestStatisticsTimestamp(datastore);
-      Query.Builder query = Query.newBuilder();
-      if (namespace == null) {
-        query.addKindBuilder().setName("__Stat_Kind__");
-      } else {
-        query.addKindBuilder().setName("__Ns_Stat_Kind__");
-      }
-      query.setFilter(makeAndFilter(
-          makeFilter("kind_name", EQUAL, makeValue(ourKind)).build(),
-          makeFilter("timestamp", EQUAL, makeValue(latestTimestamp)).build()));
-      RunQueryRequest request = makeRequest(query.build());
-
-      long now = System.currentTimeMillis();
-      RunQueryResponse response = datastore.runQuery(request);
-      LOG.info("Query for per-kind statistics took {}ms", System.currentTimeMillis() - now);
-
-      QueryResultBatch batch = response.getBatch();
-      if (batch.getEntityResultsCount() == 0) {
-        throw new NoSuchElementException(
-            "Datastore statistics for kind " + ourKind + " unavailable");
-      }
-      Entity entity = batch.getEntityResults(0).getEntity();
-      return entity.getProperties().get("entity_bytes").getIntegerValue();
-    }
-
-    @Override
-    public void populateDisplayData(DisplayData.Builder builder) {
-      super.populateDisplayData(builder);
-      builder
-          .addIfNotNull(DisplayData.item("project", projectId)
-            .withLabel("Input Project"))
-          .addIfNotNull(DisplayData.item("namespace", namespace)
-            .withLabel("App Engine Namespace"));
-
-      if (query != null) {
-        builder.add(DisplayData.item("query", query.toString())
-          .withLabel("Query"));
-      }
-    }
-
-    @Override
-    public String toString() {
-      return MoreObjects.toStringHelper(getClass())
-          .add("project", projectId)
-          .add("query", query)
-          .add("namespace", namespace)
-          .toString();
-    }
-
-    ///////////////////////////////////////////////////////////////////////////////////////////
-
-    private static final Logger LOG = LoggerFactory.getLogger(Source.class);
-    /** Not really nullable, but it may be {@code null} for in-progress {@code Source}s. */
-    @Nullable
-    private final String projectId;
-    /** Not really nullable, but it may be {@code null} for in-progress {@code Source}s. */
-    @Nullable
-    private final Query query;
-    @Nullable
-    private final String namespace;
-
-    /** For testing only. TODO: This could be much cleaner with dependency injection. */
-    @Nullable
-    private QuerySplitter mockSplitter;
-    @Nullable
-    private Long mockEstimateSizeBytes;
-
-    /**
-     * Note that only {@code namespace} is really {@code @Nullable}. The other parameters may be
-     * {@code null} as a matter of build order, but if they are {@code null} at instantiation time,
-     * an error will be thrown.
-     */
-    private Source(
-        @Nullable String projectId, @Nullable Query query,
-        @Nullable String namespace) {
-      this.projectId = projectId;
-      this.query = query;
-      this.namespace = namespace;
-    }
-
-    /**
-     * A helper function to get the split queries, taking into account the optional
-     * {@code namespace} and whether there is a mock splitter.
-     */
-    private List<Query> getSplitQueries(int numSplits, PipelineOptions options)
-        throws DatastoreException {
-      // If namespace is set, include it in the split request so splits are calculated accordingly.
-      PartitionId.Builder partitionBuilder = PartitionId.newBuilder();
-      if (namespace != null) {
-        partitionBuilder.setNamespaceId(namespace);
-      }
-
-      if (mockSplitter != null) {
-        // For testing.
-        return mockSplitter.getSplits(query, partitionBuilder.build(), numSplits, null);
-      }
-
-      return DatastoreHelper.getQuerySplitter().getSplits(
-          query, partitionBuilder.build(), numSplits, getDatastore(options));
-    }
-
-    /**
-     * Builds a {@link RunQueryRequest} from the {@code query}, using the properties set on this
-     * {@code Source}. For example, sets the {@code namespace} for the request.
-     */
-    private RunQueryRequest makeRequest(Query query) {
-      RunQueryRequest.Builder requestBuilder = RunQueryRequest.newBuilder().setQuery(query);
-      if (namespace != null) {
-        requestBuilder.getPartitionIdBuilder().setNamespaceId(namespace);
-      }
-      return requestBuilder.build();
-    }
-
-    /**
-     * Datastore system tables with statistics are periodically updated. This method fetches
-     * the latest timestamp of statistics update using the {@code __Stat_Total__} table.
-     */
-    private long queryLatestStatisticsTimestamp(Datastore datastore) throws DatastoreException {
-      Query.Builder query = Query.newBuilder();
-      query.addKindBuilder().setName("__Stat_Total__");
-      query.addOrder(makeOrder("timestamp", DESCENDING));
-      query.setLimit(Int32Value.newBuilder().setValue(1));
-      RunQueryRequest request = makeRequest(query.build());
-
-      long now = System.currentTimeMillis();
-      RunQueryResponse response = datastore.runQuery(request);
-      LOG.info("Query for latest stats timestamp of project {} took {}ms", projectId,
-          System.currentTimeMillis() - now);
-      QueryResultBatch batch = response.getBatch();
-      if (batch.getEntityResultsCount() == 0) {
-        throw new NoSuchElementException(
-            "Datastore total statistics for project " + projectId + " unavailable");
-      }
-      Entity entity = batch.getEntityResults(0).getEntity();
-      return entity.getProperties().get("timestamp").getTimestampValue().getNanos();
-    }
-
-    private Datastore getDatastore(PipelineOptions pipelineOptions) {
-      DatastoreOptions.Builder builder =
-          new DatastoreOptions.Builder()
-              .projectId(projectId)
-              .initializer(
-                new RetryHttpRequestInitializer()
-              );
-
-      Credential credential = pipelineOptions.as(GcpOptions.class).getGcpCredential();
-      if (credential != null) {
-        builder.credential(credential);
-      }
-      return DatastoreFactory.get().create(builder.build());
-    }
-
-    /** For testing only. */
-    Source withMockSplitter(QuerySplitter splitter) {
-      Source res = new Source(projectId, query, namespace);
-      res.mockSplitter = splitter;
-      res.mockEstimateSizeBytes = mockEstimateSizeBytes;
-      return res;
-    }
-
-    /** For testing only. */
-    Source withMockEstimateSizeBytes(Long estimateSizeBytes) {
-      Source res = new Source(projectId, query, namespace);
-      res.mockSplitter = mockSplitter;
-      res.mockEstimateSizeBytes = estimateSizeBytes;
-      return res;
-    }
-  }
-
-  ///////////////////// Write Class /////////////////////////////////
-
-  /**
-   * Returns a new {@link DatastoreIO.Sink} builder.
-   * You need to further configure it using {@link DatastoreIO.Sink#withProject}, before using it
-   * in a {@link Write} transform.
-   *
-   * <p>For example: {@code p.apply(Write.to(DatastoreIO.sink().withProject(projectId)));}
-   */
-  public static Sink sink() {
-    return new Sink(null);
-  }
-
-  /**
-   * Returns a new {@link Write} transform that will write to a {@link Sink}.
-   *
-   * <p>For example: {@code p.apply(DatastoreIO.writeTo(projectId));}
-   */
-  public static Write.Bound<Entity> writeTo(String projectId) {
-    return Write.to(sink().withProject(projectId));
-  }
-
-  /**
-   * A {@link Sink} that writes a {@link PCollection} containing
-   * {@link Entity Entities} to a Datastore kind.
-   *
-   */
-  public static class Sink extends org.apache.beam.sdk.io.Sink<Entity> {
-    final String projectId;
-
-    /**
-     * Returns a {@link Sink} that is like this one, but will write to the specified project.
-     */
-    public Sink withProject(String projectId) {
-      checkNotNull(projectId, "projectId");
-      return new Sink(projectId);
-    }
-
-    /**
-     * Constructs a Sink with the given project.
-     */
-    protected Sink(String projectId) {
-      this.projectId = projectId;
-    }
-
-    /**
-     * Ensures the project is set.
-     */
-    @Override
-    public void validate(PipelineOptions options) {
-      checkNotNull(
-          projectId,
-          "Project ID is a required parameter. Please use withProject to to set the projectId.");
-    }
-
-    @Override
-    public DatastoreWriteOperation createWriteOperation(PipelineOptions options) {
-      return new DatastoreWriteOperation(this);
-    }
-
-    @Override
-    public void populateDisplayData(DisplayData.Builder builder) {
-      super.populateDisplayData(builder);
-      builder
-          .addIfNotNull(DisplayData.item("project", projectId)
-            .withLabel("Output Project"));
-    }
-  }
-
-  /**
-   * A {@link WriteOperation} that will manage a parallel write to a Datastore sink.
-   */
-  private static class DatastoreWriteOperation
-      extends WriteOperation<Entity, DatastoreWriteResult> {
-    private static final Logger LOG = LoggerFactory.getLogger(DatastoreWriteOperation.class);
-
-    private final DatastoreIO.Sink sink;
-
-    public DatastoreWriteOperation(DatastoreIO.Sink sink) {
-      this.sink = sink;
-    }
-
-    @Override
-    public Coder<DatastoreWriteResult> getWriterResultCoder() {
-      return SerializableCoder.of(DatastoreWriteResult.class);
-    }
-
-    @Override
-    public void initialize(PipelineOptions options) throws Exception {}
-
-    /**
-     * Finalizes the write.  Logs the number of entities written to the Datastore.
-     */
-    @Override
-    public void finalize(Iterable<DatastoreWriteResult> writerResults, PipelineOptions options)
-        throws Exception {
-      long totalEntities = 0;
-      for (DatastoreWriteResult result : writerResults) {
-        totalEntities += result.entitiesWritten;
-      }
-      LOG.info("Wrote {} elements.", totalEntities);
-    }
-
-    @Override
-    public DatastoreWriter createWriter(PipelineOptions options) throws Exception {
-      DatastoreOptions.Builder builder =
-          new DatastoreOptions.Builder()
-              .projectId(sink.projectId)
-              .initializer(new RetryHttpRequestInitializer());
-      Credential credential = options.as(GcpOptions.class).getGcpCredential();
-      if (credential != null) {
-        builder.credential(credential);
-      }
-      Datastore datastore = DatastoreFactory.get().create(builder.build());
-
-      return new DatastoreWriter(this, datastore);
-    }
-
-    @Override
-    public DatastoreIO.Sink getSink() {
-      return sink;
-    }
-  }
-
-  /**
-   * {@link Writer} that writes entities to a Datastore Sink.  Entities are written in batches,
-   * where the maximum batch size is {@link DatastoreIO#DATASTORE_BATCH_UPDATE_LIMIT}.  Entities
-   * are committed as upsert mutations (either update if the key already exists, or insert if it is
-   * a new key).  If an entity does not have a complete key (i.e., it has no name or id), the bundle
-   * will fail.
-   *
-   * <p>See <a
-   * href="https://cloud.google.com/datastore/docs/concepts/entities#Datastore_Creating_an_entity">
-   * Datastore: Entities, Properties, and Keys</a> for information about entity keys and upsert
-   * mutations.
-   *
-   * <p>Commits are non-transactional.  If a commit fails because of a conflict over an entity
-   * group, the commit will be retried (up to {@link DatastoreIO#DATASTORE_BATCH_UPDATE_LIMIT}
-   * times).
-   *
-   * <p>Visible for testing purposes.
-   */
-  static class DatastoreWriter extends Writer<Entity, DatastoreWriteResult> {
-    private static final Logger LOG = LoggerFactory.getLogger(DatastoreWriter.class);
-    private final DatastoreWriteOperation writeOp;
-    private final Datastore datastore;
-    private long totalWritten = 0;
-
-    // Visible for testing.
-    final List<Entity> entities = new ArrayList<>();
-
-    /**
-     * Since a bundle is written in batches, we should retry the commit of a batch in order to
-     * prevent transient errors from causing the bundle to fail.
-     */
-    private static final int MAX_RETRIES = 5;
-
-    /**
-     * Initial backoff time for exponential backoff for retry attempts.
-     */
-    private static final int INITIAL_BACKOFF_MILLIS = 5000;
-
-    /**
-     * Returns true if a Datastore key is complete.  A key is complete if its last element
-     * has either an id or a name.
-     */
-    static boolean isValidKey(Key key) {
-      List<PathElement> elementList = key.getPathList();
-      if (elementList.isEmpty()) {
-        return false;
-      }
-      PathElement lastElement = elementList.get(elementList.size() - 1);
-      return (lastElement.getId() != 0 || !lastElement.getName().isEmpty());
-    }
-
-    // Visible for testing
-    DatastoreWriter(DatastoreWriteOperation writeOp, Datastore datastore) {
-      this.writeOp = writeOp;
-      this.datastore = datastore;
-    }
-
-    @Override
-    public void open(String uId) throws Exception {}
-
-    /**
-     * Writes an entity to the Datastore.  Writes are batched, up to {@link
-     * DatastoreIO#DATASTORE_BATCH_UPDATE_LIMIT}. If an entity does not have a complete key, an
-     * {@link IllegalArgumentException} will be thrown.
-     */
-    @Override
-    public void write(Entity value) throws Exception {
-      // Verify that the entity to write has a complete key.
-      if (!isValidKey(value.getKey())) {
-        throw new IllegalArgumentException(
-            "Entities to be written to the Datastore must have complete keys");
-      }
-
-      entities.add(value);
-
-      if (entities.size() >= DatastoreIO.DATASTORE_BATCH_UPDATE_LIMIT) {
-        flushBatch();
-      }
-    }
-
-    /**
-     * Flushes any pending batch writes and returns a DatastoreWriteResult.
-     */
-    @Override
-    public DatastoreWriteResult close() throws Exception {
-      if (entities.size() > 0) {
-        flushBatch();
-      }
-      return new DatastoreWriteResult(totalWritten);
-    }
-
-    @Override
-    public DatastoreWriteOperation getWriteOperation() {
-      return writeOp;
-    }
-
-    /**
-     * Writes a batch of entities to the Datastore.
-     *
-     * <p>If a commit fails, it will be retried (up to {@link DatastoreWriter#MAX_RETRIES}
-     * times).  All entities in the batch will be committed again, even if the commit was partially
-     * successful. If the retry limit is exceeded, the last exception from the Datastore will be
-     * thrown.
-     *
-     * @throws DatastoreException if the commit fails or IOException or InterruptedException if
-     * backing off between retries fails.
-     */
-    private void flushBatch() throws DatastoreException, IOException, InterruptedException {
-      LOG.debug("Writing batch of {} entities", entities.size());
-      Sleeper sleeper = Sleeper.DEFAULT;
-      BackOff backoff = new AttemptBoundedExponentialBackOff(MAX_RETRIES, INITIAL_BACKOFF_MILLIS);
-
-      while (true) {
-        // Batch upsert entities.
-        try {
-          CommitRequest.Builder commitRequest = CommitRequest.newBuilder();
-          for (Entity entity: entities) {
-            commitRequest.addMutations(makeUpsert(entity));
-          }
-          commitRequest.setMode(CommitRequest.Mode.NON_TRANSACTIONAL);
-          datastore.commit(commitRequest.build());
-          // Break if the commit threw no exception.
-          break;
-        } catch (DatastoreException exception) {
-          // Only log the code and message for potentially-transient errors. The entire exception
-          // will be propagated upon the last retry.
-          LOG.error("Error writing to the Datastore ({}): {}", exception.getCode(),
-              exception.getMessage());
-          if (!BackOffUtils.next(sleeper, backoff)) {
-            LOG.error("Aborting after {} retries.", MAX_RETRIES);
-            throw exception;
-          }
-        }
-      }
-      totalWritten += entities.size();
-      LOG.debug("Successfully wrote {} entities", entities.size());
-      entities.clear();
-    }
-  }
-
-  private static class DatastoreWriteResult implements Serializable {
-    final long entitiesWritten;
-
-    public DatastoreWriteResult(long recordsWritten) {
-      this.entitiesWritten = recordsWritten;
-    }
-  }
-
-  /**
-   * A {@link Source.Reader} over the records from a query of the datastore.
-   *
-   * <p>Timestamped records are currently not supported.
-   * All records implicitly have the timestamp of {@code BoundedWindow.TIMESTAMP_MIN_VALUE}.
-   */
-  public static class DatastoreReader extends BoundedSource.BoundedReader<Entity> {
-    private final Source source;
-
-    /**
-     * Datastore to read from.
-     */
-    private final Datastore datastore;
-
-    /**
-     * True if more results may be available.
-     */
-    private boolean moreResults;
-
-    /**
-     * Iterator over records.
-     */
-    private java.util.Iterator<EntityResult> entities;
-
-    /**
-     * Current batch of query results.
-     */
-    private QueryResultBatch currentBatch;
-
-    /**
-     * Maximum number of results to request per query.
-     *
-     * <p>Must be set, or it may result in an I/O error when querying
-     * Cloud Datastore.
-     */
-    private static final int QUERY_BATCH_LIMIT = 500;
-
-    /**
-     * Remaining user-requested limit on the number of sources to return. If the user did not set a
-     * limit, then this variable will always have the value {@link Integer#MAX_VALUE} and will never
-     * be decremented.
-     */
-    private int userLimit;
-
-    private volatile boolean done = false;
-
-    private Entity currentEntity;
-
-    /**
-     * Returns a DatastoreReader with Source and Datastore object set.
-     *
-     * @param datastore a datastore connection to use.
-     */
-    public DatastoreReader(Source source, Datastore datastore) {
-      this.source = source;
-      this.datastore = datastore;
-      // If the user set a limit on the query, remember it. Otherwise pin to MAX_VALUE.
-      userLimit = source.query.hasLimit()
-          ? source.query.getLimit().getValue() : Integer.MAX_VALUE;
-    }
-
-    @Override
-    public Entity getCurrent() {
-      return currentEntity;
-    }
-
-    @Override
-    public final long getSplitPointsConsumed() {
-      return done ? 1 : 0;
-    }
-
-    @Override
-    public final long getSplitPointsRemaining() {
-      return done ? 0 : 1;
-    }
-
-    @Override
-    public boolean start() throws IOException {
-      return advance();
-    }
-
-    @Override
-    public boolean advance() throws IOException {
-      if (entities == null || (!entities.hasNext() && moreResults)) {
-        try {
-          entities = getIteratorAndMoveCursor();
-        } catch (DatastoreException e) {
-          throw new IOException(e);
-        }
-      }
-
-      if (entities == null || !entities.hasNext()) {
-        currentEntity = null;
-        done = true;
-        return false;
-      }
-
-      currentEntity = entities.next().getEntity();
-      return true;
-    }
-
-    @Override
-    public void close() throws IOException {
-      // Nothing
-    }
-
-    @Override
-    public DatastoreIO.Source getCurrentSource() {
-      return source;
-    }
-
-    @Override
-    public DatastoreIO.Source splitAtFraction(double fraction) {
-      // Not supported.
-      return null;
-    }
-
-    @Override
-    public Double getFractionConsumed() {
-      // Not supported.
-      return null;
-    }
-
-    /**
-     * Returns an iterator over the next batch of records for the query
-     * and updates the cursor to get the next batch as needed.
-     * Query has specified limit and offset from InputSplit.
-     */
-    private Iterator<EntityResult> getIteratorAndMoveCursor() throws DatastoreException {
-      Query.Builder query = source.query.toBuilder().clone();
-      query.setLimit(Int32Value.newBuilder().setValue(Math.min(userLimit, QUERY_BATCH_LIMIT)));
-      if (currentBatch != null && !currentBatch.getEndCursor().isEmpty()) {
-        query.setStartCursor(currentBatch.getEndCursor());
-      }
-
-      RunQueryRequest request = source.makeRequest(query.build());
-      RunQueryResponse response = datastore.runQuery(request);
-
-      currentBatch = response.getBatch();
-
-      // MORE_RESULTS_AFTER_LIMIT is not implemented yet:
-      // https://groups.google.com/forum/#!topic/gcd-discuss/iNs6M1jA2Vw, so
-      // use result count to determine if more results might exist.
-      int numFetch = currentBatch.getEntityResultsCount();
-      if (source.query.hasLimit()) {
-        verify(userLimit >= numFetch,
-            "Expected userLimit %s >= numFetch %s, because query limit %s should be <= userLimit",
-            userLimit, numFetch, query.getLimit());
-        userLimit -= numFetch;
-      }
-      moreResults =
-          // User-limit does not exist (so userLimit == MAX_VALUE) and/or has not been satisfied.
-          (userLimit > 0)
-          // All indications from the API are that there are/may be more results.
-          && ((numFetch == QUERY_BATCH_LIMIT) || (currentBatch.getMoreResults() == NOT_FINISHED));
-
-      // May receive a batch of 0 results if the number of records is a multiple
-      // of the request limit.
-      if (numFetch == 0) {
-        return null;
-      }
-
-      return currentBatch.getEntityResultsList().iterator();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/d1e38443/sdks/java/core/src/main/java/org/apache/beam/sdk/io/datastore/DatastoreIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/datastore/DatastoreIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/datastore/DatastoreIO.java
new file mode 100644
index 0000000..d5043f2
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/datastore/DatastoreIO.java
@@ -0,0 +1,41 @@
+/*
+ * 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.datastore;
+
+import org.apache.beam.sdk.annotations.Experimental;
+
+/**
+ * <p>{@link DatastoreIO} provides an API for reading from and writing to
+ * <a href="https://developers.google.com/datastore/">Google Cloud Datastore</a> over different
+ * versions of the Datastore Client libraries.
+ *
+ * <p>To use the v1beta3 version see {@link V1Beta3}.
+ */
+@Experimental(Experimental.Kind.SOURCE_SINK)
+public class DatastoreIO {
+
+  private DatastoreIO() {}
+
+  /**
+   * Returns a {@link V1Beta3} that provides an API for accessing Datastore through v1beta3 version
+   * of Datastore Client library.
+   */
+  public static V1Beta3 v1beta3() {
+    return new V1Beta3();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/d1e38443/sdks/java/core/src/main/java/org/apache/beam/sdk/io/datastore/V1Beta3.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/datastore/V1Beta3.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/datastore/V1Beta3.java
new file mode 100644
index 0000000..0b9f709
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/datastore/V1Beta3.java
@@ -0,0 +1,992 @@
+/*
+ * 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.datastore;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+import static com.google.common.base.Verify.verify;
+import static com.google.datastore.v1beta3.PropertyFilter.Operator.EQUAL;
+import static com.google.datastore.v1beta3.PropertyOrder.Direction.DESCENDING;
+import static com.google.datastore.v1beta3.QueryResultBatch.MoreResultsType.NOT_FINISHED;
+import static com.google.datastore.v1beta3.client.DatastoreHelper.makeAndFilter;
+import static com.google.datastore.v1beta3.client.DatastoreHelper.makeFilter;
+import static com.google.datastore.v1beta3.client.DatastoreHelper.makeOrder;
+import static com.google.datastore.v1beta3.client.DatastoreHelper.makeUpsert;
+import static com.google.datastore.v1beta3.client.DatastoreHelper.makeValue;
+
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.SerializableCoder;
+import org.apache.beam.sdk.coders.protobuf.ProtoCoder;
+import org.apache.beam.sdk.io.BoundedSource;
+import org.apache.beam.sdk.io.Sink.WriteOperation;
+import org.apache.beam.sdk.io.Sink.Writer;
+import org.apache.beam.sdk.options.GcpOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff;
+import org.apache.beam.sdk.util.RetryHttpRequestInitializer;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PDone;
+
+import com.google.api.client.auth.oauth2.Credential;
+import com.google.api.client.util.BackOff;
+import com.google.api.client.util.BackOffUtils;
+import com.google.api.client.util.Sleeper;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.MoreObjects;
+import com.google.common.collect.ImmutableList;
+import com.google.common.primitives.Ints;
+import com.google.datastore.v1beta3.CommitRequest;
+import com.google.datastore.v1beta3.Entity;
+import com.google.datastore.v1beta3.EntityResult;
+import com.google.datastore.v1beta3.Key;
+import com.google.datastore.v1beta3.Key.PathElement;
+import com.google.datastore.v1beta3.PartitionId;
+import com.google.datastore.v1beta3.Query;
+import com.google.datastore.v1beta3.QueryResultBatch;
+import com.google.datastore.v1beta3.RunQueryRequest;
+import com.google.datastore.v1beta3.RunQueryResponse;
+import com.google.datastore.v1beta3.client.Datastore;
+import com.google.datastore.v1beta3.client.DatastoreException;
+import com.google.datastore.v1beta3.client.DatastoreFactory;
+import com.google.datastore.v1beta3.client.DatastoreHelper;
+import com.google.datastore.v1beta3.client.DatastoreOptions;
+import com.google.datastore.v1beta3.client.QuerySplitter;
+import com.google.protobuf.Int32Value;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+
+import javax.annotation.Nullable;
+
+/**
+ * <p>{@link V1Beta3} provides an API to Read and Write {@link PCollection PCollections} of
+ * <a href="https://developers.google.com/datastore/">Google Cloud Datastore</a> version v1beta3
+ * {@link Entity} objects.
+ *
+ * <p>This API currently requires an authentication workaround. To use {@link V1Beta3}, users
+ * must use the {@code gcloud} command line tool to get credentials for Datastore:
+ * <pre>
+ * $ gcloud auth login
+ * </pre>
+ *
+ * <p>To read a {@link PCollection} from a query to Datastore, use {@link V1Beta3#read} and
+ * its methods {@link V1Beta3.Read#withProjectId} and {@link V1Beta3.Read#withQuery} to
+ * specify the project to query and the query to read from. You can optionally provide a namespace
+ * to query within using {@link V1Beta3.Read#withNamespace}.
+ *
+ * <p>For example:
+ *
+ * <pre> {@code
+ * // Read a query from Datastore
+ * PipelineOptions options = PipelineOptionsFactory.fromArgs(args).create();
+ * Query query = ...;
+ * String projectId = "...";
+ *
+ * Pipeline p = Pipeline.create(options);
+ * PCollection<Entity> entities = p.apply(
+ *     DatastoreIO.v1beta3().read()
+ *         .withProjectId(projectId)
+ *         .withQuery(query));
+ * } </pre>
+ *
+ * <p><b>Note:</b> Normally, a Cloud Dataflow job will read from Cloud Datastore in parallel across
+ * many workers. However, when the {@link Query} is configured with a limit using
+ * {@link com.google.datastore.v1beta3.Query.Builder#setLimit(Int32Value)}, then
+ * all returned results will be read by a single Dataflow worker in order to ensure correct data.
+ *
+ * <p>To write a {@link PCollection} to a Datastore, use {@link V1Beta3#write},
+ * specifying the Cloud Datastore project to write to:
+ *
+ * <pre> {@code
+ * PCollection<Entity> entities = ...;
+ * entities.apply(DatastoreIO.v1beta3().write().withProjectId(projectId));
+ * p.run();
+ * } </pre>
+ *
+ * <p>{@link Entity Entities} in the {@code PCollection} to be written must have complete
+ * {@link Key Keys}. Complete {@code Keys} specify the {@code name} and {@code id} of the
+ * {@code Entity}, where incomplete {@code Keys} do not. A {@code namespace} other than
+ * {@code projectId} default may be used by specifying it in the {@code Entity} {@code Keys}.
+ *
+ * <pre>{@code
+ * Key.Builder keyBuilder = DatastoreHelper.makeKey(...);
+ * keyBuilder.getPartitionIdBuilder().setNamespace(namespace);
+ * }</pre>
+ *
+ * <p>{@code Entities} will be committed as upsert (update or insert) mutations. Please read
+ * <a href="https://cloud.google.com/datastore/docs/concepts/entities">Entities, Properties, and
+ * Keys</a> for more information about {@code Entity} keys.
+ *
+ * <p><h3>Permissions</h3>
+ * Permission requirements depend on the {@code PipelineRunner} that is used to execute the
+ * Dataflow job. Please refer to the documentation of corresponding {@code PipelineRunner}s for
+ * more details.
+ *
+ * <p>Please see <a href="https://cloud.google.com/datastore/docs/activate">Cloud Datastore Sign Up
+ * </a>for security and permission related information specific to Datastore.
+ *
+ * @see org.apache.beam.sdk.runners.PipelineRunner
+ */
+@Experimental(Experimental.Kind.SOURCE_SINK)
+public class V1Beta3 {
+
+  // A package-private constructor to prevent direct instantiation from outside of this package
+  V1Beta3() {}
+
+  /**
+   * Datastore has a limit of 500 mutations per batch operation, so we flush
+   * changes to Datastore every 500 entities.
+   */
+  private static final int DATASTORE_BATCH_UPDATE_LIMIT = 500;
+
+  /**
+   * Returns an empty {@link V1Beta3.Read} builder. Configure the source {@code projectId},
+   * {@code query}, and optionally {@code namespace} using {@link V1Beta3.Read#withProjectId},
+   * {@link V1Beta3.Read#withQuery}, and {@link V1Beta3.Read#withNamespace}.
+   */
+  public V1Beta3.Read read() {
+    return new V1Beta3.Read(null, null, null);
+  }
+
+  /**
+   * A {@link PTransform} that reads the result rows of a Datastore query as {@code Entity}
+   * objects.
+   *
+   * @see DatastoreIO
+   */
+  public static class Read extends PTransform<PBegin, PCollection<Entity>> {
+    @Nullable
+    private final String projectId;
+
+    @Nullable
+    private final Query query;
+
+    @Nullable
+    private final String namespace;
+
+    /**
+     * Note that only {@code namespace} is really {@code @Nullable}. The other parameters may be
+     * {@code null} as a matter of build order, but if they are {@code null} at instantiation time,
+     * an error will be thrown.
+     */
+    private Read(@Nullable String projectId, @Nullable Query query, @Nullable String namespace) {
+      this.projectId = projectId;
+      this.query = query;
+      this.namespace = namespace;
+    }
+
+    /**
+     * Returns a new {@link V1Beta3.Read} that reads from the Datastore for the specified project.
+     */
+    public V1Beta3.Read withProjectId(String projectId) {
+      checkNotNull(projectId, "projectId");
+      return new V1Beta3.Read(projectId, query, namespace);
+    }
+
+    /**
+     * Returns a new {@link V1Beta3.Read} that reads the results of the specified query.
+     *
+     * <p><b>Note:</b> Normally, {@code DatastoreIO} will read from Cloud Datastore in parallel
+     * across many workers. However, when the {@link Query} is configured with a limit using
+     * {@link Query.Builder#setLimit}, then all results will be read by a single worker in order
+     * to ensure correct results.
+     */
+    public V1Beta3.Read withQuery(Query query) {
+      checkNotNull(query, "query");
+      checkArgument(!query.hasLimit() || query.getLimit().getValue() > 0,
+          "Invalid query limit %s: must be positive", query.getLimit().getValue());
+      return new V1Beta3.Read(projectId, query, namespace);
+    }
+
+    /**
+     * Returns a new {@link V1Beta3.Read} that reads from the given namespace.
+     */
+    public V1Beta3.Read withNamespace(String namespace) {
+      return new V1Beta3.Read(projectId, query, namespace);
+    }
+
+    @Nullable
+    public Query getQuery() {
+      return query;
+    }
+
+    @Nullable
+    public String getProjectId() {
+      return projectId;
+    }
+
+    @Nullable
+    public String getNamespace() {
+      return namespace;
+    }
+
+    @Override
+    public PCollection<Entity> apply(PBegin input) {
+      return input.apply(org.apache.beam.sdk.io.Read.from(getSource()));
+    }
+
+    @Override
+    public void validate(PBegin input) {
+      checkNotNull(projectId, "projectId");
+      checkNotNull(query, "query");
+    }
+
+    @Override
+    public void populateDisplayData(DisplayData.Builder builder) {
+      super.populateDisplayData(builder);
+      builder
+          .addIfNotNull(DisplayData.item("projectId", projectId)
+              .withLabel("ProjectId"))
+          .addIfNotNull(DisplayData.item("namespace", namespace)
+              .withLabel("Namespace"))
+          .addIfNotNull(DisplayData.item("query", query.toString())
+              .withLabel("Query"));
+    }
+
+    @Override
+    public String toString() {
+      return MoreObjects.toStringHelper(getClass())
+          .add("projectId", projectId)
+          .add("query", query)
+          .add("namespace", namespace)
+          .toString();
+    }
+
+    @VisibleForTesting
+    DatastoreSource getSource() {
+      return new DatastoreSource(projectId, query, namespace);
+    }
+  }
+
+  /**
+   * Returns an empty {@link V1Beta3.Write} builder. Configure the destination
+   * {@code projectId} using {@link V1Beta3.Write#withProjectId}.
+   */
+  public Write write() {
+    return new Write(null);
+  }
+
+  /**
+   * A {@link PTransform} that writes {@link Entity} objects to Cloud Datastore.
+   *
+   * @see DatastoreIO
+   */
+  public static class Write extends PTransform<PCollection<Entity>, PDone> {
+    @Nullable
+    private final String projectId;
+
+    /**
+     * Note that {@code projectId} is only {@code @Nullable} as a matter of build order, but if
+     * it is {@code null} at instantiation time, an error will be thrown.
+     */
+    public Write(@Nullable String projectId) {
+      this.projectId = projectId;
+    }
+
+    /**
+     * Returns a new {@link Write} that writes to the Cloud Datastore for the specified project.
+     */
+    public Write withProjectId(String projectId) {
+      checkNotNull(projectId, "projectId");
+      return new Write(projectId);
+    }
+
+    @Override
+    public PDone apply(PCollection<Entity> input) {
+      return input.apply(
+          org.apache.beam.sdk.io.Write.to(new DatastoreSink(projectId)));
+    }
+
+    @Override
+    public void validate(PCollection<Entity> input) {
+      checkNotNull(projectId, "projectId");
+    }
+
+    @Nullable
+    public String getProjectId() {
+      return projectId;
+    }
+
+    @Override
+    public String toString() {
+      return MoreObjects.toStringHelper(getClass())
+          .add("projectId", projectId)
+          .toString();
+    }
+
+    @Override
+    public void populateDisplayData(DisplayData.Builder builder) {
+      super.populateDisplayData(builder);
+      builder
+          .addIfNotNull(DisplayData.item("projectId", projectId)
+              .withLabel("Output Project"));
+    }
+  }
+
+  /**
+   * A {@link org.apache.beam.sdk.io.Source} that reads data from Datastore.
+   */
+  static class DatastoreSource extends BoundedSource<Entity> {
+
+    @Override
+    public Coder<Entity> getDefaultOutputCoder() {
+      return ProtoCoder.of(Entity.class);
+    }
+
+    @Override
+    public boolean producesSortedKeys(PipelineOptions options) {
+      return false;
+    }
+
+    @Override
+    public List<DatastoreSource> splitIntoBundles(long desiredBundleSizeBytes,
+        PipelineOptions options) throws Exception {
+      // Users may request a limit on the number of results. We can currently support this by
+      // simply disabling parallel reads and using only a single split.
+      if (query.hasLimit()) {
+        return ImmutableList.of(this);
+      }
+
+      long numSplits;
+      try {
+        numSplits = Math.round(((double) getEstimatedSizeBytes(options)) / desiredBundleSizeBytes);
+      } catch (Exception e) {
+        // Fallback in case estimated size is unavailable. TODO: fix this, it's horrible.
+        numSplits = 12;
+      }
+
+      // If the desiredBundleSize or number of workers results in 1 split, simply return
+      // a source that reads from the original query.
+      if (numSplits <= 1) {
+        return ImmutableList.of(this);
+      }
+
+      List<Query> datastoreSplits;
+      try {
+        datastoreSplits = getSplitQueries(Ints.checkedCast(numSplits), options);
+      } catch (IllegalArgumentException | DatastoreException e) {
+        LOG.warn("Unable to parallelize the given query: {}", query, e);
+        return ImmutableList.of(this);
+      }
+
+      ImmutableList.Builder<DatastoreSource> splits = ImmutableList.builder();
+      for (Query splitQuery : datastoreSplits) {
+        splits.add(new DatastoreSource(projectId, splitQuery, namespace));
+      }
+      return splits.build();
+    }
+
+    @Override
+    public BoundedReader<Entity> createReader(PipelineOptions pipelineOptions) throws IOException {
+      return new DatastoreReader(this, getDatastore(pipelineOptions));
+    }
+
+    @Override
+    public void validate() {
+      checkNotNull(query, "query");
+      checkNotNull(projectId, "projectId");
+    }
+
+    @Override
+    public long getEstimatedSizeBytes(PipelineOptions options) throws Exception {
+      // Datastore provides no way to get a good estimate of how large the result of a query
+      // will be. As a rough approximation, we attempt to fetch the statistics of the whole
+      // entity kind being queried, using the __Stat_Kind__ system table, assuming exactly 1 kind
+      // is specified in the query.
+      //
+      // See https://cloud.google.com/datastore/docs/concepts/stats
+      if (mockEstimateSizeBytes != null) {
+        return mockEstimateSizeBytes;
+      }
+
+      Datastore datastore = getDatastore(options);
+      if (query.getKindCount() != 1) {
+        throw new UnsupportedOperationException(
+            "Can only estimate size for queries specifying exactly 1 kind.");
+      }
+      String ourKind = query.getKind(0).getName();
+      long latestTimestamp = queryLatestStatisticsTimestamp(datastore);
+      Query.Builder query = Query.newBuilder();
+      if (namespace == null) {
+        query.addKindBuilder().setName("__Stat_Kind__");
+      } else {
+        query.addKindBuilder().setName("__Ns_Stat_Kind__");
+      }
+      query.setFilter(makeAndFilter(
+          makeFilter("kind_name", EQUAL, makeValue(ourKind)).build(),
+          makeFilter("timestamp", EQUAL, makeValue(latestTimestamp)).build()));
+      RunQueryRequest request = makeRequest(query.build());
+
+      long now = System.currentTimeMillis();
+      RunQueryResponse response = datastore.runQuery(request);
+      LOG.info("Query for per-kind statistics took {}ms", System.currentTimeMillis() - now);
+
+      QueryResultBatch batch = response.getBatch();
+      if (batch.getEntityResultsCount() == 0) {
+        throw new NoSuchElementException(
+            "Datastore statistics for kind " + ourKind + " unavailable");
+      }
+      Entity entity = batch.getEntityResults(0).getEntity();
+      return entity.getProperties().get("entity_bytes").getIntegerValue();
+    }
+
+    @Override
+    public void populateDisplayData(DisplayData.Builder builder) {
+      super.populateDisplayData(builder);
+      builder
+          .addIfNotNull(DisplayData.item("projectId", projectId)
+              .withLabel("ProjectId"))
+          .addIfNotNull(DisplayData.item("namespace", namespace)
+              .withLabel("Namespace"))
+          .addIfNotNull(DisplayData.item("query", query.toString())
+              .withLabel("Query"));
+    }
+
+    @Override
+    public String toString() {
+      return MoreObjects.toStringHelper(getClass())
+          .add("projectId", projectId)
+          .add("query", query)
+          .add("namespace", namespace)
+          .toString();
+    }
+
+    private static final Logger LOG = LoggerFactory.getLogger(DatastoreSource.class);
+    private final String projectId;
+    private final Query query;
+    @Nullable
+    private final String namespace;
+
+    /** For testing only. TODO: This could be much cleaner with dependency injection. */
+    @Nullable
+    private QuerySplitter mockSplitter;
+    @Nullable
+    private Long mockEstimateSizeBytes;
+
+    DatastoreSource(String projectId, Query query, @Nullable String namespace) {
+      this.projectId = projectId;
+      this.query = query;
+      this.namespace = namespace;
+    }
+
+    /**
+     * A helper function to get the split queries, taking into account the optional
+     * {@code namespace} and whether there is a mock splitter.
+     */
+    private List<Query> getSplitQueries(int numSplits, PipelineOptions options)
+        throws DatastoreException {
+      // If namespace is set, include it in the split request so splits are calculated accordingly.
+      PartitionId.Builder partitionBuilder = PartitionId.newBuilder();
+      if (namespace != null) {
+        partitionBuilder.setNamespaceId(namespace);
+      }
+
+      if (mockSplitter != null) {
+        // For testing.
+        return mockSplitter.getSplits(query, partitionBuilder.build(), numSplits, null);
+      }
+
+      return DatastoreHelper.getQuerySplitter().getSplits(
+          query, partitionBuilder.build(), numSplits, getDatastore(options));
+    }
+
+    /**
+     * Builds a {@link RunQueryRequest} from the {@code query}, using the properties set on this
+     * {@code DatastoreSource}. For example, sets the {@code namespace} for the request.
+     */
+    private RunQueryRequest makeRequest(Query query) {
+      RunQueryRequest.Builder requestBuilder = RunQueryRequest.newBuilder().setQuery(query);
+      if (namespace != null) {
+        requestBuilder.getPartitionIdBuilder().setNamespaceId(namespace);
+      }
+      return requestBuilder.build();
+    }
+
+    /**
+     * Datastore system tables with statistics are periodically updated. This method fetches
+     * the latest timestamp of statistics update using the {@code __Stat_Total__} table.
+     */
+    private long queryLatestStatisticsTimestamp(Datastore datastore) throws DatastoreException {
+      Query.Builder query = Query.newBuilder();
+      query.addKindBuilder().setName("__Stat_Total__");
+      query.addOrder(makeOrder("timestamp", DESCENDING));
+      query.setLimit(Int32Value.newBuilder().setValue(1));
+      RunQueryRequest request = makeRequest(query.build());
+
+      long now = System.currentTimeMillis();
+      RunQueryResponse response = datastore.runQuery(request);
+      LOG.info("Query for latest stats timestamp of project {} took {}ms", projectId,
+          System.currentTimeMillis() - now);
+      QueryResultBatch batch = response.getBatch();
+      if (batch.getEntityResultsCount() == 0) {
+        throw new NoSuchElementException(
+            "Datastore total statistics for project " + projectId + " unavailable");
+      }
+      Entity entity = batch.getEntityResults(0).getEntity();
+      return entity.getProperties().get("timestamp").getTimestampValue().getNanos();
+    }
+
+    private Datastore getDatastore(PipelineOptions pipelineOptions) {
+      DatastoreOptions.Builder builder =
+          new DatastoreOptions.Builder()
+              .projectId(projectId)
+              .initializer(
+                  new RetryHttpRequestInitializer()
+              );
+
+      Credential credential = pipelineOptions.as(GcpOptions.class).getGcpCredential();
+      if (credential != null) {
+        builder.credential(credential);
+      }
+      return DatastoreFactory.get().create(builder.build());
+    }
+
+    /** For testing only. */
+    DatastoreSource withMockSplitter(QuerySplitter splitter) {
+      DatastoreSource res = new DatastoreSource(projectId, query, namespace);
+      res.mockSplitter = splitter;
+      res.mockEstimateSizeBytes = mockEstimateSizeBytes;
+      return res;
+    }
+
+    /** For testing only. */
+    DatastoreSource withMockEstimateSizeBytes(Long estimateSizeBytes) {
+      DatastoreSource res = new DatastoreSource(projectId, query, namespace);
+      res.mockSplitter = mockSplitter;
+      res.mockEstimateSizeBytes = estimateSizeBytes;
+      return res;
+    }
+
+    @VisibleForTesting
+    Query getQuery() {
+      return query;
+    }
+  }
+
+  /**
+   * A {@link DatastoreSource.Reader} over the records from a query of the datastore.
+   *
+   * <p>Timestamped records are currently not supported.
+   * All records implicitly have the timestamp of {@code BoundedWindow.TIMESTAMP_MIN_VALUE}.
+   */
+  @VisibleForTesting
+  static class DatastoreReader extends BoundedSource.BoundedReader<Entity> {
+    private final DatastoreSource source;
+
+    /**
+     * Datastore to read from.
+     */
+    private final Datastore datastore;
+
+    /**
+     * True if more results may be available.
+     */
+    private boolean moreResults;
+
+    /**
+     * Iterator over records.
+     */
+    private java.util.Iterator<EntityResult> entities;
+
+    /**
+     * Current batch of query results.
+     */
+    private QueryResultBatch currentBatch;
+
+    /**
+     * Maximum number of results to request per query.
+     *
+     * <p>Must be set, or it may result in an I/O error when querying
+     * Cloud Datastore.
+     */
+    private static final int QUERY_BATCH_LIMIT = 500;
+
+    /**
+     * Remaining user-requested limit on the number of sources to return. If the user did not set a
+     * limit, then this variable will always have the value {@link Integer#MAX_VALUE} and will never
+     * be decremented.
+     */
+    private int userLimit;
+
+    private volatile boolean done = false;
+
+    private Entity currentEntity;
+
+    /**
+     * Returns a DatastoreReader with DatastoreSource and Datastore object set.
+     *
+     * @param datastore a datastore connection to use.
+     */
+    public DatastoreReader(DatastoreSource source, Datastore datastore) {
+      this.source = source;
+      this.datastore = datastore;
+      // If the user set a limit on the query, remember it. Otherwise pin to MAX_VALUE.
+      userLimit = source.query.hasLimit()
+          ? source.query.getLimit().getValue() : Integer.MAX_VALUE;
+    }
+
+    @Override
+    public Entity getCurrent() {
+      return currentEntity;
+    }
+
+    @Override
+    public final long getSplitPointsConsumed() {
+      return done ? 1 : 0;
+    }
+
+    @Override
+    public final long getSplitPointsRemaining() {
+      return done ? 0 : 1;
+    }
+
+    @Override
+    public boolean start() throws IOException {
+      return advance();
+    }
+
+    @Override
+    public boolean advance() throws IOException {
+      if (entities == null || (!entities.hasNext() && moreResults)) {
+        try {
+          entities = getIteratorAndMoveCursor();
+        } catch (DatastoreException e) {
+          throw new IOException(e);
+        }
+      }
+
+      if (entities == null || !entities.hasNext()) {
+        currentEntity = null;
+        done = true;
+        return false;
+      }
+
+      currentEntity = entities.next().getEntity();
+      return true;
+    }
+
+    @Override
+    public void close() throws IOException {
+      // Nothing
+    }
+
+    @Override
+    public DatastoreSource getCurrentSource() {
+      return source;
+    }
+
+    @Override
+    public DatastoreSource splitAtFraction(double fraction) {
+      // Not supported.
+      return null;
+    }
+
+    @Override
+    public Double getFractionConsumed() {
+      // Not supported.
+      return null;
+    }
+
+    /**
+     * Returns an iterator over the next batch of records for the query
+     * and updates the cursor to get the next batch as needed.
+     * Query has specified limit and offset from InputSplit.
+     */
+    private Iterator<EntityResult> getIteratorAndMoveCursor() throws DatastoreException {
+      Query.Builder query = source.query.toBuilder().clone();
+      query.setLimit(Int32Value.newBuilder().setValue(Math.min(userLimit, QUERY_BATCH_LIMIT)));
+      if (currentBatch != null && !currentBatch.getEndCursor().isEmpty()) {
+        query.setStartCursor(currentBatch.getEndCursor());
+      }
+
+      RunQueryRequest request = source.makeRequest(query.build());
+      RunQueryResponse response = datastore.runQuery(request);
+
+      currentBatch = response.getBatch();
+
+      // MORE_RESULTS_AFTER_LIMIT is not implemented yet:
+      // https://groups.google.com/forum/#!topic/gcd-discuss/iNs6M1jA2Vw, so
+      // use result count to determine if more results might exist.
+      int numFetch = currentBatch.getEntityResultsCount();
+      if (source.query.hasLimit()) {
+        verify(userLimit >= numFetch,
+            "Expected userLimit %s >= numFetch %s, because query limit %s should be <= userLimit",
+            userLimit, numFetch, query.getLimit());
+        userLimit -= numFetch;
+      }
+      moreResults =
+          // User-limit does not exist (so userLimit == MAX_VALUE) and/or has not been satisfied.
+          (userLimit > 0)
+              // All indications from the API are that there are/may be more results.
+              && ((numFetch == QUERY_BATCH_LIMIT)
+              || (currentBatch.getMoreResults() == NOT_FINISHED));
+
+      // May receive a batch of 0 results if the number of records is a multiple
+      // of the request limit.
+      if (numFetch == 0) {
+        return null;
+      }
+
+      return currentBatch.getEntityResultsList().iterator();
+    }
+  }
+
+  /**
+   * A {@link org.apache.beam.sdk.io.Sink} that writes data to Datastore.
+   */
+  static class DatastoreSink extends org.apache.beam.sdk.io.Sink<Entity> {
+    final String projectId;
+
+    public DatastoreSink(String projectId) {
+      this.projectId = projectId;
+    }
+
+    @Override
+    public void validate(PipelineOptions options) {
+      checkNotNull(projectId, "projectId");
+    }
+
+    @Override
+    public DatastoreWriteOperation createWriteOperation(PipelineOptions options) {
+      return new DatastoreWriteOperation(this);
+    }
+
+    @Override
+    public void populateDisplayData(DisplayData.Builder builder) {
+      super.populateDisplayData(builder);
+      builder
+          .addIfNotNull(DisplayData.item("projectId", projectId)
+              .withLabel("Output Project"));
+    }
+  }
+
+  /**
+   * A {@link WriteOperation} that will manage a parallel write to a Datastore sink.
+   */
+  private static class DatastoreWriteOperation
+      extends WriteOperation<Entity, DatastoreWriteResult> {
+    private static final Logger LOG = LoggerFactory.getLogger(DatastoreWriteOperation.class);
+
+    private final DatastoreSink sink;
+
+    public DatastoreWriteOperation(DatastoreSink sink) {
+      this.sink = sink;
+    }
+
+    @Override
+    public Coder<DatastoreWriteResult> getWriterResultCoder() {
+      return SerializableCoder.of(DatastoreWriteResult.class);
+    }
+
+    @Override
+    public void initialize(PipelineOptions options) throws Exception {}
+
+    /**
+     * Finalizes the write.  Logs the number of entities written to the Datastore.
+     */
+    @Override
+    public void finalize(Iterable<DatastoreWriteResult> writerResults, PipelineOptions options)
+        throws Exception {
+      long totalEntities = 0;
+      for (DatastoreWriteResult result : writerResults) {
+        totalEntities += result.entitiesWritten;
+      }
+      LOG.info("Wrote {} elements.", totalEntities);
+    }
+
+    @Override
+    public DatastoreWriter createWriter(PipelineOptions options) throws Exception {
+      DatastoreOptions.Builder builder =
+          new DatastoreOptions.Builder()
+              .projectId(sink.projectId)
+              .initializer(new RetryHttpRequestInitializer());
+      Credential credential = options.as(GcpOptions.class).getGcpCredential();
+      if (credential != null) {
+        builder.credential(credential);
+      }
+      Datastore datastore = DatastoreFactory.get().create(builder.build());
+
+      return new DatastoreWriter(this, datastore);
+    }
+
+    @Override
+    public DatastoreSink getSink() {
+      return sink;
+    }
+  }
+
+  /**
+   * {@link Writer} that writes entities to a Datastore Sink.  Entities are written in batches,
+   * where the maximum batch size is {@link V1Beta3#DATASTORE_BATCH_UPDATE_LIMIT}.  Entities
+   * are committed as upsert mutations (either update if the key already exists, or insert if it is
+   * a new key).  If an entity does not have a complete key (i.e., it has no name or id), the bundle
+   * will fail.
+   *
+   * <p>See <a
+   * href="https://cloud.google.com/datastore/docs/concepts/entities#Datastore_Creating_an_entity">
+   * Datastore: Entities, Properties, and Keys</a> for information about entity keys and upsert
+   * mutations.
+   *
+   * <p>Commits are non-transactional.  If a commit fails because of a conflict over an entity
+   * group, the commit will be retried (up to {@link V1Beta3#DATASTORE_BATCH_UPDATE_LIMIT}
+   * times).
+   *
+   * <p>Visible for testing purposes.
+   */
+  @VisibleForTesting
+  static class DatastoreWriter extends Writer<Entity, DatastoreWriteResult> {
+    private static final Logger LOG = LoggerFactory.getLogger(DatastoreWriter.class);
+    private final DatastoreWriteOperation writeOp;
+    private final Datastore datastore;
+    private long totalWritten = 0;
+
+    // Visible for testing.
+    final List<Entity> entities = new ArrayList<>();
+
+    /**
+     * Since a bundle is written in batches, we should retry the commit of a batch in order to
+     * prevent transient errors from causing the bundle to fail.
+     */
+    private static final int MAX_RETRIES = 5;
+
+    /**
+     * Initial backoff time for exponential backoff for retry attempts.
+     */
+    private static final int INITIAL_BACKOFF_MILLIS = 5000;
+
+    /**
+     * Returns true if a Datastore key is complete.  A key is complete if its last element
+     * has either an id or a name.
+     */
+    static boolean isValidKey(Key key) {
+      List<PathElement> elementList = key.getPathList();
+      if (elementList.isEmpty()) {
+        return false;
+      }
+      PathElement lastElement = elementList.get(elementList.size() - 1);
+      return (lastElement.getId() != 0 || !lastElement.getName().isEmpty());
+    }
+
+    DatastoreWriter(DatastoreWriteOperation writeOp, Datastore datastore) {
+      this.writeOp = writeOp;
+      this.datastore = datastore;
+    }
+
+    @Override
+    public void open(String uId) throws Exception {}
+
+    /**
+     * Writes an entity to the Datastore.  Writes are batched, up to {@link
+     * V1Beta3#DATASTORE_BATCH_UPDATE_LIMIT}. If an entity does not have a complete key, an
+     * {@link IllegalArgumentException} will be thrown.
+     */
+    @Override
+    public void write(Entity value) throws Exception {
+      // Verify that the entity to write has a complete key.
+      if (!isValidKey(value.getKey())) {
+        throw new IllegalArgumentException(
+            "Entities to be written to the Datastore must have complete keys");
+      }
+
+      entities.add(value);
+
+      if (entities.size() >= V1Beta3.DATASTORE_BATCH_UPDATE_LIMIT) {
+        flushBatch();
+      }
+    }
+
+    /**
+     * Flushes any pending batch writes and returns a DatastoreWriteResult.
+     */
+    @Override
+    public DatastoreWriteResult close() throws Exception {
+      if (entities.size() > 0) {
+        flushBatch();
+      }
+      return new DatastoreWriteResult(totalWritten);
+    }
+
+    @Override
+    public DatastoreWriteOperation getWriteOperation() {
+      return writeOp;
+    }
+
+    /**
+     * Writes a batch of entities to the Datastore.
+     *
+     * <p>If a commit fails, it will be retried (up to {@link DatastoreWriter#MAX_RETRIES}
+     * times).  All entities in the batch will be committed again, even if the commit was partially
+     * successful. If the retry limit is exceeded, the last exception from the Datastore will be
+     * thrown.
+     *
+     * @throws DatastoreException if the commit fails or IOException or InterruptedException if
+     * backing off between retries fails.
+     */
+    private void flushBatch() throws DatastoreException, IOException, InterruptedException {
+      LOG.debug("Writing batch of {} entities", entities.size());
+      Sleeper sleeper = Sleeper.DEFAULT;
+      BackOff backoff = new AttemptBoundedExponentialBackOff(MAX_RETRIES, INITIAL_BACKOFF_MILLIS);
+
+      while (true) {
+        // Batch upsert entities.
+        try {
+          CommitRequest.Builder commitRequest = CommitRequest.newBuilder();
+          for (Entity entity: entities) {
+            commitRequest.addMutations(makeUpsert(entity));
+          }
+          commitRequest.setMode(CommitRequest.Mode.NON_TRANSACTIONAL);
+          datastore.commit(commitRequest.build());
+          // Break if the commit threw no exception.
+          break;
+        } catch (DatastoreException exception) {
+          // Only log the code and message for potentially-transient errors. The entire exception
+          // will be propagated upon the last retry.
+          LOG.error("Error writing to the Datastore ({}): {}", exception.getCode(),
+              exception.getMessage());
+          if (!BackOffUtils.next(sleeper, backoff)) {
+            LOG.error("Aborting after {} retries.", MAX_RETRIES);
+            throw exception;
+          }
+        }
+      }
+      totalWritten += entities.size();
+      LOG.debug("Successfully wrote {} entities", entities.size());
+      entities.clear();
+    }
+  }
+
+  private static class DatastoreWriteResult implements Serializable {
+    final long entitiesWritten;
+
+    public DatastoreWriteResult(long recordsWritten) {
+      this.entitiesWritten = recordsWritten;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/d1e38443/sdks/java/core/src/main/java/org/apache/beam/sdk/io/datastore/package-info.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/datastore/package-info.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/datastore/package-info.java
new file mode 100644
index 0000000..f687739
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/datastore/package-info.java
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+
+/**
+ * <p>Provides an API for reading from and writing to
+ * <a href="https://developers.google.com/datastore/">Google Cloud Datastore</a> over different
+ * versions of the Datastore Client libraries.
+ */
+package org.apache.beam.sdk.io.datastore;


[45/50] [abbrv] incubator-beam git commit: Several improvements to HDFS/Hadoop interoperability

Posted by lc...@apache.org.
Several improvements to HDFS/Hadoop interoperability

* handle NullWritable in WritableCoder
* update Function handling in HDFSFileSource#splitIntoBundles
* add AvroHDFSFileSource
* add HDFSFileSink
* add SimpleAuth HDFS IO


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

Branch: refs/heads/runners-spark2
Commit: a17a8b2e1a2279b58f32ab32b6b33522c7c4a65d
Parents: b09de0f
Author: Neville Li <ne...@spotify.com>
Authored: Mon Jun 27 15:51:17 2016 -0400
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Jul 6 10:18:53 2016 -0700

----------------------------------------------------------------------
 sdks/java/io/hdfs/pom.xml                       |  24 ++
 .../beam/sdk/io/hdfs/AvroHDFSFileSource.java    | 145 ++++++++++
 .../beam/sdk/io/hdfs/AvroWrapperCoder.java      | 116 ++++++++
 .../apache/beam/sdk/io/hdfs/HDFSFileSink.java   | 277 +++++++++++++++++++
 .../apache/beam/sdk/io/hdfs/HDFSFileSource.java |  40 +--
 .../apache/beam/sdk/io/hdfs/WritableCoder.java  |   9 +-
 .../SimpleAuthAvroHDFSFileSource.java           |  84 ++++++
 .../hdfs/simpleauth/SimpleAuthHDFSFileSink.java | 132 +++++++++
 .../simpleauth/SimpleAuthHDFSFileSource.java    | 122 ++++++++
 .../beam/sdk/io/hdfs/AvroWrapperCoderTest.java  |  52 ++++
 .../beam/sdk/io/hdfs/WritableCoderTest.java     |   9 +
 11 files changed, 989 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a17a8b2e/sdks/java/io/hdfs/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/io/hdfs/pom.xml b/sdks/java/io/hdfs/pom.xml
index 2e427b1..42175d5 100644
--- a/sdks/java/io/hdfs/pom.xml
+++ b/sdks/java/io/hdfs/pom.xml
@@ -82,6 +82,30 @@
     </dependency>
 
     <dependency>
+      <groupId>com.google.http-client</groupId>
+      <artifactId>google-http-client</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.avro</groupId>
+      <artifactId>avro</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.avro</groupId>
+      <artifactId>avro-mapred</artifactId>
+      <version>${avro.version}</version>
+      <classifier>hadoop2</classifier>
+      <exclusions>
+        <!-- exclude old Jetty version of servlet API -->
+        <exclusion>
+          <groupId>org.mortbay.jetty</groupId>
+          <artifactId>servlet-api</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-client</artifactId>
       <version>2.7.0</version>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a17a8b2e/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/AvroHDFSFileSource.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/AvroHDFSFileSource.java b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/AvroHDFSFileSource.java
new file mode 100644
index 0000000..9dc926b
--- /dev/null
+++ b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/AvroHDFSFileSource.java
@@ -0,0 +1,145 @@
+/*
+ * 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.hdfs;
+
+import org.apache.beam.sdk.coders.AvroCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.io.BoundedSource;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.util.CoderUtils;
+import org.apache.beam.sdk.values.KV;
+
+import com.google.common.base.Function;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+
+import org.apache.avro.Schema;
+import org.apache.avro.mapred.AvroKey;
+import org.apache.avro.mapreduce.AvroJob;
+import org.apache.avro.mapreduce.AvroKeyInputFormat;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+
+import java.io.IOException;
+import java.util.List;
+import javax.annotation.Nullable;
+
+/**
+ * A {@code BoundedSource} for reading Avro files resident in a Hadoop filesystem.
+ *
+ * @param <T> The type of the Avro records to be read from the source.
+ */
+public class AvroHDFSFileSource<T> extends HDFSFileSource<AvroKey<T>, NullWritable> {
+  private static final long serialVersionUID = 0L;
+
+  protected final AvroCoder<T> avroCoder;
+  private final String schemaStr;
+
+  public AvroHDFSFileSource(String filepattern, AvroCoder<T> avroCoder) {
+    this(filepattern, avroCoder, null);
+  }
+
+  public AvroHDFSFileSource(String filepattern,
+                            AvroCoder<T> avroCoder,
+                            HDFSFileSource.SerializableSplit serializableSplit) {
+    super(filepattern,
+        ClassUtil.<AvroKeyInputFormat<T>>castClass(AvroKeyInputFormat.class),
+        ClassUtil.<AvroKey<T>>castClass(AvroKey.class),
+        NullWritable.class, serializableSplit);
+    this.avroCoder = avroCoder;
+    this.schemaStr = avroCoder.getSchema().toString();
+  }
+
+  @Override
+  public Coder<KV<AvroKey<T>, NullWritable>> getDefaultOutputCoder() {
+    AvroWrapperCoder<AvroKey<T>, T> keyCoder = AvroWrapperCoder.of(this.getKeyClass(), avroCoder);
+    return KvCoder.of(keyCoder, WritableCoder.of(NullWritable.class));
+  }
+
+  @Override
+  public List<? extends AvroHDFSFileSource<T>> splitIntoBundles(
+      long desiredBundleSizeBytes, PipelineOptions options) throws Exception {
+    if (serializableSplit == null) {
+      return Lists.transform(computeSplits(desiredBundleSizeBytes),
+          new Function<InputSplit, AvroHDFSFileSource<T>>() {
+            @Override
+            public AvroHDFSFileSource<T> apply(@Nullable InputSplit inputSplit) {
+              return new AvroHDFSFileSource<>(
+                  filepattern, avroCoder, new SerializableSplit(inputSplit));
+            }
+          });
+    } else {
+      return ImmutableList.of(this);
+    }
+  }
+
+  @Override
+  public BoundedReader<KV<AvroKey<T>, NullWritable>> createReader(PipelineOptions options)
+      throws IOException {
+    this.validate();
+
+    Schema schema = new Schema.Parser().parse(schemaStr);
+    if (serializableSplit == null) {
+      return new AvroHDFSFileReader<>(this, filepattern, formatClass, schema);
+    } else {
+      return new AvroHDFSFileReader<>(this, filepattern, formatClass, schema,
+          serializableSplit.getSplit());
+    }
+  }
+
+  static class AvroHDFSFileReader<T> extends HDFSFileReader<AvroKey<T>, NullWritable> {
+    public AvroHDFSFileReader(BoundedSource<KV<AvroKey<T>, NullWritable>> source,
+                              String filepattern,
+                              Class<? extends FileInputFormat<?, ?>> formatClass,
+                              Schema schema) throws IOException {
+      this(source, filepattern, formatClass, schema, null);
+    }
+
+    public AvroHDFSFileReader(BoundedSource<KV<AvroKey<T>, NullWritable>> source,
+                              String filepattern,
+                              Class<? extends FileInputFormat<?, ?>> formatClass,
+                              Schema schema, InputSplit split) throws IOException {
+      super(source, filepattern, formatClass, split);
+      AvroJob.setInputKeySchema(job, schema);
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    protected KV<AvroKey<T>, NullWritable> nextPair() throws IOException, InterruptedException {
+      AvroKey<T> key = currentReader.getCurrentKey();
+      NullWritable value = currentReader.getCurrentValue();
+
+      // clone the record to work around identical element issue due to object reuse
+      Coder<T> avroCoder = ((AvroHDFSFileSource<T>) this.getCurrentSource()).avroCoder;
+      key = new AvroKey(CoderUtils.clone(avroCoder, key.datum()));
+
+      return KV.of(key, value);
+    }
+
+  }
+
+  static class ClassUtil {
+    @SuppressWarnings("unchecked")
+    static <T> Class<T> castClass(Class<?> aClass) {
+      return (Class<T>) aClass;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a17a8b2e/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/AvroWrapperCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/AvroWrapperCoder.java b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/AvroWrapperCoder.java
new file mode 100644
index 0000000..a831afe
--- /dev/null
+++ b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/AvroWrapperCoder.java
@@ -0,0 +1,116 @@
+/*
+ * 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.hdfs;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import org.apache.beam.sdk.coders.AvroCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.StandardCoder;
+import org.apache.beam.sdk.util.CloudObject;
+import org.apache.beam.sdk.util.PropertyNames;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.avro.mapred.AvroWrapper;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * A {@code AvroWrapperCoder} is a {@link Coder} for a Java class that implements {@link
+ * AvroWrapper}.
+ *
+ * @param <WrapperT> the type of the wrapper
+ * @param <DatumT> the type of the datum
+ */
+public class AvroWrapperCoder<WrapperT extends AvroWrapper<DatumT>, DatumT>
+    extends StandardCoder<WrapperT> {
+  private static final long serialVersionUID = 0L;
+
+  private final Class<WrapperT> wrapperType;
+  private final AvroCoder<DatumT> datumCoder;
+
+  private AvroWrapperCoder(Class<WrapperT> wrapperType, AvroCoder<DatumT> datumCoder) {
+    this.wrapperType = wrapperType;
+    this.datumCoder = datumCoder;
+  }
+
+  /**
+   * Return a {@code AvroWrapperCoder} instance for the provided element class.
+   * @param <WrapperT> the type of the wrapper
+   * @param <DatumT> the type of the datum
+   */
+  public static <WrapperT extends AvroWrapper<DatumT>, DatumT>
+  AvroWrapperCoder<WrapperT, DatumT>of(Class<WrapperT> wrapperType, AvroCoder<DatumT> datumCoder) {
+    return new AvroWrapperCoder<>(wrapperType, datumCoder);
+  }
+
+  @JsonCreator
+  @SuppressWarnings("unchecked")
+  public static AvroWrapperCoder<?, ?> of(
+      @JsonProperty("wrapperType") String wrapperType,
+      @JsonProperty(PropertyNames.COMPONENT_ENCODINGS) List<Coder<?>> components)
+      throws ClassNotFoundException {
+    Class<?> clazz = Class.forName(wrapperType);
+    if (!AvroWrapper.class.isAssignableFrom(clazz)) {
+      throw new ClassNotFoundException(
+          "Class " + wrapperType + " does not implement AvroWrapper");
+    }
+    checkArgument(components.size() == 1, "Expecting 1 component, got " + components.size());
+    return of((Class<? extends AvroWrapper>) clazz, (AvroCoder<?>) components.get(0));
+  }
+
+  @Override
+  public void encode(WrapperT value, OutputStream outStream, Context context) throws IOException {
+    datumCoder.encode(value.datum(), outStream, context);
+  }
+
+  @Override
+  public WrapperT decode(InputStream inStream, Context context) throws IOException {
+    try {
+      WrapperT wrapper = wrapperType.newInstance();
+      wrapper.datum(datumCoder.decode(inStream, context));
+      return wrapper;
+    } catch (InstantiationException | IllegalAccessException e) {
+      throw new CoderException("unable to deserialize record", e);
+    }
+  }
+
+  @Override
+  public List<? extends Coder<?>> getCoderArguments() {
+    return Collections.singletonList(datumCoder);
+  }
+
+  @Override
+  public CloudObject asCloudObject() {
+    CloudObject result = super.asCloudObject();
+    result.put("wrapperType", wrapperType.getName());
+    return result;
+  }
+
+  @Override
+  public void verifyDeterministic() throws NonDeterministicException {
+    datumCoder.verifyDeterministic();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a17a8b2e/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HDFSFileSink.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HDFSFileSink.java b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HDFSFileSink.java
new file mode 100644
index 0000000..688447a
--- /dev/null
+++ b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HDFSFileSink.java
@@ -0,0 +1,277 @@
+/*
+ * 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.hdfs;
+
+import static com.google.common.base.Preconditions.checkState;
+
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.Sink;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.values.KV;
+
+import com.google.api.client.util.Maps;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.task.JobContextImpl;
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+
+/**
+ * A {@code Sink} for writing records to a Hadoop filesystem using a Hadoop file-based output
+ * format.
+ *
+ * @param <K> The type of keys to be written to the sink.
+ * @param <V> The type of values to be written to the sink.
+ */
+public class HDFSFileSink<K, V> extends Sink<KV<K, V>> {
+
+  private static final JobID jobId = new JobID(
+      Long.toString(System.currentTimeMillis()),
+      new Random().nextInt(Integer.MAX_VALUE));
+
+  protected final String path;
+  protected final Class<? extends FileOutputFormat<K, V>> formatClass;
+
+  // workaround to make Configuration serializable
+  private final Map<String, String> map;
+
+  public HDFSFileSink(String path, Class<? extends FileOutputFormat<K, V>> formatClass) {
+    this.path = path;
+    this.formatClass = formatClass;
+    this.map = Maps.newHashMap();
+  }
+
+  public HDFSFileSink(String path, Class<? extends FileOutputFormat<K, V>> formatClass,
+                      Configuration conf) {
+    this(path, formatClass);
+    // serialize conf to map
+    for (Map.Entry<String, String> entry : conf) {
+      map.put(entry.getKey(), entry.getValue());
+    }
+  }
+
+  @Override
+  public void validate(PipelineOptions options) {
+    try {
+      Job job = jobInstance();
+      FileSystem fs = FileSystem.get(job.getConfiguration());
+      checkState(!fs.exists(new Path(path)), "Output path " + path + " already exists");
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Override
+  public Sink.WriteOperation<KV<K, V>, ?> createWriteOperation(PipelineOptions options) {
+    return new HDFSWriteOperation<>(this, path, formatClass);
+  }
+
+  private Job jobInstance() throws IOException {
+    Job job = Job.getInstance();
+    // deserialize map to conf
+    Configuration conf = job.getConfiguration();
+    for (Map.Entry<String, String> entry : map.entrySet()) {
+      conf.set(entry.getKey(), entry.getValue());
+    }
+    job.setJobID(jobId);
+    return job;
+  }
+
+  // =======================================================================
+  // WriteOperation
+  // =======================================================================
+
+  /** {{@link WriteOperation}} for HDFS. */
+  public static class HDFSWriteOperation<K, V> extends WriteOperation<KV<K, V>, String> {
+
+    private final Sink<KV<K, V>> sink;
+    protected final String path;
+    protected final Class<? extends FileOutputFormat<K, V>> formatClass;
+
+    public HDFSWriteOperation(Sink<KV<K, V>> sink,
+                              String path,
+                              Class<? extends FileOutputFormat<K, V>> formatClass) {
+      this.sink = sink;
+      this.path = path;
+      this.formatClass = formatClass;
+    }
+
+    @Override
+    public void initialize(PipelineOptions options) throws Exception {
+      Job job = ((HDFSFileSink<K, V>) getSink()).jobInstance();
+      FileOutputFormat.setOutputPath(job, new Path(path));
+    }
+
+    @Override
+    public void finalize(Iterable<String> writerResults, PipelineOptions options) throws Exception {
+      Job job = ((HDFSFileSink<K, V>) getSink()).jobInstance();
+      FileSystem fs = FileSystem.get(job.getConfiguration());
+
+      // If there are 0 output shards, just create output folder.
+      if (!writerResults.iterator().hasNext()) {
+        fs.mkdirs(new Path(path));
+        return;
+      }
+
+      // job successful
+      JobContext context = new JobContextImpl(job.getConfiguration(), job.getJobID());
+      FileOutputCommitter outputCommitter = new FileOutputCommitter(new Path(path), context);
+      outputCommitter.commitJob(context);
+
+      // get actual output shards
+      Set<String> actual = Sets.newHashSet();
+      FileStatus[] statuses = fs.listStatus(new Path(path), new PathFilter() {
+        @Override
+        public boolean accept(Path path) {
+          String name = path.getName();
+          return !name.startsWith("_") && !name.startsWith(".");
+        }
+      });
+
+      // get expected output shards
+      Set<String> expected = Sets.newHashSet(writerResults);
+      checkState(
+          expected.size() == Lists.newArrayList(writerResults).size(),
+          "Data loss due to writer results hash collision");
+      for (FileStatus s : statuses) {
+        String name = s.getPath().getName();
+        int pos = name.indexOf('.');
+        actual.add(pos > 0 ? name.substring(0, pos) : name);
+      }
+
+      checkState(actual.equals(expected), "Writer results and output files do not match");
+
+      // rename output shards to Hadoop style, i.e. part-r-00000.txt
+      int i = 0;
+      for (FileStatus s : statuses) {
+        String name = s.getPath().getName();
+        int pos = name.indexOf('.');
+        String ext = pos > 0 ? name.substring(pos) : "";
+        fs.rename(
+            s.getPath(),
+            new Path(s.getPath().getParent(), String.format("part-r-%05d%s", i, ext)));
+        i++;
+      }
+    }
+
+    @Override
+    public Writer<KV<K, V>, String> createWriter(PipelineOptions options) throws Exception {
+      return new HDFSWriter<>(this, path, formatClass);
+    }
+
+    @Override
+    public Sink<KV<K, V>> getSink() {
+      return sink;
+    }
+
+    @Override
+    public Coder<String> getWriterResultCoder() {
+      return StringUtf8Coder.of();
+    }
+
+  }
+
+  // =======================================================================
+  // Writer
+  // =======================================================================
+
+  /** {{@link Writer}} for HDFS files. */
+  public static class HDFSWriter<K, V> extends Writer<KV<K, V>, String> {
+
+    private final HDFSWriteOperation<K, V> writeOperation;
+    private final String path;
+    private final Class<? extends FileOutputFormat<K, V>> formatClass;
+
+    // unique hash for each task
+    private int hash;
+
+    private TaskAttemptContext context;
+    private RecordWriter<K, V> recordWriter;
+    private FileOutputCommitter outputCommitter;
+
+    public HDFSWriter(HDFSWriteOperation<K, V> writeOperation,
+                      String path,
+                      Class<? extends FileOutputFormat<K, V>> formatClass) {
+      this.writeOperation = writeOperation;
+      this.path = path;
+      this.formatClass = formatClass;
+    }
+
+    @Override
+    public void open(String uId) throws Exception {
+      this.hash = uId.hashCode();
+
+      Job job = ((HDFSFileSink<K, V>) getWriteOperation().getSink()).jobInstance();
+      FileOutputFormat.setOutputPath(job, new Path(path));
+
+      // Each Writer is responsible for writing one bundle of elements and is represented by one
+      // unique Hadoop task based on uId/hash. All tasks share the same job ID. Since Dataflow
+      // handles retrying of failed bundles, each task has one attempt only.
+      JobID jobId = job.getJobID();
+      TaskID taskId = new TaskID(jobId, TaskType.REDUCE, hash);
+      context = new TaskAttemptContextImpl(job.getConfiguration(), new TaskAttemptID(taskId, 0));
+
+      FileOutputFormat<K, V> outputFormat = formatClass.newInstance();
+      recordWriter = outputFormat.getRecordWriter(context);
+      outputCommitter = (FileOutputCommitter) outputFormat.getOutputCommitter(context);
+    }
+
+    @Override
+    public void write(KV<K, V> value) throws Exception {
+      recordWriter.write(value.getKey(), value.getValue());
+    }
+
+    @Override
+    public String close() throws Exception {
+      // task/attempt successful
+      recordWriter.close(context);
+      outputCommitter.commitTask(context);
+
+      // result is prefix of the output file name
+      return String.format("part-r-%d", hash);
+    }
+
+    @Override
+    public WriteOperation<KV<K, V>, String> getWriteOperation() {
+      return writeOperation;
+    }
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a17a8b2e/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HDFSFileSource.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HDFSFileSource.java b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HDFSFileSource.java
index 7a0545d..de68565 100644
--- a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HDFSFileSource.java
+++ b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HDFSFileSource.java
@@ -100,11 +100,11 @@ import javax.annotation.Nullable;
 public class HDFSFileSource<K, V> extends BoundedSource<KV<K, V>> {
   private static final long serialVersionUID = 0L;
 
-  private final String filepattern;
-  private final Class<? extends FileInputFormat<?, ?>> formatClass;
-  private final Class<K> keyClass;
-  private final Class<V> valueClass;
-  private final SerializableSplit serializableSplit;
+  protected final String filepattern;
+  protected final Class<? extends FileInputFormat<?, ?>> formatClass;
+  protected final Class<K> keyClass;
+  protected final Class<V> valueClass;
+  protected final SerializableSplit serializableSplit;
 
   /**
    * Creates a {@code Read} transform that will read from an {@code HDFSFileSource}
@@ -133,9 +133,9 @@ public class HDFSFileSource<K, V> extends BoundedSource<KV<K, V>> {
   /**
    * Create a {@code HDFSFileSource} based on a file or a file pattern specification.
    */
-  private HDFSFileSource(String filepattern,
-                         Class<? extends FileInputFormat<?, ?>> formatClass, Class<K> keyClass,
-                         Class<V> valueClass) {
+  protected HDFSFileSource(String filepattern,
+                           Class<? extends FileInputFormat<?, ?>> formatClass, Class<K> keyClass,
+                           Class<V> valueClass) {
     this(filepattern, formatClass, keyClass, valueClass, null);
   }
 
@@ -143,9 +143,9 @@ public class HDFSFileSource<K, V> extends BoundedSource<KV<K, V>> {
    * Create a {@code HDFSFileSource} based on a single Hadoop input split, which won't be
    * split up further.
    */
-  private HDFSFileSource(String filepattern,
-                         Class<? extends FileInputFormat<?, ?>> formatClass, Class<K> keyClass,
-                         Class<V> valueClass, SerializableSplit serializableSplit) {
+  protected HDFSFileSource(String filepattern,
+                           Class<? extends FileInputFormat<?, ?>> formatClass, Class<K> keyClass,
+                           Class<V> valueClass, SerializableSplit serializableSplit) {
     this.filepattern = filepattern;
     this.formatClass = formatClass;
     this.keyClass = keyClass;
@@ -183,9 +183,9 @@ public class HDFSFileSource<K, V> extends BoundedSource<KV<K, V>> {
     if (serializableSplit == null) {
       return Lists.transform(computeSplits(desiredBundleSizeBytes),
           new Function<InputSplit, BoundedSource<KV<K, V>>>() {
-        @Nullable @Override
+        @Override
         public BoundedSource<KV<K, V>> apply(@Nullable InputSplit inputSplit) {
-          return new HDFSFileSource<K, V>(filepattern, formatClass, keyClass,
+          return new HDFSFileSource<>(filepattern, formatClass, keyClass,
               valueClass, new SerializableSplit(inputSplit));
         }
       });
@@ -201,7 +201,7 @@ public class HDFSFileSource<K, V> extends BoundedSource<KV<K, V>> {
     return formatClass.newInstance();
   }
 
-  private List<InputSplit> computeSplits(long desiredBundleSizeBytes) throws IOException,
+  protected List<InputSplit> computeSplits(long desiredBundleSizeBytes) throws IOException,
       IllegalAccessException, InstantiationException {
     Job job = Job.getInstance();
     FileInputFormat.setMinInputSplitSize(job, desiredBundleSizeBytes);
@@ -276,13 +276,14 @@ public class HDFSFileSource<K, V> extends BoundedSource<KV<K, V>> {
     private final BoundedSource<KV<K, V>> source;
     private final String filepattern;
     private final Class formatClass;
+    protected Job job;
 
     private FileInputFormat<?, ?> format;
     private TaskAttemptContext attemptContext;
     private List<InputSplit> splits;
     private ListIterator<InputSplit> splitsIterator;
     private Configuration conf;
-    private RecordReader<K, V> currentReader;
+    protected RecordReader<K, V> currentReader;
     private KV<K, V> currentPair;
     private volatile boolean done = false;
 
@@ -290,7 +291,7 @@ public class HDFSFileSource<K, V> extends BoundedSource<KV<K, V>> {
      * Create a {@code HDFSFileReader} based on a file or a file pattern specification.
      */
     public HDFSFileReader(BoundedSource<KV<K, V>> source, String filepattern,
-                          Class<? extends FileInputFormat<?, ?>> formatClass) {
+                          Class<? extends FileInputFormat<?, ?>> formatClass) throws IOException {
       this(source, filepattern, formatClass, null);
     }
 
@@ -298,7 +299,8 @@ public class HDFSFileSource<K, V> extends BoundedSource<KV<K, V>> {
      * Create a {@code HDFSFileReader} based on a single Hadoop input split.
      */
     public HDFSFileReader(BoundedSource<KV<K, V>> source, String filepattern,
-                          Class<? extends FileInputFormat<?, ?>> formatClass, InputSplit split) {
+                          Class<? extends FileInputFormat<?, ?>> formatClass, InputSplit split)
+            throws IOException {
       this.source = source;
       this.filepattern = filepattern;
       this.formatClass = formatClass;
@@ -306,11 +308,11 @@ public class HDFSFileSource<K, V> extends BoundedSource<KV<K, V>> {
         this.splits = ImmutableList.of(split);
         this.splitsIterator = splits.listIterator();
       }
+      this.job = Job.getInstance(); // new instance
     }
 
     @Override
     public boolean start() throws IOException {
-      Job job = Job.getInstance(); // new instance
       Path path = new Path(filepattern);
       FileInputFormat.addInputPath(job, path);
 
@@ -369,7 +371,7 @@ public class HDFSFileSource<K, V> extends BoundedSource<KV<K, V>> {
     }
 
     @SuppressWarnings("unchecked")
-    private KV<K, V> nextPair() throws IOException, InterruptedException {
+    protected KV<K, V> nextPair() throws IOException, InterruptedException {
       K key = currentReader.getCurrentKey();
       V value = currentReader.getCurrentValue();
       // clone Writable objects since they are reused between calls to RecordReader#nextKeyValue

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a17a8b2e/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/WritableCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/WritableCoder.java b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/WritableCoder.java
index 814a762..4e913ed 100644
--- a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/WritableCoder.java
+++ b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/WritableCoder.java
@@ -23,6 +23,7 @@ import org.apache.beam.sdk.coders.StandardCoder;
 import org.apache.beam.sdk.util.CloudObject;
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.io.Writable;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
@@ -32,8 +33,7 @@ import java.io.OutputStream;
 import java.util.List;
 
 /**
- * A {@code WritableCoder} is a {@link org.apache.beam.sdk.coders.Coder} for a
- * Java class that implements {@link org.apache.hadoop.io.Writable}.
+ * A {@code WritableCoder} is a {@link Coder} for a Java class that implements {@link Writable}.
  *
  * <p> To use, specify the coder type on a PCollection:
  * <pre>
@@ -79,9 +79,14 @@ public class WritableCoder<T extends Writable> extends StandardCoder<T> {
     value.write(new DataOutputStream(outStream));
   }
 
+  @SuppressWarnings("unchecked")
   @Override
   public T decode(InputStream inStream, Context context) throws IOException {
     try {
+      if (type == NullWritable.class) {
+        // NullWritable has no default constructor
+        return (T) NullWritable.get();
+      }
       T t = type.newInstance();
       t.readFields(new DataInputStream(inStream));
       return t;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a17a8b2e/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/simpleauth/SimpleAuthAvroHDFSFileSource.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/simpleauth/SimpleAuthAvroHDFSFileSource.java b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/simpleauth/SimpleAuthAvroHDFSFileSource.java
new file mode 100644
index 0000000..5dd9673
--- /dev/null
+++ b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/simpleauth/SimpleAuthAvroHDFSFileSource.java
@@ -0,0 +1,84 @@
+/*
+ * 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.hdfs.simpleauth;
+
+import org.apache.beam.sdk.coders.AvroCoder;
+import org.apache.beam.sdk.io.hdfs.AvroHDFSFileSource;
+import org.apache.beam.sdk.io.hdfs.HDFSFileSource;
+import org.apache.beam.sdk.options.PipelineOptions;
+
+import com.google.common.base.Function;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+import org.apache.hadoop.mapreduce.InputSplit;
+
+import java.util.List;
+import javax.annotation.Nullable;
+
+/**
+ * Source for Avros on Hadoop/HDFS with Simple Authentication.
+ *
+ * Allows to set arbitrary username as HDFS user, which is used for reading Avro from HDFS.
+ */
+public class SimpleAuthAvroHDFSFileSource<T> extends AvroHDFSFileSource<T> {
+  // keep this field to pass Hadoop user between workers
+  private final String username;
+
+  /**
+   * Create a {@code SimpleAuthAvroHDFSFileSource} based on a file or a file pattern specification.
+   * @param username HDFS username.
+   */
+  public SimpleAuthAvroHDFSFileSource(String filepattern,
+                                      AvroCoder<T> avroCoder,
+                                      String username) {
+    super(filepattern, avroCoder);
+    this.username = username;
+  }
+
+  /**
+   * Create a {@code SimpleAuthAvroHDFSFileSource} based on a single Hadoop input split, which won't
+   * be split up further.
+   * @param username HDFS username.
+   */
+  public SimpleAuthAvroHDFSFileSource(String filepattern,
+                                      AvroCoder<T> avroCoder,
+                                      HDFSFileSource.SerializableSplit serializableSplit,
+                                      String username) {
+    super(filepattern, avroCoder, serializableSplit);
+    this.username = username;
+  }
+
+  @Override
+  public List<? extends AvroHDFSFileSource<T>> splitIntoBundles(long desiredBundleSizeBytes,
+                                                                PipelineOptions options)
+      throws Exception {
+    if (serializableSplit == null) {
+      return Lists.transform(computeSplits(desiredBundleSizeBytes),
+          new Function<InputSplit, AvroHDFSFileSource<T>>() {
+            @Override
+            public AvroHDFSFileSource<T> apply(@Nullable InputSplit inputSplit) {
+              return new SimpleAuthAvroHDFSFileSource<>(
+                  filepattern, avroCoder, new HDFSFileSource.SerializableSplit(inputSplit),
+                  username);
+            }
+          });
+    } else {
+      return ImmutableList.of(this);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a17a8b2e/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/simpleauth/SimpleAuthHDFSFileSink.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/simpleauth/SimpleAuthHDFSFileSink.java b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/simpleauth/SimpleAuthHDFSFileSink.java
new file mode 100644
index 0000000..d0fd8b6
--- /dev/null
+++ b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/simpleauth/SimpleAuthHDFSFileSink.java
@@ -0,0 +1,132 @@
+/*
+ * 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.hdfs.simpleauth;
+
+import org.apache.beam.sdk.io.Sink;
+import org.apache.beam.sdk.io.hdfs.HDFSFileSink;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.values.KV;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import java.security.PrivilegedExceptionAction;
+
+/**
+ * A {@code Sink} for writing records to a Hadoop filesystem using a Hadoop file-based output
+ * format with Simple Authentication.
+ *
+ * Allows arbitrary username as HDFS user, which is used for writing to HDFS.
+ *
+ * @param <K> The type of keys to be written to the sink.
+ * @param <V> The type of values to be written to the sink.
+ */
+public class SimpleAuthHDFSFileSink<K, V> extends HDFSFileSink<K, V> {
+  private final String username;
+
+  public SimpleAuthHDFSFileSink(String path,
+                                Class<? extends FileOutputFormat<K, V>> formatClass,
+                                Configuration conf,
+                                String username) {
+    super(path, formatClass, conf);
+    this.username = username;
+  }
+
+  @Override
+  public WriteOperation<KV<K, V>, ?> createWriteOperation(PipelineOptions options) {
+    return new SimpleAuthHDFSWriteOperation<>(this, path, formatClass, username);
+  }
+
+  /** {{@link WriteOperation}} for HDFS with Simple Authentication. */
+  public static class SimpleAuthHDFSWriteOperation<K, V> extends HDFSWriteOperation<K, V> {
+    private final String username;
+
+    SimpleAuthHDFSWriteOperation(Sink<KV<K, V>> sink,
+                                 String path,
+                                 Class<? extends FileOutputFormat<K, V>> formatClass,
+                                 String username) {
+      super(sink, path, formatClass);
+      this.username = username;
+    }
+
+    @Override
+    public void finalize(final Iterable<String> writerResults, final PipelineOptions options)
+        throws Exception {
+      UserGroupInformation.createRemoteUser(username).doAs(new PrivilegedExceptionAction<Void>() {
+        @Override
+        public Void run() throws Exception {
+          superFinalize(writerResults, options);
+          return null;
+        }
+      });
+    }
+
+    private void superFinalize(Iterable<String> writerResults, PipelineOptions options)
+        throws Exception {
+      super.finalize(writerResults, options);
+    }
+
+    @Override
+    public Writer<KV<K, V>, String> createWriter(PipelineOptions options) throws Exception {
+      return new SimpleAuthHDFSWriter<>(this, path, formatClass, username);
+    }
+  }
+
+  /** {{@link Writer}} for HDFS files with Simple Authentication. */
+  public static class SimpleAuthHDFSWriter<K, V> extends HDFSWriter<K, V> {
+    private final UserGroupInformation ugi;
+
+    public SimpleAuthHDFSWriter(SimpleAuthHDFSWriteOperation<K, V> writeOperation,
+                                String path,
+                                Class<? extends FileOutputFormat<K, V>> formatClass,
+                                String username) {
+      super(writeOperation, path, formatClass);
+      ugi = UserGroupInformation.createRemoteUser(username);
+    }
+
+    @Override
+    public void open(final String uId) throws Exception {
+      ugi.doAs(new PrivilegedExceptionAction<Void>() {
+        @Override
+        public Void run() throws Exception {
+          superOpen(uId);
+          return null;
+        }
+      });
+    }
+
+    private void superOpen(String uId) throws Exception {
+      super.open(uId);
+    }
+
+    @Override
+    public String close() throws Exception {
+      return ugi.doAs(new PrivilegedExceptionAction<String>() {
+        @Override
+        public String run() throws Exception {
+          return superClose();
+        }
+      });
+    }
+
+    private String superClose() throws Exception {
+      return super.close();
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a17a8b2e/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/simpleauth/SimpleAuthHDFSFileSource.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/simpleauth/SimpleAuthHDFSFileSource.java b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/simpleauth/SimpleAuthHDFSFileSource.java
new file mode 100644
index 0000000..5b768fc
--- /dev/null
+++ b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/simpleauth/SimpleAuthHDFSFileSource.java
@@ -0,0 +1,122 @@
+/*
+ * 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.hdfs.simpleauth;
+
+import org.apache.beam.sdk.io.BoundedSource;
+import org.apache.beam.sdk.io.Read;
+import org.apache.beam.sdk.io.hdfs.HDFSFileSource;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.values.KV;
+
+import com.google.common.base.Function;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+
+import java.util.List;
+import javax.annotation.Nullable;
+
+/**
+ * Source for Hadoop/HDFS with Simple Authentication.
+ *
+ * Allows to set arbitrary username as HDFS user, which is used for reading from HDFS.
+ */
+public class SimpleAuthHDFSFileSource<K, V> extends HDFSFileSource<K, V> {
+  private final String username;
+  /**
+   * Create a {@code SimpleAuthHDFSFileSource} based on a single Hadoop input split, which won't be
+   * split up further.
+   * @param username HDFS username.
+   */
+  protected SimpleAuthHDFSFileSource(String filepattern,
+                                     Class<? extends FileInputFormat<?, ?>> formatClass,
+                                     Class<K> keyClass,
+                                     Class<V> valueClass,
+                                     HDFSFileSource.SerializableSplit serializableSplit,
+                                     String username) {
+    super(filepattern, formatClass, keyClass, valueClass, serializableSplit);
+    this.username = username;
+  }
+
+  /**
+   * Create a {@code SimpleAuthHDFSFileSource} based on a file or a file pattern specification.
+   * @param username HDFS username.
+   */
+  protected SimpleAuthHDFSFileSource(String filepattern,
+                                     Class<? extends FileInputFormat<?, ?>> formatClass,
+                                     Class<K> keyClass,
+                                     Class<V> valueClass,
+                                     String username) {
+    super(filepattern, formatClass, keyClass, valueClass);
+    this.username = username;
+  }
+
+  /**
+   * Creates a {@code Read} transform that will read from an {@code SimpleAuthHDFSFileSource}
+   * with the given file name or pattern ("glob") using the given Hadoop {@link FileInputFormat},
+   * with key-value types specified
+   * by the given key class and value class.
+   * @param username HDFS username.
+   */
+  public static <K, V, T extends FileInputFormat<K, V>> Read.Bounded<KV<K, V>> readFrom(
+      String filepattern,
+      Class<T> formatClass,
+      Class<K> keyClass,
+      Class<V> valueClass,
+      String username) {
+    return Read.from(from(filepattern, formatClass, keyClass, valueClass, username));
+  }
+
+  /**
+   * Creates a {@code SimpleAuthHDFSFileSource} that reads from the given file name or pattern
+   * ("glob") using the given Hadoop {@link FileInputFormat}, with key-value types specified by the
+   * given key class and value class.
+   * @param username HDFS username.
+   */
+  public static <K, V, T extends FileInputFormat<K, V>> HDFSFileSource<K, V> from(
+      String filepattern,
+      Class<T> formatClass,
+      Class<K> keyClass,
+      Class<V> valueClass,
+      String username) {
+    @SuppressWarnings("unchecked")
+    HDFSFileSource<K, V> source = (HDFSFileSource<K, V>)
+        new SimpleAuthHDFSFileSource(filepattern, formatClass, keyClass, valueClass, username);
+    return source;
+  }
+
+  @Override
+  public List<? extends BoundedSource<KV<K, V>>> splitIntoBundles(
+      long desiredBundleSizeBytes, PipelineOptions options) throws Exception {
+    if (serializableSplit == null) {
+      return Lists.transform(computeSplits(desiredBundleSizeBytes),
+          new Function<InputSplit, BoundedSource<KV<K, V>>>() {
+            @Nullable
+            @Override
+            public BoundedSource<KV<K, V>> apply(@Nullable InputSplit inputSplit) {
+              return new SimpleAuthHDFSFileSource<>(filepattern, formatClass, keyClass,
+                  valueClass, new HDFSFileSource.SerializableSplit(inputSplit),
+                  username);
+            }
+          });
+    } else {
+      return ImmutableList.of(this);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a17a8b2e/sdks/java/io/hdfs/src/test/java/org/apache/beam/sdk/io/hdfs/AvroWrapperCoderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/hdfs/src/test/java/org/apache/beam/sdk/io/hdfs/AvroWrapperCoderTest.java b/sdks/java/io/hdfs/src/test/java/org/apache/beam/sdk/io/hdfs/AvroWrapperCoderTest.java
new file mode 100644
index 0000000..85cbd46
--- /dev/null
+++ b/sdks/java/io/hdfs/src/test/java/org/apache/beam/sdk/io/hdfs/AvroWrapperCoderTest.java
@@ -0,0 +1,52 @@
+/*
+ * 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.hdfs;
+
+import org.apache.beam.sdk.coders.AvroCoder;
+import org.apache.beam.sdk.testing.CoderProperties;
+
+import org.apache.avro.mapred.AvroKey;
+import org.apache.avro.mapred.AvroValue;
+import org.junit.Test;
+
+/**
+ * Tests for AvroWrapperCoder.
+ */
+public class AvroWrapperCoderTest {
+
+  @Test
+  public void testAvroKeyEncoding() throws Exception {
+    AvroKey<Integer> value = new AvroKey<>(42);
+    AvroWrapperCoder<AvroKey<Integer>, Integer> coder = AvroWrapperCoder.of(
+        AvroHDFSFileSource.ClassUtil.<AvroKey<Integer>>castClass(AvroKey.class),
+        AvroCoder.of(Integer.class));
+
+    CoderProperties.coderDecodeEncodeEqual(coder, value);
+  }
+
+  @Test
+  public void testAvroValueEncoding() throws Exception {
+    AvroValue<Integer> value = new AvroValue<>(42);
+    AvroWrapperCoder<AvroValue<Integer>, Integer> coder = AvroWrapperCoder.of(
+        AvroHDFSFileSource.ClassUtil.<AvroValue<Integer>>castClass(AvroValue.class),
+        AvroCoder.of(Integer.class));
+
+    CoderProperties.coderDecodeEncodeEqual(coder, value);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a17a8b2e/sdks/java/io/hdfs/src/test/java/org/apache/beam/sdk/io/hdfs/WritableCoderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/hdfs/src/test/java/org/apache/beam/sdk/io/hdfs/WritableCoderTest.java b/sdks/java/io/hdfs/src/test/java/org/apache/beam/sdk/io/hdfs/WritableCoderTest.java
index 715da8e..ac32c33 100644
--- a/sdks/java/io/hdfs/src/test/java/org/apache/beam/sdk/io/hdfs/WritableCoderTest.java
+++ b/sdks/java/io/hdfs/src/test/java/org/apache/beam/sdk/io/hdfs/WritableCoderTest.java
@@ -20,6 +20,7 @@ package org.apache.beam.sdk.io.hdfs;
 import org.apache.beam.sdk.testing.CoderProperties;
 
 import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.NullWritable;
 import org.junit.Test;
 
 /**
@@ -34,4 +35,12 @@ public class WritableCoderTest {
 
     CoderProperties.coderDecodeEncodeEqual(coder, value);
   }
+
+  @Test
+  public void testNullWritableEncoding() throws Exception {
+    NullWritable value = NullWritable.get();
+    WritableCoder<NullWritable> coder = WritableCoder.of(NullWritable.class);
+
+    CoderProperties.coderDecodeEncodeEqual(coder, value);
+  }
 }


[02/50] [abbrv] incubator-beam git commit: Remove many uses of .named methods

Posted by lc...@apache.org.
Remove many uses of .named methods

Specifically, remove uses of:
  - Window.named
  - AvroIO.named
  - PubSubIO.named
  - TextIO.named
  - BigQueryIO.named
  - Read.named


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

Branch: refs/heads/runners-spark2
Commit: 0fd0bd510ea82763383ece1a23a8ff2fbdc23b84
Parents: a25322e
Author: Ben Chambers <bc...@google.com>
Authored: Thu Jun 23 22:27:05 2016 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Jul 6 10:18:49 2016 -0700

----------------------------------------------------------------------
 .../beam/examples/DebuggingWordCount.java       |   2 +-
 .../org/apache/beam/examples/WordCount.java     |   4 +-
 .../apache/beam/examples/complete/TfIdf.java    |   3 +-
 .../examples/complete/TopWikipediaSessions.java |   2 +-
 .../examples/cookbook/DatastoreWordCount.java   |   2 +-
 .../beam/examples/cookbook/DeDupExample.java    |   5 +-
 .../beam/examples/cookbook/TriggerExample.java  |   4 +-
 .../beam/examples/complete/game/GameStats.java  |  28 ++-
 .../examples/complete/game/HourlyTeamScore.java |   5 +-
 .../examples/complete/game/LeaderBoard.java     |   8 +-
 .../beam/runners/flink/examples/TFIDF.java      |   3 +-
 .../beam/runners/flink/examples/WordCount.java  |   4 +-
 .../flink/examples/streaming/AutoComplete.java  |   9 +-
 .../flink/examples/streaming/JoinExamples.java  |  13 +-
 .../KafkaWindowedWordCountExample.java          |   2 +-
 .../examples/streaming/WindowedWordCount.java   |   3 +-
 .../beam/runners/dataflow/DataflowRunner.java   |   2 +-
 .../DataflowPipelineTranslatorTest.java         |   6 +-
 .../runners/dataflow/DataflowRunnerTest.java    |  18 +-
 .../beam/runners/spark/SimpleWordCountTest.java |   3 +-
 .../java/org/apache/beam/sdk/io/BigQueryIO.java |   1 -
 .../beam/sdk/io/AvroIOGeneratedClassTest.java   | 192 +++++--------------
 .../java/org/apache/beam/sdk/io/AvroIOTest.java |   5 +-
 .../org/apache/beam/sdk/io/BigQueryIOTest.java  |  82 +++-----
 .../apache/beam/sdk/io/FileBasedSourceTest.java |   5 +-
 .../org/apache/beam/sdk/io/PubsubIOTest.java    |   4 -
 .../java/org/apache/beam/sdk/io/TextIOTest.java |  37 +---
 .../org/apache/beam/sdk/io/XmlSourceTest.java   |  19 +-
 .../beam/sdk/runners/TransformTreeTest.java     |   4 +-
 .../sdk/transforms/windowing/WindowTest.java    |   6 +-
 .../sdk/transforms/windowing/WindowingTest.java |   2 +-
 .../src/main/java/DebuggingWordCount.java       |   2 +-
 .../src/main/java/WordCount.java                |   4 +-
 33 files changed, 158 insertions(+), 331 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fd0bd51/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java b/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java
index 85823c2..8d85d44 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java
@@ -173,7 +173,7 @@ public class DebuggingWordCount {
     Pipeline p = Pipeline.create(options);
 
     PCollection<KV<String, Long>> filteredWords =
-        p.apply(TextIO.Read.named("ReadLines").from(options.getInputFile()))
+        p.apply("ReadLines", TextIO.Read.from(options.getInputFile()))
          .apply(new WordCount.CountWords())
          .apply(ParDo.of(new FilterTextFn(options.getFilterPattern())));
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fd0bd51/examples/java/src/main/java/org/apache/beam/examples/WordCount.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/WordCount.java b/examples/java/src/main/java/org/apache/beam/examples/WordCount.java
index cf6c45a..af16c44 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/WordCount.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/WordCount.java
@@ -205,10 +205,10 @@ public class WordCount {
 
     // Concepts #2 and #3: Our pipeline applies the composite CountWords transform, and passes the
     // static FormatAsTextFn() to the ParDo transform.
-    p.apply(TextIO.Read.named("ReadLines").from(options.getInputFile()))
+    p.apply("ReadLines", TextIO.Read.from(options.getInputFile()))
      .apply(new CountWords())
      .apply(MapElements.via(new FormatAsTextFn()))
-     .apply(TextIO.Write.named("WriteCounts").to(options.getOutput()));
+     .apply("WriteCounts", TextIO.Write.to(options.getOutput()));
 
     p.run();
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fd0bd51/examples/java/src/main/java/org/apache/beam/examples/complete/TfIdf.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/TfIdf.java b/examples/java/src/main/java/org/apache/beam/examples/complete/TfIdf.java
index 23653d6..8305314 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/complete/TfIdf.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/complete/TfIdf.java
@@ -187,8 +187,7 @@ public class TfIdf {
         }
 
         PCollection<KV<URI, String>> oneUriToLines = pipeline
-            .apply(TextIO.Read.from(uriString)
-                .named("TextIO.Read(" + uriString + ")"))
+            .apply("TextIO.Read(" + uriString + ")", TextIO.Read.from(uriString))
             .apply("WithKeys(" + uriString + ")", WithKeys.<URI, String>of(uri));
 
         urisToLines = urisToLines.and(oneUriToLines);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fd0bd51/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java b/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java
index 5d95e3f..80b3ade 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java
@@ -217,7 +217,7 @@ public class TopWikipediaSessions {
         .from(options.getInput())
         .withCoder(TableRowJsonCoder.of()))
      .apply(new ComputeTopSessions(samplingThreshold))
-     .apply(TextIO.Write.named("Write").withoutSharding().to(options.getOutput()));
+     .apply("Write", TextIO.Write.withoutSharding().to(options.getOutput()));
 
     p.run();
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fd0bd51/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java
index 7578d79..b070f94 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java
@@ -193,7 +193,7 @@ public class DatastoreWordCount {
    */
   public static void writeDataToDatastore(Options options) {
       Pipeline p = Pipeline.create(options);
-      p.apply(TextIO.Read.named("ReadLines").from(options.getInput()))
+      p.apply("ReadLines", TextIO.Read.from(options.getInput()))
        .apply(ParDo.of(new CreateEntityFn(options.getNamespace(), options.getKind())))
        .apply(DatastoreIO.writeTo(options.getProject()));
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fd0bd51/examples/java/src/main/java/org/apache/beam/examples/cookbook/DeDupExample.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/DeDupExample.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DeDupExample.java
index db65543..d573bcd 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/DeDupExample.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DeDupExample.java
@@ -89,10 +89,9 @@ public class DeDupExample {
     Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
     Pipeline p = Pipeline.create(options);
 
-    p.apply(TextIO.Read.named("ReadLines").from(options.getInput()))
+    p.apply("ReadLines", TextIO.Read.from(options.getInput()))
      .apply(RemoveDuplicates.<String>create())
-     .apply(TextIO.Write.named("DedupedShakespeare")
-         .to(options.getOutput()));
+     .apply("DedupedShakespeare", TextIO.Write.to(options.getOutput()));
 
     p.run();
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fd0bd51/examples/java/src/main/java/org/apache/beam/examples/cookbook/TriggerExample.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/TriggerExample.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/TriggerExample.java
index ab1fb66..ff4909b 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/TriggerExample.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/TriggerExample.java
@@ -467,7 +467,7 @@ public class TriggerExample {
     TableReference tableRef = getTableReference(options.getProject(),
         options.getBigQueryDataset(), options.getBigQueryTable());
 
-    PCollectionList<TableRow> resultList = pipeline.apply(PubsubIO.Read.named("ReadPubsubInput")
+    PCollectionList<TableRow> resultList = pipeline.apply("ReadPubsubInput", PubsubIO.Read
         .timestampLabel(PUBSUB_TIMESTAMP_LABEL_KEY)
         .topic(options.getPubsubTopic()))
         .apply(ParDo.of(new ExtractFlowInfo()))
@@ -493,7 +493,7 @@ public class TriggerExample {
     copiedOptions.setJobName(options.getJobName() + "-injector");
     Pipeline injectorPipeline = Pipeline.create(copiedOptions);
     injectorPipeline
-    .apply(TextIO.Read.named("ReadMyFile").from(options.getInput()))
+    .apply("ReadMyFile", TextIO.Read.from(options.getInput()))
     .apply("InsertRandomDelays", ParDo.of(new InsertDelays()))
     .apply(IntraBundleParallelization.of(PubsubFileInjector
         .withTimestampLabelKey(PUBSUB_TIMESTAMP_LABEL_KEY)

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fd0bd51/examples/java8/src/main/java/org/apache/beam/examples/complete/game/GameStats.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/GameStats.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/GameStats.java
index ad8b49e..b1cb312 100644
--- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/GameStats.java
+++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/GameStats.java
@@ -260,10 +260,8 @@ public class GameStats extends LeaderBoard {
     // Calculate the total score per user over fixed windows, and
     // cumulative updates for late data.
     final PCollectionView<Map<String, Integer>> spammersView = userEvents
-      .apply(Window.named("FixedWindowsUser")
-          .<KV<String, Integer>>into(FixedWindows.of(
-              Duration.standardMinutes(options.getFixedWindowDuration())))
-          )
+      .apply("FixedWindowsUser", Window.<KV<String, Integer>>into(
+          FixedWindows.of(Duration.standardMinutes(options.getFixedWindowDuration()))))
 
       // Filter out everyone but those with (SCORE_WEIGHT * avg) clickrate.
       // These might be robots/spammers.
@@ -278,10 +276,8 @@ public class GameStats extends LeaderBoard {
     // suspected robots-- to filter out scores from those users from the sum.
     // Write the results to BigQuery.
     rawEvents
-      .apply(Window.named("WindowIntoFixedWindows")
-          .<GameActionInfo>into(FixedWindows.of(
-              Duration.standardMinutes(options.getFixedWindowDuration())))
-          )
+      .apply("WindowIntoFixedWindows", Window.<GameActionInfo>into(
+          FixedWindows.of(Duration.standardMinutes(options.getFixedWindowDuration()))))
       // Filter out the detected spammer users, using the side input derived above.
       .apply("FilterOutSpammers", ParDo
               .withSideInputs(spammersView)
@@ -299,8 +295,8 @@ public class GameStats extends LeaderBoard {
       // [END DocInclude_FilterAndCalc]
       // Write the result to BigQuery
       .apply("WriteTeamSums",
-             new WriteWindowedToBigQuery<KV<String, Integer>>(
-                options.getTablePrefix() + "_team", configureWindowedWrite()));
+          new WriteWindowedToBigQuery<KV<String, Integer>>(
+              options.getTablePrefix() + "_team", configureWindowedWrite()));
 
 
     // [START DocInclude_SessionCalc]
@@ -309,10 +305,9 @@ public class GameStats extends LeaderBoard {
     // This information could help the game designers track the changing user engagement
     // as their set of games changes.
     userEvents
-      .apply(Window.named("WindowIntoSessions")
-            .<KV<String, Integer>>into(
-                  Sessions.withGapDuration(Duration.standardMinutes(options.getSessionGap())))
-        .withOutputTimeFn(OutputTimeFns.outputAtEndOfWindow()))
+      .apply("WindowIntoSessions", Window.<KV<String, Integer>>into(
+          Sessions.withGapDuration(Duration.standardMinutes(options.getSessionGap())))
+          .withOutputTimeFn(OutputTimeFns.outputAtEndOfWindow()))
       // For this use, we care only about the existence of the session, not any particular
       // information aggregated over it, so the following is an efficient way to do that.
       .apply(Combine.perKey(x -> 0))
@@ -321,9 +316,8 @@ public class GameStats extends LeaderBoard {
       // [END DocInclude_SessionCalc]
       // [START DocInclude_Rewindow]
       // Re-window to process groups of session sums according to when the sessions complete.
-      .apply(Window.named("WindowToExtractSessionMean")
-            .<Integer>into(
-                FixedWindows.of(Duration.standardMinutes(options.getUserActivityWindowDuration()))))
+      .apply("WindowToExtractSessionMean", Window.<Integer>into(
+          FixedWindows.of(Duration.standardMinutes(options.getUserActivityWindowDuration()))))
       // Find the mean session duration in each window.
       .apply(Mean.<Integer>globally().withoutDefaults())
       // Write this info to a BigQuery table.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fd0bd51/examples/java8/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java
index 7a808ac..e489607 100644
--- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java
+++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java
@@ -178,9 +178,8 @@ public class HourlyTeamScore extends UserScore {
       // Add an element timestamp based on the event log, and apply fixed windowing.
       .apply("AddEventTimestamps",
              WithTimestamps.of((GameActionInfo i) -> new Instant(i.getTimestamp())))
-      .apply(Window.named("FixedWindowsTeam")
-          .<GameActionInfo>into(FixedWindows.of(
-                Duration.standardMinutes(options.getWindowDuration()))))
+      .apply("FixedWindowsTeam", Window.<GameActionInfo>into(
+          FixedWindows.of(Duration.standardMinutes(options.getWindowDuration()))))
       // [END DocInclude_HTSAddTsAndWindow]
 
       // Extract and sum teamname/score pairs from the event data.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fd0bd51/examples/java8/src/main/java/org/apache/beam/examples/complete/game/LeaderBoard.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/LeaderBoard.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/LeaderBoard.java
index 2c608aa..a14d533 100644
--- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/LeaderBoard.java
+++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/LeaderBoard.java
@@ -190,9 +190,8 @@ public class LeaderBoard extends HourlyTeamScore {
     // [START DocInclude_WindowAndTrigger]
     // Extract team/score pairs from the event stream, using hour-long windows by default.
     gameEvents
-        .apply(Window.named("LeaderboardTeamFixedWindows")
-          .<GameActionInfo>into(FixedWindows.of(
-              Duration.standardMinutes(options.getTeamWindowDuration())))
+        .apply("LeaderboardTeamFixedWindows", Window.<GameActionInfo>into(
+            FixedWindows.of(Duration.standardMinutes(options.getTeamWindowDuration())))
           // We will get early (speculative) results as well as cumulative
           // processing of late data.
           .triggering(
@@ -215,8 +214,7 @@ public class LeaderBoard extends HourlyTeamScore {
     // Extract user/score pairs from the event stream using processing time, via global windowing.
     // Get periodic updates on all users' running scores.
     gameEvents
-        .apply(Window.named("LeaderboardUserGlobalWindow")
-          .<GameActionInfo>into(new GlobalWindows())
+        .apply("LeaderboardUserGlobalWindow", Window.<GameActionInfo>into(new GlobalWindows())
           // Get periodic results every ten minutes.
               .triggering(Repeatedly.forever(AfterProcessingTime.pastFirstElementInPane()
                   .plusDelayOf(TEN_MINUTES)))

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fd0bd51/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/TFIDF.java
----------------------------------------------------------------------
diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/TFIDF.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/TFIDF.java
index 084ac7c..56737a4 100644
--- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/TFIDF.java
+++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/TFIDF.java
@@ -191,8 +191,7 @@ public class TFIDF {
         }
 
         PCollection<KV<URI, String>> oneUriToLines = pipeline
-            .apply(TextIO.Read.from(uriString)
-                .named("TextIO.Read(" + uriString + ")"))
+            .apply("TextIO.Read(" + uriString + ")", TextIO.Read.from(uriString))
             .apply("WithKeys(" + uriString + ")", WithKeys.<URI, String>of(uri));
 
         urisToLines = urisToLines.and(oneUriToLines);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fd0bd51/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/WordCount.java
----------------------------------------------------------------------
diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/WordCount.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/WordCount.java
index 2817622..2d95c97 100644
--- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/WordCount.java
+++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/WordCount.java
@@ -109,10 +109,10 @@ public class WordCount {
 
     Pipeline p = Pipeline.create(options);
 
-    p.apply(TextIO.Read.named("ReadLines").from(options.getInput()))
+    p.apply("ReadLines", TextIO.Read.from(options.getInput()))
         .apply(new CountWords())
         .apply(MapElements.via(new FormatAsTextFn()))
-        .apply(TextIO.Write.named("WriteCounts").to(options.getOutput()));
+        .apply("WriteCounts", TextIO.Write.to(options.getOutput()));
 
     p.run();
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fd0bd51/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/AutoComplete.java
----------------------------------------------------------------------
diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/AutoComplete.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/AutoComplete.java
index ed11781..c0ff85d 100644
--- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/AutoComplete.java
+++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/AutoComplete.java
@@ -44,7 +44,6 @@ import org.apache.beam.sdk.transforms.windowing.FixedWindows;
 import org.apache.beam.sdk.transforms.windowing.Window;
 import org.apache.beam.sdk.transforms.windowing.WindowFn;
 import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PBegin;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionList;
 
@@ -380,14 +379,14 @@ public class AutoComplete {
     options.setExecutionRetryDelay(3000L);
     options.setRunner(FlinkRunner.class);
 
-    PTransform<? super PBegin, PCollection<String>> readSource =
-            Read.from(new UnboundedSocketSource<>("localhost", 9999, '\n', 3)).named("WordStream");
-    WindowFn<Object, ?> windowFn = FixedWindows.of(Duration.standardSeconds(options.getWindowSize()));
+
+    WindowFn<Object, ?> windowFn =
+        FixedWindows.of(Duration.standardSeconds(options.getWindowSize()));
 
     // Create the pipeline.
     Pipeline p = Pipeline.create(options);
     PCollection<KV<String, List<CompletionCandidate>>> toWrite = p
-      .apply(readSource)
+      .apply("WordStream", Read.from(new UnboundedSocketSource<>("localhost", 9999, '\n', 3)))
       .apply(ParDo.of(new ExtractWordsFn()))
       .apply(Window.<String>into(windowFn)
               .triggering(AfterWatermark.pastEndOfWindow()).withAllowedLateness(Duration.ZERO)

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fd0bd51/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/JoinExamples.java
----------------------------------------------------------------------
diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/JoinExamples.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/JoinExamples.java
index 0828ddc..f456b27 100644
--- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/JoinExamples.java
+++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/JoinExamples.java
@@ -24,7 +24,6 @@ import org.apache.beam.sdk.io.Read;
 import org.apache.beam.sdk.io.TextIO;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.transforms.DoFn;
-import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.transforms.join.CoGbkResult;
 import org.apache.beam.sdk.transforms.join.CoGroupByKey;
@@ -34,7 +33,6 @@ import org.apache.beam.sdk.transforms.windowing.FixedWindows;
 import org.apache.beam.sdk.transforms.windowing.Window;
 import org.apache.beam.sdk.transforms.windowing.WindowFn;
 import org.apache.beam.sdk.values.KV;
-import org.apache.beam.sdk.values.PBegin;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.TupleTag;
 
@@ -135,22 +133,19 @@ public class JoinExamples {
     options.setExecutionRetryDelay(3000L);
     options.setRunner(FlinkRunner.class);
 
-    PTransform<? super PBegin, PCollection<String>> readSourceA =
-        Read.from(new UnboundedSocketSource<>("localhost", 9999, '\n', 3)).named("FirstStream");
-    PTransform<? super PBegin, PCollection<String>> readSourceB =
-        Read.from(new UnboundedSocketSource<>("localhost", 9998, '\n', 3)).named("SecondStream");
-
     WindowFn<Object, ?> windowFn = FixedWindows.of(Duration.standardSeconds(options.getWindowSize()));
 
     Pipeline p = Pipeline.create(options);
 
     // the following two 'applys' create multiple inputs to our pipeline, one for each
     // of our two input sources.
-    PCollection<String> streamA = p.apply(readSourceA)
+    PCollection<String> streamA = p
+        .apply("FirstStream", Read.from(new UnboundedSocketSource<>("localhost", 9999, '\n', 3)))
         .apply(Window.<String>into(windowFn)
             .triggering(AfterWatermark.pastEndOfWindow()).withAllowedLateness(Duration.ZERO)
             .discardingFiredPanes());
-    PCollection<String> streamB = p.apply(readSourceB)
+    PCollection<String> streamB = p
+        .apply("SecondStream", Read.from(new UnboundedSocketSource<>("localhost", 9998, '\n', 3)))
         .apply(Window.<String>into(windowFn)
             .triggering(AfterWatermark.pastEndOfWindow()).withAllowedLateness(Duration.ZERO)
             .discardingFiredPanes());

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fd0bd51/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaWindowedWordCountExample.java
----------------------------------------------------------------------
diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaWindowedWordCountExample.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaWindowedWordCountExample.java
index b14c5ae..4e81420 100644
--- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaWindowedWordCountExample.java
+++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaWindowedWordCountExample.java
@@ -132,7 +132,7 @@ public class KafkaWindowedWordCountExample {
         new SimpleStringSchema(), p);
 
     PCollection<String> words = pipeline
-        .apply(Read.named("StreamingWordCount").from(UnboundedFlinkSource.of(kafkaConsumer)))
+        .apply("StreamingWordCount", Read.from(UnboundedFlinkSource.of(kafkaConsumer)))
         .apply(ParDo.of(new ExtractWordsFn()))
         .apply(Window.<String>into(FixedWindows.of(Duration.standardSeconds(options.getWindowSize())))
             .triggering(AfterWatermark.pastEndOfWindow()).withAllowedLateness(Duration.ZERO)

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fd0bd51/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/WindowedWordCount.java
----------------------------------------------------------------------
diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/WindowedWordCount.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/WindowedWordCount.java
index f72b705..1b532a7 100644
--- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/WindowedWordCount.java
+++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/WindowedWordCount.java
@@ -119,7 +119,8 @@ public class WindowedWordCount {
     Pipeline pipeline = Pipeline.create(options);
 
     PCollection<String> words = pipeline
-        .apply(Read.from(new UnboundedSocketSource<>("localhost", 9999, '\n', 3)).named("StreamingWordCount"))
+        .apply("StreamingWordCount",
+            Read.from(new UnboundedSocketSource<>("localhost", 9999, '\n', 3)))
         .apply(ParDo.of(new ExtractWordsFn()))
         .apply(Window.<String>into(SlidingWindows.of(Duration.standardSeconds(options.getWindowSize()))
             .every(Duration.standardSeconds(options.getSlide())))

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fd0bd51/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
index d47d285..7ff247a 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
@@ -2686,7 +2686,7 @@ public class DataflowRunner extends PipelineRunner<DataflowPipelineJob> {
       try {
         Coder<T> coder = transform.getDefaultOutputCoder(input);
         return Pipeline.applyTransform(
-            input, PubsubIO.Read.named("StartingSignal").subscription("_starting_signal/"))
+            "StartingSignal", input, PubsubIO.Read.subscription("_starting_signal/"))
             .apply(ParDo.of(new OutputNullKv()))
             .apply("GlobalSingleton", Window.<KV<Void, Void>>into(new GlobalWindows())
                 .triggering(AfterPane.elementCountAtLeast(1))

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fd0bd51/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java
index c3a6a11..e04a1fc 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java
@@ -127,8 +127,8 @@ public class DataflowPipelineTranslatorTest implements Serializable {
     options.setRunner(DataflowRunner.class);
     Pipeline p = Pipeline.create(options);
 
-    p.apply(TextIO.Read.named("ReadMyFile").from("gs://bucket/object"))
-        .apply(TextIO.Write.named("WriteMyFile").to("gs://bucket/object"));
+    p.apply("ReadMyFile", TextIO.Read.from("gs://bucket/object"))
+     .apply("WriteMyFile", TextIO.Write.to("gs://bucket/object"));
 
     return p;
   }
@@ -458,7 +458,7 @@ public class DataflowPipelineTranslatorTest implements Serializable {
 
     // Create a pipeline that the predefined step will be embedded into
     Pipeline pipeline = Pipeline.create(options);
-    pipeline.apply(TextIO.Read.named("ReadMyFile").from("gs://bucket/in"))
+    pipeline.apply("ReadMyFile", TextIO.Read.from("gs://bucket/in"))
         .apply(ParDo.of(new NoOpFn()))
         .apply(new EmbeddedTransform(predefinedStep.clone()))
         .apply(ParDo.of(new NoOpFn()));

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fd0bd51/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java
index e094d0d..999dc3a 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java
@@ -18,6 +18,7 @@
 package org.apache.beam.runners.dataflow;
 
 import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow;
+
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.hamcrest.Matchers.containsString;
 import static org.hamcrest.Matchers.equalTo;
@@ -151,8 +152,8 @@ public class DataflowRunnerTest {
     options.setRunner(DataflowRunner.class);
     Pipeline p = Pipeline.create(options);
 
-    p.apply(TextIO.Read.named("ReadMyFile").from("gs://bucket/object"))
-        .apply(TextIO.Write.named("WriteMyFile").to("gs://bucket/object"));
+    p.apply("ReadMyFile", TextIO.Read.from("gs://bucket/object"))
+        .apply("WriteMyFile", TextIO.Write.to("gs://bucket/object"));
 
     return p;
   }
@@ -464,7 +465,7 @@ public class DataflowRunnerTest {
     ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
 
     Pipeline p = buildDataflowPipeline(buildPipelineOptions(jobCaptor));
-    p.apply(TextIO.Read.named("ReadMyNonGcsFile").from(tmpFolder.newFile().getPath()));
+    p.apply("ReadMyNonGcsFile", TextIO.Read.from(tmpFolder.newFile().getPath()));
 
     thrown.expectCause(Matchers.allOf(
         instanceOf(IllegalArgumentException.class),
@@ -477,11 +478,11 @@ public class DataflowRunnerTest {
   @Test
   public void testNonGcsFilePathInWriteFailure() throws IOException {
     Pipeline p = buildDataflowPipeline(buildPipelineOptions());
-    PCollection<String> pc = p.apply(TextIO.Read.named("ReadMyGcsFile").from("gs://bucket/object"));
+    PCollection<String> pc = p.apply("ReadMyGcsFile", TextIO.Read.from("gs://bucket/object"));
 
     thrown.expect(IllegalArgumentException.class);
     thrown.expectMessage(containsString("expected a valid 'gs://' path but was given"));
-    pc.apply(TextIO.Write.named("WriteMyNonGcsFile").to("/tmp/file"));
+    pc.apply("WriteMyNonGcsFile", TextIO.Write.to("/tmp/file"));
   }
 
   @Test
@@ -489,8 +490,7 @@ public class DataflowRunnerTest {
     ArgumentCaptor<Job> jobCaptor = ArgumentCaptor.forClass(Job.class);
 
     Pipeline p = buildDataflowPipeline(buildPipelineOptions(jobCaptor));
-    p.apply(TextIO.Read.named("ReadInvalidGcsFile")
-        .from("gs://bucket/tmp//file"));
+    p.apply("ReadInvalidGcsFile", TextIO.Read.from("gs://bucket/tmp//file"));
 
     thrown.expectCause(Matchers.allOf(
         instanceOf(IllegalArgumentException.class),
@@ -502,11 +502,11 @@ public class DataflowRunnerTest {
   @Test
   public void testMultiSlashGcsFileWritePath() throws IOException {
     Pipeline p = buildDataflowPipeline(buildPipelineOptions());
-    PCollection<String> pc = p.apply(TextIO.Read.named("ReadMyGcsFile").from("gs://bucket/object"));
+    PCollection<String> pc = p.apply("ReadMyGcsFile", TextIO.Read.from("gs://bucket/object"));
 
     thrown.expect(IllegalArgumentException.class);
     thrown.expectMessage("consecutive slashes");
-    pc.apply(TextIO.Write.named("WriteInvalidGcsFile").to("gs://bucket/tmp//file"));
+    pc.apply("WriteInvalidGcsFile", TextIO.Write.to("gs://bucket/tmp//file"));
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fd0bd51/runners/spark/src/test/java/org/apache/beam/runners/spark/SimpleWordCountTest.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/SimpleWordCountTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/SimpleWordCountTest.java
index da3fa7a..6a3edd7 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/SimpleWordCountTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/SimpleWordCountTest.java
@@ -89,8 +89,7 @@ public class SimpleWordCountTest {
     PCollection<String> output = inputWords.apply(new CountWords());
 
     File outputFile = testFolder.newFile();
-    output.apply(
-        TextIO.Write.named("WriteCounts").to(outputFile.getAbsolutePath()).withoutSharding());
+    output.apply("WriteCounts", TextIO.Write.to(outputFile.getAbsolutePath()).withoutSharding());
 
     EvaluationResult res = SparkRunner.create().run(p);
     res.close();

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fd0bd51/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BigQueryIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BigQueryIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BigQueryIO.java
index 6a36c8d..7cac705 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BigQueryIO.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BigQueryIO.java
@@ -518,7 +518,6 @@ public class BigQueryIO {
               + " query without a result flattening preference");
         }
 
-        // Only verify existence/correctness if validation is enabled.
         if (validate) {
           // Check for source table/query presence for early failure notification.
           // Note that a presence check can fail if the table or dataset are created by earlier

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fd0bd51/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOGeneratedClassTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOGeneratedClassTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOGeneratedClassTest.java
index da886de..6e26d33 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOGeneratedClassTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOGeneratedClassTest.java
@@ -136,6 +136,18 @@ public class AvroIOGeneratedClassTest {
     return users;
   }
 
+  <T> void runTestRead(
+      String applyName, AvroIO.Read.Bound<T> read, String expectedName, T[] expectedOutput)
+      throws Exception {
+    generateAvroFile(generateAvroObjects());
+
+    TestPipeline p = TestPipeline.create();
+    PCollection<T> output = p.apply(applyName, read);
+    PAssert.that(output).containsInAnyOrder(expectedOutput);
+    p.run();
+    assertEquals(expectedName, output.getName());
+  }
+
   <T> void runTestRead(AvroIO.Read.Bound<T> read, String expectedName, T[] expectedOutput)
       throws Exception {
     generateAvroFile(generateAvroObjects());
@@ -158,28 +170,16 @@ public class AvroIOGeneratedClassTest {
         AvroIO.Read.withSchema(AvroGeneratedUser.class).from(avroFile.getPath()),
         "AvroIO.Read/Read.out",
         generateAvroObjects());
-    runTestRead(
-        AvroIO.Read.named("MyRead").from(avroFile.getPath()).withSchema(AvroGeneratedUser.class),
+    runTestRead("MyRead",
+        AvroIO.Read.from(avroFile.getPath()).withSchema(AvroGeneratedUser.class),
         "MyRead/Read.out",
         generateAvroObjects());
-    runTestRead(
-        AvroIO.Read.named("MyRead").withSchema(AvroGeneratedUser.class).from(avroFile.getPath()),
+    runTestRead("MyRead",
+        AvroIO.Read.withSchema(AvroGeneratedUser.class).from(avroFile.getPath()),
         "MyRead/Read.out",
         generateAvroObjects());
-    runTestRead(
-        AvroIO.Read.from(avroFile.getPath()).withSchema(AvroGeneratedUser.class).named("HerRead"),
-        "HerRead/Read.out",
-        generateAvroObjects());
-    runTestRead(
-        AvroIO.Read.from(avroFile.getPath()).named("HerRead").withSchema(AvroGeneratedUser.class),
-        "HerRead/Read.out",
-        generateAvroObjects());
-    runTestRead(
-        AvroIO.Read.withSchema(AvroGeneratedUser.class).named("HerRead").from(avroFile.getPath()),
-        "HerRead/Read.out",
-        generateAvroObjects());
-    runTestRead(
-        AvroIO.Read.withSchema(AvroGeneratedUser.class).from(avroFile.getPath()).named("HerRead"),
+    runTestRead("HerRead",
+        AvroIO.Read.from(avroFile.getPath()).withSchema(AvroGeneratedUser.class),
         "HerRead/Read.out",
         generateAvroObjects());
   }
@@ -195,28 +195,20 @@ public class AvroIOGeneratedClassTest {
         AvroIO.Read.withSchema(schema).from(avroFile.getPath()),
         "AvroIO.Read/Read.out",
         generateAvroGenericRecords());
-    runTestRead(
-        AvroIO.Read.named("MyRead").from(avroFile.getPath()).withSchema(schema),
+    runTestRead("MyRead",
+        AvroIO.Read.from(avroFile.getPath()).withSchema(schema),
         "MyRead/Read.out",
         generateAvroGenericRecords());
-    runTestRead(
-        AvroIO.Read.named("MyRead").withSchema(schema).from(avroFile.getPath()),
+    runTestRead("MyRead",
+        AvroIO.Read.withSchema(schema).from(avroFile.getPath()),
         "MyRead/Read.out",
         generateAvroGenericRecords());
-    runTestRead(
-        AvroIO.Read.from(avroFile.getPath()).withSchema(schema).named("HerRead"),
-        "HerRead/Read.out",
-        generateAvroGenericRecords());
-    runTestRead(
-        AvroIO.Read.from(avroFile.getPath()).named("HerRead").withSchema(schema),
-        "HerRead/Read.out",
-        generateAvroGenericRecords());
-    runTestRead(
-        AvroIO.Read.withSchema(schema).named("HerRead").from(avroFile.getPath()),
+    runTestRead("HerRead",
+        AvroIO.Read.from(avroFile.getPath()).withSchema(schema),
         "HerRead/Read.out",
         generateAvroGenericRecords());
-    runTestRead(
-        AvroIO.Read.withSchema(schema).from(avroFile.getPath()).named("HerRead"),
+    runTestRead("HerRead",
+        AvroIO.Read.from(avroFile.getPath()).withSchema(schema),
         "HerRead/Read.out",
         generateAvroGenericRecords());
   }
@@ -232,28 +224,12 @@ public class AvroIOGeneratedClassTest {
         AvroIO.Read.withSchema(schemaString).from(avroFile.getPath()),
         "AvroIO.Read/Read.out",
         generateAvroGenericRecords());
-    runTestRead(
-        AvroIO.Read.named("MyRead").from(avroFile.getPath()).withSchema(schemaString),
-        "MyRead/Read.out",
-        generateAvroGenericRecords());
-    runTestRead(
-        AvroIO.Read.named("MyRead").withSchema(schemaString).from(avroFile.getPath()),
+    runTestRead("MyRead",
+        AvroIO.Read.from(avroFile.getPath()).withSchema(schemaString),
         "MyRead/Read.out",
         generateAvroGenericRecords());
-    runTestRead(
-        AvroIO.Read.from(avroFile.getPath()).withSchema(schemaString).named("HerRead"),
-        "HerRead/Read.out",
-        generateAvroGenericRecords());
-    runTestRead(
-        AvroIO.Read.from(avroFile.getPath()).named("HerRead").withSchema(schemaString),
-        "HerRead/Read.out",
-        generateAvroGenericRecords());
-    runTestRead(
-        AvroIO.Read.withSchema(schemaString).named("HerRead").from(avroFile.getPath()),
-        "HerRead/Read.out",
-        generateAvroGenericRecords());
-    runTestRead(
-        AvroIO.Read.withSchema(schemaString).from(avroFile.getPath()).named("HerRead"),
+    runTestRead("HerRead",
+        AvroIO.Read.withSchema(schemaString).from(avroFile.getPath()),
         "HerRead/Read.out",
         generateAvroGenericRecords());
   }
@@ -276,106 +252,34 @@ public class AvroIOGeneratedClassTest {
   @Test
   @Category(NeedsRunner.class)
   public void testWriteFromGeneratedClass() throws Exception {
-    runTestWrite(AvroIO.Write.to(avroFile.getPath())
-                             .withSchema(AvroGeneratedUser.class),
-                 "AvroIO.Write");
-    runTestWrite(AvroIO.Write.withSchema(AvroGeneratedUser.class)
-                             .to(avroFile.getPath()),
-                 "AvroIO.Write");
-    runTestWrite(AvroIO.Write.named("MyWrite")
-                             .to(avroFile.getPath())
-                             .withSchema(AvroGeneratedUser.class),
-                 "MyWrite");
-    runTestWrite(AvroIO.Write.named("MyWrite")
-                             .withSchema(AvroGeneratedUser.class)
-                             .to(avroFile.getPath()),
-                 "MyWrite");
-    runTestWrite(AvroIO.Write.to(avroFile.getPath())
-                             .withSchema(AvroGeneratedUser.class)
-                             .named("HerWrite"),
-                 "HerWrite");
-    runTestWrite(AvroIO.Write.to(avroFile.getPath())
-                             .named("HerWrite")
-                             .withSchema(AvroGeneratedUser.class),
-                 "HerWrite");
-    runTestWrite(AvroIO.Write.withSchema(AvroGeneratedUser.class)
-                             .named("HerWrite")
-                             .to(avroFile.getPath()),
-                 "HerWrite");
-    runTestWrite(AvroIO.Write.withSchema(AvroGeneratedUser.class)
-                             .to(avroFile.getPath())
-                             .named("HerWrite"),
-                 "HerWrite");
+    runTestWrite(
+        AvroIO.Write.to(avroFile.getPath()).withSchema(AvroGeneratedUser.class),
+        "AvroIO.Write");
+    runTestWrite(
+        AvroIO.Write.withSchema(AvroGeneratedUser.class).to(avroFile.getPath()),
+        "AvroIO.Write");
   }
 
   @Test
   @Category(NeedsRunner.class)
   public void testWriteFromSchema() throws Exception {
-    runTestWrite(AvroIO.Write.to(avroFile.getPath())
-                             .withSchema(schema),
-                 "AvroIO.Write");
-    runTestWrite(AvroIO.Write.withSchema(schema)
-                             .to(avroFile.getPath()),
-                 "AvroIO.Write");
-    runTestWrite(AvroIO.Write.named("MyWrite")
-                             .to(avroFile.getPath())
-                             .withSchema(schema),
-                 "MyWrite");
-    runTestWrite(AvroIO.Write.named("MyWrite")
-                             .withSchema(schema)
-                             .to(avroFile.getPath()),
-                 "MyWrite");
-    runTestWrite(AvroIO.Write.to(avroFile.getPath())
-                             .withSchema(schema)
-                             .named("HerWrite"),
-                 "HerWrite");
-    runTestWrite(AvroIO.Write.to(avroFile.getPath())
-                             .named("HerWrite")
-                             .withSchema(schema),
-                 "HerWrite");
-    runTestWrite(AvroIO.Write.withSchema(schema)
-                             .named("HerWrite")
-                             .to(avroFile.getPath()),
-                 "HerWrite");
-    runTestWrite(AvroIO.Write.withSchema(schema)
-                             .to(avroFile.getPath())
-                             .named("HerWrite"),
-                 "HerWrite");
+    runTestWrite(
+        AvroIO.Write.to(avroFile.getPath()).withSchema(schema),
+        "AvroIO.Write");
+    runTestWrite(
+        AvroIO.Write.withSchema(schema).to(avroFile.getPath()),
+        "AvroIO.Write");
   }
 
   @Test
   @Category(NeedsRunner.class)
   public void testWriteFromSchemaString() throws Exception {
-    runTestWrite(AvroIO.Write.to(avroFile.getPath())
-                             .withSchema(schemaString),
-                 "AvroIO.Write");
-    runTestWrite(AvroIO.Write.withSchema(schemaString)
-                             .to(avroFile.getPath()),
-                 "AvroIO.Write");
-    runTestWrite(AvroIO.Write.named("MyWrite")
-                             .to(avroFile.getPath())
-                             .withSchema(schemaString),
-                 "MyWrite");
-    runTestWrite(AvroIO.Write.named("MyWrite")
-                             .withSchema(schemaString)
-                             .to(avroFile.getPath()),
-                 "MyWrite");
-    runTestWrite(AvroIO.Write.to(avroFile.getPath())
-                             .withSchema(schemaString)
-                             .named("HerWrite"),
-                 "HerWrite");
-    runTestWrite(AvroIO.Write.to(avroFile.getPath())
-                             .named("HerWrite")
-                             .withSchema(schemaString),
-                 "HerWrite");
-    runTestWrite(AvroIO.Write.withSchema(schemaString)
-                             .named("HerWrite")
-                             .to(avroFile.getPath()),
-                 "HerWrite");
-    runTestWrite(AvroIO.Write.withSchema(schemaString)
-                             .to(avroFile.getPath())
-                             .named("HerWrite"),
-                 "HerWrite");
+    runTestWrite(
+        AvroIO.Write.to(avroFile.getPath()).withSchema(schemaString),
+        "AvroIO.Write");
+    runTestWrite(
+        AvroIO.Write.withSchema(schemaString).to(avroFile.getPath()),
+        "AvroIO.Write");
   }
 
   // TODO: for Write only, test withSuffix, withNumShards,

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fd0bd51/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOTest.java
index 13c1bcf..8625b10 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOTest.java
@@ -18,6 +18,7 @@
 package org.apache.beam.sdk.io;
 
 import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem;
+
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -81,10 +82,6 @@ public class AvroIOTest {
   public void testAvroIOGetName() {
     assertEquals("AvroIO.Read", AvroIO.Read.from("gs://bucket/foo*/baz").getName());
     assertEquals("AvroIO.Write", AvroIO.Write.to("gs://bucket/foo/baz").getName());
-    assertEquals("ReadMyFile",
-        AvroIO.Read.named("ReadMyFile").from("gs://bucket/foo*/baz").getName());
-    assertEquals("WriteMyFile",
-        AvroIO.Write.named("WriteMyFile").to("gs://bucket/foo/baz").getName());
   }
 
   @DefaultCoder(AvroCoder.class)

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fd0bd51/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BigQueryIOTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BigQueryIOTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BigQueryIOTest.java
index a1daf72..f0d3fce 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BigQueryIOTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BigQueryIOTest.java
@@ -340,8 +340,7 @@ public class BigQueryIOTest implements Serializable {
     checkReadTableObjectWithValidate(bound, project, dataset, table, true);
   }
 
-  private void checkReadQueryObject(
-      BigQueryIO.Read.Bound bound, String query) {
+  private void checkReadQueryObject(BigQueryIO.Read.Bound bound, String query) {
     checkReadQueryObjectWithValidate(bound, query, true);
   }
 
@@ -393,15 +392,13 @@ public class BigQueryIOTest implements Serializable {
 
   @Test
   public void testBuildTableBasedSource() {
-    BigQueryIO.Read.Bound bound = BigQueryIO.Read.named("ReadMyTable")
-        .from("foo.com:project:somedataset.sometable");
+    BigQueryIO.Read.Bound bound = BigQueryIO.Read.from("foo.com:project:somedataset.sometable");
     checkReadTableObject(bound, "foo.com:project", "somedataset", "sometable");
   }
 
   @Test
   public void testBuildQueryBasedSource() {
-    BigQueryIO.Read.Bound bound = BigQueryIO.Read.named("ReadMyQuery")
-        .fromQuery("foo_query");
+    BigQueryIO.Read.Bound bound = BigQueryIO.Read.fromQuery("foo_query");
     checkReadQueryObject(bound, "foo_query");
   }
 
@@ -409,8 +406,8 @@ public class BigQueryIOTest implements Serializable {
   public void testBuildTableBasedSourceWithoutValidation() {
     // This test just checks that using withoutValidation will not trigger object
     // construction errors.
-    BigQueryIO.Read.Bound bound = BigQueryIO.Read.named("ReadMyTable")
-        .from("foo.com:project:somedataset.sometable").withoutValidation();
+    BigQueryIO.Read.Bound bound =
+        BigQueryIO.Read.from("foo.com:project:somedataset.sometable").withoutValidation();
     checkReadTableObjectWithValidate(bound, "foo.com:project", "somedataset", "sometable", false);
   }
 
@@ -418,15 +415,15 @@ public class BigQueryIOTest implements Serializable {
   public void testBuildQueryBasedSourceWithoutValidation() {
     // This test just checks that using withoutValidation will not trigger object
     // construction errors.
-    BigQueryIO.Read.Bound bound = BigQueryIO.Read.named("ReadMyTable")
-        .fromQuery("some_query").withoutValidation();
+    BigQueryIO.Read.Bound bound =
+        BigQueryIO.Read.fromQuery("some_query").withoutValidation();
     checkReadQueryObjectWithValidate(bound, "some_query", false);
   }
 
   @Test
   public void testBuildTableBasedSourceWithDefaultProject() {
-    BigQueryIO.Read.Bound bound = BigQueryIO.Read.named("ReadMyTable")
-        .from("somedataset.sometable");
+    BigQueryIO.Read.Bound bound =
+        BigQueryIO.Read.from("somedataset.sometable");
     checkReadTableObject(bound, null, "somedataset", "sometable");
   }
 
@@ -436,8 +433,7 @@ public class BigQueryIOTest implements Serializable {
         .setProjectId("foo.com:project")
         .setDatasetId("somedataset")
         .setTableId("sometable");
-    BigQueryIO.Read.Bound bound = BigQueryIO.Read.named("ReadMyTable")
-        .from(table);
+    BigQueryIO.Read.Bound bound = BigQueryIO.Read.from(table);
     checkReadTableObject(bound, "foo.com:project", "somedataset", "sometable");
   }
 
@@ -457,18 +453,7 @@ public class BigQueryIOTest implements Serializable {
     thrown.expectMessage(
         Matchers.either(Matchers.containsString("Unable to confirm BigQuery dataset presence"))
             .or(Matchers.containsString("BigQuery dataset not found for table")));
-    p.apply(BigQueryIO.Read.named("ReadMyTable").from(tableRef));
-  }
-
-  @Test
-  @Category(RunnableOnService.class)
-  public void testBuildSourceWithoutTableOrQuery() {
-    Pipeline p = TestPipeline.create();
-    thrown.expect(IllegalStateException.class);
-    thrown.expectMessage(
-        "Invalid BigQuery read operation, either table reference or query has to be set");
-    p.apply(BigQueryIO.Read.named("ReadMyTable"));
-    p.run();
+    p.apply(BigQueryIO.Read.from(tableRef));
   }
 
   @Test
@@ -490,8 +475,8 @@ public class BigQueryIOTest implements Serializable {
     thrown.expectMessage(
         "Invalid BigQuery read operation. Specifies both a query and a table, only one of these"
         + " should be provided");
-    p.apply(
-        BigQueryIO.Read.named("ReadMyTable")
+    p.apply("ReadMyTable",
+        BigQueryIO.Read
             .from("foo.com:project:somedataset.sometable")
             .fromQuery("query"));
     p.run();
@@ -505,8 +490,8 @@ public class BigQueryIOTest implements Serializable {
     thrown.expectMessage(
         "Invalid BigQuery read operation. Specifies a"
               + " table with a result flattening preference, which is not configurable");
-    p.apply(
-        BigQueryIO.Read.named("ReadMyTable")
+    p.apply("ReadMyTable",
+        BigQueryIO.Read
             .from("foo.com:project:somedataset.sometable")
             .withoutResultFlattening());
     p.run();
@@ -521,7 +506,7 @@ public class BigQueryIOTest implements Serializable {
         "Invalid BigQuery read operation. Specifies a"
               + " table with a result flattening preference, which is not configurable");
     p.apply(
-        BigQueryIO.Read.named("ReadMyTable")
+        BigQueryIO.Read
             .from("foo.com:project:somedataset.sometable")
             .withoutValidation()
             .withoutResultFlattening());
@@ -644,8 +629,7 @@ public class BigQueryIOTest implements Serializable {
 
   @Test
   public void testBuildSink() {
-    BigQueryIO.Write.Bound bound = BigQueryIO.Write.named("WriteMyTable")
-        .to("foo.com:project:somedataset.sometable");
+    BigQueryIO.Write.Bound bound = BigQueryIO.Write.to("foo.com:project:somedataset.sometable");
     checkWriteObject(
         bound, "foo.com:project", "somedataset", "sometable",
         null, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_EMPTY);
@@ -655,8 +639,8 @@ public class BigQueryIOTest implements Serializable {
   public void testBuildSinkwithoutValidation() {
     // This test just checks that using withoutValidation will not trigger object
     // construction errors.
-    BigQueryIO.Write.Bound bound = BigQueryIO.Write.named("WriteMyTable")
-        .to("foo.com:project:somedataset.sometable").withoutValidation();
+    BigQueryIO.Write.Bound bound =
+        BigQueryIO.Write.to("foo.com:project:somedataset.sometable").withoutValidation();
     checkWriteObjectWithValidate(
         bound, "foo.com:project", "somedataset", "sometable",
         null, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_EMPTY, false);
@@ -664,8 +648,7 @@ public class BigQueryIOTest implements Serializable {
 
   @Test
   public void testBuildSinkDefaultProject() {
-    BigQueryIO.Write.Bound bound = BigQueryIO.Write.named("WriteMyTable")
-        .to("somedataset.sometable");
+    BigQueryIO.Write.Bound bound = BigQueryIO.Write.to("somedataset.sometable");
     checkWriteObject(
         bound, null, "somedataset", "sometable",
         null, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_EMPTY);
@@ -677,8 +660,7 @@ public class BigQueryIOTest implements Serializable {
         .setProjectId("foo.com:project")
         .setDatasetId("somedataset")
         .setTableId("sometable");
-    BigQueryIO.Write.Bound bound = BigQueryIO.Write.named("WriteMyTable")
-        .to(table);
+    BigQueryIO.Write.Bound bound = BigQueryIO.Write.to(table);
     checkWriteObject(
         bound, "foo.com:project", "somedataset", "sometable",
         null, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_EMPTY);
@@ -691,14 +673,14 @@ public class BigQueryIOTest implements Serializable {
     thrown.expect(IllegalStateException.class);
     thrown.expectMessage("must set the table reference");
     p.apply(Create.<TableRow>of().withCoder(TableRowJsonCoder.of()))
-        .apply(BigQueryIO.Write.named("WriteMyTable"));
+        .apply(BigQueryIO.Write.withoutValidation());
   }
 
   @Test
   public void testBuildSinkWithSchema() {
     TableSchema schema = new TableSchema();
-    BigQueryIO.Write.Bound bound = BigQueryIO.Write.named("WriteMyTable")
-        .to("foo.com:project:somedataset.sometable").withSchema(schema);
+    BigQueryIO.Write.Bound bound =
+        BigQueryIO.Write.to("foo.com:project:somedataset.sometable").withSchema(schema);
     checkWriteObject(
         bound, "foo.com:project", "somedataset", "sometable",
         schema, CreateDisposition.CREATE_IF_NEEDED, WriteDisposition.WRITE_EMPTY);
@@ -706,7 +688,7 @@ public class BigQueryIOTest implements Serializable {
 
   @Test
   public void testBuildSinkWithCreateDispositionNever() {
-    BigQueryIO.Write.Bound bound = BigQueryIO.Write.named("WriteMyTable")
+    BigQueryIO.Write.Bound bound = BigQueryIO.Write
         .to("foo.com:project:somedataset.sometable")
         .withCreateDisposition(CreateDisposition.CREATE_NEVER);
     checkWriteObject(
@@ -716,7 +698,7 @@ public class BigQueryIOTest implements Serializable {
 
   @Test
   public void testBuildSinkWithCreateDispositionIfNeeded() {
-    BigQueryIO.Write.Bound bound = BigQueryIO.Write.named("WriteMyTable")
+    BigQueryIO.Write.Bound bound = BigQueryIO.Write
         .to("foo.com:project:somedataset.sometable")
         .withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED);
     checkWriteObject(
@@ -726,7 +708,7 @@ public class BigQueryIOTest implements Serializable {
 
   @Test
   public void testBuildSinkWithWriteDispositionTruncate() {
-    BigQueryIO.Write.Bound bound = BigQueryIO.Write.named("WriteMyTable")
+    BigQueryIO.Write.Bound bound = BigQueryIO.Write
         .to("foo.com:project:somedataset.sometable")
         .withWriteDisposition(WriteDisposition.WRITE_TRUNCATE);
     checkWriteObject(
@@ -736,7 +718,7 @@ public class BigQueryIOTest implements Serializable {
 
   @Test
   public void testBuildSinkWithWriteDispositionAppend() {
-    BigQueryIO.Write.Bound bound = BigQueryIO.Write.named("WriteMyTable")
+    BigQueryIO.Write.Bound bound = BigQueryIO.Write
         .to("foo.com:project:somedataset.sometable")
         .withWriteDisposition(WriteDisposition.WRITE_APPEND);
     checkWriteObject(
@@ -746,7 +728,7 @@ public class BigQueryIOTest implements Serializable {
 
   @Test
   public void testBuildSinkWithWriteDispositionEmpty() {
-    BigQueryIO.Write.Bound bound = BigQueryIO.Write.named("WriteMyTable")
+    BigQueryIO.Write.Bound bound = BigQueryIO.Write
         .to("foo.com:project:somedataset.sometable")
         .withWriteDisposition(WriteDisposition.WRITE_EMPTY);
     checkWriteObject(
@@ -794,7 +776,7 @@ public class BigQueryIOTest implements Serializable {
         Matchers.either(Matchers.containsString("Unable to confirm BigQuery dataset presence"))
             .or(Matchers.containsString("BigQuery dataset not found for table")));
     p.apply(Create.<TableRow>of().withCoder(TableRowJsonCoder.of()))
-     .apply(BigQueryIO.Write.named("WriteMyTable")
+     .apply(BigQueryIO.Write
          .to(tableRef)
          .withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED)
          .withSchema(new TableSchema()));
@@ -878,8 +860,6 @@ public class BigQueryIOTest implements Serializable {
   public void testBigQueryIOGetName() {
     assertEquals("BigQueryIO.Read", BigQueryIO.Read.from("somedataset.sometable").getName());
     assertEquals("BigQueryIO.Write", BigQueryIO.Write.to("somedataset.sometable").getName());
-    assertEquals("ReadMyTable", BigQueryIO.Read.named("ReadMyTable").getName());
-    assertEquals("WriteMyTable", BigQueryIO.Write.named("WriteMyTable").getName());
   }
 
   @Test
@@ -915,7 +895,7 @@ public class BigQueryIOTest implements Serializable {
     thrown.expectMessage("must set the table reference of a BigQueryIO.Write transform");
     TestPipeline.create()
         .apply(Create.<TableRow>of())
-        .apply(BigQueryIO.Write.named("name"));
+        .apply("name", BigQueryIO.Write.withoutValidation());
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fd0bd51/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSourceTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSourceTest.java
index b0c577d..c9f4079 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSourceTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSourceTest.java
@@ -21,6 +21,7 @@ import static org.apache.beam.sdk.testing.SourceTestUtils.assertSplitAtFractionE
 import static org.apache.beam.sdk.testing.SourceTestUtils.assertSplitAtFractionFails;
 import static org.apache.beam.sdk.testing.SourceTestUtils.assertSplitAtFractionSucceedsAndConsistent;
 import static org.apache.beam.sdk.testing.SourceTestUtils.readFromSource;
+
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
@@ -717,7 +718,7 @@ public class FileBasedSourceTest {
     File file = createFileWithData(fileName, data);
 
     TestFileBasedSource source = new TestFileBasedSource(file.getPath(), 64, null);
-    PCollection<String> output = p.apply(Read.from(source).named("ReadFileData"));
+    PCollection<String> output = p.apply("ReadFileData", Read.from(source));
 
     PAssert.that(output).containsInAnyOrder(data);
     p.run();
@@ -743,7 +744,7 @@ public class FileBasedSourceTest {
     TestFileBasedSource source =
         new TestFileBasedSource(new File(file1.getParent(), "file*").getPath(), 64, null);
 
-    PCollection<String> output = p.apply(Read.from(source).named("ReadFileData"));
+    PCollection<String> output = p.apply("ReadFileData", Read.from(source));
 
     List<String> expectedResults = new ArrayList<String>();
     expectedResults.addAll(data1);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fd0bd51/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubIOTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubIOTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubIOTest.java
index eaf452d..efa1cd2 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubIOTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/PubsubIOTest.java
@@ -45,10 +45,6 @@ public class PubsubIOTest {
         PubsubIO.Read.topic("projects/myproject/topics/mytopic").getName());
     assertEquals("PubsubIO.Write",
         PubsubIO.Write.topic("projects/myproject/topics/mytopic").getName());
-    assertEquals("ReadMyTopic",
-        PubsubIO.Read.named("ReadMyTopic").topic("projects/myproject/topics/mytopic").getName());
-    assertEquals("WriteMyTopic",
-        PubsubIO.Write.named("WriteMyTopic").topic("projects/myproject/topics/mytopic").getName());
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fd0bd51/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java
index c3a5084..df598c8 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOTest.java
@@ -22,6 +22,7 @@ import static org.apache.beam.sdk.TestUtils.LINES_ARRAY;
 import static org.apache.beam.sdk.TestUtils.NO_INTS_ARRAY;
 import static org.apache.beam.sdk.TestUtils.NO_LINES_ARRAY;
 import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem;
+
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -44,14 +45,12 @@ import org.apache.beam.sdk.testing.PAssert;
 import org.apache.beam.sdk.testing.SourceTestUtils;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.display.DisplayData;
 import org.apache.beam.sdk.util.CoderUtils;
 import org.apache.beam.sdk.util.GcsUtil;
 import org.apache.beam.sdk.util.IOChannelUtils;
 import org.apache.beam.sdk.util.gcsfs.GcsPath;
 import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PDone;
 
 import com.google.common.collect.ImmutableList;
 
@@ -167,16 +166,9 @@ public class TextIOTest {
     }
 
     {
-      PCollection<String> output2 =
-          p.apply(TextIO.Read.named("MyRead").from(file));
+      PCollection<String> output2 = p.apply("MyRead", TextIO.Read.from(file));
       assertEquals("MyRead/Read.out", output2.getName());
     }
-
-    {
-      PCollection<String> output3 =
-          p.apply(TextIO.Read.from(file).named("HerRead"));
-      assertEquals("HerRead/Read.out", output3.getName());
-    }
   }
 
   @Test
@@ -299,27 +291,6 @@ public class TextIOTest {
   }
 
   @Test
-  public void testWriteNamed() {
-    {
-      PTransform<PCollection<String>, PDone> transform1 =
-        TextIO.Write.to("/tmp/file.txt");
-      assertEquals("TextIO.Write", transform1.getName());
-    }
-
-    {
-      PTransform<PCollection<String>, PDone> transform2 =
-          TextIO.Write.named("MyWrite").to("/tmp/file.txt");
-      assertEquals("MyWrite", transform2.getName());
-    }
-
-    {
-      PTransform<PCollection<String>, PDone> transform3 =
-          TextIO.Write.to("/tmp/file.txt").named("HerWrite");
-      assertEquals("HerWrite", transform3.getName());
-    }
-  }
-
-  @Test
   @Category(NeedsRunner.class)
   public void testShardedWrite() throws Exception {
     runTestWrite(LINES_ARRAY, StringUtf8Coder.of(), 5);
@@ -620,12 +591,8 @@ public class TextIOTest {
   public void testTextIOGetName() {
     assertEquals("TextIO.Read", TextIO.Read.from("somefile").getName());
     assertEquals("TextIO.Write", TextIO.Write.to("somefile").getName());
-    assertEquals("ReadMyFile", TextIO.Read.named("ReadMyFile").from("somefile").getName());
-    assertEquals("WriteMyFile", TextIO.Write.named("WriteMyFile").to("somefile").getName());
 
     assertEquals("TextIO.Read", TextIO.Read.from("somefile").toString());
-    assertEquals(
-        "ReadMyFile [TextIO.Read]", TextIO.Read.named("ReadMyFile").from("somefile").toString());
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fd0bd51/sdks/java/core/src/test/java/org/apache/beam/sdk/io/XmlSourceTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/XmlSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/XmlSourceTest.java
index eb65468..37e3881 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/XmlSourceTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/XmlSourceTest.java
@@ -21,6 +21,7 @@ import static org.apache.beam.sdk.testing.SourceTestUtils.assertSplitAtFractionE
 import static org.apache.beam.sdk.testing.SourceTestUtils.assertSplitAtFractionFails;
 import static org.apache.beam.sdk.testing.SourceTestUtils.assertSplitAtFractionSucceedsAndConsistent;
 import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem;
+
 import static org.hamcrest.Matchers.both;
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.hamcrest.Matchers.containsString;
@@ -582,7 +583,7 @@ public class XmlSourceTest {
             .withRecordClass(Train.class)
             .withMinBundleSize(1024);
 
-    PCollection<Train> output = p.apply(Read.from(source).named("ReadFileData"));
+    PCollection<Train> output = p.apply("ReadFileData", Read.from(source));
 
     List<Train> expectedResults =
         ImmutableList.of(new Train("Thomas", 1, "blue", null), new Train("Henry", 3, "green", null),
@@ -672,7 +673,7 @@ public class XmlSourceTest {
             .withRecordElement("train")
             .withRecordClass(Train.class)
             .withMinBundleSize(1024);
-    PCollection<Train> output = p.apply(Read.from(source).named("ReadFileData"));
+    PCollection<Train> output = p.apply("ReadFileData", Read.from(source));
 
     PAssert.that(output).containsInAnyOrder(trains);
     p.run();
@@ -814,13 +815,13 @@ public class XmlSourceTest {
 
     Pipeline p = TestPipeline.create();
 
-    XmlSource<Train> source = XmlSource.<Train>from(file.getParent() + "/"
-        + "temp*.xml")
-                                  .withRootElement("trains")
-                                  .withRecordElement("train")
-                                  .withRecordClass(Train.class)
-                                  .withMinBundleSize(1024);
-    PCollection<Train> output = p.apply(Read.from(source).named("ReadFileData"));
+    XmlSource<Train> source =
+        XmlSource.<Train>from(file.getParent() + "/" + "temp*.xml")
+            .withRootElement("trains")
+            .withRecordElement("train")
+            .withRecordClass(Train.class)
+            .withMinBundleSize(1024);
+    PCollection<Train> output = p.apply("ReadFileData", Read.from(source));
 
     List<Train> expectedResults = new ArrayList<>();
     expectedResults.addAll(trains1);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fd0bd51/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformTreeTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformTreeTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformTreeTest.java
index 0c992c4..08c3996 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformTreeTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformTreeTest.java
@@ -120,9 +120,9 @@ public class TransformTreeTest {
 
     Pipeline p = TestPipeline.create();
 
-    p.apply(TextIO.Read.named("ReadMyFile").from(inputFile.getPath()))
+    p.apply("ReadMyFile", TextIO.Read.from(inputFile.getPath()))
         .apply(Sample.<String>any(10))
-        .apply(TextIO.Write.named("WriteMyFile").to(outputFile.getPath()));
+        .apply("WriteMyFile", TextIO.Write.to(outputFile.getPath()));
 
     final EnumSet<TransformsSeen> visited =
         EnumSet.noneOf(TransformsSeen.class);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fd0bd51/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowTest.java
index c858f32..76bc038 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowTest.java
@@ -119,11 +119,11 @@ public class WindowTest implements Serializable {
     FixedWindows fixed25 = FixedWindows.of(Duration.standardMinutes(25));
     WindowingStrategy<?, ?> strategy = TestPipeline.create()
         .apply(Create.of("hello", "world").withCoder(StringUtf8Coder.of()))
-        .apply(Window.named("WindowInto10").<String>into(fixed10)
+        .apply("WindowInto10", Window.<String>into(fixed10)
             .withAllowedLateness(Duration.standardDays(1))
             .triggering(Repeatedly.forever(AfterPane.elementCountAtLeast(5)))
             .accumulatingFiredPanes())
-        .apply(Window.named("WindowInto25").<String>into(fixed25))
+        .apply("WindowInto25", Window.<String>into(fixed25))
         .getWindowingStrategy();
 
     assertEquals(Duration.standardDays(1), strategy.getAllowedLateness());
@@ -272,7 +272,7 @@ public class WindowTest implements Serializable {
 
   @Test
   public void testDisplayDataExcludesUnspecifiedProperties() {
-    Window.Bound<?> onlyHasAccumulationMode = Window.named("foobar").discardingFiredPanes();
+    Window.Bound<?> onlyHasAccumulationMode = Window.discardingFiredPanes();
     assertThat(DisplayData.from(onlyHasAccumulationMode), not(hasDisplayItem(hasKey(isOneOf(
         "windowFn",
         "trigger",

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fd0bd51/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowingTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowingTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowingTest.java
index 21f58df..c1e092a 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowingTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowingTest.java
@@ -221,7 +221,7 @@ public class WindowingTest implements Serializable {
 
     Pipeline p = TestPipeline.create();
     PCollection<String> output = p.begin()
-        .apply(TextIO.Read.named("ReadLines").from(filename))
+        .apply("ReadLines", TextIO.Read.from(filename))
         .apply(ParDo.of(new ExtractWordsWithTimestampsFn()))
         .apply(new WindowedCount(FixedWindows.of(Duration.millis(10))));
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fd0bd51/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/DebuggingWordCount.java
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/DebuggingWordCount.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/DebuggingWordCount.java
index 3306cb4..c0e5b17 100644
--- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/DebuggingWordCount.java
+++ b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/DebuggingWordCount.java
@@ -174,7 +174,7 @@ public class DebuggingWordCount {
     Pipeline p = Pipeline.create(options);
 
     PCollection<KV<String, Long>> filteredWords =
-        p.apply(TextIO.Read.named("ReadLines").from(options.getInputFile()))
+        p.apply("ReadLines", TextIO.Read.from(options.getInputFile()))
          .apply(new WordCount.CountWords())
          .apply(ParDo.of(new FilterTextFn(options.getFilterPattern())));
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0fd0bd51/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/WordCount.java
----------------------------------------------------------------------
diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/WordCount.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/WordCount.java
index 07ed6d0..803e800 100644
--- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/WordCount.java
+++ b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/WordCount.java
@@ -195,10 +195,10 @@ public class WordCount {
 
     // Concepts #2 and #3: Our pipeline applies the composite CountWords transform, and passes the
     // static FormatAsTextFn() to the ParDo transform.
-    p.apply(TextIO.Read.named("ReadLines").from(options.getInputFile()))
+    p.apply("ReadLines", TextIO.Read.from(options.getInputFile()))
      .apply(new CountWords())
      .apply(ParDo.of(new FormatAsTextFn()))
-     .apply(TextIO.Write.named("WriteCounts").to(options.getOutput()));
+     .apply("WriteCounts", TextIO.Write.to(options.getOutput()));
 
     p.run();
   }


[44/50] [abbrv] incubator-beam git commit: Add initial microbenchmarks directory

Posted by lc...@apache.org.
Add initial microbenchmarks directory


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

Branch: refs/heads/runners-spark2
Commit: b36aeb92003b1c613c56633b2b5a0800260d92d3
Parents: a17a8b2
Author: bchambers <bc...@google.com>
Authored: Tue Jun 28 15:55:16 2016 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Jul 6 10:18:53 2016 -0700

----------------------------------------------------------------------
 .../beam/sdk/transforms/DoFnReflector.java      |  12 +-
 sdks/java/microbenchmarks/README.md             |  42 ++++
 sdks/java/microbenchmarks/pom.xml               | 110 +++++++++
 .../coders/AvroCoderBenchmark.java              | 121 ++++++++++
 .../coders/ByteArrayCoderBenchmark.java         |  66 +++++
 .../coders/CoderBenchmarking.java               |  42 ++++
 .../coders/StringUtf8CoderBenchmark.java        |  72 ++++++
 .../transforms/DoFnReflectorBenchmark.java      | 239 +++++++++++++++++++
 sdks/java/pom.xml                               |   1 +
 9 files changed, 699 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b36aeb92/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnReflector.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnReflector.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnReflector.java
index 452ee8e..e711d04 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnReflector.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnReflector.java
@@ -123,7 +123,7 @@ public abstract class DoFnReflector {
    * @param c the {@link org.apache.beam.sdk.transforms.DoFnWithContext.ProcessContext}
    *     to pass to {@link ProcessElement}.
    */
-  abstract <InputT, OutputT> void invokeProcessElement(
+  public abstract <InputT, OutputT> void invokeProcessElement(
       DoFnWithContext<InputT, OutputT> fn,
       DoFnWithContext<InputT, OutputT>.ProcessContext c,
       ExtraContextFactory<InputT, OutputT> extra);
@@ -135,7 +135,7 @@ public abstract class DoFnReflector {
    * @param c the {@link org.apache.beam.sdk.transforms.DoFnWithContext.Context}
    *     to pass to {@link StartBundle}.
    */
-  <InputT, OutputT> void invokeStartBundle(
+  public <InputT, OutputT> void invokeStartBundle(
      DoFnWithContext<InputT, OutputT> fn,
      DoFnWithContext<InputT, OutputT>.Context c,
      ExtraContextFactory<InputT, OutputT> extra) {
@@ -149,7 +149,7 @@ public abstract class DoFnReflector {
    * @param c the {@link org.apache.beam.sdk.transforms.DoFnWithContext.Context}
    *     to pass to {@link FinishBundle}.
    */
-  abstract <InputT, OutputT> void invokeFinishBundle(
+  public abstract <InputT, OutputT> void invokeFinishBundle(
       DoFnWithContext<InputT, OutputT> fn,
       DoFnWithContext<InputT, OutputT>.Context c,
       ExtraContextFactory<InputT, OutputT> extra);
@@ -430,7 +430,7 @@ public abstract class DoFnReflector {
     }
 
     @Override
-    <InputT, OutputT> void invokeProcessElement(
+    public <InputT, OutputT> void invokeProcessElement(
         DoFnWithContext<InputT, OutputT> fn,
         DoFnWithContext<InputT, OutputT>.ProcessContext c,
         ExtraContextFactory<InputT, OutputT> extra) {
@@ -438,7 +438,7 @@ public abstract class DoFnReflector {
     }
 
     @Override
-    <InputT, OutputT> void invokeStartBundle(
+    public <InputT, OutputT> void invokeStartBundle(
         DoFnWithContext<InputT, OutputT> fn,
         DoFnWithContext<InputT, OutputT>.Context c,
         ExtraContextFactory<InputT, OutputT> extra) {
@@ -449,7 +449,7 @@ public abstract class DoFnReflector {
     }
 
     @Override
-    <InputT, OutputT> void invokeFinishBundle(
+    public <InputT, OutputT> void invokeFinishBundle(
         DoFnWithContext<InputT, OutputT> fn,
         DoFnWithContext<InputT, OutputT>.Context c,
         ExtraContextFactory<InputT, OutputT> extra) {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b36aeb92/sdks/java/microbenchmarks/README.md
----------------------------------------------------------------------
diff --git a/sdks/java/microbenchmarks/README.md b/sdks/java/microbenchmarks/README.md
new file mode 100644
index 0000000..627e669
--- /dev/null
+++ b/sdks/java/microbenchmarks/README.md
@@ -0,0 +1,42 @@
+<!--
+    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.
+-->
+
+# Microbenchmarks for parts of the Beam SDK
+
+To run benchmarks:
+
+ 1. Run `mvn install` in the top directory to install the SDK.
+
+ 2. Build the benchmark package:
+
+        cd microbenchmarks
+        mvn package
+
+ 3. run benchmark harness:
+
+        java -jar target/microbenchmarks.jar
+
+ 4. (alternate to step 3)
+    to run just a subset of benchmarks, pass a regular expression that
+    matches the benchmarks you want to run (this can match against the class
+    name, or the method name).  E.g., to run any benchmarks with
+    "DoFnReflector" in the name:
+
+        java -jar target/microbenchmarks.jar ".*DoFnReflector.*"
+

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b36aeb92/sdks/java/microbenchmarks/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/microbenchmarks/pom.xml b/sdks/java/microbenchmarks/pom.xml
new file mode 100644
index 0000000..96a76ec
--- /dev/null
+++ b/sdks/java/microbenchmarks/pom.xml
@@ -0,0 +1,110 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+    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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+
+  <modelVersion>4.0.0</modelVersion>
+
+  <parent>
+    <groupId>org.apache.beam</groupId>
+    <artifactId>beam-sdks-java-parent</artifactId>
+    <version>0.2.0-incubating-SNAPSHOT</version>
+    <relativePath>../pom.xml</relativePath>
+  </parent>
+
+  <artifactId>beam-sdks-java-microbenchmarks</artifactId>
+  <name>Apache Beam :: SDKs :: Java :: Microbenchmarks</name>
+  <description>Microbenchmarks for components in the Beam Java SDK.</description>
+  <packaging>jar</packaging>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-compiler-plugin</artifactId>
+      </plugin>
+
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-source-plugin</artifactId>
+      </plugin>
+
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+      </plugin>
+
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-checkstyle-plugin</artifactId>
+      </plugin>
+
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-shade-plugin</artifactId>
+        <executions>
+          <execution>
+            <phase>package</phase>
+            <goals>
+              <goal>shade</goal>
+            </goals>
+            <configuration>
+              <finalName>microbenchmarks</finalName>
+              <transformers>
+                <transformer implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
+                  <mainClass>org.openjdk.jmh.Main</mainClass>
+                </transformer>
+              </transformers>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-sdks-java-core</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>joda-time</groupId>
+      <artifactId>joda-time</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-jdk14</artifactId>
+      <!-- When loaded at runtime this will wire up slf4j to the JUL backend -->
+      <scope>runtime</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.openjdk.jmh</groupId>
+      <artifactId>jmh-core</artifactId>
+      <version>1.0.1</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.openjdk.jmh</groupId>
+      <artifactId>jmh-generator-annprocess</artifactId>
+      <version>1.6.1</version>
+      <scope>provided</scope>
+    </dependency>
+  </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b36aeb92/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/coders/AvroCoderBenchmark.java
----------------------------------------------------------------------
diff --git a/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/coders/AvroCoderBenchmark.java b/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/coders/AvroCoderBenchmark.java
new file mode 100644
index 0000000..39b31ef
--- /dev/null
+++ b/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/coders/AvroCoderBenchmark.java
@@ -0,0 +1,121 @@
+/*
+ * 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.microbenchmarks.coders;
+
+import org.apache.beam.sdk.coders.AvroCoder;
+import org.apache.beam.sdk.coders.DefaultCoder;
+
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.Fork;
+import org.openjdk.jmh.annotations.Param;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.annotations.Warmup;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+/**
+ * Benchmarks for {@link AvroCoder}.
+ */
+@State(Scope.Benchmark)
+@Fork(1)
+@Warmup(iterations = 5)
+public class AvroCoderBenchmark {
+
+  @DefaultCoder(AvroCoder.class)
+  private static class Pojo {
+    public String text;
+    public int count;
+
+    // Empty constructor required for Avro decoding.
+    @SuppressWarnings("unused")
+    public Pojo() {
+    }
+
+    public Pojo(String text, int count) {
+      this.text = text;
+      this.count = count;
+    }
+
+    // auto-generated
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+
+      Pojo pojo = (Pojo) o;
+
+      if (count != pojo.count) {
+        return false;
+      }
+      if (text != null
+          ? !text.equals(pojo.text)
+          : pojo.text != null) {
+        return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      return 0;
+    }
+
+    @Override
+    public String toString() {
+      return "Pojo{"
+          + "text='" + text + '\''
+          + ", count=" + count
+          + '}';
+    }
+  }
+
+  AvroCoder<Pojo> coder = AvroCoder.of(Pojo.class);
+
+  @Param({"true", "false"})
+  boolean isWholeStream;
+
+  Pojo shortPojo;
+  Pojo longPojo;
+
+  @Setup
+  public void setUp() {
+    shortPojo = new Pojo("hello world", 42);
+
+    char[] bytes60k = new char[60 * 1024];
+    Arrays.fill(bytes60k, 'a');
+    longPojo = new Pojo(new String(bytes60k), 42);
+  }
+
+  @Benchmark
+  public Pojo codeShortPojo() throws IOException {
+    return CoderBenchmarking.testCoder(coder, isWholeStream, shortPojo);
+  }
+
+  @Benchmark
+  public Pojo codeLongPojo() throws Exception {
+    return CoderBenchmarking.testCoder(coder, isWholeStream, longPojo);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b36aeb92/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/coders/ByteArrayCoderBenchmark.java
----------------------------------------------------------------------
diff --git a/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/coders/ByteArrayCoderBenchmark.java b/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/coders/ByteArrayCoderBenchmark.java
new file mode 100644
index 0000000..df20a15
--- /dev/null
+++ b/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/coders/ByteArrayCoderBenchmark.java
@@ -0,0 +1,66 @@
+/*
+ * 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.microbenchmarks.coders;
+
+import org.apache.beam.sdk.coders.ByteArrayCoder;
+
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.Fork;
+import org.openjdk.jmh.annotations.Param;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.annotations.Warmup;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+/**
+ * Benchmarks for {@link ByteArrayCoder}.
+ */
+@State(Scope.Benchmark)
+@Fork(1)
+@Warmup(iterations = 5)
+public class ByteArrayCoderBenchmark {
+
+  ByteArrayCoder coder = ByteArrayCoder.of();
+
+  @Param({"true", "false"})
+  boolean isWholeStream;
+
+  byte[] shortArray;
+  byte[] longArray;
+
+  @Setup
+  public void setUp() {
+    shortArray = new byte[10];
+    Arrays.fill(shortArray, (byte) 47);
+    longArray = new byte[60 * 1024];
+    Arrays.fill(longArray, (byte) 47);
+  }
+
+  @Benchmark
+  public byte[] codeShortArray() throws IOException {
+    return CoderBenchmarking.testCoder(coder, isWholeStream, shortArray);
+  }
+
+  @Benchmark
+  public byte[] codeLongArray() throws Exception {
+    return CoderBenchmarking.testCoder(coder, isWholeStream, longArray);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b36aeb92/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/coders/CoderBenchmarking.java
----------------------------------------------------------------------
diff --git a/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/coders/CoderBenchmarking.java b/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/coders/CoderBenchmarking.java
new file mode 100644
index 0000000..8523cb2
--- /dev/null
+++ b/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/coders/CoderBenchmarking.java
@@ -0,0 +1,42 @@
+/*
+ * 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.microbenchmarks.coders;
+
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.util.CoderUtils;
+
+import java.io.IOException;
+
+/**
+ * Utilities for writing coder benchmarks.
+ */
+class CoderBenchmarking {
+
+  /**
+   * Encodes and decodes the given value using the specified Coder.
+   *
+   * @throws IOException if there are errors during encoding or decoding
+   */
+  public static <T> T testCoder(
+      Coder<T> coder, boolean isWholeStream, T value) throws IOException {
+    Coder.Context context =
+        isWholeStream ? Coder.Context.OUTER : Coder.Context.NESTED;
+    byte[] encoded = CoderUtils.encodeToByteArray(coder, value, context);
+    return CoderUtils.decodeFromByteArray(coder, encoded, context);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b36aeb92/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/coders/StringUtf8CoderBenchmark.java
----------------------------------------------------------------------
diff --git a/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/coders/StringUtf8CoderBenchmark.java b/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/coders/StringUtf8CoderBenchmark.java
new file mode 100644
index 0000000..c0bcb45
--- /dev/null
+++ b/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/coders/StringUtf8CoderBenchmark.java
@@ -0,0 +1,72 @@
+/*
+ * 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.microbenchmarks.coders;
+
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.Fork;
+import org.openjdk.jmh.annotations.Param;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.annotations.Warmup;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+/**
+ * Benchmarks for {@link StringUtf8Coder}.
+ */
+@State(Scope.Benchmark)
+@Fork(1)
+@Warmup(iterations = 5)
+public class StringUtf8CoderBenchmark {
+
+  StringUtf8Coder coder = StringUtf8Coder.of();
+
+  @Param({"true", "false"})
+  boolean isWholeStream;
+
+  String shortString;
+  String longString;
+
+  @Setup
+  public void setUp() {
+    shortString = "hello world";
+
+    char[] bytes60k = new char[60 * 1024];
+    Arrays.fill(bytes60k, 'a');
+    longString = new String(bytes60k);
+  }
+
+  @Benchmark
+  public String codeEmptyString() throws IOException {
+    return CoderBenchmarking.testCoder(coder, isWholeStream, "");
+  }
+
+  @Benchmark
+  public String codeShortString() throws IOException {
+    return CoderBenchmarking.testCoder(coder, isWholeStream, shortString);
+  }
+
+  @Benchmark
+  public String codeLongString() throws IOException {
+    return CoderBenchmarking.testCoder(coder, isWholeStream, longString);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b36aeb92/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/transforms/DoFnReflectorBenchmark.java
----------------------------------------------------------------------
diff --git a/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/transforms/DoFnReflectorBenchmark.java b/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/transforms/DoFnReflectorBenchmark.java
new file mode 100644
index 0000000..1b8ec2a
--- /dev/null
+++ b/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/transforms/DoFnReflectorBenchmark.java
@@ -0,0 +1,239 @@
+/*
+ * 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.microbenchmarks.transforms;
+
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.DoFnReflector;
+import org.apache.beam.sdk.transforms.DoFnWithContext;
+import org.apache.beam.sdk.transforms.DoFnWithContext.ExtraContextFactory;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.WindowingInternals;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.TupleTag;
+
+import org.joda.time.Instant;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.Fork;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.annotations.Warmup;
+
+/**
+ * Benchmarks for {@link DoFn} and {@link DoFnWithContext} invocations, specifically
+ * for measuring the overhead of {@link DoFnReflector}.
+ */
+@State(Scope.Benchmark)
+@Fork(1)
+@Warmup(iterations = 5)
+public class DoFnReflectorBenchmark {
+
+  private static final String ELEMENT = "some string to use for testing";
+
+  private DoFn<String, String> doFn = new UpperCaseDoFn();
+  private DoFnWithContext<String, String> doFnWithContext = new UpperCaseDoFnWithContext();
+
+  private StubDoFnProcessContext stubDoFnContext = new StubDoFnProcessContext(doFn, ELEMENT);
+  private StubDoFnWithContextProcessContext stubDoFnWithContextContext =
+      new StubDoFnWithContextProcessContext(doFnWithContext, ELEMENT);
+  private ExtraContextFactory<String, String> extraContextFactory =
+      new ExtraContextFactory<String, String>() {
+
+    @Override
+    public BoundedWindow window() {
+      return null;
+    }
+
+    @Override
+    public WindowingInternals<String, String> windowingInternals() {
+      return null;
+    }
+  };
+
+  private DoFnReflector doFnReflector;
+  private DoFn<String, String> adaptedDoFnWithContext;
+
+  @Setup
+  public void setUp() {
+    doFnReflector = DoFnReflector.of(doFnWithContext.getClass());
+    adaptedDoFnWithContext = doFnReflector.toDoFn(doFnWithContext);
+  }
+
+  @Benchmark
+  public String invokeDoFn() throws Exception {
+    doFn.processElement(stubDoFnContext);
+    return stubDoFnContext.output;
+  }
+
+  @Benchmark
+  public String invokeDoFnWithContextViaAdaptor() throws Exception {
+    adaptedDoFnWithContext.processElement(stubDoFnContext);
+    return stubDoFnContext.output;
+  }
+
+  @Benchmark
+  public String invokeDoFnWithContext() throws Exception {
+    doFnReflector.invokeProcessElement(
+        doFnWithContext, stubDoFnWithContextContext, extraContextFactory);
+    return stubDoFnWithContextContext.output;
+  }
+
+  private static class UpperCaseDoFn extends DoFn<String, String> {
+
+    @Override
+    public void processElement(ProcessContext c) throws Exception {
+      c.output(c.element().toUpperCase());
+    }
+  }
+
+  private static class UpperCaseDoFnWithContext extends DoFnWithContext<String, String> {
+
+    @ProcessElement
+    public void processElement(ProcessContext c) throws Exception {
+      c.output(c.element().toUpperCase());
+    }
+  }
+
+  private static class StubDoFnProcessContext extends DoFn<String, String>.ProcessContext {
+
+    private final String element;
+    private String output;
+
+    public StubDoFnProcessContext(DoFn<String, String> fn, String element) {
+      fn.super();
+      this.element = element;
+    }
+
+    @Override
+    public String element() {
+      return element;
+    }
+
+    @Override
+    public <T> T sideInput(PCollectionView<T> view) {
+      return null;
+    }
+
+    @Override
+    public Instant timestamp() {
+      return null;
+    }
+
+    @Override
+    public BoundedWindow window() {
+      return null;
+    }
+
+    @Override
+    public PaneInfo pane() {
+      return null;
+    }
+
+    @Override
+    public WindowingInternals<String, String> windowingInternals() {
+      return null;
+    }
+
+    @Override
+    public PipelineOptions getPipelineOptions() {
+      return null;
+    }
+
+    @Override
+    public void output(String output) {
+      this.output = output;
+    }
+
+    @Override
+    public void outputWithTimestamp(String output, Instant timestamp) {
+      output(output);
+    }
+
+    @Override
+    public <T> void sideOutput(TupleTag<T> tag, T output) {
+    }
+
+    @Override
+    public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
+    }
+
+    @Override
+    protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT>
+        createAggregatorInternal(String name, CombineFn<AggInputT, ?, AggOutputT> combiner) {
+      return null;
+    }
+  }
+
+  private static class StubDoFnWithContextProcessContext
+      extends DoFnWithContext<String, String>.ProcessContext {
+    private final String element;
+    private  String output;
+
+    public StubDoFnWithContextProcessContext(DoFnWithContext<String, String> fn, String element) {
+      fn.super();
+      this.element = element;
+    }
+
+    @Override
+    public String element() {
+      return element;
+    }
+
+    @Override
+    public <T> T sideInput(PCollectionView<T> view) {
+      return null;
+    }
+
+    @Override
+    public Instant timestamp() {
+      return null;
+    }
+
+    @Override
+    public PaneInfo pane() {
+      return null;
+    }
+
+    @Override
+    public PipelineOptions getPipelineOptions() {
+      return null;
+    }
+
+    @Override
+    public void output(String output) {
+      this.output = output;
+    }
+
+    @Override
+    public void outputWithTimestamp(String output, Instant timestamp) {
+      output(output);
+    }
+
+    @Override
+    public <T> void sideOutput(TupleTag<T> tag, T output) {
+    }
+
+    @Override
+    public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b36aeb92/sdks/java/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/pom.xml b/sdks/java/pom.xml
index 55aea6a..0350804 100644
--- a/sdks/java/pom.xml
+++ b/sdks/java/pom.xml
@@ -41,6 +41,7 @@
          a released artifact exists, we need to modify the build order.
     <module>maven-archetypes</module> -->
     <module>extensions</module>
+    <module>microbenchmarks</module>
   </modules>
 
   <profiles>


[39/50] [abbrv] incubator-beam git commit: [BEAM-392] Update flink dependency to the most recent stable version

Posted by lc...@apache.org.
[BEAM-392] Update flink dependency to the most recent stable version


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

Branch: refs/heads/runners-spark2
Commit: 721763a3f8b79a5a583b6e881af6c02301b3a1a8
Parents: d71f385
Author: Isma�l Mej�a <ie...@gmail.com>
Authored: Wed Jun 29 23:00:02 2016 +0200
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Jul 6 10:18:52 2016 -0700

----------------------------------------------------------------------
 runners/flink/pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/721763a3/runners/flink/pom.xml
----------------------------------------------------------------------
diff --git a/runners/flink/pom.xml b/runners/flink/pom.xml
index bea639e..653fe2c 100644
--- a/runners/flink/pom.xml
+++ b/runners/flink/pom.xml
@@ -39,7 +39,7 @@
   <properties>
     <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
     <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
-    <flink.version>1.0.0</flink.version>
+    <flink.version>1.0.3</flink.version>
   </properties>
 
   <repositories>


[40/50] [abbrv] incubator-beam git commit: [BEAM-142] - BigQueryIO: don't unnecessarily initialize an ExecutorService to validate parameters

Posted by lc...@apache.org.
[BEAM-142] - BigQueryIO: don't unnecessarily initialize an ExecutorService to validate parameters

By default, BigQueryIO initializes a ExecutorContext, however AppEngine
doesn't allow modification of threads to be daemon
threads. Using GcsOptions.ExecutorContext to create the
executorContext.


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

Branch: refs/heads/runners-spark2
Commit: 6924358e701839f2b2deeb37ac4c106ae03ed731
Parents: 8853118
Author: Lucas Amorim <lu...@protonmail.com>
Authored: Tue Jun 28 14:28:17 2016 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Jul 6 10:18:52 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/beam/sdk/io/BigQueryIO.java | 10 +++----
 .../beam/sdk/util/BigQueryTableInserter.java    | 31 +++++++++++++-------
 .../sdk/util/BigQueryTableInserterTest.java     | 17 +++++++----
 .../apache/beam/sdk/util/BigQueryUtilTest.java  | 12 +++++---
 .../util/RetryHttpRequestInitializerTest.java   |  5 +++-
 5 files changed, 48 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6924358e/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BigQueryIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BigQueryIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BigQueryIO.java
index a9d85b8..790e3ff 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BigQueryIO.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BigQueryIO.java
@@ -1594,7 +1594,7 @@ public class BigQueryIO {
           TableReference table) {
         try {
           Bigquery client = Transport.newBigQueryClient(options).build();
-          BigQueryTableInserter inserter = new BigQueryTableInserter(client);
+          BigQueryTableInserter inserter = new BigQueryTableInserter(client, options);
           if (!inserter.isEmpty(table)) {
             throw new IllegalArgumentException(
                 "BigQuery table is not empty: " + BigQueryIO.toTableSpec(table));
@@ -2084,7 +2084,7 @@ public class BigQueryIO {
       for (String tableSpec : tableRows.keySet()) {
         TableReference tableReference = getOrCreateTable(options, tableSpec);
         flushRows(client, tableReference, tableRows.get(tableSpec),
-            uniqueIdsForTableRows.get(tableSpec));
+            uniqueIdsForTableRows.get(tableSpec), options);
       }
       tableRows.clear();
       uniqueIdsForTableRows.clear();
@@ -2109,7 +2109,7 @@ public class BigQueryIO {
           if (!createdTables.contains(tableSpec)) {
             TableSchema tableSchema = JSON_FACTORY.fromString(jsonTableSchema, TableSchema.class);
             Bigquery client = Transport.newBigQueryClient(options).build();
-            BigQueryTableInserter inserter = new BigQueryTableInserter(client);
+            BigQueryTableInserter inserter = new BigQueryTableInserter(client, options);
             inserter.getOrCreateTable(tableReference, WriteDisposition.WRITE_APPEND,
                 CreateDisposition.CREATE_IF_NEEDED, tableSchema);
             createdTables.add(tableSpec);
@@ -2121,10 +2121,10 @@ public class BigQueryIO {
 
     /** Writes the accumulated rows into BigQuery with streaming API. */
     private void flushRows(Bigquery client, TableReference tableReference,
-        List<TableRow> tableRows, List<String> uniqueIds) {
+        List<TableRow> tableRows, List<String> uniqueIds, BigQueryOptions options) {
       if (!tableRows.isEmpty()) {
         try {
-          BigQueryTableInserter inserter = new BigQueryTableInserter(client);
+          BigQueryTableInserter inserter = new BigQueryTableInserter(client, options);
           inserter.insertAll(tableReference, tableRows, uniqueIds, byteCountAggregator);
         } catch (IOException e) {
           throw new RuntimeException(e);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6924358e/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryTableInserter.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryTableInserter.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryTableInserter.java
index f87a3c4..84004a7 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryTableInserter.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryTableInserter.java
@@ -22,6 +22,8 @@ import static com.google.common.base.Preconditions.checkNotNull;
 import org.apache.beam.sdk.io.BigQueryIO;
 import org.apache.beam.sdk.io.BigQueryIO.Write.CreateDisposition;
 import org.apache.beam.sdk.io.BigQueryIO.Write.WriteDisposition;
+import org.apache.beam.sdk.options.GcsOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.transforms.Aggregator;
 
 import com.google.api.client.util.BackOff;
@@ -38,7 +40,6 @@ import com.google.api.services.bigquery.model.TableRow;
 import com.google.api.services.bigquery.model.TableSchema;
 import com.google.cloud.hadoop.util.ApiErrorExtractor;
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.util.concurrent.MoreExecutors;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -50,9 +51,7 @@ import java.util.List;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
-import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 
 import javax.annotation.Nullable;
@@ -83,43 +82,51 @@ public class BigQueryTableInserter {
   private final TableReference defaultRef;
   private final long maxRowsPerBatch;
 
-  private static final ExecutorService executor = MoreExecutors.getExitingExecutorService(
-      (ThreadPoolExecutor) Executors.newFixedThreadPool(100), 10, TimeUnit.SECONDS);
+  private ExecutorService executor;
 
   /**
    * Constructs a new row inserter.
    *
    * @param client a BigQuery client
+   * @param options a PipelineOptions object
    */
-  public BigQueryTableInserter(Bigquery client) {
+  public BigQueryTableInserter(Bigquery client, PipelineOptions options) {
     this.client = client;
     this.defaultRef = null;
     this.maxRowsPerBatch = MAX_ROWS_PER_BATCH;
+    this.executor = options.as(GcsOptions.class).getExecutorService();
   }
 
   /**
    * Constructs a new row inserter.
    *
    * @param client a BigQuery client
+   * @param options a PipelineOptions object
    * @param defaultRef identifies the table to insert into
-   * @deprecated replaced by {@link #BigQueryTableInserter(Bigquery)}
+   * @deprecated replaced by {@link #BigQueryTableInserter(Bigquery, PipelineOptions)}
    */
   @Deprecated
-  public BigQueryTableInserter(Bigquery client, TableReference defaultRef) {
+  public BigQueryTableInserter(Bigquery client, PipelineOptions options,
+                               TableReference defaultRef) {
     this.client = client;
     this.defaultRef = defaultRef;
     this.maxRowsPerBatch = MAX_ROWS_PER_BATCH;
+    this.executor = options.as(GcsOptions.class).getExecutorService();
   }
 
   /**
    * Constructs a new row inserter.
    *
    * @param client a BigQuery client
+   * @param options a PipelineOptions object
+   * @param maxRowsPerBatch maximum number of rows to insert per call to BigQuery
    */
-  public BigQueryTableInserter(Bigquery client, int maxRowsPerBatch) {
+  public BigQueryTableInserter(Bigquery client, PipelineOptions options,
+                               int maxRowsPerBatch) {
     this.client = client;
     this.defaultRef = null;
     this.maxRowsPerBatch = maxRowsPerBatch;
+    this.executor = options.as(GcsOptions.class).getExecutorService();
   }
 
   /**
@@ -127,13 +134,15 @@ public class BigQueryTableInserter {
    *
    * @param client a BigQuery client
    * @param defaultRef identifies the default table to insert into
-   * @deprecated replaced by {@link #BigQueryTableInserter(Bigquery, int)}
+   * @deprecated replaced by {@link #BigQueryTableInserter(Bigquery, PipelineOptions, int)}
    */
   @Deprecated
-  public BigQueryTableInserter(Bigquery client, TableReference defaultRef, int maxRowsPerBatch) {
+  public BigQueryTableInserter(Bigquery client, PipelineOptions options,
+                               TableReference defaultRef, int maxRowsPerBatch) {
     this.client = client;
     this.defaultRef = defaultRef;
     this.maxRowsPerBatch = maxRowsPerBatch;
+    this.executor = options.as(GcsOptions.class).getExecutorService();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6924358e/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BigQueryTableInserterTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BigQueryTableInserterTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BigQueryTableInserterTest.java
index 7d9c8a8..344e916 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BigQueryTableInserterTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BigQueryTableInserterTest.java
@@ -28,6 +28,8 @@ import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.verifyNoMoreInteractions;
 import static org.mockito.Mockito.when;
 
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.testing.ExpectedLogs;
 
 import com.google.api.client.googleapis.json.GoogleJsonError;
@@ -75,6 +77,7 @@ public class BigQueryTableInserterTest {
   @Rule public ExpectedLogs expectedLogs = ExpectedLogs.none(BigQueryTableInserter.class);
   @Mock private LowLevelHttpResponse response;
   private Bigquery bigquery;
+  private PipelineOptions options;
 
   @Before
   public void setUp() {
@@ -97,6 +100,8 @@ public class BigQueryTableInserterTest {
         new Bigquery.Builder(
                 transport, Transport.getJsonFactory(), new RetryHttpRequestInitializer())
             .build();
+
+    options = PipelineOptionsFactory.create();
   }
 
   @After
@@ -139,7 +144,7 @@ public class BigQueryTableInserterTest {
     when(response.getStatusCode()).thenReturn(200);
     when(response.getContent()).thenReturn(toStream(testTable));
 
-    BigQueryTableInserter inserter = new BigQueryTableInserter(bigquery);
+    BigQueryTableInserter inserter = new BigQueryTableInserter(bigquery, options);
     Table ret =
         inserter.tryCreateTable(
             new Table(),
@@ -160,7 +165,7 @@ public class BigQueryTableInserterTest {
   public void testCreateTableSucceedsAlreadyExists() throws IOException {
     when(response.getStatusCode()).thenReturn(409); // 409 means already exists
 
-    BigQueryTableInserter inserter = new BigQueryTableInserter(bigquery);
+    BigQueryTableInserter inserter = new BigQueryTableInserter(bigquery, options);
     Table ret =
         inserter.tryCreateTable(
             new Table(),
@@ -191,7 +196,7 @@ public class BigQueryTableInserterTest {
         .thenReturn(toStream(errorWithReasonAndStatus("rateLimitExceeded", 403)))
         .thenReturn(toStream(testTable));
 
-    BigQueryTableInserter inserter = new BigQueryTableInserter(bigquery);
+    BigQueryTableInserter inserter = new BigQueryTableInserter(bigquery, options);
     Table ret =
         inserter.tryCreateTable(
             testTable,
@@ -227,7 +232,7 @@ public class BigQueryTableInserterTest {
     thrown.expect(GoogleJsonResponseException.class);
     thrown.expectMessage("actually forbidden");
 
-    BigQueryTableInserter inserter = new BigQueryTableInserter(bigquery);
+    BigQueryTableInserter inserter = new BigQueryTableInserter(bigquery, options);
     try {
       inserter.tryCreateTable(
           new Table(),
@@ -261,7 +266,7 @@ public class BigQueryTableInserterTest {
         .thenReturn(toStream(errorWithReasonAndStatus("rateLimitExceeded", 403)))
         .thenReturn(toStream(new TableDataInsertAllResponse()));
 
-    BigQueryTableInserter inserter = new BigQueryTableInserter(bigquery);
+    BigQueryTableInserter inserter = new BigQueryTableInserter(bigquery, options);
 
     inserter.insertAll(ref, rows);
     verify(response, times(2)).getStatusCode();
@@ -291,7 +296,7 @@ public class BigQueryTableInserterTest {
     thrown.expect(GoogleJsonResponseException.class);
     thrown.expectMessage("actually forbidden");
 
-    BigQueryTableInserter inserter = new BigQueryTableInserter(bigquery);
+    BigQueryTableInserter inserter = new BigQueryTableInserter(bigquery, options);
 
     try {
       inserter.insertAll(ref, rows);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6924358e/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BigQueryUtilTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BigQueryUtilTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BigQueryUtilTest.java
index 65fbeb7..c033a7d 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BigQueryUtilTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BigQueryUtilTest.java
@@ -31,6 +31,8 @@ import static org.mockito.Mockito.verifyNoMoreInteractions;
 import static org.mockito.Mockito.when;
 
 import org.apache.beam.sdk.io.BigQueryIO;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.transforms.Aggregator;
 import org.apache.beam.sdk.transforms.Combine.CombineFn;
 import org.apache.beam.sdk.transforms.Sum;
@@ -81,10 +83,12 @@ public class BigQueryUtilTest {
   @Mock private Bigquery.Tables.Get mockTablesGet;
   @Mock private Bigquery.Tabledata mockTabledata;
   @Mock private Bigquery.Tabledata.List mockTabledataList;
+  private PipelineOptions options;
 
   @Before
   public void setUp() {
     MockitoAnnotations.initMocks(this);
+    this.options = PipelineOptionsFactory.create();
   }
 
   @After
@@ -369,7 +373,7 @@ public class BigQueryUtilTest {
     TableReference ref = BigQueryIO
         .parseTableSpec("project:dataset.table");
 
-    BigQueryTableInserter inserter = new BigQueryTableInserter(mockClient);
+    BigQueryTableInserter inserter = new BigQueryTableInserter(mockClient, options);
 
     inserter.getOrCreateTable(ref, BigQueryIO.Write.WriteDisposition.WRITE_APPEND,
         BigQueryIO.Write.CreateDisposition.CREATE_NEVER, null);
@@ -387,7 +391,7 @@ public class BigQueryUtilTest {
     TableReference ref = BigQueryIO
         .parseTableSpec("project:dataset.table");
 
-    BigQueryTableInserter inserter = new BigQueryTableInserter(mockClient);
+    BigQueryTableInserter inserter = new BigQueryTableInserter(mockClient, options);
 
     inserter.getOrCreateTable(ref, BigQueryIO.Write.WriteDisposition.WRITE_EMPTY,
         BigQueryIO.Write.CreateDisposition.CREATE_NEVER, null);
@@ -408,7 +412,7 @@ public class BigQueryUtilTest {
     TableReference ref = BigQueryIO
         .parseTableSpec("project:dataset.table");
 
-    BigQueryTableInserter inserter = new BigQueryTableInserter(mockClient);
+    BigQueryTableInserter inserter = new BigQueryTableInserter(mockClient, options);
 
     try {
       inserter.getOrCreateTable(ref, BigQueryIO.Write.WriteDisposition.WRITE_EMPTY,
@@ -432,7 +436,7 @@ public class BigQueryUtilTest {
 
     TableReference ref = BigQueryIO
         .parseTableSpec("project:dataset.table");
-    BigQueryTableInserter inserter = new BigQueryTableInserter(mockClient, 5);
+    BigQueryTableInserter inserter = new BigQueryTableInserter(mockClient, options, 5);
 
     List<TableRow> rows = new ArrayList<>();
     List<String> ids = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6924358e/sdks/java/core/src/test/java/org/apache/beam/sdk/util/RetryHttpRequestInitializerTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/RetryHttpRequestInitializerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/RetryHttpRequestInitializerTest.java
index 7d212d4..83ffaa1 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/RetryHttpRequestInitializerTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/RetryHttpRequestInitializerTest.java
@@ -30,6 +30,8 @@ import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.verifyNoMoreInteractions;
 import static org.mockito.Mockito.when;
 
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+
 import com.google.api.client.auth.oauth2.Credential;
 import com.google.api.client.http.HttpRequest;
 import com.google.api.client.http.HttpResponse;
@@ -281,7 +283,8 @@ public class RetryHttpRequestInitializerTest {
     // RetryHttpInitializer.
     Bigquery b = new Bigquery.Builder(
         transport, Transport.getJsonFactory(), new RetryHttpRequestInitializer()).build();
-    BigQueryTableInserter inserter = new BigQueryTableInserter(b);
+
+    BigQueryTableInserter inserter = new BigQueryTableInserter(b, PipelineOptionsFactory.create());
     TableReference t = new TableReference()
         .setProjectId("project").setDatasetId("dataset").setTableId("table");
 


[42/50] [abbrv] incubator-beam git commit: Replace PubsubIO and injector with TextIO in traffic examples

Posted by lc...@apache.org.
Replace PubsubIO and injector with TextIO in traffic examples


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

Branch: refs/heads/runners-spark2
Commit: ae75a5d8c18098356f5e96ed25ca543a846d8f5b
Parents: 151ff5f
Author: Pei He <pe...@google.com>
Authored: Fri Jul 1 14:45:43 2016 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Jul 6 10:18:53 2016 -0700

----------------------------------------------------------------------
 .../examples/complete/TrafficMaxLaneFlow.java   | 64 ++++----------------
 .../beam/examples/complete/TrafficRoutes.java   | 62 +++----------------
 2 files changed, 22 insertions(+), 104 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ae75a5d8/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficMaxLaneFlow.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficMaxLaneFlow.java b/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficMaxLaneFlow.java
index cfef311..2db7c9e 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficMaxLaneFlow.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficMaxLaneFlow.java
@@ -20,13 +20,11 @@ package org.apache.beam.examples.complete;
 import org.apache.beam.examples.common.DataflowExampleOptions;
 import org.apache.beam.examples.common.DataflowExampleUtils;
 import org.apache.beam.examples.common.ExampleBigQueryTableOptions;
-import org.apache.beam.examples.common.ExamplePubsubTopicAndSubscriptionOptions;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.PipelineResult;
 import org.apache.beam.sdk.coders.AvroCoder;
 import org.apache.beam.sdk.coders.DefaultCoder;
 import org.apache.beam.sdk.io.BigQueryIO;
-import org.apache.beam.sdk.io.PubsubIO;
 import org.apache.beam.sdk.io.TextIO;
 import org.apache.beam.sdk.options.Default;
 import org.apache.beam.sdk.options.Description;
@@ -46,7 +44,6 @@ import com.google.api.services.bigquery.model.TableFieldSchema;
 import com.google.api.services.bigquery.model.TableReference;
 import com.google.api.services.bigquery.model.TableRow;
 import com.google.api.services.bigquery.model.TableSchema;
-import com.google.common.base.Strings;
 
 import org.apache.avro.reflect.Nullable;
 import org.joda.time.Duration;
@@ -62,30 +59,19 @@ import java.util.List;
  * A Dataflow Example that runs in both batch and streaming modes with traffic sensor data.
  * You can configure the running mode by setting {@literal --streaming} to true or false.
  *
- * <p>Concepts: The batch and streaming runners, sliding windows, Google Cloud Pub/Sub
- * topic injection, use of the AvroCoder to encode a custom class, and custom Combine transforms.
+ * <p>Concepts: The batch and streaming runners, sliding windows,
+ * use of the AvroCoder to encode a custom class, and custom Combine transforms.
  *
  * <p>This example analyzes traffic sensor data using SlidingWindows. For each window,
  * it finds the lane that had the highest flow recorded, for each sensor station. It writes
  * those max values along with auxiliary info to a BigQuery table.
  *
- * <p>In batch mode, the pipeline reads traffic sensor data from {@literal --inputFile}.
+ * <p>The pipeline reads traffic sensor data from {@literal --inputFile}.
  *
- * <p>In streaming mode, the pipeline reads the data from a Pub/Sub topic.
- * By default, the example will run a separate pipeline to inject the data from the default
- * {@literal --inputFile} to the Pub/Sub {@literal --pubsubTopic}. It will make it available for
- * the streaming pipeline to process. You may override the default {@literal --inputFile} with the
- * file of your choosing. You may also set {@literal --inputFile} to an empty string, which will
- * disable the automatic Pub/Sub injection, and allow you to use separate tool to control the input
- * to this example. An example code, which publishes traffic sensor data to a Pub/Sub topic,
- * is provided in
- * <a href="https://github.com/GoogleCloudPlatform/cloud-pubsub-samples-python/tree/master/gce-cmdline-publisher"></a>.
- *
- * <p>The example is configured to use the default Pub/Sub topic and the default BigQuery table
- * from the example common package (there are no defaults for a general Dataflow pipeline).
- * You can override them by using the {@literal --pubsubTopic}, {@literal --bigQueryDataset}, and
- * {@literal --bigQueryTable} options. If the Pub/Sub topic or the BigQuery table do not exist,
- * the example will try to create them.
+ * <p>The example is configured to use the default BigQuery table from the example common package
+ * (there are no defaults for a general Dataflow pipeline).
+ * You can override them by using the {@literal --bigQueryDataset}, and {@literal --bigQueryTable}
+ * options. If the BigQuery table do not exist, the example will try to create them.
  *
  * <p>The example will try to cancel the pipelines on the signal to terminate the process (CTRL-C)
  * and then exits.
@@ -322,8 +308,8 @@ public class TrafficMaxLaneFlow {
     * <p>Inherits standard configuration options.
     */
   private interface TrafficMaxLaneFlowOptions extends DataflowExampleOptions,
-      ExamplePubsubTopicAndSubscriptionOptions, ExampleBigQueryTableOptions {
-        @Description("Input file to inject to Pub/Sub topic")
+      ExampleBigQueryTableOptions {
+    @Description("Path of the file to read from")
     @Default.String("gs://dataflow-samples/traffic_sensor/"
         + "Freeways-5Minaa2010-01-01_to_2010-02-15_test2.csv")
     String getInputFile();
@@ -364,28 +350,11 @@ public class TrafficMaxLaneFlow {
     tableRef.setDatasetId(options.getBigQueryDataset());
     tableRef.setTableId(options.getBigQueryTable());
 
-    PCollection<String> input;
-    if (options.isUnbounded()) {
-      // Read unbounded PubSubIO.
-      input = pipeline.apply(PubsubIO.Read
-          .timestampLabel(PUBSUB_TIMESTAMP_LABEL_KEY)
-          .subscription(options.getPubsubSubscription()));
-    } else {
-      // Read bounded PubSubIO.
-      input = pipeline.apply(PubsubIO.Read
-          .timestampLabel(PUBSUB_TIMESTAMP_LABEL_KEY)
-          .subscription(options.getPubsubSubscription()).maxNumRecords(VALID_INPUTS));
-
-      // To read bounded TextIO files, use:
-      // input = pipeline.apply(new ReadFileAndExtractTimestamps(options.getInputFile()));
-    }
-    input
+    pipeline
+        .apply("ReadLines", new ReadFileAndExtractTimestamps(options.getInputFile()))
         // row... => <station route, station speed> ...
         .apply(ParDo.of(new ExtractFlowInfoFn()))
-        // map the incoming data stream into sliding windows. The default window duration values
-        // work well if you're running the accompanying Pub/Sub generator script with the
-        // --replay flag, which simulates pauses in the sensor data publication. You may want to
-        // adjust them otherwise.
+        // map the incoming data stream into sliding windows.
         .apply(Window.<KV<String, LaneInfo>>into(SlidingWindows.of(
             Duration.standardMinutes(options.getWindowDuration())).
             every(Duration.standardMinutes(options.getWindowSlideEvery()))))
@@ -393,15 +362,6 @@ public class TrafficMaxLaneFlow {
         .apply(BigQueryIO.Write.to(tableRef)
             .withSchema(FormatMaxesFn.getSchema()));
 
-    // Inject the data into the Pub/Sub topic with a Dataflow batch pipeline.
-    if (!Strings.isNullOrEmpty(options.getInputFile())
-        && !Strings.isNullOrEmpty(options.getPubsubTopic())) {
-      dataflowUtils.runInjectorPipeline(
-          new ReadFileAndExtractTimestamps(options.getInputFile()),
-          options.getPubsubTopic(),
-          PUBSUB_TIMESTAMP_LABEL_KEY);
-    }
-
     // Run the pipeline.
     PipelineResult result = pipeline.run();
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ae75a5d8/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficRoutes.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficRoutes.java b/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficRoutes.java
index c9bada8..89cfbfc 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficRoutes.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficRoutes.java
@@ -20,13 +20,11 @@ package org.apache.beam.examples.complete;
 import org.apache.beam.examples.common.DataflowExampleOptions;
 import org.apache.beam.examples.common.DataflowExampleUtils;
 import org.apache.beam.examples.common.ExampleBigQueryTableOptions;
-import org.apache.beam.examples.common.ExamplePubsubTopicAndSubscriptionOptions;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.PipelineResult;
 import org.apache.beam.sdk.coders.AvroCoder;
 import org.apache.beam.sdk.coders.DefaultCoder;
 import org.apache.beam.sdk.io.BigQueryIO;
-import org.apache.beam.sdk.io.PubsubIO;
 import org.apache.beam.sdk.io.TextIO;
 import org.apache.beam.sdk.options.Default;
 import org.apache.beam.sdk.options.Description;
@@ -45,7 +43,6 @@ import com.google.api.services.bigquery.model.TableFieldSchema;
 import com.google.api.services.bigquery.model.TableReference;
 import com.google.api.services.bigquery.model.TableRow;
 import com.google.api.services.bigquery.model.TableSchema;
-import com.google.common.base.Strings;
 import com.google.common.collect.Lists;
 
 import org.apache.avro.reflect.Nullable;
@@ -66,30 +63,18 @@ import java.util.Map;
  * A Dataflow Example that runs in both batch and streaming modes with traffic sensor data.
  * You can configure the running mode by setting {@literal --streaming} to true or false.
  *
- * <p>Concepts: The batch and streaming runners, GroupByKey, sliding windows, and
- * Google Cloud Pub/Sub topic injection.
+ * <p>Concepts: The batch and streaming runners, GroupByKey, sliding windows.
  *
  * <p>This example analyzes traffic sensor data using SlidingWindows. For each window,
  * it calculates the average speed over the window for some small set of predefined 'routes',
  * and looks for 'slowdowns' in those routes. It writes its results to a BigQuery table.
  *
- * <p>In batch mode, the pipeline reads traffic sensor data from {@literal --inputFile}.
+ * <p>The pipeline reads traffic sensor data from {@literal --inputFile}.
  *
- * <p>In streaming mode, the pipeline reads the data from a Pub/Sub topic.
- * By default, the example will run a separate pipeline to inject the data from the default
- * {@literal --inputFile} to the Pub/Sub {@literal --pubsubTopic}. It will make it available for
- * the streaming pipeline to process. You may override the default {@literal --inputFile} with the
- * file of your choosing. You may also set {@literal --inputFile} to an empty string, which will
- * disable the automatic Pub/Sub injection, and allow you to use separate tool to control the input
- * to this example. An example code, which publishes traffic sensor data to a Pub/Sub topic,
- * is provided in
- * <a href="https://github.com/GoogleCloudPlatform/cloud-pubsub-samples-python/tree/master/gce-cmdline-publisher"></a>.
- *
- * <p>The example is configured to use the default Pub/Sub topic and the default BigQuery table
- * from the example common package (there are no defaults for a general Dataflow pipeline).
- * You can override them by using the {@literal --pubsubTopic}, {@literal --bigQueryDataset}, and
- * {@literal --bigQueryTable} options. If the Pub/Sub topic or the BigQuery table do not exist,
- * the example will try to create them.
+ * <p>The example is configured to use the default BigQuery table from the example common package
+ * (there are no defaults for a general Dataflow pipeline).
+ * You can override them by using the {@literal --bigQueryDataset}, and {@literal --bigQueryTable}
+ * options. If the BigQuery table do not exist, the example will try to create them.
  *
  * <p>The example will try to cancel the pipelines on the signal to terminate the process (CTRL-C)
  * and then exits.
@@ -333,8 +318,8 @@ public class TrafficRoutes {
   * <p>Inherits standard configuration options.
   */
   private interface TrafficRoutesOptions extends DataflowExampleOptions,
-      ExamplePubsubTopicAndSubscriptionOptions, ExampleBigQueryTableOptions {
-    @Description("Input file to inject to Pub/Sub topic")
+      ExampleBigQueryTableOptions {
+    @Description("Path of the file to read from")
     @Default.String("gs://dataflow-samples/traffic_sensor/"
         + "Freeways-5Minaa2010-01-01_to_2010-02-15_test2.csv")
     String getInputFile();
@@ -376,29 +361,11 @@ public class TrafficRoutes {
     tableRef.setDatasetId(options.getBigQueryDataset());
     tableRef.setTableId(options.getBigQueryTable());
 
-    PCollection<String> input;
-    if (options.isUnbounded()) {
-      // Read unbounded PubSubIO.
-      input = pipeline.apply(PubsubIO.Read
-          .timestampLabel(PUBSUB_TIMESTAMP_LABEL_KEY)
-          .subscription(options.getPubsubSubscription()));
-    } else {
-      // Read bounded PubSubIO.
-      input = pipeline.apply(PubsubIO.Read
-          .timestampLabel(PUBSUB_TIMESTAMP_LABEL_KEY)
-          .subscription(options.getPubsubSubscription()).maxNumRecords(VALID_INPUTS));
-
-      // To read bounded TextIO files, use:
-      // input = pipeline.apply(TextIO.Read.from(options.getInputFile()))
-      //    .apply(ParDo.of(new ExtractTimestamps()));
-    }
-    input
+    pipeline
+        .apply("ReadLines", new ReadFileAndExtractTimestamps(options.getInputFile()))
         // row... => <station route, station speed> ...
         .apply(ParDo.of(new ExtractStationSpeedFn()))
         // map the incoming data stream into sliding windows.
-        // The default window duration values work well if you're running the accompanying Pub/Sub
-        // generator script without the --replay flag, so that there are no simulated pauses in
-        // the sensor data publication. You may want to adjust the values otherwise.
         .apply(Window.<KV<String, StationSpeed>>into(SlidingWindows.of(
             Duration.standardMinutes(options.getWindowDuration())).
             every(Duration.standardMinutes(options.getWindowSlideEvery()))))
@@ -406,15 +373,6 @@ public class TrafficRoutes {
         .apply(BigQueryIO.Write.to(tableRef)
             .withSchema(FormatStatsFn.getSchema()));
 
-    // Inject the data into the Pub/Sub topic with a Dataflow batch pipeline.
-    if (!Strings.isNullOrEmpty(options.getInputFile())
-        && !Strings.isNullOrEmpty(options.getPubsubTopic())) {
-      dataflowUtils.runInjectorPipeline(
-          new ReadFileAndExtractTimestamps(options.getInputFile()),
-          options.getPubsubTopic(),
-          PUBSUB_TIMESTAMP_LABEL_KEY);
-    }
-
     // Run the pipeline.
     PipelineResult result = pipeline.run();
 


[14/50] [abbrv] incubator-beam git commit: Move remnants of Dataflow runner from SDK to its module

Posted by lc...@apache.org.
Move remnants of Dataflow runner from SDK to its module


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

Branch: refs/heads/runners-spark2
Commit: 5472c97e91e94bed2320b69675e7bed689b3bf68
Parents: a53450b
Author: Kenneth Knowles <kl...@google.com>
Authored: Thu Jun 23 15:59:23 2016 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Jul 6 10:18:50 2016 -0700

----------------------------------------------------------------------
 .../DataflowPipelineTranslatorTest.java         |  1 -
 .../dataflow/RecordingPipelineVisitor.java      | 46 +++++++++++++++++++
 .../sdk/runners/RecordingPipelineVisitor.java   | 47 --------------------
 .../beam/sdk/runners/TransformTreeTest.java     |  2 +-
 4 files changed, 47 insertions(+), 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5472c97e/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java
index e04a1fc..48c757f 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java
@@ -47,7 +47,6 @@ import org.apache.beam.sdk.coders.VarIntCoder;
 import org.apache.beam.sdk.coders.VoidCoder;
 import org.apache.beam.sdk.io.TextIO;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.runners.RecordingPipelineVisitor;
 import org.apache.beam.sdk.transforms.Count;
 import org.apache.beam.sdk.transforms.Create;
 import org.apache.beam.sdk.transforms.DoFn;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5472c97e/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/RecordingPipelineVisitor.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/RecordingPipelineVisitor.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/RecordingPipelineVisitor.java
new file mode 100644
index 0000000..8ebc4bd
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/RecordingPipelineVisitor.java
@@ -0,0 +1,46 @@
+/*
+ * 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.runners.dataflow;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.runners.TransformTreeNode;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PValue;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Provides a simple {@link org.apache.beam.sdk.Pipeline.PipelineVisitor}
+ * that records the transformation tree.
+ */
+class RecordingPipelineVisitor extends Pipeline.PipelineVisitor.Defaults {
+
+  public final List<PTransform<?, ?>> transforms = new ArrayList<>();
+  public final List<PValue> values = new ArrayList<>();
+
+  @Override
+  public void visitPrimitiveTransform(TransformTreeNode node) {
+    transforms.add(node.getTransform());
+  }
+
+  @Override
+  public void visitValue(PValue value, TransformTreeNode producer) {
+    values.add(value);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5472c97e/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/RecordingPipelineVisitor.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/RecordingPipelineVisitor.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/RecordingPipelineVisitor.java
deleted file mode 100644
index d64738f..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/RecordingPipelineVisitor.java
+++ /dev/null
@@ -1,47 +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.runners;
-
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.values.PValue;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * Provides a simple {@link org.apache.beam.sdk.Pipeline.PipelineVisitor}
- * that records the transformation tree.
- *
- * <p>Provided for internal unit tests.
- */
-public class RecordingPipelineVisitor extends Pipeline.PipelineVisitor.Defaults {
-
-  public final List<PTransform<?, ?>> transforms = new ArrayList<>();
-  public final List<PValue> values = new ArrayList<>();
-
-  @Override
-  public void visitPrimitiveTransform(TransformTreeNode node) {
-    transforms.add(node.getTransform());
-  }
-
-  @Override
-  public void visitValue(PValue value, TransformTreeNode producer) {
-    values.add(value);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5472c97e/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformTreeTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformTreeTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformTreeTest.java
index 08c3996..9009a77 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformTreeTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformTreeTest.java
@@ -177,7 +177,7 @@ public class TransformTreeTest {
 
     p.apply(new InvalidCompositeTransform());
 
-    p.traverseTopologically(new RecordingPipelineVisitor());
+    p.traverseTopologically(new Pipeline.PipelineVisitor.Defaults() {});
     fail("traversal should have failed with an IllegalStateException");
   }
 


[30/50] [abbrv] incubator-beam git commit: Copy and use UnboundedReadFromBoundedSource in dataflow runner

Posted by lc...@apache.org.
Copy and use UnboundedReadFromBoundedSource in dataflow runner


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

Branch: refs/heads/runners-spark2
Commit: 56ec6c3e060d8a8aa5dd5b482e93b191a8771d92
Parents: 5211c4a
Author: Pei He <pe...@google.com>
Authored: Mon Jun 27 17:29:14 2016 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Jul 6 10:18:51 2016 -0700

----------------------------------------------------------------------
 .../beam/runners/dataflow/DataflowRunner.java   |  34 +-
 .../DataflowUnboundedReadFromBoundedSource.java | 547 +++++++++++++++++++
 .../runners/dataflow/DataflowRunnerTest.java    |  30 -
 3 files changed, 577 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/56ec6c3e/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
index 70dd94f..2ba6c7b 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
@@ -30,6 +30,7 @@ import org.apache.beam.runners.dataflow.DataflowPipelineTranslator.TransformTran
 import org.apache.beam.runners.dataflow.DataflowPipelineTranslator.TranslationContext;
 import org.apache.beam.runners.dataflow.internal.AssignWindows;
 import org.apache.beam.runners.dataflow.internal.DataflowAggregatorTransforms;
+import org.apache.beam.runners.dataflow.internal.DataflowUnboundedReadFromBoundedSource;
 import org.apache.beam.runners.dataflow.internal.IsmFormat;
 import org.apache.beam.runners.dataflow.internal.IsmFormat.IsmRecord;
 import org.apache.beam.runners.dataflow.internal.IsmFormat.IsmRecordCoder;
@@ -60,6 +61,7 @@ import org.apache.beam.sdk.coders.VarIntCoder;
 import org.apache.beam.sdk.coders.VarLongCoder;
 import org.apache.beam.sdk.io.AvroIO;
 import org.apache.beam.sdk.io.BigQueryIO;
+import org.apache.beam.sdk.io.BoundedSource;
 import org.apache.beam.sdk.io.FileBasedSink;
 import org.apache.beam.sdk.io.PubsubIO;
 import org.apache.beam.sdk.io.PubsubUnboundedSink;
@@ -350,11 +352,8 @@ public class DataflowRunner extends PipelineRunner<DataflowPipelineJob> {
       builder.put(View.AsIterable.class, StreamingViewAsIterable.class);
       builder.put(Write.Bound.class, StreamingWrite.class);
       builder.put(Read.Unbounded.class, StreamingUnboundedRead.class);
-      builder.put(Read.Bounded.class, UnsupportedIO.class);
-      builder.put(AvroIO.Read.Bound.class, UnsupportedIO.class);
+      builder.put(Read.Bounded.class, StreamingBoundedRead.class);
       builder.put(AvroIO.Write.Bound.class, UnsupportedIO.class);
-      builder.put(BigQueryIO.Read.Bound.class, UnsupportedIO.class);
-      builder.put(TextIO.Read.Bound.class, UnsupportedIO.class);
       builder.put(TextIO.Write.Bound.class, UnsupportedIO.class);
       builder.put(Window.Bound.class, AssignWindows.class);
       // In streaming mode must use either the custom Pubsub unbounded source/sink or
@@ -2366,6 +2365,33 @@ public class DataflowRunner extends PipelineRunner<DataflowPipelineJob> {
   }
 
   /**
+   * Specialized implementation for {@link org.apache.beam.sdk.io.Read.Bounded Read.Bounded} for the
+   * Dataflow runner in streaming mode.
+   */
+  private static class StreamingBoundedRead<T> extends PTransform<PInput, PCollection<T>> {
+    private final BoundedSource<T> source;
+
+    /** Builds an instance of this class from the overridden transform. */
+    @SuppressWarnings("unused") // used via reflection in DataflowRunner#apply()
+    public StreamingBoundedRead(DataflowRunner runner, Read.Bounded<T> transform) {
+      this.source = transform.getSource();
+    }
+
+    @Override
+    protected Coder<T> getDefaultOutputCoder() {
+      return source.getDefaultOutputCoder();
+    }
+
+    @Override
+    public final PCollection<T> apply(PInput input) {
+      source.validate();
+
+      return Pipeline.applyTransform(input, new DataflowUnboundedReadFromBoundedSource<>(source))
+          .setIsBoundedInternal(IsBounded.BOUNDED);
+    }
+  }
+
+  /**
    * Specialized implementation for
    * {@link org.apache.beam.sdk.transforms.Create.Values Create.Values} for the
    * Dataflow runner in streaming mode.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/56ec6c3e/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/DataflowUnboundedReadFromBoundedSource.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/DataflowUnboundedReadFromBoundedSource.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/DataflowUnboundedReadFromBoundedSource.java
new file mode 100644
index 0000000..5e035bc
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/DataflowUnboundedReadFromBoundedSource.java
@@ -0,0 +1,547 @@
+/*
+ * 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.runners.dataflow.internal;
+
+import static org.apache.beam.sdk.util.StringUtils.approximateSimpleName;
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.ListCoder;
+import org.apache.beam.sdk.coders.NullableCoder;
+import org.apache.beam.sdk.coders.SerializableCoder;
+import org.apache.beam.sdk.coders.StandardCoder;
+import org.apache.beam.sdk.io.BoundedSource;
+import org.apache.beam.sdk.io.BoundedSource.BoundedReader;
+import org.apache.beam.sdk.io.Read;
+import org.apache.beam.sdk.io.UnboundedSource;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.PropertyNames;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.TimestampedValue;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Function;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import org.joda.time.Instant;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+
+import javax.annotation.Nullable;
+
+/**
+ * {@link PTransform} that converts a {@link BoundedSource} as an {@link UnboundedSource}.
+ *
+ * <p>{@link BoundedSource} is read directly without calling {@link BoundedSource#splitIntoBundles},
+ * and element timestamps are propagated. While any elements remain, the watermark is the beginning
+ * of time {@link BoundedWindow#TIMESTAMP_MIN_VALUE}, and after all elements have been produced
+ * the watermark goes to the end of time {@link BoundedWindow#TIMESTAMP_MAX_VALUE}.
+ *
+ * <p>Checkpoints are created by calling {@link BoundedReader#splitAtFraction} on inner
+ * {@link BoundedSource}.
+ * Sources that cannot be split are read entirely into memory, so this transform does not work well
+ * with large, unsplittable sources.
+ *
+ * <p>This transform is intended to be used by a runner during pipeline translation to convert
+ * a Read.Bounded into a Read.Unbounded.
+ *
+ * @deprecated This class is copied from beam runners core in order to avoid pipeline construction
+ * time dependency. It should be replaced in the dataflow worker as an execution time dependency.
+ */
+@Deprecated
+public class DataflowUnboundedReadFromBoundedSource<T> extends PTransform<PInput, PCollection<T>> {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(DataflowUnboundedReadFromBoundedSource.class);
+
+  private final BoundedSource<T> source;
+
+  /**
+   * Constructs a {@link PTransform} that performs an unbounded read from a {@link BoundedSource}.
+   */
+  public DataflowUnboundedReadFromBoundedSource(BoundedSource<T> source) {
+    this.source = source;
+  }
+
+  @Override
+  public PCollection<T> apply(PInput input) {
+    return input.getPipeline().apply(
+        Read.from(new BoundedToUnboundedSourceAdapter<>(source)));
+  }
+
+  @Override
+  protected Coder<T> getDefaultOutputCoder() {
+    return source.getDefaultOutputCoder();
+  }
+
+  @Override
+  public String getKindString() {
+    return "Read(" + approximateSimpleName(source.getClass()) + ")";
+  }
+
+  @Override
+  public void populateDisplayData(DisplayData.Builder builder) {
+    // We explicitly do not register base-class data, instead we use the delegate inner source.
+    builder
+        .add(DisplayData.item("source", source.getClass()))
+        .include(source);
+  }
+
+  /**
+   * A {@code BoundedSource} to {@code UnboundedSource} adapter.
+   */
+  @VisibleForTesting
+  public static class BoundedToUnboundedSourceAdapter<T>
+      extends UnboundedSource<T, BoundedToUnboundedSourceAdapter.Checkpoint<T>> {
+
+    private BoundedSource<T> boundedSource;
+
+    public BoundedToUnboundedSourceAdapter(BoundedSource<T> boundedSource) {
+      this.boundedSource = boundedSource;
+    }
+
+    @Override
+    public void validate() {
+      boundedSource.validate();
+    }
+
+    @Override
+    public List<BoundedToUnboundedSourceAdapter<T>> generateInitialSplits(
+        int desiredNumSplits, PipelineOptions options) throws Exception {
+      try {
+        long desiredBundleSize = boundedSource.getEstimatedSizeBytes(options) / desiredNumSplits;
+        if (desiredBundleSize <= 0) {
+          LOG.warn("BoundedSource {} cannot estimate its size, skips the initial splits.",
+              boundedSource);
+          return ImmutableList.of(this);
+        }
+        List<? extends BoundedSource<T>> splits
+            = boundedSource.splitIntoBundles(desiredBundleSize, options);
+        if (splits == null) {
+          LOG.warn("BoundedSource cannot split {}, skips the initial splits.", boundedSource);
+          return ImmutableList.of(this);
+        }
+        return Lists.transform(
+            splits,
+            new Function<BoundedSource<T>, BoundedToUnboundedSourceAdapter<T>>() {
+              @Override
+              public BoundedToUnboundedSourceAdapter<T> apply(BoundedSource<T> input) {
+                return new BoundedToUnboundedSourceAdapter<>(input);
+              }});
+      } catch (Exception e) {
+        LOG.warn("Exception while splitting {}, skips the initial splits.", boundedSource, e);
+        return ImmutableList.of(this);
+      }
+    }
+
+    @Override
+    public Reader createReader(PipelineOptions options, Checkpoint<T> checkpoint)
+        throws IOException {
+      if (checkpoint == null) {
+        return new Reader(null /* residualElements */, boundedSource, options);
+      } else {
+        return new Reader(checkpoint.residualElements, checkpoint.residualSource, options);
+      }
+    }
+
+    @Override
+    public Coder<T> getDefaultOutputCoder() {
+      return boundedSource.getDefaultOutputCoder();
+    }
+
+    @SuppressWarnings({"rawtypes", "unchecked"})
+    @Override
+    public Coder<Checkpoint<T>> getCheckpointMarkCoder() {
+      return new CheckpointCoder<>(boundedSource.getDefaultOutputCoder());
+    }
+
+    @VisibleForTesting
+    static class Checkpoint<T> implements UnboundedSource.CheckpointMark {
+      private final @Nullable List<TimestampedValue<T>> residualElements;
+      private final @Nullable BoundedSource<T> residualSource;
+
+      public Checkpoint(
+          @Nullable List<TimestampedValue<T>> residualElements,
+          @Nullable BoundedSource<T> residualSource) {
+        this.residualElements = residualElements;
+        this.residualSource = residualSource;
+      }
+
+      @Override
+      public void finalizeCheckpoint() {}
+
+      @VisibleForTesting
+      @Nullable List<TimestampedValue<T>> getResidualElements() {
+        return residualElements;
+      }
+
+      @VisibleForTesting
+      @Nullable BoundedSource<T> getResidualSource() {
+        return residualSource;
+      }
+    }
+
+    @VisibleForTesting
+    static class CheckpointCoder<T> extends StandardCoder<Checkpoint<T>> {
+
+      @JsonCreator
+      public static CheckpointCoder<?> of(
+          @JsonProperty(PropertyNames.COMPONENT_ENCODINGS)
+          List<Coder<?>> components) {
+        checkArgument(components.size() == 1,
+            "Expecting 1 components, got %s", components.size());
+        return new CheckpointCoder<>(components.get(0));
+      }
+
+      // The coder for a list of residual elements and their timestamps
+      private final Coder<List<TimestampedValue<T>>> elemsCoder;
+      // The coder from the BoundedReader for coding each element
+      private final Coder<T> elemCoder;
+      // The nullable and serializable coder for the BoundedSource.
+      @SuppressWarnings("rawtypes")
+      private final Coder<BoundedSource> sourceCoder;
+
+      CheckpointCoder(Coder<T> elemCoder) {
+        this.elemsCoder = NullableCoder.of(
+            ListCoder.of(TimestampedValue.TimestampedValueCoder.of(elemCoder)));
+        this.elemCoder = elemCoder;
+        this.sourceCoder = NullableCoder.of(SerializableCoder.of(BoundedSource.class));
+      }
+
+      @Override
+      public void encode(Checkpoint<T> value, OutputStream outStream, Context context)
+          throws CoderException, IOException {
+        Context nested = context.nested();
+        elemsCoder.encode(value.residualElements, outStream, nested);
+        sourceCoder.encode(value.residualSource, outStream, nested);
+      }
+
+      @SuppressWarnings("unchecked")
+      @Override
+      public Checkpoint<T> decode(InputStream inStream, Context context)
+          throws CoderException, IOException {
+        Context nested = context.nested();
+        return new Checkpoint<>(
+            elemsCoder.decode(inStream, nested),
+            sourceCoder.decode(inStream, nested));
+      }
+
+      @Override
+      public List<Coder<?>> getCoderArguments() {
+        return Arrays.<Coder<?>>asList(elemCoder);
+      }
+
+      @Override
+      public void verifyDeterministic() throws NonDeterministicException {
+        throw new NonDeterministicException(this,
+            "CheckpointCoder uses Java Serialization, which may be non-deterministic.");
+      }
+    }
+
+    /**
+     * An {@code UnboundedReader<T>} that wraps a {@code BoundedSource<T>} into
+     * {@link ResidualElements} and {@link ResidualSource}.
+     *
+     * <p>In the initial state, {@link ResidualElements} is null and {@link ResidualSource} contains
+     * the {@code BoundedSource<T>}. After the first checkpoint, the {@code BoundedSource<T>} will
+     * be split into {@link ResidualElements} and {@link ResidualSource}.
+     */
+    @VisibleForTesting
+    class Reader extends UnboundedReader<T> {
+      private ResidualElements residualElements;
+      private @Nullable ResidualSource residualSource;
+      private final PipelineOptions options;
+      private boolean done;
+
+      Reader(
+          @Nullable List<TimestampedValue<T>> residualElementsList,
+          @Nullable BoundedSource<T> residualSource,
+          PipelineOptions options) {
+        init(residualElementsList, residualSource, options);
+        this.options = checkNotNull(options, "options");
+        this.done = false;
+      }
+
+      private void init(
+          @Nullable List<TimestampedValue<T>> residualElementsList,
+          @Nullable BoundedSource<T> residualSource,
+          PipelineOptions options) {
+        this.residualElements = residualElementsList == null
+            ? new ResidualElements(Collections.<TimestampedValue<T>>emptyList())
+                : new ResidualElements(residualElementsList);
+        this.residualSource =
+            residualSource == null ? null : new ResidualSource(residualSource, options);
+      }
+
+      @Override
+      public boolean start() throws IOException {
+        return advance();
+      }
+
+      @Override
+      public boolean advance() throws IOException {
+        if (residualElements.advance()) {
+          return true;
+        } else if (residualSource != null && residualSource.advance()) {
+          return true;
+        } else {
+          done = true;
+          return false;
+        }
+      }
+
+      @Override
+      public void close() throws IOException {
+        if (residualSource != null) {
+          residualSource.close();
+        }
+      }
+
+      @Override
+      public T getCurrent() throws NoSuchElementException {
+        if (residualElements.hasCurrent()) {
+          return residualElements.getCurrent();
+        } else if (residualSource != null) {
+          return residualSource.getCurrent();
+        } else {
+          throw new NoSuchElementException();
+        }
+      }
+
+      @Override
+      public Instant getCurrentTimestamp() throws NoSuchElementException {
+        if (residualElements.hasCurrent()) {
+          return residualElements.getCurrentTimestamp();
+        } else if (residualSource != null) {
+          return residualSource.getCurrentTimestamp();
+        } else {
+          throw new NoSuchElementException();
+        }
+      }
+
+      @Override
+      public Instant getWatermark() {
+        return done ? BoundedWindow.TIMESTAMP_MAX_VALUE : BoundedWindow.TIMESTAMP_MIN_VALUE;
+      }
+
+      /**
+       * {@inheritDoc}
+       *
+       * <p>If only part of the {@link ResidualElements} is consumed, the new
+       * checkpoint will contain the remaining elements in {@link ResidualElements} and
+       * the {@link ResidualSource}.
+       *
+       * <p>If all {@link ResidualElements} and part of the
+       * {@link ResidualSource} are consumed, the new checkpoint is done by splitting
+       * {@link ResidualSource} into new {@link ResidualElements} and {@link ResidualSource}.
+       * {@link ResidualSource} is the source split from the current source,
+       * and {@link ResidualElements} contains rest elements from the current source after
+       * the splitting. For unsplittable source, it will put all remaining elements into
+       * the {@link ResidualElements}.
+       */
+      @Override
+      public Checkpoint<T> getCheckpointMark() {
+        Checkpoint<T> newCheckpoint;
+        if (!residualElements.done()) {
+          // Part of residualElements are consumed.
+          // Checkpoints the remaining elements and residualSource.
+          newCheckpoint = new Checkpoint<>(
+              residualElements.getRestElements(),
+              residualSource == null ? null : residualSource.getSource());
+        } else if (residualSource != null) {
+          newCheckpoint = residualSource.getCheckpointMark();
+        } else {
+          newCheckpoint = new Checkpoint<>(null /* residualElements */, null /* residualSource */);
+        }
+        // Re-initialize since the residualElements and the residualSource might be
+        // consumed or split by checkpointing.
+        init(newCheckpoint.residualElements, newCheckpoint.residualSource, options);
+        return newCheckpoint;
+      }
+
+      @Override
+      public BoundedToUnboundedSourceAdapter<T> getCurrentSource() {
+        return BoundedToUnboundedSourceAdapter.this;
+      }
+    }
+
+    private class ResidualElements {
+      private final List<TimestampedValue<T>> elementsList;
+      private @Nullable Iterator<TimestampedValue<T>> elementsIterator;
+      private @Nullable TimestampedValue<T> currentT;
+      private boolean hasCurrent;
+      private boolean done;
+
+      ResidualElements(List<TimestampedValue<T>> residualElementsList) {
+        this.elementsList = checkNotNull(residualElementsList, "residualElementsList");
+        this.elementsIterator = null;
+        this.currentT = null;
+        this.hasCurrent = false;
+        this.done = false;
+      }
+
+      public boolean advance() {
+        if (elementsIterator == null) {
+          elementsIterator = elementsList.iterator();
+        }
+        if (elementsIterator.hasNext()) {
+          currentT = elementsIterator.next();
+          hasCurrent = true;
+          return true;
+        } else {
+          done = true;
+          hasCurrent = false;
+          return false;
+        }
+      }
+
+      boolean hasCurrent() {
+        return hasCurrent;
+      }
+
+      boolean done() {
+        return done;
+      }
+
+      TimestampedValue<T> getCurrentTimestampedValue() {
+        if (!hasCurrent) {
+          throw new NoSuchElementException();
+        }
+        return currentT;
+      }
+
+      T getCurrent() {
+        return getCurrentTimestampedValue().getValue();
+      }
+
+      Instant getCurrentTimestamp() {
+        return getCurrentTimestampedValue().getTimestamp();
+      }
+
+      List<TimestampedValue<T>> getRestElements() {
+        if (elementsIterator == null) {
+          return elementsList;
+        } else {
+          List<TimestampedValue<T>> newResidualElements = Lists.newArrayList();
+          while (elementsIterator.hasNext()) {
+            newResidualElements.add(elementsIterator.next());
+          }
+          return newResidualElements;
+        }
+      }
+    }
+
+    private class ResidualSource {
+      private BoundedSource<T> residualSource;
+      private PipelineOptions options;
+      private @Nullable BoundedReader<T> reader;
+      private boolean closed;
+
+      public ResidualSource(BoundedSource<T> residualSource, PipelineOptions options) {
+        this.residualSource = checkNotNull(residualSource, "residualSource");
+        this.options = checkNotNull(options, "options");
+        this.reader = null;
+        this.closed = false;
+      }
+
+      private boolean advance() throws IOException {
+        if (reader == null && !closed) {
+          reader = residualSource.createReader(options);
+          return reader.start();
+        } else {
+          return reader.advance();
+        }
+      }
+
+      T getCurrent() throws NoSuchElementException {
+        if (reader == null) {
+          throw new NoSuchElementException();
+        }
+        return reader.getCurrent();
+      }
+
+      Instant getCurrentTimestamp() throws NoSuchElementException {
+        if (reader == null) {
+          throw new NoSuchElementException();
+        }
+        return reader.getCurrentTimestamp();
+      }
+
+      void close() throws IOException {
+        if (reader != null) {
+          reader.close();
+          reader = null;
+        }
+        closed = true;
+      }
+
+      BoundedSource<T> getSource() {
+        return residualSource;
+      }
+
+      Checkpoint<T> getCheckpointMark() {
+        if (reader == null) {
+          // Reader hasn't started, checkpoint the residualSource.
+          return new Checkpoint<>(null /* residualElements */, residualSource);
+        } else {
+          // Part of residualSource are consumed.
+          // Splits the residualSource and tracks the new residualElements in current source.
+          BoundedSource<T> residualSplit = null;
+          Double fractionConsumed = reader.getFractionConsumed();
+          if (fractionConsumed != null && 0 <= fractionConsumed && fractionConsumed <= 1) {
+            double fractionRest = 1 - fractionConsumed;
+            int splitAttempts = 8;
+            for (int i = 0; i < 8 && residualSplit == null; ++i) {
+              double fractionToSplit = fractionConsumed + fractionRest * i / splitAttempts;
+              residualSplit = reader.splitAtFraction(fractionToSplit);
+            }
+          }
+          List<TimestampedValue<T>> newResidualElements = Lists.newArrayList();
+          try {
+            while (advance()) {
+              newResidualElements.add(
+                  TimestampedValue.of(reader.getCurrent(), reader.getCurrentTimestamp()));
+            }
+          } catch (IOException e) {
+            throw new RuntimeException("Failed to read elements from the bounded reader.", e);
+          }
+          return new Checkpoint<>(newResidualElements, residualSplit);
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/56ec6c3e/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java
index 999dc3a..0cf1ade 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java
@@ -56,8 +56,6 @@ import org.apache.beam.sdk.coders.BigEndianLongCoder;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.VarLongCoder;
 import org.apache.beam.sdk.io.AvroIO;
-import org.apache.beam.sdk.io.AvroSource;
-import org.apache.beam.sdk.io.BigQueryIO;
 import org.apache.beam.sdk.io.Read;
 import org.apache.beam.sdk.io.TextIO;
 import org.apache.beam.sdk.options.PipelineOptions;
@@ -898,34 +896,6 @@ public class DataflowRunnerTest {
   }
 
   @Test
-  public void testBoundedSourceUnsupportedInStreaming() throws Exception {
-    testUnsupportedSource(
-        AvroSource.readFromFileWithClass("foo", String.class), "Read.Bounded", true);
-  }
-
-  @Test
-  public void testBigQueryIOSourceUnsupportedInStreaming() throws Exception {
-    testUnsupportedSource(
-        BigQueryIO.Read.from("project:bar.baz").withoutValidation(), "BigQueryIO.Read", true);
-  }
-
-  @Test
-  public void testAvroIOSourceUnsupportedInStreaming() throws Exception {
-    testUnsupportedSource(
-        AvroIO.Read.from("foo"), "AvroIO.Read", true);
-  }
-
-  @Test
-  public void testTextIOSourceUnsupportedInStreaming() throws Exception {
-    testUnsupportedSource(TextIO.Read.from("foo"), "TextIO.Read", true);
-  }
-
-  @Test
-  public void testReadBoundedSourceUnsupportedInStreaming() throws Exception {
-    testUnsupportedSource(Read.from(AvroSource.from("/tmp/test")), "Read.Bounded", true);
-  }
-
-  @Test
   public void testReadUnboundedUnsupportedInBatch() throws Exception {
     testUnsupportedSource(Read.from(new TestCountingSource(1)), "Read.Unbounded", false);
   }


[23/50] [abbrv] incubator-beam git commit: Added getSplitPointsConsumed() to ByteKeyRangeTracker

Posted by lc...@apache.org.
Added getSplitPointsConsumed() to ByteKeyRangeTracker


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

Branch: refs/heads/runners-spark2
Commit: ef9d1955fcdaab28633ea2d081ad342471a23359
Parents: 53c8bff
Author: Ian Zhou <ia...@google.com>
Authored: Tue Jun 21 17:34:20 2016 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Jul 6 10:18:51 2016 -0700

----------------------------------------------------------------------
 .../beam/sdk/io/range/ByteKeyRangeTracker.java  | 63 +++++++++++++++++++-
 .../sdk/io/range/ByteKeyRangeTrackerTest.java   | 58 +++++++++++++++++-
 2 files changed, 118 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ef9d1955/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKeyRangeTracker.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKeyRangeTracker.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKeyRangeTracker.java
index b165924..7dd9a2c 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKeyRangeTracker.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKeyRangeTracker.java
@@ -18,6 +18,9 @@
 package org.apache.beam.sdk.io.range;
 
 import static com.google.common.base.MoreObjects.toStringHelper;
+import static com.google.common.base.Preconditions.checkState;
+
+import org.apache.beam.sdk.io.BoundedSource.BoundedReader;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -38,6 +41,10 @@ public final class ByteKeyRangeTracker implements RangeTracker<ByteKey> {
     return new ByteKeyRangeTracker(range);
   }
 
+  public synchronized boolean isDone() {
+    return done;
+  }
+
   @Override
   public synchronized ByteKey getStartPosition() {
     return range.getStartKey();
@@ -55,13 +62,28 @@ public final class ByteKeyRangeTracker implements RangeTracker<ByteKey> {
 
   @Override
   public synchronized boolean tryReturnRecordAt(boolean isAtSplitPoint, ByteKey recordStart) {
-    if (isAtSplitPoint && !range.containsKey(recordStart)) {
+    if (done) {
       return false;
     }
+
+    checkState(!(position == null && !isAtSplitPoint), "The first record must be at a split point");
+    checkState(!(recordStart.compareTo(range.getStartKey()) < 0),
+        "Trying to return record which is before the start key");
+    checkState(!(position != null && recordStart.compareTo(position) < 0),
+        "Trying to return record which is before the last-returned record");
+
     if (position == null) {
       range = range.withStartKey(recordStart);
     }
     position = recordStart;
+
+    if (isAtSplitPoint) {
+      if (!range.containsKey(recordStart)) {
+        done = true;
+        return false;
+      }
+      ++splitPointsSeen;
+    }
     return true;
   }
 
@@ -107,13 +129,50 @@ public final class ByteKeyRangeTracker implements RangeTracker<ByteKey> {
     return range.estimateFractionForKey(position);
   }
 
+  public synchronized long getSplitPointsConsumed() {
+    if (position == null) {
+      return 0;
+    } else if (isDone()) {
+      return splitPointsSeen;
+    } else {
+      // There is a current split point, and it has not finished processing.
+      checkState(
+          splitPointsSeen > 0,
+          "A started rangeTracker should have seen > 0 split points (is %s)",
+          splitPointsSeen);
+      return splitPointsSeen - 1;
+    }
+  }
+
   ///////////////////////////////////////////////////////////////////////////////
   private ByteKeyRange range;
   @Nullable private ByteKey position;
+  private long splitPointsSeen;
+  private boolean done;
 
   private ByteKeyRangeTracker(ByteKeyRange range) {
     this.range = range;
-    this.position = null;
+    position = null;
+    splitPointsSeen = 0L;
+    done = false;
+  }
+
+  /**
+   * Marks this range tracker as being done. Specifically, this will mark the current split point,
+   * if one exists, as being finished.
+   *
+   * <p>Always returns false, so that it can be used in an implementation of
+   * {@link BoundedReader#start()} or {@link BoundedReader#advance()} as follows:
+   *
+   * <pre> {@code
+   * public boolean start() {
+   *   return startImpl() && rangeTracker.tryReturnRecordAt(isAtSplitPoint, position)
+   *       || rangeTracker.markDone();
+   * }} </pre>
+   */
+  public synchronized boolean markDone() {
+    done = true;
+    return false;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ef9d1955/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeTrackerTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeTrackerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeTrackerTest.java
index 4404414..8deaf44 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeTrackerTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeTrackerTest.java
@@ -21,14 +21,18 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.ExpectedException;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
 
 /** Tests for {@link ByteKeyRangeTracker}. */
 @RunWith(JUnit4.class)
 public class ByteKeyRangeTrackerTest {
+  private static final ByteKey BEFORE_START_KEY = ByteKey.of(0x11);
   private static final ByteKey INITIAL_START_KEY = ByteKey.of(0x12);
+  private static final ByteKey AFTER_START_KEY = ByteKey.of(0x13);
   private static final ByteKey INITIAL_MIDDLE_KEY = ByteKey.of(0x23);
   private static final ByteKey NEW_START_KEY = ByteKey.of(0x14);
   private static final ByteKey NEW_MIDDLE_KEY = ByteKey.of(0x24);
@@ -39,6 +43,8 @@ public class ByteKeyRangeTrackerTest {
   private static final double NEW_RANGE_SIZE = 0x34 - 0x14;
   private static final ByteKeyRange NEW_RANGE = ByteKeyRange.of(NEW_START_KEY, END_KEY);
 
+  @Rule public final ExpectedException expected = ExpectedException.none();
+
   /** Tests for {@link ByteKeyRangeTracker#toString}. */
   @Test
   public void testToString() {
@@ -127,7 +133,30 @@ public class ByteKeyRangeTrackerTest {
 
     assertFalse(tracker.tryReturnRecordAt(true, END_KEY)); // after end
 
-    assertTrue(tracker.tryReturnRecordAt(true, BEFORE_END_KEY)); // still succeeds
+    assertFalse(tracker.tryReturnRecordAt(true, BEFORE_END_KEY)); // false because done
+  }
+
+  @Test
+  public void testTryReturnFirstRecordNotSplitPoint() {
+    ByteKeyRangeTracker tracker = ByteKeyRangeTracker.of(INITIAL_RANGE);
+    expected.expect(IllegalStateException.class);
+    tracker.tryReturnRecordAt(false, INITIAL_START_KEY);
+  }
+
+  @Test
+  public void testTryReturnBeforeStartKey() {
+    ByteKeyRangeTracker tracker = ByteKeyRangeTracker.of(INITIAL_RANGE);
+    expected.expect(IllegalStateException.class);
+    tracker.tryReturnRecordAt(true, BEFORE_START_KEY);
+  }
+
+  @Test
+  public void testTryReturnBeforeLastReturnedRecord() {
+    ByteKeyRangeTracker tracker = ByteKeyRangeTracker.of(INITIAL_RANGE);
+    assertTrue(tracker.tryReturnRecordAt(true, INITIAL_START_KEY));
+    assertTrue(tracker.tryReturnRecordAt(true, INITIAL_MIDDLE_KEY));
+    expected.expect(IllegalStateException.class);
+    tracker.tryReturnRecordAt(true, AFTER_START_KEY);
   }
 
   /** Tests for {@link ByteKeyRangeTracker#trySplitAtPosition}. */
@@ -151,4 +180,31 @@ public class ByteKeyRangeTrackerTest {
     assertFalse(tracker.trySplitAtPosition(INITIAL_MIDDLE_KEY));
     assertTrue(tracker.tryReturnRecordAt(true, INITIAL_MIDDLE_KEY));
   }
+
+  /** Tests for {@link ByteKeyRangeTracker#getSplitPointsConsumed()}. */
+  @Test
+  public void testGetSplitPointsConsumed() {
+    ByteKeyRangeTracker tracker = ByteKeyRangeTracker.of(INITIAL_RANGE);
+    assertEquals(0, tracker.getSplitPointsConsumed());
+
+    // Started, 0 split points consumed
+    assertTrue(tracker.tryReturnRecordAt(true, INITIAL_START_KEY));
+    assertEquals(0, tracker.getSplitPointsConsumed());
+
+    // Processing new split point, 1 split point consumed
+    assertTrue(tracker.tryReturnRecordAt(true, AFTER_START_KEY));
+    assertEquals(1, tracker.getSplitPointsConsumed());
+
+    // Processing new non-split point, 1 split point consumed
+    assertTrue(tracker.tryReturnRecordAt(false, INITIAL_MIDDLE_KEY));
+    assertEquals(1, tracker.getSplitPointsConsumed());
+
+    // Processing new split point, 2 split points consumed
+    assertTrue(tracker.tryReturnRecordAt(true, BEFORE_END_KEY));
+    assertEquals(2, tracker.getSplitPointsConsumed());
+
+    // Mark tracker as done, 3 split points consumed
+    tracker.markDone();
+    assertEquals(3, tracker.getSplitPointsConsumed());
+  }
 }


[06/50] [abbrv] incubator-beam git commit: Remove TriggerBuilder backwards-compatibility adapter

Posted by lc...@apache.org.
Remove TriggerBuilder backwards-compatibility adapter


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

Branch: refs/heads/runners-spark2
Commit: 62fe4ef1ea1c8f968e8ffb5d87646562fb63f7e8
Parents: 0d4ee07
Author: Kenneth Knowles <kl...@google.com>
Authored: Thu Jun 23 21:03:29 2016 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Jul 6 10:18:49 2016 -0700

----------------------------------------------------------------------
 .../beam/sdk/util/ReduceFnRunnerTest.java       |  3 +-
 .../apache/beam/sdk/util/ReduceFnTester.java    |  5 ++-
 .../flink/streaming/GroupAlsoByWindowTest.java  |  2 +-
 .../transforms/windowing/AfterWatermark.java    | 38 ++++----------------
 .../beam/sdk/transforms/windowing/Trigger.java  |  7 +---
 .../transforms/windowing/TriggerBuilder.java    | 29 ---------------
 .../beam/sdk/transforms/windowing/Window.java   | 10 +++---
 .../windowing/AfterProcessingTimeTest.java      |  3 +-
 .../windowing/AfterWatermarkTest.java           | 24 ++++++-------
 .../org/apache/beam/sdk/util/TriggerTester.java | 11 +++---
 10 files changed, 33 insertions(+), 99 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/62fe4ef1/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnRunnerTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnRunnerTest.java
index 64fcae3..cd78107 100644
--- a/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnRunnerTest.java
+++ b/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnRunnerTest.java
@@ -111,7 +111,6 @@ public class ReduceFnRunnerTest {
     MockitoAnnotations.initMocks(this);
 
     mockTrigger = mock(Trigger.class, withSettings().serializable());
-    when(mockTrigger.buildTrigger()).thenReturn(mockTrigger);
 
     @SuppressWarnings("unchecked")
     PCollectionView<Integer> mockViewUnchecked =
@@ -271,7 +270,7 @@ public class ReduceFnRunnerTest {
     ReduceFnTester<Integer, Integer, IntervalWindow> tester =
         ReduceFnTester.combining(
             windowFn,
-            AfterWatermark.pastEndOfWindow().withLateFirings(Never.ever()).buildTrigger(),
+            AfterWatermark.pastEndOfWindow().withLateFirings(Never.ever()),
             AccumulationMode.DISCARDING_FIRED_PANES,
             new Sum.SumIntegerFn().<String>asKeyedFn(),
             VarIntCoder.of(),

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/62fe4ef1/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnTester.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnTester.java b/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnTester.java
index e897f54..fa62583 100644
--- a/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnTester.java
+++ b/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnTester.java
@@ -39,7 +39,6 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
 import org.apache.beam.sdk.transforms.windowing.OutputTimeFns;
 import org.apache.beam.sdk.transforms.windowing.PaneInfo;
 import org.apache.beam.sdk.transforms.windowing.Trigger;
-import org.apache.beam.sdk.transforms.windowing.TriggerBuilder;
 import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior;
 import org.apache.beam.sdk.transforms.windowing.WindowFn;
 import org.apache.beam.sdk.util.TimerInternals.TimerData;
@@ -131,12 +130,12 @@ public class ReduceFnTester<InputT, OutputT, W extends BoundedWindow> {
   }
 
   public static <W extends BoundedWindow> ReduceFnTester<Integer, Iterable<Integer>, W>
-      nonCombining(WindowFn<?, W> windowFn, TriggerBuilder trigger, AccumulationMode mode,
+      nonCombining(WindowFn<?, W> windowFn, Trigger trigger, AccumulationMode mode,
           Duration allowedDataLateness, ClosingBehavior closingBehavior) throws Exception {
     WindowingStrategy<?, W> strategy =
         WindowingStrategy.of(windowFn)
             .withOutputTimeFn(OutputTimeFns.outputAtEarliestInputTimestamp())
-            .withTrigger(trigger.buildTrigger())
+            .withTrigger(trigger)
             .withMode(mode)
             .withAllowedLateness(allowedDataLateness)
             .withClosingBehavior(closingBehavior);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/62fe4ef1/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/GroupAlsoByWindowTest.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/GroupAlsoByWindowTest.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/GroupAlsoByWindowTest.java
index 207fb5a..2d83fb6 100644
--- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/GroupAlsoByWindowTest.java
+++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/GroupAlsoByWindowTest.java
@@ -83,7 +83,7 @@ public class GroupAlsoByWindowTest extends StreamingMultipleProgramsTestBase {
   private final WindowingStrategy fixedWindowWithCompoundTriggerStrategy =
     fixedWindowingStrategy.withTrigger(
       AfterWatermark.pastEndOfWindow().withEarlyFirings(AfterPane.elementCountAtLeast(5))
-        .withLateFirings(AfterPane.elementCountAtLeast(5)).buildTrigger());
+        .withLateFirings(AfterPane.elementCountAtLeast(5)));
 
   /**
    * The default accumulation mode is

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/62fe4ef1/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterWatermark.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterWatermark.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterWatermark.java
index 019a68d..0d2a878 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterWatermark.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterWatermark.java
@@ -78,31 +78,9 @@ public class AfterWatermark {
   }
 
   /**
-   * Interface for building an AfterWatermarkTrigger with early firings already filled in.
+   * @see AfterWatermark
    */
-  public interface AfterWatermarkEarly extends TriggerBuilder {
-    /**
-     * Creates a new {@code Trigger} like the this, except that it fires repeatedly whenever
-     * the given {@code Trigger} fires after the watermark has passed the end of the window.
-     */
-    TriggerBuilder withLateFirings(OnceTrigger lateTrigger);
-  }
-
-  /**
-   * Interface for building an AfterWatermarkTrigger with late firings already filled in.
-   */
-  public interface AfterWatermarkLate extends TriggerBuilder {
-    /**
-     * Creates a new {@code Trigger} like the this, except that it fires repeatedly whenever
-     * the given {@code Trigger} fires before the watermark has passed the end of the window.
-     */
-    TriggerBuilder withEarlyFirings(OnceTrigger earlyTrigger);
-  }
-
-
-  private static class AfterWatermarkEarlyAndLate
-      extends Trigger
-      implements TriggerBuilder, AfterWatermarkEarly, AfterWatermarkLate {
+  public static class AfterWatermarkEarlyAndLate extends Trigger {
 
     private static final int EARLY_INDEX = 0;
     private static final int LATE_INDEX = 1;
@@ -112,7 +90,7 @@ public class AfterWatermark {
     private final OnceTrigger lateTrigger;
 
     @SuppressWarnings("unchecked")
-    private AfterWatermarkEarlyAndLate(OnceTrigger earlyTrigger, OnceTrigger lateTrigger) {
+    public AfterWatermarkEarlyAndLate(OnceTrigger earlyTrigger, OnceTrigger lateTrigger) {
       super(lateTrigger == null
           ? ImmutableList.<Trigger>of(earlyTrigger)
           : ImmutableList.<Trigger>of(earlyTrigger, lateTrigger));
@@ -120,13 +98,11 @@ public class AfterWatermark {
       this.lateTrigger = lateTrigger;
     }
 
-    @Override
-    public TriggerBuilder withEarlyFirings(OnceTrigger earlyTrigger) {
+    public Trigger withEarlyFirings(OnceTrigger earlyTrigger) {
       return new AfterWatermarkEarlyAndLate(earlyTrigger, lateTrigger);
     }
 
-    @Override
-    public TriggerBuilder withLateFirings(OnceTrigger lateTrigger) {
+    public Trigger withLateFirings(OnceTrigger lateTrigger) {
       return new AfterWatermarkEarlyAndLate(earlyTrigger, lateTrigger);
     }
 
@@ -301,7 +277,7 @@ public class AfterWatermark {
      * Creates a new {@code Trigger} like the this, except that it fires repeatedly whenever
      * the given {@code Trigger} fires before the watermark has passed the end of the window.
      */
-    public AfterWatermarkEarly withEarlyFirings(OnceTrigger earlyFirings) {
+    public AfterWatermarkEarlyAndLate withEarlyFirings(OnceTrigger earlyFirings) {
       checkNotNull(earlyFirings, "Must specify the trigger to use for early firings");
       return new AfterWatermarkEarlyAndLate(earlyFirings, null);
     }
@@ -310,7 +286,7 @@ public class AfterWatermark {
      * Creates a new {@code Trigger} like the this, except that it fires repeatedly whenever
      * the given {@code Trigger} fires after the watermark has passed the end of the window.
      */
-    public AfterWatermarkLate withLateFirings(OnceTrigger lateFirings) {
+    public AfterWatermarkEarlyAndLate withLateFirings(OnceTrigger lateFirings) {
       checkNotNull(lateFirings, "Must specify the trigger to use for late firings");
       return new AfterWatermarkEarlyAndLate(Never.ever(), lateFirings);
     }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/62fe4ef1/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Trigger.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Trigger.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Trigger.java
index e97d3bd..86801e8 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Trigger.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Trigger.java
@@ -93,7 +93,7 @@ import javax.annotation.Nullable;
  * state before the callback returns.
  */
 @Experimental(Experimental.Kind.TRIGGER)
-public abstract class Trigger implements Serializable, TriggerBuilder {
+public abstract class Trigger implements Serializable {
 
   /**
    * Interface for accessing information about the trigger being executed and other triggers in the
@@ -495,11 +495,6 @@ public abstract class Trigger implements Serializable, TriggerBuilder {
     return new OrFinallyTrigger(this, until);
   }
 
-  @Override
-  public Trigger buildTrigger() {
-    return this;
-  }
-
   /**
    * {@link Trigger}s that are guaranteed to fire at most once should extend from this, rather
    * than the general {@link Trigger} class to indicate that behavior.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/62fe4ef1/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/TriggerBuilder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/TriggerBuilder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/TriggerBuilder.java
deleted file mode 100644
index e8bd52b..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/TriggerBuilder.java
+++ /dev/null
@@ -1,29 +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.transforms.windowing;
-
-/**
- * Anything that can be used to create an instance of a {@code Trigger} implements this interface.
- *
- * <p>This includes {@code Trigger}s (which can return themselves) and any "enhanced" syntax for
- * constructing a trigger.
- */
-public interface TriggerBuilder {
-  /** Return the {@code Trigger} built by this builder. */
-  Trigger buildTrigger();
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/62fe4ef1/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java
index 7d790d4..dde5c05 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java
@@ -87,7 +87,7 @@ import javax.annotation.Nullable;
  *
  * <h2> Triggers </h2>
  *
- * <p>{@link Window.Bound#triggering(TriggerBuilder)} allows specifying a trigger to control when
+ * <p>{@link Window.Bound#triggering(Trigger)} allows specifying a trigger to control when
  * (in processing time) results for the given window can be produced. If unspecified, the default
  * behavior is to trigger first when the watermark passes the end of the window, and then trigger
  * again every time there is late arriving data.
@@ -195,7 +195,7 @@ public class Window {
    * mode using either {@link #discardingFiredPanes()} or {@link #accumulatingFiredPanes()}.
    */
   @Experimental(Kind.TRIGGER)
-  public static <T> Bound<T> triggering(TriggerBuilder trigger) {
+  public static <T> Bound<T> triggering(Trigger trigger) {
     return new Unbound().triggering(trigger);
   }
 
@@ -290,7 +290,7 @@ public class Window {
      * mode using either {@link #discardingFiredPanes()} or {@link #accumulatingFiredPanes()}.
      */
     @Experimental(Kind.TRIGGER)
-    public <T> Bound<T> triggering(TriggerBuilder trigger) {
+    public <T> Bound<T> triggering(Trigger trigger) {
       return new Bound<T>(name).triggering(trigger);
     }
 
@@ -433,11 +433,11 @@ public class Window {
      * mode using either {@link #discardingFiredPanes()} or {@link #accumulatingFiredPanes()}.
      */
     @Experimental(Kind.TRIGGER)
-    public Bound<T> triggering(TriggerBuilder trigger) {
+    public Bound<T> triggering(Trigger trigger) {
       return new Bound<T>(
           name,
           windowFn,
-          trigger.buildTrigger(),
+          trigger,
           mode,
           allowedLateness,
           closingBehavior,

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/62fe4ef1/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTimeTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTimeTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTimeTest.java
index 8d2b4a1..ea9c2b0 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTimeTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTimeTest.java
@@ -176,8 +176,7 @@ public class AfterProcessingTimeTest {
     Trigger trigger = AfterWatermark.pastEndOfWindow()
         .withLateFirings(AfterProcessingTime
             .pastFirstElementInPane()
-            .plusDelayOf(Duration.standardMinutes(10)))
-        .buildTrigger();
+            .plusDelayOf(Duration.standardMinutes(10)));
 
     String expected = "AfterWatermark.pastEndOfWindow()"
         + ".withLateFirings(AfterProcessingTime"

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/62fe4ef1/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterWatermarkTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterWatermarkTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterWatermarkTest.java
index d692cbf..418f746 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterWatermarkTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterWatermarkTest.java
@@ -346,28 +346,24 @@ public class AfterWatermarkTest {
 
   @Test
   public void testEarlyFiringsToString() {
-    Trigger trigger = AfterWatermark.pastEndOfWindow()
-        .withEarlyFirings(StubTrigger.named("t1"))
-        .buildTrigger();
+    Trigger trigger = AfterWatermark.pastEndOfWindow().withEarlyFirings(StubTrigger.named("t1"));
 
     assertEquals("AfterWatermark.pastEndOfWindow().withEarlyFirings(t1)", trigger.toString());
   }
 
   @Test
   public void testLateFiringsToString() {
-    Trigger trigger = AfterWatermark.pastEndOfWindow()
-        .withLateFirings(StubTrigger.named("t1"))
-        .buildTrigger();
+    Trigger trigger = AfterWatermark.pastEndOfWindow().withLateFirings(StubTrigger.named("t1"));
 
     assertEquals("AfterWatermark.pastEndOfWindow().withLateFirings(t1)", trigger.toString());
   }
 
   @Test
   public void testEarlyAndLateFiringsToString() {
-    Trigger trigger = AfterWatermark.pastEndOfWindow()
-        .withEarlyFirings(StubTrigger.named("t1"))
-        .withLateFirings(StubTrigger.named("t2"))
-        .buildTrigger();
+    Trigger trigger =
+        AfterWatermark.pastEndOfWindow()
+            .withEarlyFirings(StubTrigger.named("t1"))
+            .withLateFirings(StubTrigger.named("t2"));
 
     assertEquals("AfterWatermark.pastEndOfWindow().withEarlyFirings(t1).withLateFirings(t2)",
         trigger.toString());
@@ -375,10 +371,10 @@ public class AfterWatermarkTest {
 
   @Test
   public void testToStringExcludesNeverTrigger() {
-    Trigger trigger = AfterWatermark.pastEndOfWindow()
-        .withEarlyFirings(Never.ever())
-        .withLateFirings(Never.ever())
-        .buildTrigger();
+    Trigger trigger =
+        AfterWatermark.pastEndOfWindow()
+            .withEarlyFirings(Never.ever())
+            .withLateFirings(Never.ever());
 
     assertEquals("AfterWatermark.pastEndOfWindow()", trigger.toString());
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/62fe4ef1/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TriggerTester.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TriggerTester.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TriggerTester.java
index c495712..ba42c37 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TriggerTester.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TriggerTester.java
@@ -26,7 +26,6 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
 import org.apache.beam.sdk.transforms.windowing.PaneInfo;
 import org.apache.beam.sdk.transforms.windowing.Trigger;
-import org.apache.beam.sdk.transforms.windowing.TriggerBuilder;
 import org.apache.beam.sdk.transforms.windowing.WindowFn;
 import org.apache.beam.sdk.util.ActiveWindowSet.MergeCallback;
 import org.apache.beam.sdk.util.TimerInternals.TimerData;
@@ -108,7 +107,7 @@ public class TriggerTester<InputT, W extends BoundedWindow> {
   private final Map<W, W> windowToMergeResult;
 
   /**
-   * An {@link ExecutableTrigger} built from the {@link Trigger} or {@link TriggerBuilder}
+   * An {@link ExecutableTrigger} built from the {@link Trigger} or {@link Trigger}
    * under test.
    */
   private final ExecutableTrigger executableTrigger;
@@ -119,10 +118,10 @@ public class TriggerTester<InputT, W extends BoundedWindow> {
   private final Map<W, FinishedTriggers> finishedSets;
 
   public static <W extends BoundedWindow> SimpleTriggerTester<W> forTrigger(
-      TriggerBuilder trigger, WindowFn<Object, W> windowFn)
+      Trigger trigger, WindowFn<Object, W> windowFn)
           throws Exception {
     WindowingStrategy<Object, W> windowingStrategy =
-        WindowingStrategy.of(windowFn).withTrigger(trigger.buildTrigger())
+        WindowingStrategy.of(windowFn).withTrigger(trigger)
         // Merging requires accumulation mode or early firings can break up a session.
         // Not currently an issue with the tester (because we never GC) but we don't want
         // mystery failures due to violating this need.
@@ -134,9 +133,9 @@ public class TriggerTester<InputT, W extends BoundedWindow> {
   }
 
   public static <InputT, W extends BoundedWindow> TriggerTester<InputT, W> forAdvancedTrigger(
-      TriggerBuilder trigger, WindowFn<Object, W> windowFn) throws Exception {
+      Trigger trigger, WindowFn<Object, W> windowFn) throws Exception {
     WindowingStrategy<Object, W> strategy =
-        WindowingStrategy.of(windowFn).withTrigger(trigger.buildTrigger())
+        WindowingStrategy.of(windowFn).withTrigger(trigger)
         // Merging requires accumulation mode or early firings can break up a session.
         // Not currently an issue with the tester (because we never GC) but we don't want
         // mystery failures due to violating this need.


[16/50] [abbrv] incubator-beam git commit: Explicitly set UseDummyRunner in IO, Extensions

Posted by lc...@apache.org.
Explicitly set UseDummyRunner in IO, Extensions

This mitigates test failures due to forkCount=0 in travis.


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

Branch: refs/heads/runners-spark2
Commit: 942b6e02769f34d30fd8e5a96568d14ff6386a9c
Parents: 8b8615e
Author: Thomas Groh <tg...@google.com>
Authored: Mon Jun 27 15:11:58 2016 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Jul 6 10:18:50 2016 -0700

----------------------------------------------------------------------
 runners/direct-java/pom.xml                |  3 +--
 sdks/java/extensions/join-library/pom.xml  |  9 +++++++++
 sdks/java/io/google-cloud-platform/pom.xml |  9 +++++++++
 sdks/java/io/hdfs/pom.xml                  |  9 +++++++++
 sdks/java/io/kafka/pom.xml                 |  9 +++++++++
 sdks/java/java8tests/pom.xml               | 10 ++++++++++
 6 files changed, 47 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/942b6e02/runners/direct-java/pom.xml
----------------------------------------------------------------------
diff --git a/runners/direct-java/pom.xml b/runners/direct-java/pom.xml
index 13e8951..82c1e15 100644
--- a/runners/direct-java/pom.xml
+++ b/runners/direct-java/pom.xml
@@ -67,8 +67,7 @@
         <artifactId>maven-surefire-plugin</artifactId>
         <configuration>
           <systemPropertyVariables>
-            <!-- Use a dummy runner for component tests -->
-            <beamUseDummyRunner>true</beamUseDummyRunner>
+            <beamUseDummyRunner>false</beamUseDummyRunner>
           </systemPropertyVariables>
         </configuration>
         <executions>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/942b6e02/sdks/java/extensions/join-library/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/join-library/pom.xml b/sdks/java/extensions/join-library/pom.xml
index 0fec148..0dec32d 100644
--- a/sdks/java/extensions/join-library/pom.xml
+++ b/sdks/java/extensions/join-library/pom.xml
@@ -37,6 +37,15 @@
       </plugin>
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-plugin</artifactId>
+        <configuration>
+          <systemPropertyVariables>
+            <beamUseDummyRunner>false</beamUseDummyRunner>
+          </systemPropertyVariables>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-checkstyle-plugin</artifactId>
       </plugin>
       <plugin>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/942b6e02/sdks/java/io/google-cloud-platform/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/pom.xml b/sdks/java/io/google-cloud-platform/pom.xml
index bb5fd11..692ac96 100644
--- a/sdks/java/io/google-cloud-platform/pom.xml
+++ b/sdks/java/io/google-cloud-platform/pom.xml
@@ -43,6 +43,15 @@
       </plugin>
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-plugin</artifactId>
+        <configuration>
+          <systemPropertyVariables>
+            <beamUseDummyRunner>false</beamUseDummyRunner>
+          </systemPropertyVariables>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-source-plugin</artifactId>
       </plugin>
       <plugin>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/942b6e02/sdks/java/io/hdfs/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/io/hdfs/pom.xml b/sdks/java/io/hdfs/pom.xml
index 9f4b38a..f2c9f68 100644
--- a/sdks/java/io/hdfs/pom.xml
+++ b/sdks/java/io/hdfs/pom.xml
@@ -38,6 +38,15 @@
       </plugin>
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-plugin</artifactId>
+        <configuration>
+          <systemPropertyVariables>
+            <beamUseDummyRunner>false</beamUseDummyRunner>
+          </systemPropertyVariables>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-source-plugin</artifactId>
       </plugin>
       <plugin>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/942b6e02/sdks/java/io/kafka/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/io/kafka/pom.xml b/sdks/java/io/kafka/pom.xml
index 578026b..cc447cd 100644
--- a/sdks/java/io/kafka/pom.xml
+++ b/sdks/java/io/kafka/pom.xml
@@ -41,6 +41,15 @@
       </plugin>
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-plugin</artifactId>
+        <configuration>
+          <systemPropertyVariables>
+            <beamUseDummyRunner>false</beamUseDummyRunner>
+          </systemPropertyVariables>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-jar-plugin</artifactId>
       </plugin>
       <plugin>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/942b6e02/sdks/java/java8tests/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/java8tests/pom.xml b/sdks/java/java8tests/pom.xml
index 775a7c7..61606cb 100644
--- a/sdks/java/java8tests/pom.xml
+++ b/sdks/java/java8tests/pom.xml
@@ -53,6 +53,16 @@
 
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-plugin</artifactId>
+        <configuration>
+          <systemPropertyVariables>
+            <beamUseDummyRunner>false</beamUseDummyRunner>
+          </systemPropertyVariables>
+        </configuration>
+      </plugin>
+
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-checkstyle-plugin</artifactId>
       </plugin>
 


[26/50] [abbrv] incubator-beam git commit: Write: add support for setting a fixed number of shards

Posted by lc...@apache.org.
Write: add support for setting a fixed number of shards

And remove special support in Dataflow and Direct runners for it.


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

Branch: refs/heads/runners-spark2
Commit: da1dcf9b8216dc144e1958d13534c6f72f6756a1
Parents: 722bfca
Author: Dan Halperin <dh...@google.com>
Authored: Tue Jun 14 14:03:41 2016 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Jul 6 10:18:51 2016 -0700

----------------------------------------------------------------------
 .../direct/AvroIOShardedWriteFactory.java       |  76 -----
 .../beam/runners/direct/DirectRunner.java       |   4 -
 .../runners/direct/ShardControlledWrite.java    |  81 -----
 .../direct/TextIOShardedWriteFactory.java       |  78 -----
 .../direct/AvroIOShardedWriteFactoryTest.java   | 120 -------
 .../direct/TextIOShardedWriteFactoryTest.java   | 120 -------
 .../beam/runners/dataflow/DataflowRunner.java   | 258 ---------------
 .../java/org/apache/beam/sdk/io/AvroIO.java     |  12 +-
 .../java/org/apache/beam/sdk/io/TextIO.java     |  15 +-
 .../main/java/org/apache/beam/sdk/io/Write.java | 314 ++++++++++++++-----
 .../java/org/apache/beam/sdk/io/WriteTest.java  | 145 ++++++++-
 11 files changed, 391 insertions(+), 832 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/da1dcf9b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/AvroIOShardedWriteFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/AvroIOShardedWriteFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/AvroIOShardedWriteFactory.java
deleted file mode 100644
index 7422f27..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/AvroIOShardedWriteFactory.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.runners.direct;
-
-import org.apache.beam.sdk.io.AvroIO;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.util.IOChannelUtils;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PDone;
-import org.apache.beam.sdk.values.PInput;
-import org.apache.beam.sdk.values.POutput;
-
-class AvroIOShardedWriteFactory implements PTransformOverrideFactory {
-  @Override
-  public <InputT extends PInput, OutputT extends POutput> PTransform<InputT, OutputT> override(
-      PTransform<InputT, OutputT> transform) {
-    if (transform instanceof AvroIO.Write.Bound) {
-      @SuppressWarnings("unchecked")
-      AvroIO.Write.Bound<InputT> originalWrite = (AvroIO.Write.Bound<InputT>) transform;
-      if (originalWrite.getNumShards() > 1
-          || (originalWrite.getNumShards() == 1
-              && !"".equals(originalWrite.getShardNameTemplate()))) {
-        @SuppressWarnings("unchecked")
-        PTransform<InputT, OutputT> override =
-            (PTransform<InputT, OutputT>) new AvroIOShardedWrite<InputT>(originalWrite);
-        return override;
-      }
-    }
-    return transform;
-  }
-
-  private class AvroIOShardedWrite<InputT> extends ShardControlledWrite<InputT> {
-    private final AvroIO.Write.Bound<InputT> initial;
-
-    private AvroIOShardedWrite(AvroIO.Write.Bound<InputT> initial) {
-      this.initial = initial;
-    }
-
-    @Override
-    int getNumShards() {
-      return initial.getNumShards();
-    }
-
-    @Override
-    PTransform<? super PCollection<InputT>, PDone> getSingleShardTransform(int shardNum) {
-      String shardName =
-          IOChannelUtils.constructName(
-              initial.getFilenamePrefix(),
-              initial.getShardNameTemplate(),
-              initial.getFilenameSuffix(),
-              shardNum,
-              getNumShards());
-      return initial.withoutSharding().to(shardName).withSuffix("");
-    }
-
-    @Override
-    protected PTransform<PCollection<InputT>, PDone> delegate() {
-      return initial;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/da1dcf9b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
index 2584739..7408c0b 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
@@ -24,8 +24,6 @@ import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.Pipeline.PipelineExecutionException;
 import org.apache.beam.sdk.PipelineResult;
 import org.apache.beam.sdk.annotations.Experimental;
-import org.apache.beam.sdk.io.AvroIO;
-import org.apache.beam.sdk.io.TextIO;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.runners.AggregatorPipelineExtractor;
 import org.apache.beam.sdk.runners.AggregatorRetrievalException;
@@ -80,8 +78,6 @@ public class DirectRunner
           ImmutableMap.<Class<? extends PTransform>, PTransformOverrideFactory>builder()
               .put(GroupByKey.class, new DirectGroupByKeyOverrideFactory())
               .put(CreatePCollectionView.class, new ViewOverrideFactory())
-              .put(AvroIO.Write.Bound.class, new AvroIOShardedWriteFactory())
-              .put(TextIO.Write.Bound.class, new TextIOShardedWriteFactory())
               .build();
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/da1dcf9b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ShardControlledWrite.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ShardControlledWrite.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ShardControlledWrite.java
deleted file mode 100644
index 4687f85..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ShardControlledWrite.java
+++ /dev/null
@@ -1,81 +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.runners.direct;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.transforms.Partition;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PCollectionList;
-import org.apache.beam.sdk.values.PDone;
-
-import java.util.concurrent.ThreadLocalRandom;
-
-/**
- * A write that explicitly controls its number of output shards.
- */
-abstract class ShardControlledWrite<InputT>
-    extends ForwardingPTransform<PCollection<InputT>, PDone> {
-  @Override
-  public PDone apply(PCollection<InputT> input) {
-    int numShards = getNumShards();
-    checkArgument(
-        numShards >= 1,
-        "%s should only be applied if the output has a controlled number of shards (> 1); got %s",
-        getClass().getSimpleName(),
-        getNumShards());
-    PCollectionList<InputT> shards =
-        input.apply(
-            "PartitionInto" + numShards + "Shards",
-            Partition.of(getNumShards(), new RandomSeedPartitionFn<InputT>()));
-    for (int i = 0; i < shards.size(); i++) {
-      PCollection<InputT> shard = shards.get(i);
-      PTransform<? super PCollection<InputT>, PDone> writeShard = getSingleShardTransform(i);
-      shard.apply(String.format("%s(Shard:%s)", writeShard.getName(), i), writeShard);
-    }
-    return PDone.in(input.getPipeline());
-  }
-
-  /**
-   * Returns the number of shards this {@link PTransform} should write to.
-   */
-  abstract int getNumShards();
-
-  /**
-   * Returns a {@link PTransform} that performs a write to the shard with the specified shard
-   * number.
-   *
-   * <p>This method will be called n times, where n is the value of {@link #getNumShards()}, for
-   * shard numbers {@code [0...n)}.
-   */
-  abstract PTransform<? super PCollection<InputT>, PDone> getSingleShardTransform(int shardNum);
-
-  private static class RandomSeedPartitionFn<T> implements Partition.PartitionFn<T> {
-    int nextPartition = -1;
-    @Override
-    public int partitionFor(T elem, int numPartitions) {
-      if (nextPartition < 0) {
-        nextPartition = ThreadLocalRandom.current().nextInt(numPartitions);
-      }
-      nextPartition++;
-      nextPartition %= numPartitions;
-      return nextPartition;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/da1dcf9b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TextIOShardedWriteFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TextIOShardedWriteFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TextIOShardedWriteFactory.java
deleted file mode 100644
index be1bf18..0000000
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TextIOShardedWriteFactory.java
+++ /dev/null
@@ -1,78 +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.runners.direct;
-
-import org.apache.beam.sdk.io.TextIO;
-import org.apache.beam.sdk.io.TextIO.Write.Bound;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.util.IOChannelUtils;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PDone;
-import org.apache.beam.sdk.values.PInput;
-import org.apache.beam.sdk.values.POutput;
-
-class TextIOShardedWriteFactory implements PTransformOverrideFactory {
-
-  @Override
-  public <InputT extends PInput, OutputT extends POutput> PTransform<InputT, OutputT> override(
-      PTransform<InputT, OutputT> transform) {
-    if (transform instanceof TextIO.Write.Bound) {
-      @SuppressWarnings("unchecked")
-      TextIO.Write.Bound<InputT> originalWrite = (TextIO.Write.Bound<InputT>) transform;
-      if (originalWrite.getNumShards() > 1
-          || (originalWrite.getNumShards() == 1
-              && !"".equals(originalWrite.getShardNameTemplate()))) {
-        @SuppressWarnings("unchecked")
-        PTransform<InputT, OutputT> override =
-            (PTransform<InputT, OutputT>) new TextIOShardedWrite<InputT>(originalWrite);
-        return override;
-      }
-    }
-    return transform;
-  }
-
-  private static class TextIOShardedWrite<InputT> extends ShardControlledWrite<InputT> {
-    private final TextIO.Write.Bound<InputT> initial;
-
-    private TextIOShardedWrite(Bound<InputT> initial) {
-      this.initial = initial;
-    }
-
-    @Override
-    int getNumShards() {
-      return initial.getNumShards();
-    }
-
-    @Override
-    PTransform<PCollection<InputT>, PDone> getSingleShardTransform(int shardNum) {
-      String shardName =
-          IOChannelUtils.constructName(
-              initial.getFilenamePrefix(),
-              initial.getShardTemplate(),
-              initial.getFilenameSuffix(),
-              shardNum,
-              getNumShards());
-      return TextIO.Write.withCoder(initial.getCoder()).to(shardName).withoutSharding();
-    }
-
-    @Override
-    protected PTransform<PCollection<InputT>, PDone> delegate() {
-      return initial;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/da1dcf9b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/AvroIOShardedWriteFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/AvroIOShardedWriteFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/AvroIOShardedWriteFactoryTest.java
deleted file mode 100644
index d94113a..0000000
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/AvroIOShardedWriteFactoryTest.java
+++ /dev/null
@@ -1,120 +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.runners.direct;
-
-import static org.hamcrest.Matchers.not;
-import static org.hamcrest.Matchers.theInstance;
-import static org.junit.Assert.assertThat;
-
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.io.AvroIO;
-import org.apache.beam.sdk.io.AvroIOTest;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PDone;
-
-import org.hamcrest.Matchers;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-import java.io.File;
-
-/**
- * Tests for {@link AvroIOShardedWriteFactory}.
- */
-@RunWith(JUnit4.class)
-public class AvroIOShardedWriteFactoryTest {
-
-  @Rule public TemporaryFolder tmp = new TemporaryFolder();
-  private AvroIOShardedWriteFactory factory;
-
-  @Before
-  public void setup() {
-    factory = new AvroIOShardedWriteFactory();
-  }
-
-  @Test
-  public void originalWithoutShardingReturnsOriginal() throws Exception {
-    File file = tmp.newFile("foo");
-    PTransform<PCollection<String>, PDone> original =
-        AvroIO.Write.withSchema(String.class).to(file.getAbsolutePath()).withoutSharding();
-    PTransform<PCollection<String>, PDone> overridden = factory.override(original);
-
-    assertThat(overridden, theInstance(original));
-  }
-
-  @Test
-  public void originalShardingNotSpecifiedReturnsOriginal() throws Exception {
-    File file = tmp.newFile("foo");
-    PTransform<PCollection<String>, PDone> original =
-        AvroIO.Write.withSchema(String.class).to(file.getAbsolutePath());
-    PTransform<PCollection<String>, PDone> overridden = factory.override(original);
-
-    assertThat(overridden, theInstance(original));
-  }
-
-  @Test
-  public void originalShardedToOneReturnsExplicitlySharded() throws Exception {
-    File file = tmp.newFile("foo");
-    AvroIO.Write.Bound<String> original =
-        AvroIO.Write.withSchema(String.class).to(file.getAbsolutePath()).withNumShards(1);
-    PTransform<PCollection<String>, PDone> overridden = factory.override(original);
-
-    assertThat(overridden, not(Matchers.<PTransform<PCollection<String>, PDone>>equalTo(original)));
-
-    Pipeline p = getPipeline();
-    String[] elems = new String[] {"foo", "bar", "baz"};
-    p.apply(Create.<String>of(elems)).apply(overridden);
-
-    file.delete();
-
-    p.run();
-    AvroIOTest.assertTestOutputs(elems, 1, file.getAbsolutePath(), original.getShardNameTemplate());
-  }
-
-  @Test
-  public void originalShardedToManyReturnsExplicitlySharded() throws Exception {
-    File file = tmp.newFile("foo");
-    AvroIO.Write.Bound<String> original =
-        AvroIO.Write.withSchema(String.class).to(file.getAbsolutePath()).withNumShards(3);
-    PTransform<PCollection<String>, PDone> overridden = factory.override(original);
-
-    assertThat(overridden, not(Matchers.<PTransform<PCollection<String>, PDone>>equalTo(original)));
-
-    Pipeline p = getPipeline();
-    String[] elems = new String[] {"foo", "bar", "baz", "spam", "ham", "eggs"};
-    p.apply(Create.<String>of(elems)).apply(overridden);
-
-    file.delete();
-    p.run();
-    AvroIOTest.assertTestOutputs(elems, 3, file.getAbsolutePath(), original.getShardNameTemplate());
-  }
-
-  private Pipeline getPipeline() {
-    PipelineOptions options = TestPipeline.testingPipelineOptions();
-    options.setRunner(DirectRunner.class);
-    return TestPipeline.fromOptions(options);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/da1dcf9b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TextIOShardedWriteFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TextIOShardedWriteFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TextIOShardedWriteFactoryTest.java
deleted file mode 100644
index 5ede931..0000000
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TextIOShardedWriteFactoryTest.java
+++ /dev/null
@@ -1,120 +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.runners.direct;
-
-import static org.hamcrest.Matchers.not;
-import static org.hamcrest.Matchers.theInstance;
-import static org.junit.Assert.assertThat;
-
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.io.TextIO;
-import org.apache.beam.sdk.io.TextIOTest;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.PTransform;
-import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PDone;
-
-import org.hamcrest.Matchers;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-
-import java.io.File;
-
-/**
- * Tests for {@link TextIOShardedWriteFactory}.
- */
-@RunWith(JUnit4.class)
-public class TextIOShardedWriteFactoryTest {
-  @Rule public TemporaryFolder tmp = new TemporaryFolder();
-  private TextIOShardedWriteFactory factory;
-
-  @Before
-  public void setup() {
-    factory = new TextIOShardedWriteFactory();
-  }
-
-  @Test
-  public void originalWithoutShardingReturnsOriginal() throws Exception {
-    File file = tmp.newFile("foo");
-    PTransform<PCollection<String>, PDone> original =
-        TextIO.Write.to(file.getAbsolutePath()).withoutSharding();
-    PTransform<PCollection<String>, PDone> overridden = factory.override(original);
-
-    assertThat(overridden, theInstance(original));
-  }
-
-  @Test
-  public void originalShardingNotSpecifiedReturnsOriginal() throws Exception {
-    File file = tmp.newFile("foo");
-    PTransform<PCollection<String>, PDone> original = TextIO.Write.to(file.getAbsolutePath());
-    PTransform<PCollection<String>, PDone> overridden = factory.override(original);
-
-    assertThat(overridden, theInstance(original));
-  }
-
-  @Test
-  public void originalShardedToOneReturnsExplicitlySharded() throws Exception {
-    File file = tmp.newFile("foo");
-    TextIO.Write.Bound<String> original =
-        TextIO.Write.to(file.getAbsolutePath()).withNumShards(1);
-    PTransform<PCollection<String>, PDone> overridden = factory.override(original);
-
-    assertThat(overridden, not(Matchers.<PTransform<PCollection<String>, PDone>>equalTo(original)));
-
-    Pipeline p = getPipeline();
-    String[] elems = new String[] {"foo", "bar", "baz"};
-    p.apply(Create.<String>of(elems)).apply(overridden);
-
-    file.delete();
-
-    p.run();
-    TextIOTest.assertOutputFiles(
-        elems, StringUtf8Coder.of(), 1, tmp, "foo", original.getShardNameTemplate());
-  }
-
-  @Test
-  public void originalShardedToManyReturnsExplicitlySharded() throws Exception {
-    File file = tmp.newFile("foo");
-    TextIO.Write.Bound<String> original = TextIO.Write.to(file.getAbsolutePath()).withNumShards(3);
-    PTransform<PCollection<String>, PDone> overridden = factory.override(original);
-
-    assertThat(overridden, not(Matchers.<PTransform<PCollection<String>, PDone>>equalTo(original)));
-
-    Pipeline p = getPipeline();
-    String[] elems = new String[] {"foo", "bar", "baz", "spam", "ham", "eggs"};
-    p.apply(Create.<String>of(elems)).apply(overridden);
-
-    file.delete();
-    p.run();
-    TextIOTest.assertOutputFiles(
-        elems, StringUtf8Coder.of(), 3, tmp, "foo", original.getShardNameTemplate());
-  }
-
-  private Pipeline getPipeline() {
-    PipelineOptions options = TestPipeline.testingPipelineOptions();
-    options.setRunner(DirectRunner.class);
-    return TestPipeline.fromOptions(options);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/da1dcf9b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
index 33f97e6..70dd94f 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
@@ -44,7 +44,6 @@ import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.Pipeline.PipelineVisitor;
 import org.apache.beam.sdk.PipelineResult.State;
 import org.apache.beam.sdk.annotations.Experimental;
-import org.apache.beam.sdk.coders.AvroCoder;
 import org.apache.beam.sdk.coders.BigEndianLongCoder;
 import org.apache.beam.sdk.coders.CannotProvideCoderException;
 import org.apache.beam.sdk.coders.Coder;
@@ -66,7 +65,6 @@ import org.apache.beam.sdk.io.PubsubIO;
 import org.apache.beam.sdk.io.PubsubUnboundedSink;
 import org.apache.beam.sdk.io.PubsubUnboundedSource;
 import org.apache.beam.sdk.io.Read;
-import org.apache.beam.sdk.io.ShardNameTemplate;
 import org.apache.beam.sdk.io.TextIO;
 import org.apache.beam.sdk.io.UnboundedSource;
 import org.apache.beam.sdk.io.Write;
@@ -91,7 +89,6 @@ import org.apache.beam.sdk.transforms.View.CreatePCollectionView;
 import org.apache.beam.sdk.transforms.WithKeys;
 import org.apache.beam.sdk.transforms.windowing.AfterPane;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.sdk.transforms.windowing.DefaultTrigger;
 import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
 import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
 import org.apache.beam.sdk.transforms.windowing.Window;
@@ -376,8 +373,6 @@ public class DataflowRunner extends PipelineRunner<DataflowPipelineJob> {
       builder.put(Read.Unbounded.class, UnsupportedIO.class);
       builder.put(Window.Bound.class, AssignWindows.class);
       builder.put(Write.Bound.class, BatchWrite.class);
-      builder.put(AvroIO.Write.Bound.class, BatchAvroIOWrite.class);
-      builder.put(TextIO.Write.Bound.class, BatchTextIOWrite.class);
       // In batch mode must use the custom Pubsub bounded source/sink.
       builder.put(PubsubUnboundedSource.class, UnsupportedIO.class);
       builder.put(PubsubUnboundedSink.class, UnsupportedIO.class);
@@ -2048,52 +2043,6 @@ public class DataflowRunner extends PipelineRunner<DataflowPipelineJob> {
   }
 
   /**
-   * A {@link PTransform} that uses shuffle to create a fusion break. This allows pushing
-   * parallelism limits such as sharding controls further down the pipeline.
-   */
-  private static class ReshardForWrite<T> extends PTransform<PCollection<T>, PCollection<T>> {
-    @Override
-    public PCollection<T> apply(PCollection<T> input) {
-      return input
-          // TODO: This would need to be adapted to write per-window shards.
-          .apply(
-              Window.<T>into(new GlobalWindows())
-                  .triggering(DefaultTrigger.of())
-                  .discardingFiredPanes())
-          .apply(
-              "RandomKey",
-              ParDo.of(
-                  new DoFn<T, KV<Long, T>>() {
-                    transient long counter, step;
-
-                    @Override
-                    public void startBundle(Context c) {
-                      counter = (long) (Math.random() * Long.MAX_VALUE);
-                      step = 1 + 2 * (long) (Math.random() * Long.MAX_VALUE);
-                    }
-
-                    @Override
-                    public void processElement(ProcessContext c) {
-                      counter += step;
-                      c.output(KV.of(counter, c.element()));
-                    }
-                  }))
-          .apply(GroupByKey.<Long, T>create())
-          .apply(
-              "Ungroup",
-              ParDo.of(
-                  new DoFn<KV<Long, Iterable<T>>, T>() {
-                    @Override
-                    public void processElement(ProcessContext c) {
-                      for (T item : c.element().getValue()) {
-                        c.output(item);
-                      }
-                    }
-                  }));
-    }
-  }
-
-  /**
    * Specialized implementation which overrides
    * {@link org.apache.beam.sdk.io.Write.Bound Write.Bound} to provide Google
    * Cloud Dataflow specific path validation of {@link FileBasedSink}s.
@@ -2122,213 +2071,6 @@ public class DataflowRunner extends PipelineRunner<DataflowPipelineJob> {
   }
 
   /**
-   * Specialized implementation which overrides
-   * {@link org.apache.beam.sdk.io.TextIO.Write.Bound TextIO.Write.Bound} with
-   * a native sink instead of a custom sink as workaround until custom sinks
-   * have support for sharding controls.
-   */
-  private static class BatchTextIOWrite<T> extends PTransform<PCollection<T>, PDone> {
-    private final TextIO.Write.Bound<T> transform;
-    /**
-     * Builds an instance of this class from the overridden transform.
-     */
-    @SuppressWarnings("unused") // used via reflection in DataflowRunner#apply()
-    public BatchTextIOWrite(DataflowRunner runner, TextIO.Write.Bound<T> transform) {
-      this.transform = transform;
-    }
-
-    @Override
-    public PDone apply(PCollection<T> input) {
-      if (transform.getNumShards() > 0) {
-        return input
-            .apply(new ReshardForWrite<T>())
-            .apply(new BatchTextIONativeWrite<>(transform));
-      } else {
-        return transform.apply(input);
-      }
-    }
-  }
-
-  /**
-   * This {@link PTransform} is used by the {@link DataflowPipelineTranslator} as a way
-   * to provide the native definition of the Text sink.
-   */
-  private static class BatchTextIONativeWrite<T> extends PTransform<PCollection<T>, PDone> {
-    private final TextIO.Write.Bound<T> transform;
-    public BatchTextIONativeWrite(TextIO.Write.Bound<T> transform) {
-      this.transform = transform;
-    }
-
-    @Override
-    public PDone apply(PCollection<T> input) {
-      return PDone.in(input.getPipeline());
-    }
-
-    static {
-      DataflowPipelineTranslator.registerTransformTranslator(
-          BatchTextIONativeWrite.class, new BatchTextIONativeWriteTranslator());
-    }
-  }
-
-  /**
-   * TextIO.Write.Bound support code for the Dataflow backend when applying parallelism limits
-   * through user requested sharding limits.
-   */
-  private static class BatchTextIONativeWriteTranslator
-      implements TransformTranslator<BatchTextIONativeWrite<?>> {
-    @SuppressWarnings("unchecked")
-    @Override
-    public void translate(@SuppressWarnings("rawtypes") BatchTextIONativeWrite transform,
-        TranslationContext context) {
-      translateWriteHelper(transform, transform.transform, context);
-    }
-
-    private <T> void translateWriteHelper(
-        BatchTextIONativeWrite<T> transform,
-        TextIO.Write.Bound<T> originalTransform,
-        TranslationContext context) {
-      // Note that the original transform can not be used during add step/add input
-      // and is only passed in to get properties from it.
-
-      checkState(originalTransform.getNumShards() > 0,
-          "Native TextSink is expected to only be used when sharding controls are required.");
-
-      context.addStep(transform, "ParallelWrite");
-      context.addInput(PropertyNames.PARALLEL_INPUT, context.getInput(transform));
-
-      // TODO: drop this check when server supports alternative templates.
-      switch (originalTransform.getShardTemplate()) {
-        case ShardNameTemplate.INDEX_OF_MAX:
-          break;  // supported by server
-        case "":
-          // Empty shard template allowed - forces single output.
-          checkArgument(originalTransform.getNumShards() <= 1,
-              "Num shards must be <= 1 when using an empty sharding template");
-          break;
-        default:
-          throw new UnsupportedOperationException("Shard template "
-              + originalTransform.getShardTemplate()
-              + " not yet supported by Dataflow service");
-      }
-
-      // TODO: How do we want to specify format and
-      // format-specific properties?
-      context.addInput(PropertyNames.FORMAT, "text");
-      context.addInput(PropertyNames.FILENAME_PREFIX, originalTransform.getFilenamePrefix());
-      context.addInput(PropertyNames.SHARD_NAME_TEMPLATE,
-          originalTransform.getShardNameTemplate());
-      context.addInput(PropertyNames.FILENAME_SUFFIX, originalTransform.getFilenameSuffix());
-      context.addInput(PropertyNames.VALIDATE_SINK, originalTransform.needsValidation());
-      context.addInput(PropertyNames.NUM_SHARDS, (long) originalTransform.getNumShards());
-      context.addEncodingInput(
-          WindowedValue.getValueOnlyCoder(originalTransform.getCoder()));
-
-    }
-  }
-
-  /**
-   * Specialized implementation which overrides
-   * {@link org.apache.beam.sdk.io.AvroIO.Write.Bound AvroIO.Write.Bound} with
-   * a native sink instead of a custom sink as workaround until custom sinks
-   * have support for sharding controls.
-   */
-  private static class BatchAvroIOWrite<T> extends PTransform<PCollection<T>, PDone> {
-    private final AvroIO.Write.Bound<T> transform;
-    /**
-     * Builds an instance of this class from the overridden transform.
-     */
-    @SuppressWarnings("unused") // used via reflection in DataflowRunner#apply()
-    public BatchAvroIOWrite(DataflowRunner runner, AvroIO.Write.Bound<T> transform) {
-      this.transform = transform;
-    }
-
-    @Override
-    public PDone apply(PCollection<T> input) {
-      if (transform.getNumShards() > 0) {
-        return input
-            .apply(new ReshardForWrite<T>())
-            .apply(new BatchAvroIONativeWrite<>(transform));
-      } else {
-        return transform.apply(input);
-      }
-    }
-  }
-
-  /**
-   * This {@link PTransform} is used by the {@link DataflowPipelineTranslator} as a way
-   * to provide the native definition of the Avro sink.
-   */
-  private static class BatchAvroIONativeWrite<T> extends PTransform<PCollection<T>, PDone> {
-    private final AvroIO.Write.Bound<T> transform;
-    public BatchAvroIONativeWrite(AvroIO.Write.Bound<T> transform) {
-      this.transform = transform;
-    }
-
-    @Override
-    public PDone apply(PCollection<T> input) {
-      return PDone.in(input.getPipeline());
-    }
-
-    static {
-      DataflowPipelineTranslator.registerTransformTranslator(
-          BatchAvroIONativeWrite.class, new BatchAvroIONativeWriteTranslator());
-    }
-  }
-
-  /**
-   * AvroIO.Write.Bound support code for the Dataflow backend when applying parallelism limits
-   * through user requested sharding limits.
-   */
-  private static class BatchAvroIONativeWriteTranslator
-      implements TransformTranslator<BatchAvroIONativeWrite<?>> {
-    @SuppressWarnings("unchecked")
-    @Override
-    public void translate(@SuppressWarnings("rawtypes") BatchAvroIONativeWrite transform,
-        TranslationContext context) {
-      translateWriteHelper(transform, transform.transform, context);
-    }
-
-    private <T> void translateWriteHelper(
-        BatchAvroIONativeWrite<T> transform,
-        AvroIO.Write.Bound<T> originalTransform,
-        TranslationContext context) {
-      // Note that the original transform can not be used during add step/add input
-      // and is only passed in to get properties from it.
-
-      checkState(originalTransform.getNumShards() > 0,
-          "Native AvroSink is expected to only be used when sharding controls are required.");
-
-      context.addStep(transform, "ParallelWrite");
-      context.addInput(PropertyNames.PARALLEL_INPUT, context.getInput(transform));
-
-      // TODO: drop this check when server supports alternative templates.
-      switch (originalTransform.getShardTemplate()) {
-        case ShardNameTemplate.INDEX_OF_MAX:
-          break;  // supported by server
-        case "":
-          // Empty shard template allowed - forces single output.
-          checkArgument(originalTransform.getNumShards() <= 1,
-              "Num shards must be <= 1 when using an empty sharding template");
-          break;
-        default:
-          throw new UnsupportedOperationException("Shard template "
-              + originalTransform.getShardTemplate()
-              + " not yet supported by Dataflow service");
-      }
-
-      context.addInput(PropertyNames.FORMAT, "avro");
-      context.addInput(PropertyNames.FILENAME_PREFIX, originalTransform.getFilenamePrefix());
-      context.addInput(PropertyNames.SHARD_NAME_TEMPLATE, originalTransform.getShardTemplate());
-      context.addInput(PropertyNames.FILENAME_SUFFIX, originalTransform.getFilenameSuffix());
-      context.addInput(PropertyNames.VALIDATE_SINK, originalTransform.needsValidation());
-      context.addInput(PropertyNames.NUM_SHARDS, (long) originalTransform.getNumShards());
-      context.addEncodingInput(
-          WindowedValue.getValueOnlyCoder(
-              AvroCoder.of(originalTransform.getType(), originalTransform.getSchema())));
-    }
-  }
-
-  /**
    * Specialized (non-)implementation for
    * {@link org.apache.beam.sdk.io.Write.Bound Write.Bound}
    * for the Dataflow runner in streaming mode.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/da1dcf9b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java
index 280cd12..718461a 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java
@@ -647,14 +647,14 @@ public class AvroIO {
           throw new IllegalStateException("need to set the schema of an AvroIO.Write transform");
         }
 
-        // Note that custom sinks currently do not expose sharding controls.
-        // Thus pipeline runner writers need to individually add support internally to
-        // apply user requested sharding limits.
-        return input.apply(
-            "Write",
+        org.apache.beam.sdk.io.Write.Bound<T> write =
             org.apache.beam.sdk.io.Write.to(
                 new AvroSink<>(
-                    filenamePrefix, filenameSuffix, shardTemplate, AvroCoder.of(type, schema))));
+                    filenamePrefix, filenameSuffix, shardTemplate, AvroCoder.of(type, schema)));
+        if (getNumShards() > 0) {
+          write = write.withNumShards(getNumShards());
+        }
+        return input.apply("Write", write);
       }
 
       @Override

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/da1dcf9b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java
index 9dd3679..64db3f7 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java
@@ -109,7 +109,7 @@ import javax.annotation.Nullable;
  * }</pre>
  *
  * <h3>Permissions</h3>
- * <p>When run using the {@link DirectRunner}, your pipeline can read and write text files
+ * <p>When run using the {@code DirectRunner}, your pipeline can read and write text files
  * on your local drive and remote text files on Google Cloud Storage that you have access to using
  * your {@code gcloud} credentials. When running in the Dataflow service, the pipeline can only
  * read and write files from GCS. For more information about permissions, see the Cloud Dataflow
@@ -608,12 +608,13 @@ public class TextIO {
               "need to set the filename prefix of a TextIO.Write transform");
         }
 
-        // Note that custom sinks currently do not expose sharding controls.
-        // Thus pipeline runner writers need to individually add support internally to
-        // apply user requested sharding limits.
-        return input.apply("Write", org.apache.beam.sdk.io.Write.to(
-            new TextSink<>(
-                filenamePrefix, filenameSuffix, shardTemplate, coder)));
+        org.apache.beam.sdk.io.Write.Bound<T> write =
+            org.apache.beam.sdk.io.Write.to(
+                new TextSink<>(filenamePrefix, filenameSuffix, shardTemplate, coder));
+        if (getNumShards() > 0) {
+          write = write.withNumShards(getNumShards());
+        }
+        return input.apply("Write", write);
       }
 
       @Override

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/da1dcf9b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Write.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Write.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Write.java
index df6e4d2..c48933b 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Write.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Write.java
@@ -17,6 +17,8 @@
  */
 package org.apache.beam.sdk.io;
 
+import static com.google.common.base.Preconditions.checkNotNull;
+
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.coders.Coder;
@@ -26,54 +28,80 @@ import org.apache.beam.sdk.io.Sink.Writer;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.transforms.Create;
 import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
 import org.apache.beam.sdk.transforms.View;
+import org.apache.beam.sdk.transforms.WithKeys;
 import org.apache.beam.sdk.transforms.display.DisplayData;
+import org.apache.beam.sdk.transforms.windowing.DefaultTrigger;
 import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
 import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.PDone;
 
+import com.google.api.client.util.Lists;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.Collections;
+import java.util.List;
 import java.util.UUID;
+import java.util.concurrent.ThreadLocalRandom;
 
 /**
  * A {@link PTransform} that writes to a {@link Sink}. A write begins with a sequential global
  * initialization of a sink, followed by a parallel write, and ends with a sequential finalization
- * of the write. The output of a write is {@link PDone}.  In the case of an empty PCollection, only
- * the global initialization and finalization will be performed.
+ * of the write. The output of a write is {@link PDone}.
+ *
+ * <p>By default, every bundle in the input {@link PCollection} will be processed by a
+ * {@link WriteOperation}, so the number of outputs will vary based on runner behavior, though at
+ * least 1 output will always be produced. The exact parallelism of the write stage can be
+ * controlled using {@link Write.Bound#withNumShards}, typically used to control how many files are
+ * produced or to globally limit the number of workers connecting to an external service. However,
+ * this option can often hurt performance: it adds an additional {@link GroupByKey} to the pipeline.
+ *
+ * <p>{@code Write} re-windows the data into the global window, so it is typically not well suited
+ * to use in streaming pipelines.
  *
- * <p>Currently, only batch workflows can contain Write transforms.
+ * <p>Example usage with runner-controlled sharding:
  *
- * <p>Example usage:
+ * <pre>{@code p.apply(Write.to(new MySink(...)));}</pre>
+
+ * <p>Example usage with a fixed number of shards:
  *
- * <p>{@code p.apply(Write.to(new MySink(...)));}
+ * <pre>{@code p.apply(Write.to(new MySink(...)).withNumShards(3));}</pre>
  */
 @Experimental(Experimental.Kind.SOURCE_SINK)
 public class Write {
   private static final Logger LOG = LoggerFactory.getLogger(Write.class);
 
   /**
-   * Creates a Write transform that writes to the given Sink.
+   * Creates a {@link Write} transform that writes to the given {@link Sink}, letting the runner
+   * control how many different shards are produced.
    */
   public static <T> Bound<T> to(Sink<T> sink) {
-    return new Bound<>(sink);
+    checkNotNull(sink, "sink");
+    return new Bound<>(sink, 0 /* runner-controlled sharding */);
   }
 
   /**
-   * A {@link PTransform} that writes to a {@link Sink}. See {@link Write} and {@link Sink} for
-   * documentation about writing to Sinks.
+   * A {@link PTransform} that writes to a {@link Sink}. See the class-level Javadoc for more
+   * information.
+   *
+   * @see Write
+   * @see Sink
    */
   public static class Bound<T> extends PTransform<PCollection<T>, PDone> {
     private final Sink<T> sink;
+    private int numShards;
 
-    private Bound(Sink<T> sink) {
+    private Bound(Sink<T> sink, int numShards) {
       this.sink = sink;
+      this.numShards = numShards;
     }
 
     @Override
@@ -87,9 +115,20 @@ public class Write {
     public void populateDisplayData(DisplayData.Builder builder) {
       super.populateDisplayData(builder);
       builder
-          .add(DisplayData.item("sink", sink.getClass())
-            .withLabel("Write Sink"))
-          .include(sink);
+          .add(DisplayData.item("sink", sink.getClass()).withLabel("Write Sink"))
+          .include(sink)
+          .addIfNotDefault(
+              DisplayData.item("numShards", getNumShards()).withLabel("Fixed Number of Shards"),
+              0);
+    }
+
+    /**
+     * Returns the number of shards that will be produced in the output.
+     *
+     * @see Write for more information
+     */
+    public int getNumShards() {
+      return numShards;
     }
 
     /**
@@ -100,6 +139,153 @@ public class Write {
     }
 
     /**
+     * Returns a new {@link Write.Bound} that will write to the current {@link Sink} using the
+     * specified number of shards.
+     *
+     * <p>This option should be used sparingly as it can hurt performance. See {@link Write} for
+     * more information.
+     *
+     * <p>A value less than or equal to 0 will be equivalent to the default behavior of
+     * runner-controlled sharding.
+     */
+    public Bound<T> withNumShards(int numShards) {
+      return new Bound<>(sink, Math.max(numShards, 0));
+    }
+
+    /**
+     * Writes all the elements in a bundle using a {@link Writer} produced by the
+     * {@link WriteOperation} associated with the {@link Sink}.
+     */
+    private class WriteBundles<WriteT> extends DoFn<T, WriteT> {
+      // Writer that will write the records in this bundle. Lazily
+      // initialized in processElement.
+      private Writer<T, WriteT> writer = null;
+      private final PCollectionView<WriteOperation<T, WriteT>> writeOperationView;
+
+      WriteBundles(PCollectionView<WriteOperation<T, WriteT>> writeOperationView) {
+        this.writeOperationView = writeOperationView;
+      }
+
+      @Override
+      public void processElement(ProcessContext c) throws Exception {
+        // Lazily initialize the Writer
+        if (writer == null) {
+          WriteOperation<T, WriteT> writeOperation = c.sideInput(writeOperationView);
+          LOG.info("Opening writer for write operation {}", writeOperation);
+          writer = writeOperation.createWriter(c.getPipelineOptions());
+          writer.open(UUID.randomUUID().toString());
+          LOG.debug("Done opening writer {} for operation {}", writer, writeOperationView);
+        }
+        try {
+          writer.write(c.element());
+        } catch (Exception e) {
+          // Discard write result and close the write.
+          try {
+            writer.close();
+            // The writer does not need to be reset, as this DoFn cannot be reused.
+          } catch (Exception closeException) {
+            if (closeException instanceof InterruptedException) {
+              // Do not silently ignore interrupted state.
+              Thread.currentThread().interrupt();
+            }
+            // Do not mask the exception that caused the write to fail.
+            e.addSuppressed(closeException);
+          }
+          throw e;
+        }
+      }
+
+      @Override
+      public void finishBundle(Context c) throws Exception {
+        if (writer != null) {
+          WriteT result = writer.close();
+          c.output(result);
+          // Reset state in case of reuse.
+          writer = null;
+        }
+      }
+
+      @Override
+      public void populateDisplayData(DisplayData.Builder builder) {
+        Write.Bound.this.populateDisplayData(builder);
+      }
+    }
+
+    /**
+     * Like {@link WriteBundles}, but where the elements for each shard have been collected into
+     * a single iterable.
+     *
+     * @see WriteBundles
+     */
+    private class WriteShardedBundles<WriteT> extends DoFn<KV<Integer, Iterable<T>>, WriteT> {
+      private final PCollectionView<WriteOperation<T, WriteT>> writeOperationView;
+
+      WriteShardedBundles(PCollectionView<WriteOperation<T, WriteT>> writeOperationView) {
+        this.writeOperationView = writeOperationView;
+      }
+
+      @Override
+      public void processElement(ProcessContext c) throws Exception {
+        // In a sharded write, single input element represents one shard. We can open and close
+        // the writer in each call to processElement.
+        WriteOperation<T, WriteT> writeOperation = c.sideInput(writeOperationView);
+        LOG.info("Opening writer for write operation {}", writeOperation);
+        Writer<T, WriteT> writer = writeOperation.createWriter(c.getPipelineOptions());
+        writer.open(UUID.randomUUID().toString());
+        LOG.debug("Done opening writer {} for operation {}", writer, writeOperationView);
+
+        try {
+          for (T t : c.element().getValue()) {
+            writer.write(t);
+          }
+        } catch (Exception e) {
+          try {
+            writer.close();
+          } catch (Exception closeException) {
+            if (closeException instanceof InterruptedException) {
+              // Do not silently ignore interrupted state.
+              Thread.currentThread().interrupt();
+            }
+            // Do not mask the exception that caused the write to fail.
+            e.addSuppressed(closeException);
+          }
+          throw e;
+        }
+
+        // Close the writer; if this throws let the error propagate.
+        WriteT result = writer.close();
+        c.output(result);
+      }
+
+      @Override
+      public void populateDisplayData(DisplayData.Builder builder) {
+        Write.Bound.this.populateDisplayData(builder);
+      }
+    }
+
+    private static class ApplyShardingKey<T> implements SerializableFunction<T, Integer> {
+      private final int numShards;
+      private int shardNumber;
+
+      ApplyShardingKey(int numShards) {
+        this.numShards = numShards;
+        shardNumber = -1;
+      }
+
+      @Override
+      public Integer apply(T input) {
+        if (shardNumber == -1) {
+          // We want to desynchronize the first record sharding key for each instance of
+          // ApplyShardingKey, so records in a small PCollection will be statistically balanced.
+          shardNumber = ThreadLocalRandom.current().nextInt(numShards);
+        } else {
+          shardNumber = (shardNumber + 1) % numShards;
+        }
+        return shardNumber;
+      }
+    }
+
+    /**
      * A write is performed as sequence of three {@link ParDo}'s.
      *
      * <p>In the first, a do-once ParDo is applied to a singleton PCollection containing the Sink's
@@ -142,7 +328,7 @@ public class Write {
       // A singleton collection of the WriteOperation, to be used as input to a ParDo to initialize
       // the sink.
       PCollection<WriteOperation<T, WriteT>> operationCollection =
-          p.apply(Create.<WriteOperation<T, WriteT>>of(writeOperation).withCoder(operationCoder));
+          p.apply(Create.of(writeOperation).withCoder(operationCoder));
 
       // Initialize the resource in a do-once ParDo on the WriteOperation.
       operationCollection = operationCollection
@@ -165,57 +351,32 @@ public class Write {
       final PCollectionView<WriteOperation<T, WriteT>> writeOperationView =
           operationCollection.apply(View.<WriteOperation<T, WriteT>>asSingleton());
 
+      // Re-window the data into the global window and remove any existing triggers.
+      PCollection<T> inputInGlobalWindow =
+          input.apply(
+              Window.<T>into(new GlobalWindows())
+                  .triggering(DefaultTrigger.of())
+                  .discardingFiredPanes());
+
       // Perform the per-bundle writes as a ParDo on the input PCollection (with the WriteOperation
       // as a side input) and collect the results of the writes in a PCollection.
       // There is a dependency between this ParDo and the first (the WriteOperation PCollection
       // as a side input), so this will happen after the initial ParDo.
-      PCollection<WriteT> results = input
-          .apply(Window.<T>into(new GlobalWindows()))
-          .apply("WriteBundles", ParDo.of(new DoFn<T, WriteT>() {
-            // Writer that will write the records in this bundle. Lazily
-            // initialized in processElement.
-            private Writer<T, WriteT> writer = null;
-
-            @Override
-            public void processElement(ProcessContext c) throws Exception {
-              // Lazily initialize the Writer
-              if (writer == null) {
-                WriteOperation<T, WriteT> writeOperation = c.sideInput(writeOperationView);
-                LOG.info("Opening writer for write operation {}", writeOperation);
-                writer = writeOperation.createWriter(c.getPipelineOptions());
-                writer.open(UUID.randomUUID().toString());
-                LOG.debug("Done opening writer {} for operation {}", writer, writeOperationView);
-              }
-              try {
-                writer.write(c.element());
-              } catch (Exception e) {
-                // Discard write result and close the write.
-                try {
-                  writer.close();
-                  // The writer does not need to be reset, as this DoFn cannot be reused
-                } catch (Exception closeException) {
-                  // Do not mask the exception that caused the write to fail.
-                }
-                throw e;
-              }
-            }
-
-            @Override
-            public void finishBundle(Context c) throws Exception {
-              if (writer != null) {
-                WriteT result = writer.close();
-                c.output(result);
-                // Reset state in case of reuse
-                writer = null;
-              }
-            }
-
-            @Override
-            public void populateDisplayData(DisplayData.Builder builder) {
-              Write.Bound.this.populateDisplayData(builder);
-            }
-          }).withSideInputs(writeOperationView))
-          .setCoder(writeOperation.getWriterResultCoder());
+      PCollection<WriteT> results;
+      if (getNumShards() <= 0) {
+        results = inputInGlobalWindow
+            .apply("WriteBundles",
+                ParDo.of(new WriteBundles<>(writeOperationView))
+                    .withSideInputs(writeOperationView));
+      } else {
+        results = inputInGlobalWindow
+            .apply("ApplyShardLabel", WithKeys.of(new ApplyShardingKey<T>(getNumShards())))
+            .apply("GroupIntoShards", GroupByKey.<Integer, T>create())
+            .apply("WriteShardedBundles",
+                ParDo.of(new WriteShardedBundles<>(writeOperationView))
+                    .withSideInputs(writeOperationView));
+      }
+      results.setCoder(writeOperation.getWriterResultCoder());
 
       final PCollectionView<Iterable<WriteT>> resultsView =
           results.apply(View.<WriteT>asIterable());
@@ -231,17 +392,26 @@ public class Write {
             @Override
             public void processElement(ProcessContext c) throws Exception {
               WriteOperation<T, WriteT> writeOperation = c.element();
-              LOG.info("Finalizing write operation {}", writeOperation);
-              Iterable<WriteT> results = c.sideInput(resultsView);
-              LOG.debug("Side input initialized to finalize write operation {}", writeOperation);
-              if (!results.iterator().hasNext()) {
-                LOG.info("No write results, creating a single empty output.");
-                Writer<T, WriteT> writer = writeOperation.createWriter(c.getPipelineOptions());
-                writer.open(UUID.randomUUID().toString());
-                WriteT emptyWrite = writer.close();
-                results = Collections.singleton(emptyWrite);
-                LOG.debug("Done creating a single empty output.");
+              LOG.info("Finalizing write operation {}.", writeOperation);
+              List<WriteT> results = Lists.newArrayList(c.sideInput(resultsView));
+              LOG.debug("Side input initialized to finalize write operation {}.", writeOperation);
+
+              // We must always output at least 1 shard, and honor user-specified numShards if set.
+              int minShardsNeeded = Math.max(1, getNumShards());
+              int extraShardsNeeded = minShardsNeeded - results.size();
+              if (extraShardsNeeded > 0) {
+                LOG.info(
+                    "Creating {} empty output shards in addition to {} written for a total of {}.",
+                    extraShardsNeeded, results.size(), minShardsNeeded);
+                for (int i = 0; i < extraShardsNeeded; ++i) {
+                  Writer<T, WriteT> writer = writeOperation.createWriter(c.getPipelineOptions());
+                  writer.open(UUID.randomUUID().toString());
+                  WriteT emptyWrite = writer.close();
+                  results.add(emptyWrite);
+                }
+                LOG.debug("Done creating extra shards.");
               }
+
               writeOperation.finalize(results, c.getPipelineOptions());
               LOG.debug("Done finalizing write operation {}", writeOperation);
             }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/da1dcf9b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteTest.java
index abda3a5..56643f2 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteTest.java
@@ -19,9 +19,11 @@ package org.apache.beam.sdk.io;
 
 import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem;
 import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom;
+
 import static org.hamcrest.Matchers.anyOf;
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.is;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
@@ -52,7 +54,9 @@ import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 
 import com.google.common.base.MoreObjects;
+import com.google.common.base.Optional;
 
+import org.hamcrest.Matchers;
 import org.joda.time.Duration;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -62,12 +66,14 @@ import org.junit.runners.JUnit4;
 import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Objects;
 import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicInteger;
 
 /**
  * Tests for the Write PTransform.
@@ -76,6 +82,10 @@ import java.util.concurrent.ThreadLocalRandom;
 public class WriteTest {
   // Static store that can be accessed within the writer
   private static List<String> sinkContents = new ArrayList<>();
+  // Static count of output shards
+  private static AtomicInteger numShards = new AtomicInteger(0);
+  // Static counts of the number of records per shard.
+  private static List<Integer> recordsPerShard = new ArrayList<>();
 
   private static final MapElements<String, String> IDENTITY_MAP =
       MapElements.via(new SimpleFunction<String, String>() {
@@ -129,6 +139,71 @@ public class WriteTest {
   }
 
   /**
+   * Test that Write with an empty input still produces one shard.
+   */
+  @Test
+  @Category(NeedsRunner.class)
+  public void testEmptyWrite() {
+    runWrite(Collections.<String>emptyList(), IDENTITY_MAP);
+    // Note we did not request a sharded write, so runWrite will not validate the number of shards.
+    assertEquals(1, numShards.intValue());
+  }
+
+  /**
+   * Test that Write with a configured number of shards produces the desired number of shards even
+   * when there are many elements.
+   */
+  @Test
+  @Category(NeedsRunner.class)
+  public void testShardedWrite() {
+    runShardedWrite(
+        Arrays.asList("one", "two", "three", "four", "five", "six"),
+        IDENTITY_MAP,
+        Optional.of(1));
+  }
+
+  /**
+   * Test that Write with a configured number of shards produces the desired number of shards even
+   * when there are too few elements.
+   */
+  @Test
+  @Category(NeedsRunner.class)
+  public void testExpandShardedWrite() {
+    runShardedWrite(
+        Arrays.asList("one", "two", "three", "four", "five", "six"),
+        IDENTITY_MAP,
+        Optional.of(20));
+  }
+
+  /**
+   * Tests that a Write can balance many elements.
+   */
+  @Test
+  @Category(NeedsRunner.class)
+  public void testShardedWriteBalanced() {
+    int numElements = 1000;
+    List<String> inputs = new ArrayList<>(numElements);
+    for (int i = 0; i < numElements; ++i) {
+      inputs.add(String.format("elt%04d", i));
+    }
+
+    runShardedWrite(
+        inputs,
+        new WindowAndReshuffle<>(
+            Window.<String>into(Sessions.withGapDuration(Duration.millis(1)))),
+        Optional.of(10));
+
+    // Check that both the min and max number of results per shard are close to the expected.
+    int min = Integer.MAX_VALUE;
+    int max = Integer.MIN_VALUE;
+    for (Integer i : recordsPerShard) {
+      min = Math.min(min, i);
+      max = Math.max(max, i);
+    }
+    assertThat((double) min, Matchers.greaterThanOrEqualTo(max * 0.9));
+  }
+
+  /**
    * Test a Write transform with an empty PCollection.
    */
   @Test
@@ -147,7 +222,7 @@ public class WriteTest {
     List<String> inputs = Arrays.asList("Critical canary", "Apprehensive eagle",
         "Intimidating pigeon", "Pedantic gull", "Frisky finch");
     runWrite(
-        inputs, new WindowAndReshuffle(Window.<String>into(FixedWindows.of(Duration.millis(2)))));
+        inputs, new WindowAndReshuffle<>(Window.<String>into(FixedWindows.of(Duration.millis(2)))));
   }
 
   /**
@@ -161,7 +236,22 @@ public class WriteTest {
 
     runWrite(
         inputs,
-        new WindowAndReshuffle(Window.<String>into(Sessions.withGapDuration(Duration.millis(1)))));
+        new WindowAndReshuffle<>(
+            Window.<String>into(Sessions.withGapDuration(Duration.millis(1)))));
+  }
+
+  @Test
+  public void testBuildWrite() {
+    Sink<String> sink = new TestSink() {};
+    Write.Bound<String> write = Write.to(sink).withNumShards(3);
+    assertEquals(3, write.getNumShards());
+    assertThat(write.getSink(), is(sink));
+
+    Write.Bound<String> write2 = write.withNumShards(7);
+    assertEquals(7, write2.getNumShards());
+    assertThat(write2.getSink(), is(sink));
+    // original unchanged
+    assertEquals(3, write.getNumShards());
   }
 
   @Test
@@ -179,7 +269,20 @@ public class WriteTest {
     assertThat(displayData, includesDisplayDataFrom(sink));
   }
 
-
+  @Test
+  public void testShardedDisplayData() {
+    TestSink sink = new TestSink() {
+      @Override
+      public void populateDisplayData(DisplayData.Builder builder) {
+        builder.add(DisplayData.item("foo", "bar"));
+      }
+    };
+    Write.Bound<String> write = Write.to(sink).withNumShards(1);
+    DisplayData displayData = DisplayData.from(write);
+    assertThat(displayData, hasDisplayItem("sink", sink.getClass()));
+    assertThat(displayData, includesDisplayDataFrom(sink));
+    assertThat(displayData, hasDisplayItem("numShards", 1));
+  }
 
   /**
    * Performs a Write transform and verifies the Write transform calls the appropriate methods on
@@ -188,6 +291,18 @@ public class WriteTest {
    */
   private static void runWrite(
       List<String> inputs, PTransform<PCollection<String>, PCollection<String>> transform) {
+    runShardedWrite(inputs, transform, Optional.<Integer>absent());
+  }
+
+  /**
+   * Performs a Write transform with the desired number of shards. Verifies the Write transform
+   * calls the appropriate methods on a test sink in the correct order, as well as verifies that
+   * the elements of a PCollection are written to the sink. If numConfiguredShards is not null, also
+   * verifies that the output number of shards is correct.
+   */
+  private static void runShardedWrite(
+      List<String> inputs, PTransform<PCollection<String>, PCollection<String>> transform,
+      Optional<Integer> numConfiguredShards) {
     // Flag to validate that the pipeline options are passed to the Sink
     WriteOptions options = TestPipeline.testingPipelineOptions().as(WriteOptions.class);
     options.setTestFlag("test_value");
@@ -195,6 +310,10 @@ public class WriteTest {
 
     // Clear the sink's contents.
     sinkContents.clear();
+    // Reset the number of shards produced.
+    numShards.set(0);
+    // Reset the number of records in each shard.
+    recordsPerShard.clear();
 
     // Prepare timestamps for the elements.
     List<Long> timestamps = new ArrayList<>();
@@ -203,13 +322,21 @@ public class WriteTest {
     }
 
     TestSink sink = new TestSink();
+    Write.Bound<String> write = Write.to(sink);
+    if (numConfiguredShards.isPresent()) {
+      write = write.withNumShards(numConfiguredShards.get());
+    }
     p.apply(Create.timestamped(inputs, timestamps).withCoder(StringUtf8Coder.of()))
      .apply(transform)
-     .apply(Write.to(sink));
+     .apply(write);
 
     p.run();
     assertThat(sinkContents, containsInAnyOrder(inputs.toArray()));
     assertTrue(sink.hasCorrectState());
+    if (numConfiguredShards.isPresent()) {
+      assertEquals(numConfiguredShards.get().intValue(), numShards.intValue());
+      assertEquals(numConfiguredShards.get().intValue(), recordsPerShard.size());
+    }
   }
 
   // Test sink and associated write operation and writer. TestSink, TestWriteOperation, and
@@ -246,10 +373,7 @@ public class WriteTest {
      */
     @Override
     public boolean equals(Object other) {
-      if (!(other instanceof TestSink)) {
-        return false;
-      }
-      return true;
+      return (other instanceof TestSink);
     }
 
     @Override
@@ -314,6 +438,7 @@ public class WriteTest {
         idSet.add(result.uId);
         // Add the elements that were written to the sink's contents.
         sinkContents.addAll(result.elementsWritten);
+        recordsPerShard.add(result.elementsWritten.size());
       }
       // Each result came from a unique id.
       assertEquals(resultCount, idSet.size());
@@ -398,6 +523,7 @@ public class WriteTest {
 
     @Override
     public void open(String uId) throws Exception {
+      numShards.incrementAndGet();
       this.uId = uId;
       assertEquals(State.INITIAL, state);
       state = State.OPENED;
@@ -421,10 +547,9 @@ public class WriteTest {
   /**
    * Options for test, exposed for PipelineOptionsFactory.
    */
-  public static interface WriteOptions extends TestPipelineOptions {
+  public interface WriteOptions extends TestPipelineOptions {
     @Description("Test flag and value")
     String getTestFlag();
-
     void setTestFlag(String value);
   }
 }


[21/50] [abbrv] incubator-beam git commit: Added integration tests for BigtableRead and BigtableWrite

Posted by lc...@apache.org.
Added integration tests for BigtableRead and BigtableWrite


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

Branch: refs/heads/runners-spark2
Commit: 067c92a78b167d53d1791b1b912624a95c7eb57f
Parents: 49bb72d
Author: Ian Zhou <ia...@google.com>
Authored: Fri Jun 17 14:21:05 2016 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Jul 6 10:18:50 2016 -0700

----------------------------------------------------------------------
 sdks/java/io/google-cloud-platform/pom.xml      |  47 +++++
 .../io/gcp/bigtable/BigtableTestOptions.java    |  42 ++++
 .../sdk/io/gcp/bigtable/BigtableReadIT.java     |  61 ++++++
 .../sdk/io/gcp/bigtable/BigtableWriteIT.java    | 197 +++++++++++++++++++
 4 files changed, 347 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/067c92a7/sdks/java/io/google-cloud-platform/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/pom.xml b/sdks/java/io/google-cloud-platform/pom.xml
index 5786e84..bb5fd11 100644
--- a/sdks/java/io/google-cloud-platform/pom.xml
+++ b/sdks/java/io/google-cloud-platform/pom.xml
@@ -53,7 +53,39 @@
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-checkstyle-plugin</artifactId>
       </plugin>
+
+      <!-- Integration Tests -->
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-failsafe-plugin</artifactId>
+        <configuration>
+          <useManifestOnlyJar>false</useManifestOnlyJar>
+          <redirectTestOutputToFile>true</redirectTestOutputToFile>
+        </configuration>
+        <executions>
+          <execution>
+            <goals>
+              <goal>integration-test</goal>
+              <goal>verify</goal>
+            </goals>
+            <configuration>
+              <systemPropertyVariables>
+                <beamTestPipelineOptions>${integrationTestPipelineOptions}</beamTestPipelineOptions>
+              </systemPropertyVariables>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
     </plugins>
+
+    <extensions>
+      <!-- Use os-maven-plugin to initialize the "os.detected" properties -->
+      <extension>
+        <groupId>kr.motd.maven</groupId>
+        <artifactId>os-maven-plugin</artifactId>
+        <version>1.4.0.Final</version>
+      </extension>
+    </extensions>
   </build>
 
   <dependencies>
@@ -99,6 +131,14 @@
       <artifactId>jsr305</artifactId>
     </dependency>
 
+    <dependency>
+      <groupId>io.netty</groupId>
+      <artifactId>netty-tcnative-boringssl-static</artifactId>
+      <version>1.1.33.Fork13</version>
+      <classifier>${os.detected.classifier}</classifier>
+      <scope>runtime</scope>
+    </dependency>
+
     <!--  test -->
     <dependency>
       <groupId>org.apache.beam</groupId>
@@ -115,6 +155,13 @@
     </dependency>
 
     <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-runners-google-cloud-dataflow-java</artifactId>
+      <version>${project.version}</version>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
       <groupId>org.hamcrest</groupId>
       <artifactId>hamcrest-all</artifactId>
       <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/067c92a7/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableTestOptions.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableTestOptions.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableTestOptions.java
new file mode 100644
index 0000000..0cd4f57
--- /dev/null
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableTestOptions.java
@@ -0,0 +1,42 @@
+/*
+ * 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.bigtable;
+
+import org.apache.beam.sdk.options.Default;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.testing.TestPipelineOptions;
+
+/**
+ * Properties needed when using Bigtable with the Beam SDK.
+ */
+public interface BigtableTestOptions extends TestPipelineOptions {
+  @Description("Project ID for Bigtable")
+  @Default.String("apache-beam-testing")
+  String getProjectId();
+  void setProjectId(String value);
+
+  @Description("Cluster ID for Bigtable")
+  @Default.String("beam-test")
+  String getClusterId();
+  void setClusterId(String value);
+
+  @Description("Zone ID for Bigtable")
+  @Default.String("us-central1-c")
+  String getZoneId();
+  void setZoneId(String value);
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/067c92a7/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableReadIT.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableReadIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableReadIT.java
new file mode 100644
index 0000000..22d5b5b
--- /dev/null
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableReadIT.java
@@ -0,0 +1,61 @@
+/*
+ * 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.bigtable;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Count;
+import org.apache.beam.sdk.values.PCollection;
+
+import com.google.bigtable.v1.Row;
+import com.google.cloud.bigtable.config.BigtableOptions;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * End-to-end tests of BigtableRead.
+ */
+@RunWith(JUnit4.class)
+public class BigtableReadIT {
+
+  @Test
+  public void testE2EBigtableRead() throws Exception {
+    PipelineOptionsFactory.register(BigtableTestOptions.class);
+    BigtableTestOptions options = TestPipeline.testingPipelineOptions()
+        .as(BigtableTestOptions.class);
+
+    BigtableOptions.Builder bigtableOptionsBuilder = new BigtableOptions.Builder()
+        .setProjectId(options.getProjectId())
+        .setClusterId(options.getClusterId())
+        .setZoneId(options.getZoneId());
+
+    final String tableId = "BigtableReadTest";
+    final long numRows = 1000L;
+
+    Pipeline p = Pipeline.create(options);
+    PCollection<Long> count = p
+        .apply(BigtableIO.read().withBigtableOptions(bigtableOptionsBuilder).withTableId(tableId))
+        .apply(Count.<Row>globally());
+    PAssert.thatSingleton(count).isEqualTo(numRows);
+    p.run();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/067c92a7/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java
new file mode 100644
index 0000000..af7afc5
--- /dev/null
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java
@@ -0,0 +1,197 @@
+/*
+ * 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.bigtable;
+
+import static org.junit.Assert.assertThat;
+
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.io.CountingInput;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.KV;
+
+import com.google.bigtable.admin.table.v1.ColumnFamily;
+import com.google.bigtable.admin.table.v1.CreateTableRequest;
+import com.google.bigtable.admin.table.v1.DeleteTableRequest;
+import com.google.bigtable.admin.table.v1.GetTableRequest;
+import com.google.bigtable.admin.table.v1.Table;
+import com.google.bigtable.v1.Mutation;
+import com.google.bigtable.v1.ReadRowsRequest;
+import com.google.bigtable.v1.Row;
+import com.google.cloud.bigtable.config.BigtableOptions;
+import com.google.cloud.bigtable.grpc.BigtableSession;
+import com.google.cloud.bigtable.grpc.BigtableTableAdminClient;
+import com.google.cloud.bigtable.grpc.scanner.ResultScanner;
+import com.google.common.collect.ImmutableList;
+import com.google.protobuf.ByteString;
+
+import org.hamcrest.Matchers;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * End-to-end tests of BigtableWrite.
+ */
+@RunWith(JUnit4.class)
+public class BigtableWriteIT implements Serializable {
+  /**
+   * These tests requires a static instances because the writers go through a serialization step
+   * when executing the test and would not affect passed-in objects otherwise.
+   */
+  private static final String COLUMN_FAMILY_NAME = "cf";
+  private static BigtableTestOptions options;
+  private BigtableOptions bigtableOptions;
+  private static BigtableSession session;
+  private static BigtableTableAdminClient tableAdminClient;
+  private final String tableId =
+      String.format("BigtableWriteIT-%tF-%<tH-%<tM-%<tS-%<tL", new Date());
+
+  @Before
+  public void setup() throws Exception {
+    PipelineOptionsFactory.register(BigtableTestOptions.class);
+    options = TestPipeline.testingPipelineOptions().as(BigtableTestOptions.class);
+
+    BigtableOptions.Builder bigtableOptionsBuilder = new BigtableOptions.Builder()
+        .setProjectId(options.getProjectId())
+        .setClusterId(options.getClusterId())
+        .setZoneId(options.getZoneId())
+        .setUserAgent("apache-beam-test");
+    bigtableOptions = bigtableOptionsBuilder.build();
+
+    session = new BigtableSession(bigtableOptions);
+    tableAdminClient = session.getTableAdminClient();
+  }
+
+  @Test
+  public void testE2EBigtableWrite() throws Exception {
+    final String tableName = bigtableOptions.getClusterName().toTableNameStr(tableId);
+    final String clusterName = bigtableOptions.getClusterName().toString();
+    final int numRows = 1000;
+    final List<KV<ByteString, ByteString>> testData = generateTableData(numRows);
+
+    createEmptyTable(clusterName, tableId);
+
+    Pipeline p = Pipeline.create(options);
+    p.apply(CountingInput.upTo(numRows))
+        .apply(ParDo.of(new DoFn<Long, KV<ByteString, Iterable<Mutation>>>() {
+          @Override
+          public void processElement(ProcessContext c) {
+            int index = c.element().intValue();
+
+            Iterable<Mutation> mutations =
+                ImmutableList.of(Mutation.newBuilder()
+                    .setSetCell(
+                        Mutation.SetCell.newBuilder()
+                            .setValue(testData.get(index).getValue())
+                            .setFamilyName(COLUMN_FAMILY_NAME))
+                    .build());
+            c.output(KV.of(testData.get(index).getKey(), mutations));
+          }
+        }))
+        .apply(BigtableIO.write()
+          .withBigtableOptions(bigtableOptions)
+          .withTableId(tableId));
+    p.run();
+
+    // Test number of column families and column family name equality
+    Table table = getTable(tableName);
+    assertThat(table.getColumnFamilies().keySet(), Matchers.hasSize(1));
+    assertThat(table.getColumnFamilies(), Matchers.hasKey(COLUMN_FAMILY_NAME));
+
+    // Test table data equality
+    List<KV<ByteString, ByteString>> tableData = getTableData(tableName);
+    assertThat(tableData, Matchers.containsInAnyOrder(testData.toArray()));
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    final String tableName = bigtableOptions.getClusterName().toTableNameStr(tableId);
+    deleteTable(tableName);
+    session.close();
+  }
+
+  ////////////////////////////////////////////////////////////////////////////////////////////
+  /** Helper function to generate KV test data. */
+  private List<KV<ByteString, ByteString>> generateTableData(int numRows) {
+    List<KV<ByteString, ByteString>> testData = new ArrayList<>(numRows);
+    for (int i = 0; i < numRows; ++i) {
+      ByteString key = ByteString.copyFromUtf8(String.format("key%09d", i));
+      ByteString value = ByteString.copyFromUtf8(String.format("value%09d", i));
+      testData.add(KV.of(key, value));
+    }
+
+    return testData;
+  }
+
+  /** Helper function to create an empty table. */
+  private void createEmptyTable(String clusterName, String tableId) {
+    Table.Builder tableBuilder = Table.newBuilder();
+    Map<String, ColumnFamily> columnFamilies = tableBuilder.getMutableColumnFamilies();
+    columnFamilies.put(COLUMN_FAMILY_NAME, ColumnFamily.newBuilder().build());
+
+    CreateTableRequest.Builder createTableRequestBuilder = CreateTableRequest.newBuilder()
+        .setName(clusterName)
+        .setTableId(tableId)
+        .setTable(tableBuilder.build());
+    tableAdminClient.createTable(createTableRequestBuilder.build());
+  }
+
+  /** Helper function to get a table. */
+  private Table getTable(String tableName) {
+    GetTableRequest.Builder getTableRequestBuilder = GetTableRequest.newBuilder()
+        .setName(tableName);
+    return tableAdminClient.getTable(getTableRequestBuilder.build());
+  }
+
+  /** Helper function to get a table's data. */
+  private List<KV<ByteString, ByteString>> getTableData(String tableName) throws IOException {
+    List<KV<ByteString, ByteString>> tableData = new ArrayList<>();
+    ReadRowsRequest.Builder readRowsRequestBuilder = ReadRowsRequest.newBuilder()
+        .setTableName(tableName);
+    ResultScanner<Row> scanner = session.getDataClient().readRows(readRowsRequestBuilder.build());
+
+    Row currentRow;
+    while ((currentRow = scanner.next()) != null) {
+      ByteString key = currentRow.getKey();
+      ByteString value = currentRow.getFamilies(0).getColumns(0).getCells(0).getValue();
+      tableData.add(KV.of(key, value));
+    }
+    scanner.close();
+
+    return tableData;
+  }
+
+  /** Helper function to delete a table. */
+  private void deleteTable(String tableName) {
+    DeleteTableRequest.Builder deleteTableRequestBuilder = DeleteTableRequest.newBuilder()
+        .setName(tableName);
+    tableAdminClient.deleteTable(deleteTableRequestBuilder.build());
+  }
+}


[05/50] [abbrv] incubator-beam git commit: Use only WindowFn in TriggerTester

Posted by lc...@apache.org.
Use only WindowFn in TriggerTester

This change is preparatory for separating trigger syntax
from implementation.

Previously, the whole WindowingStrategy was passed in, but
not used. Since the tester is really a test of the state
machine, it will be moved to runners-core alongside the trigger
implementation. The requirement to provide a WindowingStrategy
with the original syntax is extraneous.


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

Branch: refs/heads/runners-spark2
Commit: ebea5a7342885f17b15f11601b6c126fc4454039
Parents: 62fe4ef
Author: Kenneth Knowles <kl...@google.com>
Authored: Thu Jun 23 21:24:22 2016 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Jul 6 10:18:49 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/beam/sdk/util/ReduceFnRunner.java    |  3 ++-
 .../org/apache/beam/sdk/util/TriggerContextFactory.java  | 11 ++++++-----
 .../java/org/apache/beam/sdk/util/TriggerTester.java     |  2 +-
 3 files changed, 9 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ebea5a73/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnRunner.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnRunner.java
index 864e8e7..2efc859 100644
--- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnRunner.java
+++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnRunner.java
@@ -236,7 +236,8 @@ public class ReduceFnRunner<K, InputT, OutputT, W extends BoundedWindow> {
     this.triggerRunner =
         new TriggerRunner<>(
             windowingStrategy.getTrigger(),
-            new TriggerContextFactory<>(windowingStrategy, stateInternals, activeWindows));
+            new TriggerContextFactory<>(
+                windowingStrategy.getWindowFn(), stateInternals, activeWindows));
   }
 
   private ActiveWindowSet<W> createActiveWindowSet() {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ebea5a73/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TriggerContextFactory.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TriggerContextFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TriggerContextFactory.java
index 4855654..f7635d3 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TriggerContextFactory.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TriggerContextFactory.java
@@ -22,6 +22,7 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.transforms.windowing.Trigger;
 import org.apache.beam.sdk.transforms.windowing.Trigger.MergingTriggerInfo;
 import org.apache.beam.sdk.transforms.windowing.Trigger.TriggerInfo;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
 import org.apache.beam.sdk.util.state.MergingStateAccessor;
 import org.apache.beam.sdk.util.state.State;
 import org.apache.beam.sdk.util.state.StateAccessor;
@@ -50,19 +51,19 @@ import javax.annotation.Nullable;
  */
 public class TriggerContextFactory<W extends BoundedWindow> {
 
-  private final WindowingStrategy<?, W> windowingStrategy;
+  private final WindowFn<?, W> windowFn;
   private StateInternals<?> stateInternals;
   // Future triggers may be able to exploit the active window to state address window mapping.
   @SuppressWarnings("unused")
   private ActiveWindowSet<W> activeWindows;
   private final Coder<W> windowCoder;
 
-  public TriggerContextFactory(WindowingStrategy<?, W> windowingStrategy,
+  public TriggerContextFactory(WindowFn<?, W> windowFn,
       StateInternals<?> stateInternals, ActiveWindowSet<W> activeWindows) {
-    this.windowingStrategy = windowingStrategy;
+    this.windowFn = windowFn;
     this.stateInternals = stateInternals;
     this.activeWindows = activeWindows;
-    this.windowCoder = windowingStrategy.getWindowFn().windowCoder();
+    this.windowCoder = windowFn.windowCoder();
   }
 
   public Trigger.TriggerContext base(W window, Timers timers,
@@ -106,7 +107,7 @@ public class TriggerContextFactory<W extends BoundedWindow> {
 
     @Override
     public boolean isMerging() {
-      return !windowingStrategy.getWindowFn().isNonMerging();
+      return !windowFn.isNonMerging();
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ebea5a73/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TriggerTester.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TriggerTester.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TriggerTester.java
index ba42c37..715d3ef 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TriggerTester.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TriggerTester.java
@@ -159,7 +159,7 @@ public class TriggerTester<InputT, W extends BoundedWindow> {
     this.windowToMergeResult = new HashMap<>();
 
     this.contextFactory =
-        new TriggerContextFactory<>(windowingStrategy, stateInternals, activeWindows);
+        new TriggerContextFactory<>(windowingStrategy.getWindowFn(), stateInternals, activeWindows);
   }
 
   /**


[11/50] [abbrv] incubator-beam git commit: [BEAM-377] Validate BigQueryIO.Read is properly configured

Posted by lc...@apache.org.
[BEAM-377] Validate BigQueryIO.Read is properly configured

Previously, using withoutValidation would disable all validation,
leading to a NullPointerException if there wasn't a table or schema
provided.

The intention of the withoutValidation parameter is to bypass more
expensive (and possibly incorrect checks, such as the existence of
the table prior to pipeline execution in cases where earlier stages
create the table).

This moves the basic usage validation to always happen, while the
extended validation is still disabled by withoutValidation.


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

Branch: refs/heads/runners-spark2
Commit: a25322e1bc217e1e185dc2beed72e28d1f5a9cc1
Parents: 38dfb63
Author: Ben Chambers <bc...@google.com>
Authored: Sat Jun 25 14:11:17 2016 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Jul 6 10:18:49 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/beam/sdk/io/BigQueryIO.java | 47 ++++++++++++--------
 .../org/apache/beam/sdk/io/BigQueryIOTest.java  | 31 ++++++++++++-
 2 files changed, 58 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a25322e1/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BigQueryIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BigQueryIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BigQueryIO.java
index 1c666ed..6a36c8d 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BigQueryIO.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BigQueryIO.java
@@ -389,6 +389,12 @@ public class BigQueryIO {
     public static class Bound extends PTransform<PInput, PCollection<TableRow>> {
       @Nullable final String jsonTableRef;
       @Nullable final String query;
+
+      /**
+       * Disable validation that the table exists or the query succeeds prior to pipeline
+       * submission. Basic validation (such as ensuring that a query or table is specified) still
+       * occurs.
+       */
       final boolean validate;
       @Nullable final Boolean flattenResults;
       @Nullable final BigQueryServices testBigQueryServices;
@@ -467,7 +473,9 @@ public class BigQueryIO {
       }
 
       /**
-       * Disable table validation.
+       * Disable validation that the table exists or the query succeeds prior to pipeline
+       * submission. Basic validation (such as ensuring that a query or table is specified) still
+       * occurs.
        */
       public Bound withoutValidation() {
         return new Bound(name, query, jsonTableRef, false, flattenResults, testBigQueryServices);
@@ -491,24 +499,27 @@ public class BigQueryIO {
 
       @Override
       public void validate(PInput input) {
-        if (validate) {
-          BigQueryOptions bqOptions = input.getPipeline().getOptions().as(BigQueryOptions.class);
-
-          TableReference table = getTableWithDefaultProject(bqOptions);
-          if (table == null && query == null) {
-            throw new IllegalStateException(
-                "Invalid BigQuery read operation, either table reference or query has to be set");
-          } else if (table != null && query != null) {
-            throw new IllegalStateException("Invalid BigQuery read operation. Specifies both a"
-                + " query and a table, only one of these should be provided");
-          } else if (table != null && flattenResults != null) {
-            throw new IllegalStateException("Invalid BigQuery read operation. Specifies a"
-                + " table with a result flattening preference, which is not configurable");
-          } else if (query != null && flattenResults == null) {
-            throw new IllegalStateException("Invalid BigQuery read operation. Specifies a"
-                + " query without a result flattening preference");
-          }
+        // Even if existence validation is disabled, we need to make sure that the BigQueryIO
+        // read is properly specified.
+        BigQueryOptions bqOptions = input.getPipeline().getOptions().as(BigQueryOptions.class);
+
+        TableReference table = getTableWithDefaultProject(bqOptions);
+        if (table == null && query == null) {
+          throw new IllegalStateException(
+              "Invalid BigQuery read operation, either table reference or query has to be set");
+        } else if (table != null && query != null) {
+          throw new IllegalStateException("Invalid BigQuery read operation. Specifies both a"
+              + " query and a table, only one of these should be provided");
+        } else if (table != null && flattenResults != null) {
+          throw new IllegalStateException("Invalid BigQuery read operation. Specifies a"
+              + " table with a result flattening preference, which is not configurable");
+        } else if (query != null && flattenResults == null) {
+          throw new IllegalStateException("Invalid BigQuery read operation. Specifies a"
+              + " query without a result flattening preference");
+        }
 
+        // Only verify existence/correctness if validation is enabled.
+        if (validate) {
           // Check for source table/query presence for early failure notification.
           // Note that a presence check can fail if the table or dataset are created by earlier
           // stages of the pipeline or if a query depends on earlier stages of a pipeline. For these

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a25322e1/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BigQueryIOTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BigQueryIOTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BigQueryIOTest.java
index 2a135ec..a1daf72 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BigQueryIOTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BigQueryIOTest.java
@@ -26,8 +26,8 @@ import static com.google.common.base.Preconditions.checkArgument;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertThat;
-import static org.mockito.Mockito.anyString;
-import static org.mockito.Mockito.eq;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Matchers.eq;
 import static org.mockito.Mockito.when;
 
 import org.apache.beam.sdk.Pipeline;
@@ -473,6 +473,17 @@ public class BigQueryIOTest implements Serializable {
 
   @Test
   @Category(RunnableOnService.class)
+  public void testBuildSourceWithoutTableQueryOrValidation() {
+    Pipeline p = TestPipeline.create();
+    thrown.expect(IllegalStateException.class);
+    thrown.expectMessage(
+        "Invalid BigQuery read operation, either table reference or query has to be set");
+    p.apply(BigQueryIO.Read.withoutValidation());
+    p.run();
+  }
+
+  @Test
+  @Category(RunnableOnService.class)
   public void testBuildSourceWithTableAndQuery() {
     Pipeline p = TestPipeline.create();
     thrown.expect(IllegalStateException.class);
@@ -502,6 +513,22 @@ public class BigQueryIOTest implements Serializable {
   }
 
   @Test
+  @Category(RunnableOnService.class)
+  public void testBuildSourceWithTableAndFlattenWithoutValidation() {
+    Pipeline p = TestPipeline.create();
+    thrown.expect(IllegalStateException.class);
+    thrown.expectMessage(
+        "Invalid BigQuery read operation. Specifies a"
+              + " table with a result flattening preference, which is not configurable");
+    p.apply(
+        BigQueryIO.Read.named("ReadMyTable")
+            .from("foo.com:project:somedataset.sometable")
+            .withoutValidation()
+            .withoutResultFlattening());
+    p.run();
+  }
+
+  @Test
   @Category(NeedsRunner.class)
   public void testReadFromTable() {
     FakeBigQueryServices fakeBqServices = new FakeBigQueryServices()


[27/50] [abbrv] incubator-beam git commit: Add direct runner package-info

Posted by lc...@apache.org.
Add direct runner package-info


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

Branch: refs/heads/runners-spark2
Commit: 53c8bff4431d8bd1979bd2f363c32538a5a766b9
Parents: bc8e6fe
Author: Thomas Groh <tg...@google.com>
Authored: Tue Jun 14 16:16:39 2016 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Jul 6 10:18:51 2016 -0700

----------------------------------------------------------------------
 .../beam/runners/direct/package-info.java       | 25 ++++++++++++++++++++
 1 file changed, 25 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/53c8bff4/runners/direct-java/src/main/java/org/apache/beam/runners/direct/package-info.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/package-info.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/package-info.java
new file mode 100644
index 0000000..e6d4ead
--- /dev/null
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/package-info.java
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+/**
+ * Defines the {@link org.apache.beam.sdk.options.PipelineOptions.DirectRunner}
+ * which executes both Bounded and Unbounded {@code Pipelines} on the local machine.
+ *
+ * <p>See {@link org.apache.beam.sdk.runners} for more information about Pipeline Runners.
+ */
+package org.apache.beam.runners.direct;


[38/50] [abbrv] incubator-beam git commit: Replacing BigQuery direct calls with BigQueryServices abstraction

Posted by lc...@apache.org.
Replacing BigQuery direct calls with BigQueryServices abstraction


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

Branch: refs/heads/runners-spark2
Commit: d71f38503cc283d1fd0d444c8452c699ebe819e8
Parents: ed32d10
Author: Pei He <pe...@google.com>
Authored: Tue Jun 28 17:49:52 2016 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Jul 6 10:18:52 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/beam/sdk/io/BigQueryIO.java | 118 ++++++++-----------
 .../apache/beam/sdk/util/BigQueryServices.java  |  12 ++
 .../beam/sdk/util/BigQueryServicesImpl.java     |  37 ++++++
 .../org/apache/beam/sdk/io/BigQueryIOTest.java  |  39 ++++--
 4 files changed, 128 insertions(+), 78 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/d71f3850/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BigQueryIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BigQueryIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BigQueryIO.java
index 790e3ff..7955022 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BigQueryIO.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BigQueryIO.java
@@ -54,7 +54,6 @@ import org.apache.beam.sdk.util.BigQueryServices.DatasetService;
 import org.apache.beam.sdk.util.BigQueryServices.JobService;
 import org.apache.beam.sdk.util.BigQueryServicesImpl;
 import org.apache.beam.sdk.util.BigQueryTableInserter;
-import org.apache.beam.sdk.util.BigQueryTableRowIterator;
 import org.apache.beam.sdk.util.GcsUtil.GcsUtilFactory;
 import org.apache.beam.sdk.util.IOChannelFactory;
 import org.apache.beam.sdk.util.IOChannelUtils;
@@ -85,7 +84,6 @@ import com.google.api.services.bigquery.model.JobConfigurationQuery;
 import com.google.api.services.bigquery.model.JobReference;
 import com.google.api.services.bigquery.model.JobStatistics;
 import com.google.api.services.bigquery.model.JobStatus;
-import com.google.api.services.bigquery.model.QueryRequest;
 import com.google.api.services.bigquery.model.TableReference;
 import com.google.api.services.bigquery.model.TableRow;
 import com.google.api.services.bigquery.model.TableSchema;
@@ -386,7 +384,7 @@ public class BigQueryIO {
        */
       final boolean validate;
       @Nullable final Boolean flattenResults;
-      @Nullable final BigQueryServices testBigQueryServices;
+      @Nullable BigQueryServices bigQueryServices;
 
       private static final String QUERY_VALIDATION_FAILURE_ERROR =
           "Validation of query \"%1$s\" failed. If the query depends on an earlier stage of the"
@@ -403,18 +401,18 @@ public class BigQueryIO {
             null /* jsonTableRef */,
             true /* validate */,
             null /* flattenResults */,
-            null /* testBigQueryServices */);
+            null /* bigQueryServices */);
       }
 
       private Bound(
           String name, @Nullable String query, @Nullable String jsonTableRef, boolean validate,
-          @Nullable Boolean flattenResults, @Nullable BigQueryServices testBigQueryServices) {
+          @Nullable Boolean flattenResults, @Nullable BigQueryServices bigQueryServices) {
         super(name);
         this.jsonTableRef = jsonTableRef;
         this.query = query;
         this.validate = validate;
         this.flattenResults = flattenResults;
-        this.testBigQueryServices = testBigQueryServices;
+        this.bigQueryServices = bigQueryServices;
       }
 
       /**
@@ -434,7 +432,7 @@ public class BigQueryIO {
        */
       public Bound from(TableReference table) {
         return new Bound(
-            name, query, toJsonString(table), validate, flattenResults, testBigQueryServices);
+            name, query, toJsonString(table), validate, flattenResults, bigQueryServices);
       }
 
       /**
@@ -449,7 +447,7 @@ public class BigQueryIO {
        */
       public Bound fromQuery(String query) {
         return new Bound(name, query, jsonTableRef, validate,
-            MoreObjects.firstNonNull(flattenResults, Boolean.TRUE), testBigQueryServices);
+            MoreObjects.firstNonNull(flattenResults, Boolean.TRUE), bigQueryServices);
       }
 
       /**
@@ -458,7 +456,7 @@ public class BigQueryIO {
        * occurs.
        */
       public Bound withoutValidation() {
-        return new Bound(name, query, jsonTableRef, false, flattenResults, testBigQueryServices);
+        return new Bound(name, query, jsonTableRef, false, flattenResults, bigQueryServices);
       }
 
       /**
@@ -469,7 +467,7 @@ public class BigQueryIO {
        * from a table will cause an error during validation.
        */
       public Bound withoutResultFlattening() {
-        return new Bound(name, query, jsonTableRef, validate, false, testBigQueryServices);
+        return new Bound(name, query, jsonTableRef, validate, false, bigQueryServices);
       }
 
       @VisibleForTesting
@@ -499,36 +497,28 @@ public class BigQueryIO {
         }
 
         if (validate) {
+          BigQueryServices bqServices = getBigQueryServices();
           // Check for source table/query presence for early failure notification.
           // Note that a presence check can fail if the table or dataset are created by earlier
           // stages of the pipeline or if a query depends on earlier stages of a pipeline. For these
           // cases the withoutValidation method can be used to disable the check.
           if (table != null) {
-            verifyDatasetPresence(bqOptions, table);
-            verifyTablePresence(bqOptions, table);
+            DatasetService datasetService = bqServices.getDatasetService(bqOptions);
+            verifyDatasetPresence(datasetService, table);
+            verifyTablePresence(datasetService, table);
           }
           if (query != null) {
-            dryRunQuery(bqOptions, query);
+            JobService jobService = bqServices.getJobService(bqOptions);
+            try {
+              jobService.dryRunQuery(bqOptions.getProject(), query);
+            } catch (Exception e) {
+              throw new IllegalArgumentException(
+                  String.format(QUERY_VALIDATION_FAILURE_ERROR, query), e);
+            }
           }
         }
       }
 
-      private static void dryRunQuery(BigQueryOptions options, String query) {
-        Bigquery client = Transport.newBigQueryClient(options).build();
-        QueryRequest request = new QueryRequest();
-        request.setQuery(query);
-        request.setDryRun(true);
-
-        try {
-          BigQueryTableRowIterator.executeWithBackOff(
-              client.jobs().query(options.getProject(), request), QUERY_VALIDATION_FAILURE_ERROR,
-              query);
-        } catch (Exception e) {
-          throw new IllegalArgumentException(
-              String.format(QUERY_VALIDATION_FAILURE_ERROR, query), e);
-        }
-      }
-
       @Override
       public PCollection<TableRow> apply(PInput input) {
         String uuid = randomUUIDString();
@@ -669,11 +659,10 @@ public class BigQueryIO {
       }
 
       private BigQueryServices getBigQueryServices() {
-        if (testBigQueryServices != null) {
-          return testBigQueryServices;
-        } else {
-          return new BigQueryServicesImpl();
+        if (bigQueryServices == null) {
+          bigQueryServices = new BigQueryServicesImpl();
         }
+        return bigQueryServices;
       }
     }
 
@@ -1443,8 +1432,7 @@ public class BigQueryIO {
       // An option to indicate if table validation is desired. Default is true.
       final boolean validate;
 
-      // A fake or mock BigQueryServices for tests.
-      @Nullable private BigQueryServices testBigQueryServices;
+      @Nullable private BigQueryServices bigQueryServices;
 
       private static class TranslateTableSpecFunction implements
           SerializableFunction<BoundedWindow, TableReference> {
@@ -1475,14 +1463,14 @@ public class BigQueryIO {
             CreateDisposition.CREATE_IF_NEEDED,
             WriteDisposition.WRITE_EMPTY,
             true /* validate */,
-            null /* testBigQueryServices */);
+            null /* bigQueryServices */);
       }
 
       private Bound(String name, @Nullable String jsonTableRef,
           @Nullable SerializableFunction<BoundedWindow, TableReference> tableRefFunction,
           @Nullable String jsonSchema,
           CreateDisposition createDisposition, WriteDisposition writeDisposition, boolean validate,
-          @Nullable BigQueryServices testBigQueryServices) {
+          @Nullable BigQueryServices bigQueryServices) {
         super(name);
         this.jsonTableRef = jsonTableRef;
         this.tableRefFunction = tableRefFunction;
@@ -1490,7 +1478,7 @@ public class BigQueryIO {
         this.createDisposition = checkNotNull(createDisposition, "createDisposition");
         this.writeDisposition = checkNotNull(writeDisposition, "writeDisposition");
         this.validate = validate;
-        this.testBigQueryServices = testBigQueryServices;
+        this.bigQueryServices = bigQueryServices;
       }
 
       /**
@@ -1510,7 +1498,7 @@ public class BigQueryIO {
        */
       public Bound to(TableReference table) {
         return new Bound(name, toJsonString(table), tableRefFunction, jsonSchema, createDisposition,
-            writeDisposition, validate, testBigQueryServices);
+            writeDisposition, validate, bigQueryServices);
       }
 
       /**
@@ -1539,7 +1527,7 @@ public class BigQueryIO {
       public Bound toTableReference(
           SerializableFunction<BoundedWindow, TableReference> tableRefFunction) {
         return new Bound(name, jsonTableRef, tableRefFunction, jsonSchema, createDisposition,
-            writeDisposition, validate, testBigQueryServices);
+            writeDisposition, validate, bigQueryServices);
       }
 
       /**
@@ -1550,7 +1538,7 @@ public class BigQueryIO {
        */
       public Bound withSchema(TableSchema schema) {
         return new Bound(name, jsonTableRef, tableRefFunction, toJsonString(schema),
-            createDisposition, writeDisposition, validate, testBigQueryServices);
+            createDisposition, writeDisposition, validate, bigQueryServices);
       }
 
       /**
@@ -1560,7 +1548,7 @@ public class BigQueryIO {
        */
       public Bound withCreateDisposition(CreateDisposition createDisposition) {
         return new Bound(name, jsonTableRef, tableRefFunction, jsonSchema, createDisposition,
-            writeDisposition, validate, testBigQueryServices);
+            writeDisposition, validate, bigQueryServices);
       }
 
       /**
@@ -1570,7 +1558,7 @@ public class BigQueryIO {
        */
       public Bound withWriteDisposition(WriteDisposition writeDisposition) {
         return new Bound(name, jsonTableRef, tableRefFunction, jsonSchema, createDisposition,
-            writeDisposition, validate, testBigQueryServices);
+            writeDisposition, validate, bigQueryServices);
       }
 
       /**
@@ -1580,7 +1568,7 @@ public class BigQueryIO {
        */
       public Bound withoutValidation() {
         return new Bound(name, jsonTableRef, tableRefFunction, jsonSchema, createDisposition,
-            writeDisposition, false, testBigQueryServices);
+            writeDisposition, false, bigQueryServices);
       }
 
       @VisibleForTesting
@@ -1590,18 +1578,18 @@ public class BigQueryIO {
       }
 
       private static void verifyTableEmpty(
-          BigQueryOptions options,
+          DatasetService datasetService,
           TableReference table) {
         try {
-          Bigquery client = Transport.newBigQueryClient(options).build();
-          BigQueryTableInserter inserter = new BigQueryTableInserter(client, options);
-          if (!inserter.isEmpty(table)) {
+          boolean isEmpty = datasetService.isTableEmpty(
+              table.getProjectId(), table.getDatasetId(), table.getTableId());
+          if (!isEmpty) {
             throw new IllegalArgumentException(
                 "BigQuery table is not empty: " + BigQueryIO.toTableSpec(table));
           }
-        } catch (IOException e) {
+        } catch (IOException | InterruptedException e) {
           ApiErrorExtractor errorExtractor = new ApiErrorExtractor();
-          if (errorExtractor.itemNotFound(e)) {
+          if (e instanceof IOException && errorExtractor.itemNotFound((IOException) e)) {
             // Nothing to do. If the table does not exist, it is considered empty.
           } else {
             throw new RuntimeException(
@@ -1633,16 +1621,17 @@ public class BigQueryIO {
         if (jsonTableRef != null && validate) {
           TableReference table = getTableWithDefaultProject(options);
 
+          DatasetService datasetService = getBigQueryServices().getDatasetService(options);
           // Check for destination table presence and emptiness for early failure notification.
           // Note that a presence check can fail when the table or dataset is created by an earlier
           // stage of the pipeline. For these cases the #withoutValidation method can be used to
           // disable the check.
-          verifyDatasetPresence(options, table);
+          verifyDatasetPresence(datasetService, table);
           if (getCreateDisposition() == BigQueryIO.Write.CreateDisposition.CREATE_NEVER) {
-            verifyTablePresence(options, table);
+            verifyTablePresence(datasetService, table);
           }
           if (getWriteDisposition() == BigQueryIO.Write.WriteDisposition.WRITE_EMPTY) {
-            verifyTableEmpty(options, table);
+            verifyTableEmpty(datasetService, table);
           }
         }
 
@@ -1663,7 +1652,7 @@ public class BigQueryIO {
           checkArgument(
               !Strings.isNullOrEmpty(tempLocation),
               "BigQueryIO.Write needs a GCS temp location to store temp files.");
-          if (testBigQueryServices == null) {
+          if (bigQueryServices == null) {
             try {
               GcsPath.fromUri(tempLocation);
             } catch (IllegalArgumentException e) {
@@ -1789,11 +1778,10 @@ public class BigQueryIO {
       }
 
       private BigQueryServices getBigQueryServices() {
-        if (testBigQueryServices != null) {
-          return testBigQueryServices;
-        } else {
-          return new BigQueryServicesImpl();
+        if (bigQueryServices == null) {
+          bigQueryServices = new BigQueryServicesImpl();
         }
+        return bigQueryServices;
       }
     }
 
@@ -1985,12 +1973,9 @@ public class BigQueryIO {
     }
   }
 
-  private static void verifyDatasetPresence(BigQueryOptions options, TableReference table) {
+  private static void verifyDatasetPresence(DatasetService datasetService, TableReference table) {
     try {
-      Bigquery client = Transport.newBigQueryClient(options).build();
-      BigQueryTableRowIterator.executeWithBackOff(
-          client.datasets().get(table.getProjectId(), table.getDatasetId()),
-          RESOURCE_NOT_FOUND_ERROR, "dataset", BigQueryIO.toTableSpec(table));
+      datasetService.getDataset(table.getProjectId(), table.getDatasetId());
     } catch (Exception e) {
       ApiErrorExtractor errorExtractor = new ApiErrorExtractor();
       if ((e instanceof IOException) && errorExtractor.itemNotFound((IOException) e)) {
@@ -2006,12 +1991,9 @@ public class BigQueryIO {
     }
   }
 
-  private static void verifyTablePresence(BigQueryOptions options, TableReference table) {
+  private static void verifyTablePresence(DatasetService datasetService, TableReference table) {
     try {
-      Bigquery client = Transport.newBigQueryClient(options).build();
-      BigQueryTableRowIterator.executeWithBackOff(
-          client.tables().get(table.getProjectId(), table.getDatasetId(), table.getTableId()),
-          RESOURCE_NOT_FOUND_ERROR, "table", BigQueryIO.toTableSpec(table));
+      datasetService.getTable(table.getProjectId(), table.getDatasetId(), table.getTableId());
     } catch (Exception e) {
       ApiErrorExtractor errorExtractor = new ApiErrorExtractor();
       if ((e instanceof IOException) && errorExtractor.itemNotFound((IOException) e)) {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/d71f3850/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryServices.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryServices.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryServices.java
index f82edf4..514e005 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryServices.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryServices.java
@@ -116,6 +116,18 @@ public interface BigQueryServices extends Serializable {
         throws IOException, InterruptedException;
 
     /**
+     * Returns true if the table is empty.
+     */
+    boolean isTableEmpty(String projectId, String datasetId, String tableId)
+        throws IOException, InterruptedException;
+
+    /**
+     * Gets the specified {@link Dataset} resource by dataset ID.
+     */
+    Dataset getDataset(String projectId, String datasetId)
+        throws IOException, InterruptedException;
+
+    /**
      * Create a {@link Dataset} with the given {@code location} and {@code description}.
      */
     void createDataset(String projectId, String datasetId, String location, String description)

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/d71f3850/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryServicesImpl.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryServicesImpl.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryServicesImpl.java
index 01ea45f..1aadeb2 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryServicesImpl.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BigQueryServicesImpl.java
@@ -36,6 +36,7 @@ import com.google.api.services.bigquery.model.JobReference;
 import com.google.api.services.bigquery.model.JobStatistics;
 import com.google.api.services.bigquery.model.JobStatus;
 import com.google.api.services.bigquery.model.Table;
+import com.google.api.services.bigquery.model.TableDataList;
 import com.google.api.services.bigquery.model.TableReference;
 import com.google.api.services.bigquery.model.TableRow;
 import com.google.cloud.hadoop.util.ApiErrorExtractor;
@@ -307,6 +308,42 @@ public class BigQueryServicesImpl implements BigQueryServices {
           backoff);
     }
 
+    @Override
+    public boolean isTableEmpty(String projectId, String datasetId, String tableId)
+        throws IOException, InterruptedException {
+      BackOff backoff =
+          new AttemptBoundedExponentialBackOff(MAX_RPC_ATTEMPTS, INITIAL_RPC_BACKOFF_MILLIS);
+      TableDataList dataList = executeWithRetries(
+          client.tabledata().list(projectId, datasetId, tableId),
+          String.format(
+              "Unable to list table data: %s, aborting after %d retries.",
+              tableId, MAX_RPC_ATTEMPTS),
+          Sleeper.DEFAULT,
+          backoff);
+      return dataList.getRows() == null || dataList.getRows().isEmpty();
+    }
+
+    /**
+     * {@inheritDoc}
+     *
+     * <p> the RPC for at most {@code MAX_RPC_ATTEMPTS} times until it succeeds.
+     *
+     * @throws IOException if it exceeds max RPC .
+     */
+    @Override
+    public Dataset getDataset(String projectId, String datasetId)
+        throws IOException, InterruptedException {
+      BackOff backoff =
+          new AttemptBoundedExponentialBackOff(MAX_RPC_ATTEMPTS, INITIAL_RPC_BACKOFF_MILLIS);
+      return executeWithRetries(
+          client.datasets().get(projectId, datasetId),
+          String.format(
+              "Unable to get dataset: %s, aborting after %d retries.",
+              datasetId, MAX_RPC_ATTEMPTS),
+          Sleeper.DEFAULT,
+          backoff);
+    }
+
     /**
      * {@inheritDoc}
      *

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/d71f3850/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BigQueryIOTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BigQueryIOTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BigQueryIOTest.java
index f0d3fce..43bf314 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BigQueryIOTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BigQueryIOTest.java
@@ -438,14 +438,22 @@ public class BigQueryIOTest implements Serializable {
   }
 
   @Test
-  public void testValidateReadSetsDefaultProject() {
+  public void testValidateReadSetsDefaultProject() throws Exception {
+    String projectId = "someproject";
+    String datasetId = "somedataset";
     BigQueryOptions options = TestPipeline.testingPipelineOptions().as(BigQueryOptions.class);
-    options.setProject("someproject");
+    options.setProject(projectId);
+
+    FakeBigQueryServices fakeBqServices = new FakeBigQueryServices()
+        .withJobService(mockJobService)
+        .withDatasetService(mockDatasetService);
+    when(mockDatasetService.getDataset(projectId, datasetId)).thenThrow(
+        new RuntimeException("Unable to confirm BigQuery dataset presence"));
 
     Pipeline p = TestPipeline.create(options);
 
     TableReference tableRef = new TableReference();
-    tableRef.setDatasetId("somedataset");
+    tableRef.setDatasetId(datasetId);
     tableRef.setTableId("sometable");
 
     thrown.expect(RuntimeException.class);
@@ -453,7 +461,8 @@ public class BigQueryIOTest implements Serializable {
     thrown.expectMessage(
         Matchers.either(Matchers.containsString("Unable to confirm BigQuery dataset presence"))
             .or(Matchers.containsString("BigQuery dataset not found for table")));
-    p.apply(BigQueryIO.Read.from(tableRef));
+    p.apply(BigQueryIO.Read.from(tableRef)
+        .withTestServices(fakeBqServices));
   }
 
   @Test
@@ -759,15 +768,24 @@ public class BigQueryIOTest implements Serializable {
     assertThat(displayData, hasDisplayItem("validation", false));
   }
 
-  private void testWriteValidatesDataset(boolean streaming) {
+  private void testWriteValidatesDataset(boolean streaming) throws Exception {
+    String projectId = "someproject";
+    String datasetId = "somedataset";
+
     BigQueryOptions options = TestPipeline.testingPipelineOptions().as(BigQueryOptions.class);
-    options.setProject("someproject");
+    options.setProject(projectId);
     options.setStreaming(streaming);
 
+    FakeBigQueryServices fakeBqServices = new FakeBigQueryServices()
+        .withJobService(mockJobService)
+        .withDatasetService(mockDatasetService);
+    when(mockDatasetService.getDataset(projectId, datasetId)).thenThrow(
+        new RuntimeException("Unable to confirm BigQuery dataset presence"));
+
     Pipeline p = TestPipeline.create(options);
 
     TableReference tableRef = new TableReference();
-    tableRef.setDatasetId("somedataset");
+    tableRef.setDatasetId(datasetId);
     tableRef.setTableId("sometable");
 
     thrown.expect(RuntimeException.class);
@@ -779,16 +797,17 @@ public class BigQueryIOTest implements Serializable {
      .apply(BigQueryIO.Write
          .to(tableRef)
          .withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED)
-         .withSchema(new TableSchema()));
+         .withSchema(new TableSchema())
+         .withTestServices(fakeBqServices));
   }
 
   @Test
-  public void testWriteValidatesDatasetBatch() {
+  public void testWriteValidatesDatasetBatch() throws Exception {
     testWriteValidatesDataset(false);
   }
 
   @Test
-  public void testWriteValidatesDatasetStreaming() {
+  public void testWriteValidatesDatasetStreaming() throws Exception {
     testWriteValidatesDataset(true);
   }
 


[48/50] [abbrv] incubator-beam git commit: Make examples only have optional runtime deps on runners

Posted by lc...@apache.org.
Make examples only have optional runtime deps on runners


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

Branch: refs/heads/runners-spark2
Commit: 023a58e7db98659560cfae27050aaed726c7e909
Parents: ae75a5d
Author: Kenneth Knowles <kl...@google.com>
Authored: Fri Jul 1 13:45:14 2016 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Jul 6 10:18:53 2016 -0700

----------------------------------------------------------------------
 examples/java/pom.xml  | 1 +
 examples/java8/pom.xml | 2 ++
 2 files changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/023a58e7/examples/java/pom.xml
----------------------------------------------------------------------
diff --git a/examples/java/pom.xml b/examples/java/pom.xml
index 6173ce3..0d2f505 100644
--- a/examples/java/pom.xml
+++ b/examples/java/pom.xml
@@ -268,6 +268,7 @@
       <artifactId>beam-runners-direct-java</artifactId>
       <version>${project.version}</version>
       <scope>runtime</scope>
+      <optional>true</optional>
     </dependency>
 
     <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/023a58e7/examples/java8/pom.xml
----------------------------------------------------------------------
diff --git a/examples/java8/pom.xml b/examples/java8/pom.xml
index ebc6f3d..61b8cb4 100644
--- a/examples/java8/pom.xml
+++ b/examples/java8/pom.xml
@@ -170,6 +170,7 @@
       <artifactId>beam-runners-direct-java</artifactId>
       <version>${project.version}</version>
       <scope>runtime</scope>
+      <optional>true</optional>
     </dependency>
 
     <dependency>
@@ -177,6 +178,7 @@
       <artifactId>beam-runners-flink_2.10</artifactId>
       <version>${project.version}</version>
       <scope>runtime</scope>
+      <optional>true</optional>
     </dependency>
   </dependencies>
 </project>


[41/50] [abbrv] incubator-beam git commit: [BEAM-390] Update spark dependency to the most recent stable version

Posted by lc...@apache.org.
[BEAM-390] Update spark dependency to the most recent stable version


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

Branch: refs/heads/runners-spark2
Commit: a313e33764a9b966023515b8dee3c7a72e245955
Parents: 4ff5233
Author: Isma�l Mej�a <ie...@gmail.com>
Authored: Wed Jun 29 23:01:09 2016 +0200
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Jul 6 10:18:52 2016 -0700

----------------------------------------------------------------------
 runners/spark/pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a313e337/runners/spark/pom.xml
----------------------------------------------------------------------
diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml
index 94c42bd..2a38923 100644
--- a/runners/spark/pom.xml
+++ b/runners/spark/pom.xml
@@ -34,7 +34,7 @@
   <properties>
     <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
     <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
-    <spark.version>1.6.1</spark.version>
+    <spark.version>1.6.2</spark.version>
     <hadoop.version>2.2.0</hadoop.version>
     <kafka.version>0.8.2.1</kafka.version>
   </properties>


[29/50] [abbrv] incubator-beam git commit: Implement getSplitPointsConsumed() in BigtableIO

Posted by lc...@apache.org.
Implement getSplitPointsConsumed() in BigtableIO


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

Branch: refs/heads/runners-spark2
Commit: 5211c4adbdd1f6e01e4d26d2375d878312a46e8a
Parents: 89b22c8
Author: Ian Zhou <ia...@google.com>
Authored: Wed Jun 29 12:03:27 2016 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Jul 6 10:18:51 2016 -0700

----------------------------------------------------------------------
 .../beam/sdk/io/gcp/bigtable/BigtableIO.java    | 11 +++++--
 .../sdk/io/gcp/bigtable/BigtableIOTest.java     | 32 +++++++++++++++++++-
 2 files changed, 40 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5211c4ad/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java
index f725a66..cddb333 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java
@@ -765,7 +765,8 @@ public class BigtableIO {
       reader = service.createReader(getCurrentSource());
       boolean hasRecord =
           reader.start()
-              && rangeTracker.tryReturnRecordAt(true, ByteKey.of(reader.getCurrentRow().getKey()));
+              && rangeTracker.tryReturnRecordAt(true, ByteKey.of(reader.getCurrentRow().getKey()))
+              || rangeTracker.markDone();
       if (hasRecord) {
         ++recordsReturned;
       }
@@ -781,7 +782,8 @@ public class BigtableIO {
     public boolean advance() throws IOException {
       boolean hasRecord =
           reader.advance()
-              && rangeTracker.tryReturnRecordAt(true, ByteKey.of(reader.getCurrentRow().getKey()));
+              && rangeTracker.tryReturnRecordAt(true, ByteKey.of(reader.getCurrentRow().getKey()))
+              || rangeTracker.markDone();
       if (hasRecord) {
         ++recordsReturned;
       }
@@ -808,6 +810,11 @@ public class BigtableIO {
     }
 
     @Override
+    public final long getSplitPointsConsumed() {
+      return rangeTracker.getSplitPointsConsumed();
+    }
+
+    @Override
     public final synchronized BigtableSource splitAtFraction(double fraction) {
       ByteKey splitKey;
       try {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/5211c4ad/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java
index c09943b..cdbaaac 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java
@@ -25,7 +25,6 @@ import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisp
 
 import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Verify.verifyNotNull;
-
 import static org.hamcrest.Matchers.hasSize;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
@@ -33,6 +32,7 @@ import static org.junit.Assert.assertThat;
 
 import org.apache.beam.sdk.Pipeline.PipelineExecutionException;
 import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.io.BoundedSource.BoundedReader;
 import org.apache.beam.sdk.io.gcp.bigtable.BigtableIO.BigtableSource;
 import org.apache.beam.sdk.io.range.ByteKey;
 import org.apache.beam.sdk.io.range.ByteKeyRange;
@@ -490,6 +490,36 @@ public class BigtableIOTest {
     assertThat(displayData, hasDisplayItem("tableId", "fooTable"));
   }
 
+  @Test
+  public void testGetSplitPointsConsumed() throws Exception {
+    final String table = "TEST-TABLE";
+    final int numRows = 100;
+    int splitPointsConsumed = 0;
+
+    makeTableData(table, numRows);
+
+    BigtableSource source =
+        new BigtableSource(service, table, null, ByteKeyRange.ALL_KEYS, null);
+
+    BoundedReader<Row> reader = source.createReader(TestPipeline.testingPipelineOptions());
+
+    reader.start();
+    // Started, 0 split points consumed
+    assertEquals("splitPointsConsumed starting",
+        splitPointsConsumed, reader.getSplitPointsConsumed());
+
+    // Split points consumed increases for each row read
+    while (reader.advance()) {
+      assertEquals("splitPointsConsumed advancing",
+          ++splitPointsConsumed, reader.getSplitPointsConsumed());
+    }
+
+    // Reader marked as done, 100 split points consumed
+    assertEquals("splitPointsConsumed done", numRows, reader.getSplitPointsConsumed());
+
+    reader.close();
+  }
+
   ////////////////////////////////////////////////////////////////////////////////////////////
   private static final String COLUMN_FAMILY_NAME = "family";
   private static final ByteString COLUMN_NAME = ByteString.copyFromUtf8("column");


[31/50] [abbrv] incubator-beam git commit: Deprecate WindowedValue.valueInEmptyWindows

Posted by lc...@apache.org.
Deprecate WindowedValue.valueInEmptyWindows


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

Branch: refs/heads/runners-spark2
Commit: 1779bf88d42738c4a39502d0df995b8421ecf4c8
Parents: 9f79e2b
Author: Kenneth Knowles <kl...@google.com>
Authored: Wed Jun 22 07:44:45 2016 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Jul 6 10:18:52 2016 -0700

----------------------------------------------------------------------
 .../org/apache/beam/sdk/util/WindowedValue.java   | 18 ++++++++++++++----
 1 file changed, 14 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/1779bf88/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowedValue.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowedValue.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowedValue.java
index d21b6c8..676848c 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowedValue.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowedValue.java
@@ -130,7 +130,11 @@ public abstract class WindowedValue<T> {
   /**
    * Returns a {@code WindowedValue} with the given value in no windows, and the default timestamp
    * and pane.
+   *
+   * @deprecated a value in no windows technically is not "in" a PCollection. It is allowed to drop
+   *     it at any point, and benign runner implementation details could cause silent data loss.
    */
+  @Deprecated
   public static <T> WindowedValue<T> valueInEmptyWindows(T value) {
     return new ValueInEmptyWindows<T>(value, PaneInfo.NO_FIRING);
   }
@@ -138,7 +142,11 @@ public abstract class WindowedValue<T> {
   /**
    * Returns a {@code WindowedValue} with the given value in no windows, and the default timestamp
    * and the specified pane.
+   *
+   * @deprecated a value in no windows technically is not "in" a PCollection. It is allowed to drop
+   *     it at any point, and benign runner implementation details could cause silent data loss.
    */
+  @Deprecated
   public static <T> WindowedValue<T> valueInEmptyWindows(T value, PaneInfo pane) {
     return new ValueInEmptyWindows<T>(value, pane);
   }
@@ -295,11 +303,13 @@ public abstract class WindowedValue<T> {
   }
 
   /**
-   * The representation of a WindowedValue where timestamp == MIN and
-   * windows == {}.
+   * The representation of a WindowedValue where timestamp == MIN and windows == {}.
+   *
+   * @deprecated a value in no windows technically is not "in" a PCollection. It is allowed to drop
+   *     it at any point, and benign runner implementation details could cause silent data loss.
    */
-  private static class ValueInEmptyWindows<T>
-      extends MinTimestampWindowedValue<T> {
+  @Deprecated
+  private static class ValueInEmptyWindows<T> extends MinTimestampWindowedValue<T> {
     public ValueInEmptyWindows(T value, PaneInfo pane) {
       super(value, pane);
     }


[08/50] [abbrv] incubator-beam git commit: Disable integration tests in parent pom

Posted by lc...@apache.org.
Disable integration tests in parent pom


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

Branch: refs/heads/runners-spark2
Commit: d1d7a2c141798fc5708db68bf78712126ddb82df
Parents: 8b0cbf9
Author: Ian Zhou <ia...@google.com>
Authored: Fri Jun 24 13:39:57 2016 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Jul 6 10:18:49 2016 -0700

----------------------------------------------------------------------
 examples/java/pom.xml | 4 ----
 pom.xml               | 3 +++
 2 files changed, 3 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/d1d7a2c1/examples/java/pom.xml
----------------------------------------------------------------------
diff --git a/examples/java/pom.xml b/examples/java/pom.xml
index 223334f..ba1e6e5 100644
--- a/examples/java/pom.xml
+++ b/examples/java/pom.xml
@@ -34,10 +34,6 @@
 
   <packaging>jar</packaging>
 
-  <properties>
-    <skipITs>true</skipITs>
-  </properties>
-
   <build>
     <plugins>
       <plugin>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/d1d7a2c1/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 31beef1..d933e8b 100644
--- a/pom.xml
+++ b/pom.xml
@@ -98,6 +98,9 @@
     <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
     <dataflow.javadoc_opts />
 
+    <!-- Disable integration tests by default -->
+    <skipITs>true</skipITs>
+
     <!-- If updating dependencies, please update any relevant javadoc offlineLinks -->
     <avro.version>1.7.7</avro.version>
     <bigquery.version>v2-rev295-1.22.0</bigquery.version>


[32/50] [abbrv] incubator-beam git commit: Fix missing Override Annotations in the DirectRunner

Posted by lc...@apache.org.
Fix missing Override Annotations in the DirectRunner


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

Branch: refs/heads/runners-spark2
Commit: 8f7e24a2222754054be01e4c78205fedf14d11ac
Parents: 1779bf8
Author: Thomas Groh <tg...@google.com>
Authored: Wed Jun 29 13:55:53 2016 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Jul 6 10:18:52 2016 -0700

----------------------------------------------------------------------
 .../apache/beam/runners/direct/UnboundedReadEvaluatorFactory.java   | 1 +
 .../java/org/apache/beam/runners/direct/SideInputContainerTest.java | 1 +
 2 files changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8f7e24a2/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactory.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactory.java
index 9a287b7..b226a2a 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactory.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactory.java
@@ -241,6 +241,7 @@ class UnboundedReadEvaluatorFactory implements TransformEvaluatorFactory {
             GlobalWindow.INSTANCE,
             transform.getOutput().getWindowingStrategy(),
             new Runnable() {
+              @Override
               public void run() {
                 try {
                   mark.finalizeCheckpoint();

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8f7e24a2/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SideInputContainerTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SideInputContainerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SideInputContainerTest.java
index c0242ed..ec589da 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SideInputContainerTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SideInputContainerTest.java
@@ -483,6 +483,7 @@ public class SideInputContainerTest {
             Object callback = invocation.getArguments()[3];
             final Runnable callbackRunnable = (Runnable) callback;
             Executors.newSingleThreadExecutor().submit(new Runnable() {
+              @Override
               public void run() {
                 try {
                   if (!runLatch.await(1500L, TimeUnit.MILLISECONDS)) {


[35/50] [abbrv] incubator-beam git commit: Make PTransform.apply Abstract

Posted by lc...@apache.org.
Make PTransform.apply Abstract

Primitive PTransforms should create the appropriate primitive output
type. Composite PTransforms should override apply.


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

Branch: refs/heads/runners-spark2
Commit: 9a17d496bc67b0ebd601b717cc810d290142481f
Parents: 8f7e24a
Author: Thomas Groh <tg...@google.com>
Authored: Wed Jun 29 16:12:21 2016 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Jul 6 10:18:52 2016 -0700

----------------------------------------------------------------------
 .../runners/direct/CommittedResultTest.java     |  4 ++++
 .../apache/beam/sdk/transforms/PTransform.java  | 20 ++++----------------
 .../beam/sdk/transforms/PTransformTest.java     |  7 ++++++-
 .../display/DisplayDataMatchersTest.java        | 18 ++++++++++++++++--
 .../sdk/transforms/display/DisplayDataTest.java |  6 +++++-
 .../apache/beam/sdk/util/StringUtilsTest.java   | 19 +++++++++++++++++--
 6 files changed, 52 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9a17d496/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CommittedResultTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CommittedResultTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CommittedResultTest.java
index 1e51b55..c0c06e5 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CommittedResultTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CommittedResultTest.java
@@ -52,6 +52,10 @@ public class CommittedResultTest implements Serializable {
   private transient PCollection<Integer> created = p.apply(Create.of(1, 2));
   private transient AppliedPTransform<?, ?, ?> transform =
       AppliedPTransform.of("foo", p.begin(), PDone.in(p), new PTransform<PBegin, PDone>() {
+        @Override
+        public PDone apply(PBegin begin) {
+          throw new IllegalArgumentException("Should never be applied");
+        }
       });
   private transient BundleFactory bundleFactory = ImmutableListBundleFactory.create();
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9a17d496/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PTransform.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PTransform.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PTransform.java
index a56eefc..fe6e8ad 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PTransform.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PTransform.java
@@ -181,18 +181,8 @@ public abstract class PTransform<InputT extends PInput, OutputT extends POutput>
    * transforms, which do not apply any transforms internally, should return
    * a new unbound output and register evaluators (via backend-specific
    * registration methods).
-   *
-   * <p>The default implementation throws an exception.  A derived class must
-   * either implement apply, or else each runner must supply a custom
-   * implementation via
-   * {@link org.apache.beam.sdk.runners.PipelineRunner#apply}.
    */
-  public OutputT apply(InputT input) {
-    throw new IllegalArgumentException(
-        "Runner " + input.getPipeline().getRunner()
-            + " has not registered an implementation for the required primitive operation "
-            + this);
-  }
+  public abstract OutputT apply(InputT input);
 
   /**
    * Called before invoking apply (which may be intercepted by the runner) to
@@ -201,7 +191,7 @@ public abstract class PTransform<InputT extends PInput, OutputT extends POutput>
    *
    * <p>By default, does nothing.
    */
-  public void validate(InputT input) { }
+  public void validate(InputT input) {}
 
   /**
    * Returns the transform name.
@@ -279,8 +269,7 @@ public abstract class PTransform<InputT extends PInput, OutputT extends POutput>
    * @throws CannotProvideCoderException if no coder can be inferred
    */
   protected Coder<?> getDefaultOutputCoder() throws CannotProvideCoderException {
-    throw new CannotProvideCoderException(
-      "PTransform.getDefaultOutputCoder called.");
+    throw new CannotProvideCoderException("PTransform.getDefaultOutputCoder called.");
   }
 
   /**
@@ -319,6 +308,5 @@ public abstract class PTransform<InputT extends PInput, OutputT extends POutput>
    * to provide their own display data.
    */
   @Override
-  public void populateDisplayData(Builder builder) {
-  }
+  public void populateDisplayData(Builder builder) {}
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9a17d496/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PTransformTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PTransformTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PTransformTest.java
index 0ef13c5..f7074c3 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PTransformTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PTransformTest.java
@@ -35,7 +35,12 @@ public class PTransformTest {
   @Test
   public void testPopulateDisplayDataDefaultBehavior() {
     PTransform<PCollection<String>, PCollection<String>> transform =
-        new PTransform<PCollection<String>, PCollection<String>>() {};
+        new PTransform<PCollection<String>, PCollection<String>>() {
+          @Override
+          public PCollection<String> apply(PCollection<String> begin) {
+            throw new IllegalArgumentException("Should never be applied");
+          }
+        };
     DisplayData displayData = DisplayData.from(transform);
     assertThat(displayData.items(), empty());
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9a17d496/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataMatchersTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataMatchersTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataMatchersTest.java
index f9f2911..07029e9 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataMatchersTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataMatchersTest.java
@@ -22,7 +22,6 @@ import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasName
 import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasType;
 import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasValue;
 import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom;
-
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertThat;
@@ -79,6 +78,11 @@ public class DisplayDataMatchersTest {
 
     DisplayData data = DisplayData.from(new PTransform<PCollection<String>, PCollection<String>>() {
       @Override
+      public PCollection<String> apply(PCollection<String> input) {
+        throw new IllegalArgumentException("Should never be applied");
+      }
+
+      @Override
       public void populateDisplayData(Builder builder) {
         builder.add(DisplayData.item("foo", DisplayDataMatchersTest.class));
       }
@@ -101,7 +105,12 @@ public class DisplayDataMatchersTest {
     Matcher<DisplayData> matcher = hasDisplayItem(hasNamespace(SampleTransform.class));
 
     assertFalse(matcher.matches(DisplayData.from(
-        new PTransform<PCollection<String>, PCollection<String>>(){})));
+        new PTransform<PCollection<String>, PCollection<String>>(){
+          @Override
+          public PCollection<String> apply(PCollection<String> input) {
+            throw new IllegalArgumentException("Should never be applied");
+          }
+        })));
     assertThat(createDisplayDataWithItem("foo", "bar"), matcher);
   }
 
@@ -149,6 +158,11 @@ public class DisplayDataMatchersTest {
     }
 
     @Override
+    public PCollection<String> apply(PCollection<String> input) {
+      throw new IllegalArgumentException("Should never be applied");
+    }
+
+    @Override
     public void populateDisplayData(Builder builder) {
       builder.add(DisplayData.item(key, value));
     }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9a17d496/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataTest.java
index 88973ff..a1189bb 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataTest.java
@@ -23,7 +23,6 @@ import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasName
 import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasType;
 import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasValue;
 import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom;
-
 import static org.hamcrest.Matchers.allOf;
 import static org.hamcrest.Matchers.empty;
 import static org.hamcrest.Matchers.everyItem;
@@ -112,6 +111,11 @@ public class DisplayDataTest implements Serializable {
           Instant startTime = defaultStartTime;
 
           @Override
+          public PCollection<String> apply(PCollection<String> begin) {
+            throw new IllegalArgumentException("Should never be applied");
+          }
+
+          @Override
           public void populateDisplayData(DisplayData.Builder builder) {
             builder
                 .include(subComponent1)

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9a17d496/sdks/java/core/src/test/java/org/apache/beam/sdk/util/StringUtilsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/StringUtilsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/StringUtilsTest.java
index 408af83..7e68df9 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/StringUtilsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/StringUtilsTest.java
@@ -69,7 +69,17 @@ public class StringUtilsTest {
   }
 
   private class EmbeddedPTransform extends PTransform<PBegin, PDone> {
-    private class Bound extends PTransform<PBegin, PDone> {}
+    @Override
+    public PDone apply(PBegin begin) {
+      throw new IllegalArgumentException("Should never be applied");
+    }
+
+    private class Bound extends PTransform<PBegin, PDone> {
+      @Override
+      public PDone apply(PBegin begin) {
+        throw new IllegalArgumentException("Should never be applied");
+      }
+    }
 
     private Bound getBound() {
       return new Bound();
@@ -118,7 +128,12 @@ public class StringUtilsTest {
   @Test
   public void testPTransformNameWithAnonOuterClass() throws Exception {
     AnonymousClass anonymousClassObj = new AnonymousClass() {
-      class NamedInnerClass extends PTransform<PBegin, PDone> {}
+      class NamedInnerClass extends PTransform<PBegin, PDone> {
+        @Override
+        public PDone apply(PBegin begin) {
+          throw new IllegalArgumentException("Should never be applied");
+        }
+      }
 
       @Override
       public Object getInnerClassInstance() {


[33/50] [abbrv] incubator-beam git commit: KafkaIO tests : change category annotation from RunnableOnService to NeedsRunner

Posted by lc...@apache.org.
KafkaIO tests : change category annotation from RunnableOnService to NeedsRunner


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

Branch: refs/heads/runners-spark2
Commit: ed32d10e6474a64dd17280f0f755d6eb03c3a21b
Parents: 9a17d49
Author: Raghu Angadi <ra...@google.com>
Authored: Thu Jun 30 10:51:35 2016 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Jul 6 10:18:52 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java    | 10 +++++-----
 1 file changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ed32d10e/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java
index 7d4337d..587e3e2 100644
--- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java
+++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java
@@ -29,8 +29,8 @@ import org.apache.beam.sdk.io.Read;
 import org.apache.beam.sdk.io.UnboundedSource;
 import org.apache.beam.sdk.io.UnboundedSource.UnboundedReader;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.testing.NeedsRunner;
 import org.apache.beam.sdk.testing.PAssert;
-import org.apache.beam.sdk.testing.RunnableOnService;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.Count;
 import org.apache.beam.sdk.transforms.DoFn;
@@ -237,7 +237,7 @@ public class KafkaIOTest {
   }
 
   @Test
-  @Category(RunnableOnService.class)
+  @Category(NeedsRunner.class)
   public void testUnboundedSource() {
     Pipeline p = TestPipeline.create();
     int numElements = 1000;
@@ -252,7 +252,7 @@ public class KafkaIOTest {
   }
 
   @Test
-  @Category(RunnableOnService.class)
+  @Category(NeedsRunner.class)
   public void testUnboundedSourceWithExplicitPartitions() {
     Pipeline p = TestPipeline.create();
     int numElements = 1000;
@@ -290,7 +290,7 @@ public class KafkaIOTest {
   }
 
   @Test
-  @Category(RunnableOnService.class)
+  @Category(NeedsRunner.class)
   public void testUnboundedSourceTimestamps() {
     Pipeline p = TestPipeline.create();
     int numElements = 1000;
@@ -318,7 +318,7 @@ public class KafkaIOTest {
   }
 
   @Test
-  @Category(RunnableOnService.class)
+  @Category(NeedsRunner.class)
   public void testUnboundedSourceSplits() throws Exception {
     Pipeline p = TestPipeline.create();
     int numElements = 1000;


[43/50] [abbrv] incubator-beam git commit: Fix timestamps in GroupAlsoByWindowsProperties

Posted by lc...@apache.org.
Fix timestamps in GroupAlsoByWindowsProperties

Some of the timestamps were not adjusted when
BEAM-145 was fixed to respect the WindowFn's
timestamps.


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

Branch: refs/heads/runners-spark2
Commit: 151ff5f7119a4d81b4cb3d6c0e81f05ad5801090
Parents: acec0a5
Author: Kenneth Knowles <kl...@google.com>
Authored: Fri Jul 1 09:53:37 2016 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Jul 6 10:18:53 2016 -0700

----------------------------------------------------------------------
 .../apache/beam/sdk/util/GroupAlsoByWindowsProperties.java   | 8 ++++++--
 1 file changed, 6 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/151ff5f7/runners/core-java/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsProperties.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsProperties.java b/runners/core-java/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsProperties.java
index f653f49..fe2a495 100644
--- a/runners/core-java/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsProperties.java
+++ b/runners/core-java/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsProperties.java
@@ -165,11 +165,13 @@ public class GroupAlsoByWindowsProperties {
     TimestampedValue<KV<String, Iterable<String>>> item1 =
         Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 20)));
     assertThat(item1.getValue().getValue(), containsInAnyOrder("v1", "v2"));
+    // Timestamp adjusted by WindowFn to exceed the end of the prior sliding window
     assertThat(item1.getTimestamp(), equalTo(new Instant(10)));
 
     TimestampedValue<KV<String, Iterable<String>>> item2 =
         Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(10, 30)));
     assertThat(item2.getValue().getValue(), contains("v2"));
+    // Timestamp adjusted by WindowFn to exceed the end of the prior sliding window
     assertThat(item2.getTimestamp(), equalTo(new Instant(20)));
   }
 
@@ -218,13 +220,15 @@ public class GroupAlsoByWindowsProperties {
         Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 20)));
     assertThat(item1.getValue().getKey(), equalTo("k"));
     assertThat(item1.getValue().getValue(), equalTo(combineFn.apply(ImmutableList.of(1L, 2L, 4L))));
-    assertThat(item1.getTimestamp(), equalTo(new Instant(5L)));
+    // Timestamp adjusted by WindowFn to exceed the end of the prior sliding window
+    assertThat(item1.getTimestamp(), equalTo(new Instant(10L)));
 
     TimestampedValue<KV<String, Long>> item2 =
         Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(10, 30)));
     assertThat(item2.getValue().getKey(), equalTo("k"));
     assertThat(item2.getValue().getValue(), equalTo(combineFn.apply(ImmutableList.of(2L, 4L))));
-    assertThat(item2.getTimestamp(), equalTo(new Instant(15L)));
+    // Timestamp adjusted by WindowFn to exceed the end of the prior sliding window
+    assertThat(item2.getTimestamp(), equalTo(new Instant(20L)));
   }
 
   /**


[17/50] [abbrv] incubator-beam git commit: DisplayDataEvaluator: Add support for source transforms

Posted by lc...@apache.org.
DisplayDataEvaluator: Add support for source transforms


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

Branch: refs/heads/runners-spark2
Commit: 49bb72d91f3c8f265c12d658902fe552cca2de03
Parents: 37f4ad4
Author: Vikas Kedigehalli <vi...@google.com>
Authored: Mon Jun 27 11:14:11 2016 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Jul 6 10:18:50 2016 -0700

----------------------------------------------------------------------
 .../display/DisplayDataEvaluator.java           | 29 +++++++++++++++++---
 .../display/DisplayDataEvaluatorTest.java       | 14 ++++++++++
 2 files changed, 39 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/49bb72d9/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataEvaluator.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataEvaluator.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataEvaluator.java
index a17e06f..a78a4ad 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataEvaluator.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataEvaluator.java
@@ -25,6 +25,7 @@ import org.apache.beam.sdk.runners.TransformTreeNode;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.Create;
 import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PBegin;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.POutput;
 
@@ -79,8 +80,8 @@ public class DisplayDataEvaluator {
    * @return the set of {@link DisplayData} for primitive {@link PTransform PTransforms}.
    */
   public <InputT> Set<DisplayData> displayDataForPrimitiveTransforms(
-    final PTransform<? super PCollection<InputT>, ? extends POutput> root,
-    Coder<InputT> inputCoder) {
+      final PTransform<? super PCollection<InputT>, ? extends POutput> root,
+      Coder<InputT> inputCoder) {
 
     Create.Values<InputT> input = Create.of();
     if (inputCoder != null) {
@@ -89,9 +90,29 @@ public class DisplayDataEvaluator {
 
     Pipeline pipeline = Pipeline.create(options);
     pipeline
-      .apply(input)
-      .apply(root);
+        .apply(input)
+        .apply(root);
 
+    return displayDataForPipeline(pipeline, root);
+  }
+
+  /**
+   * Traverse the specified source {@link PTransform}, collecting {@link DisplayData} registered
+   * on the inner primitive {@link PTransform PTransforms}.
+   *
+   * @param root The source root {@link PTransform} to traverse
+   * @return the set of {@link DisplayData} for primitive source {@link PTransform PTransforms}.
+   */
+  public Set<DisplayData> displayDataForPrimitiveSourceTransforms(
+      final PTransform<? super PBegin, ? extends POutput> root) {
+    Pipeline pipeline = Pipeline.create(options);
+    pipeline
+        .apply(root);
+
+    return displayDataForPipeline(pipeline, root);
+  }
+
+  private static Set<DisplayData> displayDataForPipeline(Pipeline pipeline, PTransform root) {
     PrimitiveDisplayDataPTransformVisitor visitor = new PrimitiveDisplayDataPTransformVisitor(root);
     pipeline.traverseTopologically(visitor);
     return visitor.getPrimitivesDisplayData();

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/49bb72d9/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataEvaluatorTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataEvaluatorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataEvaluatorTest.java
index 318c116..ce32b7d 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataEvaluatorTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataEvaluatorTest.java
@@ -23,9 +23,11 @@ import static org.hamcrest.Matchers.hasItem;
 import static org.hamcrest.Matchers.not;
 import static org.junit.Assert.assertThat;
 
+import org.apache.beam.sdk.io.TextIO;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.PBegin;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.POutput;
 
@@ -92,4 +94,16 @@ public class DisplayDataEvaluatorTest implements Serializable {
 
     assertThat(displayData, hasItem(hasDisplayItem("foo")));
   }
+
+  @Test
+  public void testSourceTransform() {
+    PTransform<? super PBegin, ? extends POutput> myTransform = TextIO.Read
+        .from("foo.*")
+        .withoutValidation();
+
+    DisplayDataEvaluator evaluator = DisplayDataEvaluator.create();
+    Set<DisplayData> displayData = evaluator.displayDataForPrimitiveSourceTransforms(myTransform);
+
+    assertThat(displayData, hasItem(hasDisplayItem("filePattern", "foo.*")));
+  }
 }


[07/50] [abbrv] incubator-beam git commit: Add StaticWindows

Posted by lc...@apache.org.
Add StaticWindows

This is a windowFn that ignores the input and always assigns to the same
input. It returns the provided window as the side input window if and
only if that window is present within its set of windows.


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

Branch: refs/heads/runners-spark2
Commit: f146f06f9fb9a5d100a322b2747ccc89f13c70d3
Parents: ebea5a7
Author: Thomas Groh <tg...@google.com>
Authored: Tue Jun 21 10:44:43 2016 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Jul 6 10:18:49 2016 -0700

----------------------------------------------------------------------
 .../apache/beam/sdk/testing/StaticWindows.java  | 110 +++++++++++++++++++
 .../apache/beam/sdk/testing/WindowSupplier.java |  83 ++++++++++++++
 .../beam/sdk/testing/StaticWindowsTest.java     |  94 ++++++++++++++++
 .../beam/sdk/testing/WindowSupplierTest.java    |  89 +++++++++++++++
 4 files changed, 376 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/f146f06f/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/StaticWindows.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/StaticWindows.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/StaticWindows.java
new file mode 100644
index 0000000..08d2355
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/StaticWindows.java
@@ -0,0 +1,110 @@
+/*
+ * 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.testing;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.NonMergingWindowFn;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+
+import com.google.common.base.Supplier;
+import com.google.common.collect.Iterables;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Objects;
+
+/**
+ * A {@link WindowFn} that assigns all elements to a static collection of
+ * {@link BoundedWindow BoundedWindows}. Side inputs windowed into static windows only support
+ * main input windows in the provided collection of windows.
+ */
+final class StaticWindows extends NonMergingWindowFn<Object, BoundedWindow> {
+  private final Supplier<Collection<BoundedWindow>> windows;
+  private final Coder<BoundedWindow> coder;
+
+  private final boolean onlyExisting;
+
+  private StaticWindows(
+      Supplier<Collection<BoundedWindow>> windows,
+      Coder<BoundedWindow> coder,
+      boolean onlyExisting) {
+    this.windows = windows;
+    this.coder = coder;
+    this.onlyExisting = onlyExisting;
+  }
+
+  public static <W extends BoundedWindow> StaticWindows of(Coder<W> coder, Iterable<W> windows) {
+    checkArgument(!Iterables.isEmpty(windows), "Input windows to StaticWindows may not be empty");
+    @SuppressWarnings("unchecked")
+    StaticWindows windowFn =
+        new StaticWindows(
+            WindowSupplier.of((Coder<BoundedWindow>) coder, (Iterable<BoundedWindow>) windows),
+            (Coder<BoundedWindow>) coder,
+            false);
+    return windowFn;
+  }
+
+  public static <W extends BoundedWindow> StaticWindows of(Coder<W> coder, W window) {
+    return of(coder, Collections.singleton(window));
+  }
+
+  public StaticWindows intoOnlyExisting() {
+    return new StaticWindows(windows, coder, true);
+  }
+
+  public Collection<BoundedWindow> getWindows() {
+    return windows.get();
+  }
+
+  @Override
+  public Collection<BoundedWindow> assignWindows(AssignContext c) throws Exception {
+    if (onlyExisting) {
+      checkArgument(
+          windows.get().contains(c.window()),
+          "Tried to assign windows to an element that is not already windowed into a provided "
+              + "window when onlyExisting is set to true");
+      return Collections.singleton(c.window());
+    } else {
+      return getWindows();
+    }
+  }
+
+  @Override
+  public boolean isCompatible(WindowFn<?, ?> other) {
+    if (!(other instanceof StaticWindows)) {
+      return false;
+    }
+    StaticWindows that = (StaticWindows) other;
+    return Objects.equals(this.windows.get(), that.windows.get());
+  }
+
+  @Override
+  public Coder<BoundedWindow> windowCoder() {
+    return coder;
+  }
+
+  @Override
+  public BoundedWindow getSideInputWindow(BoundedWindow window) {
+    checkArgument(windows.get().contains(window),
+        "StaticWindows only supports side input windows for main input windows that it contains");
+    return window;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/f146f06f/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/WindowSupplier.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/WindowSupplier.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/WindowSupplier.java
new file mode 100644
index 0000000..62bc09f
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/WindowSupplier.java
@@ -0,0 +1,83 @@
+/*
+ * 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.testing;
+
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.CoderUtils;
+
+import com.google.common.base.Supplier;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+
+import java.io.Serializable;
+import java.util.Collection;
+
+/**
+ * A {@link Supplier} that returns a static set of {@link BoundedWindow BoundedWindows}. The
+ * supplier is {@link Serializable}, and handles encoding and decoding the windows with a
+ * {@link Coder} provided for the windows.
+ */
+final class WindowSupplier implements Supplier<Collection<BoundedWindow>>, Serializable {
+  private final Coder<? extends BoundedWindow> coder;
+  private final Collection<byte[]> encodedWindows;
+
+  private transient Collection<BoundedWindow> windows;
+
+  public static <W extends BoundedWindow> WindowSupplier of(Coder<W> coder, Iterable<W> windows) {
+    ImmutableSet.Builder<byte[]> windowsBuilder = ImmutableSet.builder();
+    for (W window : windows) {
+      try {
+        windowsBuilder.add(CoderUtils.encodeToByteArray(coder, window));
+      } catch (CoderException e) {
+        throw new IllegalArgumentException(
+            "Could not encode provided windows with the provided window coder", e);
+      }
+    }
+    return new WindowSupplier(coder, windowsBuilder.build());
+  }
+
+  private WindowSupplier(Coder<? extends BoundedWindow> coder, Collection<byte[]> encodedWindows) {
+    this.coder = coder;
+    this.encodedWindows = encodedWindows;
+  }
+
+  @Override
+  public Collection<BoundedWindow> get() {
+    if (windows == null) {
+      decodeWindows();
+    }
+    return windows;
+  }
+
+  private synchronized void decodeWindows() {
+    if (windows == null) {
+      ImmutableList.Builder<BoundedWindow> windowsBuilder = ImmutableList.builder();
+      for (byte[] encoded : encodedWindows) {
+        try {
+          windowsBuilder.add(CoderUtils.decodeFromByteArray(coder, encoded));
+        } catch (CoderException e) {
+          throw new IllegalArgumentException(
+              "Could not decode provided windows with the provided window coder", e);
+        }
+      }
+      this.windows = windowsBuilder.build();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/f146f06f/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/StaticWindowsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/StaticWindowsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/StaticWindowsTest.java
new file mode 100644
index 0000000..fd715dc
--- /dev/null
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/StaticWindowsTest.java
@@ -0,0 +1,94 @@
+/*
+ * 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.testing;
+
+import static org.junit.Assert.assertThat;
+
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+
+import com.google.common.collect.ImmutableList;
+
+import org.hamcrest.Matchers;
+import org.joda.time.Instant;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/**
+ * Tests for {@link StaticWindows}.
+ */
+@RunWith(JUnit4.class)
+public class StaticWindowsTest {
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
+
+  private final IntervalWindow first = new IntervalWindow(new Instant(0), new Instant(100_000L));
+  private final IntervalWindow second =
+      new IntervalWindow(new Instant(1_000_000L), GlobalWindow.INSTANCE.maxTimestamp());
+
+  @Test
+  public void singleWindowSucceeds() throws Exception {
+    WindowFn<Object, BoundedWindow> fn = StaticWindows.of(IntervalWindow.getCoder(), first);
+    assertThat(WindowFnTestUtils.assignedWindows(fn, 100L),
+        Matchers.<BoundedWindow>contains(first));
+    assertThat(WindowFnTestUtils.assignedWindows(fn, -100L),
+        Matchers.<BoundedWindow>contains(first));
+  }
+
+  @Test
+  public void multipleWindowsSucceeds() throws Exception {
+    WindowFn<Object, BoundedWindow> fn =
+        StaticWindows.of(IntervalWindow.getCoder(), ImmutableList.of(first, second));
+    assertThat(WindowFnTestUtils.assignedWindows(fn, 100L),
+        Matchers.<BoundedWindow>containsInAnyOrder(first, second));
+    assertThat(WindowFnTestUtils.assignedWindows(fn, 1_000_000_000L),
+        Matchers.<BoundedWindow>containsInAnyOrder(first, second));
+    assertThat(WindowFnTestUtils.assignedWindows(fn, -100L),
+        Matchers.<BoundedWindow>containsInAnyOrder(first, second));
+  }
+
+  @Test
+  public void getSideInputWindowIdentity() {
+    WindowFn<Object, BoundedWindow> fn =
+        StaticWindows.of(IntervalWindow.getCoder(), ImmutableList.of(first, second));
+
+    assertThat(fn.getSideInputWindow(first), Matchers.<BoundedWindow>equalTo(first));
+    assertThat(fn.getSideInputWindow(second), Matchers.<BoundedWindow>equalTo(second));
+  }
+
+  @Test
+  public void getSideInputWindowNotPresent() {
+    WindowFn<Object, BoundedWindow> fn =
+        StaticWindows.of(IntervalWindow.getCoder(), ImmutableList.of(second));
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage("contains");
+    fn.getSideInputWindow(first);
+  }
+
+  @Test
+  public void emptyIterableThrows() {
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage("may not be empty");
+    StaticWindows.of(GlobalWindow.Coder.INSTANCE, ImmutableList.<GlobalWindow>of());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/f146f06f/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/WindowSupplierTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/WindowSupplierTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/WindowSupplierTest.java
new file mode 100644
index 0000000..178c67c
--- /dev/null
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/WindowSupplierTest.java
@@ -0,0 +1,89 @@
+/*
+ * 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.testing;
+
+import static org.junit.Assert.assertThat;
+
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.util.SerializableUtils;
+
+import com.google.common.collect.ImmutableList;
+
+import org.hamcrest.Matchers;
+import org.joda.time.Instant;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.Collections;
+
+/**
+ * Tests for {@link WindowSupplier}.
+ */
+public class WindowSupplierTest {
+  private final IntervalWindow window = new IntervalWindow(new Instant(0L), new Instant(100L));
+  private final IntervalWindow otherWindow =
+      new IntervalWindow(new Instant(-100L), new Instant(100L));
+  @Rule public ExpectedException thrown = ExpectedException.none();
+
+  @Test
+  public void getReturnsProvidedWindows() {
+    assertThat(
+        WindowSupplier.of(IntervalWindow.getCoder(), ImmutableList.of(window, otherWindow)).get(),
+        Matchers.<BoundedWindow>containsInAnyOrder(otherWindow, window));
+  }
+
+  @Test
+  public void getAfterSerialization() {
+    WindowSupplier supplier =
+        WindowSupplier.of(IntervalWindow.getCoder(), ImmutableList.of(window, otherWindow));
+    assertThat(
+        SerializableUtils.clone(supplier).get(),
+        Matchers.<BoundedWindow>containsInAnyOrder(otherWindow, window));
+  }
+
+  @Test
+  public void unencodableWindowFails() {
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage("Could not encode");
+    WindowSupplier.of(
+        new FailingCoder(),
+        Collections.<BoundedWindow>singleton(window));
+  }
+
+  private static class FailingCoder extends AtomicCoder<BoundedWindow>  {
+    @Override
+    public void encode(
+        BoundedWindow value, OutputStream outStream, Context context)
+        throws CoderException, IOException {
+      throw new CoderException("Test Enccode Exception");
+    }
+
+    @Override
+    public BoundedWindow decode(
+        InputStream inStream, Context context) throws CoderException, IOException {
+      throw new CoderException("Test Decode Exception");
+    }
+  }
+}


[34/50] [abbrv] incubator-beam git commit: Update to bigtable-client-core-0.3.0 and use bulk writes

Posted by lc...@apache.org.
Update to bigtable-client-core-0.3.0 and use bulk writes

Generally more stable, plus bulk writes bring 5x write throughput in
batch jobs by more efficiently using the network.


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

Branch: refs/heads/runners-spark2
Commit: 4ff523344f82508573decaa54b81e3cef6a22e6b
Parents: 6924358
Author: Ian Zhou <ia...@google.com>
Authored: Thu Jun 16 13:52:11 2016 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Jul 6 10:18:52 2016 -0700

----------------------------------------------------------------------
 pom.xml                                         |  2 +-
 sdks/java/core/pom.xml                          |  2 +-
 sdks/java/io/google-cloud-platform/pom.xml      |  2 +-
 .../beam/sdk/io/gcp/bigtable/BigtableIO.java    | 60 +++++++++++++++++++-
 .../io/gcp/bigtable/BigtableServiceImpl.java    | 31 +++++-----
 .../sdk/io/gcp/bigtable/BigtableIOTest.java     | 59 +++++++++++++++++++
 .../sdk/io/gcp/bigtable/BigtableWriteIT.java    | 18 +++++-
 7 files changed, 150 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4ff52334/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 6f1eaac..14a9c67 100644
--- a/pom.xml
+++ b/pom.xml
@@ -119,7 +119,7 @@
     <google-cloud-bigdataoss.version>1.4.5</google-cloud-bigdataoss.version>
     <google-cloud-dataflow-java-proto-library-all.version>0.5.160304</google-cloud-dataflow-java-proto-library-all.version>
     <guava.version>19.0</guava.version>
-    <grpc.version>0.12.0</grpc.version>
+    <grpc.version>0.13.1</grpc.version>
     <hamcrest.version>1.3</hamcrest.version>
     <jackson.version>2.7.2</jackson.version>
     <findbugs.version>3.0.1</findbugs.version>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4ff52334/sdks/java/core/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/core/pom.xml b/sdks/java/core/pom.xml
index 67c7fe9..9ec8f3d 100644
--- a/sdks/java/core/pom.xml
+++ b/sdks/java/core/pom.xml
@@ -311,7 +311,7 @@
     <dependency>
       <groupId>io.netty</groupId>
       <artifactId>netty-handler</artifactId>
-      <version>4.1.0.Beta8</version>
+      <version>4.1.0.CR1</version>
     </dependency>
 
     <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4ff52334/sdks/java/io/google-cloud-platform/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/pom.xml b/sdks/java/io/google-cloud-platform/pom.xml
index c95ea71..c7e77f1 100644
--- a/sdks/java/io/google-cloud-platform/pom.xml
+++ b/sdks/java/io/google-cloud-platform/pom.xml
@@ -32,7 +32,7 @@
   <packaging>jar</packaging>
 
   <properties>
-    <bigtable.version>0.2.3</bigtable.version>
+    <bigtable.version>0.3.0</bigtable.version>
   </properties>
 
   <build>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4ff52334/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java
index cddb333..47c68dd 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java
@@ -47,6 +47,8 @@ import com.google.bigtable.v1.Row;
 import com.google.bigtable.v1.RowFilter;
 import com.google.bigtable.v1.SampleRowKeysResponse;
 import com.google.cloud.bigtable.config.BigtableOptions;
+import com.google.cloud.bigtable.config.BulkOptions;
+import com.google.cloud.bigtable.config.RetryOptions;
 import com.google.common.base.MoreObjects;
 import com.google.common.collect.ImmutableList;
 import com.google.common.util.concurrent.FutureCallback;
@@ -54,6 +56,8 @@ import com.google.common.util.concurrent.Futures;
 import com.google.protobuf.ByteString;
 import com.google.protobuf.Empty;
 
+import io.grpc.Status;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -204,6 +208,8 @@ public class BigtableIO {
       checkNotNull(optionsBuilder, "optionsBuilder");
       // TODO: is there a better way to clone a Builder? Want it to be immune from user changes.
       BigtableOptions.Builder clonedBuilder = optionsBuilder.build().toBuilder();
+      clonedBuilder.setDataChannelCount(1);
+      clonedBuilder = addRetryOptions(clonedBuilder);
       BigtableOptions optionsWithAgent = clonedBuilder.setUserAgent(getUserAgent()).build();
       return new Read(optionsWithAgent, tableId, filter, bigtableService);
     }
@@ -388,6 +394,8 @@ public class BigtableIO {
       checkNotNull(optionsBuilder, "optionsBuilder");
       // TODO: is there a better way to clone a Builder? Want it to be immune from user changes.
       BigtableOptions.Builder clonedBuilder = optionsBuilder.build().toBuilder();
+      clonedBuilder = addBulkOptions(clonedBuilder);
+      clonedBuilder = addRetryOptions(clonedBuilder);
       BigtableOptions optionsWithAgent = clonedBuilder.setUserAgent(getUserAgent()).build();
       return new Write(optionsWithAgent, tableId, bigtableService);
     }
@@ -1024,6 +1032,56 @@ public class BigtableIO {
         info.getName(),
         info.getVersion(),
         javaVersion,
-        "0.2.3" /* TODO get Bigtable client version directly from jar. */);
+        "0.3.0" /* TODO get Bigtable client version directly from jar. */);
+  }
+
+  /**
+   * A helper function to add appropriate bulk options. See
+   * <a href="https://github.com/GoogleCloudPlatform/cloud-bigtable-client/issues/899">RetryOptions
+   * toBuilder</a> for issue.
+   */
+  static BigtableOptions.Builder addBulkOptions(BigtableOptions.Builder builder) {
+    BulkOptions bulkOptions = builder.build().getBulkOptions();
+
+    BulkOptions.Builder bulkOptionsBuilder = new BulkOptions.Builder()
+        .setAsyncMutatorWorkerCount(bulkOptions.getAsyncMutatorCount())
+        .setUseBulkApi(true)
+        .setBulkMaxRowKeyCount(bulkOptions.getBulkMaxRowKeyCount())
+        .setBulkMaxRequestSize(bulkOptions.getBulkMaxRequestSize())
+        .setMaxInflightRpcs(bulkOptions.getMaxInflightRpcs())
+        .setMaxMemory(bulkOptions.getMaxMemory());
+
+    builder.setBulkOptions(bulkOptionsBuilder.build());
+    return builder;
+  }
+
+  /**
+   * A helper function to add appropriate retry options. See
+   * <a href="https://github.com/GoogleCloudPlatform/cloud-bigtable-client/issues/899">RetryOptions
+   * toBuilder</a> for issue.
+   */
+  static BigtableOptions.Builder addRetryOptions(BigtableOptions.Builder builder) {
+    RetryOptions retryOptions = builder.build().getRetryOptions();
+
+    RetryOptions.Builder retryOptionsBuilder = new RetryOptions.Builder()
+        .setEnableRetries(retryOptions.enableRetries())
+        .setInitialBackoffMillis(retryOptions.getInitialBackoffMillis())
+        .setBackoffMultiplier(retryOptions.getBackoffMultiplier())
+        .setMaxElapsedBackoffMillis(retryOptions.getMaxElaspedBackoffMillis())
+        .setStreamingBufferSize(retryOptions.getStreamingBufferSize())
+        .setStreamingBatchSize(Math.min(retryOptions.getStreamingBatchSize(),
+            retryOptions.getStreamingBufferSize() / 2))
+        .setReadPartialRowTimeoutMillis(retryOptions.getReadPartialRowTimeoutMillis())
+        .setMaxScanTimeoutRetries(retryOptions.getMaxScanTimeoutRetries())
+        .setAllowRetriesWithoutTimestamp(retryOptions.allowRetriesWithoutTimestamp());
+
+    for (Status.Code code : Status.Code.values()) {
+      if (retryOptions.isRetryable(code)) {
+        retryOptionsBuilder.addStatusToRetryOn(code);
+      }
+    }
+
+    builder.setRetryOptions(retryOptionsBuilder.build());
+    return builder;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4ff52334/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java
index 5933e13..a0e6b29 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java
@@ -30,8 +30,9 @@ import com.google.bigtable.v1.SampleRowKeysRequest;
 import com.google.bigtable.v1.SampleRowKeysResponse;
 import com.google.cloud.bigtable.config.BigtableOptions;
 import com.google.cloud.bigtable.grpc.BigtableSession;
+import com.google.cloud.bigtable.grpc.BigtableTableName;
 import com.google.cloud.bigtable.grpc.async.AsyncExecutor;
-import com.google.cloud.bigtable.grpc.async.HeapSizeManager;
+import com.google.cloud.bigtable.grpc.async.BulkMutation;
 import com.google.cloud.bigtable.grpc.scanner.ResultScanner;
 import com.google.common.base.MoreObjects;
 import com.google.common.io.Closer;
@@ -65,7 +66,7 @@ class BigtableServiceImpl implements BigtableService {
   @Override
   public BigtableWriterImpl openForWriting(String tableId) throws IOException {
     BigtableSession session = new BigtableSession(options);
-    String tableName = options.getClusterName().toTableNameStr(tableId);
+    BigtableTableName tableName = options.getClusterName().toTableName(tableId);
     return new BigtableWriterImpl(session, tableName);
   }
 
@@ -170,24 +171,23 @@ class BigtableServiceImpl implements BigtableService {
   private static class BigtableWriterImpl implements Writer {
     private BigtableSession session;
     private AsyncExecutor executor;
+    private BulkMutation bulkMutation;
     private final MutateRowRequest.Builder partialBuilder;
 
-    public BigtableWriterImpl(BigtableSession session, String tableName) {
+    public BigtableWriterImpl(BigtableSession session, BigtableTableName tableName) {
       this.session = session;
-      this.executor =
-          new AsyncExecutor(
-              session.getDataClient(),
-              new HeapSizeManager(
-                  AsyncExecutor.ASYNC_MUTATOR_MAX_MEMORY_DEFAULT,
-                  AsyncExecutor.MAX_INFLIGHT_RPCS_DEFAULT));
-
-      partialBuilder = MutateRowRequest.newBuilder().setTableName(tableName);
+      executor = session.createAsyncExecutor();
+      bulkMutation = session.createBulkMutation(tableName, executor);
+
+      partialBuilder = MutateRowRequest.newBuilder().setTableName(tableName.toString());
     }
 
     @Override
     public void close() throws IOException {
       try {
-        if (executor != null) {
+        if (bulkMutation != null) {
+          bulkMutation.flush();
+          bulkMutation = null;
           executor.flush();
           executor = null;
         }
@@ -208,12 +208,7 @@ class BigtableServiceImpl implements BigtableService {
               .setRowKey(record.getKey())
               .addAllMutations(record.getValue())
               .build();
-      try {
-        return executor.mutateRowAsync(r);
-      } catch (InterruptedException e) {
-        Thread.currentThread().interrupt();
-        throw new IOException("Write interrupted", e);
-      }
+      return bulkMutation.add(r);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4ff52334/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java
index cdbaaac..6a6197e 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java
@@ -27,6 +27,7 @@ import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Verify.verifyNotNull;
 import static org.hamcrest.Matchers.hasSize;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertThat;
 
@@ -54,6 +55,8 @@ import com.google.bigtable.v1.Row;
 import com.google.bigtable.v1.RowFilter;
 import com.google.bigtable.v1.SampleRowKeysResponse;
 import com.google.cloud.bigtable.config.BigtableOptions;
+import com.google.cloud.bigtable.config.BulkOptions;
+import com.google.cloud.bigtable.config.RetryOptions;
 import com.google.common.base.Predicate;
 import com.google.common.base.Predicates;
 import com.google.common.collect.ImmutableList;
@@ -63,6 +66,8 @@ import com.google.common.util.concurrent.ListenableFuture;
 import com.google.protobuf.ByteString;
 import com.google.protobuf.Empty;
 
+import io.grpc.Status;
+
 import org.hamcrest.Matchers;
 import org.junit.Before;
 import org.junit.Rule;
@@ -76,10 +81,12 @@ import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Comparator;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.NoSuchElementException;
+import java.util.Set;
 import java.util.SortedMap;
 import java.util.TreeMap;
 
@@ -520,6 +527,58 @@ public class BigtableIOTest {
     reader.close();
   }
 
+  @Test
+  public void testAddBulkOptions() {
+    BigtableOptions.Builder optionsBuilder = BIGTABLE_OPTIONS.toBuilder();
+    optionsBuilder = BigtableIO.addBulkOptions(optionsBuilder);
+
+    BulkOptions bulkOptions = optionsBuilder.build().getBulkOptions();
+    assertEquals(BulkOptions.BIGTABLE_ASYNC_MUTATOR_COUNT_DEFAULT,
+        bulkOptions.getAsyncMutatorCount());
+    assertEquals(true, bulkOptions.useBulkApi());
+    assertEquals(BulkOptions.BIGTABLE_BULK_MAX_ROW_KEY_COUNT_DEFAULT,
+        bulkOptions.getBulkMaxRowKeyCount());
+    assertEquals(BulkOptions.BIGTABLE_BULK_MAX_REQUEST_SIZE_BYTES_DEFAULT,
+        bulkOptions.getBulkMaxRequestSize());
+    assertEquals(BulkOptions.BIGTABLE_MAX_INFLIGHT_RPCS_PER_CHANNEL_DEFAULT
+        * optionsBuilder.getDataChannelCount(), bulkOptions.getMaxInflightRpcs());
+    assertEquals(BulkOptions.BIGTABLE_MAX_MEMORY_DEFAULT, bulkOptions.getMaxMemory());
+  }
+
+  @Test
+  public void testAddRetryOptions() {
+    final double delta = 0.0000001;
+    BigtableOptions.Builder optionsBuilder = BIGTABLE_OPTIONS.toBuilder();
+    optionsBuilder = BigtableIO.addRetryOptions(optionsBuilder);
+
+    RetryOptions retryOptions = optionsBuilder.build().getRetryOptions();
+    assertEquals(RetryOptions.DEFAULT_ENABLE_GRPC_RETRIES, retryOptions.enableRetries());
+    assertEquals(RetryOptions.DEFAULT_INITIAL_BACKOFF_MILLIS,
+        retryOptions.getInitialBackoffMillis());
+    assertEquals(RetryOptions.DEFAULT_BACKOFF_MULTIPLIER, retryOptions.getBackoffMultiplier(),
+        delta);
+    assertEquals(RetryOptions.DEFAULT_MAX_ELAPSED_BACKOFF_MILLIS,
+        retryOptions.getMaxElaspedBackoffMillis());
+    assertEquals(RetryOptions.DEFAULT_STREAMING_BUFFER_SIZE, retryOptions.getStreamingBufferSize());
+    assertEquals(RetryOptions.DEFAULT_STREAMING_BATCH_SIZE, retryOptions.getStreamingBatchSize());
+    assertEquals(RetryOptions.DEFAULT_READ_PARTIAL_ROW_TIMEOUT_MS,
+        retryOptions.getReadPartialRowTimeoutMillis());
+    assertEquals(RetryOptions.DEFAULT_MAX_SCAN_TIMEOUT_RETRIES,
+        retryOptions.getMaxScanTimeoutRetries());
+    assertFalse(retryOptions.allowRetriesWithoutTimestamp());
+
+    Set<Status.Code> statusToRetryOn = new HashSet<>();
+    for (Status.Code code : Status.Code.values()) {
+      if (retryOptions.isRetryable(code)) {
+        statusToRetryOn.add(code);
+      }
+    }
+
+    Set<Status.Code> defaultStatusToRetryOn =
+        new HashSet<>(RetryOptions.DEFAULT_ENABLE_GRPC_RETRIES_SET);
+    assertThat(statusToRetryOn, Matchers.containsInAnyOrder(defaultStatusToRetryOn.toArray()));
+  }
+
   ////////////////////////////////////////////////////////////////////////////////////////////
   private static final String COLUMN_FAMILY_NAME = "family";
   private static final ByteString COLUMN_NAME = ByteString.copyFromUtf8("column");

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4ff52334/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java
index af7afc5..8e17761 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java
@@ -35,7 +35,9 @@ import com.google.bigtable.admin.table.v1.Table;
 import com.google.bigtable.v1.Mutation;
 import com.google.bigtable.v1.ReadRowsRequest;
 import com.google.bigtable.v1.Row;
+import com.google.bigtable.v1.RowRange;
 import com.google.cloud.bigtable.config.BigtableOptions;
+import com.google.cloud.bigtable.config.RetryOptions;
 import com.google.cloud.bigtable.grpc.BigtableSession;
 import com.google.cloud.bigtable.grpc.BigtableTableAdminClient;
 import com.google.cloud.bigtable.grpc.scanner.ResultScanner;
@@ -78,11 +80,17 @@ public class BigtableWriteIT implements Serializable {
     PipelineOptionsFactory.register(BigtableTestOptions.class);
     options = TestPipeline.testingPipelineOptions().as(BigtableTestOptions.class);
 
+    // RetryOptions streamingBatchSize must be explicitly set for getTableData()
+    RetryOptions.Builder retryOptionsBuilder = new RetryOptions.Builder();
+    retryOptionsBuilder.setStreamingBatchSize(
+        retryOptionsBuilder.build().getStreamingBufferSize() / 2);
+
     BigtableOptions.Builder bigtableOptionsBuilder = new BigtableOptions.Builder()
         .setProjectId(options.getProjectId())
         .setClusterId(options.getClusterId())
         .setZoneId(options.getZoneId())
-        .setUserAgent("apache-beam-test");
+        .setUserAgent("apache-beam-test")
+        .setRetryOptions(retryOptionsBuilder.build());
     bigtableOptions = bigtableOptionsBuilder.build();
 
     session = new BigtableSession(bigtableOptions);
@@ -172,9 +180,15 @@ public class BigtableWriteIT implements Serializable {
 
   /** Helper function to get a table's data. */
   private List<KV<ByteString, ByteString>> getTableData(String tableName) throws IOException {
+    // Add empty range to avoid TARGET_NOT_SET error
+    RowRange range = RowRange.newBuilder()
+        .setStartKey(ByteString.EMPTY)
+        .setEndKey(ByteString.EMPTY)
+        .build();
     List<KV<ByteString, ByteString>> tableData = new ArrayList<>();
     ReadRowsRequest.Builder readRowsRequestBuilder = ReadRowsRequest.newBuilder()
-        .setTableName(tableName);
+        .setTableName(tableName)
+        .setRowRange(range);
     ResultScanner<Row> scanner = session.getDataClient().readRows(readRowsRequestBuilder.build());
 
     Row currentRow;


[04/50] [abbrv] incubator-beam git commit: Key with integers in GatherAllPanes

Posted by lc...@apache.org.
Key with integers in GatherAllPanes

Ensures that runners which do not support null values can handle
GatherAllPanes.


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

Branch: refs/heads/runners-spark2
Commit: 947a54250ef8b2d8e707e52934890b80119ac408
Parents: f146f06
Author: Thomas Groh <tg...@google.com>
Authored: Fri Jun 17 15:19:26 2016 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Jul 6 10:18:49 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/beam/sdk/util/GatherAllPanes.java   | 12 +++++++-----
 1 file changed, 7 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/947a5425/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GatherAllPanes.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GatherAllPanes.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GatherAllPanes.java
index ab40678..0f2ecd0 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GatherAllPanes.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GatherAllPanes.java
@@ -57,15 +57,17 @@ public class GatherAllPanes<T>
     WindowFn<?, ?> originalWindowFn = input.getWindowingStrategy().getWindowFn();
 
     return input
-        .apply(WithKeys.<Void, T>of((Void) null).withKeyType(new TypeDescriptor<Void>() {}))
-        .apply(new ReifyTimestampsAndWindows<Void, T>())
+        .apply(WithKeys.<Integer, T>of(0).withKeyType(new TypeDescriptor<Integer>() {}))
+        .apply(new ReifyTimestampsAndWindows<Integer, T>())
         .apply(
             Window.into(
-                    new IdentityWindowFn<KV<Void, WindowedValue<T>>>(
+                    new IdentityWindowFn<KV<Integer, WindowedValue<T>>>(
                         originalWindowFn.windowCoder()))
-                .triggering(Never.ever()))
+                .triggering(Never.ever())
+                .withAllowedLateness(input.getWindowingStrategy().getAllowedLateness())
+                .discardingFiredPanes())
         // all values have the same key so they all appear as a single output element
-        .apply(GroupByKey.<Void, WindowedValue<T>>create())
+        .apply(GroupByKey.<Integer, WindowedValue<T>>create())
         .apply(Values.<Iterable<WindowedValue<T>>>create())
         .setWindowingStrategyInternal(input.getWindowingStrategy());
   }


[25/50] [abbrv] incubator-beam git commit: Add Surefire Max Heap Size argument

Posted by lc...@apache.org.
Add Surefire Max Heap Size argument

Remove Travis Maven overrides. Fork in Travis CI.


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

Branch: refs/heads/runners-spark2
Commit: bc8e6fee59a63e909446cd904530d04fe1c4ddb0
Parents: da1dcf9
Author: Thomas Groh <tg...@google.com>
Authored: Tue Jun 28 09:26:31 2016 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Jul 6 10:18:51 2016 -0700

----------------------------------------------------------------------
 .travis.yml           | 9 +++++----
 pom.xml               | 4 ++++
 runners/flink/pom.xml | 4 ++--
 3 files changed, 11 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/bc8e6fee/.travis.yml
----------------------------------------------------------------------
diff --git a/.travis.yml b/.travis.yml
index 6d81689..45bad9a 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -35,16 +35,17 @@ matrix:
       env: MAVEN_OVERRIDE=""
     # On Linux, run with specific JDKs only.
     - os: linux
-      env: CUSTOM_JDK="oraclejdk8" MAVEN_OVERRIDE="-DforkCount=0"
+      env: CUSTOM_JDK="oraclejdk8" MAVEN_OVERRIDE="-DbeamSurefireArgline='-Xmx512m'"
     - os: linux
-      env: CUSTOM_JDK="oraclejdk7" MAVEN_OVERRIDE="-DforkCount=0"
+      env: CUSTOM_JDK="oraclejdk7" MAVEN_OVERRIDE="-DbeamSurefireArgline='-Xmx512m'"
     - os: linux
-      env: CUSTOM_JDK="openjdk7" MAVEN_OVERRIDE="-DforkCount=0"
+      env: CUSTOM_JDK="openjdk7" MAVEN_OVERRIDE="-DbeamSurefireArgline='-Xmx512m'"
 
 before_install:
-  - echo "MAVEN_OPTS='-Xmx2048m -XX:MaxPermSize=512m'" > ~/.mavenrc
+  - echo "MAVEN_OPTS='-Xmx1024m -XX:MaxPermSize=512m'" > ~/.mavenrc
   - if [ "$TRAVIS_OS_NAME" == "osx" ]; then export JAVA_HOME=$(/usr/libexec/java_home); fi
   - if [ "$TRAVIS_OS_NAME" == "linux" ]; then jdk_switcher use "$CUSTOM_JDK"; fi
+  - export BEAM_SUREFIRE_ARGLINE="-Xmx512m"
 
 install:
   - travis_retry mvn -B install clean -U -DskipTests=true

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/bc8e6fee/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index bf59132..6848973 100644
--- a/pom.xml
+++ b/pom.xml
@@ -101,6 +101,9 @@
     <!-- Disable integration tests by default -->
     <skipITs>true</skipITs>
 
+    <!-- Do not add additional surefire arguments by default -->
+    <beamSurefireArgline></beamSurefireArgline>
+
     <!-- If updating dependencies, please update any relevant javadoc offlineLinks -->
     <avro.version>1.7.7</avro.version>
     <bigquery.version>v2-rev295-1.22.0</bigquery.version>
@@ -790,6 +793,7 @@
             <useManifestOnlyJar>false</useManifestOnlyJar>
             <trimStackTrace>false</trimStackTrace>
             <redirectTestOutputToFile>true</redirectTestOutputToFile>
+            <argLine>${beamSurefireArgline}</argLine>
           </configuration>
           <dependencies>
             <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/bc8e6fee/runners/flink/pom.xml
----------------------------------------------------------------------
diff --git a/runners/flink/pom.xml b/runners/flink/pom.xml
index d69eb88..bea639e 100644
--- a/runners/flink/pom.xml
+++ b/runners/flink/pom.xml
@@ -73,7 +73,7 @@
           </executions>
           <configuration>
             <forkCount>1</forkCount>
-            <argLine>-Dlog4j.configuration=log4j-test.properties  -XX:-UseGCOverheadLimit</argLine>
+            <argLine>-Dlog4j.configuration=log4j-test.properties  -XX:-UseGCOverheadLimit ${beamSurefireArgline}</argLine>
           </configuration>
         </plugin>
 
@@ -82,7 +82,7 @@
           <groupId>org.apache.maven.plugins</groupId>
           <artifactId>maven-surefire-plugin</artifactId>
           <configuration>
-            <argLine>-Dlog4j.configuration=log4j-test.properties  -XX:-UseGCOverheadLimit</argLine>
+            <argLine>-Dlog4j.configuration=log4j-test.properties  -XX:-UseGCOverheadLimit ${beamSurefireArgline}</argLine>
           </configuration>
         </plugin>
 


[13/50] [abbrv] incubator-beam git commit: Increase the MESSAGES_POLLING_ATTEMPTS for slow backend starts

Posted by lc...@apache.org.
Increase the MESSAGES_POLLING_ATTEMPTS for slow backend starts

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

Branch: refs/heads/runners-spark2
Commit: a53450b539da957a35b6cd53c7a085710e657932
Parents: 2c53b19
Author: peihe <pe...@users.noreply.github.com>
Authored: Tue Jun 28 11:10:47 2016 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Jul 6 10:18:50 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a53450b5/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java
index 548d2b9..0c79a92 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java
@@ -104,7 +104,7 @@ public class DataflowPipelineJob implements PipelineResult {
   /**
    * The amount of polling attempts for job status and messages information.
    */
-  static final int MESSAGES_POLLING_ATTEMPTS = 10;
+  static final int MESSAGES_POLLING_ATTEMPTS = 12;
   static final int STATUS_POLLING_ATTEMPTS = 5;
 
   /**


[22/50] [abbrv] incubator-beam git commit: Add FindBugs analysis to the build

Posted by lc...@apache.org.
Add FindBugs analysis to the build

FindBugs is a static analysis tool which looks for common
coding issues in Java code. See: http://findbugs.sourceforge.net/

This commit integrates FindBugs into the build process using
findbugs-maven-plugin, and subscribes java-sdk-all for analysis.

The codebase currently generates many Findbugs issues, which have
been added in the findbugs-filter.xml file as a baseline. These issues
should be triaged and fixed over time.

There will be some cases where we choose to ignore issues reported by
FindBugs. In such a case, the issue should be suppressed in code via
the @SuppressFBWarnings annotation with a comment describing why it
is suppressed.


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

Branch: refs/heads/runners-spark2
Commit: 88531184da2bda8b6e5e7ed37cda950882630393
Parents: 56ec6c3
Author: Scott Wegner <sw...@google.com>
Authored: Mon May 16 15:44:45 2016 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Jul 6 10:18:51 2016 -0700

----------------------------------------------------------------------
 examples/java/pom.xml                           |   2 +-
 pom.xml                                         |  56 +-
 runners/core-java/pom.xml                       |   4 +-
 runners/direct-java/pom.xml                     |   2 +-
 runners/google-cloud-dataflow-java/pom.xml      |   2 +-
 .../src/main/resources/beam/findbugs-filter.xml | 561 +++++++++++++++++++
 sdks/java/core/pom.xml                          |   7 +-
 .../java/org/apache/beam/sdk/util/ZipFiles.java |  11 +
 sdks/java/io/google-cloud-platform/pom.xml      |  14 +-
 sdks/java/io/hdfs/pom.xml                       |   2 +-
 sdks/java/io/kafka/pom.xml                      |   2 +-
 11 files changed, 651 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/88531184/examples/java/pom.xml
----------------------------------------------------------------------
diff --git a/examples/java/pom.xml b/examples/java/pom.xml
index 3b67797..6173ce3 100644
--- a/examples/java/pom.xml
+++ b/examples/java/pom.xml
@@ -240,7 +240,7 @@
 
     <dependency>
       <groupId>com.google.code.findbugs</groupId>
-      <artifactId>jsr305</artifactId>
+      <artifactId>annotations</artifactId>
     </dependency>
 
     <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/88531184/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 6848973..6f1eaac 100644
--- a/pom.xml
+++ b/pom.xml
@@ -122,8 +122,8 @@
     <grpc.version>0.12.0</grpc.version>
     <hamcrest.version>1.3</hamcrest.version>
     <jackson.version>2.7.2</jackson.version>
+    <findbugs.version>3.0.1</findbugs.version>
     <joda.version>2.4</joda.version>
-    <jsr305.version>3.0.1</jsr305.version>
     <junit.version>4.11</junit.version>
     <mockito.version>1.9.5</mockito.version>
     <protobuf.version>3.0.0-beta-1</protobuf.version>
@@ -338,6 +338,10 @@
             <groupId>com.google.guava</groupId>
             <artifactId>guava-jdk5</artifactId>
           </exclusion>
+          <exclusion>
+            <groupId>com.google.code.findbugs</groupId>
+            <artifactId>jsr305</artifactId>
+          </exclusion>
         </exclusions>
       </dependency>
 
@@ -362,6 +366,10 @@
             <groupId>com.google.guava</groupId>
             <artifactId>guava-jdk5</artifactId>
           </exclusion>
+          <exclusion>
+            <groupId>com.google.code.findbugs</groupId>
+            <artifactId>jsr305</artifactId>
+          </exclusion>
         </exclusions>
       </dependency>
 
@@ -469,20 +477,32 @@
 
       <dependency>
         <groupId>com.google.code.findbugs</groupId>
-        <artifactId>jsr305</artifactId>
-        <version>${jsr305.version}</version>
+        <artifactId>annotations</artifactId>
+        <version>${findbugs.version}</version>
       </dependency>
 
       <dependency>
         <groupId>com.google.cloud.bigdataoss</groupId>
         <artifactId>gcsio</artifactId>
         <version>${google-cloud-bigdataoss.version}</version>
+        <exclusions>
+          <exclusion>
+            <groupId>com.google.code.findbugs</groupId>
+            <artifactId>jsr305</artifactId>
+          </exclusion>
+        </exclusions>
       </dependency>
 
       <dependency>
         <groupId>com.google.cloud.bigdataoss</groupId>
         <artifactId>util</artifactId>
         <version>${google-cloud-bigdataoss.version}</version>
+        <exclusions>
+          <exclusion>
+            <groupId>com.google.code.findbugs</groupId>
+            <artifactId>jsr305</artifactId>
+          </exclusion>
+        </exclusions>
       </dependency>
 
       <dependency>
@@ -608,6 +628,12 @@
         <artifactId>guava-testlib</artifactId>
         <version>${guava.version}</version>
         <scope>test</scope>
+        <exclusions>
+          <exclusion>
+            <groupId>com.google.code.findbugs</groupId>
+            <artifactId>jsr305</artifactId>
+          </exclusion>
+        </exclusions>
       </dependency>
 
       <dependency>
@@ -911,6 +937,30 @@
             </execution>
           </executions>
         </plugin>
+
+        <plugin>
+          <groupId>org.codehaus.mojo</groupId>
+          <artifactId>findbugs-maven-plugin</artifactId>
+          <version>${findbugs.version}</version>
+          <dependencies>
+            <dependency>
+              <groupId>org.apache.beam</groupId>
+              <artifactId>beam-sdks-java-build-tools</artifactId>
+              <version>${project.version}</version>
+            </dependency>
+          </dependencies>
+          <configuration>
+            <excludeFilterFile>beam/findbugs-filter.xml</excludeFilterFile>
+          </configuration>
+          <executions>
+            <execution>
+              <phase>test</phase>
+              <goals>
+                <goal>check</goal>
+              </goals>
+            </execution>
+          </executions>
+        </plugin>
       </plugins>
     </pluginManagement>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/88531184/runners/core-java/pom.xml
----------------------------------------------------------------------
diff --git a/runners/core-java/pom.xml b/runners/core-java/pom.xml
index 3a91f7e..c7eea4a 100644
--- a/runners/core-java/pom.xml
+++ b/runners/core-java/pom.xml
@@ -199,7 +199,7 @@
 
     <dependency>
       <groupId>com.google.code.findbugs</groupId>
-      <artifactId>jsr305</artifactId>
+      <artifactId>annotations</artifactId>
     </dependency>
 
     <dependency>
@@ -216,7 +216,7 @@
 
     <dependency>
       <groupId>com.google.code.findbugs</groupId>
-      <artifactId>jsr305</artifactId>
+      <artifactId>annotations</artifactId>
     </dependency>
 
     <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/88531184/runners/direct-java/pom.xml
----------------------------------------------------------------------
diff --git a/runners/direct-java/pom.xml b/runners/direct-java/pom.xml
index 82c1e15..2e45e1d 100644
--- a/runners/direct-java/pom.xml
+++ b/runners/direct-java/pom.xml
@@ -276,7 +276,7 @@
 
     <dependency>
       <groupId>com.google.code.findbugs</groupId>
-      <artifactId>jsr305</artifactId>
+      <artifactId>annotations</artifactId>
     </dependency>
 
     <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/88531184/runners/google-cloud-dataflow-java/pom.xml
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/pom.xml b/runners/google-cloud-dataflow-java/pom.xml
index 38ffa49..76e5f80 100644
--- a/runners/google-cloud-dataflow-java/pom.xml
+++ b/runners/google-cloud-dataflow-java/pom.xml
@@ -363,7 +363,7 @@
 
     <dependency>
       <groupId>com.google.code.findbugs</groupId>
-      <artifactId>jsr305</artifactId>
+      <artifactId>annotations</artifactId>
     </dependency>
 
     <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/88531184/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml
----------------------------------------------------------------------
diff --git a/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml b/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml
new file mode 100644
index 0000000..3f15835
--- /dev/null
+++ b/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml
@@ -0,0 +1,561 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+    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.
+-->
+<FindBugsFilter>
+  <!-- Ignored bug categories. Bugs in these categories should not block the build. -->
+  <Bug category="I18N"/>
+  <Bug pattern="DM_STRING_CTOR"/>
+  <Bug pattern="SE_NO_SERIALVERSIONID"/>
+
+
+  <!--
+          Baseline issues below. No new issues should be added to this list. Instead, suppress
+          the issue inline using @SuppressFBWarnings and a documented reason, or (preferably) fix
+          the issue.
+        -->
+  <Match>
+    <Class name="org.apache.beam.sdk.coders.AvroCoder"/>
+    <Field name="decoder"/>
+    <Bug pattern="SE_BAD_FIELD"/>
+    <!--Non-transient non-serializable instance field in serializable class-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.coders.AvroCoder"/>
+    <Field name="encoder"/>
+    <Bug pattern="SE_BAD_FIELD"/>
+    <!--Non-transient non-serializable instance field in serializable class-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.coders.AvroCoder"/>
+    <Field name="reader"/>
+    <Bug pattern="SE_BAD_FIELD"/>
+    <!--Non-transient non-serializable instance field in serializable class-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.coders.AvroCoder"/>
+    <Field name="writer"/>
+    <Bug pattern="SE_BAD_FIELD"/>
+    <!--Non-transient non-serializable instance field in serializable class-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.coders.AvroCoder"/>
+    <Field name="reader"/>
+    <Bug pattern="SE_BAD_FIELD_STORE"/>
+    <!--Non-serializable value stored into instance field of a serializable class-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.coders.AvroCoder"/>
+    <Field name="writer"/>
+    <Bug pattern="SE_BAD_FIELD_STORE"/>
+    <!--Non-serializable value stored into instance field of a serializable class-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.coders.Coder$NonDeterministicException"/>
+    <Bug pattern="NM_CLASS_NOT_EXCEPTION"/>
+    <!--Class is not derived from an Exception, even though it is named as such-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.coders.Coder$NonDeterministicException"/>
+    <Method name="getMessage"/>
+    <Bug pattern="VA_FORMAT_STRING_USES_NEWLINE"/>
+    <!--Format string should use %n rather than \n-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.coders.EntityCoder"/>
+    <Method name="decode"/>
+    <Bug pattern="RR_NOT_CHECKED"/>
+    <!--Method ignores results of InputStream.read()-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.coders.JAXBCoder"/>
+    <Method name="getContext"/>
+    <Bug pattern="DC_DOUBLECHECK"/>
+    <!--Possible double check of field-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.coders.JAXBCoder"/>
+    <Field name="jaxbContext"/>
+    <Bug pattern="IS2_INCONSISTENT_SYNC"/>
+    <!--Inconsistent synchronization-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.coders.InstantCoder$1"/>
+    <Bug pattern="HE_INHERITS_EQUALS_USE_HASHCODE"/>
+    <!--Class inherits equals() and uses Object.hashCode()-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.coders.SerializableCoder"/>
+    <Method name="equals"/>
+    <Bug pattern="NP_EQUALS_SHOULD_HANDLE_NULL_ARGUMENT"/>
+    <!--equals() method does not check for null argument-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.coders.StandardCoder"/>
+    <Method name="toString"/>
+    <Bug pattern="SBSC_USE_STRINGBUFFER_CONCATENATION"/>
+    <!--Method invokes inefficient new String(String) constructor-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.coders.StructuralByteArray"/>
+    <Method name="getValue"/>
+    <Bug pattern="EI_EXPOSE_REP"/>
+    <!--May expose internal representation by returning reference to mutable object-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.coders.StructuralByteArray"/>
+    <Method name="&lt;init&gt;"/>
+    <Bug pattern="EI_EXPOSE_REP2"/>
+    <!--May expose internal representation by incorporating reference to mutable object-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.io.BigQueryIO$StreamingWriteFn"/>
+    <Method name="finishBundle"/>
+    <Bug pattern="WMI_WRONG_MAP_ITERATOR"/>
+    <!--Inefficient use of keySet iterator instead of entrySet iterator-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.io.PubsubIO$Read$Bound$PubsubBoundedReader"/>
+    <Method name="processElement"/>
+    <Bug pattern="DMI_RANDOM_USED_ONLY_ONCE"/>
+    <!--Random object created and used only once-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.io.PubsubUnboundedSink$WriterFn"/>
+    <Method name="publishBatch"/>
+    <Bug pattern="DLS_DEAD_LOCAL_STORE"/>
+    <!--Dead store to local variable-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.io.Write$Bound"/>
+    <Method name="createWrite"/>
+    <Bug pattern="DLS_DEAD_LOCAL_STORE"/>
+    <!--Dead store to local variable-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.io.range.OffsetRangeTracker"/>
+    <Field name="done"/>
+    <Bug pattern="IS2_INCONSISTENT_SYNC"/>
+    <!--Inconsistent synchronization-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.io.range.OffsetRangeTracker"/>
+    <Field name="lastRecordStart"/>
+    <Bug pattern="IS2_INCONSISTENT_SYNC"/>
+    <!--Inconsistent synchronization-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.io.range.OffsetRangeTracker"/>
+    <Field name="offsetOfLastSplitPoint"/>
+    <Bug pattern="IS2_INCONSISTENT_SYNC"/>
+    <!--Inconsistent synchronization-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.io.range.OffsetRangeTracker"/>
+    <Field name="splitPointsSeen"/>
+    <Bug pattern="IS2_INCONSISTENT_SYNC"/>
+    <!--Inconsistent synchronization-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.io.range.OffsetRangeTracker"/>
+    <Field name="startOffset"/>
+    <Bug pattern="IS2_INCONSISTENT_SYNC"/>
+    <!--Inconsistent synchronization-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.io.range.OffsetRangeTracker"/>
+    <Field name="stopOffset"/>
+    <Bug pattern="IS2_INCONSISTENT_SYNC"/>
+    <!--Inconsistent synchronization-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.options.PipelineOptionsFactory$1"/>
+    <Method name="apply"/>
+    <Bug pattern="NP_PARAMETER_MUST_BE_NONNULL_BUT_MARKED_AS_NULLABLE"/>
+    <!--Parameter must be non-null but is marked as nullable-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.options.PipelineOptionsFactory$2"/>
+    <Method name="apply"/>
+    <Bug pattern="NP_PARAMETER_MUST_BE_NONNULL_BUT_MARKED_AS_NULLABLE"/>
+    <!--Parameter must be non-null but is marked as nullable-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.options.PipelineOptionsFactory$3"/>
+    <Method name="apply"/>
+    <Bug pattern="NP_PARAMETER_MUST_BE_NONNULL_BUT_MARKED_AS_NULLABLE"/>
+    <!--Parameter must be non-null but is marked as nullable-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.options.PipelineOptionsFactory$4"/>
+    <Method name="apply"/>
+    <Bug pattern="NP_PARAMETER_MUST_BE_NONNULL_BUT_MARKED_AS_NULLABLE"/>
+    <!--Parameter must be non-null but is marked as nullable-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.options.ProxyInvocationHandler"/>
+    <Method name="&lt;init&gt;"/>
+    <Bug pattern="DM_NEXTINT_VIA_NEXTDOUBLE"/>
+    <!--Use the nextInt method of Random rather than nextDouble to generate a random integer-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.runners.DirectPipelineRunner"/>
+    <Method name="&lt;init&gt;"/>
+    <Bug pattern="DMI_RANDOM_USED_ONLY_ONCE"/>
+    <!--Random object created and used only once-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.testing.CoderProperties"/>
+    <Field name="ALL_CONTEXTS"/>
+    <Bug pattern="MS_MUTABLE_COLLECTION_PKGPROTECT"/>
+    <!--Field is a mutable collection which should be package protected-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.testing.PAssert$IterableAssert"/>
+    <Method name="equals"/>
+    <Bug pattern="EQ_UNUSUAL"/>
+    <!--Unusual equals method -->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.testing.PAssert$IterableAssert"/>
+    <Field name="pipeline"/>
+    <Bug pattern="SE_BAD_FIELD"/>
+    <!--Non-transient non-serializable instance field in serializable class-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.testing.PAssert$PCollectionContentsAssert"/>
+    <Method name="equals"/>
+    <Bug pattern="EQ_UNUSUAL"/>
+    <!--Unusual equals method-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.testing.PAssert$PCollectionViewAssert"/>
+    <Method name="equals"/>
+    <Bug pattern="EQ_UNUSUAL"/>
+    <!--Unusual equals method-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.testing.PAssert$SingletonAssert"/>
+    <Method name="equals"/>
+    <Bug pattern="EQ_UNUSUAL"/>
+    <!--Unusual equals method -->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.testing.PAssert$SingletonAssert"/>
+    <Field name="pipeline"/>
+    <Bug pattern="SE_BAD_FIELD"/>
+    <!--Non-transient non-serializable instance field in serializable class-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.testing.SerializableMatchers$SerializableArrayViaCoder"/>
+    <Field name="value"/>
+    <Bug pattern="SE_TRANSIENT_FIELD_NOT_RESTORED"/>
+    <!--Transient field that isn't set by deserialization. -->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.testing.WindowSupplier"/>
+    <Field name="windows"/>
+    <Bug pattern="IS2_INCONSISTENT_SYNC"/>
+    <!--Inconsistent synchronization -->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.transforms.ApproximateQuantiles$ApproximateQuantilesCombineFn"/>
+    <Method name="create"/>
+    <Bug pattern="ICAST_INT_CAST_TO_DOUBLE_PASSED_TO_CEIL"/>
+    <!--Integral value cast to double and then passed to Math.ceil-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.transforms.ApproximateQuantiles$QuantileBuffer"/>
+    <Method name="compareTo"/>
+    <Bug pattern="EQ_COMPARETO_USE_OBJECT_EQUALS"/>
+    <!--Class defines compareTo(...) and uses Object.equals()-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.transforms.Create$TimestampedValues"/>
+    <Field name="elems"/>
+    <Bug pattern="SE_TRANSIENT_FIELD_NOT_RESTORED"/>
+    <!--Transient field that isn't set by deserialization. -->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.transforms.DoFnTester"/>
+    <Field name="sideOutputTags"/>
+    <Bug pattern="URF_UNREAD_FIELD"/>
+    <!--Unread field-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.transforms.Mean$CountSum"/>
+    <Method name="equals"/>
+    <Bug pattern="FE_FLOATING_POINT_EQUALITY"/>
+    <!--Test for floating point equality-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.transforms.display.DisplayData"/>
+    <Field name="entries"/>
+    <Bug pattern="SE_BAD_FIELD"/>
+    <!--Non-transient non-serializable instance field in serializable class-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.transforms.display.DisplayData$InternalBuilder$PopulateDisplayDataException"/>
+    <Bug pattern="SIC_INNER_SHOULD_BE_STATIC"/>
+    <!--Should be a static inner class-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.transforms.join.CoGbkResult$UnionValueIterator"/>
+    <Method name="hasNext"/>
+    <Bug pattern="RC_REF_COMPARISON_BAD_PRACTICE_BOOLEAN"/>
+    <!--Suspicious reference comparison of Boolean values-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.transforms.windowing.AfterDelayFromFirstElement"/>
+    <Method name="prefetchOnElement"/>
+    <Bug pattern="RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT"/>
+    <!--Return value of method without side effect is ignored-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.transforms.windowing.AfterDelayFromFirstElement"/>
+    <Method name="prefetchShouldFire"/>
+    <Bug pattern="RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT"/>
+    <!--Return value of method without side effect is ignored-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.transforms.windowing.AfterPane"/>
+    <Method name="prefetchShouldFire"/>
+    <Bug pattern="RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT"/>
+    <!--Return value of method without side effect is ignored-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.util.AttemptAndTimeBoundedExponentialBackOff"/>
+    <Method name="reset"/>
+    <Bug pattern="UR_UNINIT_READ_CALLED_FROM_SUPER_CONSTRUCTOR"/>
+    <!--Uninitialized read of field method called from constructor of superclass-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.util.AttemptAndTimeBoundedExponentialBackOff"/>
+    <Method name="reset"/>
+    <Bug pattern="UR_UNINIT_READ_CALLED_FROM_SUPER_CONSTRUCTOR"/>
+    <!--Uninitialized read of field method called from constructor of superclass-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.util.AvroUtils$AvroMetadata"/>
+    <Method name="getSyncMarker"/>
+    <Bug pattern="EI_EXPOSE_REP"/>
+    <!--May expose internal representation by returning reference to mutable object-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.util.BitSetCoder"/>
+    <Field name="byteArrayCoder"/>
+    <Bug pattern="SE_TRANSIENT_FIELD_NOT_RESTORED"/>
+    <!--Transient field that isn't set by deserialization. -->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.util.CombineFnUtil$NonSerializableBoundedKeyedCombineFn"/>
+    <Field name="context"/>
+    <Bug pattern="SE_BAD_FIELD"/>
+    <!--Non-transient non-serializable instance field in serializable class-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.util.ExposedByteArrayInputStream"/>
+    <Method name="readAll"/>
+    <Bug pattern="EI_EXPOSE_REP"/>
+    <!--May expose internal representation by returning reference to mutable object-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.util.ExposedByteArrayOutputStream"/>
+    <Method name="toByteArray"/>
+    <Bug pattern="EI_EXPOSE_REP"/>
+    <!--May expose internal representation by returning reference to mutable object-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.util.ExposedByteArrayOutputStream"/>
+    <Method name="writeAndOwn"/>
+    <Bug pattern="EI_EXPOSE_REP2"/>
+    <!--May expose internal representation by incorporating reference to mutable object-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.util.PCollectionViews$PCollectionViewBase"/>
+    <Method name="equals"/>
+    <Bug pattern="RCN_REDUNDANT_NULLCHECK_OF_NONNULL_VALUE"/>
+    <!--Redundant nullcheck of value known to be non-null-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.util.PCollectionViews$SingletonPCollectionView"/>
+    <Field name="defaultValue"/>
+    <Bug pattern="IS2_INCONSISTENT_SYNC"/>
+    <!--Inconsistent synchronization-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.util.PaneInfoTracker$1"/>
+    <Method name="readLater"/>
+    <Bug pattern="RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT"/>
+    <!--Return value of method without side effect is ignored-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.util.PubsubClient$IncomingMessage"/>
+    <Method name="&lt;init&gt;"/>
+    <Bug pattern="EI_EXPOSE_REP2"/>
+    <!--May expose internal representation by incorporating reference to mutable object-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.util.PubsubClient$IncomingMessage"/>
+    <Method name="&lt;init&gt;"/>
+    <Bug pattern="EI_EXPOSE_REP2"/>
+    <!--May expose internal representation by incorporating reference to mutable object-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.util.PubsubClient$OutgoingMessage"/>
+    <Method name="&lt;init&gt;"/>
+    <Bug pattern="EI_EXPOSE_REP2"/>
+    <!--May expose internal representation by incorporating reference to mutable object-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.util.ReleaseInfo"/>
+    <Method name="&lt;init&gt;"/>
+    <Bug pattern="OBL_UNSATISFIED_OBLIGATION"/>
+    <!--Method may fail to clean up stream or resource-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.util.SystemReduceFn"/>
+    <Method name="prefetchOnTrigger"/>
+    <Bug pattern="RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT"/>
+    <!--Return value of method without side effect is ignored-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.util.TriggerContextFactory"/>
+    <Field name="activeWindows"/>
+    <Bug pattern="URF_UNREAD_FIELD"/>
+    <!--Unread field-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.util.TriggerRunner"/>
+    <Method name="prefetchForMerge"/>
+    <Bug pattern="RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT"/>
+    <!--Return value of method without side effect is ignored-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.util.TriggerRunner"/>
+    <Method name="prefetchForValue"/>
+    <Bug pattern="RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT"/>
+    <!--Return value of method without side effect is ignored-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.util.TriggerRunner"/>
+    <Method name="prefetchOnFire"/>
+    <Bug pattern="RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT"/>
+    <!--Return value of method without side effect is ignored-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.util.TriggerRunner"/>
+    <Method name="prefetchShouldFire"/>
+    <Bug pattern="RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT"/>
+    <!--Return value of method without side effect is ignored-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.util.ValueWithRecordId"/>
+    <Method name="getId"/>
+    <Bug pattern="EI_EXPOSE_REP"/>
+    <!--May expose internal representation by returning reference to mutable object-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.util.ValueWithRecordId"/>
+    <Method name="&lt;init&gt;"/>
+    <Bug pattern="EI_EXPOSE_REP2"/>
+    <!--May expose internal representation by incorporating reference to mutable object-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.util.WatermarkHold"/>
+    <Field name="timerInternals"/>
+    <Bug pattern="SE_BAD_FIELD"/>
+    <!--Non-transient non-serializable instance field in serializable class-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.util.WatermarkHold$1"/>
+    <Method name="readLater"/>
+    <Bug pattern="RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT"/>
+    <!--Return value of method without side effect is ignored-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.util.common.ReflectHelpers$1"/>
+    <Method name="apply"/>
+    <Bug pattern="NP_PARAMETER_MUST_BE_NONNULL_BUT_MARKED_AS_NULLABLE"/>
+    <!--Parameter must be non-null but is marked as nullable-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.util.common.ReflectHelpers$2"/>
+    <Method name="apply"/>
+    <Bug pattern="NP_PARAMETER_MUST_BE_NONNULL_BUT_MARKED_AS_NULLABLE"/>
+    <!--Parameter must be non-null but is marked as nullable-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.util.common.ReflectHelpers$3"/>
+    <Method name="apply"/>
+    <Bug pattern="NP_PARAMETER_MUST_BE_NONNULL_BUT_MARKED_AS_NULLABLE"/>
+    <!--Parameter must be non-null but is marked as nullable-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.util.common.ReflectHelpers$4"/>
+    <Method name="apply"/>
+    <Bug pattern="NP_PARAMETER_MUST_BE_NONNULL_BUT_MARKED_AS_NULLABLE"/>
+    <!--Parameter must be non-null but is marked as nullable-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.util.common.ReflectHelpers$5"/>
+    <Method name="apply"/>
+    <Bug pattern="NP_PARAMETER_MUST_BE_NONNULL_BUT_MARKED_AS_NULLABLE"/>
+    <!--Parameter must be non-null but is marked as nullable-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.util.common.ReflectHelpers$6"/>
+    <Method name="apply"/>
+    <Bug pattern="NP_PARAMETER_MUST_BE_NONNULL_BUT_MARKED_AS_NULLABLE"/>
+    <!--Parameter must be non-null but is marked as nullable-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.util.state.StateMerging"/>
+    <Method name="mergeBags"/>
+    <Bug pattern="RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT"/>
+    <!--Return value of method without side effect is ignored-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.util.state.StateMerging"/>
+    <Method name="mergeCombiningValues"/>
+    <Bug pattern="RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT"/>
+    <!--Return value of method without side effect is ignored-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.util.state.StateMerging"/>
+    <Method name="prefetchBags"/>
+    <Bug pattern="RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT"/>
+    <!--Return value of method without side effect is ignored-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.util.state.StateMerging"/>
+    <Method name="prefetchCombiningValues"/>
+    <Bug pattern="RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT"/>
+    <!--Return value of method without side effect is ignored-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.util.state.StateMerging"/>
+    <Method name="prefetchWatermarks"/>
+    <Bug pattern="RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT"/>
+    <!--Return value of method without side effect is ignored-->
+  </Match>
+  <Match>
+    <Class name="org.apache.beam.sdk.util.state.StateTags$CombiningValueStateTag"/>
+    <Method name="equals"/>
+    <Bug pattern="EQ_DOESNT_OVERRIDE_EQUALS"/>
+    <!--Class doesn't override equals in superclass-->
+  </Match>
+</FindBugsFilter>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/88531184/sdks/java/core/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/core/pom.xml b/sdks/java/core/pom.xml
index 3b650c6..67c7fe9 100644
--- a/sdks/java/core/pom.xml
+++ b/sdks/java/core/pom.xml
@@ -234,6 +234,11 @@
         </executions>
       </plugin>
 
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>findbugs-maven-plugin</artifactId>
+      </plugin>
+
       <!-- Coverage analysis for unit tests. -->
       <plugin>
         <groupId>org.jacoco</groupId>
@@ -415,7 +420,7 @@
 
     <dependency>
       <groupId>com.google.code.findbugs</groupId>
-      <artifactId>jsr305</artifactId>
+      <artifactId>annotations</artifactId>
     </dependency>
 
     <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/88531184/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ZipFiles.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ZipFiles.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ZipFiles.java
index 038b9cb..c93e18d 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ZipFiles.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ZipFiles.java
@@ -40,6 +40,8 @@ import java.util.zip.ZipEntry;
 import java.util.zip.ZipFile;
 import java.util.zip.ZipOutputStream;
 
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+
 /**
  * Functions for zipping a directory (including a subdirectory) into a ZIP-file
  * or unzipping it again.
@@ -226,6 +228,10 @@ public final class ZipFiles {
    * @throws IOException the zipping failed, e.g. because the input was not
    *     readable.
    */
+  @SuppressFBWarnings(value = "NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE",
+      justification = "File.listFiles() will return null if the File instance is not a directory. "
+          + "Null dereference is not a possibility here since we validate sourceDirectory is "
+          + "directory via sourceDirectory.isDirectory()")
   public static void zipDirectory(
       File sourceDirectory,
       OutputStream outputStream) throws IOException {
@@ -235,6 +241,7 @@ public final class ZipFiles {
         sourceDirectory.isDirectory(),
         "%s is not a valid directory",
         sourceDirectory.getAbsolutePath());
+
     ZipOutputStream zos = new ZipOutputStream(outputStream);
     for (File file : sourceDirectory.listFiles()) {
       zipDirectoryInternal(file, "", zos);
@@ -257,6 +264,10 @@ public final class ZipFiles {
    * @throws IOException the zipping failed, e.g. because the output was not
    *     writeable.
    */
+  @SuppressFBWarnings(value = "NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE",
+      justification = "File.listFiles() will return null if the File instance is not a directory. "
+          + "Null dereference is not a possibility here since we validate inputFile is directory "
+          + "via inputFile.isDirectory()")
   private static void zipDirectoryInternal(
       File inputFile,
       String directoryName,

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/88531184/sdks/java/io/google-cloud-platform/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/pom.xml b/sdks/java/io/google-cloud-platform/pom.xml
index 692ac96..c95ea71 100644
--- a/sdks/java/io/google-cloud-platform/pom.xml
+++ b/sdks/java/io/google-cloud-platform/pom.xml
@@ -112,12 +112,24 @@
       <groupId>com.google.cloud.bigtable</groupId>
       <artifactId>bigtable-protos</artifactId>
       <version>${bigtable.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>com.google.code.findbugs</groupId>
+          <artifactId>jsr305</artifactId>
+        </exclusion>
+      </exclusions>
     </dependency>
 
     <dependency>
       <groupId>com.google.cloud.bigtable</groupId>
       <artifactId>bigtable-client-core</artifactId>
       <version>${bigtable.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>com.google.code.findbugs</groupId>
+          <artifactId>jsr305</artifactId>
+        </exclusion>
+      </exclusions>
     </dependency>
 
     <dependency>
@@ -137,7 +149,7 @@
 
     <dependency>
       <groupId>com.google.code.findbugs</groupId>
-      <artifactId>jsr305</artifactId>
+      <artifactId>annotations</artifactId>
     </dependency>
 
     <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/88531184/sdks/java/io/hdfs/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/io/hdfs/pom.xml b/sdks/java/io/hdfs/pom.xml
index f2c9f68..2e427b1 100644
--- a/sdks/java/io/hdfs/pom.xml
+++ b/sdks/java/io/hdfs/pom.xml
@@ -78,7 +78,7 @@
 
     <dependency>
       <groupId>com.google.code.findbugs</groupId>
-      <artifactId>jsr305</artifactId>
+      <artifactId>annotations</artifactId>
     </dependency>
 
     <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/88531184/sdks/java/io/kafka/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/io/kafka/pom.xml b/sdks/java/io/kafka/pom.xml
index cc447cd..9777543 100644
--- a/sdks/java/io/kafka/pom.xml
+++ b/sdks/java/io/kafka/pom.xml
@@ -100,7 +100,7 @@
 
     <dependency>
       <groupId>com.google.code.findbugs</groupId>
-      <artifactId>jsr305</artifactId>
+      <artifactId>annotations</artifactId>
     </dependency>
 
     <!-- test dependencies-->


[10/50] [abbrv] incubator-beam git commit: Update test for GatherAllPanes

Posted by lc...@apache.org.
Update test for GatherAllPanes


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

Branch: refs/heads/runners-spark2
Commit: c8babc1b93d20d583a26820fd3ffff2720185180
Parents: 947a542
Author: Thomas Groh <tg...@google.com>
Authored: Mon Jun 20 14:39:54 2016 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Jul 6 10:18:49 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/beam/sdk/util/GatherAllPanesTest.java    | 8 +++++++-
 1 file changed, 7 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c8babc1b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GatherAllPanesTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GatherAllPanesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GatherAllPanesTest.java
index e9be41e..a6522ef 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GatherAllPanesTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GatherAllPanesTest.java
@@ -23,6 +23,7 @@ import org.apache.beam.sdk.io.CountingInput;
 import org.apache.beam.sdk.testing.NeedsRunner;
 import org.apache.beam.sdk.testing.PAssert;
 import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Flatten;
 import org.apache.beam.sdk.transforms.GroupByKey;
 import org.apache.beam.sdk.transforms.SerializableFunction;
 import org.apache.beam.sdk.transforms.Values;
@@ -33,6 +34,7 @@ import org.apache.beam.sdk.transforms.windowing.AfterWatermark;
 import org.apache.beam.sdk.transforms.windowing.FixedWindows;
 import org.apache.beam.sdk.transforms.windowing.Window;
 import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
 import org.apache.beam.sdk.values.TypeDescriptor;
 
 import com.google.common.collect.Iterables;
@@ -98,8 +100,12 @@ public class GatherAllPanesTest implements Serializable {
   public void multiplePanesMultipleReifiedPane() {
     TestPipeline p = TestPipeline.create();
 
+    PCollection<Long> someElems = p.apply("someLongs", CountingInput.upTo(20000));
+    PCollection<Long> otherElems = p.apply("otherLongs", CountingInput.upTo(20000));
     PCollection<Iterable<WindowedValue<Iterable<Long>>>> accumulatedPanes =
-        p.apply(CountingInput.upTo(20000))
+        PCollectionList.of(someElems)
+            .and(otherElems)
+            .apply(Flatten.<Long>pCollections())
             .apply(
                 WithTimestamps.of(
                     new SerializableFunction<Long, Instant>() {


[46/50] [abbrv] incubator-beam git commit: DatastoreIO: add Read/Write Transforms and version-specific class

Posted by lc...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/d1e38443/sdks/java/core/src/test/java/org/apache/beam/sdk/io/DatastoreIOTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/DatastoreIOTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/DatastoreIOTest.java
deleted file mode 100644
index 2aca190..0000000
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/DatastoreIOTest.java
+++ /dev/null
@@ -1,594 +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;
-
-import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem;
-
-import static com.google.datastore.v1beta3.client.DatastoreHelper.makeKey;
-
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.hamcrest.Matchers.greaterThanOrEqualTo;
-import static org.hamcrest.Matchers.lessThanOrEqualTo;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.eq;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.never;
-import static org.mockito.Mockito.spy;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.verifyNoMoreInteractions;
-import static org.mockito.Mockito.when;
-
-import org.apache.beam.sdk.io.DatastoreIO.DatastoreReader;
-import org.apache.beam.sdk.io.DatastoreIO.DatastoreWriter;
-import org.apache.beam.sdk.options.GcpOptions;
-import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.sdk.testing.ExpectedLogs;
-import org.apache.beam.sdk.transforms.display.DisplayData;
-import org.apache.beam.sdk.util.TestCredential;
-
-import com.google.common.collect.Lists;
-import com.google.datastore.v1beta3.Entity;
-import com.google.datastore.v1beta3.EntityResult;
-import com.google.datastore.v1beta3.Key;
-import com.google.datastore.v1beta3.KindExpression;
-import com.google.datastore.v1beta3.PartitionId;
-import com.google.datastore.v1beta3.PropertyFilter;
-import com.google.datastore.v1beta3.Query;
-import com.google.datastore.v1beta3.QueryResultBatch;
-import com.google.datastore.v1beta3.RunQueryRequest;
-import com.google.datastore.v1beta3.RunQueryResponse;
-import com.google.datastore.v1beta3.Value;
-import com.google.datastore.v1beta3.client.Datastore;
-import com.google.datastore.v1beta3.client.DatastoreHelper;
-import com.google.datastore.v1beta3.client.QuerySplitter;
-import com.google.protobuf.Int32Value;
-
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-import org.mockito.Mock;
-import org.mockito.MockitoAnnotations;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.NoSuchElementException;
-
-/**
- * Tests for {@link DatastoreIO}.
- */
-@RunWith(JUnit4.class)
-public class DatastoreIOTest {
-  private static final String PROJECT = "testProject";
-  private static final String NAMESPACE = "testNamespace";
-  private static final String KIND = "testKind";
-  private static final Query QUERY;
-  static {
-    Query.Builder q = Query.newBuilder();
-    q.addKindBuilder().setName(KIND);
-    QUERY = q.build();
-  }
-  private DatastoreIO.Source initialSource;
-
-  @Mock
-  Datastore mockDatastore;
-
-  @Rule
-  public final ExpectedException thrown = ExpectedException.none();
-
-  @Rule public final ExpectedLogs logged = ExpectedLogs.none(DatastoreIO.Source.class);
-
-  @Before
-  public void setUp() {
-    MockitoAnnotations.initMocks(this);
-    initialSource = DatastoreIO.source()
-        .withProject(PROJECT).withQuery(QUERY).withNamespace(NAMESPACE);
-  }
-
-  /**
-   * Helper function to create a test {@code DataflowPipelineOptions}.
-   */
-  static final GcpOptions testPipelineOptions() {
-    GcpOptions options = PipelineOptionsFactory.as(GcpOptions.class);
-    options.setGcpCredential(new TestCredential());
-    return options;
-  }
-
-  @Test
-  public void testBuildSource() throws Exception {
-    DatastoreIO.Source source = DatastoreIO.source()
-        .withProject(PROJECT).withQuery(QUERY).withNamespace(NAMESPACE);
-    assertEquals(QUERY, source.getQuery());
-    assertEquals(PROJECT, source.getProjectId());
-    assertEquals(NAMESPACE, source.getNamespace());
-  }
-
-  /**
-   * {@link #testBuildSource} but constructed in a different order.
-   */
-  @Test
-  public void testBuildSourceAlt() throws Exception {
-    DatastoreIO.Source source = DatastoreIO.source()
-        .withProject(PROJECT).withNamespace(NAMESPACE).withQuery(QUERY);
-    assertEquals(QUERY, source.getQuery());
-    assertEquals(PROJECT, source.getProjectId());
-    assertEquals(NAMESPACE, source.getNamespace());
-  }
-
-  @Test
-  public void testSourceValidationFailsProject() throws Exception {
-    DatastoreIO.Source source = DatastoreIO.source().withQuery(QUERY);
-    thrown.expect(NullPointerException.class);
-    thrown.expectMessage("project");
-    source.validate();
-  }
-
-  @Test
-  public void testSourceValidationFailsQuery() throws Exception {
-    DatastoreIO.Source source = DatastoreIO.source().withProject(PROJECT);
-    thrown.expect(NullPointerException.class);
-    thrown.expectMessage("query");
-    source.validate();
-  }
-
-  @Test
-  public void testSourceValidationFailsQueryLimitZero() throws Exception {
-    Query invalidLimit = Query.newBuilder().setLimit(Int32Value.newBuilder().setValue(0)).build();
-    thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("Invalid query limit 0: must be positive");
-
-    DatastoreIO.source().withQuery(invalidLimit);
-  }
-
-  @Test
-  public void testSourceValidationFailsQueryLimitNegative() throws Exception {
-    Query invalidLimit = Query.newBuilder().setLimit(Int32Value.newBuilder().setValue(-5)).build();
-    thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("Invalid query limit -5: must be positive");
-
-    DatastoreIO.source().withQuery(invalidLimit);
-  }
-
-  @Test
-  public void testSourceValidationSucceedsNamespace() throws Exception {
-    DatastoreIO.Source source = DatastoreIO.source().withProject(PROJECT).withQuery(QUERY);
-    /* Should succeed, as a null namespace is fine. */
-    source.validate();
-  }
-
-  @Test
-  public void testSourceDisplayData() {
-  DatastoreIO.Source source = DatastoreIO.source()
-      .withProject(PROJECT)
-      .withQuery(QUERY)
-      .withNamespace(NAMESPACE);
-
-    DisplayData displayData = DisplayData.from(source);
-
-    assertThat(displayData, hasDisplayItem("project", PROJECT));
-    assertThat(displayData, hasDisplayItem("query", QUERY.toString()));
-    assertThat(displayData, hasDisplayItem("namespace", NAMESPACE));
-  }
-
-  @Test
-  public void testSinkDoesNotAllowNullProject() throws Exception {
-    thrown.expect(NullPointerException.class);
-    thrown.expectMessage("projectId");
-
-    DatastoreIO.sink().withProject(null);
-  }
-
-  @Test
-  public void testSinkValidationFailsWithNoProject() throws Exception {
-    DatastoreIO.Sink sink = DatastoreIO.sink();
-
-    thrown.expect(NullPointerException.class);
-    thrown.expectMessage("Project");
-
-    sink.validate(testPipelineOptions());
-  }
-
-  @Test
-  public void testSinkValidationSucceedsWithProject() throws Exception {
-    DatastoreIO.Sink sink = DatastoreIO.sink().withProject(PROJECT);
-    sink.validate(testPipelineOptions());
-  }
-
-  @Test
-  public void testSinkDisplayData() {
-    DatastoreIO.Sink sink = DatastoreIO.sink()
-        .withProject(PROJECT);
-
-    DisplayData displayData = DisplayData.from(sink);
-
-    assertThat(displayData, hasDisplayItem("project", PROJECT));
-  }
-
-  @Test
-  public void testQuerySplitBasic() throws Exception {
-    KindExpression mykind = KindExpression.newBuilder().setName("mykind").build();
-    Query query = Query.newBuilder().addKind(mykind).build();
-
-    List<Query> mockSplits = new ArrayList<>();
-    for (int i = 0; i < 8; ++i) {
-      mockSplits.add(
-          Query.newBuilder()
-              .addKind(mykind)
-              .setFilter(
-                  DatastoreHelper.makeFilter("foo", PropertyFilter.Operator.EQUAL,
-                      Value.newBuilder().setIntegerValue(i).build()))
-              .build());
-    }
-
-    QuerySplitter splitter = mock(QuerySplitter.class);
-    /* No namespace */
-    PartitionId partition = PartitionId.newBuilder().build();
-    when(splitter.getSplits(any(Query.class), eq(partition), eq(8), any(Datastore.class)))
-        .thenReturn(mockSplits);
-
-    DatastoreIO.Source io = initialSource
-        .withNamespace(null)
-        .withQuery(query)
-        .withMockSplitter(splitter)
-        .withMockEstimateSizeBytes(8 * 1024L);
-
-    List<DatastoreIO.Source> bundles = io.splitIntoBundles(1024, testPipelineOptions());
-    assertEquals(8, bundles.size());
-    for (int i = 0; i < 8; ++i) {
-      DatastoreIO.Source bundle = bundles.get(i);
-      Query bundleQuery = bundle.getQuery();
-      assertEquals("mykind", bundleQuery.getKind(0).getName());
-      assertEquals(i, bundleQuery.getFilter().getPropertyFilter().getValue().getIntegerValue());
-    }
-  }
-
-  /**
-   * Verifies that when namespace is set in the source, the split request includes the namespace.
-   */
-  @Test
-  public void testSourceWithNamespace() throws Exception {
-    QuerySplitter splitter = mock(QuerySplitter.class);
-    DatastoreIO.Source io = initialSource
-        .withMockSplitter(splitter)
-        .withMockEstimateSizeBytes(8 * 1024L);
-
-    io.splitIntoBundles(1024, testPipelineOptions());
-
-    PartitionId partition = PartitionId.newBuilder().setNamespaceId(NAMESPACE).build();
-    verify(splitter).getSplits(eq(QUERY), eq(partition), eq(8), any(Datastore.class));
-    verifyNoMoreInteractions(splitter);
-  }
-
-  @Test
-  public void testQuerySplitWithZeroSize() throws Exception {
-    KindExpression mykind = KindExpression.newBuilder().setName("mykind").build();
-    Query query = Query.newBuilder().addKind(mykind).build();
-
-    List<Query> mockSplits = Lists.newArrayList(
-        Query.newBuilder()
-            .addKind(mykind)
-            .build());
-
-    QuerySplitter splitter = mock(QuerySplitter.class);
-    when(splitter.getSplits(any(Query.class), any(PartitionId.class), eq(1), any(Datastore.class)))
-        .thenReturn(mockSplits);
-
-    DatastoreIO.Source io = initialSource
-        .withQuery(query)
-        .withMockSplitter(splitter)
-        .withMockEstimateSizeBytes(0L);
-
-    List<DatastoreIO.Source> bundles = io.splitIntoBundles(1024, testPipelineOptions());
-    assertEquals(1, bundles.size());
-    verify(splitter, never())
-        .getSplits(any(Query.class), any(PartitionId.class), eq(1), any(Datastore.class));
-    DatastoreIO.Source bundle = bundles.get(0);
-    Query bundleQuery = bundle.getQuery();
-    assertEquals("mykind", bundleQuery.getKind(0).getName());
-    assertFalse(bundleQuery.hasFilter());
-  }
-
-  /**
-   * Tests that a query with a user-provided limit field does not split, and does not even
-   * interact with a query splitter.
-   */
-  @Test
-  public void testQueryDoesNotSplitWithLimitSet() throws Exception {
-    // Minimal query with a limit
-    Query query = Query.newBuilder().setLimit(Int32Value.newBuilder().setValue(5)).build();
-
-    // Mock query splitter, should not be invoked.
-    QuerySplitter splitter = mock(QuerySplitter.class);
-    when(splitter.getSplits(any(Query.class), any(PartitionId.class), eq(2), any(Datastore.class)))
-        .thenThrow(new AssertionError("Splitter should not be invoked"));
-
-    List<DatastoreIO.Source> bundles =
-        initialSource
-            .withQuery(query)
-            .withMockSplitter(splitter)
-            .splitIntoBundles(1024, testPipelineOptions());
-
-    assertEquals(1, bundles.size());
-    assertEquals(query, bundles.get(0).getQuery());
-    verifyNoMoreInteractions(splitter);
-  }
-
-  /**
-   * Tests that when {@link QuerySplitter} cannot split a query, {@link DatastoreIO} falls back to
-   * a single split.
-   */
-  @Test
-  public void testQuerySplitterThrows() throws Exception {
-    // Mock query splitter that throws IllegalArgumentException
-    IllegalArgumentException exception =
-        new IllegalArgumentException("query not supported by splitter");
-    QuerySplitter splitter = mock(QuerySplitter.class);
-    when(
-            splitter.getSplits(
-                any(Query.class), any(PartitionId.class), any(Integer.class), any(Datastore.class)))
-        .thenThrow(exception);
-
-    Query query = Query.newBuilder().addKind(KindExpression.newBuilder().setName("myKind")).build();
-    List<DatastoreIO.Source> bundles =
-        initialSource
-            .withQuery(query)
-            .withMockSplitter(splitter)
-            .withMockEstimateSizeBytes(10240L)
-            .splitIntoBundles(1024, testPipelineOptions());
-
-    assertEquals(1, bundles.size());
-    assertEquals(query, bundles.get(0).getQuery());
-    verify(splitter, times(1))
-        .getSplits(
-            any(Query.class), any(PartitionId.class), any(Integer.class), any(Datastore.class));
-    logged.verifyWarn("Unable to parallelize the given query", exception);
-  }
-
-  @Test
-  public void testQuerySplitSizeUnavailable() throws Exception {
-    KindExpression mykind = KindExpression.newBuilder().setName("mykind").build();
-    Query query = Query.newBuilder().addKind(mykind).build();
-
-    List<Query> mockSplits = Lists.newArrayList(Query.newBuilder().addKind(mykind).build());
-
-    QuerySplitter splitter = mock(QuerySplitter.class);
-    when(splitter.getSplits(any(Query.class), any(PartitionId.class), eq(12), any(Datastore.class)))
-        .thenReturn(mockSplits);
-
-    DatastoreIO.Source io = initialSource
-        .withQuery(query)
-        .withMockSplitter(splitter)
-        .withMockEstimateSizeBytes(8 * 1024L);
-
-    DatastoreIO.Source spiedIo = spy(io);
-    when(spiedIo.getEstimatedSizeBytes(any(PipelineOptions.class)))
-        .thenThrow(new NoSuchElementException());
-
-    List<DatastoreIO.Source> bundles = spiedIo.splitIntoBundles(1024, testPipelineOptions());
-    assertEquals(1, bundles.size());
-    verify(splitter, never())
-        .getSplits(any(Query.class), any(PartitionId.class), eq(1), any(Datastore.class));
-    DatastoreIO.Source bundle = bundles.get(0);
-    Query bundleQuery = bundle.getQuery();
-    assertEquals("mykind", bundleQuery.getKind(0).getName());
-    assertFalse(bundleQuery.hasFilter());
-  }
-
-  /**
-   * Test building a Sink using builder methods.
-   */
-  @Test
-  public void testBuildSink() throws Exception {
-    DatastoreIO.Sink sink = DatastoreIO.sink().withProject(PROJECT);
-    assertEquals(PROJECT, sink.projectId);
-
-    sink = DatastoreIO.sink().withProject(PROJECT);
-    assertEquals(PROJECT, sink.projectId);
-
-    sink = DatastoreIO.sink().withProject(PROJECT);
-    assertEquals(PROJECT, sink.projectId);
-  }
-
-  /**
-   * Test building a sink using the default host.
-   */
-  @Test
-  public void testBuildSinkDefaults() throws Exception {
-    DatastoreIO.Sink sink = DatastoreIO.sink().withProject(PROJECT);
-    assertEquals(PROJECT, sink.projectId);
-
-    sink = DatastoreIO.sink().withProject(PROJECT);
-    assertEquals(PROJECT, sink.projectId);
-  }
-
-  /**
-   * Test the detection of complete and incomplete keys.
-   */
-  @Test
-  public void testHasNameOrId() {
-    Key key;
-    // Complete with name, no ancestor
-    key = makeKey("bird", "finch").build();
-    assertTrue(DatastoreWriter.isValidKey(key));
-
-    // Complete with id, no ancestor
-    key = makeKey("bird", 123).build();
-    assertTrue(DatastoreWriter.isValidKey(key));
-
-    // Incomplete, no ancestor
-    key = makeKey("bird").build();
-    assertFalse(DatastoreWriter.isValidKey(key));
-
-    // Complete with name and ancestor
-    key = makeKey("bird", "owl").build();
-    key = makeKey(key, "bird", "horned").build();
-    assertTrue(DatastoreWriter.isValidKey(key));
-
-    // Complete with id and ancestor
-    key = makeKey("bird", "owl").build();
-    key = makeKey(key, "bird", 123).build();
-    assertTrue(DatastoreWriter.isValidKey(key));
-
-    // Incomplete with ancestor
-    key = makeKey("bird", "owl").build();
-    key = makeKey(key, "bird").build();
-    assertFalse(DatastoreWriter.isValidKey(key));
-
-    key = makeKey().build();
-    assertFalse(DatastoreWriter.isValidKey(key));
-  }
-
-  /**
-   * Test that entities with incomplete keys cannot be updated.
-   */
-  @Test
-  public void testAddEntitiesWithIncompleteKeys() throws Exception {
-    Key key = makeKey("bird").build();
-    Entity entity = Entity.newBuilder().setKey(key).build();
-    DatastoreWriter writer = new DatastoreIO.DatastoreWriter(null, mockDatastore);
-
-    thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("Entities to be written to the Datastore must have complete keys");
-
-    writer.write(entity);
-  }
-
-  /**
-   * Test that entities are added to the batch to update.
-   */
-  @Test
-  public void testAddingEntities() throws Exception {
-    List<Entity> expected = Lists.newArrayList(
-        Entity.newBuilder().setKey(makeKey("bird", "jay").build()).build(),
-        Entity.newBuilder().setKey(makeKey("bird", "condor").build()).build(),
-        Entity.newBuilder().setKey(makeKey("bird", "robin").build()).build());
-
-    List<Entity> allEntities = Lists.newArrayList(expected);
-    Collections.shuffle(allEntities);
-
-    DatastoreWriter writer = new DatastoreIO.DatastoreWriter(null, mockDatastore);
-    writer.open("test_id");
-    for (Entity entity : allEntities) {
-      writer.write(entity);
-    }
-
-    assertEquals(expected.size(), writer.entities.size());
-    assertThat(writer.entities, containsInAnyOrder(expected.toArray()));
-  }
-
-  /** Datastore batch API limit in number of records per query. */
-  private static final int DATASTORE_QUERY_BATCH_LIMIT = 500;
-
-  /**
-   * A helper function that creates mock {@link Entity} results in response to a query. Always
-   * indicates that more results are available, unless the batch is limited to fewer than
-   * {@link #DATASTORE_QUERY_BATCH_LIMIT} results.
-   */
-  private static RunQueryResponse mockResponseForQuery(Query q) {
-    // Every query DatastoreIO sends should have a limit.
-    assertTrue(q.hasLimit());
-
-    // The limit should be in the range [1, DATASTORE_QUERY_BATCH_LIMIT]
-    int limit = q.getLimit().getValue();
-    assertThat(limit, greaterThanOrEqualTo(1));
-    assertThat(limit, lessThanOrEqualTo(DATASTORE_QUERY_BATCH_LIMIT));
-
-    // Create the requested number of entities.
-    List<EntityResult> entities = new ArrayList<>(limit);
-    for (int i = 0; i < limit; ++i) {
-      entities.add(
-          EntityResult.newBuilder()
-              .setEntity(Entity.newBuilder().setKey(makeKey("key" + i, i + 1)))
-              .build());
-    }
-
-    // Fill out the other parameters on the returned result batch.
-    RunQueryResponse.Builder ret = RunQueryResponse.newBuilder();
-    ret.getBatchBuilder()
-        .addAllEntityResults(entities)
-        .setEntityResultType(EntityResult.ResultType.FULL)
-        .setMoreResults(
-            limit == DATASTORE_QUERY_BATCH_LIMIT
-                ? QueryResultBatch.MoreResultsType.NOT_FINISHED
-                : QueryResultBatch.MoreResultsType.NO_MORE_RESULTS);
-
-    return ret.build();
-  }
-
-  /** Helper function to run a test reading from a limited-result query. */
-  private void runQueryLimitReadTest(int numEntities) throws Exception {
-    // An empty query to read entities.
-    Query query = Query.newBuilder().setLimit(
-        Int32Value.newBuilder().setValue(numEntities)).build();
-    DatastoreIO.Source source = DatastoreIO.source().withQuery(query).withProject("mockProject");
-
-    // Use mockResponseForQuery to generate results.
-    when(mockDatastore.runQuery(any(RunQueryRequest.class)))
-        .thenAnswer(
-            new Answer<RunQueryResponse>() {
-              @Override
-              public RunQueryResponse answer(InvocationOnMock invocation) throws Throwable {
-                Query q = ((RunQueryRequest) invocation.getArguments()[0]).getQuery();
-                return mockResponseForQuery(q);
-              }
-            });
-
-    // Actually instantiate the reader.
-    DatastoreReader reader = new DatastoreReader(source, mockDatastore);
-
-    // Simply count the number of results returned by the reader.
-    assertTrue(reader.start());
-    int resultCount = 1;
-    while (reader.advance()) {
-      resultCount++;
-    }
-    reader.close();
-
-    // Validate the number of results.
-    assertEquals(numEntities, resultCount);
-  }
-
-  /** Tests reading with a query limit less than one batch. */
-  @Test
-  public void testReadingWithLimitOneBatch() throws Exception {
-    runQueryLimitReadTest(5);
-  }
-
-  /** Tests reading with a query limit more than one batch, and not a multiple. */
-  @Test
-  public void testReadingWithLimitMultipleBatches() throws Exception {
-    runQueryLimitReadTest(DATASTORE_QUERY_BATCH_LIMIT + 5);
-  }
-
-  /** Tests reading several batches, using an exact multiple of batch size results. */
-  @Test
-  public void testReadingWithLimitMultipleBatchesExactMultiple() throws Exception {
-    runQueryLimitReadTest(5 * DATASTORE_QUERY_BATCH_LIMIT);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/d1e38443/sdks/java/core/src/test/java/org/apache/beam/sdk/io/datastore/V1Beta3Test.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/datastore/V1Beta3Test.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/datastore/V1Beta3Test.java
new file mode 100644
index 0000000..9a87ed3
--- /dev/null
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/datastore/V1Beta3Test.java
@@ -0,0 +1,584 @@
+/*
+ * 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.datastore;
+
+import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem;
+
+import static com.google.datastore.v1beta3.client.DatastoreHelper.makeKey;
+
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.greaterThanOrEqualTo;
+import static org.hamcrest.Matchers.lessThanOrEqualTo;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoMoreInteractions;
+import static org.mockito.Mockito.when;
+
+import org.apache.beam.sdk.io.datastore.V1Beta3.DatastoreReader;
+import org.apache.beam.sdk.io.datastore.V1Beta3.DatastoreSource;
+import org.apache.beam.sdk.io.datastore.V1Beta3.DatastoreWriter;
+import org.apache.beam.sdk.options.GcpOptions;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.testing.ExpectedLogs;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+import org.apache.beam.sdk.util.TestCredential;
+
+import com.google.common.collect.Lists;
+import com.google.datastore.v1beta3.Entity;
+import com.google.datastore.v1beta3.EntityResult;
+import com.google.datastore.v1beta3.Key;
+import com.google.datastore.v1beta3.KindExpression;
+import com.google.datastore.v1beta3.PartitionId;
+import com.google.datastore.v1beta3.PropertyFilter;
+import com.google.datastore.v1beta3.Query;
+import com.google.datastore.v1beta3.QueryResultBatch;
+import com.google.datastore.v1beta3.RunQueryRequest;
+import com.google.datastore.v1beta3.RunQueryResponse;
+import com.google.datastore.v1beta3.Value;
+import com.google.datastore.v1beta3.client.Datastore;
+import com.google.datastore.v1beta3.client.DatastoreHelper;
+import com.google.datastore.v1beta3.client.QuerySplitter;
+import com.google.protobuf.Int32Value;
+
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+import org.mockito.Mock;
+import org.mockito.MockitoAnnotations;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.NoSuchElementException;
+
+/**
+ * Tests for {@link V1Beta3}.
+ */
+@RunWith(JUnit4.class)
+public class V1Beta3Test {
+  private static final String PROJECT_ID = "testProject";
+  private static final String NAMESPACE = "testNamespace";
+  private static final String KIND = "testKind";
+  private static final Query QUERY;
+  static {
+    Query.Builder q = Query.newBuilder();
+    q.addKindBuilder().setName(KIND);
+    QUERY = q.build();
+  }
+  private V1Beta3.Read initialRead;
+
+  @Mock
+  Datastore mockDatastore;
+
+  @Rule
+  public final ExpectedException thrown = ExpectedException.none();
+
+  @Rule public final ExpectedLogs logged = ExpectedLogs.none(DatastoreSource.class);
+
+  @Before
+  public void setUp() {
+    MockitoAnnotations.initMocks(this);
+
+    initialRead = DatastoreIO.v1beta3().read()
+        .withProjectId(PROJECT_ID).withQuery(QUERY).withNamespace(NAMESPACE);
+  }
+
+  /**
+   * Helper function to create a test {@code DataflowPipelineOptions}.
+   */
+  static final GcpOptions testPipelineOptions() {
+    GcpOptions options = PipelineOptionsFactory.as(GcpOptions.class);
+    options.setGcpCredential(new TestCredential());
+    return options;
+  }
+
+  @Test
+  public void testBuildRead() throws Exception {
+    V1Beta3.Read read = DatastoreIO.v1beta3().read()
+        .withProjectId(PROJECT_ID).withQuery(QUERY).withNamespace(NAMESPACE);
+    assertEquals(QUERY, read.getQuery());
+    assertEquals(PROJECT_ID, read.getProjectId());
+    assertEquals(NAMESPACE, read.getNamespace());
+  }
+
+  /**
+   * {@link #testBuildRead} but constructed in a different order.
+   */
+  @Test
+  public void testBuildReadAlt() throws Exception {
+    V1Beta3.Read read =  DatastoreIO.v1beta3().read()
+        .withProjectId(PROJECT_ID).withNamespace(NAMESPACE).withQuery(QUERY);
+    assertEquals(QUERY, read.getQuery());
+    assertEquals(PROJECT_ID, read.getProjectId());
+    assertEquals(NAMESPACE, read.getNamespace());
+  }
+
+  @Test
+  public void testReadValidationFailsProject() throws Exception {
+    V1Beta3.Read read =  DatastoreIO.v1beta3().read().withQuery(QUERY);
+    thrown.expect(NullPointerException.class);
+    thrown.expectMessage("project");
+    read.validate(null);
+  }
+
+  @Test
+  public void testReadValidationFailsQuery() throws Exception {
+    V1Beta3.Read read =  DatastoreIO.v1beta3().read().withProjectId(PROJECT_ID);
+    thrown.expect(NullPointerException.class);
+    thrown.expectMessage("query");
+    read.validate(null);
+  }
+
+  @Test
+  public void testReadValidationFailsQueryLimitZero() throws Exception {
+    Query invalidLimit = Query.newBuilder().setLimit(Int32Value.newBuilder().setValue(0)).build();
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage("Invalid query limit 0: must be positive");
+
+    DatastoreIO.v1beta3().read().withQuery(invalidLimit);
+  }
+
+  @Test
+  public void testReadValidationFailsQueryLimitNegative() throws Exception {
+    Query invalidLimit = Query.newBuilder().setLimit(Int32Value.newBuilder().setValue(-5)).build();
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage("Invalid query limit -5: must be positive");
+
+    DatastoreIO.v1beta3().read().withQuery(invalidLimit);
+  }
+
+  @Test
+  public void testReadValidationSucceedsNamespace() throws Exception {
+    V1Beta3.Read read =  DatastoreIO.v1beta3().read().withProjectId(PROJECT_ID).withQuery(QUERY);
+    /* Should succeed, as a null namespace is fine. */
+    read.validate(null);
+  }
+
+  @Test
+  public void testReadDisplayData() {
+    V1Beta3.Read read =  DatastoreIO.v1beta3().read()
+      .withProjectId(PROJECT_ID)
+      .withQuery(QUERY)
+      .withNamespace(NAMESPACE);
+
+    DisplayData displayData = DisplayData.from(read);
+
+    assertThat(displayData, hasDisplayItem("projectId", PROJECT_ID));
+    assertThat(displayData, hasDisplayItem("query", QUERY.toString()));
+    assertThat(displayData, hasDisplayItem("namespace", NAMESPACE));
+  }
+
+  @Test
+  public void testWriteDoesNotAllowNullProject() throws Exception {
+    thrown.expect(NullPointerException.class);
+    thrown.expectMessage("projectId");
+
+    DatastoreIO.v1beta3().write().withProjectId(null);
+  }
+
+  @Test
+  public void testWriteValidationFailsWithNoProject() throws Exception {
+    V1Beta3.Write write =  DatastoreIO.v1beta3().write();
+
+    thrown.expect(NullPointerException.class);
+    thrown.expectMessage("projectId");
+
+    write.validate(null);
+  }
+
+  @Test
+  public void testSinkValidationSucceedsWithProject() throws Exception {
+    V1Beta3.Write write =  DatastoreIO.v1beta3().write().withProjectId(PROJECT_ID);
+    write.validate(null);
+  }
+
+  @Test
+  public void testWriteDisplayData() {
+    V1Beta3.Write write =  DatastoreIO.v1beta3().write()
+        .withProjectId(PROJECT_ID);
+
+    DisplayData displayData = DisplayData.from(write);
+
+    assertThat(displayData, hasDisplayItem("projectId", PROJECT_ID));
+  }
+
+  @Test
+  public void testQuerySplitBasic() throws Exception {
+    KindExpression mykind = KindExpression.newBuilder().setName("mykind").build();
+    Query query = Query.newBuilder().addKind(mykind).build();
+
+    List<Query> mockSplits = new ArrayList<>();
+    for (int i = 0; i < 8; ++i) {
+      mockSplits.add(
+          Query.newBuilder()
+              .addKind(mykind)
+              .setFilter(
+                  DatastoreHelper.makeFilter("foo", PropertyFilter.Operator.EQUAL,
+                      Value.newBuilder().setIntegerValue(i).build()))
+              .build());
+    }
+
+    QuerySplitter splitter = mock(QuerySplitter.class);
+    /* No namespace */
+    PartitionId partition = PartitionId.newBuilder().build();
+    when(splitter.getSplits(any(Query.class), eq(partition), eq(8), any(Datastore.class)))
+        .thenReturn(mockSplits);
+
+    DatastoreSource io = initialRead
+        .withNamespace(null)
+        .withQuery(query)
+        .getSource()
+        .withMockSplitter(splitter)
+        .withMockEstimateSizeBytes(8 * 1024L);
+
+    List<DatastoreSource> bundles = io.splitIntoBundles(1024, testPipelineOptions());
+    assertEquals(8, bundles.size());
+    for (int i = 0; i < 8; ++i) {
+      DatastoreSource bundle = bundles.get(i);
+      Query bundleQuery = bundle.getQuery();
+      assertEquals("mykind", bundleQuery.getKind(0).getName());
+      assertEquals(i, bundleQuery.getFilter().getPropertyFilter().getValue().getIntegerValue());
+    }
+  }
+
+  /**
+   * Verifies that when namespace is set in the source, the split request includes the namespace.
+   */
+  @Test
+  public void testSourceWithNamespace() throws Exception {
+    QuerySplitter splitter = mock(QuerySplitter.class);
+    DatastoreSource io = initialRead
+        .getSource()
+        .withMockSplitter(splitter)
+        .withMockEstimateSizeBytes(8 * 1024L);
+
+    io.splitIntoBundles(1024, testPipelineOptions());
+
+    PartitionId partition = PartitionId.newBuilder().setNamespaceId(NAMESPACE).build();
+    verify(splitter).getSplits(eq(QUERY), eq(partition), eq(8), any(Datastore.class));
+    verifyNoMoreInteractions(splitter);
+  }
+
+  @Test
+  public void testQuerySplitWithZeroSize() throws Exception {
+    KindExpression mykind = KindExpression.newBuilder().setName("mykind").build();
+    Query query = Query.newBuilder().addKind(mykind).build();
+
+    List<Query> mockSplits = Lists.newArrayList(
+        Query.newBuilder()
+            .addKind(mykind)
+            .build());
+
+    QuerySplitter splitter = mock(QuerySplitter.class);
+    when(splitter.getSplits(any(Query.class), any(PartitionId.class), eq(1), any(Datastore.class)))
+        .thenReturn(mockSplits);
+
+    DatastoreSource io = initialRead
+        .withQuery(query)
+        .getSource()
+        .withMockSplitter(splitter)
+        .withMockEstimateSizeBytes(0L);
+
+    List<DatastoreSource> bundles = io.splitIntoBundles(1024, testPipelineOptions());
+    assertEquals(1, bundles.size());
+    verify(splitter, never())
+        .getSplits(any(Query.class), any(PartitionId.class), eq(1), any(Datastore.class));
+    DatastoreSource bundle = bundles.get(0);
+    Query bundleQuery = bundle.getQuery();
+    assertEquals("mykind", bundleQuery.getKind(0).getName());
+    assertFalse(bundleQuery.hasFilter());
+  }
+
+  /**
+   * Tests that a query with a user-provided limit field does not split, and does not even
+   * interact with a query splitter.
+   */
+  @Test
+  public void testQueryDoesNotSplitWithLimitSet() throws Exception {
+    // Minimal query with a limit
+    Query query = Query.newBuilder().setLimit(Int32Value.newBuilder().setValue(5)).build();
+
+    // Mock query splitter, should not be invoked.
+    QuerySplitter splitter = mock(QuerySplitter.class);
+    when(splitter.getSplits(any(Query.class), any(PartitionId.class), eq(2), any(Datastore.class)))
+        .thenThrow(new AssertionError("Splitter should not be invoked"));
+
+    List<DatastoreSource> bundles =
+        initialRead
+            .withQuery(query)
+            .getSource()
+            .withMockSplitter(splitter)
+            .splitIntoBundles(1024, testPipelineOptions());
+
+    assertEquals(1, bundles.size());
+    assertEquals(query, bundles.get(0).getQuery());
+    verifyNoMoreInteractions(splitter);
+  }
+
+  /**
+   * Tests that when {@link QuerySplitter} cannot split a query, {@link V1Beta3} falls back to
+   * a single split.
+   */
+  @Test
+  public void testQuerySplitterThrows() throws Exception {
+    // Mock query splitter that throws IllegalArgumentException
+    IllegalArgumentException exception =
+        new IllegalArgumentException("query not supported by splitter");
+    QuerySplitter splitter = mock(QuerySplitter.class);
+    when(
+            splitter.getSplits(
+                any(Query.class), any(PartitionId.class), any(Integer.class), any(Datastore.class)))
+        .thenThrow(exception);
+
+    Query query = Query.newBuilder().addKind(KindExpression.newBuilder().setName("myKind")).build();
+    List<DatastoreSource> bundles =
+        initialRead
+            .withQuery(query)
+            .getSource()
+            .withMockSplitter(splitter)
+            .withMockEstimateSizeBytes(10240L)
+            .splitIntoBundles(1024, testPipelineOptions());
+
+    assertEquals(1, bundles.size());
+    assertEquals(query, bundles.get(0).getQuery());
+    verify(splitter, times(1))
+        .getSplits(
+            any(Query.class), any(PartitionId.class), any(Integer.class), any(Datastore.class));
+    logged.verifyWarn("Unable to parallelize the given query", exception);
+  }
+
+  @Test
+  public void testQuerySplitSizeUnavailable() throws Exception {
+    KindExpression mykind = KindExpression.newBuilder().setName("mykind").build();
+    Query query = Query.newBuilder().addKind(mykind).build();
+
+    List<Query> mockSplits = Lists.newArrayList(Query.newBuilder().addKind(mykind).build());
+
+    QuerySplitter splitter = mock(QuerySplitter.class);
+    when(splitter.getSplits(any(Query.class), any(PartitionId.class), eq(12), any(Datastore.class)))
+        .thenReturn(mockSplits);
+
+    DatastoreSource io = initialRead
+        .withQuery(query)
+        .getSource()
+        .withMockSplitter(splitter)
+        .withMockEstimateSizeBytes(8 * 1024L);
+
+    DatastoreSource spiedIo = spy(io);
+    when(spiedIo.getEstimatedSizeBytes(any(PipelineOptions.class)))
+        .thenThrow(new NoSuchElementException());
+
+    List<DatastoreSource> bundles = spiedIo.splitIntoBundles(1024, testPipelineOptions());
+    assertEquals(1, bundles.size());
+    verify(splitter, never())
+        .getSplits(any(Query.class), any(PartitionId.class), eq(1), any(Datastore.class));
+    DatastoreSource bundle = bundles.get(0);
+    Query bundleQuery = bundle.getQuery();
+    assertEquals("mykind", bundleQuery.getKind(0).getName());
+    assertFalse(bundleQuery.hasFilter());
+  }
+
+  /**
+   * Test building a Write using builder methods.
+   */
+  @Test
+  public void testBuildWrite() throws Exception {
+    V1Beta3.Write write =  DatastoreIO.v1beta3().write().withProjectId(PROJECT_ID);
+    assertEquals(PROJECT_ID, write.getProjectId());
+  }
+
+  /**
+   * Test the detection of complete and incomplete keys.
+   */
+  @Test
+  public void testHasNameOrId() {
+    Key key;
+    // Complete with name, no ancestor
+    key = makeKey("bird", "finch").build();
+    assertTrue(DatastoreWriter.isValidKey(key));
+
+    // Complete with id, no ancestor
+    key = makeKey("bird", 123).build();
+    assertTrue(DatastoreWriter.isValidKey(key));
+
+    // Incomplete, no ancestor
+    key = makeKey("bird").build();
+    assertFalse(DatastoreWriter.isValidKey(key));
+
+    // Complete with name and ancestor
+    key = makeKey("bird", "owl").build();
+    key = makeKey(key, "bird", "horned").build();
+    assertTrue(DatastoreWriter.isValidKey(key));
+
+    // Complete with id and ancestor
+    key = makeKey("bird", "owl").build();
+    key = makeKey(key, "bird", 123).build();
+    assertTrue(DatastoreWriter.isValidKey(key));
+
+    // Incomplete with ancestor
+    key = makeKey("bird", "owl").build();
+    key = makeKey(key, "bird").build();
+    assertFalse(DatastoreWriter.isValidKey(key));
+
+    key = makeKey().build();
+    assertFalse(DatastoreWriter.isValidKey(key));
+  }
+
+  /**
+   * Test that entities with incomplete keys cannot be updated.
+   */
+  @Test
+  public void testAddEntitiesWithIncompleteKeys() throws Exception {
+    Key key = makeKey("bird").build();
+    Entity entity = Entity.newBuilder().setKey(key).build();
+    DatastoreWriter writer = new DatastoreWriter(null, mockDatastore);
+
+    thrown.expect(IllegalArgumentException.class);
+    thrown.expectMessage("Entities to be written to the Datastore must have complete keys");
+
+    writer.write(entity);
+  }
+
+  /**
+   * Test that entities are added to the batch to update.
+   */
+  @Test
+  public void testAddingEntities() throws Exception {
+    List<Entity> expected = Lists.newArrayList(
+        Entity.newBuilder().setKey(makeKey("bird", "jay").build()).build(),
+        Entity.newBuilder().setKey(makeKey("bird", "condor").build()).build(),
+        Entity.newBuilder().setKey(makeKey("bird", "robin").build()).build());
+
+    List<Entity> allEntities = Lists.newArrayList(expected);
+    Collections.shuffle(allEntities);
+
+    DatastoreWriter writer = new DatastoreWriter(null, mockDatastore);
+    writer.open("test_id");
+    for (Entity entity : allEntities) {
+      writer.write(entity);
+    }
+
+    assertEquals(expected.size(), writer.entities.size());
+    assertThat(writer.entities, containsInAnyOrder(expected.toArray()));
+  }
+
+  /** Datastore batch API limit in number of records per query. */
+  private static final int DATASTORE_QUERY_BATCH_LIMIT = 500;
+
+  /**
+   * A helper function that creates mock {@link Entity} results in response to a query. Always
+   * indicates that more results are available, unless the batch is limited to fewer than
+   * {@link #DATASTORE_QUERY_BATCH_LIMIT} results.
+   */
+  private static RunQueryResponse mockResponseForQuery(Query q) {
+    // Every query V1Beta3 sends should have a limit.
+    assertTrue(q.hasLimit());
+
+    // The limit should be in the range [1, DATASTORE_QUERY_BATCH_LIMIT]
+    int limit = q.getLimit().getValue();
+    assertThat(limit, greaterThanOrEqualTo(1));
+    assertThat(limit, lessThanOrEqualTo(DATASTORE_QUERY_BATCH_LIMIT));
+
+    // Create the requested number of entities.
+    List<EntityResult> entities = new ArrayList<>(limit);
+    for (int i = 0; i < limit; ++i) {
+      entities.add(
+          EntityResult.newBuilder()
+              .setEntity(Entity.newBuilder().setKey(makeKey("key" + i, i + 1)))
+              .build());
+    }
+
+    // Fill out the other parameters on the returned result batch.
+    RunQueryResponse.Builder ret = RunQueryResponse.newBuilder();
+    ret.getBatchBuilder()
+        .addAllEntityResults(entities)
+        .setEntityResultType(EntityResult.ResultType.FULL)
+        .setMoreResults(
+            limit == DATASTORE_QUERY_BATCH_LIMIT
+                ? QueryResultBatch.MoreResultsType.NOT_FINISHED
+                : QueryResultBatch.MoreResultsType.NO_MORE_RESULTS);
+
+    return ret.build();
+  }
+
+  /** Helper function to run a test reading from a limited-result query. */
+  private void runQueryLimitReadTest(int numEntities) throws Exception {
+    // An empty query to read entities.
+    Query query = Query.newBuilder().setLimit(
+        Int32Value.newBuilder().setValue(numEntities)).build();
+    V1Beta3.Read read =  DatastoreIO.v1beta3().read().withQuery(query).withProjectId("mockProject");
+
+    // Use mockResponseForQuery to generate results.
+    when(mockDatastore.runQuery(any(RunQueryRequest.class)))
+        .thenAnswer(
+            new Answer<RunQueryResponse>() {
+              @Override
+              public RunQueryResponse answer(InvocationOnMock invocation) throws Throwable {
+                Query q = ((RunQueryRequest) invocation.getArguments()[0]).getQuery();
+                return mockResponseForQuery(q);
+              }
+            });
+
+    // Actually instantiate the reader.
+    DatastoreReader reader = new DatastoreReader(read.getSource(), mockDatastore);
+
+    // Simply count the number of results returned by the reader.
+    assertTrue(reader.start());
+    int resultCount = 1;
+    while (reader.advance()) {
+      resultCount++;
+    }
+    reader.close();
+
+    // Validate the number of results.
+    assertEquals(numEntities, resultCount);
+  }
+
+  /** Tests reading with a query limit less than one batch. */
+  @Test
+  public void testReadingWithLimitOneBatch() throws Exception {
+    runQueryLimitReadTest(5);
+  }
+
+  /** Tests reading with a query limit more than one batch, and not a multiple. */
+  @Test
+  public void testReadingWithLimitMultipleBatches() throws Exception {
+    runQueryLimitReadTest(DATASTORE_QUERY_BATCH_LIMIT + 5);
+  }
+
+  /** Tests reading several batches, using an exact multiple of batch size results. */
+  @Test
+  public void testReadingWithLimitMultipleBatchesExactMultiple() throws Exception {
+    runQueryLimitReadTest(5 * DATASTORE_QUERY_BATCH_LIMIT);
+  }
+}



[49/50] [abbrv] incubator-beam git commit: Replace PubsubIO and injector with TextIO in beam-examples

Posted by lc...@apache.org.
Replace PubsubIO and injector with TextIO in beam-examples


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

Branch: refs/heads/runners-spark2
Commit: b09de0f0dbc055cd859b9e2b5350e6ddb42e69c8
Parents: 023a58e
Author: Pei He <pe...@google.com>
Authored: Wed Jun 29 13:01:10 2016 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Jul 6 10:18:53 2016 -0700

----------------------------------------------------------------------
 .../examples/complete/StreamingWordExtract.java | 35 ++++++--------------
 1 file changed, 10 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/b09de0f0/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java b/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java
index aca4222..4ea199c 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java
@@ -19,11 +19,10 @@ package org.apache.beam.examples.complete;
 
 import org.apache.beam.examples.common.DataflowExampleUtils;
 import org.apache.beam.examples.common.ExampleBigQueryTableOptions;
-import org.apache.beam.examples.common.ExamplePubsubTopicOptions;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.PipelineResult;
 import org.apache.beam.sdk.io.BigQueryIO;
-import org.apache.beam.sdk.io.PubsubIO;
+import org.apache.beam.sdk.io.TextIO;
 import org.apache.beam.sdk.options.Default;
 import org.apache.beam.sdk.options.Description;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
@@ -40,22 +39,14 @@ import java.util.ArrayList;
 /**
  * A streaming Dataflow Example using BigQuery output.
  *
- * <p>This pipeline example reads lines of text from a PubSub topic, splits each line
+ * <p>This pipeline example reads lines of the input text file, splits each line
  * into individual words, capitalizes those words, and writes the output to
  * a BigQuery table.
  *
- * <p>By default, the example will run a separate pipeline to inject the data from the default
- * {@literal --inputFile} to the Pub/Sub {@literal --pubsubTopic}. It will make it available for
- * the streaming pipeline to process. You may override the default {@literal --inputFile} with the
- * file of your choosing. You may also set {@literal --inputFile} to an empty string, which will
- * disable the automatic Pub/Sub injection, and allow you to use separate tool to control the input
- * to this example.
- *
- * <p>The example is configured to use the default Pub/Sub topic and the default BigQuery table
- * from the example common package (there are no defaults for a general Dataflow pipeline).
- * You can override them by using the {@literal --pubsubTopic}, {@literal --bigQueryDataset}, and
- * {@literal --bigQueryTable} options. If the Pub/Sub topic or the BigQuery table do not exist,
- * the example will try to create them.
+ * <p>The example is configured to use the default BigQuery table from the example common package
+ * (there are no defaults for a general Dataflow pipeline).
+ * You can override them by using the {@literal --bigQueryDataset}, and {@literal --bigQueryTable}
+ * options. If the BigQuery table do not exist, the example will try to create them.
  *
  * <p>The example will try to cancel the pipelines on the signal to terminate the process (CTRL-C)
  * and then exits.
@@ -110,9 +101,8 @@ public class StreamingWordExtract {
    *
    * <p>Inherits standard configuration options.
    */
-  private interface StreamingWordExtractOptions
-      extends ExamplePubsubTopicOptions, ExampleBigQueryTableOptions {
-    @Description("Input file to inject to Pub/Sub topic")
+  private interface StreamingWordExtractOptions extends ExampleBigQueryTableOptions {
+    @Description("Path of the file to read from")
     @Default.String("gs://dataflow-samples/shakespeare/kinglear.txt")
     String getInputFile();
     void setInputFile(String value);
@@ -141,7 +131,7 @@ public class StreamingWordExtract {
         .append(options.getBigQueryTable())
         .toString();
     pipeline
-        .apply(PubsubIO.Read.topic(options.getPubsubTopic()))
+        .apply("ReadLines", TextIO.Read.from(options.getInputFile()))
         .apply(ParDo.of(new ExtractWords()))
         .apply(ParDo.of(new Uppercase()))
         .apply(ParDo.of(new StringToRowConverter()))
@@ -150,12 +140,7 @@ public class StreamingWordExtract {
 
     PipelineResult result = pipeline.run();
 
-    if (!options.getInputFile().isEmpty()) {
-      // Inject the data into the Pub/Sub topic with a Dataflow batch pipeline.
-      dataflowUtils.runInjectorPipeline(options.getInputFile(), options.getPubsubTopic());
-    }
-
-    // dataflowUtils will try to cancel the pipeline and the injector before the program exists.
+    // dataflowUtils will try to cancel the pipeline before the program exists.
     dataflowUtils.waitToFinish(result);
   }
 }


[18/50] [abbrv] incubator-beam git commit: Static import Preconditions.checkX everywhere

Posted by lc...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/main/java/org/apache/beam/sdk/util/InstanceBuilder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/InstanceBuilder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/InstanceBuilder.java
index e5d9916..08e07ce 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/InstanceBuilder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/InstanceBuilder.java
@@ -17,10 +17,12 @@
  */
 package org.apache.beam.sdk.util;
 
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkState;
+
 import org.apache.beam.sdk.values.TypeDescriptor;
 
 import com.google.common.base.Joiner;
-import com.google.common.base.Preconditions;
 
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
@@ -80,8 +82,7 @@ public class InstanceBuilder<T> {
    */
   public InstanceBuilder<T> fromClassName(String name)
       throws ClassNotFoundException {
-    Preconditions.checkArgument(factoryClass == null,
-        "Class name may only be specified once");
+    checkArgument(factoryClass == null, "Class name may only be specified once");
     if (name.indexOf('.') == -1) {
       name = type.getPackage().getName() + "." + name;
     }
@@ -114,7 +115,7 @@ public class InstanceBuilder<T> {
    * <p>Modifies and returns the {@code InstanceBuilder} for chaining.
    */
   public InstanceBuilder<T> fromFactoryMethod(String methodName) {
-    Preconditions.checkArgument(this.methodName == null,
+    checkArgument(this.methodName == null,
         "Factory method name may only be specified once");
     this.methodName = methodName;
     return this;
@@ -201,18 +202,18 @@ public class InstanceBuilder<T> {
   }
 
   private T buildFromMethod(Class<?>[] types) {
-    Preconditions.checkState(factoryClass != null);
-    Preconditions.checkState(methodName != null);
+    checkState(factoryClass != null);
+    checkState(methodName != null);
 
     try {
       Method method = factoryClass.getDeclaredMethod(methodName, types);
 
-      Preconditions.checkState(Modifier.isStatic(method.getModifiers()),
+      checkState(Modifier.isStatic(method.getModifiers()),
           "Factory method must be a static method for "
               + factoryClass.getName() + "#" + method.getName()
       );
 
-      Preconditions.checkState(type.isAssignableFrom(method.getReturnType()),
+      checkState(type.isAssignableFrom(method.getReturnType()),
           "Return type for " + factoryClass.getName() + "#" + method.getName()
               + " must be assignable to " + type.getSimpleName());
 
@@ -241,12 +242,12 @@ public class InstanceBuilder<T> {
   }
 
   private T buildFromConstructor(Class<?>[] types) {
-    Preconditions.checkState(factoryClass != null);
+    checkState(factoryClass != null);
 
     try {
       Constructor<?> constructor = factoryClass.getDeclaredConstructor(types);
 
-      Preconditions.checkState(type.isAssignableFrom(factoryClass),
+      checkState(type.isAssignableFrom(factoryClass),
           "Instance type " + factoryClass.getName()
               + " must be assignable to " + type.getSimpleName());
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IntervalBoundedExponentialBackOff.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IntervalBoundedExponentialBackOff.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IntervalBoundedExponentialBackOff.java
index 92ff2f0..519776a 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IntervalBoundedExponentialBackOff.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IntervalBoundedExponentialBackOff.java
@@ -17,8 +17,10 @@
  */
 package org.apache.beam.sdk.util;
 
+import static com.google.common.base.Preconditions.checkArgument;
+
 import com.google.api.client.util.BackOff;
-import com.google.common.base.Preconditions;
+
 
 /**
  * Implementation of {@link BackOff} that increases the back off period for each retry attempt
@@ -55,10 +57,8 @@ public class IntervalBoundedExponentialBackOff implements BackOff {
   private int currentAttempt;
 
   public IntervalBoundedExponentialBackOff(long maximumIntervalMillis, long initialIntervalMillis) {
-    Preconditions.checkArgument(
-        maximumIntervalMillis > 0, "Maximum interval must be greater than zero.");
-    Preconditions.checkArgument(
-        initialIntervalMillis > 0, "Initial interval must be greater than zero.");
+    checkArgument(maximumIntervalMillis > 0, "Maximum interval must be greater than zero.");
+    checkArgument(initialIntervalMillis > 0, "Initial interval must be greater than zero.");
     this.maximumIntervalMillis = maximumIntervalMillis;
     this.initialIntervalMillis = initialIntervalMillis;
     reset();

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MergingActiveWindowSet.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MergingActiveWindowSet.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MergingActiveWindowSet.java
index 07e47aa..4702fe2 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MergingActiveWindowSet.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MergingActiveWindowSet.java
@@ -17,6 +17,9 @@
  */
 package org.apache.beam.sdk.util;
 
+import static com.google.common.base.Preconditions.checkNotNull;
+import static com.google.common.base.Preconditions.checkState;
+
 import org.apache.beam.sdk.coders.MapCoder;
 import org.apache.beam.sdk.coders.SetCoder;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
@@ -28,7 +31,6 @@ import org.apache.beam.sdk.util.state.StateTags;
 import org.apache.beam.sdk.util.state.ValueState;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Sets;
 
@@ -142,7 +144,7 @@ public class MergingActiveWindowSet<W extends BoundedWindow> implements ActiveWi
   @Override
   public void ensureWindowIsActive(W window) {
     Set<W> stateAddressWindows = activeWindowToStateAddressWindows.get(window);
-    Preconditions.checkState(stateAddressWindows != null,
+    checkState(stateAddressWindows != null,
                              "Cannot ensure window %s is active since it is neither ACTIVE nor NEW",
                              window);
     if (stateAddressWindows.isEmpty()) {
@@ -196,25 +198,22 @@ public class MergingActiveWindowSet<W extends BoundedWindow> implements ActiveWi
     @Override
     public void merge(Collection<W> toBeMerged, W mergeResult) throws Exception {
       // The arguments have come from userland.
-      Preconditions.checkNotNull(toBeMerged);
-      Preconditions.checkNotNull(mergeResult);
+      checkNotNull(toBeMerged);
+      checkNotNull(mergeResult);
       List<W> copyOfToBeMerged = new ArrayList<>(toBeMerged.size());
       boolean includesMergeResult = false;
       for (W window : toBeMerged) {
-        Preconditions.checkNotNull(window);
-        Preconditions.checkState(
-            isActiveOrNew(window), "Expecting merge window %s to be ACTIVE or NEW", window);
+        checkNotNull(window);
+        checkState(isActiveOrNew(window), "Expecting merge window %s to be ACTIVE or NEW", window);
         if (window.equals(mergeResult)) {
           includesMergeResult = true;
         }
         boolean notDup = seen.add(window);
-        Preconditions.checkState(
-            notDup, "Expecting merge window %s to appear in at most one merge set", window);
+        checkState(notDup, "Expecting merge window %s to appear in at most one merge set", window);
         copyOfToBeMerged.add(window);
       }
       if (!includesMergeResult) {
-        Preconditions.checkState(
-            !isActive(mergeResult), "Expecting result window %s to be NEW", mergeResult);
+        checkState(!isActive(mergeResult), "Expecting result window %s to be NEW", mergeResult);
       }
       allToBeMerged.add(copyOfToBeMerged);
       allMergeResults.add(mergeResult);
@@ -267,7 +266,7 @@ public class MergingActiveWindowSet<W extends BoundedWindow> implements ActiveWi
 
     for (W other : toBeMerged) {
       Set<W> otherStateAddressWindows = activeWindowToStateAddressWindows.get(other);
-      Preconditions.checkState(otherStateAddressWindows != null,
+      checkState(otherStateAddressWindows != null,
                                "Window %s is not ACTIVE or NEW", other);
 
       for (W otherStateAddressWindow : otherStateAddressWindows) {
@@ -295,7 +294,7 @@ public class MergingActiveWindowSet<W extends BoundedWindow> implements ActiveWi
   public void merged(W window) {
     // Take just the first state address window.
     Set<W> stateAddressWindows = activeWindowToStateAddressWindows.get(window);
-    Preconditions.checkState(stateAddressWindows != null, "Window %s is not ACTIVE", window);
+    checkState(stateAddressWindows != null, "Window %s is not ACTIVE", window);
     W first = Iterables.getFirst(stateAddressWindows, null);
     stateAddressWindows.clear();
     stateAddressWindows.add(first);
@@ -308,7 +307,7 @@ public class MergingActiveWindowSet<W extends BoundedWindow> implements ActiveWi
   @Override
   public Set<W> readStateAddresses(W window) {
     Set<W> stateAddressWindows = activeWindowToStateAddressWindows.get(window);
-    Preconditions.checkState(stateAddressWindows != null, "Window %s is not ACTIVE", window);
+    checkState(stateAddressWindows != null, "Window %s is not ACTIVE", window);
     return stateAddressWindows;
   }
 
@@ -319,9 +318,9 @@ public class MergingActiveWindowSet<W extends BoundedWindow> implements ActiveWi
   @Override
   public W writeStateAddress(W window) {
     Set<W> stateAddressWindows = activeWindowToStateAddressWindows.get(window);
-    Preconditions.checkState(stateAddressWindows != null, "Window %s is not ACTIVE", window);
+    checkState(stateAddressWindows != null, "Window %s is not ACTIVE", window);
     W result = Iterables.getFirst(stateAddressWindows, null);
-    Preconditions.checkState(result != null, "Window %s is still NEW", window);
+    checkState(result != null, "Window %s is still NEW", window);
     return result;
   }
 
@@ -345,11 +344,11 @@ public class MergingActiveWindowSet<W extends BoundedWindow> implements ActiveWi
     Set<W> knownStateAddressWindows = new HashSet<>();
     for (Map.Entry<W, Set<W>> entry : activeWindowToStateAddressWindows.entrySet()) {
       W active = entry.getKey();
-      Preconditions.checkState(!entry.getValue().isEmpty(),
+      checkState(!entry.getValue().isEmpty(),
                                "Unexpected empty state address window set for ACTIVE window %s",
                                active);
       for (W stateAddressWindow : entry.getValue()) {
-        Preconditions.checkState(knownStateAddressWindows.add(stateAddressWindow),
+        checkState(knownStateAddressWindows.add(stateAddressWindow),
                                  "%s is in more than one state address window set",
                                  stateAddressWindow);
       }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SerializableUtils.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SerializableUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SerializableUtils.java
index 06b5bb2..45f6c4a 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SerializableUtils.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SerializableUtils.java
@@ -20,11 +20,11 @@ package org.apache.beam.sdk.util;
 import static org.apache.beam.sdk.util.CoderUtils.decodeFromByteArray;
 import static org.apache.beam.sdk.util.CoderUtils.encodeToByteArray;
 
+import static com.google.common.base.Preconditions.checkState;
+
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.CoderException;
 
-import com.google.common.base.Preconditions;
-
 import org.xerial.snappy.SnappyInputStream;
 import org.xerial.snappy.SnappyOutputStream;
 
@@ -120,11 +120,10 @@ public class SerializableUtils {
               + "See Coder for details.", coder), e
       );
     }
-    Preconditions.checkState(coder.equals(decoded),
-        String.format("Coder not equal to original after serialization, "
-            + "indicating that the Coder may not implement serialization "
-            + "correctly.  Before: %s, after: %s, cloud encoding: %s",
-            coder, decoded, cloudObject));
+    checkState(coder.equals(decoded),
+        "Coder not equal to original after serialization, indicating that the Coder may not "
+        + "implement serialization correctly.  Before: %s, after: %s, cloud encoding: %s",
+        coder, decoded, cloudObject);
 
     return cloudObject;
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/main/java/org/apache/beam/sdk/util/StringUtils.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/StringUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/StringUtils.java
index 428b350..53201a4 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/StringUtils.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/StringUtils.java
@@ -17,10 +17,12 @@
  */
 package org.apache.beam.sdk.util;
 
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+
 import org.apache.beam.sdk.transforms.PTransform;
 
 import com.google.common.base.Joiner;
-import com.google.common.base.Preconditions;
 
 import java.util.ArrayList;
 import java.util.List;
@@ -148,7 +150,7 @@ public class StringUtils {
    * </ul>
    */
   public static String approximatePTransformName(Class<?> clazz) {
-    Preconditions.checkArgument(PTransform.class.isAssignableFrom(clazz));
+    checkArgument(PTransform.class.isAssignableFrom(clazz));
     return approximateSimpleName(clazz, /* dropOuterClassNames */ false)
         .replaceFirst("\\.Bound$", "");
   }
@@ -160,8 +162,8 @@ public class StringUtils {
    * (i.e. insertions, deletions or substitutions) required to change one string into the other.
    */
   public static int getLevenshteinDistance(final String s, final String t) {
-    Preconditions.checkNotNull(s);
-    Preconditions.checkNotNull(t);
+    checkNotNull(s);
+    checkNotNull(t);
 
     // base cases
     if (s.equals(t)) {
@@ -206,7 +208,7 @@ public class StringUtils {
   }
 
   private static String approximateSimpleName(Class<?> clazz, boolean dropOuterClassNames) {
-    Preconditions.checkArgument(!clazz.isAnonymousClass(),
+    checkArgument(!clazz.isAnonymousClass(),
         "Attempted to get simple name of anonymous class");
 
     String fullName = clazz.getName();

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TimerInternals.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TimerInternals.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TimerInternals.java
index 5f67131..c03ab4d 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TimerInternals.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TimerInternals.java
@@ -17,6 +17,7 @@
  */
 package org.apache.beam.sdk.util;
 
+import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkNotNull;
 
 import org.apache.beam.sdk.coders.Coder;
@@ -29,7 +30,6 @@ import org.apache.beam.sdk.util.state.StateNamespace;
 import org.apache.beam.sdk.util.state.StateNamespaces;
 
 import com.google.common.base.MoreObjects;
-import com.google.common.base.Preconditions;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
@@ -229,8 +229,7 @@ public interface TimerInternals {
     public static TimerDataCoder of(
         @JsonProperty(PropertyNames.COMPONENT_ENCODINGS)
         List<Coder<?>> components) {
-      Preconditions.checkArgument(components.size() == 1,
-          "Expecting 1 components, got " + components.size());
+      checkArgument(components.size() == 1, "Expecting 1 components, got %s", components.size());
       return of((Coder<? extends BoundedWindow>) components.get(0));
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ValueWithRecordId.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ValueWithRecordId.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ValueWithRecordId.java
index 80dfcae..e724349 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ValueWithRecordId.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ValueWithRecordId.java
@@ -17,13 +17,14 @@
  */
 package org.apache.beam.sdk.util;
 
+import static com.google.common.base.Preconditions.checkArgument;
+
 import org.apache.beam.sdk.coders.ByteArrayCoder;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.StandardCoder;
 import org.apache.beam.sdk.transforms.DoFn;
 
 import com.google.common.base.MoreObjects;
-import com.google.common.base.Preconditions;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
@@ -96,8 +97,7 @@ public class ValueWithRecordId<ValueT> {
     public static <ValueT> ValueWithRecordIdCoder<ValueT> of(
          @JsonProperty(PropertyNames.COMPONENT_ENCODINGS)
         List<Coder<ValueT>> components) {
-      Preconditions.checkArgument(components.size() == 1,
-          "Expecting 1 component, got " + components.size());
+      checkArgument(components.size() == 1, "Expecting 1 component, got %s", components.size());
       return of(components.get(0));
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowedValue.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowedValue.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowedValue.java
index 1bbdbd9..d78da41 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowedValue.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowedValue.java
@@ -34,7 +34,6 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo.PaneInfoCoder;
 import org.apache.beam.sdk.util.common.ElementByteSizeObserver;
 
 import com.google.common.base.MoreObjects;
-import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
@@ -72,7 +71,7 @@ public abstract class WindowedValue<T> {
       Instant timestamp,
       Collection<? extends BoundedWindow> windows,
       PaneInfo pane) {
-    Preconditions.checkNotNull(pane);
+    checkNotNull(pane);
 
     if (windows.size() == 0 && BoundedWindow.TIMESTAMP_MIN_VALUE.equals(timestamp)) {
       return valueInEmptyWindows(value, pane);
@@ -91,7 +90,7 @@ public abstract class WindowedValue<T> {
       Instant timestamp,
       BoundedWindow window,
       PaneInfo pane) {
-    Preconditions.checkNotNull(pane);
+    checkNotNull(pane);
 
     boolean isGlobal = GlobalWindow.INSTANCE.equals(window);
     if (isGlobal && BoundedWindow.TIMESTAMP_MIN_VALUE.equals(timestamp)) {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ReflectHelpers.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ReflectHelpers.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ReflectHelpers.java
index 3201271..2b6f6bc 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ReflectHelpers.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ReflectHelpers.java
@@ -17,11 +17,12 @@
  */
 package org.apache.beam.sdk.util.common;
 
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
 import static java.util.Arrays.asList;
 
 import com.google.common.base.Function;
 import com.google.common.base.Joiner;
-import com.google.common.base.Preconditions;
 import com.google.common.collect.FluentIterable;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Queues;
@@ -158,7 +159,7 @@ public class ReflectHelpers {
    * @return
    */
   public static FluentIterable<Class<?>> getClosureOfInterfaces(Class<?> clazz) {
-    Preconditions.checkNotNull(clazz);
+    checkNotNull(clazz);
     Queue<Class<?>> interfacesToProcess = Queues.newArrayDeque();
     Collections.addAll(interfacesToProcess, clazz.getInterfaces());
 
@@ -196,8 +197,8 @@ public class ReflectHelpers {
    * @return An iterable of {@link Method}s which {@code iface} exposes.
    */
   public static Iterable<Method> getClosureOfMethodsOnInterface(Class<?> iface) {
-    Preconditions.checkNotNull(iface);
-    Preconditions.checkArgument(iface.isInterface());
+    checkNotNull(iface);
+    checkArgument(iface.isInterface());
     ImmutableSet.Builder<Method> builder = ImmutableSet.builder();
     Queue<Class<?>> interfacesToProcess = Queues.newArrayDeque();
     interfacesToProcess.add(iface);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/main/java/org/apache/beam/sdk/util/gcsfs/GcsPath.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/gcsfs/GcsPath.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/gcsfs/GcsPath.java
index 3e4b791..e629d64 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/gcsfs/GcsPath.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/gcsfs/GcsPath.java
@@ -17,9 +17,10 @@
  */
 package org.apache.beam.sdk.util.gcsfs;
 
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Strings.isNullOrEmpty;
+
 import com.google.api.services.storage.model.StorageObject;
-import com.google.common.base.Preconditions;
-import com.google.common.base.Strings;
 
 import java.io.File;
 import java.io.IOException;
@@ -80,18 +81,17 @@ public class GcsPath implements Path {
    * contain a port, user info, a query, or a fragment.
    */
   public static GcsPath fromUri(URI uri) {
-    Preconditions.checkArgument(uri.getScheme().equalsIgnoreCase(SCHEME),
-        "URI: %s is not a GCS URI", uri);
-    Preconditions.checkArgument(uri.getPort() == -1,
+    checkArgument(uri.getScheme().equalsIgnoreCase(SCHEME), "URI: %s is not a GCS URI", uri);
+    checkArgument(uri.getPort() == -1,
         "GCS URI may not specify port: %s (%i)", uri, uri.getPort());
-    Preconditions.checkArgument(
-        Strings.isNullOrEmpty(uri.getUserInfo()),
+    checkArgument(
+        isNullOrEmpty(uri.getUserInfo()),
         "GCS URI may not specify userInfo: %s (%s)", uri, uri.getUserInfo());
-    Preconditions.checkArgument(
-        Strings.isNullOrEmpty(uri.getQuery()),
+    checkArgument(
+        isNullOrEmpty(uri.getQuery()),
         "GCS URI may not specify query: %s (%s)", uri, uri.getQuery());
-    Preconditions.checkArgument(
-        Strings.isNullOrEmpty(uri.getFragment()),
+    checkArgument(
+        isNullOrEmpty(uri.getFragment()),
         "GCS URI may not specify fragment: %s (%s)", uri, uri.getFragment());
 
     return fromUri(uri.toString());
@@ -114,9 +114,9 @@ public class GcsPath implements Path {
    */
   public static GcsPath fromUri(String uri) {
     Matcher m = GCS_URI.matcher(uri);
-    Preconditions.checkArgument(m.matches(), "Invalid GCS URI: %s", uri);
+    checkArgument(m.matches(), "Invalid GCS URI: %s", uri);
 
-    Preconditions.checkArgument(m.group("SCHEME").equalsIgnoreCase(SCHEME),
+    checkArgument(m.group("SCHEME").equalsIgnoreCase(SCHEME),
         "URI: %s is not a GCS URI", uri);
     return new GcsPath(null, m.group("BUCKET"), m.group("OBJECT"));
   }
@@ -132,7 +132,7 @@ public class GcsPath implements Path {
    */
   public static GcsPath fromResourceName(String name) {
     Matcher m = GCS_RESOURCE_NAME.matcher(name);
-    Preconditions.checkArgument(m.matches(), "Invalid GCS resource name: %s", name);
+    checkArgument(m.matches(), "Invalid GCS resource name: %s", name);
 
     return new GcsPath(null, m.group("BUCKET"), m.group("OBJECT"));
   }
@@ -202,10 +202,9 @@ public class GcsPath implements Path {
     if (bucket == null) {
       bucket = "";
     }
-    Preconditions.checkArgument(!bucket.contains("/"),
+    checkArgument(!bucket.contains("/"),
         "GCS bucket may not contain a slash");
-    Preconditions
-        .checkArgument(bucket.isEmpty()
+    checkArgument(bucket.isEmpty()
                 || bucket.matches("[a-z0-9][-_a-z0-9.]+[a-z0-9]"),
             "GCS bucket names must contain only lowercase letters, numbers, "
                 + "dashes (-), underscores (_), and dots (.). Bucket names "
@@ -216,7 +215,7 @@ public class GcsPath implements Path {
     if (object == null) {
       object = "";
     }
-    Preconditions.checkArgument(
+    checkArgument(
         object.indexOf('\n') < 0 && object.indexOf('\r') < 0,
         "GCS object names must not contain Carriage Return or "
             + "Line Feed characters.");
@@ -318,32 +317,32 @@ public class GcsPath implements Path {
 
   @Override
   public GcsPath getName(int count) {
-    Preconditions.checkArgument(count >= 0);
+    checkArgument(count >= 0);
 
     Iterator<Path> iterator = iterator();
     for (int i = 0; i < count; ++i) {
-      Preconditions.checkArgument(iterator.hasNext());
+      checkArgument(iterator.hasNext());
       iterator.next();
     }
 
-    Preconditions.checkArgument(iterator.hasNext());
+    checkArgument(iterator.hasNext());
     return (GcsPath) iterator.next();
   }
 
   @Override
   public GcsPath subpath(int beginIndex, int endIndex) {
-    Preconditions.checkArgument(beginIndex >= 0);
-    Preconditions.checkArgument(endIndex > beginIndex);
+    checkArgument(beginIndex >= 0);
+    checkArgument(endIndex > beginIndex);
 
     Iterator<Path> iterator = iterator();
     for (int i = 0; i < beginIndex; ++i) {
-      Preconditions.checkArgument(iterator.hasNext());
+      checkArgument(iterator.hasNext());
       iterator.next();
     }
 
     GcsPath path = null;
     while (beginIndex < endIndex) {
-      Preconditions.checkArgument(iterator.hasNext());
+      checkArgument(iterator.hasNext());
       if (path == null) {
         path = (GcsPath) iterator.next();
       } else {
@@ -492,7 +491,7 @@ public class GcsPath implements Path {
 
     @Override
     public boolean hasNext() {
-      return !Strings.isNullOrEmpty(name);
+      return !isNullOrEmpty(name);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateMerging.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateMerging.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateMerging.java
index b02f01d..843933f 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateMerging.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateMerging.java
@@ -17,9 +17,9 @@
  */
 package org.apache.beam.sdk.util.state;
 
-import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import static com.google.common.base.Preconditions.checkState;
 
-import com.google.common.base.Preconditions;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 
 import org.joda.time.Instant;
 
@@ -228,7 +228,7 @@ public class StateMerging {
       // Update directly from window-derived hold.
       Instant hold = result.getOutputTimeFn().assignOutputTime(
           BoundedWindow.TIMESTAMP_MIN_VALUE, resultWindow);
-      Preconditions.checkState(hold.isAfter(BoundedWindow.TIMESTAMP_MIN_VALUE));
+      checkState(hold.isAfter(BoundedWindow.TIMESTAMP_MIN_VALUE));
       result.add(hold);
     } else {
       // Prefetch.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java
index b0ca70b..6f6c4a1 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java
@@ -21,7 +21,10 @@ import static org.apache.beam.sdk.TestUtils.checkCombineFn;
 import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem;
 import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasNamespace;
 import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom;
+
+import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkNotNull;
+import static com.google.common.base.Preconditions.checkState;
 import static org.hamcrest.Matchers.hasItem;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
@@ -62,7 +65,6 @@ import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionView;
 
 import com.google.common.base.MoreObjects;
-import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Sets;
 
@@ -766,8 +768,7 @@ public class CombineTest implements Serializable {
     public static SetCoder<?> of(
         @JsonProperty(PropertyNames.COMPONENT_ENCODINGS)
         List<Coder<?>> components) {
-      Preconditions.checkArgument(components.size() == 1,
-          "Expecting 1 component, got " + components.size());
+      checkArgument(components.size() == 1, "Expecting 1 component, got %s", components.size());
       return of((Coder<?>) components.get(0));
     }
 
@@ -1094,8 +1095,8 @@ public class CombineTest implements Serializable {
 
       @Override
       public void addInput(Integer element) {
-        Preconditions.checkState(merges == 0);
-        Preconditions.checkState(outputs == 0);
+        checkState(merges == 0);
+        checkState(outputs == 0);
 
         inputs++;
         sum += element;
@@ -1103,8 +1104,8 @@ public class CombineTest implements Serializable {
 
       @Override
       public void mergeAccumulator(Counter accumulator) {
-        Preconditions.checkState(outputs == 0);
-        Preconditions.checkArgument(accumulator.outputs == 0);
+        checkState(outputs == 0);
+        checkArgument(accumulator.outputs == 0);
 
         merges += accumulator.merges + 1;
         inputs += accumulator.inputs;
@@ -1113,7 +1114,7 @@ public class CombineTest implements Serializable {
 
       @Override
       public Iterable<Long> extractOutput() {
-        Preconditions.checkState(outputs == 0);
+        checkState(outputs == 0);
 
         return Arrays.asList(sum, inputs, merges, outputs);
       }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java
index db32fa6..868270c 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java
@@ -25,6 +25,7 @@ import static org.apache.beam.sdk.util.SerializableUtils.serializeToByteArray;
 import static org.apache.beam.sdk.util.StringUtils.byteArrayToJsonString;
 import static org.apache.beam.sdk.util.StringUtils.jsonStringToByteArray;
 
+import static com.google.common.base.Preconditions.checkNotNull;
 import static org.hamcrest.Matchers.allOf;
 import static org.hamcrest.Matchers.anyOf;
 import static org.hamcrest.Matchers.equalTo;
@@ -57,8 +58,6 @@ import org.apache.beam.sdk.values.TimestampedValue;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.sdk.values.TupleTagList;
 
-import com.google.common.base.Preconditions;
-
 import com.fasterxml.jackson.annotation.JsonCreator;
 
 import org.joda.time.Duration;
@@ -292,7 +291,7 @@ public class ParDoTest implements Serializable {
     @Override
     public void processElement(ProcessContext c) {
       Instant timestamp = c.timestamp();
-      Preconditions.checkNotNull(timestamp);
+      checkNotNull(timestamp);
       Integer value = c.element();
       c.outputWithTimestamp(value, timestamp.plus(durationToShift));
     }
@@ -301,7 +300,7 @@ public class ParDoTest implements Serializable {
   static class TestFormatTimestampDoFn extends DoFn<Integer, String> {
     @Override
     public void processElement(ProcessContext c) {
-      Preconditions.checkNotNull(c.timestamp());
+      checkNotNull(c.timestamp());
       c.output("processing: " + c.element() + ", timestamp: " + c.timestamp().getMillis());
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SampleTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SampleTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SampleTest.java
index 4cc0c9b..fe02573 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SampleTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SampleTest.java
@@ -20,6 +20,8 @@ package org.apache.beam.sdk.transforms;
 import static org.apache.beam.sdk.TestUtils.LINES;
 import static org.apache.beam.sdk.TestUtils.NO_LINES;
 import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem;
+
+import static com.google.common.base.Preconditions.checkArgument;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
@@ -35,7 +37,6 @@ import org.apache.beam.sdk.transforms.display.DisplayData;
 import org.apache.beam.sdk.values.PCollection;
 
 import com.google.common.base.Joiner;
-import com.google.common.base.Preconditions;
 
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -219,8 +220,7 @@ public class SampleTest {
   }
 
   void runPickAnyTest(final List<String> lines, int limit) {
-    Preconditions.checkArgument(new HashSet<String>(lines).size() == lines.size(),
-        "Duplicates are unsupported.");
+    checkArgument(new HashSet<String>(lines).size() == lines.size(), "Duplicates are unsupported.");
     Pipeline p = TestPipeline.create();
 
     PCollection<String> input = p.apply(Create.of(lines)

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java
index 5e6e6a3..738b492 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java
@@ -17,6 +17,7 @@
  */
 package org.apache.beam.sdk.transforms;
 
+import static com.google.common.base.Preconditions.checkArgument;
 import static org.hamcrest.Matchers.isA;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -49,7 +50,6 @@ import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.TimestampedValue;
 import org.apache.beam.sdk.values.TypeDescriptor;
 
-import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 
 import org.hamcrest.Matchers;
@@ -204,8 +204,8 @@ public class ViewTest implements Serializable {
             .apply("OutputSideInputs", ParDo.withSideInputs(view).of(new DoFn<Integer, Integer>() {
               @Override
               public void processElement(ProcessContext c) {
-                Preconditions.checkArgument(c.sideInput(view).size() == 4);
-                Preconditions.checkArgument(c.sideInput(view).get(0) == c.sideInput(view).get(0));
+                checkArgument(c.sideInput(view).size() == 4);
+                checkArgument(c.sideInput(view).get(0) == c.sideInput(view).get(0));
                 for (Integer i : c.sideInput(view)) {
                   c.output(i);
                 }
@@ -243,8 +243,8 @@ public class ViewTest implements Serializable {
             .apply("OutputSideInputs", ParDo.withSideInputs(view).of(new DoFn<Integer, Integer>() {
               @Override
               public void processElement(ProcessContext c) {
-                Preconditions.checkArgument(c.sideInput(view).size() == 4);
-                Preconditions.checkArgument(c.sideInput(view).get(0) == c.sideInput(view).get(0));
+                checkArgument(c.sideInput(view).size() == 4);
+                checkArgument(c.sideInput(view).get(0) == c.sideInput(view).get(0));
                 for (Integer i : c.sideInput(view)) {
                   c.output(i);
                 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TriggerTester.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TriggerTester.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TriggerTester.java
index 715d3ef..b321c8f 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TriggerTester.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TriggerTester.java
@@ -42,7 +42,6 @@ import org.apache.beam.sdk.util.state.WatermarkHoldState;
 import org.apache.beam.sdk.values.TimestampedValue;
 
 import com.google.common.base.MoreObjects;
-import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 
@@ -487,7 +486,7 @@ public class TriggerTester<InputT, W extends BoundedWindow> {
 
     @Override
     public Instant currentInputWatermarkTime() {
-      return Preconditions.checkNotNull(inputWatermarkTime);
+      return checkNotNull(inputWatermarkTime);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/extensions/join-library/src/main/java/org/apache/beam/sdk/extensions/joinlibrary/Join.java
----------------------------------------------------------------------
diff --git a/sdks/java/extensions/join-library/src/main/java/org/apache/beam/sdk/extensions/joinlibrary/Join.java b/sdks/java/extensions/join-library/src/main/java/org/apache/beam/sdk/extensions/joinlibrary/Join.java
index 968a613..72abaea 100644
--- a/sdks/java/extensions/join-library/src/main/java/org/apache/beam/sdk/extensions/joinlibrary/Join.java
+++ b/sdks/java/extensions/join-library/src/main/java/org/apache/beam/sdk/extensions/joinlibrary/Join.java
@@ -17,6 +17,8 @@
  */
 package org.apache.beam.sdk.extensions.joinlibrary;
 
+import static com.google.common.base.Preconditions.checkNotNull;
+
 import org.apache.beam.sdk.coders.KvCoder;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.ParDo;
@@ -26,7 +28,6 @@ import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.TupleTag;
-import com.google.common.base.Preconditions;
 
 /**
  * Utility class with different versions of joins. All methods join two collections of
@@ -46,8 +47,8 @@ public class Join {
    */
   public static <K, V1, V2> PCollection<KV<K, KV<V1, V2>>> innerJoin(
     final PCollection<KV<K, V1>> leftCollection, final PCollection<KV<K, V2>> rightCollection) {
-    Preconditions.checkNotNull(leftCollection);
-    Preconditions.checkNotNull(rightCollection);
+    checkNotNull(leftCollection);
+    checkNotNull(rightCollection);
 
     final TupleTag<V1> v1Tuple = new TupleTag<>();
     final TupleTag<V2> v2Tuple = new TupleTag<>();
@@ -94,9 +95,9 @@ public class Join {
     final PCollection<KV<K, V1>> leftCollection,
     final PCollection<KV<K, V2>> rightCollection,
     final V2 nullValue) {
-    Preconditions.checkNotNull(leftCollection);
-    Preconditions.checkNotNull(rightCollection);
-    Preconditions.checkNotNull(nullValue);
+    checkNotNull(leftCollection);
+    checkNotNull(rightCollection);
+    checkNotNull(nullValue);
 
     final TupleTag<V1> v1Tuple = new TupleTag<>();
     final TupleTag<V2> v2Tuple = new TupleTag<>();
@@ -147,9 +148,9 @@ public class Join {
     final PCollection<KV<K, V1>> leftCollection,
     final PCollection<KV<K, V2>> rightCollection,
     final V1 nullValue) {
-    Preconditions.checkNotNull(leftCollection);
-    Preconditions.checkNotNull(rightCollection);
-    Preconditions.checkNotNull(nullValue);
+    checkNotNull(leftCollection);
+    checkNotNull(rightCollection);
+    checkNotNull(nullValue);
 
     final TupleTag<V1> v1Tuple = new TupleTag<>();
     final TupleTag<V2> v2Tuple = new TupleTag<>();

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HDFSFileSource.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HDFSFileSource.java b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HDFSFileSource.java
index 41a271c..7a0545d 100644
--- a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HDFSFileSource.java
+++ b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HDFSFileSource.java
@@ -17,6 +17,9 @@
  */
 package org.apache.beam.sdk.io.hdfs;
 
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.KvCoder;
 import org.apache.beam.sdk.coders.VoidCoder;
@@ -24,10 +27,11 @@ import org.apache.beam.sdk.io.BoundedSource;
 import org.apache.beam.sdk.io.Read;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.values.KV;
+
 import com.google.common.base.Function;
-import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
@@ -41,6 +45,7 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
+
 import java.io.Externalizable;
 import java.io.IOException;
 import java.io.ObjectInput;
@@ -50,6 +55,7 @@ import java.lang.reflect.Method;
 import java.util.List;
 import java.util.ListIterator;
 import java.util.NoSuchElementException;
+
 import javax.annotation.Nullable;
 
 /**
@@ -165,14 +171,10 @@ public class HDFSFileSource<K, V> extends BoundedSource<KV<K, V>> {
 
   @Override
   public void validate() {
-    Preconditions.checkNotNull(filepattern,
-        "need to set the filepattern of a HDFSFileSource");
-    Preconditions.checkNotNull(formatClass,
-        "need to set the format class of a HDFSFileSource");
-    Preconditions.checkNotNull(keyClass,
-        "need to set the key class of a HDFSFileSource");
-    Preconditions.checkNotNull(valueClass,
-        "need to set the value class of a HDFSFileSource");
+    checkNotNull(filepattern, "need to set the filepattern of a HDFSFileSource");
+    checkNotNull(formatClass, "need to set the format class of a HDFSFileSource");
+    checkNotNull(keyClass, "need to set the key class of a HDFSFileSource");
+    checkNotNull(valueClass, "need to set the value class of a HDFSFileSource");
   }
 
   @Override
@@ -467,8 +469,7 @@ public class HDFSFileSource<K, V> extends BoundedSource<KV<K, V>> {
     }
 
     public SerializableSplit(InputSplit split) {
-      Preconditions.checkArgument(split instanceof Writable, "Split is not writable: "
-          + split);
+      checkArgument(split instanceof Writable, "Split is not writable: %s", split);
       this.split = split;
     }
 



[12/50] [abbrv] incubator-beam git commit: pom.xml: upgrade to dataflow version v1b3-rev30-1.22.0

Posted by lc...@apache.org.
pom.xml: upgrade to dataflow version v1b3-rev30-1.22.0


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

Branch: refs/heads/runners-spark2
Commit: 2c53b191ce40746f29bd331f169cbb07c00da5ad
Parents: 562beaf
Author: Joshua Litt <jo...@joshualitt.mtv.corp.google.com>
Authored: Tue Jun 28 09:39:05 2016 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Jul 6 10:18:50 2016 -0700

----------------------------------------------------------------------
 pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/2c53b191/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 582ff62..bf59132 100644
--- a/pom.xml
+++ b/pom.xml
@@ -106,7 +106,7 @@
     <bigquery.version>v2-rev295-1.22.0</bigquery.version>
     <pubsubgrpc.version>0.0.2</pubsubgrpc.version>
     <clouddebugger.version>v2-rev8-1.22.0</clouddebugger.version>
-    <dataflow.version>v1b3-rev26-1.22.0</dataflow.version>
+    <dataflow.version>v1b3-rev30-1.22.0</dataflow.version>
     <dataflow.proto.version>0.5.160222</dataflow.proto.version>
     <datastore.client.version>1.0.0-beta.2</datastore.client.version>
     <datastore.proto.version>1.0.0-beta</datastore.proto.version>


[50/50] [abbrv] incubator-beam git commit: Merge master into runners-spark2 branch to keep up-to-date

Posted by lc...@apache.org.
Merge master into runners-spark2 branch to keep up-to-date

This closes #585


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

Branch: refs/heads/runners-spark2
Commit: 19dd6ccefd66e1c24b43125007cf68b3d9000c9f
Parents: f57e66c d1e3844
Author: Luke Cwik <lc...@google.com>
Authored: Wed Jul 6 10:19:45 2016 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Jul 6 10:19:45 2016 -0700

----------------------------------------------------------------------
 .travis.yml                                     |    9 +-
 README.md                                       |    6 +-
 examples/java/README.md                         |    4 +-
 examples/java/pom.xml                           |   56 +-
 .../beam/examples/DebuggingWordCount.java       |    4 +-
 .../apache/beam/examples/MinimalWordCount.java  |   28 +-
 .../apache/beam/examples/WindowedWordCount.java |    2 +-
 .../org/apache/beam/examples/WordCount.java     |    6 +-
 .../examples/common/DataflowExampleUtils.java   |   28 +-
 .../beam/examples/complete/AutoComplete.java    |   51 +-
 .../examples/complete/StreamingWordExtract.java |   35 +-
 .../apache/beam/examples/complete/TfIdf.java    |   18 +-
 .../examples/complete/TopWikipediaSessions.java |   11 +-
 .../examples/complete/TrafficMaxLaneFlow.java   |   64 +-
 .../beam/examples/complete/TrafficRoutes.java   |   62 +-
 .../examples/cookbook/BigQueryTornadoes.java    |    2 +-
 .../cookbook/CombinePerKeyExamples.java         |    2 +-
 .../examples/cookbook/DatastoreWordCount.java   |   66 +-
 .../beam/examples/cookbook/DeDupExample.java    |    7 +-
 .../beam/examples/cookbook/FilterExamples.java  |    5 +-
 .../beam/examples/cookbook/JoinExamples.java    |    6 +-
 .../examples/cookbook/MaxPerKeyExamples.java    |    2 +-
 .../beam/examples/cookbook/TriggerExample.java  |    6 +-
 examples/java8/pom.xml                          |   36 +-
 .../beam/examples/MinimalWordCountJava8.java    |   30 +-
 .../beam/examples/complete/game/GameStats.java  |   37 +-
 .../examples/complete/game/HourlyTeamScore.java |   11 +-
 .../examples/complete/game/LeaderBoard.java     |   14 +-
 .../beam/examples/complete/game/UserScore.java  |    4 +-
 .../complete/game/injector/InjectorUtils.java   |    6 +-
 .../injector/RetryHttpInitializerWrapper.java   |    5 +-
 .../complete/game/utils/WriteToBigQuery.java    |    2 +-
 .../game/utils/WriteWindowedToBigQuery.java     |    2 +-
 .../complete/game/HourlyTeamScoreTest.java      |    2 +-
 pom.xml                                         |  136 +-
 runners/core-java/pom.xml                       |   68 +-
 .../core/UnboundedReadFromBoundedSource.java    |  542 +++
 .../org/apache/beam/sdk/util/AssignWindows.java |   46 +
 .../apache/beam/sdk/util/AssignWindowsDoFn.java |   80 +
 .../beam/sdk/util/BatchTimerInternals.java      |  141 +
 .../org/apache/beam/sdk/util/DoFnRunner.java    |   62 +
 .../apache/beam/sdk/util/DoFnRunnerBase.java    |  559 +++
 .../org/apache/beam/sdk/util/DoFnRunners.java   |  144 +
 .../beam/sdk/util/GroupAlsoByWindowsDoFn.java   |   59 +
 .../GroupAlsoByWindowsViaOutputBufferDoFn.java  |  100 +
 .../sdk/util/GroupByKeyViaGroupByKeyOnly.java   |  212 ++
 .../sdk/util/LateDataDroppingDoFnRunner.java    |  147 +
 .../org/apache/beam/sdk/util/NonEmptyPanes.java |  150 +
 .../apache/beam/sdk/util/PaneInfoTracker.java   |  155 +
 .../sdk/util/PushbackSideInputDoFnRunner.java   |  115 +
 .../java/org/apache/beam/sdk/util/ReduceFn.java |  130 +
 .../beam/sdk/util/ReduceFnContextFactory.java   |  497 +++
 .../apache/beam/sdk/util/ReduceFnRunner.java    |  988 ++++++
 .../apache/beam/sdk/util/SimpleDoFnRunner.java  |   56 +
 .../apache/beam/sdk/util/SystemReduceFn.java    |  135 +
 .../org/apache/beam/sdk/util/TriggerRunner.java |  235 ++
 .../org/apache/beam/sdk/util/WatermarkHold.java |  537 +++
 .../util/common/ElementByteSizeObservable.java  |   42 +
 .../beam/sdk/util/common/PeekingReiterator.java |   99 +
 .../UnboundedReadFromBoundedSourceTest.java     |  374 ++
 .../beam/sdk/util/BatchTimerInternalsTest.java  |  118 +
 .../sdk/util/GroupAlsoByWindowsProperties.java  |  623 ++++
 ...oupAlsoByWindowsViaOutputBufferDoFnTest.java |  106 +
 .../util/LateDataDroppingDoFnRunnerTest.java    |  117 +
 .../util/PushbackSideInputDoFnRunnerTest.java   |  234 ++
 .../beam/sdk/util/ReduceFnRunnerTest.java       | 1446 ++++++++
 .../apache/beam/sdk/util/ReduceFnTester.java    |  788 +++++
 .../beam/sdk/util/SimpleDoFnRunnerTest.java     |   86 +
 runners/direct-java/pom.xml                     |   18 +-
 .../direct/AvroIOShardedWriteFactory.java       |   76 -
 .../beam/runners/direct/DirectGroupByKey.java   |    2 +-
 .../beam/runners/direct/DirectRunner.java       |    4 -
 .../runners/direct/ShardControlledWrite.java    |   81 -
 .../direct/TextIOShardedWriteFactory.java       |   78 -
 .../direct/UnboundedReadDeduplicator.java       |  102 +
 .../direct/UnboundedReadEvaluatorFactory.java   |   18 +-
 .../beam/runners/direct/WatermarkManager.java   |    5 +-
 .../runners/direct/WindowEvaluatorFactory.java  |   18 +-
 .../beam/runners/direct/package-info.java       |   25 +
 .../direct/AvroIOShardedWriteFactoryTest.java   |  120 -
 .../runners/direct/CommittedResultTest.java     |    4 +
 .../direct/GroupByKeyEvaluatorFactoryTest.java  |    2 +-
 .../GroupByKeyOnlyEvaluatorFactoryTest.java     |    2 +-
 .../runners/direct/SideInputContainerTest.java  |    1 +
 .../direct/TextIOShardedWriteFactoryTest.java   |  120 -
 .../direct/UnboundedReadDeduplicatorTest.java   |  134 +
 .../UnboundedReadEvaluatorFactoryTest.java      |   50 +-
 .../direct/WindowEvaluatorFactoryTest.java      |  178 +-
 runners/flink/README.md                         |    4 +-
 runners/flink/examples/pom.xml                  |   14 +
 .../beam/runners/flink/examples/TFIDF.java      |   57 +-
 .../beam/runners/flink/examples/WordCount.java  |    8 +-
 .../flink/examples/streaming/AutoComplete.java  |   17 +-
 .../flink/examples/streaming/JoinExamples.java  |   21 +-
 .../examples/streaming/KafkaIOExamples.java     |    4 +-
 .../KafkaWindowedWordCountExample.java          |    6 +-
 .../examples/streaming/WindowedWordCount.java   |    7 +-
 runners/flink/pom.xml                           |    6 +-
 runners/flink/runner/pom.xml                    |   40 +-
 .../FlinkPipelineExecutionEnvironment.java      |    6 +-
 .../beam/runners/flink/FlinkPipelineRunner.java |  180 -
 .../apache/beam/runners/flink/FlinkRunner.java  |  179 +
 .../runners/flink/FlinkRunnerRegistrar.java     |    4 +-
 .../beam/runners/flink/FlinkRunnerResult.java   |    8 +
 .../runners/flink/TestFlinkPipelineRunner.java  |   81 -
 .../beam/runners/flink/TestFlinkRunner.java     |   81 +
 .../FlinkBatchTransformTranslators.java         |   49 +-
 .../FlinkBatchTranslationContext.java           |   13 +-
 .../FlinkStreamingTransformTranslators.java     |    5 +-
 .../FlinkStreamingTranslationContext.java       |    8 +-
 .../functions/FlinkAssignContext.java           |   15 +-
 .../functions/FlinkNoElementAssignContext.java  |    4 +-
 .../functions/FlinkProcessContext.java          |   25 +-
 .../translation/types/CoderComparator.java      |  217 --
 .../translation/types/CoderTypeInformation.java |   20 +-
 .../translation/types/CoderTypeSerializer.java  |   37 +-
 .../types/EncodedValueComparator.java           |  197 ++
 .../types/EncodedValueSerializer.java           |  113 +
 .../types/EncodedValueTypeInformation.java      |  111 +
 .../translation/types/KvCoderComperator.java    |  259 --
 .../types/KvCoderTypeInformation.java           |  207 --
 .../flink/translation/types/KvKeySelector.java  |   51 +
 .../utils/SerializedPipelineOptions.java        |    8 +-
 .../streaming/FlinkAbstractParDoWrapper.java    |   14 +-
 .../FlinkGroupAlsoByWindowWrapper.java          |   23 +-
 .../streaming/FlinkParDoBoundMultiWrapper.java  |    8 +-
 .../streaming/io/UnboundedFlinkSink.java        |    2 +-
 .../streaming/io/UnboundedFlinkSource.java      |   13 +-
 .../streaming/state/FlinkStateInternals.java    |    9 +-
 .../flink/EncodedValueComparatorTest.java       |   71 +
 .../runners/flink/FlinkRunnerRegistrarTest.java |    8 +-
 .../beam/runners/flink/FlinkTestPipeline.java   |    4 +-
 .../flink/streaming/GroupAlsoByWindowTest.java  |    4 +-
 runners/google-cloud-dataflow-java/pom.xml      |   27 +-
 .../BlockingDataflowPipelineRunner.java         |  186 -
 .../dataflow/BlockingDataflowRunner.java        |  186 +
 .../dataflow/DataflowJobCancelledException.java |    2 +-
 .../dataflow/DataflowJobExecutionException.java |    2 +-
 .../dataflow/DataflowJobUpdatedException.java   |    2 +-
 .../runners/dataflow/DataflowPipelineJob.java   |    4 +-
 .../dataflow/DataflowPipelineRegistrar.java     |    8 +-
 .../dataflow/DataflowPipelineRunner.java        | 3229 ------------------
 .../dataflow/DataflowPipelineRunnerHooks.java   |   39 -
 .../dataflow/DataflowPipelineTranslator.java    |   25 +-
 .../beam/runners/dataflow/DataflowRunner.java   | 2997 ++++++++++++++++
 .../runners/dataflow/DataflowRunnerHooks.java   |   39 +
 .../dataflow/internal/AssignWindows.java        |    2 +-
 .../DataflowUnboundedReadFromBoundedSource.java |  547 +++
 .../runners/dataflow/internal/IsmFormat.java    |    8 +-
 .../runners/dataflow/internal/package-info.java |    2 +-
 .../BlockingDataflowPipelineOptions.java        |    6 +-
 .../options/DataflowPipelineOptions.java        |    4 +-
 .../DataflowPipelineWorkerPoolOptions.java      |    6 +-
 .../options/DataflowWorkerLoggingOptions.java   |   14 +-
 .../testing/TestDataflowPipelineRunner.java     |  271 --
 .../dataflow/testing/TestDataflowRunner.java    |  271 ++
 .../dataflow/util/DataflowPathValidator.java    |   14 +-
 .../dataflow/util/DataflowTransport.java        |   11 -
 .../beam/runners/dataflow/util/GcsStager.java   |    5 +-
 .../util/GroupAlsoByWindowViaWindowSetDoFn.java |   40 -
 .../BlockingDataflowPipelineRunnerTest.java     |  304 --
 .../dataflow/BlockingDataflowRunnerTest.java    |  304 ++
 .../dataflow/DataflowPipelineRegistrarTest.java |    4 +-
 .../dataflow/DataflowPipelineRunnerTest.java    | 1417 --------
 .../DataflowPipelineTranslatorTest.java         |   63 +-
 .../runners/dataflow/DataflowRunnerTest.java    | 1387 ++++++++
 .../dataflow/RecordingPipelineVisitor.java      |   46 +
 .../runners/dataflow/io/DataflowAvroIOTest.java |    4 +-
 .../dataflow/io/DataflowDatastoreIOTest.java    |   25 +-
 .../dataflow/io/DataflowPubsubIOTest.java       |    4 +-
 .../runners/dataflow/io/DataflowTextIOTest.java |    4 +-
 .../testing/TestDataflowPipelineRunnerTest.java |  601 ----
 .../testing/TestDataflowRunnerTest.java         |  601 ++++
 .../DataflowDisplayDataEvaluator.java           |   10 +-
 .../transforms/DataflowGroupByKeyTest.java      |    8 +-
 .../dataflow/transforms/DataflowViewTest.java   |    8 +-
 .../util/DataflowPathValidatorTest.java         |    8 +-
 runners/spark/README.md                         |    8 +-
 runners/spark/pom.xml                           |  203 +-
 .../beam/runners/spark/SparkPipelineRunner.java |  255 --
 .../apache/beam/runners/spark/SparkRunner.java  |  255 ++
 .../runners/spark/SparkRunnerRegistrar.java     |    7 +-
 .../beam/runners/spark/TestSparkRunner.java     |   77 +
 .../beam/runners/spark/io/CreateStream.java     |    7 +-
 .../apache/beam/runners/spark/io/KafkaIO.java   |   23 +-
 .../beam/runners/spark/io/hadoop/HadoopIO.java  |   38 +-
 .../translation/SparkPipelineEvaluator.java     |    6 +-
 .../StreamingWindowPipelineDetector.java        |    6 +-
 .../apache/beam/runners/spark/DeDupTest.java    |    4 +-
 .../beam/runners/spark/EmptyInputTest.java      |    4 +-
 .../beam/runners/spark/SimpleWordCountTest.java |   13 +-
 .../runners/spark/SparkRunnerRegistrarTest.java |    2 +-
 .../apache/beam/runners/spark/TfIdfTest.java    |    4 +-
 .../beam/runners/spark/io/AvroPipelineTest.java |    4 +-
 .../beam/runners/spark/io/NumShardsTest.java    |    6 +-
 .../io/hadoop/HadoopFileFormatPipelineTest.java |    4 +-
 .../spark/translation/CombineGloballyTest.java  |    6 +-
 .../spark/translation/CombinePerKeyTest.java    |    4 +-
 .../spark/translation/DoFnOutputTest.java       |    6 +-
 .../translation/MultiOutputWordCountTest.java   |    4 +-
 .../spark/translation/SerializationTest.java    |    8 +-
 .../spark/translation/SideEffectsTest.java      |    4 +-
 .../translation/TransformTranslatorTest.java    |    6 +-
 .../translation/WindowedWordCountTest.java      |    8 +-
 .../streaming/FlattenStreamingTest.java         |    6 +-
 .../streaming/KafkaStreamingTest.java           |    6 +-
 .../streaming/SimpleStreamingWordCountTest.java |    6 +-
 .../src/main/resources/beam/findbugs-filter.xml |  561 +++
 sdks/java/core/pom.xml                          |   33 +-
 .../main/java/org/apache/beam/sdk/Pipeline.java |    7 +-
 .../org/apache/beam/sdk/coders/AvroCoder.java   |   10 +-
 .../java/org/apache/beam/sdk/coders/Coder.java  |    6 +-
 .../apache/beam/sdk/coders/CoderRegistry.java   |    5 +-
 .../apache/beam/sdk/coders/CollectionCoder.java |    7 +-
 .../org/apache/beam/sdk/coders/EntityCoder.java |   87 -
 .../apache/beam/sdk/coders/IterableCoder.java   |    7 +-
 .../beam/sdk/coders/IterableLikeCoder.java      |   10 +-
 .../org/apache/beam/sdk/coders/JAXBCoder.java   |   28 +-
 .../org/apache/beam/sdk/coders/KvCoder.java     |    7 +-
 .../org/apache/beam/sdk/coders/ListCoder.java   |    7 +-
 .../org/apache/beam/sdk/coders/MapCoder.java    |    6 +-
 .../apache/beam/sdk/coders/NullableCoder.java   |    6 +-
 .../org/apache/beam/sdk/coders/SetCoder.java    |    7 +-
 .../java/org/apache/beam/sdk/io/AvroIO.java     |   71 +-
 .../java/org/apache/beam/sdk/io/AvroSource.java |   12 +-
 .../java/org/apache/beam/sdk/io/BigQueryIO.java |  214 +-
 .../sdk/io/BoundedReadFromUnboundedSource.java  |    6 +-
 .../apache/beam/sdk/io/CompressedSource.java    |   81 +-
 .../org/apache/beam/sdk/io/DatastoreIO.java     |  988 ------
 .../org/apache/beam/sdk/io/FileBasedSink.java   |   13 +-
 .../apache/beam/sdk/io/OffsetBasedSource.java   |   38 +-
 .../java/org/apache/beam/sdk/io/PubsubIO.java   |   37 +-
 .../apache/beam/sdk/io/PubsubUnboundedSink.java |   24 +-
 .../beam/sdk/io/PubsubUnboundedSource.java      |    5 +-
 .../main/java/org/apache/beam/sdk/io/Read.java  |   29 +-
 .../java/org/apache/beam/sdk/io/TextIO.java     |   87 +-
 .../org/apache/beam/sdk/io/UnboundedSource.java |   28 +-
 .../main/java/org/apache/beam/sdk/io/Write.java |  314 +-
 .../java/org/apache/beam/sdk/io/XmlSink.java    |   10 +-
 .../java/org/apache/beam/sdk/io/XmlSource.java  |   11 +-
 .../beam/sdk/io/datastore/DatastoreIO.java      |   41 +
 .../apache/beam/sdk/io/datastore/V1Beta3.java   |  992 ++++++
 .../beam/sdk/io/datastore/package-info.java     |   24 +
 .../org/apache/beam/sdk/io/package-info.java    |    6 +-
 .../beam/sdk/io/range/ByteKeyRangeTracker.java  |   63 +-
 .../beam/sdk/io/range/OffsetRangeTracker.java   |   13 +-
 .../beam/sdk/options/DirectPipelineOptions.java |   74 -
 .../beam/sdk/options/PipelineOptions.java       |    7 +-
 .../sdk/options/PipelineOptionsFactory.java     |   28 +-
 .../sdk/options/PipelineOptionsValidator.java   |   18 +-
 .../sdk/options/ProxyInvocationHandler.java     |    9 +-
 .../apache/beam/sdk/runners/PipelineRunner.java |    6 +-
 .../sdk/runners/RecordingPipelineVisitor.java   |   47 -
 .../beam/sdk/runners/TransformHierarchy.java    |   10 +-
 .../beam/sdk/runners/TransformTreeNode.java     |   13 +-
 .../apache/beam/sdk/runners/package-info.java   |   12 +-
 .../org/apache/beam/sdk/testing/PAssert.java    |  371 +-
 .../apache/beam/sdk/testing/StaticWindows.java  |  110 +
 .../apache/beam/sdk/testing/TestPipeline.java   |    2 +-
 .../beam/sdk/testing/WindowFnTestUtils.java     |    5 +-
 .../apache/beam/sdk/testing/WindowSupplier.java |   83 +
 .../apache/beam/sdk/transforms/Aggregator.java  |    2 +-
 .../sdk/transforms/ApproximateQuantiles.java    |    9 +-
 .../org/apache/beam/sdk/transforms/Combine.java |   54 +-
 .../org/apache/beam/sdk/transforms/Count.java   |    2 +-
 .../org/apache/beam/sdk/transforms/Create.java  |    5 +-
 .../beam/sdk/transforms/DoFnReflector.java      |   12 +-
 .../beam/sdk/transforms/FlatMapElements.java    |    2 +-
 .../org/apache/beam/sdk/transforms/Flatten.java |    2 +-
 .../transforms/IntraBundleParallelization.java  |    9 +-
 .../org/apache/beam/sdk/transforms/Keys.java    |   13 +-
 .../org/apache/beam/sdk/transforms/KvSwap.java  |   15 +-
 .../apache/beam/sdk/transforms/MapElements.java |    2 +-
 .../apache/beam/sdk/transforms/PTransform.java  |   27 +-
 .../org/apache/beam/sdk/transforms/ParDo.java   |   70 +-
 .../beam/sdk/transforms/RemoveDuplicates.java   |   13 +-
 .../org/apache/beam/sdk/transforms/Sample.java  |    6 +-
 .../org/apache/beam/sdk/transforms/Top.java     |    7 +-
 .../org/apache/beam/sdk/transforms/Values.java  |   13 +-
 .../apache/beam/sdk/transforms/WithKeys.java    |   15 +-
 .../beam/sdk/transforms/WithTimestamps.java     |    2 +-
 .../sdk/transforms/display/DisplayData.java     |    4 +-
 .../beam/sdk/transforms/join/CoGbkResult.java   |    6 +-
 .../beam/sdk/transforms/join/CoGroupByKey.java  |   10 +-
 .../beam/sdk/transforms/windowing/AfterAll.java |    6 +-
 .../sdk/transforms/windowing/AfterFirst.java    |    9 +-
 .../transforms/windowing/AfterWatermark.java    |   38 +-
 .../sdk/transforms/windowing/GlobalWindows.java |    5 -
 .../beam/sdk/transforms/windowing/PaneInfo.java |   15 +-
 .../windowing/PartitioningWindowFn.java         |    5 -
 .../beam/sdk/transforms/windowing/Trigger.java  |    7 +-
 .../transforms/windowing/TriggerBuilder.java    |   29 -
 .../beam/sdk/transforms/windowing/Window.java   |   73 +-
 .../beam/sdk/transforms/windowing/WindowFn.java |   11 +-
 .../org/apache/beam/sdk/util/AssignWindows.java |   46 -
 .../apache/beam/sdk/util/AssignWindowsDoFn.java |   75 -
 ...AttemptAndTimeBoundedExponentialBackOff.java |   11 +-
 .../util/AttemptBoundedExponentialBackOff.java  |    9 +-
 .../beam/sdk/util/BatchTimerInternals.java      |  140 -
 .../apache/beam/sdk/util/BigQueryServices.java  |   12 +
 .../beam/sdk/util/BigQueryServicesImpl.java     |   37 +
 .../beam/sdk/util/BigQueryTableInserter.java    |   36 +-
 .../beam/sdk/util/BigQueryTableRowIterator.java |    4 +-
 .../org/apache/beam/sdk/util/Credentials.java   |    5 +-
 .../org/apache/beam/sdk/util/DoFnRunner.java    |   62 -
 .../apache/beam/sdk/util/DoFnRunnerBase.java    |  558 ---
 .../org/apache/beam/sdk/util/DoFnRunners.java   |  144 -
 .../apache/beam/sdk/util/ExecutableTrigger.java |   11 +-
 .../apache/beam/sdk/util/GatherAllPanes.java    |   16 +-
 .../java/org/apache/beam/sdk/util/GcsUtil.java  |   10 +-
 .../beam/sdk/util/GroupAlsoByWindowsDoFn.java   |   59 -
 .../GroupAlsoByWindowsViaOutputBufferDoFn.java  |  100 -
 .../sdk/util/GroupByKeyViaGroupByKeyOnly.java   |  247 --
 .../apache/beam/sdk/util/IdentityWindowFn.java  |   20 +-
 .../apache/beam/sdk/util/InstanceBuilder.java   |   21 +-
 .../util/IntervalBoundedExponentialBackOff.java |   10 +-
 .../sdk/util/LateDataDroppingDoFnRunner.java    |  147 -
 .../beam/sdk/util/MergingActiveWindowSet.java   |   35 +-
 .../org/apache/beam/sdk/util/NonEmptyPanes.java |  150 -
 .../apache/beam/sdk/util/PaneInfoTracker.java   |  154 -
 .../sdk/util/PushbackSideInputDoFnRunner.java   |  115 -
 .../java/org/apache/beam/sdk/util/ReduceFn.java |  130 -
 .../beam/sdk/util/ReduceFnContextFactory.java   |  497 ---
 .../apache/beam/sdk/util/ReduceFnRunner.java    |  985 ------
 .../sdk/util/ReifyTimestampsAndWindows.java     |   63 +
 .../org/apache/beam/sdk/util/Reshuffle.java     |    6 +-
 .../apache/beam/sdk/util/SerializableUtils.java |   13 +-
 .../apache/beam/sdk/util/SimpleDoFnRunner.java  |   56 -
 .../org/apache/beam/sdk/util/StringUtils.java   |   12 +-
 .../apache/beam/sdk/util/SystemReduceFn.java    |  135 -
 .../apache/beam/sdk/util/TimerInternals.java    |    5 +-
 .../beam/sdk/util/TriggerContextFactory.java    |   11 +-
 .../org/apache/beam/sdk/util/TriggerRunner.java |  234 --
 .../apache/beam/sdk/util/ValueWithRecordId.java |   25 +-
 .../org/apache/beam/sdk/util/WatermarkHold.java |  536 ---
 .../org/apache/beam/sdk/util/WindowedValue.java |  204 +-
 .../java/org/apache/beam/sdk/util/ZipFiles.java |   11 +
 .../util/common/ElementByteSizeObservable.java  |   42 -
 .../beam/sdk/util/common/PeekingReiterator.java |   99 -
 .../beam/sdk/util/common/ReflectHelpers.java    |    9 +-
 .../org/apache/beam/sdk/util/gcsfs/GcsPath.java |   51 +-
 .../beam/sdk/util/state/StateMerging.java       |    6 +-
 .../org/apache/beam/sdk/WindowMatchers.java     |   80 +-
 .../org/apache/beam/sdk/WindowMatchersTest.java |   84 +
 .../apache/beam/sdk/coders/AvroCoderTest.java   |   26 +
 .../apache/beam/sdk/coders/EntityCoderTest.java |  110 -
 .../apache/beam/sdk/coders/JAXBCoderTest.java   |   69 +-
 .../beam/sdk/coders/SerializableCoderTest.java  |    5 +
 .../sdk/coders/protobuf/ProtobufUtilTest.java   |    7 +-
 .../beam/sdk/io/AvroIOGeneratedClassTest.java   |  192 +-
 .../java/org/apache/beam/sdk/io/AvroIOTest.java |    5 +-
 .../org/apache/beam/sdk/io/BigQueryIOTest.java  |  128 +-
 .../beam/sdk/io/CompressedSourceTest.java       |   85 +
 .../org/apache/beam/sdk/io/DatastoreIOTest.java |  621 ----
 .../apache/beam/sdk/io/FileBasedSourceTest.java |    5 +-
 .../beam/sdk/io/OffsetBasedSourceTest.java      |    5 +-
 .../org/apache/beam/sdk/io/PubsubIOTest.java    |    4 -
 .../java/org/apache/beam/sdk/io/TextIOTest.java |  210 +-
 .../java/org/apache/beam/sdk/io/WriteTest.java  |  145 +-
 .../org/apache/beam/sdk/io/XmlSourceTest.java   |   19 +-
 .../beam/sdk/io/datastore/V1Beta3Test.java      |  584 ++++
 .../sdk/io/range/ByteKeyRangeTrackerTest.java   |   58 +-
 .../sdk/io/range/OffsetRangeTrackerTest.java    |   91 +-
 .../beam/sdk/runners/PipelineRunnerTest.java    |   26 +-
 .../beam/sdk/runners/TransformTreeTest.java     |    6 +-
 .../apache/beam/sdk/testing/PAssertTest.java    |  116 +
 .../beam/sdk/testing/StaticWindowsTest.java     |   94 +
 .../beam/sdk/testing/WindowSupplierTest.java    |   89 +
 .../apache/beam/sdk/transforms/CombineTest.java |   17 +-
 .../apache/beam/sdk/transforms/CreateTest.java  |    4 +
 .../beam/sdk/transforms/GroupByKeyTest.java     |  206 +-
 .../beam/sdk/transforms/PTransformTest.java     |    7 +-
 .../apache/beam/sdk/transforms/ParDoTest.java   |   34 +-
 .../apache/beam/sdk/transforms/SampleTest.java  |    6 +-
 .../apache/beam/sdk/transforms/ViewTest.java    |   10 +-
 .../display/DisplayDataEvaluator.java           |   29 +-
 .../display/DisplayDataEvaluatorTest.java       |   14 +
 .../display/DisplayDataMatchersTest.java        |   18 +-
 .../sdk/transforms/display/DisplayDataTest.java |    6 +-
 .../windowing/AfterProcessingTimeTest.java      |    3 +-
 .../windowing/AfterWatermarkTest.java           |   24 +-
 .../sdk/transforms/windowing/WindowTest.java    |    6 +-
 .../sdk/transforms/windowing/WindowingTest.java |   10 +-
 .../apache/beam/sdk/util/ApiSurfaceTest.java    |    4 +-
 .../beam/sdk/util/BatchTimerInternalsTest.java  |  118 -
 .../sdk/util/BigQueryTableInserterTest.java     |   17 +-
 .../apache/beam/sdk/util/BigQueryUtilTest.java  |   12 +-
 .../beam/sdk/util/GatherAllPanesTest.java       |   16 +-
 .../sdk/util/GroupAlsoByWindowsProperties.java  |  619 ----
 ...oupAlsoByWindowsViaOutputBufferDoFnTest.java |  106 -
 .../sdk/util/IdentitySideInputWindowFn.java     |    3 +-
 .../util/LateDataDroppingDoFnRunnerTest.java    |  117 -
 .../sdk/util/MergingActiveWindowSetTest.java    |    6 +-
 .../util/PushbackSideInputDoFnRunnerTest.java   |  234 --
 .../beam/sdk/util/ReduceFnRunnerTest.java       | 1448 --------
 .../apache/beam/sdk/util/ReduceFnTester.java    |  784 -----
 .../util/RetryHttpRequestInitializerTest.java   |    5 +-
 .../beam/sdk/util/SimpleDoFnRunnerTest.java     |   86 -
 .../apache/beam/sdk/util/StringUtilsTest.java   |   19 +-
 .../org/apache/beam/sdk/util/TriggerTester.java |   30 +-
 sdks/java/extensions/join-library/pom.xml       |    9 +
 .../beam/sdk/extensions/joinlibrary/Join.java   |   19 +-
 sdks/java/io/google-cloud-platform/pom.xml      |   92 +-
 .../beam/sdk/io/gcp/bigtable/BigtableIO.java    |   71 +-
 .../io/gcp/bigtable/BigtableServiceImpl.java    |   31 +-
 .../io/gcp/bigtable/BigtableTestOptions.java    |   42 +
 .../sdk/io/gcp/bigtable/BigtableIOTest.java     |   94 +-
 .../sdk/io/gcp/bigtable/BigtableReadIT.java     |   61 +
 .../sdk/io/gcp/bigtable/BigtableWriteIT.java    |  211 ++
 sdks/java/io/hdfs/pom.xml                       |   62 +
 .../beam/sdk/io/hdfs/AvroHDFSFileSource.java    |  145 +
 .../beam/sdk/io/hdfs/AvroWrapperCoder.java      |  116 +
 .../apache/beam/sdk/io/hdfs/HDFSFileSink.java   |  277 ++
 .../apache/beam/sdk/io/hdfs/HDFSFileSource.java |   63 +-
 .../apache/beam/sdk/io/hdfs/WritableCoder.java  |    9 +-
 .../SimpleAuthAvroHDFSFileSource.java           |   84 +
 .../hdfs/simpleauth/SimpleAuthHDFSFileSink.java |  132 +
 .../simpleauth/SimpleAuthHDFSFileSource.java    |  122 +
 .../beam/sdk/io/hdfs/AvroWrapperCoderTest.java  |   52 +
 .../beam/sdk/io/hdfs/WritableCoderTest.java     |    9 +
 sdks/java/io/kafka/pom.xml                      |   34 +
 .../org/apache/beam/sdk/io/kafka/KafkaIO.java   |   11 +-
 .../apache/beam/sdk/io/kafka/KafkaIOTest.java   |   10 +-
 sdks/java/java8tests/pom.xml                    |   15 +-
 .../src/main/java/DebuggingWordCount.java       |    4 +-
 .../src/main/java/MinimalWordCount.java         |    8 +-
 .../src/main/java/WindowedWordCount.java        |    2 +-
 .../src/main/java/WordCount.java                |    6 +-
 .../main/java/common/DataflowExampleUtils.java  |   16 +-
 .../src/main/java/StarterPipeline.java          |    4 +-
 .../src/main/java/it/pkg/StarterPipeline.java   |    4 +-
 sdks/java/microbenchmarks/README.md             |   42 +
 sdks/java/microbenchmarks/pom.xml               |  110 +
 .../coders/AvroCoderBenchmark.java              |  121 +
 .../coders/ByteArrayCoderBenchmark.java         |   66 +
 .../coders/CoderBenchmarking.java               |   42 +
 .../coders/StringUtf8CoderBenchmark.java        |   72 +
 .../transforms/DoFnReflectorBenchmark.java      |  239 ++
 sdks/java/pom.xml                               |    3 +-
 439 files changed, 26154 insertions(+), 20743 deletions(-)
----------------------------------------------------------------------



[24/50] [abbrv] incubator-beam git commit: Move some more easy stuff to runners-core

Posted by lc...@apache.org.
Move some more easy stuff to runners-core


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

Branch: refs/heads/runners-spark2
Commit: 722bfcaf0f8d908e263fb578a25aa7bc12cc602e
Parents: 5472c97
Author: Kenneth Knowles <kl...@google.com>
Authored: Thu Jun 23 16:46:41 2016 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Jul 6 10:18:51 2016 -0700

----------------------------------------------------------------------
 .../util/common/ElementByteSizeObservable.java  | 42 +++++++++
 .../beam/sdk/util/common/PeekingReiterator.java | 99 ++++++++++++++++++++
 .../util/common/ElementByteSizeObservable.java  | 42 ---------
 .../beam/sdk/util/common/PeekingReiterator.java | 99 --------------------
 4 files changed, 141 insertions(+), 141 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/722bfcaf/runners/core-java/src/main/java/org/apache/beam/sdk/util/common/ElementByteSizeObservable.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/common/ElementByteSizeObservable.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/common/ElementByteSizeObservable.java
new file mode 100644
index 0000000..613aa4b
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/common/ElementByteSizeObservable.java
@@ -0,0 +1,42 @@
+/*
+ * 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.util.common;
+
+/**
+ * An interface for things that allow observing the size in bytes of
+ * encoded values of type {@code T}.
+ *
+ * @param <T> the type of the values being observed
+ */
+public interface ElementByteSizeObservable<T> {
+  /**
+   * Returns whether {@link #registerByteSizeObserver} is cheap enough
+   * to call for every element, that is, if this
+   * {@code ElementByteSizeObservable} can calculate the byte size of
+   * the element to be coded in roughly constant time (or lazily).
+   */
+  public boolean isRegisterByteSizeObserverCheap(T value);
+
+  /**
+   * Notifies the {@code ElementByteSizeObserver} about the byte size
+   * of the encoded value using this {@code ElementByteSizeObservable}.
+   */
+  public void registerByteSizeObserver(T value,
+                                       ElementByteSizeObserver observer)
+      throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/722bfcaf/runners/core-java/src/main/java/org/apache/beam/sdk/util/common/PeekingReiterator.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/common/PeekingReiterator.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/common/PeekingReiterator.java
new file mode 100644
index 0000000..1e3c17f
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/common/PeekingReiterator.java
@@ -0,0 +1,99 @@
+/*
+ * 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.util.common;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+import static com.google.common.base.Preconditions.checkState;
+
+import java.util.NoSuchElementException;
+
+/**
+ * A {@link Reiterator} that supports one-element lookahead during iteration.
+ *
+ * @param <T> the type of elements returned by this iterator
+ */
+public final class PeekingReiterator<T> implements Reiterator<T> {
+  private T nextElement;
+  private boolean nextElementComputed;
+  private final Reiterator<T> iterator;
+
+  public PeekingReiterator(Reiterator<T> iterator) {
+    this.iterator = checkNotNull(iterator);
+  }
+
+  PeekingReiterator(PeekingReiterator<T> it) {
+    this.iterator = checkNotNull(checkNotNull(it).iterator.copy());
+    this.nextElement = it.nextElement;
+    this.nextElementComputed = it.nextElementComputed;
+  }
+
+  @Override
+  public boolean hasNext() {
+    computeNext();
+    return nextElementComputed;
+  }
+
+  @Override
+  public T next() {
+    T result = peek();
+    nextElementComputed = false;
+    return result;
+  }
+
+  /**
+   * {@inheritDoc}
+   *
+   * <p>If {@link #peek} is called, {@code remove} is disallowed until
+   * {@link #next} has been subsequently called.
+   */
+  @Override
+  public void remove() {
+    checkState(!nextElementComputed,
+        "After peek(), remove() is disallowed until next() is called");
+    iterator.remove();
+  }
+
+  @Override
+  public PeekingReiterator<T> copy() {
+    return new PeekingReiterator<>(this);
+  }
+
+  /**
+   * Returns the element that would be returned by {@link #next}, without
+   * actually consuming the element.
+   * @throws NoSuchElementException if there is no next element
+   */
+  public T peek() {
+    computeNext();
+    if (!nextElementComputed) {
+      throw new NoSuchElementException();
+    }
+    return nextElement;
+  }
+
+  private void computeNext() {
+    if (nextElementComputed) {
+      return;
+    }
+    if (!iterator.hasNext()) {
+      return;
+    }
+    nextElement = iterator.next();
+    nextElementComputed = true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/722bfcaf/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ElementByteSizeObservable.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ElementByteSizeObservable.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ElementByteSizeObservable.java
deleted file mode 100644
index 613aa4b..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ElementByteSizeObservable.java
+++ /dev/null
@@ -1,42 +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.util.common;
-
-/**
- * An interface for things that allow observing the size in bytes of
- * encoded values of type {@code T}.
- *
- * @param <T> the type of the values being observed
- */
-public interface ElementByteSizeObservable<T> {
-  /**
-   * Returns whether {@link #registerByteSizeObserver} is cheap enough
-   * to call for every element, that is, if this
-   * {@code ElementByteSizeObservable} can calculate the byte size of
-   * the element to be coded in roughly constant time (or lazily).
-   */
-  public boolean isRegisterByteSizeObserverCheap(T value);
-
-  /**
-   * Notifies the {@code ElementByteSizeObserver} about the byte size
-   * of the encoded value using this {@code ElementByteSizeObservable}.
-   */
-  public void registerByteSizeObserver(T value,
-                                       ElementByteSizeObserver observer)
-      throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/722bfcaf/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/PeekingReiterator.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/PeekingReiterator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/PeekingReiterator.java
deleted file mode 100644
index 1e3c17f..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/PeekingReiterator.java
+++ /dev/null
@@ -1,99 +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.util.common;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-import static com.google.common.base.Preconditions.checkState;
-
-import java.util.NoSuchElementException;
-
-/**
- * A {@link Reiterator} that supports one-element lookahead during iteration.
- *
- * @param <T> the type of elements returned by this iterator
- */
-public final class PeekingReiterator<T> implements Reiterator<T> {
-  private T nextElement;
-  private boolean nextElementComputed;
-  private final Reiterator<T> iterator;
-
-  public PeekingReiterator(Reiterator<T> iterator) {
-    this.iterator = checkNotNull(iterator);
-  }
-
-  PeekingReiterator(PeekingReiterator<T> it) {
-    this.iterator = checkNotNull(checkNotNull(it).iterator.copy());
-    this.nextElement = it.nextElement;
-    this.nextElementComputed = it.nextElementComputed;
-  }
-
-  @Override
-  public boolean hasNext() {
-    computeNext();
-    return nextElementComputed;
-  }
-
-  @Override
-  public T next() {
-    T result = peek();
-    nextElementComputed = false;
-    return result;
-  }
-
-  /**
-   * {@inheritDoc}
-   *
-   * <p>If {@link #peek} is called, {@code remove} is disallowed until
-   * {@link #next} has been subsequently called.
-   */
-  @Override
-  public void remove() {
-    checkState(!nextElementComputed,
-        "After peek(), remove() is disallowed until next() is called");
-    iterator.remove();
-  }
-
-  @Override
-  public PeekingReiterator<T> copy() {
-    return new PeekingReiterator<>(this);
-  }
-
-  /**
-   * Returns the element that would be returned by {@link #next}, without
-   * actually consuming the element.
-   * @throws NoSuchElementException if there is no next element
-   */
-  public T peek() {
-    computeNext();
-    if (!nextElementComputed) {
-      throw new NoSuchElementException();
-    }
-    return nextElement;
-  }
-
-  private void computeNext() {
-    if (nextElementComputed) {
-      return;
-    }
-    if (!iterator.hasNext()) {
-      return;
-    }
-    nextElement = iterator.next();
-    nextElementComputed = true;
-  }
-}


[36/50] [abbrv] incubator-beam git commit: Make WindowedValue like an interface, allow external implementations

Posted by lc...@apache.org.
Make WindowedValue like an interface, allow external implementations


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

Branch: refs/heads/runners-spark2
Commit: 9f79e2bd0384f2894b0fc6e45c2ab44136541fa3
Parents: a313e33
Author: Kenneth Knowles <kl...@google.com>
Authored: Wed Jun 22 07:40:38 2016 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Jul 6 10:18:52 2016 -0700

----------------------------------------------------------------------
 .../org/apache/beam/sdk/util/WindowedValue.java | 181 ++++++++++++-------
 1 file changed, 112 insertions(+), 69 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9f79e2bd/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowedValue.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowedValue.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowedValue.java
index d78da41..d21b6c8 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowedValue.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowedValue.java
@@ -59,9 +59,6 @@ import java.util.Set;
  */
 public abstract class WindowedValue<T> {
 
-  protected final T value;
-  protected final PaneInfo pane;
-
   /**
    * Returns a {@code WindowedValue} with the given value, timestamp,
    * and windows.
@@ -146,11 +143,6 @@ public abstract class WindowedValue<T> {
     return new ValueInEmptyWindows<T>(value, pane);
   }
 
-  private WindowedValue(T value, PaneInfo pane) {
-    this.value = value;
-    this.pane = checkNotNull(pane);
-  }
-
   /**
    * Returns a new {@code WindowedValue} that is a copy of this one, but with a different value,
    * which may have a new type {@code NewT}.
@@ -160,9 +152,7 @@ public abstract class WindowedValue<T> {
   /**
    * Returns the value of this {@code WindowedValue}.
    */
-  public T getValue() {
-    return value;
-  }
+  public abstract T getValue();
 
   /**
    * Returns the timestamp of this {@code WindowedValue}.
@@ -175,6 +165,11 @@ public abstract class WindowedValue<T> {
   public abstract Collection<? extends BoundedWindow> getWindows();
 
   /**
+   * Returns the pane of this {@code WindowedValue} in its window.
+   */
+  public abstract PaneInfo getPane();
+
+  /**
    * Returns a collection of {@link WindowedValue WindowedValues} identical to this one, except each
    * is in exactly one of the windows that this {@link WindowedValue} is in.
    */
@@ -186,18 +181,28 @@ public abstract class WindowedValue<T> {
     return windowedValues.build();
   }
 
-  /**
-   * Returns the pane of this {@code WindowedValue} in its window.
-   */
-  public PaneInfo getPane() {
-    return pane;
-  }
-
   @Override
-  public abstract boolean equals(Object o);
+  public boolean equals(Object other) {
+    if (!(other instanceof WindowedValue)) {
+      return false;
+    } else {
+      WindowedValue<?> that = (WindowedValue<?>) other;
+
+      // Compare timestamps first as they are most likely to differ.
+      // Also compare timestamps according to millis-since-epoch because otherwise expensive
+      // comparisons are made on their Chronology objects.
+      return this.getTimestamp().isEqual(that.getTimestamp())
+          && Objects.equals(this.getValue(), that.getValue())
+          && Objects.equals(this.getWindows(), that.getWindows())
+          && Objects.equals(this.getPane(), that.getPane());
+    }
+  }
 
   @Override
-  public abstract int hashCode();
+  public int hashCode() {
+    // Hash only the millis of the timestamp to be consistent with equals
+    return Objects.hash(getValue(), getTimestamp().getMillis(), getWindows(), getPane());
+  }
 
   @Override
   public abstract String toString();
@@ -206,11 +211,34 @@ public abstract class WindowedValue<T> {
       Collections.singletonList(GlobalWindow.INSTANCE);
 
   /**
+   * An abstract superclass for implementations of {@link WindowedValue} that stores the value
+   * and pane info.
+   */
+  private abstract static class SimpleWindowedValue<T> extends WindowedValue<T> {
+    private final T value;
+    private final PaneInfo pane;
+
+    protected SimpleWindowedValue(T value, PaneInfo pane) {
+      this.value = value;
+      this.pane = checkNotNull(pane);
+    }
+
+    @Override
+    public PaneInfo getPane() {
+      return pane;
+    }
+    @Override
+    public T getValue() {
+      return value;
+    }
+  }
+
+  /**
    * The abstract superclass of WindowedValue representations where
    * timestamp == MIN.
    */
   private abstract static class MinTimestampWindowedValue<T>
-      extends WindowedValue<T> {
+      extends SimpleWindowedValue<T> {
     public MinTimestampWindowedValue(T value, PaneInfo pane) {
       super(value, pane);
     }
@@ -232,8 +260,8 @@ public abstract class WindowedValue<T> {
     }
 
     @Override
-    public <NewT> WindowedValue<NewT> withValue(NewT value) {
-      return new ValueInGlobalWindow<>(value, pane);
+    public <NewT> WindowedValue<NewT> withValue(NewT newValue) {
+      return new ValueInGlobalWindow<>(newValue, getPane());
     }
 
     @Override
@@ -245,23 +273,23 @@ public abstract class WindowedValue<T> {
     public boolean equals(Object o) {
       if (o instanceof ValueInGlobalWindow) {
         ValueInGlobalWindow<?> that = (ValueInGlobalWindow<?>) o;
-        return Objects.equals(that.pane, this.pane)
-            && Objects.equals(that.value, this.value);
+        return Objects.equals(that.getPane(), this.getPane())
+            && Objects.equals(that.getValue(), this.getValue());
       } else {
-        return false;
+        return super.equals(o);
       }
     }
 
     @Override
     public int hashCode() {
-      return Objects.hash(value, pane);
+      return Objects.hash(getValue(), getPane());
     }
 
     @Override
     public String toString() {
       return MoreObjects.toStringHelper(getClass())
-          .add("value", value)
-          .add("pane", pane)
+          .add("value", getValue())
+          .add("pane", getPane())
           .toString();
     }
   }
@@ -277,8 +305,8 @@ public abstract class WindowedValue<T> {
     }
 
     @Override
-    public <NewT> WindowedValue<NewT> withValue(NewT value) {
-      return new ValueInEmptyWindows<>(value, pane);
+    public <NewT> WindowedValue<NewT> withValue(NewT newValue) {
+      return new ValueInEmptyWindows<>(newValue, getPane());
     }
 
     @Override
@@ -290,23 +318,23 @@ public abstract class WindowedValue<T> {
     public boolean equals(Object o) {
       if (o instanceof ValueInEmptyWindows) {
         ValueInEmptyWindows<?> that = (ValueInEmptyWindows<?>) o;
-        return Objects.equals(that.pane, this.pane)
-            && Objects.equals(that.value, this.value);
+        return Objects.equals(that.getPane(), this.getPane())
+            && Objects.equals(that.getValue(), this.getValue());
       } else {
-        return false;
+        return super.equals(o);
       }
     }
 
     @Override
     public int hashCode() {
-      return Objects.hash(value, pane);
+      return Objects.hash(getValue(), getPane());
     }
 
     @Override
     public String toString() {
       return MoreObjects.toStringHelper(getClass())
-          .add("value", value)
-          .add("pane", pane)
+          .add("value", getValue())
+          .add("pane", getPane())
           .toString();
     }
   }
@@ -316,8 +344,8 @@ public abstract class WindowedValue<T> {
    * timestamp is arbitrary.
    */
   private abstract static class TimestampedWindowedValue<T>
-      extends WindowedValue<T> {
-    protected final Instant timestamp;
+      extends SimpleWindowedValue<T> {
+    private final Instant timestamp;
 
     public TimestampedWindowedValue(T value,
                                     Instant timestamp,
@@ -345,8 +373,8 @@ public abstract class WindowedValue<T> {
     }
 
     @Override
-    public <NewT> WindowedValue<NewT> withValue(NewT value) {
-      return new TimestampedValueInGlobalWindow<>(value, timestamp, pane);
+    public <NewT> WindowedValue<NewT> withValue(NewT newValue) {
+      return new TimestampedValueInGlobalWindow<>(newValue, getTimestamp(), getPane());
     }
 
     @Override
@@ -359,25 +387,29 @@ public abstract class WindowedValue<T> {
       if (o instanceof TimestampedValueInGlobalWindow) {
         TimestampedValueInGlobalWindow<?> that =
             (TimestampedValueInGlobalWindow<?>) o;
-        return this.timestamp.isEqual(that.timestamp) // don't compare chronology objects
-            && Objects.equals(that.pane, this.pane)
-            && Objects.equals(that.value, this.value);
+        // Compare timestamps first as they are most likely to differ.
+        // Also compare timestamps according to millis-since-epoch because otherwise expensive
+        // comparisons are made on their Chronology objects.
+        return this.getTimestamp().isEqual(that.getTimestamp())
+            && Objects.equals(that.getPane(), this.getPane())
+            && Objects.equals(that.getValue(), this.getValue());
       } else {
-        return false;
+        return super.equals(o);
       }
     }
 
     @Override
     public int hashCode() {
-      return Objects.hash(value, pane, timestamp.getMillis());
+      // Hash only the millis of the timestamp to be consistent with equals
+      return Objects.hash(getValue(), getPane(), getTimestamp().getMillis());
     }
 
     @Override
     public String toString() {
       return MoreObjects.toStringHelper(getClass())
-          .add("value", value)
-          .add("timestamp", timestamp)
-          .add("pane", pane)
+          .add("value", getValue())
+          .add("timestamp", getTimestamp())
+          .add("pane", getPane())
           .toString();
     }
   }
@@ -399,8 +431,8 @@ public abstract class WindowedValue<T> {
     }
 
     @Override
-    public <NewT> WindowedValue<NewT> withValue(NewT value) {
-      return new TimestampedValueInSingleWindow<>(value, timestamp, window, pane);
+    public <NewT> WindowedValue<NewT> withValue(NewT newValue) {
+      return new TimestampedValueInSingleWindow<>(newValue, getTimestamp(), window, getPane());
     }
 
     @Override
@@ -413,27 +445,31 @@ public abstract class WindowedValue<T> {
       if (o instanceof TimestampedValueInSingleWindow) {
         TimestampedValueInSingleWindow<?> that =
             (TimestampedValueInSingleWindow<?>) o;
-        return Objects.equals(that.value, this.value)
-            && this.timestamp.isEqual(that.timestamp) // don't compare chronology objects
-            && Objects.equals(that.pane, this.pane)
+        // Compare timestamps first as they are most likely to differ.
+        // Also compare timestamps according to millis-since-epoch because otherwise expensive
+        // comparisons are made on their Chronology objects.
+        return this.getTimestamp().isEqual(that.getTimestamp())
+            && Objects.equals(that.getValue(), this.getValue())
+            && Objects.equals(that.getPane(), this.getPane())
             && Objects.equals(that.window, this.window);
       } else {
-        return false;
+        return super.equals(o);
       }
     }
 
     @Override
     public int hashCode() {
-      return Objects.hash(value, timestamp.getMillis(), pane, window);
+      // Hash only the millis of the timestamp to be consistent with equals
+      return Objects.hash(getValue(), getTimestamp().getMillis(), getPane(), window);
     }
 
     @Override
     public String toString() {
       return MoreObjects.toStringHelper(getClass())
-          .add("value", value)
-          .add("timestamp", timestamp)
+          .add("value", getValue())
+          .add("timestamp", getTimestamp())
           .add("window", window)
-          .add("pane", pane)
+          .add("pane", getPane())
           .toString();
     }
   }
@@ -456,8 +492,8 @@ public abstract class WindowedValue<T> {
     }
 
     @Override
-    public <NewT> WindowedValue<NewT> withValue(NewT value) {
-      return new TimestampedValueInMultipleWindows<>(value, timestamp, windows, pane);
+    public <NewT> WindowedValue<NewT> withValue(NewT newValue) {
+      return new TimestampedValueInMultipleWindows<>(newValue, getTimestamp(), windows, getPane());
     }
 
     @Override
@@ -470,30 +506,37 @@ public abstract class WindowedValue<T> {
       if (o instanceof TimestampedValueInMultipleWindows) {
         TimestampedValueInMultipleWindows<?> that =
             (TimestampedValueInMultipleWindows<?>) o;
-        if (this.timestamp.isEqual(that.timestamp) // don't compare chronology objects
-            && Objects.equals(that.value, this.value)
-            && Objects.equals(that.pane, this.pane)) {
+        // Compare timestamps first as they are most likely to differ.
+        // Also compare timestamps according to millis-since-epoch because otherwise expensive
+        // comparisons are made on their Chronology objects.
+        if (this.getTimestamp().isEqual(that.getTimestamp())
+            && Objects.equals(that.getValue(), this.getValue())
+            && Objects.equals(that.getPane(), this.getPane())) {
           ensureWindowsAreASet();
           that.ensureWindowsAreASet();
           return that.windows.equals(this.windows);
+        } else {
+          return false;
         }
+      } else {
+        return super.equals(o);
       }
-      return false;
     }
 
     @Override
     public int hashCode() {
+      // Hash only the millis of the timestamp to be consistent with equals
       ensureWindowsAreASet();
-      return Objects.hash(value, timestamp.getMillis(), pane, windows);
+      return Objects.hash(getValue(), getTimestamp().getMillis(), getPane(), windows);
     }
 
     @Override
     public String toString() {
       return MoreObjects.toStringHelper(getClass())
-          .add("value", value)
-          .add("timestamp", timestamp)
+          .add("value", getValue())
+          .add("timestamp", getTimestamp())
           .add("windows", windows)
-          .add("pane", pane)
+          .add("pane", getPane())
           .toString();
     }
 


[37/50] [abbrv] incubator-beam git commit: [BEAM-392] Rename missing references to FlinkPipelineRunner

Posted by lc...@apache.org.
[BEAM-392] Rename missing references to FlinkPipelineRunner


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

Branch: refs/heads/runners-spark2
Commit: acec0a5984ef7e7dd73f943afe25d35f0f828d6b
Parents: 721763a
Author: Isma�l Mej�a <ie...@gmail.com>
Authored: Wed Jun 29 23:12:06 2016 +0200
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Jul 6 10:18:52 2016 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/beam/examples/MinimalWordCount.java   | 6 +++---
 .../java/org/apache/beam/examples/MinimalWordCountJava8.java   | 6 +++---
 2 files changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/acec0a59/examples/java/src/main/java/org/apache/beam/examples/MinimalWordCount.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/MinimalWordCount.java b/examples/java/src/main/java/org/apache/beam/examples/MinimalWordCount.java
index 2c67609..9f6d61a 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/MinimalWordCount.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/MinimalWordCount.java
@@ -67,17 +67,17 @@ public class MinimalWordCount {
     // In order to run your pipeline, you need to make following runner specific changes:
     //
     // CHANGE 1/3: Select a Beam runner, such as BlockingDataflowRunner
-    // or FlinkPipelineRunner.
+    // or FlinkRunner.
     // CHANGE 2/3: Specify runner-required options.
     // For BlockingDataflowRunner, set project and temp location as follows:
     //   DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class);
     //   dataflowOptions.setRunner(BlockingDataflowRunner.class);
     //   dataflowOptions.setProject("SET_YOUR_PROJECT_ID_HERE");
     //   dataflowOptions.setTempLocation("gs://SET_YOUR_BUCKET_NAME_HERE/AND_TEMP_DIRECTORY");
-    // For FlinkPipelineRunner, set the runner as follows. See {@code FlinkPipelineOptions}
+    // For FlinkRunner, set the runner as follows. See {@code FlinkPipelineOptions}
     // for more details.
     //   options.as(FlinkPipelineOptions.class)
-    //      .setRunner(FlinkPipelineRunner.class);
+    //      .setRunner(FlinkRunner.class);
 
     // Create the Pipeline object with the options we defined above.
     Pipeline p = Pipeline.create(options);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/acec0a59/examples/java8/src/main/java/org/apache/beam/examples/MinimalWordCountJava8.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/main/java/org/apache/beam/examples/MinimalWordCountJava8.java b/examples/java8/src/main/java/org/apache/beam/examples/MinimalWordCountJava8.java
index 6362b96..ff8ca55 100644
--- a/examples/java8/src/main/java/org/apache/beam/examples/MinimalWordCountJava8.java
+++ b/examples/java8/src/main/java/org/apache/beam/examples/MinimalWordCountJava8.java
@@ -42,17 +42,17 @@ public class MinimalWordCountJava8 {
     // In order to run your pipeline, you need to make following runner specific changes:
     //
     // CHANGE 1/3: Select a Beam runner, such as BlockingDataflowRunner
-    // or FlinkPipelineRunner.
+    // or FlinkRunner.
     // CHANGE 2/3: Specify runner-required options.
     // For BlockingDataflowRunner, set project and temp location as follows:
     //   DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class);
     //   dataflowOptions.setRunner(BlockingDataflowRunner.class);
     //   dataflowOptions.setProject("SET_YOUR_PROJECT_ID_HERE");
     //   dataflowOptions.setTempLocation("gs://SET_YOUR_BUCKET_NAME_HERE/AND_TEMP_DIRECTORY");
-    // For FlinkPipelineRunner, set the runner as follows. See {@code FlinkPipelineOptions}
+    // For FlinkRunner, set the runner as follows. See {@code FlinkPipelineOptions}
     // for more details.
     //   options.as(FlinkPipelineOptions.class)
-    //      .setRunner(FlinkPipelineRunner.class);
+    //      .setRunner(FlinkRunner.class);
 
     Pipeline p = Pipeline.create(options);
 


[03/50] [abbrv] incubator-beam git commit: Enable dependency-plugin at global level

Posted by lc...@apache.org.
Enable dependency-plugin at global level


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

Branch: refs/heads/runners-spark2
Commit: 38dfb634f20d203aa80a2a62e621afcb9dd999a1
Parents: d1d7a2c
Author: Pei He <pe...@google.com>
Authored: Tue Jun 21 15:25:06 2016 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Jul 6 10:18:49 2016 -0700

----------------------------------------------------------------------
 examples/java/pom.xml                      | 13 ----
 examples/java8/pom.xml                     | 14 ----
 pom.xml                                    | 25 ++++---
 runners/core-java/pom.xml                  | 13 ----
 runners/direct-java/pom.xml                | 13 ----
 runners/flink/examples/pom.xml             | 14 ++++
 runners/flink/runner/pom.xml               | 14 ++++
 runners/google-cloud-dataflow-java/pom.xml | 13 ----
 runners/spark/pom.xml                      | 89 ++++++++++++++++++++++++-
 sdks/java/core/pom.xml                     | 15 -----
 sdks/java/io/google-cloud-platform/pom.xml | 22 +++++-
 sdks/java/io/hdfs/pom.xml                  | 29 ++++++++
 sdks/java/io/kafka/pom.xml                 | 25 +++++++
 sdks/java/java8tests/pom.xml               | 13 ----
 14 files changed, 204 insertions(+), 108 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/38dfb634/examples/java/pom.xml
----------------------------------------------------------------------
diff --git a/examples/java/pom.xml b/examples/java/pom.xml
index ba1e6e5..3b67797 100644
--- a/examples/java/pom.xml
+++ b/examples/java/pom.xml
@@ -54,19 +54,6 @@
 
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-dependency-plugin</artifactId>
-        <executions>
-          <execution>
-            <goals><goal>analyze-only</goal></goals>
-            <configuration>
-              <failOnWarning>true</failOnWarning>
-            </configuration>
-          </execution>
-        </executions>
-      </plugin>
-
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-checkstyle-plugin</artifactId>
       </plugin>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/38dfb634/examples/java8/pom.xml
----------------------------------------------------------------------
diff --git a/examples/java8/pom.xml b/examples/java8/pom.xml
index 82b1c46..ebc6f3d 100644
--- a/examples/java8/pom.xml
+++ b/examples/java8/pom.xml
@@ -66,20 +66,6 @@
         </configuration>
       </plugin>
 
-
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-dependency-plugin</artifactId>
-        <executions>
-          <execution>
-            <goals><goal>analyze-only</goal></goals>
-            <configuration>
-              <failOnWarning>true</failOnWarning>
-            </configuration>
-          </execution>
-        </executions>
-      </plugin>
-
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-checkstyle-plugin</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/38dfb634/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index d933e8b..582ff62 100644
--- a/pom.xml
+++ b/pom.xml
@@ -863,16 +863,6 @@
           </configuration>
         </plugin>
 
-        <!-- Ignore runtime-only dependencies in analysis -->
-        <plugin>
-          <groupId>org.apache.maven.plugins</groupId>
-          <artifactId>maven-dependency-plugin</artifactId>
-          <version>2.10</version>
-          <configuration>
-            <ignoreNonCompile>true</ignoreNonCompile>
-          </configuration>
-        </plugin>
-
         <plugin>
           <groupId>org.codehaus.mojo</groupId>
           <artifactId>build-helper-maven-plugin</artifactId>
@@ -923,6 +913,21 @@
     <plugins>
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-dependency-plugin</artifactId>
+        <version>2.10</version>
+        <executions>
+          <execution>
+            <goals><goal>analyze-only</goal></goals>
+            <configuration>
+              <!-- Ignore runtime-only dependencies in analysis -->
+              <ignoreNonCompile>true</ignoreNonCompile>
+              <failOnWarning>true</failOnWarning>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-enforcer-plugin</artifactId>
         <version>1.3.1</version>
         <executions>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/38dfb634/runners/core-java/pom.xml
----------------------------------------------------------------------
diff --git a/runners/core-java/pom.xml b/runners/core-java/pom.xml
index 98d80bb..3a91f7e 100644
--- a/runners/core-java/pom.xml
+++ b/runners/core-java/pom.xml
@@ -54,19 +54,6 @@
 
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-dependency-plugin</artifactId>
-        <executions>
-          <execution>
-            <goals><goal>analyze-only</goal></goals>
-            <configuration>
-              <failOnWarning>true</failOnWarning>
-            </configuration>
-          </execution>
-        </executions>
-      </plugin>
-
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-checkstyle-plugin</artifactId>
       </plugin>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/38dfb634/runners/direct-java/pom.xml
----------------------------------------------------------------------
diff --git a/runners/direct-java/pom.xml b/runners/direct-java/pom.xml
index 3010757..13e8951 100644
--- a/runners/direct-java/pom.xml
+++ b/runners/direct-java/pom.xml
@@ -48,19 +48,6 @@
 
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-dependency-plugin</artifactId>
-        <executions>
-          <execution>
-            <goals><goal>analyze-only</goal></goals>
-            <configuration>
-              <failOnWarning>true</failOnWarning>
-            </configuration>
-          </execution>
-        </executions>
-      </plugin>
-
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-checkstyle-plugin</artifactId>
       </plugin>
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/38dfb634/runners/flink/examples/pom.xml
----------------------------------------------------------------------
diff --git a/runners/flink/examples/pom.xml b/runners/flink/examples/pom.xml
index 1fba81c..b0ee2ed 100644
--- a/runners/flink/examples/pom.xml
+++ b/runners/flink/examples/pom.xml
@@ -96,6 +96,20 @@
         <artifactId>maven-source-plugin</artifactId>
       </plugin>
 
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-dependency-plugin</artifactId>
+        <executions>
+          <execution>
+            <goals><goal>analyze-only</goal></goals>
+            <configuration>
+              <!-- disable for now until dependencies are cleaned up -->
+              <failOnWarning>false</failOnWarning>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+
       <!-- Checkstyle errors for now
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/38dfb634/runners/flink/runner/pom.xml
----------------------------------------------------------------------
diff --git a/runners/flink/runner/pom.xml b/runners/flink/runner/pom.xml
index dd32063..1c522fc 100644
--- a/runners/flink/runner/pom.xml
+++ b/runners/flink/runner/pom.xml
@@ -152,6 +152,20 @@
         <artifactId>maven-source-plugin</artifactId>
       </plugin>
 
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-dependency-plugin</artifactId>
+        <executions>
+          <execution>
+            <goals><goal>analyze-only</goal></goals>
+            <configuration>
+              <!-- disable for now until dependencies are cleaned up -->
+              <failOnWarning>false</failOnWarning>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+
       <!-- Checkstyle errors for now
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/38dfb634/runners/google-cloud-dataflow-java/pom.xml
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/pom.xml b/runners/google-cloud-dataflow-java/pom.xml
index 5408462..38ffa49 100644
--- a/runners/google-cloud-dataflow-java/pom.xml
+++ b/runners/google-cloud-dataflow-java/pom.xml
@@ -93,19 +93,6 @@
         </configuration>
       </plugin>
 
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-dependency-plugin</artifactId>
-        <executions>
-          <execution>
-            <goals><goal>analyze-only</goal></goals>
-            <configuration>
-              <failOnWarning>true</failOnWarning>
-            </configuration>
-          </execution>
-        </executions>
-      </plugin>
-
       <!-- Run CheckStyle pass on transforms, as they are release in
            source form. -->
       <plugin>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/38dfb634/runners/spark/pom.xml
----------------------------------------------------------------------
diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml
index 741f2db..94c42bd 100644
--- a/runners/spark/pom.xml
+++ b/runners/spark/pom.xml
@@ -35,6 +35,8 @@
     <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
     <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
     <spark.version>1.6.1</spark.version>
+    <hadoop.version>2.2.0</hadoop.version>
+    <kafka.version>0.8.2.1</kafka.version>
   </properties>
 
   <profiles>
@@ -113,9 +115,33 @@
       <scope>provided</scope>
     </dependency>
     <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-network-common_2.10</artifactId>
+      <version>${spark.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
       <groupId>org.apache.kafka</groupId>
       <artifactId>kafka_2.10</artifactId>
-      <version>0.8.2.1</version>
+      <version>${kafka.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.kafka</groupId>
+      <artifactId>kafka-clients</artifactId>
+      <version>${kafka.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <version>${hadoop.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-mapreduce-client-core</artifactId>
+      <version>${hadoop.version}</version>
       <scope>provided</scope>
     </dependency>
     <dependency>
@@ -128,10 +154,67 @@
       <optional>true</optional>
     </dependency>
     <dependency>
+      <groupId>com.google.http-client</groupId>
+      <artifactId>google-http-client</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-core</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-annotations</artifactId>
+    </dependency>
+      <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-databind</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.avro</groupId>
+      <artifactId>avro</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-api</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>joda-time</groupId>
+      <artifactId>joda-time</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>commons-io</groupId>
+      <artifactId>commons-io</artifactId>
+      <version>2.4</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.zookeeper</groupId>
+      <artifactId>zookeeper</artifactId>
+      <version>3.4.6</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.scala-lang</groupId>
+      <artifactId>scala-library</artifactId>
+      <version>2.10.5</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
       <groupId>org.apache.beam</groupId>
       <artifactId>beam-sdks-java-core</artifactId>
       <exclusions>
-        <!-- Use Hadoop/Spark's backend logger -->
+        <!-- Use Hadoop/Spark's backend logger instead of jdk14 for tests -->
+        <exclusion>
+          <groupId>org.slf4j</groupId>
+          <artifactId>slf4j-jdk14</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.beam</groupId>
+      <artifactId>beam-runners-core-java</artifactId>
+      <exclusions>
+        <!-- Use Hadoop/Spark's backend logger instead of jdk14 for tests -->
         <exclusion>
           <groupId>org.slf4j</groupId>
           <artifactId>slf4j-jdk14</artifactId>
@@ -142,7 +225,7 @@
       <groupId>org.apache.beam</groupId>
       <artifactId>beam-examples-java</artifactId>
       <exclusions>
-        <!-- Use Hadoop/Spark's backend logger -->
+        <!-- Use Hadoop/Spark's backend logger instead of jdk14 for tests -->
         <exclusion>
           <groupId>org.slf4j</groupId>
           <artifactId>slf4j-jdk14</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/38dfb634/sdks/java/core/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/core/pom.xml b/sdks/java/core/pom.xml
index bbba77b..3b650c6 100644
--- a/sdks/java/core/pom.xml
+++ b/sdks/java/core/pom.xml
@@ -137,21 +137,6 @@
 
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-dependency-plugin</artifactId>
-        <executions>
-          <execution>
-            <goals>
-              <goal>analyze-only</goal>
-            </goals>
-            <configuration>
-              <failOnWarning>true</failOnWarning>
-            </configuration>
-          </execution>
-        </executions>
-      </plugin>
-
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-checkstyle-plugin</artifactId>
         <configuration>
           <!-- Set testSourceDirectory in order to exclude generated-test-sources -->

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/38dfb634/sdks/java/io/google-cloud-platform/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/pom.xml b/sdks/java/io/google-cloud-platform/pom.xml
index f567261..5786e84 100644
--- a/sdks/java/io/google-cloud-platform/pom.xml
+++ b/sdks/java/io/google-cloud-platform/pom.xml
@@ -79,7 +79,27 @@
       <version>${bigtable.version}</version>
     </dependency>
 
-    <!-- test -->
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-api</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.protobuf</groupId>
+      <artifactId>protobuf-java</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.code.findbugs</groupId>
+      <artifactId>jsr305</artifactId>
+    </dependency>
+
+    <!--  test -->
     <dependency>
       <groupId>org.apache.beam</groupId>
       <artifactId>beam-sdks-java-core</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/38dfb634/sdks/java/io/hdfs/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/io/hdfs/pom.xml b/sdks/java/io/hdfs/pom.xml
index 9c30792..9f4b38a 100644
--- a/sdks/java/io/hdfs/pom.xml
+++ b/sdks/java/io/hdfs/pom.xml
@@ -58,12 +58,41 @@
     </dependency>
 
     <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-annotations</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.code.findbugs</groupId>
+      <artifactId>jsr305</artifactId>
+    </dependency>
+
+    <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-client</artifactId>
       <version>2.7.0</version>
       <scope>provided</scope>
     </dependency>
 
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <version>2.7.0</version>
+      <scope>provided</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-mapreduce-client-core</artifactId>
+      <version>2.7.0</version>
+      <scope>provided</scope>
+    </dependency>
+
     <!-- test dependencies -->
     <dependency>
       <groupId>org.apache.beam</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/38dfb634/sdks/java/io/kafka/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/io/kafka/pom.xml b/sdks/java/io/kafka/pom.xml
index 76c0eb6..578026b 100644
--- a/sdks/java/io/kafka/pom.xml
+++ b/sdks/java/io/kafka/pom.xml
@@ -69,6 +69,31 @@
       <version>0.9.0.1</version>
     </dependency>
 
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-api</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>joda-time</groupId>
+      <artifactId>joda-time</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-annotations</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.google.code.findbugs</groupId>
+      <artifactId>jsr305</artifactId>
+    </dependency>
+
     <!-- test dependencies-->
     <dependency>
       <groupId>org.apache.beam</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/38dfb634/sdks/java/java8tests/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/java/java8tests/pom.xml b/sdks/java/java8tests/pom.xml
index 8e20228..775a7c7 100644
--- a/sdks/java/java8tests/pom.xml
+++ b/sdks/java/java8tests/pom.xml
@@ -53,19 +53,6 @@
 
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-dependency-plugin</artifactId>
-        <executions>
-          <execution>
-            <goals><goal>analyze-only</goal></goals>
-            <configuration>
-              <failOnWarning>true</failOnWarning>
-            </configuration>
-          </execution>
-        </executions>
-      </plugin>
-
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-checkstyle-plugin</artifactId>
       </plugin>
 


[20/50] [abbrv] incubator-beam git commit: Static import Preconditions.checkX everywhere

Posted by lc...@apache.org.
Static import Preconditions.checkX everywhere


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

Branch: refs/heads/runners-spark2
Commit: 8b8615ea2b8f047fbd82f887fb11ad30989a279e
Parents: 067c92a
Author: Dan Halperin <dh...@google.com>
Authored: Sun Jun 26 02:05:08 2016 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Jul 6 10:18:50 2016 -0700

----------------------------------------------------------------------
 .../beam/examples/complete/AutoComplete.java    |  4 +-
 .../complete/game/injector/InjectorUtils.java   |  6 +--
 .../injector/RetryHttpInitializerWrapper.java   |  5 +-
 .../beam/sdk/util/BatchTimerInternals.java      |  7 +--
 .../apache/beam/sdk/util/DoFnRunnerBase.java    | 15 +++---
 .../apache/beam/sdk/util/PaneInfoTracker.java   | 11 +++--
 .../beam/sdk/util/ReduceFnContextFactory.java   |  6 +--
 .../apache/beam/sdk/util/ReduceFnRunner.java    | 26 +++++-----
 .../org/apache/beam/sdk/util/TriggerRunner.java |  5 +-
 .../org/apache/beam/sdk/util/WatermarkHold.java | 19 ++++----
 .../beam/sdk/util/ReduceFnRunnerTest.java       | 18 +++----
 .../apache/beam/sdk/util/ReduceFnTester.java    | 19 ++++----
 .../beam/runners/direct/WatermarkManager.java   |  5 +-
 .../FlinkPipelineExecutionEnvironment.java      |  6 +--
 .../FlinkStreamingTranslationContext.java       |  8 +--
 .../functions/FlinkProcessContext.java          | 25 +++++-----
 .../translation/types/CoderTypeInformation.java |  6 +--
 .../utils/SerializedPipelineOptions.java        |  8 +--
 .../streaming/FlinkAbstractParDoWrapper.java    | 10 ++--
 .../FlinkGroupAlsoByWindowWrapper.java          | 23 ++++-----
 .../streaming/FlinkParDoBoundMultiWrapper.java  |  8 +--
 .../streaming/io/UnboundedFlinkSink.java        |  2 +-
 .../streaming/io/UnboundedFlinkSource.java      |  7 ++-
 .../streaming/state/FlinkStateInternals.java    |  7 +--
 .../dataflow/DataflowPipelineTranslator.java    | 15 +++---
 .../beam/runners/dataflow/DataflowRunner.java   | 19 ++++----
 .../runners/dataflow/internal/IsmFormat.java    |  8 +--
 .../options/DataflowWorkerLoggingOptions.java   | 14 +++---
 .../dataflow/util/DataflowPathValidator.java    | 14 +++---
 .../beam/runners/dataflow/util/GcsStager.java   |  5 +-
 .../beam/runners/spark/io/CreateStream.java     |  7 +--
 .../apache/beam/runners/spark/io/KafkaIO.java   | 23 ++++-----
 .../beam/runners/spark/io/hadoop/HadoopIO.java  | 38 +++++++--------
 .../main/java/org/apache/beam/sdk/Pipeline.java |  7 +--
 .../java/org/apache/beam/sdk/coders/Coder.java  |  6 +--
 .../apache/beam/sdk/coders/CoderRegistry.java   |  5 +-
 .../apache/beam/sdk/coders/CollectionCoder.java |  7 ++-
 .../apache/beam/sdk/coders/IterableCoder.java   |  7 ++-
 .../beam/sdk/coders/IterableLikeCoder.java      | 10 ++--
 .../org/apache/beam/sdk/coders/KvCoder.java     |  7 ++-
 .../org/apache/beam/sdk/coders/ListCoder.java   |  7 ++-
 .../org/apache/beam/sdk/coders/MapCoder.java    |  6 +--
 .../apache/beam/sdk/coders/NullableCoder.java   |  6 +--
 .../org/apache/beam/sdk/coders/SetCoder.java    |  7 ++-
 .../java/org/apache/beam/sdk/io/AvroIO.java     |  6 +--
 .../java/org/apache/beam/sdk/io/AvroSource.java | 12 ++---
 .../apache/beam/sdk/io/CompressedSource.java    | 10 ++--
 .../org/apache/beam/sdk/io/DatastoreIO.java     |  7 ++-
 .../org/apache/beam/sdk/io/FileBasedSink.java   | 13 +++--
 .../apache/beam/sdk/io/OffsetBasedSource.java   | 12 ++---
 .../java/org/apache/beam/sdk/io/TextIO.java     |  6 +--
 .../java/org/apache/beam/sdk/io/XmlSink.java    | 10 ++--
 .../java/org/apache/beam/sdk/io/XmlSource.java  | 11 +++--
 .../sdk/options/PipelineOptionsFactory.java     | 28 +++++------
 .../sdk/options/PipelineOptionsValidator.java   | 18 ++++---
 .../sdk/options/ProxyInvocationHandler.java     |  9 ++--
 .../apache/beam/sdk/runners/PipelineRunner.java |  6 +--
 .../beam/sdk/runners/TransformHierarchy.java    | 10 ++--
 .../beam/sdk/runners/TransformTreeNode.java     | 13 ++---
 .../sdk/transforms/ApproximateQuantiles.java    |  9 ++--
 .../org/apache/beam/sdk/transforms/Combine.java |  8 +--
 .../org/apache/beam/sdk/transforms/Create.java  |  5 +-
 .../transforms/IntraBundleParallelization.java  |  9 ++--
 .../org/apache/beam/sdk/transforms/Sample.java  |  6 +--
 .../org/apache/beam/sdk/transforms/Top.java     |  7 ++-
 .../sdk/transforms/display/DisplayData.java     |  4 +-
 .../beam/sdk/transforms/join/CoGbkResult.java   |  6 +--
 .../beam/sdk/transforms/windowing/AfterAll.java |  6 +--
 .../sdk/transforms/windowing/AfterFirst.java    |  9 ++--
 .../beam/sdk/transforms/windowing/PaneInfo.java | 11 +++--
 ...AttemptAndTimeBoundedExponentialBackOff.java | 11 +++--
 .../util/AttemptBoundedExponentialBackOff.java  |  9 ++--
 .../beam/sdk/util/BigQueryTableInserter.java    |  5 +-
 .../org/apache/beam/sdk/util/Credentials.java   |  5 +-
 .../apache/beam/sdk/util/ExecutableTrigger.java | 11 +++--
 .../java/org/apache/beam/sdk/util/GcsUtil.java  | 10 ++--
 .../apache/beam/sdk/util/InstanceBuilder.java   | 21 ++++----
 .../util/IntervalBoundedExponentialBackOff.java | 10 ++--
 .../beam/sdk/util/MergingActiveWindowSet.java   | 35 +++++++-------
 .../apache/beam/sdk/util/SerializableUtils.java | 13 +++--
 .../org/apache/beam/sdk/util/StringUtils.java   | 12 +++--
 .../apache/beam/sdk/util/TimerInternals.java    |  5 +-
 .../apache/beam/sdk/util/ValueWithRecordId.java |  6 +--
 .../org/apache/beam/sdk/util/WindowedValue.java |  5 +-
 .../beam/sdk/util/common/ReflectHelpers.java    |  9 ++--
 .../org/apache/beam/sdk/util/gcsfs/GcsPath.java | 51 ++++++++++----------
 .../beam/sdk/util/state/StateMerging.java       |  6 +--
 .../apache/beam/sdk/transforms/CombineTest.java | 17 ++++---
 .../apache/beam/sdk/transforms/ParDoTest.java   |  7 ++-
 .../apache/beam/sdk/transforms/SampleTest.java  |  6 +--
 .../apache/beam/sdk/transforms/ViewTest.java    | 10 ++--
 .../org/apache/beam/sdk/util/TriggerTester.java |  3 +-
 .../beam/sdk/extensions/joinlibrary/Join.java   | 19 ++++----
 .../apache/beam/sdk/io/hdfs/HDFSFileSource.java | 23 ++++-----
 94 files changed, 525 insertions(+), 509 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java
----------------------------------------------------------------------
diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java b/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java
index 2732aa5..f278ce3 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java
@@ -17,6 +17,7 @@
  */
 package org.apache.beam.examples.complete;
 
+import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.datastore.v1beta3.client.DatastoreHelper.makeValue;
 
 import org.apache.beam.examples.common.DataflowExampleUtils;
@@ -58,7 +59,6 @@ import com.google.api.services.bigquery.model.TableReference;
 import com.google.api.services.bigquery.model.TableRow;
 import com.google.api.services.bigquery.model.TableSchema;
 import com.google.common.base.MoreObjects;
-import com.google.common.base.Preconditions;
 import com.google.datastore.v1beta3.Entity;
 import com.google.datastore.v1beta3.Key;
 import com.google.datastore.v1beta3.Value;
@@ -460,7 +460,7 @@ public class AutoComplete {
     PTransform<? super PBegin, PCollection<String>> readSource;
     WindowFn<Object, ?> windowFn;
     if (options.isStreaming()) {
-      Preconditions.checkArgument(
+      checkArgument(
           !options.getOutputToDatastore(), "DatastoreIO is not supported in streaming.");
       dataflowUtils.setupPubsub();
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/examples/java8/src/main/java/org/apache/beam/examples/complete/game/injector/InjectorUtils.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/injector/InjectorUtils.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/injector/InjectorUtils.java
index dbb5b8f..53e644d 100644
--- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/injector/InjectorUtils.java
+++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/injector/InjectorUtils.java
@@ -17,6 +17,7 @@
  */
 package org.apache.beam.examples.complete.game.injector;
 
+import static com.google.common.base.Preconditions.checkNotNull;
 
 import com.google.api.client.googleapis.auth.oauth2.GoogleCredential;
 import com.google.api.client.googleapis.json.GoogleJsonResponseException;
@@ -28,7 +29,6 @@ import com.google.api.client.json.JsonFactory;
 import com.google.api.services.pubsub.Pubsub;
 import com.google.api.services.pubsub.PubsubScopes;
 import com.google.api.services.pubsub.model.Topic;
-import com.google.common.base.Preconditions;
 
 import java.io.IOException;
 
@@ -42,8 +42,8 @@ class InjectorUtils {
   public static Pubsub getClient(final HttpTransport httpTransport,
                                  final JsonFactory jsonFactory)
            throws IOException {
-      Preconditions.checkNotNull(httpTransport);
-      Preconditions.checkNotNull(jsonFactory);
+      checkNotNull(httpTransport);
+      checkNotNull(jsonFactory);
       GoogleCredential credential =
           GoogleCredential.getApplicationDefault(httpTransport, jsonFactory);
       if (credential.createScopedRequired()) {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/examples/java8/src/main/java/org/apache/beam/examples/complete/game/injector/RetryHttpInitializerWrapper.java
----------------------------------------------------------------------
diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/injector/RetryHttpInitializerWrapper.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/injector/RetryHttpInitializerWrapper.java
index 3183a05..45be287 100644
--- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/injector/RetryHttpInitializerWrapper.java
+++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/injector/RetryHttpInitializerWrapper.java
@@ -17,6 +17,8 @@
  */
 package org.apache.beam.examples.complete.game.injector;
 
+import static com.google.common.base.Preconditions.checkNotNull;
+
 import com.google.api.client.auth.oauth2.Credential;
 import com.google.api.client.http.HttpBackOffIOExceptionHandler;
 import com.google.api.client.http.HttpBackOffUnsuccessfulResponseHandler;
@@ -26,7 +28,6 @@ import com.google.api.client.http.HttpResponse;
 import com.google.api.client.http.HttpUnsuccessfulResponseHandler;
 import com.google.api.client.util.ExponentialBackOff;
 import com.google.api.client.util.Sleeper;
-import com.google.common.base.Preconditions;
 
 import java.io.IOException;
 import java.util.logging.Logger;
@@ -81,7 +82,7 @@ public class RetryHttpInitializerWrapper implements HttpRequestInitializer {
      */
     RetryHttpInitializerWrapper(
             final Credential wrappedCredential, final Sleeper sleeper) {
-        this.wrappedCredential = Preconditions.checkNotNull(wrappedCredential);
+        this.wrappedCredential = checkNotNull(wrappedCredential);
         this.sleeper = sleeper;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/runners/core-java/src/main/java/org/apache/beam/sdk/util/BatchTimerInternals.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/BatchTimerInternals.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/BatchTimerInternals.java
index d0c0b2f..0dd03ba 100644
--- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/BatchTimerInternals.java
+++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/BatchTimerInternals.java
@@ -17,10 +17,11 @@
  */
 package org.apache.beam.sdk.util;
 
+import static com.google.common.base.Preconditions.checkState;
+
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 
 import com.google.common.base.MoreObjects;
-import com.google.common.base.Preconditions;
 
 import org.joda.time.Instant;
 
@@ -105,7 +106,7 @@ public class BatchTimerInternals implements TimerInternals {
 
   public void advanceInputWatermark(ReduceFnRunner<?, ?, ?, ?> runner, Instant newInputWatermark)
       throws Exception {
-    Preconditions.checkState(!newInputWatermark.isBefore(inputWatermarkTime),
+    checkState(!newInputWatermark.isBefore(inputWatermarkTime),
         "Cannot move input watermark time backwards from %s to %s", inputWatermarkTime,
         newInputWatermark);
     inputWatermarkTime = newInputWatermark;
@@ -114,7 +115,7 @@ public class BatchTimerInternals implements TimerInternals {
 
   public void advanceProcessingTime(ReduceFnRunner<?, ?, ?, ?> runner, Instant newProcessingTime)
       throws Exception {
-    Preconditions.checkState(!newProcessingTime.isBefore(processingTime),
+    checkState(!newProcessingTime.isBefore(processingTime),
         "Cannot move processing time backwards from %s to %s", processingTime, newProcessingTime);
     processingTime = newProcessingTime;
     advance(runner, newProcessingTime, TimeDomain.PROCESSING_TIME);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java
index a849eb2..3f8f8e0 100644
--- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java
+++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java
@@ -17,6 +17,8 @@
  */
 package org.apache.beam.sdk.util;
 
+import static com.google.common.base.Preconditions.checkNotNull;
+
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.IterableCoder;
 import org.apache.beam.sdk.options.PipelineOptions;
@@ -36,7 +38,6 @@ import org.apache.beam.sdk.util.state.StateInternals;
 import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.TupleTag;
 
-import com.google.common.base.Preconditions;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
@@ -333,13 +334,13 @@ public abstract class DoFnRunnerBase<InputT, OutputT> implements DoFnRunner<Inpu
 
     @Override
     public <T> void sideOutput(TupleTag<T> tag, T output) {
-      Preconditions.checkNotNull(tag, "TupleTag passed to sideOutput cannot be null");
+      checkNotNull(tag, "TupleTag passed to sideOutput cannot be null");
       sideOutputWindowedValue(tag, output, null, null, PaneInfo.NO_FIRING);
     }
 
     @Override
     public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
-      Preconditions.checkNotNull(tag, "TupleTag passed to sideOutputWithTimestamp cannot be null");
+      checkNotNull(tag, "TupleTag passed to sideOutputWithTimestamp cannot be null");
       sideOutputWindowedValue(tag, output, timestamp, null, PaneInfo.NO_FIRING);
     }
 
@@ -351,7 +352,7 @@ public abstract class DoFnRunnerBase<InputT, OutputT> implements DoFnRunner<Inpu
     @Override
     protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregatorInternal(
         String name, CombineFn<AggInputT, ?, AggOutputT> combiner) {
-      Preconditions.checkNotNull(combiner,
+      checkNotNull(combiner,
           "Combiner passed to createAggregator cannot be null");
       return new CounterAggregator<>(generateInternalAggregatorName(name),
           combiner, addCounterMutator);
@@ -409,7 +410,7 @@ public abstract class DoFnRunnerBase<InputT, OutputT> implements DoFnRunner<Inpu
 
     @Override
     public <T> T sideInput(PCollectionView<T> view) {
-      Preconditions.checkNotNull(view, "View passed to sideInput cannot be null");
+      checkNotNull(view, "View passed to sideInput cannot be null");
       Iterator<? extends BoundedWindow> windowIter = windows().iterator();
       BoundedWindow window;
       if (!windowIter.hasNext()) {
@@ -467,13 +468,13 @@ public abstract class DoFnRunnerBase<InputT, OutputT> implements DoFnRunner<Inpu
 
     @Override
     public <T> void sideOutput(TupleTag<T> tag, T output) {
-      Preconditions.checkNotNull(tag, "Tag passed to sideOutput cannot be null");
+      checkNotNull(tag, "Tag passed to sideOutput cannot be null");
       context.sideOutputWindowedValue(tag, windowedValue.withValue(output));
     }
 
     @Override
     public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
-      Preconditions.checkNotNull(tag, "Tag passed to sideOutputWithTimestamp cannot be null");
+      checkNotNull(tag, "Tag passed to sideOutputWithTimestamp cannot be null");
       checkTimestamp(timestamp);
       context.sideOutputWindowedValue(
           tag, output, timestamp, windowedValue.getWindows(), windowedValue.getPane());

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/runners/core-java/src/main/java/org/apache/beam/sdk/util/PaneInfoTracker.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/PaneInfoTracker.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/PaneInfoTracker.java
index 5e08031..0a47feb 100644
--- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/PaneInfoTracker.java
+++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/PaneInfoTracker.java
@@ -17,6 +17,8 @@
  */
 package org.apache.beam.sdk.util;
 
+import static com.google.common.base.Preconditions.checkState;
+
 import org.apache.beam.sdk.transforms.windowing.AfterWatermark;
 import org.apache.beam.sdk.transforms.windowing.PaneInfo;
 import org.apache.beam.sdk.transforms.windowing.PaneInfo.PaneInfoCoder;
@@ -28,7 +30,6 @@ import org.apache.beam.sdk.util.state.StateTags;
 import org.apache.beam.sdk.util.state.ValueState;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
 
 import org.joda.time.Instant;
 
@@ -132,21 +133,21 @@ public class PaneInfoTracker {
       // Timing transitions should follow EARLY* ON_TIME? LATE*
       switch (previousTiming) {
         case EARLY:
-          Preconditions.checkState(
+          checkState(
               timing == Timing.EARLY || timing == Timing.ON_TIME || timing == Timing.LATE,
               "EARLY cannot transition to %s", timing);
           break;
         case ON_TIME:
-          Preconditions.checkState(
+          checkState(
               timing == Timing.LATE, "ON_TIME cannot transition to %s", timing);
           break;
         case LATE:
-          Preconditions.checkState(timing == Timing.LATE, "LATE cannot transtion to %s", timing);
+          checkState(timing == Timing.LATE, "LATE cannot transtion to %s", timing);
           break;
         case UNKNOWN:
           break;
       }
-      Preconditions.checkState(!previousPane.isLast(), "Last pane was not last after all.");
+      checkState(!previousPane.isLast(), "Last pane was not last after all.");
     }
 
     return PaneInfo.createPane(isFirst, isFinal, timing, index, nonSpeculativeIndex);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnContextFactory.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnContextFactory.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnContextFactory.java
index c90940e..2d86508 100644
--- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnContextFactory.java
+++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnContextFactory.java
@@ -17,6 +17,7 @@
  */
 package org.apache.beam.sdk.util;
 
+import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkNotNull;
 
 import org.apache.beam.sdk.coders.Coder;
@@ -36,7 +37,6 @@ import org.apache.beam.sdk.util.state.StateNamespaces;
 import org.apache.beam.sdk.util.state.StateNamespaces.WindowNamespace;
 import org.apache.beam.sdk.util.state.StateTag;
 
-import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableMap;
 
 import org.joda.time.Instant;
@@ -123,7 +123,7 @@ class ReduceFnContextFactory<K, InputT, OutputT, W extends BoundedWindow> {
     private final StateNamespace namespace;
 
     public TimersImpl(StateNamespace namespace) {
-      Preconditions.checkArgument(namespace instanceof WindowNamespace);
+      checkArgument(namespace instanceof WindowNamespace);
       this.namespace = namespace;
     }
 
@@ -248,7 +248,7 @@ class ReduceFnContextFactory<K, InputT, OutputT, W extends BoundedWindow> {
             namespace = namespaceFor(activeWindows.writeStateAddress(mergingWindow));
             break;
         }
-        Preconditions.checkNotNull(namespace); // cases are exhaustive.
+        checkNotNull(namespace); // cases are exhaustive.
         builder.put(mergingWindow, stateInternals.state(namespace, address, context));
       }
       return builder.build();

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnRunner.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnRunner.java
index 2efc859..c879409 100644
--- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnRunner.java
+++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnRunner.java
@@ -17,6 +17,9 @@
  */
 package org.apache.beam.sdk.util;
 
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkState;
+
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.transforms.Aggregator;
 import org.apache.beam.sdk.transforms.DoFn;
@@ -40,7 +43,6 @@ import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableMap;
 
 import org.joda.time.Duration;
@@ -527,7 +529,7 @@ public class ReduceFnRunner<K, InputT, OutputT, W extends BoundedWindow> {
         // Assert that holds have a proximate timer.
         boolean holdInWindow = !hold.isAfter(window.maxTimestamp());
         boolean timerInWindow = !timer.isAfter(window.maxTimestamp());
-        Preconditions.checkState(
+        checkState(
             holdInWindow == timerInWindow,
             "set a hold at %s, a timer at %s, which disagree as to whether they are in window %s",
             hold,
@@ -559,7 +561,7 @@ public class ReduceFnRunner<K, InputT, OutputT, W extends BoundedWindow> {
    */
   public void onTimer(TimerData timer) throws Exception {
     // Which window is the timer for?
-    Preconditions.checkArgument(timer.getNamespace() instanceof WindowNamespace,
+    checkArgument(timer.getNamespace() instanceof WindowNamespace,
         "Expected timer to be in WindowNamespace, but was in %s", timer.getNamespace());
     @SuppressWarnings("unchecked")
     WindowNamespace<W> windowNamespace = (WindowNamespace<W>) timer.getNamespace();
@@ -607,7 +609,7 @@ public class ReduceFnRunner<K, InputT, OutputT, W extends BoundedWindow> {
         // and the watermark has passed the end of the window.
         @Nullable Instant newHold =
             onTrigger(directContext, renamedContext, true/* isFinished */, isEndOfWindow);
-        Preconditions.checkState(newHold == null,
+        checkState(newHold == null,
             "Hold placed at %s despite isFinished being true.", newHold);
       }
 
@@ -635,7 +637,7 @@ public class ReduceFnRunner<K, InputT, OutputT, W extends BoundedWindow> {
         // cleanup event and handled by the above).
         // Note we must do this even if the trigger is finished so that we are sure to cleanup
         // any final trigger finished bits.
-        Preconditions.checkState(
+        checkState(
             windowingStrategy.getAllowedLateness().isLongerThan(Duration.ZERO),
             "Unexpected zero getAllowedLateness");
         WindowTracing.debug(
@@ -643,7 +645,7 @@ public class ReduceFnRunner<K, InputT, OutputT, W extends BoundedWindow> {
             + "inputWatermark:{}; outputWatermark:{}",
             key, directContext.window(), cleanupTime, timerInternals.currentInputWatermarkTime(),
             timerInternals.currentOutputWatermarkTime());
-        Preconditions.checkState(!cleanupTime.isAfter(BoundedWindow.TIMESTAMP_MAX_VALUE),
+        checkState(!cleanupTime.isAfter(BoundedWindow.TIMESTAMP_MAX_VALUE),
                                  "Cleanup time %s is beyond end-of-time", cleanupTime);
         directContext.timers().setTimer(cleanupTime, TimeDomain.EVENT_TIME);
       }
@@ -822,14 +824,14 @@ public class ReduceFnRunner<K, InputT, OutputT, W extends BoundedWindow> {
 
     if (newHold != null) {
       // We can't be finished yet.
-      Preconditions.checkState(
+      checkState(
         !isFinished, "new hold at %s but finished %s", newHold, directContext.window());
       // The hold cannot be behind the input watermark.
-      Preconditions.checkState(
+      checkState(
         !newHold.isBefore(inputWM), "new hold %s is before input watermark %s", newHold, inputWM);
       if (newHold.isAfter(directContext.window().maxTimestamp())) {
         // The hold must be for garbage collection, which can't have happened yet.
-        Preconditions.checkState(
+        checkState(
           newHold.isEqual(garbageCollectionTime(directContext.window())),
           "new hold %s should be at garbage collection for window %s plus %s",
           newHold,
@@ -837,12 +839,12 @@ public class ReduceFnRunner<K, InputT, OutputT, W extends BoundedWindow> {
           windowingStrategy.getAllowedLateness());
       } else {
         // The hold must be for the end-of-window, which can't have happened yet.
-        Preconditions.checkState(
+        checkState(
           newHold.isEqual(directContext.window().maxTimestamp()),
           "new hold %s should be at end of window %s",
           newHold,
           directContext.window());
-        Preconditions.checkState(
+        checkState(
           !isEndOfWindow,
           "new hold at %s for %s but this is the watermark trigger",
           newHold,
@@ -915,7 +917,7 @@ public class ReduceFnRunner<K, InputT, OutputT, W extends BoundedWindow> {
         directContext.window(),
         inputWM,
         timerInternals.currentOutputWatermarkTime());
-    Preconditions.checkState(!timer.isAfter(BoundedWindow.TIMESTAMP_MAX_VALUE),
+    checkState(!timer.isAfter(BoundedWindow.TIMESTAMP_MAX_VALUE),
                              "Timer %s is beyond end-of-time", timer);
     directContext.timers().setTimer(timer, TimeDomain.EVENT_TIME);
     return timer;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/runners/core-java/src/main/java/org/apache/beam/sdk/util/TriggerRunner.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/TriggerRunner.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/TriggerRunner.java
index f104f6a..878d1d7 100644
--- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/TriggerRunner.java
+++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/TriggerRunner.java
@@ -17,6 +17,8 @@
  */
 package org.apache.beam.sdk.util;
 
+import static com.google.common.base.Preconditions.checkState;
+
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.transforms.windowing.DefaultTrigger;
 import org.apache.beam.sdk.transforms.windowing.Trigger;
@@ -27,7 +29,6 @@ import org.apache.beam.sdk.util.state.StateTags;
 import org.apache.beam.sdk.util.state.ValueState;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableMap;
 
 import org.joda.time.Instant;
@@ -66,7 +67,7 @@ public class TriggerRunner<W extends BoundedWindow> {
   private final TriggerContextFactory<W> contextFactory;
 
   public TriggerRunner(ExecutableTrigger rootTrigger, TriggerContextFactory<W> contextFactory) {
-    Preconditions.checkState(rootTrigger.getTriggerIndex() == 0);
+    checkState(rootTrigger.getTriggerIndex() == 0);
     this.rootTrigger = rootTrigger;
     this.contextFactory = contextFactory;
   }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/runners/core-java/src/main/java/org/apache/beam/sdk/util/WatermarkHold.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/WatermarkHold.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/WatermarkHold.java
index 14ec082..61ab44a 100644
--- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/WatermarkHold.java
+++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/WatermarkHold.java
@@ -17,6 +17,8 @@
  */
 package org.apache.beam.sdk.util;
 
+import static com.google.common.base.Preconditions.checkState;
+
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.transforms.windowing.OutputTimeFn;
 import org.apache.beam.sdk.transforms.windowing.OutputTimeFns;
@@ -30,7 +32,6 @@ import org.apache.beam.sdk.util.state.StateTags;
 import org.apache.beam.sdk.util.state.WatermarkHoldState;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
 
 import org.joda.time.Duration;
 import org.joda.time.Instant;
@@ -205,10 +206,10 @@ class WatermarkHold<W extends BoundedWindow> implements Serializable {
    */
   private Instant shift(Instant timestamp, W window) {
     Instant shifted = windowingStrategy.getOutputTimeFn().assignOutputTime(timestamp, window);
-    Preconditions.checkState(!shifted.isBefore(timestamp),
+    checkState(!shifted.isBefore(timestamp),
         "OutputTimeFn moved element from %s to earlier time %s for window %s",
         timestamp, shifted, window);
-    Preconditions.checkState(timestamp.isAfter(window.maxTimestamp())
+    checkState(timestamp.isAfter(window.maxTimestamp())
             || !shifted.isAfter(window.maxTimestamp()),
         "OutputTimeFn moved element from %s to %s which is beyond end of "
             + "window %s",
@@ -254,8 +255,8 @@ class WatermarkHold<W extends BoundedWindow> implements Serializable {
     } else {
       which = "on time";
       tooLate = false;
-      Preconditions.checkState(!elementHold.isAfter(BoundedWindow.TIMESTAMP_MAX_VALUE),
-                               "Element hold %s is beyond end-of-time", elementHold);
+      checkState(!elementHold.isAfter(BoundedWindow.TIMESTAMP_MAX_VALUE),
+          "Element hold %s is beyond end-of-time", elementHold);
       context.state().access(elementHoldTag).add(elementHold);
     }
     WindowTracing.trace(
@@ -316,10 +317,10 @@ class WatermarkHold<W extends BoundedWindow> implements Serializable {
       return null;
     }
 
-    Preconditions.checkState(outputWM == null || !eowHold.isBefore(outputWM),
+    checkState(outputWM == null || !eowHold.isBefore(outputWM),
         "End-of-window hold %s cannot be before output watermark %s",
         eowHold, outputWM);
-    Preconditions.checkState(!eowHold.isAfter(BoundedWindow.TIMESTAMP_MAX_VALUE),
+    checkState(!eowHold.isAfter(BoundedWindow.TIMESTAMP_MAX_VALUE),
         "End-of-window hold %s is beyond end-of-time", eowHold);
     // If paneIsEmpty then this hold is just for empty ON_TIME panes, so we want to keep
     // the hold away from the combining function in elementHoldTag.
@@ -387,10 +388,10 @@ class WatermarkHold<W extends BoundedWindow> implements Serializable {
       return null;
     }
 
-    Preconditions.checkState(!gcHold.isBefore(inputWM),
+    checkState(!gcHold.isBefore(inputWM),
         "Garbage collection hold %s cannot be before input watermark %s",
         gcHold, inputWM);
-    Preconditions.checkState(!gcHold.isAfter(BoundedWindow.TIMESTAMP_MAX_VALUE),
+    checkState(!gcHold.isAfter(BoundedWindow.TIMESTAMP_MAX_VALUE),
         "Garbage collection hold %s is beyond end-of-time", gcHold);
     // Same EXTRA_HOLD_TAG vs elementHoldTag discussion as in addEndOfWindowHold above.
     context.state().access(EXTRA_HOLD_TAG).add(gcHold);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnRunnerTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnRunnerTest.java
index cd78107..dc2413a 100644
--- a/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnRunnerTest.java
+++ b/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnRunnerTest.java
@@ -20,6 +20,7 @@ package org.apache.beam.sdk.util;
 import static org.apache.beam.sdk.WindowMatchers.isSingleWindowedValue;
 import static org.apache.beam.sdk.WindowMatchers.isWindowedValue;
 
+import static com.google.common.base.Preconditions.checkArgument;
 import static org.hamcrest.Matchers.contains;
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.hamcrest.Matchers.emptyIterable;
@@ -66,7 +67,6 @@ import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode;
 import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.TimestampedValue;
 
-import com.google.common.base.Preconditions;
 import com.google.common.collect.Iterables;
 
 import org.joda.time.Duration;
@@ -1391,26 +1391,26 @@ public class ReduceFnRunnerTest {
     }
     @Override
     public int[] createAccumulator(Context c) {
-      Preconditions.checkArgument(
+      checkArgument(
           c.getPipelineOptions().as(TestOptions.class).getValue() == expectedValue);
-      Preconditions.checkArgument(c.sideInput(view) == expectedValue);
+      checkArgument(c.sideInput(view) == expectedValue);
       return wrap(0);
     }
 
     @Override
     public int[] addInput(int[] accumulator, Integer input, Context c) {
-      Preconditions.checkArgument(
+      checkArgument(
           c.getPipelineOptions().as(TestOptions.class).getValue() == expectedValue);
-      Preconditions.checkArgument(c.sideInput(view) == expectedValue);
+      checkArgument(c.sideInput(view) == expectedValue);
       accumulator[0] += input.intValue();
       return accumulator;
     }
 
     @Override
     public int[] mergeAccumulators(Iterable<int[]> accumulators, Context c) {
-      Preconditions.checkArgument(
+      checkArgument(
           c.getPipelineOptions().as(TestOptions.class).getValue() == expectedValue);
-      Preconditions.checkArgument(c.sideInput(view) == expectedValue);
+      checkArgument(c.sideInput(view) == expectedValue);
       Iterator<int[]> iter = accumulators.iterator();
       if (!iter.hasNext()) {
         return createAccumulator(c);
@@ -1425,9 +1425,9 @@ public class ReduceFnRunnerTest {
 
     @Override
     public Integer extractOutput(int[] accumulator, Context c) {
-      Preconditions.checkArgument(
+      checkArgument(
           c.getPipelineOptions().as(TestOptions.class).getValue() == expectedValue);
-      Preconditions.checkArgument(c.sideInput(view) == expectedValue);
+      checkArgument(c.sideInput(view) == expectedValue);
       return accumulator[0];
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnTester.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnTester.java b/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnTester.java
index fa62583..e0ff879 100644
--- a/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnTester.java
+++ b/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnTester.java
@@ -17,6 +17,8 @@
  */
 package org.apache.beam.sdk.util;
 
+import static com.google.common.base.Preconditions.checkNotNull;
+import static com.google.common.base.Preconditions.checkState;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -57,7 +59,6 @@ import org.apache.beam.sdk.values.TupleTag;
 
 import com.google.common.base.Function;
 import com.google.common.base.MoreObjects;
-import com.google.common.base.Preconditions;
 import com.google.common.collect.FluentIterable;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
@@ -637,7 +638,7 @@ public class ReduceFnTester<InputT, OutputT, W extends BoundedWindow> {
           data = processingTimers.peek();
           break;
       }
-      Preconditions.checkNotNull(data); // cases exhaustive
+      checkNotNull(data); // cases exhaustive
       return data == null ? null : data.getTimestamp();
     }
 
@@ -680,7 +681,7 @@ public class ReduceFnTester<InputT, OutputT, W extends BoundedWindow> {
 
     @Override
     public Instant currentInputWatermarkTime() {
-      return Preconditions.checkNotNull(inputWatermarkTime);
+      return checkNotNull(inputWatermarkTime);
     }
 
     @Override
@@ -702,8 +703,8 @@ public class ReduceFnTester<InputT, OutputT, W extends BoundedWindow> {
 
     public void advanceInputWatermark(
         ReduceFnRunner<?, ?, ?, ?> runner, Instant newInputWatermark) throws Exception {
-      Preconditions.checkNotNull(newInputWatermark);
-      Preconditions.checkState(
+      checkNotNull(newInputWatermark);
+      checkState(
           !newInputWatermark.isBefore(inputWatermarkTime),
           "Cannot move input watermark time backwards from %s to %s", inputWatermarkTime,
           newInputWatermark);
@@ -724,14 +725,14 @@ public class ReduceFnTester<InputT, OutputT, W extends BoundedWindow> {
     }
 
     public void advanceOutputWatermark(Instant newOutputWatermark) {
-      Preconditions.checkNotNull(newOutputWatermark);
+      checkNotNull(newOutputWatermark);
       if (newOutputWatermark.isAfter(inputWatermarkTime)) {
         WindowTracing.trace(
             "TestTimerInternals.advanceOutputWatermark: clipping output watermark from {} to {}",
             newOutputWatermark, inputWatermarkTime);
         newOutputWatermark = inputWatermarkTime;
       }
-      Preconditions.checkState(
+      checkState(
           outputWatermarkTime == null || !newOutputWatermark.isBefore(outputWatermarkTime),
           "Cannot move output watermark time backwards from %s to %s", outputWatermarkTime,
           newOutputWatermark);
@@ -742,7 +743,7 @@ public class ReduceFnTester<InputT, OutputT, W extends BoundedWindow> {
 
     public void advanceProcessingTime(
         ReduceFnRunner<?, ?, ?, ?> runner, Instant newProcessingTime) throws Exception {
-      Preconditions.checkState(!newProcessingTime.isBefore(processingTime),
+      checkState(!newProcessingTime.isBefore(processingTime),
           "Cannot move processing time backwards from %s to %s", processingTime, newProcessingTime);
       WindowTracing.trace("TestTimerInternals.advanceProcessingTime: from {} to {}", processingTime,
           newProcessingTime);
@@ -752,7 +753,7 @@ public class ReduceFnTester<InputT, OutputT, W extends BoundedWindow> {
 
     public void advanceSynchronizedProcessingTime(
         ReduceFnRunner<?, ?, ?, ?> runner, Instant newSynchronizedProcessingTime) throws Exception {
-      Preconditions.checkState(!newSynchronizedProcessingTime.isBefore(synchronizedProcessingTime),
+      checkState(!newSynchronizedProcessingTime.isBefore(synchronizedProcessingTime),
           "Cannot move processing time backwards from %s to %s", processingTime,
           newSynchronizedProcessingTime);
       WindowTracing.trace("TestTimerInternals.advanceProcessingTime: from {} to {}",

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java
index b8f9987..c8dfa8c 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java
@@ -17,6 +17,8 @@
  */
 package org.apache.beam.runners.direct;
 
+import static com.google.common.base.Preconditions.checkNotNull;
+
 import org.apache.beam.runners.direct.DirectRunner.CommittedBundle;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.transforms.AppliedPTransform;
@@ -33,7 +35,6 @@ import com.google.auto.value.AutoValue;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Function;
 import com.google.common.base.MoreObjects;
-import com.google.common.base.Preconditions;
 import com.google.common.collect.ComparisonChain;
 import com.google.common.collect.FluentIterable;
 import com.google.common.collect.ImmutableList;
@@ -1071,7 +1072,7 @@ public class WatermarkManager {
      * Returns the input watermark of the {@link AppliedPTransform}.
      */
     public Instant getInputWatermark() {
-      return Preconditions.checkNotNull(inputWatermark.get());
+      return checkNotNull(inputWatermark.get());
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java
index 4cd8fb3..f4d4ea6 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java
@@ -17,6 +17,8 @@
  */
 package org.apache.beam.runners.flink;
 
+import static com.google.common.base.Preconditions.checkNotNull;
+
 import org.apache.beam.runners.flink.translation.FlinkBatchPipelineTranslator;
 import org.apache.beam.runners.flink.translation.FlinkPipelineTranslator;
 import org.apache.beam.runners.flink.translation.FlinkStreamingPipelineTranslator;
@@ -24,8 +26,6 @@ import org.apache.beam.runners.flink.translation.PipelineTranslationOptimizer;
 import org.apache.beam.runners.flink.translation.TranslationMode;
 import org.apache.beam.sdk.Pipeline;
 
-import com.google.common.base.Preconditions;
-
 import org.apache.flink.api.common.JobExecutionResult;
 import org.apache.flink.api.java.CollectionEnvironment;
 import org.apache.flink.api.java.ExecutionEnvironment;
@@ -74,7 +74,7 @@ public class FlinkPipelineExecutionEnvironment {
    * @param options the user-defined pipeline options.
    * */
   FlinkPipelineExecutionEnvironment(FlinkPipelineOptions options) {
-    this.options = Preconditions.checkNotNull(options);
+    this.options = checkNotNull(options);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTranslationContext.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTranslationContext.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTranslationContext.java
index 0cb80ba..a75ef03 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTranslationContext.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTranslationContext.java
@@ -17,6 +17,8 @@
  */
 package org.apache.beam.runners.flink.translation;
 
+import static com.google.common.base.Preconditions.checkNotNull;
+
 import org.apache.beam.runners.flink.translation.types.CoderTypeInformation;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.options.PipelineOptions;
@@ -28,8 +30,6 @@ import org.apache.beam.sdk.values.PInput;
 import org.apache.beam.sdk.values.POutput;
 import org.apache.beam.sdk.values.PValue;
 
-import com.google.common.base.Preconditions;
-
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
@@ -56,8 +56,8 @@ public class FlinkStreamingTranslationContext {
   private AppliedPTransform<?, ?, ?> currentTransform;
 
   public FlinkStreamingTranslationContext(StreamExecutionEnvironment env, PipelineOptions options) {
-    this.env = Preconditions.checkNotNull(env);
-    this.options = Preconditions.checkNotNull(options);
+    this.env = checkNotNull(env);
+    this.options = checkNotNull(options);
     this.dataStreams = new HashMap<>();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkProcessContext.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkProcessContext.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkProcessContext.java
index 0f1885c..0ee8198 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkProcessContext.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkProcessContext.java
@@ -17,6 +17,8 @@
  */
 package org.apache.beam.runners.flink.translation.functions;
 
+import static com.google.common.base.Preconditions.checkNotNull;
+
 import org.apache.beam.runners.flink.translation.wrappers.SerializableFnAggregatorWrapper;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.options.PipelineOptions;
@@ -33,7 +35,6 @@ import org.apache.beam.sdk.util.state.StateInternals;
 import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.TupleTag;
 
-import com.google.common.base.Preconditions;
 import com.google.common.collect.Iterables;
 
 import org.apache.flink.api.common.functions.RuntimeContext;
@@ -71,10 +72,10 @@ class FlinkProcessContext<InputT, OutputT>
       Collector<WindowedValue<OutputT>> collector,
       Map<PCollectionView<?>, WindowingStrategy<?, ?>> sideInputs) {
     doFn.super();
-    Preconditions.checkNotNull(pipelineOptions);
-    Preconditions.checkNotNull(runtimeContext);
-    Preconditions.checkNotNull(doFn);
-    Preconditions.checkNotNull(collector);
+    checkNotNull(pipelineOptions);
+    checkNotNull(runtimeContext);
+    checkNotNull(doFn);
+    checkNotNull(collector);
 
     this.pipelineOptions = pipelineOptions;
     this.runtimeContext = runtimeContext;
@@ -93,9 +94,9 @@ class FlinkProcessContext<InputT, OutputT>
       WindowingStrategy<?, ?> windowingStrategy,
       Map<PCollectionView<?>, WindowingStrategy<?, ?>> sideInputs) {
     doFn.super();
-    Preconditions.checkNotNull(pipelineOptions);
-    Preconditions.checkNotNull(runtimeContext);
-    Preconditions.checkNotNull(doFn);
+    checkNotNull(pipelineOptions);
+    checkNotNull(runtimeContext);
+    checkNotNull(doFn);
 
     this.pipelineOptions = pipelineOptions;
     this.runtimeContext = runtimeContext;
@@ -196,8 +197,8 @@ class FlinkProcessContext<InputT, OutputT>
           PCollectionView<ViewT> view,
           BoundedWindow mainInputWindow) {
 
-        Preconditions.checkNotNull(view, "View passed to sideInput cannot be null");
-        Preconditions.checkNotNull(
+        checkNotNull(view, "View passed to sideInput cannot be null");
+        checkNotNull(
             sideInputs.get(view),
             "Side input for " + view + " not available.");
 
@@ -222,8 +223,8 @@ class FlinkProcessContext<InputT, OutputT>
 
   @Override
   public <ViewT> ViewT sideInput(PCollectionView<ViewT> view) {
-    Preconditions.checkNotNull(view, "View passed to sideInput cannot be null");
-    Preconditions.checkNotNull(sideInputs.get(view), "Side input for " + view + " not available.");
+    checkNotNull(view, "View passed to sideInput cannot be null");
+    checkNotNull(sideInputs.get(view), "Side input for " + view + " not available.");
     Iterator<? extends BoundedWindow> windowIter = windowedValue.getWindows().iterator();
     BoundedWindow window;
     if (!windowIter.hasNext()) {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java
index 0e85486..71cc6b7 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java
@@ -17,9 +17,9 @@
  */
 package org.apache.beam.runners.flink.translation.types;
 
-import org.apache.beam.sdk.coders.Coder;
+import static com.google.common.base.Preconditions.checkNotNull;
 
-import com.google.common.base.Preconditions;
+import org.apache.beam.sdk.coders.Coder;
 
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.typeinfo.AtomicType;
@@ -36,7 +36,7 @@ public class CoderTypeInformation<T> extends TypeInformation<T> implements Atomi
   private final Coder<T> coder;
 
   public CoderTypeInformation(Coder<T> coder) {
-    Preconditions.checkNotNull(coder);
+    checkNotNull(coder);
     this.coder = coder;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/utils/SerializedPipelineOptions.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/utils/SerializedPipelineOptions.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/utils/SerializedPipelineOptions.java
index 2b35c31..44af0ea 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/utils/SerializedPipelineOptions.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/utils/SerializedPipelineOptions.java
@@ -18,10 +18,12 @@
 
 package org.apache.beam.runners.flink.translation.utils;
 
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.google.common.base.Preconditions;
+import static com.google.common.base.Preconditions.checkNotNull;
+
 import org.apache.beam.sdk.options.PipelineOptions;
 
+import com.fasterxml.jackson.databind.ObjectMapper;
+
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.Serializable;
@@ -37,7 +39,7 @@ public class SerializedPipelineOptions implements Serializable {
   private transient PipelineOptions pipelineOptions;
 
   public SerializedPipelineOptions(PipelineOptions options) {
-    Preconditions.checkNotNull(options, "PipelineOptions must not be null.");
+    checkNotNull(options, "PipelineOptions must not be null.");
 
     try (ByteArrayOutputStream baos = new ByteArrayOutputStream()) {
       new ObjectMapper().writeValue(baos, options);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java
index f68a519..d8222b6 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java
@@ -17,6 +17,8 @@
  */
 package org.apache.beam.runners.flink.translation.wrappers.streaming;
 
+import static com.google.common.base.Preconditions.checkNotNull;
+
 import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions;
 import org.apache.beam.runners.flink.translation.wrappers.SerializableFnAggregatorWrapper;
 import org.apache.beam.sdk.options.PipelineOptions;
@@ -33,8 +35,6 @@ import org.apache.beam.sdk.util.WindowingStrategy;
 import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.TupleTag;
 
-import com.google.common.base.Preconditions;
-
 import org.apache.flink.api.common.accumulators.Accumulator;
 import org.apache.flink.api.common.accumulators.AccumulatorHelper;
 import org.apache.flink.api.common.functions.RichFlatMapFunction;
@@ -59,9 +59,9 @@ public abstract class FlinkAbstractParDoWrapper<IN, OUTDF, OUTFL> extends RichFl
   private DoFnProcessContext context;
 
   public FlinkAbstractParDoWrapper(PipelineOptions options, WindowingStrategy<?, ?> windowingStrategy, DoFn<IN, OUTDF> doFn) {
-    Preconditions.checkNotNull(options);
-    Preconditions.checkNotNull(windowingStrategy);
-    Preconditions.checkNotNull(doFn);
+    checkNotNull(options);
+    checkNotNull(windowingStrategy);
+    checkNotNull(doFn);
 
     this.doFn = doFn;
     this.serializedPipelineOptions = new SerializedPipelineOptions(options);

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java
index 9d2cad8..3f845cf 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java
@@ -17,6 +17,8 @@
  */
 package org.apache.beam.runners.flink.translation.wrappers.streaming;
 
+import static com.google.common.base.Preconditions.checkNotNull;
+
 import org.apache.beam.runners.core.GroupAlsoByWindowViaWindowSetDoFn;
 import org.apache.beam.runners.flink.translation.types.CoderTypeInformation;
 import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions;
@@ -50,7 +52,6 @@ import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.TupleTag;
 
-import com.google.common.base.Preconditions;
 import com.google.common.collect.HashMultimap;
 import com.google.common.collect.Multimap;
 
@@ -152,7 +153,7 @@ public class FlinkGroupAlsoByWindowWrapper<K, VIN, VACC, VOUT>
       KeyedStream<WindowedValue<KV<K, VIN>>, K> groupedStreamByKey,
       Combine.KeyedCombineFn<K, VIN, VACC, VOUT> combiner,
       KvCoder<K, VOUT> outputKvCoder) {
-    Preconditions.checkNotNull(options);
+    checkNotNull(options);
 
     KvCoder<K, VIN> inputKvCoder = (KvCoder<K, VIN>) input.getCoder();
     FlinkGroupAlsoByWindowWrapper windower = new FlinkGroupAlsoByWindowWrapper<>(options,
@@ -190,7 +191,7 @@ public class FlinkGroupAlsoByWindowWrapper<K, VIN, VACC, VOUT>
       PipelineOptions options,
       PCollection input,
       KeyedStream<WindowedValue<KV<K, VIN>>, K> groupedStreamByKey) {
-    Preconditions.checkNotNull(options);
+    checkNotNull(options);
 
     KvCoder<K, VIN> inputKvCoder = (KvCoder<K, VIN>) input.getCoder();
     Coder<K> keyCoder = inputKvCoder.getKeyCoder();
@@ -224,7 +225,7 @@ public class FlinkGroupAlsoByWindowWrapper<K, VIN, VACC, VOUT>
                    WindowingStrategy<KV<K, VIN>, BoundedWindow> windowingStrategy,
                    KvCoder<K, VIN> inputCoder,
                    Combine.KeyedCombineFn<K, VIN, VACC, VOUT> combiner) {
-    Preconditions.checkNotNull(options);
+    checkNotNull(options);
 
     return new FlinkGroupAlsoByWindowWrapper(options, registry, windowingStrategy, inputCoder, combiner);
   }
@@ -234,12 +235,12 @@ public class FlinkGroupAlsoByWindowWrapper<K, VIN, VACC, VOUT>
                                         WindowingStrategy<KV<K, VIN>, BoundedWindow> windowingStrategy,
                                         KvCoder<K, VIN> inputCoder,
                                         Combine.KeyedCombineFn<K, VIN, VACC, VOUT> combiner) {
-    Preconditions.checkNotNull(options);
+    checkNotNull(options);
 
-    this.serializedOptions = new SerializedPipelineOptions(Preconditions.checkNotNull(options));
-    this.coderRegistry = Preconditions.checkNotNull(registry);
-    this.inputKvCoder = Preconditions.checkNotNull(inputCoder);//(KvCoder<K, VIN>) input.getCoder();
-    this.windowingStrategy = Preconditions.checkNotNull(windowingStrategy);//input.getWindowingStrategy();
+    this.serializedOptions = new SerializedPipelineOptions(checkNotNull(options));
+    this.coderRegistry = checkNotNull(registry);
+    this.inputKvCoder = checkNotNull(inputCoder);//(KvCoder<K, VIN>) input.getCoder();
+    this.windowingStrategy = checkNotNull(windowingStrategy);//input.getWindowingStrategy();
     this.combineFn = combiner;
     this.operator = createGroupAlsoByWindowOperator();
     this.chainingStrategy = ChainingStrategy.ALWAYS;
@@ -447,8 +448,8 @@ public class FlinkGroupAlsoByWindowWrapper<K, VIN, VACC, VOUT>
       function.super();
       super.setupDelegateAggregators();
 
-      this.collector = Preconditions.checkNotNull(outCollector);
-      this.timerInternals = Preconditions.checkNotNull(timerInternals);
+      this.collector = checkNotNull(outCollector);
+      this.timerInternals = checkNotNull(timerInternals);
     }
 
     public void setElement(KeyedWorkItem<K, VIN> element,

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java
index a30a544..619b887 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java
@@ -17,6 +17,8 @@
  */
 package org.apache.beam.runners.flink.translation.wrappers.streaming;
 
+import static com.google.common.base.Preconditions.checkNotNull;
+
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.join.RawUnionValue;
@@ -25,8 +27,6 @@ import org.apache.beam.sdk.util.WindowingInternals;
 import org.apache.beam.sdk.util.WindowingStrategy;
 import org.apache.beam.sdk.values.TupleTag;
 
-import com.google.common.base.Preconditions;
-
 import org.apache.flink.util.Collector;
 import org.joda.time.Instant;
 
@@ -42,8 +42,8 @@ public class FlinkParDoBoundMultiWrapper<IN, OUT> extends FlinkAbstractParDoWrap
 
   public FlinkParDoBoundMultiWrapper(PipelineOptions options, WindowingStrategy<?, ?> windowingStrategy, DoFn<IN, OUT> doFn, TupleTag<?> mainTag, Map<TupleTag<?>, Integer> tagsToLabels) {
     super(options, windowingStrategy, doFn);
-    this.mainTag = Preconditions.checkNotNull(mainTag);
-    this.outputLabels = Preconditions.checkNotNull(tagsToLabels);
+    this.mainTag = checkNotNull(mainTag);
+    this.outputLabels = checkNotNull(tagsToLabels);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSink.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSink.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSink.java
index 77c195a..098473d 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSink.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSink.java
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.runners.flink.translation.wrappers.streaming.io;
 
-import com.google.common.base.Preconditions;
+
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.CoderException;
 import org.apache.beam.sdk.io.Sink;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSource.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSource.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSource.java
index b636036..94b73ce 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSource.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSource.java
@@ -17,13 +17,12 @@
  */
 package org.apache.beam.runners.flink.translation.wrappers.streaming.io;
 
-import org.apache.beam.runners.flink.translation.types.CoderTypeInformation;
+import static com.google.common.base.Preconditions.checkNotNull;
+
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.io.UnboundedSource;
 import org.apache.beam.sdk.options.PipelineOptions;
 
-import com.google.common.base.Preconditions;
-
 import org.apache.flink.streaming.api.functions.source.SourceFunction;
 
 import java.util.List;
@@ -42,7 +41,7 @@ public class UnboundedFlinkSource<T> extends UnboundedSource<T, UnboundedSource.
   private Coder<T> coder;
 
   public UnboundedFlinkSource(SourceFunction<T> source) {
-    flinkSource = Preconditions.checkNotNull(source);
+    flinkSource = checkNotNull(source);
   }
 
   public SourceFunction<T> getFlinkSource() {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkStateInternals.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkStateInternals.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkStateInternals.java
index 18d4c3c..e6a43dc 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkStateInternals.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkStateInternals.java
@@ -17,6 +17,8 @@
  */
 package org.apache.beam.runners.flink.translation.wrappers.streaming.state;
 
+import static com.google.common.base.Preconditions.checkNotNull;
+
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.transforms.Combine;
@@ -38,7 +40,6 @@ import org.apache.beam.sdk.util.state.ValueState;
 import org.apache.beam.sdk.util.state.WatermarkHoldState;
 import org.apache.beam.sdk.values.PCollectionView;
 
-import com.google.common.base.Preconditions;
 import com.google.protobuf.ByteString;
 
 import org.apache.flink.util.InstantiationUtil;
@@ -534,8 +535,8 @@ public class FlinkStateInternals<K> implements StateInternals<K> {
                                              CombineWithContext.KeyedCombineFnWithContext<? super K, InputT, AccumT, OutputT> combineFn,
                                              Coder<AccumT> accumCoder,
                                              final StateContext<?> stateContext) {
-      Preconditions.checkNotNull(combineFn);
-      Preconditions.checkNotNull(accumCoder);
+      checkNotNull(combineFn);
+      checkNotNull(accumCoder);
 
       this.stateKey = stateKey;
       this.combineFn = combineFn;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java
index e15b9d2..7fd203f 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java
@@ -29,6 +29,9 @@ import static org.apache.beam.sdk.util.Structs.addString;
 import static org.apache.beam.sdk.util.Structs.getString;
 
 import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+import static com.google.common.base.Preconditions.checkState;
+import static com.google.common.base.Strings.isNullOrEmpty;
 
 import org.apache.beam.runners.dataflow.DataflowRunner.GroupByKeyAndSortValuesOnly;
 import org.apache.beam.runners.dataflow.internal.ReadTranslator;
@@ -76,8 +79,6 @@ import com.google.api.services.dataflow.model.Environment;
 import com.google.api.services.dataflow.model.Job;
 import com.google.api.services.dataflow.model.Step;
 import com.google.api.services.dataflow.model.WorkerPool;
-import com.google.common.base.Preconditions;
-import com.google.common.base.Strings;
 
 import com.fasterxml.jackson.core.JsonProcessingException;
 import com.fasterxml.jackson.databind.ObjectMapper;
@@ -435,13 +436,13 @@ public class DataflowPipelineTranslator {
         disk.setDiskType(options.getWorkerDiskType());
         workerPool.setDataDisks(Collections.singletonList(disk));
       }
-      if (!Strings.isNullOrEmpty(options.getZone())) {
+      if (!isNullOrEmpty(options.getZone())) {
         workerPool.setZone(options.getZone());
       }
-      if (!Strings.isNullOrEmpty(options.getNetwork())) {
+      if (!isNullOrEmpty(options.getNetwork())) {
         workerPool.setNetwork(options.getNetwork());
       }
-      if (!Strings.isNullOrEmpty(options.getSubnetwork())) {
+      if (!isNullOrEmpty(options.getSubnetwork())) {
         workerPool.setSubnetwork(options.getSubnetwork());
       }
       if (options.getDiskSizeGb() > 0) {
@@ -669,11 +670,11 @@ public class DataflowPipelineTranslator {
                                                PValue inputValue,
                                                PValue outputValue) {
       Coder<?> inputValueCoder =
-          Preconditions.checkNotNull(outputCoders.get(inputValue));
+          checkNotNull(outputCoders.get(inputValue));
       // The inputValueCoder for the input PCollection should be some
       // WindowedValueCoder of the input PCollection's element
       // coder.
-      Preconditions.checkState(
+      checkState(
           inputValueCoder instanceof WindowedValue.WindowedValueCoder);
       // The outputValueCoder for the output should be an
       // IterableCoder of the inputValueCoder. This is a property

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
index 7ff247a..33f97e6 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
@@ -23,6 +23,7 @@ import static org.apache.beam.sdk.util.WindowedValue.valueInEmptyWindows;
 
 import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkState;
+import static com.google.common.base.Strings.isNullOrEmpty;
 
 import org.apache.beam.runners.dataflow.DataflowPipelineTranslator.JobSpecification;
 import org.apache.beam.runners.dataflow.DataflowPipelineTranslator.TransformTranslator;
@@ -135,8 +136,6 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Function;
 import com.google.common.base.Joiner;
 import com.google.common.base.Optional;
-import com.google.common.base.Preconditions;
-import com.google.common.base.Strings;
 import com.google.common.base.Utf8;
 import com.google.common.collect.ForwardingMap;
 import com.google.common.collect.HashMultimap;
@@ -261,8 +260,8 @@ public class DataflowRunner extends PipelineRunner<DataflowPipelineJob> {
     }
 
     PathValidator validator = dataflowOptions.getPathValidator();
-    Preconditions.checkArgument(!(Strings.isNullOrEmpty(dataflowOptions.getTempLocation())
-        && Strings.isNullOrEmpty(dataflowOptions.getStagingLocation())),
+    checkArgument(!(isNullOrEmpty(dataflowOptions.getTempLocation())
+        && isNullOrEmpty(dataflowOptions.getStagingLocation())),
         "Missing required value: at least one of tempLocation or stagingLocation must be set.");
 
     if (dataflowOptions.getStagingLocation() != null) {
@@ -271,9 +270,9 @@ public class DataflowRunner extends PipelineRunner<DataflowPipelineJob> {
     if (dataflowOptions.getTempLocation() != null) {
       validator.validateOutputFilePrefixSupported(dataflowOptions.getTempLocation());
     }
-    if (Strings.isNullOrEmpty(dataflowOptions.getTempLocation())) {
+    if (isNullOrEmpty(dataflowOptions.getTempLocation())) {
       dataflowOptions.setTempLocation(dataflowOptions.getStagingLocation());
-    } else if (Strings.isNullOrEmpty(dataflowOptions.getStagingLocation())) {
+    } else if (isNullOrEmpty(dataflowOptions.getStagingLocation())) {
       try {
         dataflowOptions.setStagingLocation(
             IOChannelUtils.resolve(dataflowOptions.getTempLocation(), "staging"));
@@ -545,7 +544,7 @@ public class DataflowRunner extends PipelineRunner<DataflowPipelineJob> {
     newJob.getEnvironment().setUserAgent(ReleaseInfo.getReleaseInfo());
     // The Dataflow Service may write to the temporary directory directly, so
     // must be verified.
-    if (!Strings.isNullOrEmpty(options.getTempLocation())) {
+    if (!isNullOrEmpty(options.getTempLocation())) {
       newJob.getEnvironment().setTempStoragePrefix(
           dataflowOptions.getPathValidator().verifyPath(options.getTempLocation()));
     }
@@ -578,7 +577,7 @@ public class DataflowRunner extends PipelineRunner<DataflowPipelineJob> {
       hooks.modifyEnvironmentBeforeSubmission(newJob.getEnvironment());
     }
 
-    if (!Strings.isNullOrEmpty(options.getDataflowJobFile())) {
+    if (!isNullOrEmpty(options.getDataflowJobFile())) {
       try (PrintWriter printWriter = new PrintWriter(
           new File(options.getDataflowJobFile()))) {
         String workSpecJson = DataflowPipelineTranslator.jobToString(newJob);
@@ -2203,7 +2202,7 @@ public class DataflowRunner extends PipelineRunner<DataflowPipelineJob> {
           break;  // supported by server
         case "":
           // Empty shard template allowed - forces single output.
-          Preconditions.checkArgument(originalTransform.getNumShards() <= 1,
+          checkArgument(originalTransform.getNumShards() <= 1,
               "Num shards must be <= 1 when using an empty sharding template");
           break;
         default:
@@ -2308,7 +2307,7 @@ public class DataflowRunner extends PipelineRunner<DataflowPipelineJob> {
           break;  // supported by server
         case "":
           // Empty shard template allowed - forces single output.
-          Preconditions.checkArgument(originalTransform.getNumShards() <= 1,
+          checkArgument(originalTransform.getNumShards() <= 1,
               "Num shards must be <= 1 when using an empty sharding template");
           break;
         default:

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/IsmFormat.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/IsmFormat.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/IsmFormat.java
index 1969cfb..d8bfe42 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/IsmFormat.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/IsmFormat.java
@@ -39,7 +39,6 @@ import org.apache.beam.sdk.util.VarInt;
 import org.apache.beam.sdk.values.PCollection;
 
 import com.google.auto.value.AutoValue;
-import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import com.google.common.hash.HashFunction;
 import com.google.common.hash.Hashing;
@@ -54,6 +53,7 @@ import java.io.InputStream;
 import java.io.OutputStream;
 import java.util.ArrayList;
 import java.util.List;
+
 import javax.annotation.Nullable;
 
 /**
@@ -215,7 +215,7 @@ public class IsmFormat {
         @JsonProperty(PropertyNames.NUM_SHARD_CODERS) int numberOfShardCoders,
         @JsonProperty(PropertyNames.NUM_METADATA_SHARD_CODERS) int numberOfMetadataShardCoders,
         @JsonProperty(PropertyNames.COMPONENT_ENCODINGS) List<Coder<?>> components) {
-      Preconditions.checkArgument(components.size() >= 2,
+      checkArgument(components.size() >= 2,
           "Expecting at least 2 components, got " + components.size());
       return of(
           numberOfShardCoders,
@@ -497,7 +497,7 @@ public class IsmFormat {
     @JsonCreator
     public static MetadataKeyCoder<?> of(
         @JsonProperty(PropertyNames.COMPONENT_ENCODINGS) List<Coder<?>> components) {
-      Preconditions.checkArgument(components.size() == 1,
+      checkArgument(components.size() == 1,
           "Expecting one component, got " + components.size());
       return of(components.get(0));
     }
@@ -725,7 +725,7 @@ public class IsmFormat {
     @Override
     public long getEncodedElementByteSize(KeyPrefix value, Coder.Context context)
         throws Exception {
-      Preconditions.checkNotNull(value);
+      checkNotNull(value);
       return VarInt.getLength(value.getSharedKeySize())
           + VarInt.getLength(value.getUnsharedKeySize());
     }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowWorkerLoggingOptions.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowWorkerLoggingOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowWorkerLoggingOptions.java
index dc7748a..91ac62a 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowWorkerLoggingOptions.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowWorkerLoggingOptions.java
@@ -17,12 +17,12 @@
  */
 package org.apache.beam.runners.dataflow.options;
 
+import static com.google.common.base.Preconditions.checkNotNull;
+
 import org.apache.beam.sdk.options.Default;
 import org.apache.beam.sdk.options.Description;
 import org.apache.beam.sdk.options.PipelineOptions;
 
-import com.google.common.base.Preconditions;
-
 import com.fasterxml.jackson.annotation.JsonCreator;
 
 import java.util.Arrays;
@@ -144,7 +144,7 @@ public interface DataflowWorkerLoggingOptions extends PipelineOptions {
      * and passing in the {@link Class#getName() class name}.
      */
     public WorkerLogLevelOverrides addOverrideForClass(Class<?> klass, Level level) {
-      Preconditions.checkNotNull(klass, "Expected class to be not null.");
+      checkNotNull(klass, "Expected class to be not null.");
       addOverrideForName(klass.getName(), level);
       return this;
     }
@@ -157,7 +157,7 @@ public interface DataflowWorkerLoggingOptions extends PipelineOptions {
      * and passing in the {@link Package#getName() package name}.
      */
     public WorkerLogLevelOverrides addOverrideForPackage(Package pkg, Level level) {
-      Preconditions.checkNotNull(pkg, "Expected package to be not null.");
+      checkNotNull(pkg, "Expected package to be not null.");
       addOverrideForName(pkg.getName(), level);
       return this;
     }
@@ -170,8 +170,8 @@ public interface DataflowWorkerLoggingOptions extends PipelineOptions {
      * a parent logger that has the passed in name.
      */
     public WorkerLogLevelOverrides addOverrideForName(String name, Level level) {
-      Preconditions.checkNotNull(name, "Expected name to be not null.");
-      Preconditions.checkNotNull(level,
+      checkNotNull(name, "Expected name to be not null.");
+      checkNotNull(level,
           "Expected level to be one of %s.", Arrays.toString(Level.values()));
       put(name, level);
       return this;
@@ -186,7 +186,7 @@ public interface DataflowWorkerLoggingOptions extends PipelineOptions {
      */
     @JsonCreator
     public static WorkerLogLevelOverrides from(Map<String, String> values) {
-      Preconditions.checkNotNull(values, "Expected values to be not null.");
+      checkNotNull(values, "Expected values to be not null.");
       WorkerLogLevelOverrides overrides = new WorkerLogLevelOverrides();
       for (Map.Entry<String, String> entry : values.entrySet()) {
         try {

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DataflowPathValidator.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DataflowPathValidator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DataflowPathValidator.java
index ddc5d6f..ec10b28 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DataflowPathValidator.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DataflowPathValidator.java
@@ -17,13 +17,13 @@
  */
 package org.apache.beam.runners.dataflow.util;
 
+import static com.google.common.base.Preconditions.checkArgument;
+
 import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.util.PathValidator;
 import org.apache.beam.sdk.util.gcsfs.GcsPath;
 
-import com.google.common.base.Preconditions;
-
 import java.io.IOException;
 
 /**
@@ -48,8 +48,7 @@ public class DataflowPathValidator implements PathValidator {
   @Override
   public String validateInputFilePatternSupported(String filepattern) {
     GcsPath gcsPath = getGcsPath(filepattern);
-    Preconditions.checkArgument(
-        dataflowOptions.getGcsUtil().isGcsPatternSupported(gcsPath.getObject()));
+    checkArgument(dataflowOptions.getGcsUtil().isGcsPatternSupported(gcsPath.getObject()));
     String returnValue = verifyPath(filepattern);
     verifyPathIsAccessible(filepattern, "Could not find file %s");
     return returnValue;
@@ -69,9 +68,8 @@ public class DataflowPathValidator implements PathValidator {
   @Override
   public String verifyPath(String path) {
     GcsPath gcsPath = getGcsPath(path);
-    Preconditions.checkArgument(gcsPath.isAbsolute(),
-        "Must provide absolute paths for Dataflow");
-    Preconditions.checkArgument(!gcsPath.getObject().contains("//"),
+    checkArgument(gcsPath.isAbsolute(), "Must provide absolute paths for Dataflow");
+    checkArgument(!gcsPath.getObject().contains("//"),
         "Dataflow Service does not allow objects with consecutive slashes");
     return gcsPath.toResourceName();
   }
@@ -79,7 +77,7 @@ public class DataflowPathValidator implements PathValidator {
   private void verifyPathIsAccessible(String path, String errorMessage) {
     GcsPath gcsPath = getGcsPath(path);
     try {
-      Preconditions.checkArgument(dataflowOptions.getGcsUtil().bucketExists(gcsPath),
+      checkArgument(dataflowOptions.getGcsUtil().bucketExists(gcsPath),
         errorMessage, path);
     } catch (IOException e) {
       throw new RuntimeException(

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/GcsStager.java
----------------------------------------------------------------------
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/GcsStager.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/GcsStager.java
index 8e7cbbe..bf25ce4 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/GcsStager.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/GcsStager.java
@@ -17,12 +17,13 @@
  */
 package org.apache.beam.runners.dataflow.util;
 
+import static com.google.common.base.Preconditions.checkNotNull;
+
 import org.apache.beam.runners.dataflow.options.DataflowPipelineDebugOptions;
 import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptions;
 
 import com.google.api.services.dataflow.model.DataflowPackage;
-import com.google.common.base.Preconditions;
 
 import java.util.List;
 
@@ -42,7 +43,7 @@ public class GcsStager implements Stager {
 
   @Override
   public List<DataflowPackage> stageFiles() {
-    Preconditions.checkNotNull(options.getStagingLocation());
+    checkNotNull(options.getStagingLocation());
     List<String> filesToStage = options.getFilesToStage();
     String windmillBinary =
         options.as(DataflowPipelineDebugOptions.class).getOverrideWindmillBinary();

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/runners/spark/src/main/java/org/apache/beam/runners/spark/io/CreateStream.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/CreateStream.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/CreateStream.java
index e7a9971..b3beae6 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/CreateStream.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/CreateStream.java
@@ -17,12 +17,13 @@
  */
 package org.apache.beam.runners.spark.io;
 
+import static com.google.common.base.Preconditions.checkNotNull;
+
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.util.WindowingStrategy;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PInput;
 
-import com.google.common.base.Preconditions;
 
 /**
  * Create an input stream from Queue.
@@ -53,8 +54,8 @@ public final class CreateStream<T> {
     private final Iterable<Iterable<T>> queuedValues;
 
     QueuedValues(Iterable<Iterable<T>> queuedValues) {
-      Preconditions.checkNotNull(queuedValues,
-              "need to set the queuedValues of an Create.QueuedValues transform");
+      checkNotNull(
+          queuedValues, "need to set the queuedValues of an Create.QueuedValues transform");
       this.queuedValues = queuedValues;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b8615ea/runners/spark/src/main/java/org/apache/beam/runners/spark/io/KafkaIO.java
----------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/KafkaIO.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/KafkaIO.java
index a97d86e..13171f3 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/KafkaIO.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/KafkaIO.java
@@ -17,14 +17,14 @@
  */
 package org.apache.beam.runners.spark.io;
 
+import static com.google.common.base.Preconditions.checkNotNull;
+
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.util.WindowingStrategy;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PInput;
 
-import com.google.common.base.Preconditions;
-
 import java.util.Map;
 import java.util.Set;
 
@@ -82,18 +82,13 @@ public final class KafkaIO {
       Unbound(Class<? extends Decoder<K>> keyDecoder,
               Class<? extends Decoder<V>> valueDecoder, Class<K> key,
               Class<V> value, Set<String> topics, Map<String, String> kafkaParams) {
-        Preconditions.checkNotNull(keyDecoder,
-            "need to set the key decoder class of a KafkaIO.Read transform");
-        Preconditions.checkNotNull(valueDecoder,
-            "need to set the value decoder class of a KafkaIO.Read transform");
-        Preconditions.checkNotNull(key,
-            "need to set the key class of aKafkaIO.Read transform");
-        Preconditions.checkNotNull(value,
-            "need to set the value class of a KafkaIO.Read transform");
-        Preconditions.checkNotNull(topics,
-            "need to set the topics of a KafkaIO.Read transform");
-        Preconditions.checkNotNull(kafkaParams,
-            "need to set the kafkaParams of a KafkaIO.Read transform");
+        checkNotNull(keyDecoder, "need to set the key decoder class of a KafkaIO.Read transform");
+        checkNotNull(
+            valueDecoder, "need to set the value decoder class of a KafkaIO.Read transform");
+        checkNotNull(key, "need to set the key class of a KafkaIO.Read transform");
+        checkNotNull(value, "need to set the value class of a KafkaIO.Read transform");
+        checkNotNull(topics, "need to set the topics of a KafkaIO.Read transform");
+        checkNotNull(kafkaParams, "need to set the kafkaParams of a KafkaIO.Read transform");
         this.keyDecoderClass = keyDecoder;
         this.valueDecoderClass = valueDecoder;
         this.keyClass = key;



[15/50] [abbrv] incubator-beam git commit: Fix NPE in UnboundedReadFromBoundedSource

Posted by lc...@apache.org.
Fix NPE in UnboundedReadFromBoundedSource


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

Branch: refs/heads/runners-spark2
Commit: 562beafb59e14b568369259ac9c6f855ee9c80a3
Parents: 942b6e0
Author: Pei He <pe...@google.com>
Authored: Mon Jun 27 18:21:37 2016 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Jul 6 10:18:50 2016 -0700

----------------------------------------------------------------------
 .../core/UnboundedReadFromBoundedSource.java     | 19 +++++++++----------
 .../core/UnboundedReadFromBoundedSourceTest.java |  9 +++++++++
 2 files changed, 18 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/562beafb/runners/core-java/src/main/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSource.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSource.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSource.java
index 2b3d1c7..f54af3b 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSource.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSource.java
@@ -167,10 +167,7 @@ public class UnboundedReadFromBoundedSource<T> extends PTransform<PInput, PColle
     public Reader createReader(PipelineOptions options, Checkpoint<T> checkpoint)
         throws IOException {
       if (checkpoint == null) {
-        return new Reader(
-            Collections.<TimestampedValue<T>>emptyList() /* residualElements */,
-            boundedSource,
-            options);
+        return new Reader(null /* residualElements */, boundedSource, options);
       } else {
         return new Reader(checkpoint.residualElements, checkpoint.residualSource, options);
       }
@@ -189,11 +186,11 @@ public class UnboundedReadFromBoundedSource<T> extends PTransform<PInput, PColle
 
     @VisibleForTesting
     static class Checkpoint<T> implements UnboundedSource.CheckpointMark {
-      private final List<TimestampedValue<T>> residualElements;
+      private final @Nullable List<TimestampedValue<T>> residualElements;
       private final @Nullable BoundedSource<T> residualSource;
 
       public Checkpoint(
-          List<TimestampedValue<T>> residualElements,
+          @Nullable List<TimestampedValue<T>> residualElements,
           @Nullable BoundedSource<T> residualSource) {
         this.residualElements = residualElements;
         this.residualSource = residualSource;
@@ -203,7 +200,7 @@ public class UnboundedReadFromBoundedSource<T> extends PTransform<PInput, PColle
       public void finalizeCheckpoint() {}
 
       @VisibleForTesting
-      List<TimestampedValue<T>> getResidualElements() {
+      @Nullable List<TimestampedValue<T>> getResidualElements() {
         return residualElements;
       }
 
@@ -286,7 +283,7 @@ public class UnboundedReadFromBoundedSource<T> extends PTransform<PInput, PColle
       private boolean done;
 
       Reader(
-          List<TimestampedValue<T>> residualElementsList,
+          @Nullable List<TimestampedValue<T>> residualElementsList,
           @Nullable BoundedSource<T> residualSource,
           PipelineOptions options) {
         init(residualElementsList, residualSource, options);
@@ -295,10 +292,12 @@ public class UnboundedReadFromBoundedSource<T> extends PTransform<PInput, PColle
       }
 
       private void init(
-          List<TimestampedValue<T>> residualElementsList,
+          @Nullable List<TimestampedValue<T>> residualElementsList,
           @Nullable BoundedSource<T> residualSource,
           PipelineOptions options) {
-        this.residualElements = new ResidualElements(residualElementsList);
+        this.residualElements = residualElementsList == null
+            ? new ResidualElements(Collections.<TimestampedValue<T>>emptyList())
+                : new ResidualElements(residualElementsList);
         this.residualSource =
             residualSource == null ? null : new ResidualSource(residualSource, options);
       }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/562beafb/runners/core-java/src/test/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSourceTest.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSourceTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSourceTest.java
index afd0927..dfbc675 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSourceTest.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/UnboundedReadFromBoundedSourceTest.java
@@ -19,6 +19,7 @@ package org.apache.beam.runners.core;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
 
 import org.apache.beam.runners.core.UnboundedReadFromBoundedSource.BoundedToUnboundedSourceAdapter;
 import org.apache.beam.runners.core.UnboundedReadFromBoundedSource.BoundedToUnboundedSourceAdapter.Checkpoint;
@@ -169,6 +170,10 @@ public class UnboundedReadFromBoundedSourceTest {
         checkpoint.finalizeCheckpoint();
       }
     }
+    Checkpoint<T> checkpointDone = reader.getCheckpointMark();
+    assertTrue(checkpointDone.getResidualElements() == null
+        || checkpointDone.getResidualElements().isEmpty());
+
     assertEquals(expectedElements.size(), actual.size());
     assertEquals(Sets.newHashSet(expectedElements), Sets.newHashSet(actual));
   }
@@ -230,6 +235,10 @@ public class UnboundedReadFromBoundedSourceTest {
         hasNext = reader.advance();
       }
     }
+    Checkpoint<T> checkpointDone = reader.getCheckpointMark();
+    assertTrue(checkpointDone.getResidualElements() == null
+        || checkpointDone.getResidualElements().isEmpty());
+
     assertEquals(expectedElements.size(), actual.size());
     assertEquals(Sets.newHashSet(expectedElements), Sets.newHashSet(actual));
   }


[09/50] [abbrv] incubator-beam git commit: Use GatherAllPanes in PAssert

Posted by lc...@apache.org.
Use GatherAllPanes in PAssert

Instead of explicitly grouping by key, gather all the panes across the
input window.


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

Branch: refs/heads/runners-spark2
Commit: 8b0cbf9a0d32837260449a0b4ab38eb9712b5fd3
Parents: c8babc1
Author: Thomas Groh <tg...@google.com>
Authored: Mon Jun 20 14:38:11 2016 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Jul 6 10:18:49 2016 -0700

----------------------------------------------------------------------
 .../org/apache/beam/sdk/testing/PAssert.java    | 369 +++++++++++++++----
 .../apache/beam/sdk/testing/PAssertTest.java    | 116 ++++++
 2 files changed, 418 insertions(+), 67 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b0cbf9a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java
index 1a3d85d..883b2b3 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java
@@ -18,7 +18,6 @@
 package org.apache.beam.sdk.testing;
 
 import static com.google.common.base.Preconditions.checkState;
-
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.hamcrest.Matchers.equalTo;
 import static org.hamcrest.Matchers.not;
@@ -36,18 +35,24 @@ import org.apache.beam.sdk.runners.PipelineRunner;
 import org.apache.beam.sdk.transforms.Aggregator;
 import org.apache.beam.sdk.transforms.Create;
 import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.DoFn.RequiresWindowAccess;
 import org.apache.beam.sdk.transforms.Flatten;
 import org.apache.beam.sdk.transforms.GroupByKey;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.transforms.SerializableFunction;
 import org.apache.beam.sdk.transforms.Sum;
+import org.apache.beam.sdk.transforms.Values;
 import org.apache.beam.sdk.transforms.View;
 import org.apache.beam.sdk.transforms.WithKeys;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
 import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
 import org.apache.beam.sdk.transforms.windowing.Never;
 import org.apache.beam.sdk.transforms.windowing.Window;
 import org.apache.beam.sdk.util.CoderUtils;
+import org.apache.beam.sdk.util.GatherAllPanes;
+import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PBegin;
 import org.apache.beam.sdk.values.PCollection;
@@ -63,9 +68,11 @@ import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.io.Serializable;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.List;
 import java.util.Map;
 import java.util.NoSuchElementException;
 
@@ -116,6 +123,14 @@ public class PAssert {
    * Builder interface for assertions applicable to iterables and PCollection contents.
    */
   public interface IterableAssert<T> {
+    /**
+     * Creates a new {@link IterableAssert} like this one, but with the assertion restricted to only
+     * run on the provided window.
+     *
+     * @return a new {@link IterableAssert} like this one but with the assertion only applied to the
+     * specified window.
+     */
+    IterableAssert<T> inWindow(BoundedWindow window);
 
     /**
      * Asserts that the iterable in question contains the provided elements.
@@ -152,6 +167,15 @@ public class PAssert {
    */
   public interface SingletonAssert<T> {
     /**
+     * Creates a new {@link SingletonAssert} like this one, but with the assertion restricted to
+     * only run on the provided window.
+     *
+     * @return a new {@link SingletonAssert} like this one but with the assertion only applied to
+     * the specified window.
+     */
+    SingletonAssert<T> inWindow(BoundedWindow window);
+
+    /**
      * Asserts that the value in question is equal to the provided value, according to
      * {@link Object#equals}.
      *
@@ -250,9 +274,23 @@ public class PAssert {
    */
   private static class PCollectionContentsAssert<T> implements IterableAssert<T> {
     private final PCollection<T> actual;
+    private final AssertionWindows rewindowingStrategy;
 
     public PCollectionContentsAssert(PCollection<T> actual) {
+      this(actual, IntoGlobalWindow.<T>of());
+    }
+
+    public PCollectionContentsAssert(PCollection<T> actual, AssertionWindows rewindowingStrategy) {
       this.actual = actual;
+      this.rewindowingStrategy = rewindowingStrategy;
+    }
+
+    @Override
+    public PCollectionContentsAssert<T> inWindow(BoundedWindow window) {
+      @SuppressWarnings({"unchecked", "rawtypes"})
+      Coder<BoundedWindow> windowCoder =
+          (Coder) actual.getWindowingStrategy().getWindowFn().windowCoder();
+      return new PCollectionContentsAssert<>(actual, IntoStaticWindows.<T>of(windowCoder, window));
     }
 
     /**
@@ -285,7 +323,7 @@ public class PAssert {
     @Override
     public PCollectionContentsAssert<T> satisfies(
         SerializableFunction<Iterable<T>, Void> checkerFn) {
-      actual.apply(nextAssertionName(), new GroupThenAssert<>(checkerFn));
+      actual.apply(nextAssertionName(), new GroupThenAssert<>(checkerFn, rewindowingStrategy));
       return this;
     }
 
@@ -325,7 +363,8 @@ public class PAssert {
       @SuppressWarnings({"rawtypes", "unchecked"})
       SerializableFunction<Iterable<T>, Void> checkerFn =
           (SerializableFunction) new MatcherCheckerFn<>(matcher);
-      actual.apply("PAssert$" + (assertCount++), new GroupThenAssert<>(checkerFn));
+      actual.apply(
+          "PAssert$" + (assertCount++), new GroupThenAssert<>(checkerFn, rewindowingStrategy));
       return this;
     }
 
@@ -374,13 +413,30 @@ public class PAssert {
   private static class PCollectionSingletonIterableAssert<T> implements IterableAssert<T> {
     private final PCollection<Iterable<T>> actual;
     private final Coder<T> elementCoder;
+    private final AssertionWindows rewindowingStrategy;
 
     public PCollectionSingletonIterableAssert(PCollection<Iterable<T>> actual) {
+      this(actual, IntoGlobalWindow.<Iterable<T>>of());
+    }
+
+    public PCollectionSingletonIterableAssert(
+        PCollection<Iterable<T>> actual, AssertionWindows rewindowingStrategy) {
       this.actual = actual;
 
       @SuppressWarnings("unchecked")
       Coder<T> typedCoder = (Coder<T>) actual.getCoder().getCoderArguments().get(0);
       this.elementCoder = typedCoder;
+
+      this.rewindowingStrategy = rewindowingStrategy;
+    }
+
+    @Override
+    public PCollectionSingletonIterableAssert<T> inWindow(BoundedWindow window) {
+      @SuppressWarnings({"unchecked", "rawtypes"})
+      Coder<BoundedWindow> windowCoder =
+          (Coder) actual.getWindowingStrategy().getWindowFn().windowCoder();
+      return new PCollectionSingletonIterableAssert<>(
+          actual, IntoStaticWindows.<Iterable<T>>of(windowCoder, window));
     }
 
     @Override
@@ -402,7 +458,9 @@ public class PAssert {
     @Override
     public PCollectionSingletonIterableAssert<T> satisfies(
         SerializableFunction<Iterable<T>, Void> checkerFn) {
-      actual.apply("PAssert$" + (assertCount++), new GroupThenAssertForSingleton<>(checkerFn));
+      actual.apply(
+          "PAssert$" + (assertCount++),
+          new GroupThenAssertForSingleton<>(checkerFn, rewindowingStrategy));
       return this;
     }
 
@@ -421,18 +479,38 @@ public class PAssert {
   private static class PCollectionViewAssert<ElemT, ViewT> implements SingletonAssert<ViewT> {
     private final PCollection<ElemT> actual;
     private final PTransform<PCollection<ElemT>, PCollectionView<ViewT>> view;
+    private final AssertionWindows rewindowActuals;
     private final Coder<ViewT> coder;
 
     protected PCollectionViewAssert(
         PCollection<ElemT> actual,
         PTransform<PCollection<ElemT>, PCollectionView<ViewT>> view,
         Coder<ViewT> coder) {
+      this(actual, view, IntoGlobalWindow.<ElemT>of(), coder);
+    }
+
+    private PCollectionViewAssert(
+        PCollection<ElemT> actual,
+        PTransform<PCollection<ElemT>, PCollectionView<ViewT>> view,
+        AssertionWindows rewindowActuals,
+        Coder<ViewT> coder) {
       this.actual = actual;
       this.view = view;
+      this.rewindowActuals = rewindowActuals;
       this.coder = coder;
     }
 
     @Override
+    public PCollectionViewAssert<ElemT, ViewT> inWindow(BoundedWindow window) {
+      return new PCollectionViewAssert<>(
+          actual,
+          view,
+          IntoStaticWindows.of(
+              (Coder) actual.getWindowingStrategy().getWindowFn().windowCoder(), window),
+          coder);
+    }
+
+    @Override
     public PCollectionViewAssert<ElemT, ViewT> isEqualTo(ViewT expectedValue) {
       return satisfies(new AssertIsEqualToRelation<ViewT>(), expectedValue);
     }
@@ -449,7 +527,10 @@ public class PAssert {
           .getPipeline()
           .apply(
               "PAssert$" + (assertCount++),
-              new OneSideInputAssert<ViewT>(CreateActual.from(actual, view), checkerFn));
+              new OneSideInputAssert<ViewT>(
+                  CreateActual.from(actual, rewindowActuals, view),
+                  rewindowActuals.<Integer>windowDummy(),
+                  checkerFn));
       return this;
     }
 
@@ -496,16 +577,22 @@ public class PAssert {
       extends PTransform<PBegin, PCollectionView<ActualT>> {
 
     private final transient PCollection<T> actual;
+    private final transient AssertionWindows rewindowActuals;
     private final transient PTransform<PCollection<T>, PCollectionView<ActualT>> actualView;
 
     public static <T, ActualT> CreateActual<T, ActualT> from(
-        PCollection<T> actual, PTransform<PCollection<T>, PCollectionView<ActualT>> actualView) {
-      return new CreateActual<>(actual, actualView);
+        PCollection<T> actual,
+        AssertionWindows rewindowActuals,
+        PTransform<PCollection<T>, PCollectionView<ActualT>> actualView) {
+      return new CreateActual<>(actual, rewindowActuals, actualView);
     }
 
     private CreateActual(
-        PCollection<T> actual, PTransform<PCollection<T>, PCollectionView<ActualT>> actualView) {
+        PCollection<T> actual,
+        AssertionWindows rewindowActuals,
+        PTransform<PCollection<T>, PCollectionView<ActualT>> actualView) {
       this.actual = actual;
+      this.rewindowActuals = rewindowActuals;
       this.actualView = actualView;
     }
 
@@ -513,7 +600,8 @@ public class PAssert {
     public PCollectionView<ActualT> apply(PBegin input) {
       final Coder<T> coder = actual.getCoder();
       return actual
-          .apply(Window.<T>into(new GlobalWindows()))
+          .apply("FilterActuals", rewindowActuals.<T>prepareActuals())
+          .apply("RewindowActuals", rewindowActuals.<T>windowActuals())
           .apply(
               ParDo.of(
                   new DoFn<T, T>() {
@@ -565,84 +653,83 @@ public class PAssert {
    * <p>If the {@link PCollection} is empty, this transform returns a {@link PCollection} containing
    * a single empty iterable, even though in practice most runners will not produce any element.
    */
-  private static class GroupGlobally<T> extends PTransform<PCollection<T>, PCollection<Iterable<T>>>
+  private static class GroupGlobally<T>
+      extends PTransform<PCollection<T>, PCollection<Iterable<WindowedValue<T>>>>
       implements Serializable {
+    private final AssertionWindows rewindowingStrategy;
 
-    public GroupGlobally() {}
+    public GroupGlobally(AssertionWindows rewindowingStrategy) {
+      this.rewindowingStrategy = rewindowingStrategy;
+    }
 
     @Override
-    public PCollection<Iterable<T>> apply(PCollection<T> input) {
-
-      final int contentsKey = 0;
-      final int dummyKey = 1;
+    public PCollection<Iterable<WindowedValue<T>>> apply(PCollection<T> input) {
       final int combinedKey = 42;
 
+      // Remove the triggering on both
+      PTransform<
+              PCollection<KV<Integer, Iterable<WindowedValue<T>>>>,
+              PCollection<KV<Integer, Iterable<WindowedValue<T>>>>>
+          removeTriggering =
+              Window.<KV<Integer, Iterable<WindowedValue<T>>>>triggering(Never.ever())
+                  .discardingFiredPanes()
+                  .withAllowedLateness(input.getWindowingStrategy().getAllowedLateness());
       // Group the contents by key. If it is empty, this PCollection will be empty, too.
       // Then key it again with a dummy key.
-      PCollection<KV<Integer, KV<Integer, Iterable<T>>>> doubleKeyedGroupedInput =
+      PCollection<KV<Integer, Iterable<WindowedValue<T>>>> groupedContents =
+          // TODO: Split the filtering from the rewindowing, and apply filtering before the Gather
+          // if the grouping of extra records
           input
-              .apply("GloballyWindow", Window.<T>into(new GlobalWindows()))
-              .apply("ContentsWithKeys", WithKeys.<Integer, T>of(contentsKey))
+              .apply(rewindowingStrategy.<T>prepareActuals())
+              .apply("GatherAllOutputs", GatherAllPanes.<T>globally())
               .apply(
-                  "NeverTriggerContents",
-                  Window.<KV<Integer, T>>triggering(Never.ever()).discardingFiredPanes())
-              .apply("ContentsGBK", GroupByKey.<Integer, T>create())
-              .apply(
-                  "DoubleKeyContents", WithKeys.<Integer, KV<Integer, Iterable<T>>>of(combinedKey));
+                  "RewindowActuals",
+                  rewindowingStrategy.<Iterable<WindowedValue<T>>>windowActuals())
+              .apply("KeyForDummy", WithKeys.<Integer, Iterable<WindowedValue<T>>>of(combinedKey))
+              .apply("RemoveActualsTriggering", removeTriggering);
 
       // Create another non-empty PCollection that is keyed with a distinct dummy key
-      PCollection<KV<Integer, KV<Integer, Iterable<T>>>> doubleKeyedDummy =
+      PCollection<KV<Integer, Iterable<WindowedValue<T>>>> keyedDummy =
           input
               .getPipeline()
               .apply(
                   Create.of(
                           KV.of(
                               combinedKey,
-                              KV.of(dummyKey, (Iterable<T>) Collections.<T>emptyList())))
-                      .withCoder(doubleKeyedGroupedInput.getCoder()))
-              .setWindowingStrategyInternal(doubleKeyedGroupedInput.getWindowingStrategy());
+                              (Iterable<WindowedValue<T>>)
+                                  Collections.<WindowedValue<T>>emptyList()))
+                      .withCoder(groupedContents.getCoder()))
+              .apply(
+                  "WindowIntoDummy",
+                  rewindowingStrategy.<KV<Integer, Iterable<WindowedValue<T>>>>windowDummy())
+              .apply("RemoveDummyTriggering", removeTriggering);
 
       // Flatten them together and group by the combined key to get a single element
-      PCollection<KV<Integer, Iterable<KV<Integer, Iterable<T>>>>> dummyAndContents =
-          PCollectionList.<KV<Integer, KV<Integer, Iterable<T>>>>of(doubleKeyedGroupedInput)
-              .and(doubleKeyedDummy)
+      PCollection<KV<Integer, Iterable<Iterable<WindowedValue<T>>>>> dummyAndContents =
+          PCollectionList.of(groupedContents)
+              .and(keyedDummy)
               .apply(
                   "FlattenDummyAndContents",
-                  Flatten.<KV<Integer, KV<Integer, Iterable<T>>>>pCollections())
+                  Flatten.<KV<Integer, Iterable<WindowedValue<T>>>>pCollections())
               .apply(
-                  "GroupDummyAndContents", GroupByKey.<Integer, KV<Integer, Iterable<T>>>create());
+                  "NeverTrigger",
+                  Window.<KV<Integer, Iterable<WindowedValue<T>>>>triggering(Never.ever())
+                      .withAllowedLateness(input.getWindowingStrategy().getAllowedLateness())
+                      .discardingFiredPanes())
+              .apply(
+                  "GroupDummyAndContents",
+                  GroupByKey.<Integer, Iterable<WindowedValue<T>>>create());
 
-      // Extract the contents if they exist else empty contents.
       return dummyAndContents
-          .apply(
-              "GetContents",
-              ParDo.of(
-                  new DoFn<KV<Integer, Iterable<KV<Integer, Iterable<T>>>>, Iterable<T>>() {
-                    @Override
-                    public void processElement(ProcessContext ctx) {
-                      Iterable<KV<Integer, Iterable<T>>> groupedDummyAndContents =
-                          ctx.element().getValue();
-
-                      if (Iterables.size(groupedDummyAndContents) == 1) {
-                        // Only the dummy value, so just output empty
-                        ctx.output(Collections.<T>emptyList());
-                      } else {
-                        checkState(
-                            Iterables.size(groupedDummyAndContents) == 2,
-                            "Internal error: PAssert grouped contents with a"
-                                + " dummy value resulted in more than 2 groupings: %s",
-                                groupedDummyAndContents);
-
-                        if (Iterables.get(groupedDummyAndContents, 0).getKey() == contentsKey) {
-                          // The first iterable in the group holds the real contents
-                          ctx.output(Iterables.get(groupedDummyAndContents, 0).getValue());
-                        } else {
-                          // The second iterable holds the real contents
-                          ctx.output(Iterables.get(groupedDummyAndContents, 1).getValue());
-                        }
-                      }
-                    }
-                  }));
+          .apply(Values.<Iterable<Iterable<WindowedValue<T>>>>create())
+          .apply(ParDo.of(new ConcatFn<WindowedValue<T>>()));
+    }
+  }
+
+  private static final class ConcatFn<T> extends DoFn<Iterable<Iterable<T>>, Iterable<T>> {
+    @Override
+    public void processElement(ProcessContext c) throws Exception {
+      c.output(Iterables.concat(c.element()));
     }
   }
 
@@ -653,15 +740,20 @@ public class PAssert {
   public static class GroupThenAssert<T> extends PTransform<PCollection<T>, PDone>
       implements Serializable {
     private final SerializableFunction<Iterable<T>, Void> checkerFn;
+    private final AssertionWindows rewindowingStrategy;
 
-    private GroupThenAssert(SerializableFunction<Iterable<T>, Void> checkerFn) {
+    private GroupThenAssert(
+        SerializableFunction<Iterable<T>, Void> checkerFn, AssertionWindows rewindowingStrategy) {
       this.checkerFn = checkerFn;
+      this.rewindowingStrategy = rewindowingStrategy;
     }
 
     @Override
     public PDone apply(PCollection<T> input) {
       input
-          .apply("GroupGlobally", new GroupGlobally<T>())
+          .apply("GroupGlobally", new GroupGlobally<T>(rewindowingStrategy))
+          .apply("GetOnlyPane", ParDo.of(new ExtractOnlyPane<T>()))
+          .setCoder(IterableCoder.of(input.getCoder()))
           .apply("RunChecks", ParDo.of(new GroupedValuesCheckerDoFn<>(checkerFn)));
 
       return PDone.in(input.getPipeline());
@@ -675,15 +767,20 @@ public class PAssert {
   public static class GroupThenAssertForSingleton<T>
       extends PTransform<PCollection<Iterable<T>>, PDone> implements Serializable {
     private final SerializableFunction<Iterable<T>, Void> checkerFn;
+    private final AssertionWindows rewindowingStrategy;
 
-    private GroupThenAssertForSingleton(SerializableFunction<Iterable<T>, Void> checkerFn) {
+    private GroupThenAssertForSingleton(
+        SerializableFunction<Iterable<T>, Void> checkerFn, AssertionWindows rewindowingStrategy) {
       this.checkerFn = checkerFn;
+      this.rewindowingStrategy = rewindowingStrategy;
     }
 
     @Override
     public PDone apply(PCollection<Iterable<T>> input) {
       input
-          .apply("GroupGlobally", new GroupGlobally<Iterable<T>>())
+          .apply("GroupGlobally", new GroupGlobally<Iterable<T>>(rewindowingStrategy))
+          .apply("GetOnlyPane", ParDo.of(new ExtractOnlyPane<Iterable<T>>()))
+          .setCoder(IterableCoder.of(input.getCoder()))
           .apply("RunChecks", ParDo.of(new SingletonCheckerDoFn<>(checkerFn)));
 
       return PDone.in(input.getPipeline());
@@ -703,12 +800,15 @@ public class PAssert {
   public static class OneSideInputAssert<ActualT> extends PTransform<PBegin, PDone>
       implements Serializable {
     private final transient PTransform<PBegin, PCollectionView<ActualT>> createActual;
+    private final transient PTransform<PCollection<Integer>, PCollection<Integer>> windowToken;
     private final SerializableFunction<ActualT, Void> checkerFn;
 
     private OneSideInputAssert(
         PTransform<PBegin, PCollectionView<ActualT>> createActual,
+        PTransform<PCollection<Integer>, PCollection<Integer>> windowToken,
         SerializableFunction<ActualT, Void> checkerFn) {
       this.createActual = createActual;
+      this.windowToken = windowToken;
       this.checkerFn = checkerFn;
     }
 
@@ -718,7 +818,9 @@ public class PAssert {
 
       input
           .apply(Create.of(0).withCoder(VarIntCoder.of()))
-          .apply("RunChecks",
+          .apply("WindowToken", windowToken)
+          .apply(
+              "RunChecks",
               ParDo.withSideInputs(actual).of(new SideInputCheckerDoFn<>(checkerFn, actual)));
 
       return PDone.in(input.getPipeline());
@@ -760,6 +862,23 @@ public class PAssert {
     }
   }
 
+  private static class ExtractOnlyPane<T> extends DoFn<Iterable<WindowedValue<T>>, Iterable<T>> {
+    @Override
+    public void processElement(ProcessContext c) throws Exception {
+      List<T> outputs = new ArrayList<>();
+      for (WindowedValue<T> value : c.element()) {
+        checkState(
+            value.getPane().isFirst() && value.getPane().isLast(),
+            "Expected elements to be produced by a trigger that fires at most once, but got"
+                + "a value in a pane that is %s. Actual Pane Info: %s",
+            value.getPane().isFirst() ? "not the last pane" : "not the first pane",
+            value.getPane());
+        outputs.add(value.getValue());
+      }
+      c.output(outputs);
+    }
+  }
+
   /**
    * A {@link DoFn} that runs a checking {@link SerializableFunction} on the contents of
    * the single iterable element of the input {@link PCollection} and adjusts counters and
@@ -948,4 +1067,120 @@ public class PAssert {
       return new AssertContainsInAnyOrder<T>(expectedElements);
     }
   }
+
+  ////////////////////////////////////////////////////////////////////////////////////////////////
+
+  /**
+   * A strategy for filtering and rewindowing the actual and dummy {@link PCollection PCollections}
+   * within a {@link PAssert}.
+   *
+   * <p>This must ensure that the windowing strategies of the output of {@link #windowActuals()} and
+   * {@link #windowDummy()} are compatible (and can be {@link Flatten Flattened}).
+   *
+   * <p>The {@link PCollection} produced by {@link #prepareActuals()} will be a parent (though not
+   * a direct parent) of the transform provided to {@link #windowActuals()}.
+   */
+  private interface AssertionWindows {
+    /**
+     * Returns a transform that assigns the dummy element into the appropriate
+     * {@link BoundedWindow windows}.
+     */
+    <T> PTransform<PCollection<T>, PCollection<T>> windowDummy();
+
+    /**
+     * Returns a transform that filters and reassigns windows of the actual elements if necessary.
+     */
+    <T> PTransform<PCollection<T>, PCollection<T>> prepareActuals();
+
+    /**
+     * Returns a transform that assigns the actual elements into the appropriate
+     * {@link BoundedWindow windows}. Will be called after {@link #prepareActuals()}.
+     */
+    <T> PTransform<PCollection<T>, PCollection<T>> windowActuals();
+  }
+
+  /**
+   * An {@link AssertionWindows} which assigns all elements to the {@link GlobalWindow}.
+   */
+  private static class IntoGlobalWindow implements AssertionWindows, Serializable {
+    public static AssertionWindows of() {
+      return new IntoGlobalWindow();
+    }
+
+    private <T> PTransform<PCollection<T>, PCollection<T>> window() {
+      return Window.into(new GlobalWindows());
+    }
+
+    @Override
+    public <T> PTransform<PCollection<T>, PCollection<T>> windowDummy() {
+      return window();
+    }
+
+    /**
+     * Rewindows all input elements into the {@link GlobalWindow}. This ensures that the result
+     * PCollection will contain all of the elements of the PCollection when the window is not
+     * specified.
+     */
+    @Override
+    public <T> PTransform<PCollection<T>, PCollection<T>> prepareActuals() {
+      return window();
+    }
+
+    @Override
+    public <T> PTransform<PCollection<T>, PCollection<T>> windowActuals() {
+      return window();
+    }
+  }
+
+  private static class IntoStaticWindows implements AssertionWindows {
+    private final StaticWindows windowFn;
+
+    public static AssertionWindows of(Coder<BoundedWindow> windowCoder, BoundedWindow window) {
+      return new IntoStaticWindows(StaticWindows.of(windowCoder, window));
+    }
+
+    private IntoStaticWindows(StaticWindows windowFn) {
+      this.windowFn = windowFn;
+    }
+
+    @Override
+    public <T> PTransform<PCollection<T>, PCollection<T>> windowDummy() {
+      return Window.into(windowFn);
+    }
+
+    @Override
+    public <T> PTransform<PCollection<T>, PCollection<T>> prepareActuals() {
+      return new FilterWindows<>(windowFn);
+    }
+
+    @Override
+    public <T> PTransform<PCollection<T>, PCollection<T>> windowActuals() {
+      return Window.into(windowFn.intoOnlyExisting());
+    }
+  }
+
+  /**
+   * A DoFn that filters elements based on their presence in a static collection of windows.
+   */
+  private static final class FilterWindows<T> extends PTransform<PCollection<T>, PCollection<T>> {
+    private final StaticWindows windows;
+
+    public FilterWindows(StaticWindows windows) {
+      this.windows = windows;
+    }
+
+    @Override
+    public PCollection<T> apply(PCollection<T> input) {
+      return input.apply("FilterWindows", ParDo.of(new Fn()));
+    }
+
+    private class Fn extends DoFn<T, T> implements RequiresWindowAccess {
+      @Override
+      public void processElement(ProcessContext c) throws Exception {
+        if (windows.getWindows().contains(c.window())) {
+          c.output(c.element());
+        }
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8b0cbf9a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PAssertTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PAssertTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PAssertTest.java
index fdc8719..bafd897 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PAssertTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PAssertTest.java
@@ -17,6 +17,8 @@
  */
 package org.apache.beam.sdk.testing;
 
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -25,11 +27,21 @@ import org.apache.beam.sdk.coders.AtomicCoder;
 import org.apache.beam.sdk.coders.CoderException;
 import org.apache.beam.sdk.transforms.Create;
 import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.SlidingWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
 import org.apache.beam.sdk.util.common.ElementByteSizeObserver;
 import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TimestampedValue;
+
+import com.google.common.collect.Iterables;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 
+import org.joda.time.Duration;
+import org.joda.time.Instant;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -148,6 +160,45 @@ public class PAssertTest implements Serializable {
   }
 
   /**
+   * A {@link PAssert} about the contents of a {@link PCollection}
+   * is allows to be verified by an arbitrary {@link SerializableFunction},
+   * though.
+   */
+  @Test
+  @Category(RunnableOnService.class)
+  public void testWindowedSerializablePredicate() throws Exception {
+    Pipeline pipeline = TestPipeline.create();
+
+    PCollection<NotSerializableObject> pcollection = pipeline
+        .apply(Create.timestamped(
+            TimestampedValue.of(new NotSerializableObject(), new Instant(250L)),
+            TimestampedValue.of(new NotSerializableObject(), new Instant(500L)))
+            .withCoder(NotSerializableObjectCoder.of()))
+        .apply(Window.<NotSerializableObject>into(FixedWindows.of(Duration.millis(300L))));
+
+    PAssert.that(pcollection)
+        .inWindow(new IntervalWindow(new Instant(0L), new Instant(300L)))
+        .satisfies(new SerializableFunction<Iterable<NotSerializableObject>, Void>() {
+          @Override
+          public Void apply(Iterable<NotSerializableObject> contents) {
+            assertThat(Iterables.isEmpty(contents), is(false));
+            return null; // no problem!
+          }
+        });
+    PAssert.that(pcollection)
+        .inWindow(new IntervalWindow(new Instant(300L), new Instant(600L)))
+        .satisfies(new SerializableFunction<Iterable<NotSerializableObject>, Void>() {
+          @Override
+          public Void apply(Iterable<NotSerializableObject> contents) {
+            assertThat(Iterables.isEmpty(contents), is(false));
+            return null; // no problem!
+          }
+        });
+
+    pipeline.run();
+  }
+
+  /**
    * Test that we throw an error at pipeline construction time when the user mistakenly uses
    * {@code PAssert.thatSingleton().equals()} instead of the test method {@code .isEqualTo}.
    */
@@ -220,6 +271,26 @@ public class PAssertTest implements Serializable {
   }
 
   /**
+   * Basic test for {@code isEqualTo}.
+   */
+  @Test
+  @Category(RunnableOnService.class)
+  public void testWindowedIsEqualTo() throws Exception {
+    Pipeline pipeline = TestPipeline.create();
+    PCollection<Integer> pcollection =
+        pipeline.apply(Create.timestamped(TimestampedValue.of(43, new Instant(250L)),
+            TimestampedValue.of(22, new Instant(-250L))))
+            .apply(Window.<Integer>into(FixedWindows.of(Duration.millis(500L))));
+    PAssert.thatSingleton(pcollection)
+        .inWindow(new IntervalWindow(new Instant(0L), new Instant(500L)))
+        .isEqualTo(43);
+    PAssert.thatSingleton(pcollection)
+        .inWindow(new IntervalWindow(new Instant(-500L), new Instant(0L)))
+        .isEqualTo(22);
+    pipeline.run();
+  }
+
+  /**
    * Basic test for {@code notEqualTo}.
    */
   @Test
@@ -244,6 +315,51 @@ public class PAssertTest implements Serializable {
   }
 
   /**
+   * Tests that {@code containsInAnyOrder} is actually order-independent.
+   */
+  @Test
+  @Category(RunnableOnService.class)
+  public void testGlobalWindowContainsInAnyOrder() throws Exception {
+    Pipeline pipeline = TestPipeline.create();
+    PCollection<Integer> pcollection = pipeline.apply(Create.of(1, 2, 3, 4));
+    PAssert.that(pcollection).inWindow(GlobalWindow.INSTANCE).containsInAnyOrder(2, 1, 4, 3);
+    pipeline.run();
+  }
+
+  /**
+   * Tests that windowed {@code containsInAnyOrder} is actually order-independent.
+   */
+  @Test
+  @Category(RunnableOnService.class)
+  public void testWindowedContainsInAnyOrder() throws Exception {
+    Pipeline pipeline = TestPipeline.create();
+    PCollection<Integer> pcollection =
+        pipeline.apply(Create.timestamped(TimestampedValue.of(1, new Instant(100L)),
+            TimestampedValue.of(2, new Instant(200L)),
+            TimestampedValue.of(3, new Instant(300L)),
+            TimestampedValue.of(4, new Instant(400L))))
+            .apply(Window.<Integer>into(SlidingWindows.of(Duration.millis(200L))
+                .every(Duration.millis(100L))
+                .withOffset(Duration.millis(50L))));
+
+    PAssert.that(pcollection)
+        .inWindow(new IntervalWindow(new Instant(-50L), new Instant(150L))).containsInAnyOrder(1);
+    PAssert.that(pcollection)
+        .inWindow(new IntervalWindow(new Instant(50L), new Instant(250L)))
+        .containsInAnyOrder(2, 1);
+    PAssert.that(pcollection)
+        .inWindow(new IntervalWindow(new Instant(150L), new Instant(350L)))
+        .containsInAnyOrder(2, 3);
+    PAssert.that(pcollection)
+        .inWindow(new IntervalWindow(new Instant(250L), new Instant(450L)))
+        .containsInAnyOrder(4, 3);
+    PAssert.that(pcollection)
+        .inWindow(new IntervalWindow(new Instant(350L), new Instant(550L)))
+        .containsInAnyOrder(4);
+    pipeline.run();
+  }
+
+  /**
    * Tests that {@code containsInAnyOrder} fails when and how it should.
    */
   @Test


[28/50] [abbrv] incubator-beam git commit: Update Checkpoint Documentation

Posted by lc...@apache.org.
Update Checkpoint Documentation

Checkpoint finalization is best effort. A checkpoint that is committed
to durable state is permitted to be reused to start a reader, regardless
of if it is finalized.

Note that checkpoints which have an affect on their source (via
finalize) should generally require Deduping, due to the potential for
arbitrary checkpoint finalization failures.


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

Branch: refs/heads/runners-spark2
Commit: 89b22c88b7df353a7990cc36db8181a83d6b82a2
Parents: ef9d195
Author: Thomas Groh <tg...@google.com>
Authored: Mon Jun 20 13:48:17 2016 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Jul 6 10:18:51 2016 -0700

----------------------------------------------------------------------
 .../org/apache/beam/sdk/io/UnboundedSource.java | 26 +++++++++++---------
 1 file changed, 14 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/89b22c88/sdks/java/core/src/main/java/org/apache/beam/sdk/io/UnboundedSource.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/UnboundedSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/UnboundedSource.java
index ea3004e..dded8e2 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/UnboundedSource.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/UnboundedSource.java
@@ -91,6 +91,10 @@ public abstract class UnboundedSource<
    * <p>This is needed if the underlying data source can return the same record multiple times,
    * such a queuing system with a pull-ack model.  Sources where the records read are uniquely
    * identified by the persisted state in the CheckpointMark do not need this.
+   *
+   * <p>Generally, if {@link CheckpointMark#finalizeCheckpoint()} is overridden, this method should
+   * return true. Checkpoint finalization is best-effort, and readers can be resumed from a
+   * checkpoint that has not been finalized.
    */
   public boolean requiresDeduping() {
     return false;
@@ -106,7 +110,7 @@ public abstract class UnboundedSource<
     /**
      * Called by the system to signal that this checkpoint mark has been committed along with
      * all the records which have been read from the {@link UnboundedReader} since the
-     * previous finalized checkpoint was taken.
+     * previous checkpoint was taken.
      *
      * <p>For example, this method could send acknowledgements to an external data source
      * such as Pubsub.
@@ -120,15 +124,9 @@ public abstract class UnboundedSource<
      * {@link UnboundedReader} has not yet be finalized.
      * <li>In the absence of failures, all checkpoints will be finalized and they will be
      * finalized in the same order they were taken from the {@link UnboundedReader}.
-     * <li>It is possible for a checkpoint to be taken but this method never called if
-     * the checkpoint could not be committed for any reason.
-     * <li>If this call throws an exception then the entire checkpoint will be abandoned and the
-     * reader restarted from an earlier, successfully-finalized checkpoint.
-     * <li>If a checkpoint fails for any reason then no later checkpoint will be allowed to be
-     * finalized without the reader first being restarted.
-     * <li>If an {@link UnboundedReader} is restarted from an earlier checkpoint, the checkpoint
-     * instance will be deserialized from the serialized form of the earlier checkpoint using
-     * {@link UnboundedSource#getCheckpointMarkCoder()}.
+     * <li>It is possible for a checkpoint to be taken but this method never called. This method
+     * will never be called if the checkpoint could not be committed, and other failures may cause
+     * this method to never be called.
      * <li>It is not safe to assume the {@link UnboundedReader} from which this checkpoint was
      * created still exists at the time this method is called.
      * </ul>
@@ -230,8 +228,12 @@ public abstract class UnboundedSource<
      * <p>All elements read up until this method is called will be processed together as a bundle.
      * (An element is considered 'read' if it could be returned by a call to {@link #getCurrent}.)
      * Once the result of processing those elements and the returned checkpoint have been durably
-     * committed, {@link CheckpointMark#finalizeCheckpoint} will (eventually) be called on the
-     * returned {@link CheckpointMark} object.
+     * committed, {@link CheckpointMark#finalizeCheckpoint} will be called at most once at some
+     * later point on the returned {@link CheckpointMark} object. Checkpoint finalization is
+     * best-effort, and checkpoints may not be finalized. If duplicate elements may be produced if
+     * checkpoints are not finalized in a timely manner, {@link UnboundedSource#requiresDeduping()}
+     * should be overridden to return true, and {@link UnboundedReader#getCurrentRecordId()} should
+     * be overriden to return unique record IDs.
      *
      * <p>The returned object should not be modified.
      *